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



##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/AcquireReleaseColumnsSegmentOperator.java
##########
@@ -0,0 +1,70 @@
+/**
+ * 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 java.util.Set;
+import org.apache.pinot.core.common.Block;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.segment.spi.IndexSegment;
+
+
+/**
+ * A common wrapper around the segment-level operator.
+ * Provides an opportunity to acquire and release column buffers before 
reading data
+ */
+public class AcquireReleaseColumnsSegmentOperator extends BaseOperator {
+  private static final String OPERATOR_NAME = "SegmentOperator";
+
+  private final Operator _childOperator;
+  private final IndexSegment _indexSegment;
+  private final Set<String> _columns;
+
+  public AcquireReleaseColumnsSegmentOperator(Operator childOperator, 
IndexSegment indexSegment, Set<String> columns) {
+    _childOperator = childOperator;
+    _indexSegment = indexSegment;
+    _columns = columns;
+  }
+
+  /**
+   * Makes a call to acquire column buffers from {@link IndexSegment} before 
getting nextBlock from childOperator,
+   * and
+   * a call to release the column buffers from {@link IndexSegment} after.
+   */
+  @Override
+  protected Block getNextBlock() {
+    _indexSegment.acquire(_columns);
+    Block nextBlock;
+    try {
+      nextBlock = _childOperator.nextBlock();
+    } finally {
+      _indexSegment.release(_columns);
+    }
+    return nextBlock;

Review comment:
       (nit) Can be simplified
   ```suggestion
       try {
         return _childOperator.nextBlock();
       } finally {
         _indexSegment.release(_columns);
       }
   ```

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java
##########
@@ -80,6 +83,8 @@
   // set as pinot.server.query.executor.groupby.trim.threshold
   public static final String GROUPBY_TRIM_THRESHOLD_KEY = 
"groupby.trim.threshold";
   public static final int DEFAULT_GROUPBY_TRIM_THRESHOLD = 1_000_000;
+  public static final String ENABLE_ACQUIRE_AND_RELEASE = 
"enable.buffer.acquire.release";
+  public static final boolean DEFAULT_ENABLE_ACQUIRE_AND_RELEASE = false;

Review comment:
       (nit)
   ```suggestion
     public static final String ENABLE_BUFFER_ACQUIRE_RELEASE = 
"enable.buffer.acquire.release";
     public static final boolean DEFAULT_ENABLE_BUFFER_ACQUIRE_RELEASE = false;
   ```

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java
##########
@@ -132,17 +140,40 @@ public InstancePlanMakerImplV2(QueryExecutorConfig 
queryExecutorConfig) {
     Preconditions
         .checkState(_groupByTrimThreshold > 0, "Invalid configurable: 
groupByTrimThreshold: %d must be positive",
             _groupByTrimThreshold);
-    LOGGER.info(
-        "Initializing plan maker with maxInitialResultHolderCapacity: {}, 
numGroupsLimit: {}, minSegmentGroupTrimSize: {}, minServerGroupTrimSize: {}",
-        _maxInitialResultHolderCapacity, _numGroupsLimit, 
_minSegmentGroupTrimSize, _minServerGroupTrimSize);
+    _enableBufferAcquireRelease = 
Boolean.parseBoolean(config.getProperty(ENABLE_ACQUIRE_AND_RELEASE));
+    LOGGER.info("Initializing plan maker with maxInitialResultHolderCapacity: 
{}, numGroupsLimit: {}, "
+            + "minSegmentGroupTrimSize: {}, minServerGroupTrimSize: {}, 
enableBufferAcquireRelease: {}",
+        _maxInitialResultHolderCapacity, _numGroupsLimit, 
_minSegmentGroupTrimSize, _minServerGroupTrimSize,
+        _enableBufferAcquireRelease);
   }
 
   @Override
   public Plan makeInstancePlan(List<IndexSegment> indexSegments, QueryContext 
queryContext,
       ExecutorService executorService, long endTimeMs) {
+    boolean prefetch = false;
+    if (_enableBufferAcquireRelease) {
+      prefetch = queryContext.getQueryOptions() != null && 
QueryOptions.isPrefetchBuffers(queryContext.getQueryOptions());

Review comment:
       Let's move the new logic into this `if` block, and keep the original 
logic in the `else` block to avoid always adding an extra operator
   ```suggestion
       List<PlanNode> planNodes = new ArrayList<>(indexSegments.size());
       if (_enableBufferAcquireRelease) {
         boolean prefetch = ...
       } else {
         for (IndexSegment indexSegment : indexSegments) {
           planNodes.add(makeSegmentPlanNode(indexSegment, queryContext));
         }
       }
   ```

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/AcquireReleaseColumnsSegmentOperator.java
##########
@@ -0,0 +1,70 @@
+/**
+ * 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 java.util.Set;
+import org.apache.pinot.core.common.Block;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.segment.spi.IndexSegment;
+
+
+/**
+ * A common wrapper around the segment-level operator.
+ * Provides an opportunity to acquire and release column buffers before 
reading data
+ */
+public class AcquireReleaseColumnsSegmentOperator extends BaseOperator {
+  private static final String OPERATOR_NAME = "SegmentOperator";

Review comment:
       ```suggestion
     private static final String OPERATOR_NAME = 
"AcquireReleaseColumnsSegmentOperator";
   ```




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to