You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "walterddr (via GitHub)" <gi...@apache.org> on 2023/02/21 17:03:16 UTC

[GitHub] [pinot] walterddr opened a new pull request, #10315: [multistage] properly support query hint

walterddr opened a new pull request, #10315:
URL: https://github.com/apache/pinot/pull/10315

   previously we support hinting by directly creating relHint with builder. this is sort of a short-term solution.
   This PR creates HintStrategyTable based on Calcite's recommended way of hinting planner. 
   
   See: https://calcite.apache.org/javadocAggregate/org/apache/calcite/rel/hint/package-summary.html


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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113940359


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java:
##########
@@ -83,8 +83,8 @@ public boolean matches(RelOptRuleCall call) {
     }
     if (call.rel(0) instanceof Aggregate) {
       Aggregate agg = call.rel(0);
-      return !agg.getHints().contains(PinotRelationalHints.AGG_LEAF_STAGE)
-          && !agg.getHints().contains(PinotRelationalHints.AGG_INTERMEDIATE_STAGE);
+      return !agg.getHints().contains(AggregateNode.INTERMEDIATE_STAGE_HINT)

Review Comment:
   > these 2 hints are not suppose to be directly used by user. 
   
   You both may have already discussed this but not sure I follow this. At least in the other PR, we want to create a hint and expose it to user. 
   
   Fair to say that if I want to add a new hint and expose it to the user, this will have to be evolved / extended or there is a different plan ?
   
   ```
   public static final HintStrategyTable PINOT_HINT_STRATEGY_TABLE = HintStrategyTable.builder()
         .hintStrategy(INTERNAL_AGG_INTERMEDIATE_STAGE, HintPredicates.AGGREGATE)
         .hintStrategy(INTERNAL_AGG_FINAL_STAGE, HintPredicates.AGGREGATE)
   ```



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


[GitHub] [pinot] vvivekiyer commented on pull request #10315: [multistage] properly support query hint

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on PR #10315:
URL: https://github.com/apache/pinot/pull/10315#issuecomment-1438830176

   @walterddr I have not looked at the code changes here (will look in some time). But I do have a PR out that adds supports for SQL Hints in Pinot  - https://github.com/apache/pinot/pull/10248. Can you please take a look? We can discuss about how to converge the changes. 


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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113410216


##########
pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java:
##########
@@ -109,10 +109,17 @@ private static String removeTerminatingSemicolon(String sql) {
 
   public static SqlNodeAndOptions compileToSqlNodeAndOptions(String sql)
       throws SqlCompilationException {
+    return compileToSqlNodeAndOptions(sql, true);
+  }
+
+  public static SqlNodeAndOptions compileToSqlNodeAndOptions(String sql, boolean isRemoveComments)
+      throws SqlCompilationException {
     long parseStartTimeNs = System.nanoTime();
 
     // Remove the comments from the query

Review Comment:
   The code for removing comments was added in https://github.com/apache/pinot/issues/7714 to avoid misusing the older support for `OPTIONS`. As per my understanding, this logic is not needed anymore because we have moved to SET based options. 
   
   So we can remove this logic entirely? 



##########
pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java:
##########
@@ -172,12 +173,12 @@ public String explainQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions)
 
   @VisibleForTesting
   public QueryPlan planQuery(String sqlQuery) {

Review Comment:
   This function is only called from tests, right? 
   
   The actual code path is `MultiStageBrokerRequestHandler.handleRequest()` -> `QueryEnvironment.planQuery(sqlQuery, sqlNodeAndOptions)`. 



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


[GitHub] [pinot] walterddr commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113548317


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java:
##########
@@ -16,19 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.query.planner.hints;
-
-import org.apache.calcite.rel.hint.RelHint;
-
+package org.apache.calcite.rel.hint;
 
 /**
- * Provide certain relational hint to query planner for better optimization.
+ * Default hint strategy set for Pinot query.
  */
-public class PinotRelationalHints {
-  public static final RelHint AGG_INTERMEDIATE_STAGE = RelHint.builder("AGG_INTERMEDIATE_STAGE").build();
-  public static final RelHint AGG_LEAF_STAGE = RelHint.builder("AGG_LEAF_STAGE").build();
+public class PinotHintStrategyTable {
+  public static final String INTERNAL_AGG_INTERMEDIATE_STAGE = "aggIntermediateStage";

Review Comment:
   yes. again whether a hint is valid or not is up to the implementation. but that's actually a good point. i can make this as `aggregateHint(final)` and `aggregateHint(intermediate)` this way we only have 1 hint key but different hint options. 
   
   i think there's a restriction on whether a hint can accept multiple option key in order to make this more fool proof



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


[GitHub] [pinot] walterddr commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113794124


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java:
##########
@@ -83,8 +83,8 @@ public boolean matches(RelOptRuleCall call) {
     }
     if (call.rel(0) instanceof Aggregate) {
       Aggregate agg = call.rel(0);
-      return !agg.getHints().contains(PinotRelationalHints.AGG_LEAF_STAGE)
-          && !agg.getHints().contains(PinotRelationalHints.AGG_INTERMEDIATE_STAGE);
+      return !agg.getHints().contains(AggregateNode.INTERMEDIATE_STAGE_HINT)

Review Comment:
   for now we will add a safety check in AggregateNode to not allow both hints at the same time. 



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


[GitHub] [pinot] siddharthteotia merged pull request #10315: [multistage] properly support query hint

Posted by "siddharthteotia (via GitHub)" <gi...@apache.org>.
siddharthteotia merged PR #10315:
URL: https://github.com/apache/pinot/pull/10315


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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113436552


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java:
##########
@@ -16,19 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.query.planner.hints;
-
-import org.apache.calcite.rel.hint.RelHint;
-
+package org.apache.calcite.rel.hint;
 
 /**
- * Provide certain relational hint to query planner for better optimization.
+ * Default hint strategy set for Pinot query.
  */
-public class PinotRelationalHints {
-  public static final RelHint AGG_INTERMEDIATE_STAGE = RelHint.builder("AGG_INTERMEDIATE_STAGE").build();
-  public static final RelHint AGG_LEAF_STAGE = RelHint.builder("AGG_LEAF_STAGE").build();
+public class PinotHintStrategyTable {
+  public static final String INTERNAL_AGG_INTERMEDIATE_STAGE = "aggIntermediateStage";
+  public static final String INTERNAL_AGG_FINAL_STAGE = "aggFinalStage";

Review Comment:
   nit: can we convert this to a static class for each hint? That way we'll be able to colocate all properties for each hint like name, HintPredicate, Options Checker, etc? 
   I was thinking something similar to the code here - https://github.com/apache/pinot/pull/10248/files#diff-5d04e3ba2f89fa9f8a646c2a24f79901f0eb107cfc4ccbf574d78213a614b756



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


[GitHub] [pinot] walterddr commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113793078


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java:
##########
@@ -16,19 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.query.planner.hints;
-
-import org.apache.calcite.rel.hint.RelHint;
-
+package org.apache.calcite.rel.hint;
 
 /**
- * Provide certain relational hint to query planner for better optimization.
+ * Default hint strategy set for Pinot query.
  */
-public class PinotRelationalHints {
-  public static final RelHint AGG_INTERMEDIATE_STAGE = RelHint.builder("AGG_INTERMEDIATE_STAGE").build();
-  public static final RelHint AGG_LEAF_STAGE = RelHint.builder("AGG_LEAF_STAGE").build();
+public class PinotHintStrategyTable {
+  public static final String INTERNAL_AGG_INTERMEDIATE_STAGE = "aggIntermediateStage";
+  public static final String INTERNAL_AGG_FINAL_STAGE = "aggFinalStage";

Review Comment:
   actually let's defer this decision to later. b/c the current framework adds INTERNAL_* RelHints that are most likely going to be hintName without options (neither kvOptions or listOptions) 
   
   we can explicitly add static classes for listOptions and kvOptions later, WDYT?



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


[GitHub] [pinot] walterddr commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113418803


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java:
##########
@@ -172,12 +173,12 @@ public String explainQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions)
 
   @VisibleForTesting
   public QueryPlan planQuery(String sqlQuery) {

Review Comment:
   ahh good catch. yes i think in this case we need multiple entrypoint to support isRemoveComment flag. will follow up



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113463751


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java:
##########
@@ -16,19 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.query.planner.hints;
-
-import org.apache.calcite.rel.hint.RelHint;
-
+package org.apache.calcite.rel.hint;
 
 /**
- * Provide certain relational hint to query planner for better optimization.
+ * Default hint strategy set for Pinot query.
  */
-public class PinotRelationalHints {
-  public static final RelHint AGG_INTERMEDIATE_STAGE = RelHint.builder("AGG_INTERMEDIATE_STAGE").build();
-  public static final RelHint AGG_LEAF_STAGE = RelHint.builder("AGG_LEAF_STAGE").build();
+public class PinotHintStrategyTable {
+  public static final String INTERNAL_AGG_INTERMEDIATE_STAGE = "aggIntermediateStage";

Review Comment:
   Basic question - if the user passes a hint that contains both or one of the `aggIntermediateStage` and `aggFinalStage` hints, wouldn't we completely skip applying the `PinotAggregateExchangeNodeInsertRule` RelOptRule because of the `PinotAggregateExchangeNodeInsertRule.matches` function will evaluate to false? Is this the expected behavior?



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


[GitHub] [pinot] walterddr commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113545907


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java:
##########
@@ -16,19 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.query.planner.hints;
-
-import org.apache.calcite.rel.hint.RelHint;
-
+package org.apache.calcite.rel.hint;
 
 /**
- * Provide certain relational hint to query planner for better optimization.
+ * Default hint strategy set for Pinot query.
  */
-public class PinotRelationalHints {
-  public static final RelHint AGG_INTERMEDIATE_STAGE = RelHint.builder("AGG_INTERMEDIATE_STAGE").build();
-  public static final RelHint AGG_LEAF_STAGE = RelHint.builder("AGG_LEAF_STAGE").build();
+public class PinotHintStrategyTable {
+  public static final String INTERNAL_AGG_INTERMEDIATE_STAGE = "aggIntermediateStage";
+  public static final String INTERNAL_AGG_FINAL_STAGE = "aggFinalStage";

Review Comment:
   sounds good. 



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


[GitHub] [pinot] walterddr commented on pull request #10315: [multistage] properly support query hint

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on PR #10315:
URL: https://github.com/apache/pinot/pull/10315#issuecomment-1438836279

   > @walterddr I have not looked at the code changes here (will look in some time). But I do have a PR out that adds supports for SQL Hints in Pinot - #10248. Can you please take a look? We can discuss about how to converge the changes.
   
   ah. I haven't taken a look at #10248 will try to get to it today. 
   The goal of this PR is simply setting up a framework for hints. we can add hint strategies later. 
   
   As of the test i added. I am not actually suggesting we use the aggFinalStage as the hint to indicate --> the AGG_FINAL_STAGE is suppose to be only used by internal pinot rules. 
   
   we can discuss what to add later, but could you please take a look at the overall framework and see if the HintStrategyTable aligns with your proposed PR?


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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113436552


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java:
##########
@@ -16,19 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.query.planner.hints;
-
-import org.apache.calcite.rel.hint.RelHint;
-
+package org.apache.calcite.rel.hint;
 
 /**
- * Provide certain relational hint to query planner for better optimization.
+ * Default hint strategy set for Pinot query.
  */
-public class PinotRelationalHints {
-  public static final RelHint AGG_INTERMEDIATE_STAGE = RelHint.builder("AGG_INTERMEDIATE_STAGE").build();
-  public static final RelHint AGG_LEAF_STAGE = RelHint.builder("AGG_LEAF_STAGE").build();
+public class PinotHintStrategyTable {
+  public static final String INTERNAL_AGG_INTERMEDIATE_STAGE = "aggIntermediateStage";
+  public static final String INTERNAL_AGG_FINAL_STAGE = "aggFinalStage";

Review Comment:
   nit: can we convert this to a static class for each hint? That way we'll be able to colocate all properties for each hint like name, HintPredicate, Options Checker, etc? 



##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java:
##########
@@ -83,8 +83,8 @@ public boolean matches(RelOptRuleCall call) {
     }
     if (call.rel(0) instanceof Aggregate) {
       Aggregate agg = call.rel(0);
-      return !agg.getHints().contains(PinotRelationalHints.AGG_LEAF_STAGE)
-          && !agg.getHints().contains(PinotRelationalHints.AGG_INTERMEDIATE_STAGE);
+      return !agg.getHints().contains(AggregateNode.INTERMEDIATE_STAGE_HINT)

Review Comment:
   I think we need to define what would happen in the following cases:
   1. User Hint passed is both `aggFinalStage` and `aggIntermediateStage`
   2. User hint passed is only `aggFinalStage`
   3. User hint passed is only `aggIntermediateStage` 
   
   
   Should we also consider adding tests for each of these cases in `ResourceBasedQueryPlansTest.java`?
   



##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java:
##########
@@ -16,19 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.query.planner.hints;
-
-import org.apache.calcite.rel.hint.RelHint;
-
+package org.apache.calcite.rel.hint;
 
 /**
- * Provide certain relational hint to query planner for better optimization.
+ * Default hint strategy set for Pinot query.
  */
-public class PinotRelationalHints {
-  public static final RelHint AGG_INTERMEDIATE_STAGE = RelHint.builder("AGG_INTERMEDIATE_STAGE").build();
-  public static final RelHint AGG_LEAF_STAGE = RelHint.builder("AGG_LEAF_STAGE").build();
+public class PinotHintStrategyTable {
+  public static final String INTERNAL_AGG_INTERMEDIATE_STAGE = "aggIntermediateStage";

Review Comment:
   Basic question - if the user passes a hint that contains both or one of the `aggIntermediateStage` and `aggFinalStage` hints, wouldn't we completely skip applying the PinotAggregateExchangeNodeInsertRule RelOptRule because of the `PinotAggregateExchangeNodeInsertRule.matches` function will evaluate to false? Is this the expected behavior?



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113463751


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java:
##########
@@ -16,19 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.query.planner.hints;
-
-import org.apache.calcite.rel.hint.RelHint;
-
+package org.apache.calcite.rel.hint;
 
 /**
- * Provide certain relational hint to query planner for better optimization.
+ * Default hint strategy set for Pinot query.
  */
-public class PinotRelationalHints {
-  public static final RelHint AGG_INTERMEDIATE_STAGE = RelHint.builder("AGG_INTERMEDIATE_STAGE").build();
-  public static final RelHint AGG_LEAF_STAGE = RelHint.builder("AGG_LEAF_STAGE").build();
+public class PinotHintStrategyTable {
+  public static final String INTERNAL_AGG_INTERMEDIATE_STAGE = "aggIntermediateStage";

Review Comment:
   Basic question - if the user passes a hint that contains both or one of the `aggIntermediateStage` and `aggFinalStage` hints, wouldn't we completely skip applying the `PinotAggregateExchangeNodeInsertRule` RelOptRule because of the `PinotAggregateExchangeNodeInsertRule.matches()` function will evaluate to false? Is this the expected behavior?



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113878213


##########
pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java:
##########
@@ -225,6 +225,31 @@ public void testPlanQueryMultiThread()
     }
   }
 
+  @Test
+  public void testQueryWithHint()

Review Comment:
   Should we add a test for the other hint? and a combination of both hints? 



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


[GitHub] [pinot] walterddr commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113416904


##########
pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java:
##########
@@ -109,10 +109,17 @@ private static String removeTerminatingSemicolon(String sql) {
 
   public static SqlNodeAndOptions compileToSqlNodeAndOptions(String sql)
       throws SqlCompilationException {
+    return compileToSqlNodeAndOptions(sql, true);
+  }
+
+  public static SqlNodeAndOptions compileToSqlNodeAndOptions(String sql, boolean isRemoveComments)
+      throws SqlCompilationException {
     long parseStartTimeNs = System.nanoTime();
 
     // Remove the comments from the query

Review Comment:
   no we are still supporting OPTIONS until next release :-/ so we can't remove it now unfortunately lol



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


[GitHub] [pinot] walterddr commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113547117


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java:
##########
@@ -83,8 +83,8 @@ public boolean matches(RelOptRuleCall call) {
     }
     if (call.rel(0) instanceof Aggregate) {
       Aggregate agg = call.rel(0);
-      return !agg.getHints().contains(PinotRelationalHints.AGG_LEAF_STAGE)
-          && !agg.getHints().contains(PinotRelationalHints.AGG_INTERMEDIATE_STAGE);
+      return !agg.getHints().contains(AggregateNode.INTERMEDIATE_STAGE_HINT)

Review Comment:
   not really. as i mentioned above these 2 hints are not suppose to be directly used by user. this PR only changes the relhint to sqlhint. we are actually not planning to open these 2 up to users. 
   
   do you prefer me changing this to a more explicit internal wording?



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


[GitHub] [pinot] codecov-commenter commented on pull request #10315: [multistage] properly support query hint

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #10315:
URL: https://github.com/apache/pinot/pull/10315#issuecomment-1439419831

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10315](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0f374ed) into [master](https://codecov.io/gh/apache/pinot/commit/442bfc3af1bdae230fda6aa5112f028e7620c5b8?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (442bfc3) will **decrease** coverage by `56.70%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #10315       +/-   ##
   =============================================
   - Coverage     70.43%   13.74%   -56.70%     
   + Complexity     5830      221     -5609     
   =============================================
     Files          2015     1972       -43     
     Lines        109656   107552     -2104     
     Branches      16686    16407      -279     
   =============================================
   - Hits          77236    14782    -62454     
   - Misses        27003    91594    +64591     
   + Partials       5417     1176     -4241     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `13.74% <0.00%> (+0.03%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...org/apache/pinot/sql/parsers/CalciteSqlParser.java](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvcGFyc2Vycy9DYWxjaXRlU3FsUGFyc2VyLmphdmE=) | `0.00% <ø> (-85.06%)` | :arrow_down: |
   | [.../java/org/apache/pinot/query/QueryEnvironment.java](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcGxhbm5lci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvUXVlcnlFbnZpcm9ubWVudC5qYXZh) | `0.00% <0.00%> (-92.86%)` | :arrow_down: |
   | [.../org/apache/pinot/query/planner/StageMetadata.java](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcGxhbm5lci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcGxhbm5lci9TdGFnZU1ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (-93.34%)` | :arrow_down: |
   | [...pache/pinot/query/planner/stage/AggregateNode.java](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcXVlcnktcGxhbm5lci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcGxhbm5lci9zdGFnZS9BZ2dyZWdhdGVOb2RlLmphdmE=) | `0.00% <0.00%> (-91.67%)` | :arrow_down: |
   | [...src/main/java/org/apache/pinot/sql/FilterKind.java](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvRmlsdGVyS2luZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0tleS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...in/java/org/apache/pinot/spi/utils/BytesUtils.java](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQnl0ZXNVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ain/java/org/apache/pinot/common/CustomObject.java](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vQ3VzdG9tT2JqZWN0LmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...n/java/org/apache/pinot/core/data/table/Table.java](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1RhYmxlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1599 more](https://codecov.io/gh/apache/pinot/pull/10315?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #10315: [multistage] properly support query hint

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on code in PR #10315:
URL: https://github.com/apache/pinot/pull/10315#discussion_r1113877703


##########
pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java:
##########
@@ -16,19 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.query.planner.hints;
-
-import org.apache.calcite.rel.hint.RelHint;
-
+package org.apache.calcite.rel.hint;
 
 /**
- * Provide certain relational hint to query planner for better optimization.
+ * Default hint strategy set for Pinot query.
  */
-public class PinotRelationalHints {
-  public static final RelHint AGG_INTERMEDIATE_STAGE = RelHint.builder("AGG_INTERMEDIATE_STAGE").build();
-  public static final RelHint AGG_LEAF_STAGE = RelHint.builder("AGG_LEAF_STAGE").build();
+public class PinotHintStrategyTable {
+  public static final String INTERNAL_AGG_INTERMEDIATE_STAGE = "aggIntermediateStage";
+  public static final String INTERNAL_AGG_FINAL_STAGE = "aggFinalStage";

Review Comment:
   Sure, we can add a TODO for later. 



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