Jackie-Jiang commented on a change in pull request #7839:
URL: https://github.com/apache/pinot/pull/7839#discussion_r763528060



##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/request/context/utils/QueryContextUtils.java
##########
@@ -41,6 +41,15 @@ public static boolean isSelectionQuery(QueryContext query) {
     return query.getAggregationFunctions() == null;
   }
 
+  /**
+   * Returns {@code true} if the given query is a simple selection query, 
{@code false} otherwise.
+   *
+   * Simple selection query at this moment means selection query without 
order-by.
+   */
+  public static boolean isSimpleSelectionQuery(QueryContext query) {

Review comment:
       (minor) we usually call it selection-only query
   ```suggestion
     public static boolean isSelectionOnlyQuery(QueryContext query) {
   ```

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingInstanceResponsePlanNode.java
##########
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.plan;
+
+import io.grpc.stub.StreamObserver;
+import java.util.List;
+import org.apache.pinot.common.proto.Server;
+import org.apache.pinot.core.operator.InstanceResponseOperator;
+import org.apache.pinot.core.operator.StreamingInstanceResponseOperator;
+import org.apache.pinot.segment.spi.FetchContext;
+import org.apache.pinot.segment.spi.IndexSegment;
+
+
+public class StreamingInstanceResponsePlanNode extends 
InstanceResponsePlanNode {
+  private final CombinePlanNode _combinePlanNode;
+  private final List<IndexSegment> _indexSegments;
+  private final List<FetchContext> _fetchContexts;

Review comment:
       Change these 3 variables in `InstanceResponsePlanNode` to be protected. 
No need to declare them again

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/common/datatable/DataTableUtils.java
##########
@@ -107,6 +108,34 @@ public static DataTable buildEmptyDataTable(QueryContext 
queryContext)
     }
   }
 
+  public static DataTable constructMetadataOnlyDataTable(DataTable 
origDataTable) {

Review comment:
       (minor) for consistency
   ```suggestion
     public static DataTable buildMetadataOnlyDataTable(DataTable 
origDataTable) {
   ```
   
   We should also have one to `buildDataOnlyDataTable`

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingResponseUtils.java
##########
@@ -34,6 +35,12 @@ private StreamingResponseUtils() {
     return getResponse(dataTable, Response.ResponseType.DATA);
   }
 
+  public static Server.ServerResponse getDataResponse(InstanceResponseBlock 
resultsBlock)

Review comment:
       (major) We should remove the metadata from the data response
   (minor) Suggest not introducing `InstanceResponseBlock` into this class, 
convert the `DataTable` on the caller side instead

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/StreamingInstanceResponseOperator.java
##########
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.operator;
+
+import io.grpc.stub.StreamObserver;
+import java.io.IOException;
+import java.util.List;
+import org.apache.pinot.common.proto.Server;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.datatable.DataTableUtils;
+import org.apache.pinot.core.operator.blocks.InstanceResponseBlock;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.core.operator.combine.BaseCombineOperator;
+import org.apache.pinot.core.operator.streaming.StreamingResponseUtils;
+import org.apache.pinot.segment.spi.FetchContext;
+import org.apache.pinot.segment.spi.IndexSegment;
+
+
+public class StreamingInstanceResponseOperator extends 
InstanceResponseOperator {
+  private static final String OPERATOR_NAME = 
"StreamingInstanceResponseOperator";
+  private final StreamObserver<Server.ServerResponse> _streamObserver;
+
+  public StreamingInstanceResponseOperator(BaseCombineOperator 
combinedOperator, List<IndexSegment> indexSegments,
+      List<FetchContext> fetchContexts, StreamObserver<Server.ServerResponse> 
streamObserver) {
+    super(combinedOperator, indexSegments, fetchContexts);
+    _streamObserver = streamObserver;
+  }
+
+  @Override
+  protected InstanceResponseBlock getNextBlock() {
+    InstanceResponseBlock nextBlock = super.getNextBlock();
+    try {
+      
_streamObserver.onNext(StreamingResponseUtils.getDataResponse(nextBlock));
+      DataTable instanceResponseDataTable = 
nextBlock.getInstanceResponseDataTable();
+      // return a metadata-only block.
+      DataTable dataTable = 
DataTableUtils.constructMetadataOnlyDataTable(instanceResponseDataTable);
+      return new InstanceResponseBlock(dataTable);
+    } catch (IOException e) {
+      return new InstanceResponseBlock(new IntermediateResultsBlock());

Review comment:
       Add the exception to the result block




-- 
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-unsubscr...@pinot.apache.org

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



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

Reply via email to