ndimiduk commented on a change in pull request #4146:
URL: https://github.com/apache/hbase/pull/4146#discussion_r819696759



##########
File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestAsyncTableScan.java
##########
@@ -229,12 +237,15 @@ private void testReversedScan(int start, boolean 
startInclusive, int stop, boole
     if (limit > 0) {
       scan.setLimit(limit);
     }
-    List<Result> results = doScan(scan);
+    List<Result> results = doScan(scan, -1);
     int actualStart = startInclusive ? start : start - 1;
     int actualStop = stopInclusive ? stop - 1 : stop;
     int count = actualStart - actualStop;
     if (limit > 0) {
       count = Math.min(count, limit);
+    }
+    if (scan.getBatch() > 0) {

Review comment:
       ??

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java
##########
@@ -188,6 +188,7 @@ private void startScan(OpenScannerResponse resp) {
   private CompletableFuture<OpenScannerResponse> openScanner(int replicaId) {
     return conn.callerFactory.<OpenScannerResponse> single().table(tableName)
       
.row(scan.getStartRow()).replicaId(replicaId).locateType(getLocateType(scan))
+      .priority(scan.getPriority())

Review comment:
       This absence was a bug, no?

##########
File path: 
hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java
##########
@@ -478,53 +472,123 @@ public void testCheckAndMutateMetaTable() throws 
IOException {
       any(ClientProtos.MultiRequest.class), any());
   }
 
+  private void mockScan(int scanPriority) {
+    int scannerId = 1;
+    AtomicInteger scanNextCalled = new AtomicInteger(0);
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        threadPool.submit(() ->{
+          ScanRequest req = invocation.getArgument(1);
+          RpcCallback<ScanResponse> done = invocation.getArgument(2);
+          if (!req.hasScannerId()) {
+            done.run(
+              
ScanResponse.newBuilder().setScannerId(scannerId).setTtl(800).setMoreResultsInRegion(true).setMoreResults(true).build());
+          } else {
+            assertFalse("close scanner should not come in with scan priority " 
+ scanPriority,
+              req.hasCloseScanner() && req.getCloseScanner());
+
+              Cell cell = 
CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setType(Type.Put).setRow(Bytes.toBytes(scanNextCalled.incrementAndGet()))
+                
.setFamily(Bytes.toBytes("cf")).setQualifier(Bytes.toBytes("cq")).setValue(Bytes.toBytes("v")).build();
+              Result result = Result.create(Arrays.asList(cell));
+              done.run(
+                
ScanResponse.newBuilder().setScannerId(scannerId).setTtl(800).setMoreResultsInRegion(true).setMoreResults(true).addResults(ProtobufUtil.toResult(result)).build());
+          }
+        });
+        return null;
+      }
+    }).when(stub).scan(assertPriority(scanPriority), any(ScanRequest.class), 
any());
+
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        threadPool.submit(() ->{
+          ScanRequest req = invocation.getArgument(1);
+          RpcCallback<ScanResponse> done = invocation.getArgument(2);
+          assertTrue("close request should have scannerId", 
req.hasScannerId());
+          assertEquals("close request's scannerId should match", scannerId, 
req.getScannerId());
+          assertTrue("close request should have closerScanner set", 
req.hasCloseScanner() && req.getCloseScanner());
+
+          done.run(ScanResponse.getDefaultInstance());
+        });
+        return null;
+      }
+    }).when(stub).scan(assertPriority(HIGH_QOS), assertScannerCloseRequest(), 
any());
+  }
+
   @Test
   public void testScan() throws IOException, InterruptedException {
+    mockScan(19);
     try (ResultScanner scanner = 
conn.getTable(TableName.valueOf(name.getMethodName()))
       .getScanner(new 
Scan().setCaching(1).setMaxResultSize(1).setPriority(19))) {
       assertNotNull(scanner.next());
       Thread.sleep(1000);
     }
-    Thread.sleep(1000);
-    // open, next, several renew lease, and then close
-    verify(stub, atLeast(4)).scan(assertPriority(19), any(ScanRequest.class), 
any());
+    // ensures the close thread has time to finish before asserting

Review comment:
       Instead of relying on time delays, which will result in a flakey test, 
can you instead use something like a countdown latch to track completion of the 
callback actions that the test code must wait on? For that matter, maybe you 
can improve the test by getting rid of the sleep up in the try block as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@hbase.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to