001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.client;
019
020import static org.apache.hadoop.hbase.client.ConnectionConfiguration.HBASE_CLIENT_META_READ_RPC_TIMEOUT_KEY;
021import static org.apache.hadoop.hbase.client.ConnectionConfiguration.HBASE_CLIENT_META_SCANNER_TIMEOUT;
022import static org.apache.hadoop.hbase.client.ConnectionConfiguration.HBASE_CLIENT_USE_SCANNER_TIMEOUT_PERIOD_FOR_NEXT_CALLS;
023import static org.junit.Assert.assertEquals;
024import static org.junit.Assert.assertFalse;
025import static org.junit.Assert.assertTrue;
026import static org.junit.Assert.fail;
027
028import java.io.IOException;
029import java.net.SocketTimeoutException;
030import java.util.Collection;
031import java.util.concurrent.TimeUnit;
032import java.util.function.Supplier;
033import org.apache.hadoop.conf.Configuration;
034import org.apache.hadoop.hbase.HBaseClassTestRule;
035import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
036import org.apache.hadoop.hbase.HBaseTestingUtility;
037import org.apache.hadoop.hbase.HConstants;
038import org.apache.hadoop.hbase.MiniHBaseCluster;
039import org.apache.hadoop.hbase.NamespaceDescriptor;
040import org.apache.hadoop.hbase.TableName;
041import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
042import org.apache.hadoop.hbase.ipc.CallTimeoutException;
043import org.apache.hadoop.hbase.regionserver.HRegionServer;
044import org.apache.hadoop.hbase.regionserver.RSRpcServices;
045import org.apache.hadoop.hbase.testclassification.ClientTests;
046import org.apache.hadoop.hbase.testclassification.MediumTests;
047import org.apache.hadoop.hbase.util.Bytes;
048import org.junit.After;
049import org.junit.AfterClass;
050import org.junit.Before;
051import org.junit.BeforeClass;
052import org.junit.ClassRule;
053import org.junit.Rule;
054import org.junit.Test;
055import org.junit.experimental.categories.Category;
056import org.junit.rules.TestName;
057import org.junit.runner.RunWith;
058import org.junit.runners.Parameterized;
059import org.slf4j.Logger;
060import org.slf4j.LoggerFactory;
061
062import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
063import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
064
065import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
066import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
067
068@RunWith(Parameterized.class)
069@Category({ MediumTests.class, ClientTests.class })
070public class TestClientScannerTimeouts {
071
072  @ClassRule
073  public static final HBaseClassTestRule CLASS_RULE =
074    HBaseClassTestRule.forClass(TestClientScannerTimeouts.class);
075
076  private static final Logger LOG = LoggerFactory.getLogger(TestClientScannerTimeouts.class);
077  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
078
079  private AsyncConnection ASYNC_CONN;
080  private Connection CONN;
081  private static final byte[] FAMILY = Bytes.toBytes("testFamily");
082  private static final byte[] QUALIFIER = Bytes.toBytes("testQualifier");
083  private static final byte[] VALUE = Bytes.toBytes("testValue");
084
085  private static final byte[] ROW0 = Bytes.toBytes("row-0");
086  private static final byte[] ROW1 = Bytes.toBytes("row-1");
087  private static final byte[] ROW2 = Bytes.toBytes("row-2");
088  private static final byte[] ROW3 = Bytes.toBytes("row-3");
089  private static final int rpcTimeout = 1000;
090  private static final int scanTimeout = 3 * rpcTimeout;
091  private static final int metaReadRpcTimeout = 6 * rpcTimeout;
092  private static final int metaScanTimeout = 9 * rpcTimeout;
093  private static final int CLIENT_RETRIES_NUMBER = 3;
094
095  private static TableName tableName;
096
097  @Rule
098  public TestName name = new TestName();
099
100  @Parameterized.Parameter
101  public boolean useScannerTimeoutPeriodForNextCalls;
102
103  @Parameterized.Parameters
104  public static Collection<Object[]> parameters() {
105    return HBaseCommonTestingUtility.BOOLEAN_PARAMETERIZED;
106  }
107
108  @BeforeClass
109  public static void setUpBeforeClass() throws Exception {
110    Configuration conf = TEST_UTIL.getConfiguration();
111    // Don't report so often so easier to see other rpcs
112    conf.setInt("hbase.regionserver.msginterval", 3 * 10000);
113    conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, rpcTimeout);
114    conf.setStrings(HConstants.REGION_SERVER_IMPL, RegionServerWithScanTimeout.class.getName());
115    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, CLIENT_RETRIES_NUMBER);
116    conf.setInt(HConstants.HBASE_CLIENT_PAUSE, 1000);
117    TEST_UTIL.startMiniCluster(1);
118  }
119
120  @Before
121  public void setUp() throws Exception {
122    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
123    conf.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, scanTimeout);
124    conf.setInt(HBASE_CLIENT_META_READ_RPC_TIMEOUT_KEY, metaReadRpcTimeout);
125    conf.setInt(HBASE_CLIENT_META_SCANNER_TIMEOUT, metaScanTimeout);
126    conf.setBoolean(HBASE_CLIENT_USE_SCANNER_TIMEOUT_PERIOD_FOR_NEXT_CALLS,
127      useScannerTimeoutPeriodForNextCalls);
128    ASYNC_CONN = ConnectionFactory.createAsyncConnection(conf).get();
129    CONN = ConnectionFactory.createConnection(conf);
130  }
131
132  @After
133  public void after() throws Exception {
134    CONN.close();
135    ASYNC_CONN.close();
136  }
137
138  @AfterClass
139  public static void tearDownAfterClass() throws Exception {
140    TEST_UTIL.shutdownMiniCluster();
141  }
142
143  public void setup(boolean isSystemTable) throws IOException {
144    RSRpcServicesWithScanTimeout.reset();
145
146    // parameterization adds non-alphanumeric chars to the method name. strip them so
147    // it parses as a table name
148    String nameAsString = name.getMethodName().replaceAll("[^a-zA-Z0-9]", "_") + "-"
149      + useScannerTimeoutPeriodForNextCalls;
150    if (isSystemTable) {
151      nameAsString = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR + ":" + nameAsString;
152    }
153    tableName = TableName.valueOf(nameAsString);
154    TEST_UTIL.createTable(tableName, FAMILY);
155
156    Table table = CONN.getTable(tableName);
157    putToTable(table, ROW0);
158    putToTable(table, ROW1);
159    putToTable(table, ROW2);
160    putToTable(table, ROW3);
161    LOG.info("Wrote our four values");
162
163    table.getRegionLocator().getAllRegionLocations();
164
165    // reset again incase the creation/population caused anything to trigger
166    RSRpcServicesWithScanTimeout.reset();
167  }
168
169  private void expectRow(byte[] expected, Result result) {
170    assertTrue("Expected row: " + Bytes.toString(expected),
171      Bytes.equals(expected, result.getRow()));
172  }
173
174  private void expectNumTries(int expected) {
175    assertEquals(
176      "Expected tryNumber=" + expected + ", actual=" + RSRpcServicesWithScanTimeout.tryNumber,
177      expected, RSRpcServicesWithScanTimeout.tryNumber);
178    // reset for next
179    RSRpcServicesWithScanTimeout.tryNumber = 0;
180  }
181
182  /**
183   * verify that we don't miss any data when encountering an OutOfOrderScannerNextException.
184   * Typically, the only way to naturally trigger this is if a client-side timeout causes an
185   * erroneous next() call. This is relatively hard to do these days because the server attempts to
186   * always return before the timeout. In this test we force the server to throw this exception, so
187   * that we can test the retry logic appropriately.
188   */
189  @Test
190  public void testRetryOutOfOrderScannerNextException() throws IOException {
191    expectRetryOutOfOrderScannerNext(() -> getScanner(CONN));
192  }
193
194  /**
195   * AsyncTable version of above
196   */
197  @Test
198  public void testRetryOutOfOrderScannerNextExceptionAsync() throws IOException {
199    expectRetryOutOfOrderScannerNext(this::getAsyncScanner);
200  }
201
202  @Test
203  public void testNormalScanTimeoutOnNext() throws IOException {
204    setup(false);
205    testScanTimeoutOnNext(rpcTimeout, scanTimeout);
206  }
207
208  /**
209   * AsyncTable version of above
210   */
211  @Test
212  public void testNormalScanTimeoutOnNextAsync() throws IOException {
213    setup(false);
214    expectTimeoutOnNext(scanTimeout, this::getAsyncScanner);
215  }
216
217  /**
218   * verify that we honor {@link HConstants#HBASE_RPC_READ_TIMEOUT_KEY} for openScanner() calls for
219   * meta scans
220   */
221  @Test
222  public void testNormalScanTimeoutOnOpenScanner() throws IOException {
223    setup(false);
224    expectTimeoutOnOpenScanner(rpcTimeout, this::getScanner);
225  }
226
227  /**
228   * AsyncTable version of above
229   */
230  @Test
231  public void testNormalScanTimeoutOnOpenScannerAsync() throws IOException {
232    setup(false);
233    expectTimeoutOnOpenScanner(rpcTimeout, this::getAsyncScanner);
234  }
235
236  /**
237   * verify that we honor {@link ConnectionConfiguration#HBASE_CLIENT_META_SCANNER_TIMEOUT} for
238   * next() calls in meta scans
239   */
240  @Test
241  public void testMetaScanTimeoutOnNext() throws IOException {
242    setup(true);
243    testScanTimeoutOnNext(metaReadRpcTimeout, metaScanTimeout);
244  }
245
246  private void testScanTimeoutOnNext(int rpcTimeout, int scannerTimeout) throws IOException {
247    if (useScannerTimeoutPeriodForNextCalls) {
248      // Since this has HBASE_CLIENT_USE_SCANNER_TIMEOUT_FOR_NEXT_CALLS enabled, we pass
249      // scannerTimeout as the expected timeout duration.
250      expectTimeoutOnNext(scannerTimeout, this::getScanner);
251    } else {
252      // Otherwise we pass rpcTimeout as the expected timeout duration.
253      // In this case we need a special connection which disables retries, otherwise the scanner
254      // will retry the timed out next() call, which will cause out of order exception and mess up
255      // the test
256      try (Connection conn = getNoRetriesConnection()) {
257        // Now since we disabled HBASE_CLIENT_USE_SCANNER_TIMEOUT_FOR_NEXT_CALLS, verify rpcTimeout
258        expectTimeoutOnNext(rpcTimeout, () -> getScanner(conn));
259      }
260    }
261  }
262
263  private Connection getNoRetriesConnection() throws IOException {
264    Configuration confNoRetries = new Configuration(CONN.getConfiguration());
265    confNoRetries.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);
266    return ConnectionFactory.createConnection(confNoRetries);
267  }
268
269  /**
270   * AsyncTable version of above
271   */
272  @Test
273  public void testMetaScanTimeoutOnNextAsync() throws IOException {
274    setup(true);
275    expectTimeoutOnNext(metaScanTimeout, this::getAsyncScanner);
276  }
277
278  /**
279   * verify that we honor {@link ConnectionConfiguration#HBASE_CLIENT_META_READ_RPC_TIMEOUT_KEY} for
280   * openScanner() calls for meta scans
281   */
282  @Test
283  public void testMetaScanTimeoutOnOpenScanner() throws IOException {
284    setup(true);
285    expectTimeoutOnOpenScanner(metaReadRpcTimeout, this::getScanner);
286  }
287
288  /**
289   * AsyncTable version of above
290   */
291  @Test
292  public void testMetaScanTimeoutOnOpenScannerAsync() throws IOException {
293    setup(true);
294    expectTimeoutOnOpenScanner(metaReadRpcTimeout, this::getAsyncScanner);
295  }
296
297  /**
298   * Test renewLease timeout for non-async scanner, which should use rpcTimeout. Async scanner does
299   * lease renewal automatically in the background, so renewLease() always returns false. So this
300   * test doesn't have an Async counterpart like the others.
301   */
302  @Test
303  public void testNormalScanTimeoutOnRenewLease() throws IOException {
304    setup(false);
305    expectTimeoutOnRenewScanner(rpcTimeout, this::getScanner);
306  }
307
308  /**
309   * Test renewLease timeout for non-async scanner, which should use rpcTimeout. Async scanner does
310   * lease renewal automatically in the background, so renewLease() always returns false. So this
311   * test doesn't have an Async counterpart like the others.
312   */
313  @Test
314  public void testMetaScanTimeoutOnRenewLease() throws IOException {
315    setup(true);
316    expectTimeoutOnRenewScanner(metaReadRpcTimeout, this::getScanner);
317  }
318
319  /**
320   * Test close timeout for non-async scanner, which should use rpcTimeout. Async scanner does
321   * closes async and always returns immediately. So this test doesn't have an Async counterpart
322   * like the others.
323   */
324  @Test
325  public void testNormalScanTimeoutOnClose() throws IOException {
326    setup(false);
327    expectTimeoutOnCloseScanner(rpcTimeout, this::getScanner);
328  }
329
330  /**
331   * Test close timeout for non-async scanner, which should use rpcTimeout. Async scanner does
332   * closes async and always returns immediately. So this test doesn't have an Async counterpart
333   * like the others.
334   */
335  @Test
336  public void testMetaScanTimeoutOnClose() throws IOException {
337    setup(true);
338    expectTimeoutOnCloseScanner(metaReadRpcTimeout, this::getScanner);
339  }
340
341  private void expectRetryOutOfOrderScannerNext(Supplier<ResultScanner> scannerSupplier)
342    throws IOException {
343    setup(false);
344    RSRpcServicesWithScanTimeout.seqNoToThrowOn = 1;
345
346    LOG.info(
347      "Opening scanner, expecting no errors from first next() call from openScanner response");
348    ResultScanner scanner = scannerSupplier.get();
349    Result result = scanner.next();
350    expectRow(ROW0, result);
351    expectNumTries(0);
352
353    LOG.info("Making first next() RPC, expecting no errors for seqNo 0");
354    result = scanner.next();
355    expectRow(ROW1, result);
356    expectNumTries(0);
357
358    LOG.info(
359      "Making second next() RPC, expecting OutOfOrderScannerNextException and appropriate retry");
360    result = scanner.next();
361    expectRow(ROW2, result);
362    expectNumTries(1);
363
364    // reset so no errors. since last call restarted the scan and following
365    // call would otherwise fail
366    RSRpcServicesWithScanTimeout.seqNoToThrowOn = -1;
367
368    LOG.info("Finishing scan, expecting no errors");
369    result = scanner.next();
370    expectRow(ROW3, result);
371    scanner.close();
372
373    LOG.info("Testing always throw exception");
374    byte[][] expectedResults = new byte[][] { ROW0, ROW1, ROW2, ROW3 };
375    int i = 0;
376
377    // test the case that RPC always throws
378    scanner = scannerSupplier.get();
379    RSRpcServicesWithScanTimeout.throwAlways = true;
380
381    while (true) {
382      LOG.info("Calling scanner.next()");
383      result = scanner.next();
384      if (result == null) {
385        break;
386      } else {
387        byte[] expectedResult = expectedResults[i++];
388        expectRow(expectedResult, result);
389      }
390    }
391
392    // ensure we verified all rows. this along with the expectRow check above
393    // proves that we didn't miss any rows.
394    assertEquals("Expected to exhaust expectedResults array length=" + expectedResults.length
395      + ", actual index=" + i, expectedResults.length, i);
396
397    // expect all but the first row (which came from initial openScanner) to have thrown an error
398    expectNumTries(expectedResults.length - 1);
399
400  }
401
402  private void expectTimeoutOnNext(int timeout, Supplier<ResultScanner> scannerSupplier)
403    throws IOException {
404    RSRpcServicesWithScanTimeout.seqNoToSleepOn = 1;
405    RSRpcServicesWithScanTimeout.setSleepForTimeout(timeout);
406
407    LOG.info(
408      "Opening scanner, expecting no timeouts from first next() call from openScanner response");
409    ResultScanner scanner = scannerSupplier.get();
410    Result result = scanner.next();
411    expectRow(ROW0, result);
412
413    LOG.info("Making first next() RPC, expecting no timeout for seqNo 0");
414    result = scanner.next();
415    expectRow(ROW1, result);
416
417    LOG.info("Making second next() RPC, expecting timeout");
418    long start = System.nanoTime();
419    try {
420      scanner.next();
421      fail("Expected CallTimeoutException");
422    } catch (RetriesExhaustedException e) {
423      assertTrue("Expected CallTimeoutException", e.getCause() instanceof CallTimeoutException
424        || e.getCause() instanceof SocketTimeoutException);
425    }
426    expectTimeout(start, timeout);
427  }
428
429  private void expectTimeoutOnOpenScanner(int timeout, Supplier<ResultScanner> scannerSupplier)
430    throws IOException {
431    RSRpcServicesWithScanTimeout.setSleepForTimeout(timeout);
432    RSRpcServicesWithScanTimeout.sleepOnOpen = true;
433    LOG.info("Opening scanner, expecting timeout from first next() call from openScanner response");
434    long start = System.nanoTime();
435    try {
436      scannerSupplier.get().next();
437      fail("Expected SocketTimeoutException or CallTimeoutException");
438    } catch (RetriesExhaustedException e) {
439      LOG.info("Got error", e);
440      assertTrue("Expected SocketTimeoutException or CallTimeoutException, but was " + e.getCause(),
441        e.getCause() instanceof CallTimeoutException
442          || e.getCause() instanceof SocketTimeoutException);
443    }
444    expectTimeout(start, timeout);
445  }
446
447  private void expectTimeoutOnRenewScanner(int timeout, Supplier<ResultScanner> scannerSupplier)
448    throws IOException {
449    RSRpcServicesWithScanTimeout.setSleepForTimeout(timeout);
450    RSRpcServicesWithScanTimeout.sleepOnRenew = true;
451    LOG.info(
452      "Opening scanner, expecting no timeouts from first next() call from openScanner response");
453    long start = System.nanoTime();
454    ResultScanner scanner = scannerSupplier.get();
455    scanner.next();
456    assertFalse("Expected renewLease to fail due to timeout", scanner.renewLease());
457    expectTimeout(start, timeout);
458  }
459
460  private void expectTimeoutOnCloseScanner(int timeout, Supplier<ResultScanner> scannerSupplier)
461    throws IOException {
462    RSRpcServicesWithScanTimeout.setSleepForTimeout(timeout);
463    RSRpcServicesWithScanTimeout.sleepOnClose = true;
464    LOG.info(
465      "Opening scanner, expecting no timeouts from first next() call from openScanner response");
466    long start = System.nanoTime();
467    ResultScanner scanner = scannerSupplier.get();
468    scanner.next();
469    // close doesnt throw or return anything, so we can't verify it directly.
470    // but we can verify that it took as long as we expect below
471    scanner.close();
472    expectTimeout(start, timeout);
473  }
474
475  private void expectTimeout(long start, int timeout) {
476    long duration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
477    LOG.info("Expected duration >= {}, and got {}", timeout, duration);
478    assertTrue("Expected duration >= " + timeout + ", but was " + duration, duration >= timeout);
479  }
480
481  private ResultScanner getScanner() {
482    return getScanner(CONN);
483  }
484
485  private ResultScanner getScanner(Connection conn) {
486    Scan scan = new Scan();
487    scan.setCaching(1);
488    try {
489      return conn.getTable(tableName).getScanner(scan);
490    } catch (IOException e) {
491      throw new RuntimeException(e);
492    }
493  }
494
495  private ResultScanner getAsyncScanner() {
496    Scan scan = new Scan();
497    scan.setCaching(1);
498    return ASYNC_CONN.getTable(tableName).getScanner(scan);
499  }
500
501  private void putToTable(Table ht, byte[] rowkey) throws IOException {
502    Put put = new Put(rowkey);
503    put.addColumn(FAMILY, QUALIFIER, VALUE);
504    ht.put(put);
505  }
506
507  private static class RegionServerWithScanTimeout
508    extends MiniHBaseCluster.MiniHBaseClusterRegionServer {
509    public RegionServerWithScanTimeout(Configuration conf)
510      throws IOException, InterruptedException {
511      super(conf);
512    }
513
514    @Override
515    protected RSRpcServices createRpcServices() throws IOException {
516      return new RSRpcServicesWithScanTimeout(this);
517    }
518  }
519
520  private static class RSRpcServicesWithScanTimeout extends RSRpcServices {
521    private long tableScannerId;
522
523    private static long seqNoToThrowOn = -1;
524    private static boolean throwAlways = false;
525    private static boolean threw;
526
527    private static long seqNoToSleepOn = -1;
528    private static boolean sleepOnOpen = false;
529    private static boolean sleepOnRenew = false;
530    private static boolean sleepOnClose = false;
531    private static volatile boolean slept;
532    private static int tryNumber = 0;
533
534    private static int sleepTime = rpcTimeout + 500;
535
536    public static void setSleepForTimeout(int timeout) {
537      sleepTime = timeout + 500;
538    }
539
540    public static void reset() {
541      setSleepForTimeout(scanTimeout);
542
543      seqNoToSleepOn = -1;
544      seqNoToThrowOn = -1;
545      throwAlways = false;
546      threw = false;
547      sleepOnOpen = false;
548      sleepOnRenew = false;
549      sleepOnClose = false;
550      slept = false;
551      tryNumber = 0;
552    }
553
554    public RSRpcServicesWithScanTimeout(HRegionServer rs) throws IOException {
555      super(rs);
556    }
557
558    @Override
559    public ScanResponse scan(final RpcController controller, final ScanRequest request)
560      throws ServiceException {
561      if (request.hasScannerId()) {
562        LOG.info("Got request {}", request);
563        ScanResponse scanResponse = super.scan(controller, request);
564        if (tableScannerId != request.getScannerId()) {
565          return scanResponse;
566        }
567        if (request.getCloseScanner()) {
568          if (!slept && sleepOnClose) {
569            try {
570              LOG.info("SLEEPING " + sleepTime);
571              Thread.sleep(sleepTime);
572            } catch (InterruptedException e) {
573            }
574            slept = true;
575            tryNumber++;
576          }
577          return scanResponse;
578        }
579
580        if (
581          throwAlways
582            || (!threw && request.hasNextCallSeq() && seqNoToThrowOn == request.getNextCallSeq())
583        ) {
584          threw = true;
585          tryNumber++;
586          LOG.info("THROWING exception, tryNumber={}, tableScannerId={}", tryNumber,
587            tableScannerId);
588          throw new ServiceException(new OutOfOrderScannerNextException());
589        }
590
591        if (
592          !slept && (request.hasNextCallSeq() && seqNoToSleepOn == request.getNextCallSeq()
593            || sleepOnRenew && request.getRenew())
594        ) {
595          try {
596            LOG.info("SLEEPING " + sleepTime);
597            Thread.sleep(sleepTime);
598          } catch (InterruptedException e) {
599          }
600          slept = true;
601          tryNumber++;
602        }
603        return scanResponse;
604      } else {
605        ScanResponse scanRes = super.scan(controller, request);
606        String regionName = Bytes.toString(request.getRegion().getValue().toByteArray());
607        if (!regionName.contains(TableName.META_TABLE_NAME.getNameAsString())) {
608          tableScannerId = scanRes.getScannerId();
609          if (sleepOnOpen) {
610            try {
611              LOG.info("openScanner SLEEPING " + sleepTime);
612              Thread.sleep(sleepTime);
613            } catch (InterruptedException e) {
614            }
615          }
616        }
617        return scanRes;
618      }
619    }
620  }
621}