HBASE-26700 The way we bypass broken track file is not enough in StoreFileListFile...
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / regionserver / TestSyncTimeRangeTracker.java
blobd9f661abc32e38b49f9503d9ef8a678e63aad6f9
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.regionserver;
20 import static org.junit.Assert.assertTrue;
21 import java.util.concurrent.ThreadLocalRandom;
22 import org.apache.hadoop.hbase.HBaseClassTestRule;
23 import org.apache.hadoop.hbase.testclassification.MediumTests;
24 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
25 import org.junit.ClassRule;
26 import org.junit.Test;
27 import org.junit.experimental.categories.Category;
29 @Category({RegionServerTests.class, MediumTests.class})
30 public class TestSyncTimeRangeTracker extends TestSimpleTimeRangeTracker {
32 @ClassRule
33 public static final HBaseClassTestRule CLASS_RULE =
34 HBaseClassTestRule.forClass(TestSyncTimeRangeTracker.class);
36 private static final int NUM_KEYS = 10000000;
37 private static final int NUM_OF_THREADS = 20;
39 @Override
40 protected TimeRangeTracker getTimeRangeTracker() {
41 return TimeRangeTracker.create(TimeRangeTracker.Type.SYNC);
44 @Override
45 protected TimeRangeTracker getTimeRangeTracker(long min, long max) {
46 return TimeRangeTracker.create(TimeRangeTracker.Type.SYNC, min, max);
49 /**
50 * Run a bunch of threads against a single TimeRangeTracker and ensure we arrive
51 * at right range. Here we do ten threads each incrementing over 100k at an offset
52 * of the thread index; max is 10 * 10k and min is 0.
54 @Test
55 public void testArriveAtRightAnswer() throws InterruptedException {
56 final TimeRangeTracker trr = getTimeRangeTracker();
57 final int threadCount = 10;
58 final int calls = 1000 * 1000;
59 Thread [] threads = new Thread[threadCount];
60 for (int i = 0; i < threads.length; i++) {
61 Thread t = new Thread("" + i) {
62 @Override
63 public void run() {
64 int offset = Integer.parseInt(getName());
65 boolean even = offset % 2 == 0;
66 if (even) {
67 for (int i = (offset * calls); i < calls; i++) {
68 trr.includeTimestamp(i);
70 } else {
71 int base = offset * calls;
72 for (int i = base + calls; i >= base; i--) {
73 trr.includeTimestamp(i);
78 t.start();
79 threads[i] = t;
81 for (int i = 0; i < threads.length; i++) {
82 threads[i].join();
85 assertTrue(trr.getMax() == calls * threadCount);
86 assertTrue(trr.getMin() == 0);
89 static class RandomTestData {
90 private long[] keys = new long[NUM_KEYS];
91 private long min = Long.MAX_VALUE;
92 private long max = 0;
94 public RandomTestData() {
95 if (ThreadLocalRandom.current().nextInt(NUM_OF_THREADS) % 2 == 0) {
96 for (int i = 0; i < NUM_KEYS; i++) {
97 keys[i] = i + ThreadLocalRandom.current().nextLong(NUM_OF_THREADS);
98 if (keys[i] < min) {
99 min = keys[i];
101 if (keys[i] > max) {
102 max = keys[i];
105 } else {
106 for (int i = NUM_KEYS - 1; i >= 0; i--) {
107 keys[i] = i + ThreadLocalRandom.current().nextLong(NUM_OF_THREADS);
108 if (keys[i] < min) {
109 min = keys[i];
111 if (keys[i] > max) {
112 max = keys[i];
118 public long getMax() {
119 return this.max;
122 public long getMin() {
123 return this.min;
127 static class TrtUpdateRunnable implements Runnable {
129 private TimeRangeTracker trt;
130 private RandomTestData data;
131 public TrtUpdateRunnable(final TimeRangeTracker trt, final RandomTestData data) {
132 this.trt = trt;
133 this.data = data;
136 @Override
137 public void run() {
138 for (long key : data.keys) {
139 trt.includeTimestamp(key);
145 * Run a bunch of threads against a single TimeRangeTracker and ensure we arrive
146 * at right range. The data chosen is going to ensure that there are lots collisions, i.e,
147 * some other threads may already update the value while one tries to update min/max value.
149 @Test
150 public void testConcurrentIncludeTimestampCorrectness() {
151 RandomTestData[] testData = new RandomTestData[NUM_OF_THREADS];
152 long min = Long.MAX_VALUE, max = 0;
153 for (int i = 0; i < NUM_OF_THREADS; i ++) {
154 testData[i] = new RandomTestData();
155 if (testData[i].getMin() < min) {
156 min = testData[i].getMin();
158 if (testData[i].getMax() > max) {
159 max = testData[i].getMax();
163 TimeRangeTracker trt = TimeRangeTracker.create(TimeRangeTracker.Type.SYNC);
165 Thread[] t = new Thread[NUM_OF_THREADS];
166 for (int i = 0; i < NUM_OF_THREADS; i++) {
167 t[i] = new Thread(new TrtUpdateRunnable(trt, testData[i]));
168 t[i].start();
171 for (Thread thread : t) {
172 try {
173 thread.join();
174 } catch (InterruptedException e) {
175 e.printStackTrace();
179 assertTrue(min == trt.getMin());
180 assertTrue(max == trt.getMax());