You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2023/01/03 21:55:58 UTC

[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #10006: [refactor] combine operator API changes

Jackie-Jiang commented on code in PR #10006:
URL: https://github.com/apache/pinot/pull/10006#discussion_r1060983403


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java:
##########
@@ -34,25 +41,53 @@
 public class StreamingInstanceResponseOperator extends InstanceResponseOperator {
 
   private final StreamObserver<Server.ServerResponse> _streamObserver;
+  private final int _limit;
+  private int _numRowsCollected;
 
   public StreamingInstanceResponseOperator(BaseCombineOperator<?> combinedOperator, List<IndexSegment> indexSegments,
       List<FetchContext> fetchContexts, StreamObserver<Server.ServerResponse> streamObserver,
       QueryContext queryContext) {
     super(combinedOperator, indexSegments, fetchContexts, queryContext);
     _streamObserver = streamObserver;
+    _limit = queryContext.getLimit();
+    _numRowsCollected = 0;
   }
 
   @Override
   protected InstanceResponseBlock getNextBlock() {
-    InstanceResponseBlock responseBlock = super.getNextBlock();
-    InstanceResponseBlock metadataOnlyResponseBlock = responseBlock.toMetadataOnlyResponseBlock();
+    prefetchAll();
+    InstanceResponseBlock exceptionResultBlock = null;
+    BaseResultsBlock combinedResult;
     try {
-      _streamObserver.onNext(StreamingResponseUtils.getDataResponse(responseBlock.toDataOnlyDataTable()));
+      combinedResult = _combineOperator.nextBlock();
+      while (!(combinedResult instanceof MetadataResultsBlock)) {
+        if (combinedResult instanceof ExceptionResultsBlock) {
+          exceptionResultBlock = new InstanceResponseBlock(combinedResult, _queryContext);
+          return exceptionResultBlock;
+        } else {
+          sendBlock(combinedResult);
+        }
+        combinedResult = _combineOperator.nextBlock();
+      }
     } catch (IOException e) {
-      metadataOnlyResponseBlock.addException(
+      exceptionResultBlock = new InstanceResponseBlock();
+      exceptionResultBlock.addException(
           QueryException.getException(QueryException.DATA_TABLE_SERIALIZATION_ERROR, e));
+      return exceptionResultBlock;
+    } finally {
+      releaseAll();

Review Comment:
   We can clean up the combine operator sub-tasks here. No need to add clean up everywhere in the combine operator



##########
pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java:
##########
@@ -34,25 +41,53 @@
 public class StreamingInstanceResponseOperator extends InstanceResponseOperator {
 
   private final StreamObserver<Server.ServerResponse> _streamObserver;
+  private final int _limit;
+  private int _numRowsCollected;
 
   public StreamingInstanceResponseOperator(BaseCombineOperator<?> combinedOperator, List<IndexSegment> indexSegments,
       List<FetchContext> fetchContexts, StreamObserver<Server.ServerResponse> streamObserver,
       QueryContext queryContext) {
     super(combinedOperator, indexSegments, fetchContexts, queryContext);
     _streamObserver = streamObserver;
+    _limit = queryContext.getLimit();
+    _numRowsCollected = 0;
   }
 
   @Override
   protected InstanceResponseBlock getNextBlock() {
-    InstanceResponseBlock responseBlock = super.getNextBlock();
-    InstanceResponseBlock metadataOnlyResponseBlock = responseBlock.toMetadataOnlyResponseBlock();
+    prefetchAll();
+    InstanceResponseBlock exceptionResultBlock = null;
+    BaseResultsBlock combinedResult;
     try {
-      _streamObserver.onNext(StreamingResponseUtils.getDataResponse(responseBlock.toDataOnlyDataTable()));
+      combinedResult = _combineOperator.nextBlock();
+      while (!(combinedResult instanceof MetadataResultsBlock)) {

Review Comment:
   (minor)
   ```suggestion
         while (!((combinedResult = _combineOperator.nextBlock()) instanceof MetadataResultsBlock)) {
   ```



##########
pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java:
##########
@@ -34,25 +41,53 @@
 public class StreamingInstanceResponseOperator extends InstanceResponseOperator {
 
   private final StreamObserver<Server.ServerResponse> _streamObserver;
+  private final int _limit;
+  private int _numRowsCollected;

Review Comment:
   Seems unused



-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org