001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.util;
019
020import static org.junit.jupiter.api.Assertions.assertEquals;
021import static org.junit.jupiter.api.Assertions.assertTrue;
022
023import java.util.Map;
024import java.util.Random;
025import java.util.concurrent.Callable;
026import java.util.concurrent.ConcurrentHashMap;
027import java.util.concurrent.ExecutorCompletionService;
028import java.util.concurrent.ExecutorService;
029import java.util.concurrent.Executors;
030import java.util.concurrent.Future;
031import java.util.concurrent.ThreadLocalRandom;
032import java.util.concurrent.TimeUnit;
033import java.util.concurrent.locks.Lock;
034import java.util.concurrent.locks.ReentrantReadWriteLock;
035import java.util.stream.Stream;
036import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate;
037import org.apache.hadoop.hbase.testclassification.MediumTests;
038import org.apache.hadoop.hbase.testclassification.MiscTests;
039import org.apache.hadoop.hbase.util.IdReadWriteLockWithObjectPool.ReferenceType;
040import org.junit.jupiter.api.Tag;
041import org.junit.jupiter.api.TestTemplate;
042import org.junit.jupiter.params.provider.Arguments;
043import org.slf4j.Logger;
044import org.slf4j.LoggerFactory;
045
046@HBaseParameterizedTestTemplate
047@Tag(MiscTests.TAG)
048@Tag(MediumTests.TAG)
049// Medium as it creates 100 threads; seems better to run it isolated
050public class TestIdReadWriteLockWithObjectPool {
051
052  private static final Logger LOG =
053    LoggerFactory.getLogger(TestIdReadWriteLockWithObjectPool.class);
054
055  private static final int NUM_IDS = 16;
056  private static final int NUM_THREADS = 128;
057  private static final int NUM_SECONDS = 15;
058
059  private IdReadWriteLockWithObjectPool<Long> idLock;
060
061  public TestIdReadWriteLockWithObjectPool(IdReadWriteLockWithObjectPool<Long> idLock) {
062    this.idLock = idLock;
063  }
064
065  public static Stream<Arguments> parameters() {
066    return Stream.of(Arguments.of(new IdReadWriteLockWithObjectPool<Long>(ReferenceType.WEAK)),
067      Arguments.of(new IdReadWriteLockWithObjectPool<Long>(ReferenceType.SOFT)));
068  }
069
070  private Map<Long, String> idOwner = new ConcurrentHashMap<>();
071
072  private class IdLockTestThread implements Callable<Boolean> {
073
074    private String clientId;
075
076    public IdLockTestThread(String clientId) {
077      this.clientId = clientId;
078    }
079
080    @Override
081    public Boolean call() throws Exception {
082      Thread.currentThread().setName(clientId);
083      Random rand = ThreadLocalRandom.current();
084      long endTime = EnvironmentEdgeManager.currentTime() + NUM_SECONDS * 1000;
085      while (EnvironmentEdgeManager.currentTime() < endTime) {
086        long id = rand.nextInt(NUM_IDS);
087        boolean readLock = rand.nextBoolean();
088
089        ReentrantReadWriteLock readWriteLock = idLock.getLock(id);
090        Lock lock = readLock ? readWriteLock.readLock() : readWriteLock.writeLock();
091        try {
092          lock.lock();
093          int sleepMs = 1 + rand.nextInt(4);
094          String owner = idOwner.get(id);
095          if (owner != null && LOG.isDebugEnabled()) {
096            LOG.debug((readLock ? "Read" : "Write") + "lock of Id " + id + " already taken by "
097              + owner + ", we are " + clientId);
098          }
099
100          idOwner.put(id, clientId);
101          Thread.sleep(sleepMs);
102          idOwner.remove(id);
103
104        } finally {
105          lock.unlock();
106          if (LOG.isDebugEnabled()) {
107            LOG.debug("Release " + (readLock ? "Read" : "Write") + " lock of Id" + id + ", we are "
108              + clientId);
109          }
110        }
111      }
112      return true;
113    }
114
115  }
116
117  @TestTemplate
118  public void testMultipleClients() throws Exception {
119    ExecutorService exec = Executors.newFixedThreadPool(NUM_THREADS);
120    try {
121      ExecutorCompletionService<Boolean> ecs = new ExecutorCompletionService<>(exec);
122      for (int i = 0; i < NUM_THREADS; ++i)
123        ecs.submit(new IdLockTestThread("client_" + i));
124      for (int i = 0; i < NUM_THREADS; ++i) {
125        Future<Boolean> result = ecs.take();
126        assertTrue(result.get());
127      }
128      int entryPoolSize = idLock.purgeAndGetEntryPoolSize();
129      LOG.debug("Size of entry pool after gc and purge: " + entryPoolSize);
130      ReferenceType refType = idLock.getReferenceType();
131      switch (refType) {
132        case WEAK:
133          // make sure the entry pool will be cleared after GC and purge call
134          assertEquals(0, entryPoolSize);
135          break;
136        case SOFT:
137          // make sure the entry pool won't be cleared when JVM memory is enough
138          // even after GC and purge call
139          assertEquals(NUM_IDS, entryPoolSize);
140          break;
141        default:
142          break;
143      }
144    } finally {
145      exec.shutdown();
146      exec.awaitTermination(5000, TimeUnit.MILLISECONDS);
147    }
148  }
149
150}