1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.client;
19
20 import static org.junit.Assert.assertEquals;
21 import static org.junit.Assert.assertTrue;
22
23 import java.io.IOException;
24 import java.util.ArrayList;
25 import java.util.List;
26 import java.util.Random;
27 import java.util.concurrent.Callable;
28 import java.util.concurrent.CountDownLatch;
29 import java.util.concurrent.ExecutorService;
30 import java.util.concurrent.Executors;
31 import java.util.concurrent.Future;
32 import java.util.concurrent.atomic.AtomicInteger;
33 import java.util.concurrent.atomic.AtomicLong;
34
35 import org.apache.commons.logging.Log;
36 import org.apache.commons.logging.LogFactory;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.hbase.ClusterStatus;
39 import org.apache.hadoop.hbase.HBaseConfiguration;
40 import org.apache.hadoop.hbase.HBaseTestingUtility;
41 import org.apache.hadoop.hbase.HColumnDescriptor;
42 import org.apache.hadoop.hbase.HConstants;
43 import org.apache.hadoop.hbase.HTableDescriptor;
44 import org.apache.hadoop.hbase.ServerName;
45 import org.apache.hadoop.hbase.TableName;
46 import org.apache.hadoop.hbase.exceptions.PreemptiveFastFailException;
47 import org.apache.hadoop.hbase.regionserver.HRegionServer;
48 import org.apache.hadoop.hbase.regionserver.Region;
49 import org.apache.hadoop.hbase.ipc.RpcExecutor;
50 import org.apache.hadoop.hbase.ipc.SimpleRpcScheduler;
51 import org.apache.hadoop.hbase.testclassification.MediumTests;
52 import org.apache.hadoop.hbase.util.Bytes;
53 import org.apache.hadoop.hbase.util.LoadTestKVGenerator;
54 import org.junit.After;
55 import org.junit.AfterClass;
56 import org.junit.Before;
57 import org.junit.BeforeClass;
58 import org.junit.Test;
59 import org.junit.Ignore;
60 import org.junit.experimental.categories.Category;
61
62 @Category({MediumTests.class})
63 public class TestFastFail {
64 private static final Log LOG = LogFactory.getLog(TestFastFail.class);
65 private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
66 private static byte[] FAMILY = Bytes.toBytes("testFamily");
67 private static final Random random = new Random();
68 private static int SLAVES = 1;
69 private static byte[] QUALIFIER = Bytes.toBytes("testQualifier");
70 private static final int SLEEPTIME = 5000;
71
72
73
74
75 @BeforeClass
76 public static void setUpBeforeClass() throws Exception {
77
78 TEST_UTIL.getConfiguration().set(
79 RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY, "deadline");
80 TEST_UTIL.startMiniCluster(SLAVES);
81 }
82
83
84
85
86 @AfterClass
87 public static void tearDownAfterClass() throws Exception {
88 TEST_UTIL.shutdownMiniCluster();
89 }
90
91
92
93
94 @Before
95 public void setUp() throws Exception {
96 MyPreemptiveFastFailInterceptor.numBraveSouls.set(0);
97 CallQueueTooBigPffeInterceptor.numCallQueueTooBig.set(0);
98 }
99
100
101
102
103 @After
104 public void tearDown() throws Exception {
105
106 }
107
108 @Ignore ("Can go zombie -- see HBASE-14421; FIX") @Test
109 public void testFastFail() throws IOException, InterruptedException {
110 Admin admin = TEST_UTIL.getHBaseAdmin();
111
112 final String tableName = "testClientRelearningExperiment";
113 HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes
114 .toBytes(tableName)));
115 desc.addFamily(new HColumnDescriptor(FAMILY));
116 admin.createTable(desc, Bytes.toBytes("aaaa"), Bytes.toBytes("zzzz"), 32);
117 final long numRows = 1000;
118
119 Configuration conf = TEST_UTIL.getConfiguration();
120 conf.setLong(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, SLEEPTIME * 100);
121 conf.setInt(HConstants.HBASE_CLIENT_PAUSE, SLEEPTIME / 10);
122 conf.setBoolean(HConstants.HBASE_CLIENT_FAST_FAIL_MODE_ENABLED, true);
123 conf.setLong(HConstants.HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS, 0);
124 conf.setClass(HConstants.HBASE_CLIENT_FAST_FAIL_INTERCEPTOR_IMPL,
125 MyPreemptiveFastFailInterceptor.class,
126 PreemptiveFastFailInterceptor.class);
127
128 final Connection connection = ConnectionFactory.createConnection(conf);
129
130
131
132
133 List<Put> puts = new ArrayList<>();
134 for (long i = 0; i < numRows; i++) {
135 byte[] rowKey = longToByteArrayKey(i);
136 Put put = new Put(rowKey);
137 byte[] value = rowKey;
138 put.add(FAMILY, QUALIFIER, value);
139 puts.add(put);
140 }
141 try (Table table = connection.getTable(TableName.valueOf(tableName))) {
142 table.put(puts);
143 LOG.info("Written all puts.");
144 }
145
146
147
148
149
150 int nThreads = 100;
151 ExecutorService service = Executors.newFixedThreadPool(nThreads);
152 final CountDownLatch continueOtherHalf = new CountDownLatch(1);
153 final CountDownLatch doneHalfway = new CountDownLatch(nThreads);
154
155 final AtomicInteger numSuccessfullThreads = new AtomicInteger(0);
156 final AtomicInteger numFailedThreads = new AtomicInteger(0);
157
158
159 final AtomicLong totalTimeTaken = new AtomicLong(0);
160 final AtomicInteger numBlockedWorkers = new AtomicInteger(0);
161 final AtomicInteger numPreemptiveFastFailExceptions = new AtomicInteger(0);
162
163 List<Future<Boolean>> futures = new ArrayList<Future<Boolean>>();
164 for (int i = 0; i < nThreads; i++) {
165 futures.add(service.submit(new Callable<Boolean>() {
166
167
168
169
170
171 public Boolean call() throws Exception {
172 try (Table table = connection.getTable(TableName.valueOf(tableName))) {
173
174 int sleep = random.nextInt(250);
175 Thread.sleep(sleep);
176 long key = random.nextLong();
177 if (key < 0) {
178 key = -key;
179 }
180 key %= numRows;
181 byte[] row = longToByteArrayKey(key);
182 Get g = new Get(row);
183 g.addColumn(FAMILY, QUALIFIER);
184 try {
185 table.get(g);
186 } catch (Exception e) {
187 LOG.debug("Get failed : ", e);
188 doneHalfway.countDown();
189 return false;
190 }
191
192
193 doneHalfway.countDown();
194 continueOtherHalf.await();
195
196 long startTime = System.currentTimeMillis();
197 g = new Get(row);
198 g.addColumn(FAMILY, QUALIFIER);
199 try {
200 table.get(g);
201
202 numSuccessfullThreads.addAndGet(1);
203 } catch (Exception e) {
204 if (e instanceof PreemptiveFastFailException) {
205
206 numPreemptiveFastFailExceptions.addAndGet(1);
207 }
208
209 numFailedThreads.addAndGet(1);
210 return false;
211 } finally {
212 long enTime = System.currentTimeMillis();
213 totalTimeTaken.addAndGet(enTime - startTime);
214 if ((enTime - startTime) >= SLEEPTIME) {
215
216
217
218
219
220 numBlockedWorkers.addAndGet(1);
221 }
222 }
223 return true;
224 } catch (Exception e) {
225 LOG.error("Caught unknown exception", e);
226 doneHalfway.countDown();
227 return false;
228 }
229 }
230 }));
231 }
232
233 doneHalfway.await();
234
235 ClusterStatus status = TEST_UTIL.getHBaseCluster().getClusterStatus();
236
237
238 for (int i = 0; i < SLAVES; i++) {
239 HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(i);
240 List<Region> regions = server.getOnlineRegions(TableName.META_TABLE_NAME);
241 if (regions.size() > 0) continue;
242
243 server.getRpcServer().stop();
244 server.stop("Testing");
245 }
246
247
248 continueOtherHalf.countDown();
249
250 Thread.sleep(2 * SLEEPTIME);
251
252 TEST_UTIL.getHBaseCluster().restoreClusterStatus(status);
253
254 int numThreadsReturnedFalse = 0;
255 int numThreadsReturnedTrue = 0;
256 int numThreadsThrewExceptions = 0;
257 for (Future<Boolean> f : futures) {
258 try {
259 numThreadsReturnedTrue += f.get() ? 1 : 0;
260 numThreadsReturnedFalse += f.get() ? 0 : 1;
261 } catch (Exception e) {
262 numThreadsThrewExceptions++;
263 }
264 }
265 LOG.debug("numThreadsReturnedFalse:"
266 + numThreadsReturnedFalse
267 + " numThreadsReturnedTrue:"
268 + numThreadsReturnedTrue
269 + " numThreadsThrewExceptions:"
270 + numThreadsThrewExceptions
271 + " numFailedThreads:"
272 + numFailedThreads.get()
273 + " numSuccessfullThreads:"
274 + numSuccessfullThreads.get()
275 + " numBlockedWorkers:"
276 + numBlockedWorkers.get()
277 + " totalTimeWaited: "
278 + totalTimeTaken.get()
279 / (numBlockedWorkers.get() == 0 ? Long.MAX_VALUE : numBlockedWorkers
280 .get()) + " numPFFEs: " + numPreemptiveFastFailExceptions.get());
281
282 assertEquals("The expected number of all the successfull and the failed "
283 + "threads should equal the total number of threads that we spawned",
284 nThreads, numFailedThreads.get() + numSuccessfullThreads.get());
285 assertEquals(
286 "All the failures should be coming from the secondput failure",
287 numFailedThreads.get(), numThreadsReturnedFalse);
288 assertEquals("Number of threads that threw execution exceptions "
289 + "otherwise should be 0", numThreadsThrewExceptions, 0);
290 assertEquals("The regionservers that returned true should equal to the"
291 + " number of successful threads", numThreadsReturnedTrue,
292 numSuccessfullThreads.get());
293
294
295
296
297
298
299
300
301
302
303
304
305
306 assertTrue(
307 "Only few thread should ideally be waiting for the dead "
308 + "regionserver to be coming back. numBlockedWorkers:"
309 + numBlockedWorkers.get() + " threads that retried : "
310 + MyPreemptiveFastFailInterceptor.numBraveSouls.get(),
311 numBlockedWorkers.get() <= MyPreemptiveFastFailInterceptor.numBraveSouls
312 .get());
313 }
314
315 @Test
316 public void testCallQueueTooBigExceptionDoesntTriggerPffe() throws Exception {
317 Admin admin = TEST_UTIL.getHBaseAdmin();
318
319 final String tableName = "testCallQueueTooBigException";
320 HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes
321 .toBytes(tableName)));
322 desc.addFamily(new HColumnDescriptor(FAMILY));
323 admin.createTable(desc, Bytes.toBytes("aaaa"), Bytes.toBytes("zzzz"), 3);
324
325 Configuration conf = TEST_UTIL.getConfiguration();
326 conf.setLong(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 100);
327 conf.setInt(HConstants.HBASE_CLIENT_PAUSE, 500);
328 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
329
330 conf.setBoolean(HConstants.HBASE_CLIENT_FAST_FAIL_MODE_ENABLED, true);
331 conf.setLong(HConstants.HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS, 0);
332 conf.setClass(HConstants.HBASE_CLIENT_FAST_FAIL_INTERCEPTOR_IMPL,
333 CallQueueTooBigPffeInterceptor.class, PreemptiveFastFailInterceptor.class);
334
335 final Connection connection = ConnectionFactory.createConnection(conf);
336
337
338 SimpleRpcScheduler srs = (SimpleRpcScheduler)
339 TEST_UTIL.getHBaseCluster().getRegionServer(0).getRpcServer().getScheduler();
340 Configuration newConf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
341 newConf.setInt("hbase.ipc.server.max.callqueue.length", 0);
342 srs.onConfigurationChange(newConf);
343
344 try (Table table = connection.getTable(TableName.valueOf(tableName))) {
345 Get get = new Get(new byte[1]);
346 table.get(get);
347 } catch (Throwable ex) {
348 }
349
350 assertEquals("We should have not entered PFFE mode on CQTBE, but we did;"
351 + " number of times this mode should have been entered:", 0,
352 CallQueueTooBigPffeInterceptor.numCallQueueTooBig.get());
353
354 newConf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
355 newConf.setInt("hbase.ipc.server.max.callqueue.length", 250);
356 srs.onConfigurationChange(newConf);
357 }
358
359 public static class MyPreemptiveFastFailInterceptor extends
360 PreemptiveFastFailInterceptor {
361 public static AtomicInteger numBraveSouls = new AtomicInteger();
362
363 @Override
364 protected boolean shouldRetryInspiteOfFastFail(FailureInfo fInfo) {
365 boolean ret = super.shouldRetryInspiteOfFastFail(fInfo);
366 if (ret)
367 numBraveSouls.addAndGet(1);
368 return ret;
369 }
370
371 public MyPreemptiveFastFailInterceptor(Configuration conf) {
372 super(conf);
373 }
374 }
375
376 private byte[] longToByteArrayKey(long rowKey) {
377 return LoadTestKVGenerator.md5PrefixedKey(rowKey).getBytes();
378 }
379
380 public static class CallQueueTooBigPffeInterceptor extends
381 PreemptiveFastFailInterceptor {
382 public static AtomicInteger numCallQueueTooBig = new AtomicInteger();
383
384 @Override
385 protected void handleFailureToServer(ServerName serverName, Throwable t) {
386 super.handleFailureToServer(serverName, t);
387 numCallQueueTooBig.incrementAndGet();
388 }
389
390 public CallQueueTooBigPffeInterceptor(Configuration conf) {
391 super(conf);
392 }
393 }
394 }