walterddr commented on code in PR #9344:
URL: https://github.com/apache/pinot/pull/9344#discussion_r966214894


##########
pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java:
##########
@@ -148,6 +149,36 @@ public void testQueryProjectFilterPushDownForJoin() {
     }
   }
 
+  // Test that plan query can be run as multi-thread.
+  @Test
+  public void testPlanQueryMultiThread()
+      throws Exception {
+    Runnable planQuery = () -> {
+      String query = "SELECT a.col1, a.ts, b.col2, b.col3 FROM a JOIN b ON 
a.col1 = b.col2 "
+          + "WHERE a.col3 >= 0 AND a.col2 IN  ('a', 'b') AND b.col3 < 0";
+      QueryPlan queryPlan = _queryEnvironment.planQuery(query);
+      List<StageNode> intermediateStageRoots =
+          queryPlan.getStageMetadataMap().entrySet().stream().filter(e -> 
e.getValue().getScannedTables().size() == 0)
+              .map(e -> 
queryPlan.getQueryStageMap().get(e.getKey())).collect(Collectors.toList());
+      // Assert that no project of filter node for any intermediate stage 
because all should've been pushed down.
+      for (StageNode roots : intermediateStageRoots) {
+        assertNodeTypeNotIn(roots, ImmutableList.of(ProjectNode.class, 
FilterNode.class));
+      }
+    };

Review Comment:
   no need to assert this. we should test something like this.
   - plan 2 type of SQL (say 
     - type a: SELECT * FROM tbl and 
     - type b: SELECT xxx JOIN)
   - run 4 planning (a1, b1, a2, b2) at the same time
   - assert that a1 plan == a2 plan, same for b type



##########
pinot-query-planner/src/main/java/org/apache/pinot/query/context/PlannerContext.java:
##########
@@ -19,15 +19,50 @@
 package org.apache.pinot.query.context;
 
 import java.util.Map;
-
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.hep.HepProgram;
+import org.apache.calcite.prepare.PlannerImpl;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.pinot.query.planner.logical.LogicalPlanner;
+import org.apache.pinot.query.validate.Validator;
 
 /**
  * PlannerContext is an object that holds all contextual information during 
planning phase.
  *
- * TODO: currently the planner context is not used since we don't support 
option or query rewrite. This construct is
- * here as a placeholder for the parsed out options.
+ * TODO: currently we don't support option or query rewrite.
+ * It is used to hold per query context for query planning, which cannot be 
shared across queries.
  */
 public class PlannerContext {

Review Comment:
   let's make this closeable and make sure the planner is closed 
   ```suggestion
   public class PlannerContext implement Closeable {
   ```



##########
pinot-query-planner/src/main/java/org/apache/pinot/query/context/PlannerContext.java:
##########
@@ -19,15 +19,50 @@
 package org.apache.pinot.query.context;
 
 import java.util.Map;
-
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.hep.HepProgram;
+import org.apache.calcite.prepare.PlannerImpl;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.pinot.query.planner.logical.LogicalPlanner;
+import org.apache.pinot.query.validate.Validator;
 
 /**
  * PlannerContext is an object that holds all contextual information during 
planning phase.
  *
- * TODO: currently the planner context is not used since we don't support 
option or query rewrite. This construct is
- * here as a placeholder for the parsed out options.
+ * TODO: currently we don't support option or query rewrite.
+ * It is used to hold per query context for query planning, which cannot be 
shared across queries.
  */
 public class PlannerContext {
+  public PlannerContext(FrameworkConfig config, Prepare.CatalogReader 
catalogReader, RelDataTypeFactory typeFactory,

Review Comment:
   great idea encoding these non-threadsafe objects into plannerContext!



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