ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r377213128
 
 

 ##########
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##########
 @@ -3889,4 +3891,69 @@ private void getProcedureResult(long procId, 
CompletableFuture<Void> future, int
         .call();
   }
 
+  @Override
+  public CompletableFuture<List<SlowLogPayload>> getSlowLogResponses(
+      List<ServerName> serverNames) {
+    CompletableFuture<List<SlowLogPayload>> slowLoadPayloads =
+      CompletableFuture.completedFuture(Collections.emptyList());
+    if (CollectionUtils.isEmpty(serverNames)) {
+      return slowLoadPayloads;
+    }
+    for (ServerName serverName : serverNames) {
+      CompletableFuture<List<SlowLogPayload>> slowLogPayloadFromServer =
+        getSlowLogResponseFromServer(serverName);
+      slowLoadPayloads = 
slowLoadPayloads.thenCombine(slowLogPayloadFromServer, (l1, l2) -> {
+        List<SlowLogPayload> finalSlowLogPayloads = new LinkedList<>();
 
 Review comment:
   nit: For larger clusters, you can save yourself a lot of extra allocations 
by using a stream-based approach, something like
   
   ```java
       if (CollectionUtils.isEmpty(serverNames)) {
         return CompletableFuture.completedFuture(Collections.emptyList());
       }
   
       return CompletableFuture.supplyAsync(() -> {
        return serverNames.stream()
           .map(this::getSlowLogResponseFromServer)
           .map(CompletableFuture::join)
           .flatMap(List::stream)
           .collect(Collectors.toList());
       });
   ```
   
   Actually, what I think you really want is a solution that won't fall over in 
the face of a very large cluster with lots of slow results. Such a solution 
would involve a user-provided limit on the number of results returned, 
partitioning the server list in batches of size N, processing a batch, and 
short-circuiting the return when a result-count limit is reached. So, to 
support this use case, you'll need to add a limit parameter to the methods (and 
maybe a configuration point providing a default limit) and then implement 
pagination (which I think can also be implemented as a stream over partitions 
plus `Stream.limit`).

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to