You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "xiangfu0 (via GitHub)" <gi...@apache.org> on 2023/05/08 22:20:49 UTC

[GitHub] [pinot] xiangfu0 commented on a diff in pull request #10711: [multistage][cleanup] make leaf stage also registered with SchedulerService

xiangfu0 commented on code in PR #10711:
URL: https://github.com/apache/pinot/pull/10711#discussion_r1184382252


##########
pinot-query-runtime/src/test/resources/queries/MetadataTestQuery.json:
##########
@@ -0,0 +1,66 @@
+{
+  "metadata_test": {
+    "comments": "metadata_test is used to test Tracing and Stats metadata returns",
+    "limitation": "due to current limitation, metadata test requires all data to be located on a single partition, see: https://github.com/apache/pinot/issues/10399",
+    "tables": {
+      "tbl1" : {
+        "schema": [
+          {"name": "col1", "type": "STRING"},
+          {"name": "col2", "type": "INT"}
+        ],
+        "inputs": [
+          ["foo", 1],
+          ["bar", 2]
+        ],
+        "partitionColumns": [ ]
+      },
+      "tbl2" : {
+        "schema": [
+          {"name": "col1", "type": "STRING"},
+          {"name": "col2", "type": "INT"},
+          {"name": "col3", "type": "DOUBLE"},
+          {"name": "partitionCol1", "type": "INT"},
+          {"name": "partitionCol2", "type": "STRING"}
+        ],
+        "inputs": [
+          ["foo", 1, 3.1416, 1, "charlie"],
+          ["foo", 3, 3.1416, 1, "charlie"],
+          ["bar", 2, 2.7183, 1, "charlie"],
+          ["------"],
+          ["bar", 4, 2.7183, 1, "charlie"]
+        ],
+        "partitionColumns": [
+          "partitionCol1", "partitionCol2"
+        ]
+      }
+    },
+    "queries": [
+      {
+        "sql": "SELECT * FROM {tbl1}",
+        "comment": "only 1 segment for tbl1",
+        "expectedNumSegments": 1
+      },
+      {
+        "sql": "SELECT col1, COUNT(*) FROM {tbl2} GROUP BY col1",
+        "comment": "2 segments for tbl2",
+        "expectedNumSegments": 2
+      },
+      {
+        "sql": "SELECT {tbl1}.col1, {tbl1}.col2, {tbl2}.col3 FROM {tbl1} JOIN {tbl2} ON {tbl1}.col1 = {tbl2}.col1",
+        "expectedNumSegments": 3
+      },
+      {
+        "sql": "SELECT {tbl1}.col1, {tbl1}.col2, COUNT(*) FROM {tbl1} JOIN {tbl2} ON {tbl1}.col1 = {tbl2}.col1 GROUP BY {tbl1}.col1, {tbl1}.col2",
+        "expectedNumSegments": 3
+      },
+      {
+        "sql": "SELECT {tbl1}.col1 FROM {tbl1} WHERE {tbl1}.col2 > (SELECT 0.5 * SUM({tbl2}.col3) FROM {tbl2} WHERE {tbl1}.col2 = {tbl1}.col2 AND {tbl1}.col1 = {tbl2}.col1)",
+        "comment": "This correlated subquery test is decorrelated to 2 JOINs, so one table is scanned twice, hence expected to scan 8 segments",
+        "expectedNumSegments": 4
+      }
+    ],
+    "extraProps": {
+      "noEmptySegment": "true"
+    }
+  }
+}

Review Comment:
   Nit: new line



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java:
##########
@@ -147,18 +149,18 @@ public void init(PinotConfiguration config, InstanceDataManager instanceDataMana
   }
 
   public void start()
-      throws TimeoutException {
+      throws Exception {
     _helixPropertyStore = _helixManager.getHelixPropertyStore();
     _mailboxService.start();
     _serverExecutor.start();
-    _scheduler.startAsync().awaitRunning(30, TimeUnit.SECONDS);
+    _yieldingScheduler.start();
   }
 
   public void shutDown()
-      throws TimeoutException {
+      throws Exception {
     _serverExecutor.shutDown();
     _mailboxService.shutdown();
-    _scheduler.stopAsync().awaitTerminated(30, TimeUnit.SECONDS);
+    _yieldingScheduler.stop();

Review Comment:
   stop this _yieldingScheduler first?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/ThreadPoolSchedulerService.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.query.runtime.executor;
+
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.operator.OpChain;
+import org.apache.pinot.query.runtime.operator.OpChainId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ThreadPoolSchedulerService implements SchedulerService {
+  private static final Logger LOGGER = LoggerFactory.getLogger(ThreadPoolSchedulerService.class);
+
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
+
+  public ThreadPoolSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
+  }

Review Comment:
   nit: new line



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/ThreadPoolSchedulerService.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.query.runtime.executor;
+
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.operator.OpChain;
+import org.apache.pinot.query.runtime.operator.OpChainId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ThreadPoolSchedulerService implements SchedulerService {
+  private static final Logger LOGGER = LoggerFactory.getLogger(ThreadPoolSchedulerService.class);
+
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
+
+  public ThreadPoolSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
+  }
+  @Override
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
+      @Override
+      public void runJob() {
+        boolean isFinished = false;
+        boolean returnedErrorBlock = false;
+        Throwable thrown = null;
+        try {
+          LOGGER.trace("({}): Executing", operatorChain);
+          operatorChain.getStats().executing();
+          TransferableBlock result = operatorChain.getRoot().nextBlock();
+          while (!result.isEndOfStreamBlock()) {
+            result = operatorChain.getRoot().nextBlock();
+          }
+          isFinished = true;
+          if (result.isErrorBlock()) {
+            returnedErrorBlock = true;
+            LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                result.getDataBlock().getExceptions());
+          } else {
+            LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
+          }
+        } catch (Exception e) {
+          LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
+          thrown = e;
+        } finally {
+          if (returnedErrorBlock || thrown != null) {
+            cancelOpChain(operatorChain, thrown);
+          } else if (isFinished) {
+            closeOpChain(operatorChain);
+          }
+        }
+      }
+    });
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
+  }
+
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. if there's dangling opChain we will use a cache to clean it up.
+    // TODO: make sure this is thread-safe.

Review Comment:
   Is it possible that a request is canceled first then OpChain got registered?



-- 
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