HBASE-26921 Rewrite the counting cells part in TestMultiVersions (#4316)
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / util / TestIdReadWriteLockWithObjectPool.java
blob47d6dfd627b7ce3a0e7c10f07e2aa7805f24a62a
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase.util;
20 import static org.junit.Assert.assertEquals;
21 import static org.junit.Assert.assertTrue;
23 import java.util.Arrays;
24 import java.util.Map;
25 import java.util.Random;
26 import java.util.concurrent.Callable;
27 import java.util.concurrent.ConcurrentHashMap;
28 import java.util.concurrent.ExecutorCompletionService;
29 import java.util.concurrent.ExecutorService;
30 import java.util.concurrent.Executors;
31 import java.util.concurrent.Future;
32 import java.util.concurrent.ThreadLocalRandom;
33 import java.util.concurrent.TimeUnit;
34 import java.util.concurrent.locks.Lock;
35 import java.util.concurrent.locks.ReentrantReadWriteLock;
36 import org.apache.hadoop.hbase.HBaseClassTestRule;
37 import org.apache.hadoop.hbase.testclassification.MediumTests;
38 import org.apache.hadoop.hbase.testclassification.MiscTests;
39 import org.apache.hadoop.hbase.util.IdReadWriteLockWithObjectPool.ReferenceType;
40 import org.junit.ClassRule;
41 import org.junit.Test;
42 import org.junit.experimental.categories.Category;
43 import org.junit.runner.RunWith;
44 import org.junit.runners.Parameterized;
45 import org.slf4j.Logger;
46 import org.slf4j.LoggerFactory;
48 @RunWith(Parameterized.class)
49 @Category({MiscTests.class, MediumTests.class})
50 // Medium as it creates 100 threads; seems better to run it isolated
51 public class TestIdReadWriteLockWithObjectPool {
53 @ClassRule
54 public static final HBaseClassTestRule CLASS_RULE =
55 HBaseClassTestRule.forClass(TestIdReadWriteLockWithObjectPool.class);
57 private static final Logger LOG =
58 LoggerFactory.getLogger(TestIdReadWriteLockWithObjectPool.class);
60 private static final int NUM_IDS = 16;
61 private static final int NUM_THREADS = 128;
62 private static final int NUM_SECONDS = 15;
64 @Parameterized.Parameter
65 public IdReadWriteLockWithObjectPool<Long> idLock;
67 @Parameterized.Parameters
68 public static Iterable<Object[]> data() {
69 return Arrays.asList(new Object[][] {
70 { new IdReadWriteLockWithObjectPool<Long>(ReferenceType.WEAK) },
71 { new IdReadWriteLockWithObjectPool<Long>(ReferenceType.SOFT) } });
74 private Map<Long, String> idOwner = new ConcurrentHashMap<>();
76 private class IdLockTestThread implements Callable<Boolean> {
78 private String clientId;
80 public IdLockTestThread(String clientId) {
81 this.clientId = clientId;
84 @Override
85 public Boolean call() throws Exception {
86 Thread.currentThread().setName(clientId);
87 Random rand = ThreadLocalRandom.current();
88 long endTime = EnvironmentEdgeManager.currentTime() + NUM_SECONDS * 1000;
89 while (EnvironmentEdgeManager.currentTime() < endTime) {
90 long id = rand.nextInt(NUM_IDS);
91 boolean readLock = rand.nextBoolean();
93 ReentrantReadWriteLock readWriteLock = idLock.getLock(id);
94 Lock lock = readLock ? readWriteLock.readLock() : readWriteLock.writeLock();
95 try {
96 lock.lock();
97 int sleepMs = 1 + rand.nextInt(4);
98 String owner = idOwner.get(id);
99 if (owner != null && LOG.isDebugEnabled()) {
100 LOG.debug((readLock ? "Read" : "Write") + "lock of Id " + id + " already taken by "
101 + owner + ", we are " + clientId);
104 idOwner.put(id, clientId);
105 Thread.sleep(sleepMs);
106 idOwner.remove(id);
108 } finally {
109 lock.unlock();
110 if (LOG.isDebugEnabled()) {
111 LOG.debug("Release " + (readLock ? "Read" : "Write") + " lock of Id" + id + ", we are "
112 + clientId);
116 return true;
121 @Test
122 public void testMultipleClients() throws Exception {
123 ExecutorService exec = Executors.newFixedThreadPool(NUM_THREADS);
124 try {
125 ExecutorCompletionService<Boolean> ecs = new ExecutorCompletionService<>(exec);
126 for (int i = 0; i < NUM_THREADS; ++i)
127 ecs.submit(new IdLockTestThread("client_" + i));
128 for (int i = 0; i < NUM_THREADS; ++i) {
129 Future<Boolean> result = ecs.take();
130 assertTrue(result.get());
132 int entryPoolSize = idLock.purgeAndGetEntryPoolSize();
133 LOG.debug("Size of entry pool after gc and purge: " + entryPoolSize);
134 ReferenceType refType = idLock.getReferenceType();
135 switch (refType) {
136 case WEAK:
137 // make sure the entry pool will be cleared after GC and purge call
138 assertEquals(0, entryPoolSize);
139 break;
140 case SOFT:
141 // make sure the entry pool won't be cleared when JVM memory is enough
142 // even after GC and purge call
143 assertEquals(NUM_IDS, entryPoolSize);
144 break;
145 default:
146 break;
148 } finally {
149 exec.shutdown();
150 exec.awaitTermination(5000, TimeUnit.MILLISECONDS);