1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.test;
20
21 import java.io.IOException;
22 import java.util.List;
23 import java.util.concurrent.Executors;
24 import java.util.concurrent.ScheduledExecutorService;
25 import java.util.concurrent.ScheduledFuture;
26 import java.util.concurrent.TimeUnit;
27 import java.util.concurrent.atomic.AtomicLong;
28
29 import org.apache.commons.lang.math.RandomUtils;
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.hbase.HBaseConfiguration;
34 import org.apache.hadoop.hbase.IntegrationTestIngest;
35 import org.apache.hadoop.hbase.IntegrationTestingUtility;
36 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
37 import org.apache.hadoop.hbase.TableName;
38 import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
39 import org.apache.hadoop.hbase.client.Admin;
40 import org.apache.hadoop.hbase.client.Consistency;
41 import org.apache.hadoop.hbase.client.Get;
42 import org.apache.hadoop.hbase.client.Result;
43 import org.apache.hadoop.hbase.client.Table;
44 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
45 import org.apache.hadoop.hbase.util.LoadTestTool;
46 import org.apache.hadoop.hbase.util.MultiThreadedReader;
47 import org.apache.hadoop.hbase.util.Threads;
48 import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
49 import org.apache.hadoop.util.StringUtils;
50 import org.apache.hadoop.util.ToolRunner;
51 import org.junit.Assert;
52 import org.junit.experimental.categories.Category;
53
54 import com.google.common.collect.Lists;
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92 @Category(IntegrationTests.class)
93 public class IntegrationTestTimeBoundedRequestsWithRegionReplicas extends IntegrationTestIngest {
94
95 private static final Log LOG = LogFactory.getLog(
96 IntegrationTestTimeBoundedRequestsWithRegionReplicas.class);
97
98 private static final String TEST_NAME
99 = IntegrationTestTimeBoundedRequestsWithRegionReplicas.class.getSimpleName();
100
101 protected static final long DEFAULT_GET_TIMEOUT = 5000;
102 protected static final String GET_TIMEOUT_KEY = "get_timeout_ms";
103
104 protected static final long DEFAUL_CHAOS_MONKEY_DELAY = 20 * 1000;
105 protected static final String CHAOS_MONKEY_DELAY_KEY = "chaos_monkey_delay";
106
107 protected static final int DEFAULT_REGION_REPLICATION = 3;
108
109 @Override
110 protected void startMonkey() throws Exception {
111
112 }
113
114 @Override
115 protected MonkeyFactory getDefaultMonkeyFactory() {
116 return MonkeyFactory.getFactory(MonkeyFactory.CALM);
117 }
118
119 @Override
120 public void setConf(Configuration conf) {
121 super.setConf(conf);
122
123 String clazz = this.getClass().getSimpleName();
124 conf.setIfUnset(String.format("%s.%s", clazz, LoadTestTool.OPT_REGION_REPLICATION),
125 Integer.toString(DEFAULT_REGION_REPLICATION));
126 }
127
128 protected void writeData(int colsPerKey, int recordSize, int writeThreads,
129 long startKey, long numKeys) throws IOException {
130 int ret = loadTool.run(getArgsForLoadTestTool("-write",
131 String.format("%d:%d:%d", colsPerKey, recordSize, writeThreads), startKey, numKeys));
132 if (0 != ret) {
133 String errorMsg = "Load failed with error code " + ret;
134 LOG.error(errorMsg);
135 Assert.fail(errorMsg);
136 }
137 }
138
139 @Override
140 protected void runIngestTest(long defaultRunTime, long keysPerServerPerIter, int colsPerKey,
141 int recordSize, int writeThreads, int readThreads) throws Exception {
142 LOG.info("Cluster size:"+
143 util.getHBaseClusterInterface().getClusterStatus().getServersSize());
144
145 long start = System.currentTimeMillis();
146 String runtimeKey = String.format(RUN_TIME_KEY, this.getClass().getSimpleName());
147 long runtime = util.getConfiguration().getLong(runtimeKey, defaultRunTime);
148 long startKey = 0;
149
150 long numKeys = getNumKeys(keysPerServerPerIter);
151
152
153
154 LOG.info("Writing some data to the table");
155 writeData(colsPerKey, recordSize, writeThreads, startKey, numKeys);
156
157
158 LOG.info("Flushing the table");
159 Admin admin = util.getHBaseAdmin();
160 admin.flush(getTablename());
161
162
163 long refreshTime = conf.getLong(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 0);
164 if (refreshTime > 0 && refreshTime <= 10000) {
165 LOG.info("Sleeping " + refreshTime + "ms to ensure that the data is replicated");
166 Threads.sleep(refreshTime);
167 } else {
168 LOG.info("Reopening the table");
169 admin.disableTable(getTablename());
170 admin.enableTable(getTablename());
171 }
172
173
174
175
176
177
178 long chaosMonkeyDelay = conf.getLong(String.format("%s.%s", TEST_NAME, CHAOS_MONKEY_DELAY_KEY)
179 , DEFAUL_CHAOS_MONKEY_DELAY);
180 ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor();
181 LOG.info(String.format("ChaosMonkey delay is : %d seconds. Will start %s " +
182 "ChaosMonkey after delay", chaosMonkeyDelay / 1000, monkeyToUse));
183 ScheduledFuture<?> result = executorService.schedule(new Runnable() {
184 @Override
185 public void run() {
186 try {
187 LOG.info("Starting ChaosMonkey");
188 monkey.start();
189 monkey.waitForStop();
190 } catch (Exception e) {
191 LOG.warn(StringUtils.stringifyException(e));
192 }
193
194 }
195 }, chaosMonkeyDelay, TimeUnit.MILLISECONDS);
196
197
198
199 long remainingTime = runtime - (System.currentTimeMillis() - start);
200 LOG.info("Reading random keys from the table for " + remainingTime/60000 + " min");
201 this.conf.setLong(
202 String.format(RUN_TIME_KEY, TimeBoundedMultiThreadedReader.class.getSimpleName())
203 , remainingTime);
204
205
206 try {
207 int ret = loadTool.run(getArgsForLoadTestTool("-read", String.format("100:%d", readThreads)
208 , startKey, numKeys));
209 if (0 != ret) {
210 String errorMsg = "Verification failed with error code " + ret;
211 LOG.error(errorMsg);
212 Assert.fail(errorMsg);
213 }
214 } finally {
215 if (result != null) result.cancel(false);
216 monkey.stop("Stopping the test");
217 monkey.waitForStop();
218 executorService.shutdown();
219 }
220 }
221
222 @Override
223 protected String[] getArgsForLoadTestTool(String mode, String modeSpecificArg, long startKey,
224 long numKeys) {
225 List<String> args = Lists.newArrayList(super.getArgsForLoadTestTool(
226 mode, modeSpecificArg, startKey, numKeys));
227 args.add("-reader");
228 args.add(TimeBoundedMultiThreadedReader.class.getName());
229 return args.toArray(new String[args.size()]);
230 }
231
232 public static class TimeBoundedMultiThreadedReader extends MultiThreadedReader {
233 protected long timeoutNano;
234 protected AtomicLong timedOutReads = new AtomicLong();
235 protected long runTime;
236 protected Thread timeoutThread;
237 protected AtomicLong staleReads = new AtomicLong();
238
239 public TimeBoundedMultiThreadedReader(LoadTestDataGenerator dataGen, Configuration conf,
240 TableName tableName, double verifyPercent) throws IOException {
241 super(dataGen, conf, tableName, verifyPercent);
242 long timeoutMs = conf.getLong(
243 String.format("%s.%s", TEST_NAME, GET_TIMEOUT_KEY), DEFAULT_GET_TIMEOUT);
244 timeoutNano = timeoutMs * 1000000;
245 LOG.info("Timeout for gets: " + timeoutMs);
246 String runTimeKey = String.format(RUN_TIME_KEY, this.getClass().getSimpleName());
247 this.runTime = conf.getLong(runTimeKey, -1);
248 if (this.runTime <= 0) {
249 throw new IllegalArgumentException("Please configure " + runTimeKey);
250 }
251 }
252
253 @Override
254 public void waitForFinish() {
255 try {
256 this.timeoutThread.join();
257 } catch (InterruptedException e) {
258 e.printStackTrace();
259 }
260 this.aborted = true;
261 super.waitForFinish();
262 }
263
264 @Override
265 protected String progressInfo() {
266 StringBuilder builder = new StringBuilder(super.progressInfo());
267 appendToStatus(builder, "stale_reads", staleReads.get());
268 appendToStatus(builder, "get_timeouts", timedOutReads.get());
269 return builder.toString();
270 }
271
272 @Override
273 public void start(long startKey, long endKey, int numThreads) throws IOException {
274 super.start(startKey, endKey, numThreads);
275 this.timeoutThread = new TimeoutThread(this.runTime);
276 this.timeoutThread.start();
277 }
278
279 @Override
280 protected HBaseReaderThread createReaderThread(int readerId) throws IOException {
281 return new TimeBoundedMultiThreadedReaderThread(readerId);
282 }
283
284 private class TimeoutThread extends Thread {
285 long timeout;
286 long reportInterval = 60000;
287 public TimeoutThread(long timeout) {
288 this.timeout = timeout;
289 }
290
291 @Override
292 public void run() {
293 while (true) {
294 long rem = Math.min(timeout, reportInterval);
295 if (rem <= 0) {
296 break;
297 }
298 LOG.info("Remaining execution time:" + timeout / 60000 + " min");
299 Threads.sleep(rem);
300 timeout -= rem;
301 }
302 }
303 }
304
305 public class TimeBoundedMultiThreadedReaderThread
306 extends MultiThreadedReader.HBaseReaderThread {
307
308 public TimeBoundedMultiThreadedReaderThread(int readerId) throws IOException {
309 super(readerId);
310 }
311
312 @Override
313 protected Get createGet(long keyToRead) throws IOException {
314 Get get = super.createGet(keyToRead);
315 get.setConsistency(Consistency.TIMELINE);
316 return get;
317 }
318
319 @Override
320 protected long getNextKeyToRead() {
321
322 long key = startKey + Math.abs(RandomUtils.nextLong())
323 % (endKey - startKey);
324 return key;
325 }
326
327 @Override
328 protected void verifyResultsAndUpdateMetrics(boolean verify, Get[] gets, long elapsedNano,
329 Result[] results, Table table, boolean isNullExpected)
330 throws IOException {
331 super.verifyResultsAndUpdateMetrics(verify, gets, elapsedNano, results, table, isNullExpected);
332 for (Result r : results) {
333 if (r.isStale()) staleReads.incrementAndGet();
334 }
335
336
337 if (elapsedNano > timeoutNano) {
338 timedOutReads.incrementAndGet();
339 numReadFailures.addAndGet(1);
340 }
341 }
342 }
343 }
344
345 public static void main(String[] args) throws Exception {
346 Configuration conf = HBaseConfiguration.create();
347 IntegrationTestingUtility.setUseDistributedCluster(conf);
348 int ret = ToolRunner.run(conf, new IntegrationTestTimeBoundedRequestsWithRegionReplicas(), args);
349 System.exit(ret);
350 }
351 }