View Javadoc

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
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
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.
17   */
18  
19  package org.apache.hadoop.hbase.test;
20  
21  import java.io.IOException;
22  import java.util.HashSet;
23  import java.util.List;
24  import java.util.Set;
25  import java.util.concurrent.Executors;
26  import java.util.concurrent.ScheduledExecutorService;
27  import java.util.concurrent.ScheduledFuture;
28  import java.util.concurrent.TimeUnit;
29  import java.util.concurrent.atomic.AtomicLong;
30  
31  import org.apache.commons.lang.math.RandomUtils;
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.hbase.HBaseConfiguration;
36  import org.apache.hadoop.hbase.HRegionLocation;
37  import org.apache.hadoop.hbase.IntegrationTestIngest;
38  import org.apache.hadoop.hbase.IntegrationTestingUtility;
39  import org.apache.hadoop.hbase.RegionLocations;
40  import org.apache.hadoop.hbase.testclassification.IntegrationTests;
41  import org.apache.hadoop.hbase.TableName;
42  import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
43  import org.apache.hadoop.hbase.client.Admin;
44  import org.apache.hadoop.hbase.client.ClusterConnection;
45  import org.apache.hadoop.hbase.client.Consistency;
46  import org.apache.hadoop.hbase.client.Get;
47  import org.apache.hadoop.hbase.client.Result;
48  import org.apache.hadoop.hbase.client.Table;
49  import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
50  import org.apache.hadoop.hbase.util.LoadTestTool;
51  import org.apache.hadoop.hbase.util.MultiThreadedReader;
52  import org.apache.hadoop.hbase.util.Threads;
53  import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
54  import org.apache.hadoop.util.StringUtils;
55  import org.apache.hadoop.util.ToolRunner;
56  import org.junit.Assert;
57  import org.junit.experimental.categories.Category;
58  
59  import com.google.common.collect.Lists;
60  
61  /**
62   * An IntegrationTest for doing reads with a timeout, to a read-only table with region
63   * replicas. ChaosMonkey is run which kills the region servers and master, but ensures
64   * that meta region server is not killed, and at most 2 region servers are dead at any point
65   * in time. The expected behavior is that all reads with stale mode true will return
66   * before the timeout (5 sec by default). The test fails if the read requests does not finish
67   * in time.
68   *
69   * <p> This test uses LoadTestTool to read and write the data from a single client but
70   * multiple threads. The data is written first, then we allow the region replicas to catch
71   * up. Then we start the reader threads doing get requests with stale mode true. Chaos Monkey is
72   * started after some delay (20 sec by default) after the reader threads are started so that
73   * there is enough time to fully cache meta.
74   *
75   * These parameters (and some other parameters from LoadTestTool) can be used to
76   * control behavior, given values are default:
77   * <pre>
78   * -Dhbase.IntegrationTestTimeBoundedRequestsWithRegionReplicas.runtime=600000
79   * -DIntegrationTestTimeBoundedRequestsWithRegionReplicas.num_regions_per_server=5
80   * -DIntegrationTestTimeBoundedRequestsWithRegionReplicas.get_timeout_ms=5000
81   * -DIntegrationTestTimeBoundedRequestsWithRegionReplicas.num_keys_per_server=2500
82   * -DIntegrationTestTimeBoundedRequestsWithRegionReplicas.region_replication=3
83   * -DIntegrationTestTimeBoundedRequestsWithRegionReplicas.num_read_threads=20
84   * -DIntegrationTestTimeBoundedRequestsWithRegionReplicas.num_write_threads=20
85   * -DIntegrationTestTimeBoundedRequestsWithRegionReplicas.num_regions_per_server=5
86   * -DIntegrationTestTimeBoundedRequestsWithRegionReplicas.chaos_monkey_delay=20000
87   * </pre>
88   * Use this test with "serverKilling" ChaosMonkey. Sample usage:
89   * <pre>
90   * hbase org.apache.hadoop.hbase.test.IntegrationTestTimeBoundedRequestsWithRegionReplicas
91   * -Dhbase.IntegrationTestTimeBoundedRequestsWithRegionReplicas.runtime=600000
92   * -DIntegrationTestTimeBoundedRequestsWithRegionReplicas.num_write_threads=40
93   * -DIntegrationTestTimeBoundedRequestsWithRegionReplicas.num_read_threads=40
94   * -Dhbase.ipc.client.allowsInterrupt=true --monkey serverKilling
95   * </pre>
96   */
97  @Category(IntegrationTests.class)
98  public class IntegrationTestTimeBoundedRequestsWithRegionReplicas extends IntegrationTestIngest {
99  
100   private static final Log LOG = LogFactory.getLog(
101     IntegrationTestTimeBoundedRequestsWithRegionReplicas.class);
102 
103   private static final String TEST_NAME
104     = IntegrationTestTimeBoundedRequestsWithRegionReplicas.class.getSimpleName();
105 
106   protected static final long DEFAULT_GET_TIMEOUT = 5000; // 5 sec
107   protected static final String GET_TIMEOUT_KEY = "get_timeout_ms";
108 
109   protected static final long DEFAUL_CHAOS_MONKEY_DELAY = 20 * 1000; // 20 sec
110   protected static final String CHAOS_MONKEY_DELAY_KEY = "chaos_monkey_delay";
111 
112   protected static final int DEFAULT_REGION_REPLICATION = 3;
113 
114   @Override
115   protected void startMonkey() throws Exception {
116     // we do not want to start the monkey at the start of the test.
117   }
118 
119   @Override
120   protected MonkeyFactory getDefaultMonkeyFactory() {
121     return MonkeyFactory.getFactory(MonkeyFactory.CALM);
122   }
123 
124   @Override
125   public void setConf(Configuration conf) {
126     super.setConf(conf);
127     // default replication for this test is 3
128     String clazz = this.getClass().getSimpleName();
129     conf.setIfUnset(String.format("%s.%s", clazz, LoadTestTool.OPT_REGION_REPLICATION),
130       Integer.toString(DEFAULT_REGION_REPLICATION));
131   }
132 
133   protected void writeData(int colsPerKey, int recordSize, int writeThreads,
134       long startKey, long numKeys) throws IOException {
135     int ret = loadTool.run(getArgsForLoadTestTool("-write",
136       String.format("%d:%d:%d", colsPerKey, recordSize, writeThreads), startKey, numKeys));
137     if (0 != ret) {
138       String errorMsg = "Load failed with error code " + ret;
139       LOG.error(errorMsg);
140       Assert.fail(errorMsg);
141     }
142   }
143 
144   @Override
145   protected void runIngestTest(long defaultRunTime, long keysPerServerPerIter, int colsPerKey,
146       int recordSize, int writeThreads, int readThreads) throws Exception {
147     LOG.info("Cluster size:"+
148       util.getHBaseClusterInterface().getClusterStatus().getServersSize());
149 
150     long start = System.currentTimeMillis();
151     String runtimeKey = String.format(RUN_TIME_KEY, this.getClass().getSimpleName());
152     long runtime = util.getConfiguration().getLong(runtimeKey, defaultRunTime);
153     long startKey = 0;
154 
155     long numKeys = getNumKeys(keysPerServerPerIter);
156 
157 
158     // write data once
159     LOG.info("Writing some data to the table");
160     writeData(colsPerKey, recordSize, writeThreads, startKey, numKeys);
161 
162     // flush the table
163     LOG.info("Flushing the table");
164     Admin admin = util.getHBaseAdmin();
165     admin.flush(getTablename());
166 
167     // re-open the regions to make sure that the replicas are up to date
168     long refreshTime = conf.getLong(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 0);
169     if (refreshTime > 0 && refreshTime <= 10000) {
170       LOG.info("Sleeping " + refreshTime + "ms to ensure that the data is replicated");
171       Threads.sleep(refreshTime*3);
172     } else {
173       LOG.info("Reopening the table");
174       admin.disableTable(getTablename());
175       admin.enableTable(getTablename());
176     }
177 
178     // We should only start the ChaosMonkey after the readers are started and have cached
179     // all of the region locations. Because the meta is not replicated, the timebounded reads
180     // will timeout if meta server is killed.
181     // We will start the chaos monkey after 1 minute, and since the readers are reading random
182     // keys, it should be enough to cache every region entry.
183     long chaosMonkeyDelay = conf.getLong(String.format("%s.%s", TEST_NAME, CHAOS_MONKEY_DELAY_KEY)
184       , DEFAUL_CHAOS_MONKEY_DELAY);
185     ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor();
186     LOG.info(String.format("ChaosMonkey delay is : %d seconds. Will start %s " +
187         "ChaosMonkey after delay", chaosMonkeyDelay / 1000, monkeyToUse));
188     ScheduledFuture<?> result = executorService.schedule(new Runnable() {
189       @Override
190       public void run() {
191         try {
192           LOG.info("Starting ChaosMonkey");
193           monkey.start();
194           monkey.waitForStop();
195         } catch (Exception e) {
196           LOG.warn(StringUtils.stringifyException(e));
197         }
198 
199       }
200     }, chaosMonkeyDelay, TimeUnit.MILLISECONDS);
201 
202     // set the intended run time for the reader. The reader will do read requests
203     // to random keys for this amount of time.
204     long remainingTime = runtime - (System.currentTimeMillis() - start);
205     if (remainingTime <= 0) {
206       LOG.error("The amount of time left for the test to perform random reads is "
207           + "non-positive. Increase the test execution time via "
208           + String.format(RUN_TIME_KEY,
209                 IntegrationTestTimeBoundedRequestsWithRegionReplicas.class.getSimpleName())
210           + " or reduce the amount of data written per server via "
211           + IntegrationTestTimeBoundedRequestsWithRegionReplicas.class.getSimpleName()
212           + "." + IntegrationTestIngest.NUM_KEYS_PER_SERVER_KEY);
213       throw new IllegalArgumentException("No time remains to execute random reads");
214     }
215     LOG.info("Reading random keys from the table for " + remainingTime/60000 + " min");
216     this.conf.setLong(
217       String.format(RUN_TIME_KEY, TimeBoundedMultiThreadedReader.class.getSimpleName())
218       , remainingTime); // load tool shares the same conf
219 
220     // now start the readers which will run for configured run time
221     try {
222       int ret = loadTool.run(getArgsForLoadTestTool("-read", String.format("100:%d", readThreads)
223         , startKey, numKeys));
224       if (0 != ret) {
225         String errorMsg = "Verification failed with error code " + ret;
226         LOG.error(errorMsg);
227         Assert.fail(errorMsg);
228       }
229     } finally {
230       if (result != null) result.cancel(false);
231       monkey.stop("Stopping the test");
232       monkey.waitForStop();
233       executorService.shutdown();
234     }
235   }
236 
237   @Override
238   protected String[] getArgsForLoadTestTool(String mode, String modeSpecificArg, long startKey,
239       long numKeys) {
240     List<String> args = Lists.newArrayList(super.getArgsForLoadTestTool(
241       mode, modeSpecificArg, startKey, numKeys));
242     args.add("-reader");
243     args.add(TimeBoundedMultiThreadedReader.class.getName());
244     return args.toArray(new String[args.size()]);
245   }
246 
247   public static class TimeBoundedMultiThreadedReader extends MultiThreadedReader {
248     protected long timeoutNano;
249     protected AtomicLong timedOutReads = new AtomicLong();
250     protected long runTime;
251     protected Thread timeoutThread;
252     protected AtomicLong staleReads = new AtomicLong();
253 
254     public TimeBoundedMultiThreadedReader(LoadTestDataGenerator dataGen, Configuration conf,
255         TableName tableName, double verifyPercent) throws IOException {
256       super(dataGen, conf, tableName, verifyPercent);
257       long timeoutMs = conf.getLong(
258         String.format("%s.%s", TEST_NAME, GET_TIMEOUT_KEY), DEFAULT_GET_TIMEOUT);
259       timeoutNano = timeoutMs * 1000000;
260       LOG.info("Timeout for gets: " + timeoutMs);
261       String runTimeKey = String.format(RUN_TIME_KEY, this.getClass().getSimpleName());
262       this.runTime = conf.getLong(runTimeKey, -1);
263       if (this.runTime <= 0) {
264         throw new IllegalArgumentException("Please configure " + runTimeKey);
265       }
266     }
267 
268     @Override
269     public void waitForFinish() {
270       try {
271         this.timeoutThread.join();
272       } catch (InterruptedException e) {
273         e.printStackTrace();
274       }
275       this.aborted = true;
276       super.waitForFinish();
277     }
278 
279     @Override
280     protected String progressInfo() {
281       StringBuilder builder = new StringBuilder(super.progressInfo());
282       appendToStatus(builder, "stale_reads", staleReads.get());
283       appendToStatus(builder, "get_timeouts", timedOutReads.get());
284       return builder.toString();
285     }
286 
287     @Override
288     public void start(long startKey, long endKey, int numThreads) throws IOException {
289       super.start(startKey, endKey, numThreads);
290       this.timeoutThread = new TimeoutThread(this.runTime);
291       this.timeoutThread.start();
292     }
293 
294     @Override
295     protected HBaseReaderThread createReaderThread(int readerId) throws IOException {
296       return new TimeBoundedMultiThreadedReaderThread(readerId);
297     }
298 
299     private class TimeoutThread extends Thread {
300       long timeout;
301       long reportInterval = 60000;
302       public TimeoutThread(long timeout) {
303         this.timeout = timeout;
304       }
305 
306       @Override
307       public void run() {
308         while (true) {
309           long rem = Math.min(timeout, reportInterval);
310           if (rem <= 0) {
311             break;
312           }
313           LOG.info("Remaining execution time:" + timeout / 60000 + " min");
314           Threads.sleep(rem);
315           timeout -= rem;
316         }
317       }
318     }
319 
320     public class TimeBoundedMultiThreadedReaderThread
321       extends MultiThreadedReader.HBaseReaderThread {
322 
323       public TimeBoundedMultiThreadedReaderThread(int readerId) throws IOException {
324         super(readerId);
325       }
326 
327       @Override
328       protected Get createGet(long keyToRead) throws IOException {
329         Get get = super.createGet(keyToRead);
330         get.setConsistency(Consistency.TIMELINE);
331         return get;
332       }
333 
334       @Override
335       protected long getNextKeyToRead() {
336         // always read a random key, assuming that the writer has finished writing all keys
337         long key = startKey + Math.abs(RandomUtils.nextLong())
338             % (endKey - startKey);
339         return key;
340       }
341 
342       @Override
343       protected void verifyResultsAndUpdateMetrics(boolean verify, Get[] gets, long elapsedNano,
344           Result[] results, Table table, boolean isNullExpected)
345           throws IOException {
346         super.verifyResultsAndUpdateMetrics(verify, gets, elapsedNano, results, table, isNullExpected);
347         for (Result r : results) {
348           if (r.isStale()) staleReads.incrementAndGet();
349         }
350         // we actually do not timeout and cancel the reads after timeout. We just wait for the RPC
351         // to complete, but if the request took longer than timeout, we treat that as error.
352         if (elapsedNano > timeoutNano) {
353           timedOutReads.incrementAndGet();
354           numReadFailures.addAndGet(1); // fail the test
355           for (Result r : results) {
356             LOG.error("FAILED FOR " + r);
357             RegionLocations rl = ((ClusterConnection)connection).
358                 locateRegion(tableName, r.getRow(), true, true);
359             HRegionLocation locations[] = rl.getRegionLocations();
360             for (HRegionLocation h : locations) {
361               LOG.error("LOCATION " + h);
362             }
363           }
364         }
365       }
366     }
367   }
368 
369   public static void main(String[] args) throws Exception {
370     Configuration conf = HBaseConfiguration.create();
371     IntegrationTestingUtility.setUseDistributedCluster(conf);
372     int ret = ToolRunner.run(conf, new IntegrationTestTimeBoundedRequestsWithRegionReplicas(), args);
373     System.exit(ret);
374   }
375 }