[GitHub] [calcite] libenchao commented on a diff in pull request #2848: [CALCITE-5201] Improve SemiJoinRule to match Join's right input which is unique for Join keys

2022-11-19 Thread GitBox


libenchao commented on code in PR #2848:
URL: https://github.com/apache/calcite/pull/2848#discussion_r1027061254


##
core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java:
##
@@ -232,6 +234,84 @@ default JoinToSemiJoinRuleConfig 
withOperandFor(Class joinClass,
 }
   }
 
+  /**
+   * SemiJoinRule that matches a Project on top of a Join with a RelNode
+   * which is unique for Join's right keys.
+   *
+   * @see CoreRules#JOIN_ON_UNIQUE_TO_SEMI_JOIN */
+  public static class JoinOnUniqueToSemiJoinRule extends SemiJoinRule {
+
+/** Creates a JoinOnUniqueToSemiJoinRule. */
+protected JoinOnUniqueToSemiJoinRule(JoinOnUniqueToSemiJoinRuleConfig 
config) {
+  super(config);
+}
+
+@Override public boolean matches(RelOptRuleCall call) {
+  final Project project = call.rel(0);
+  final Join join = call.rel(1);
+  final RelNode left = call.rel(2);
+
+  final ImmutableBitSet bits =
+  RelOptUtil.InputFinder.bits(project.getProjects(), null);
+  final ImmutableBitSet rightBits =
+  ImmutableBitSet.range(left.getRowType().getFieldCount(),
+  join.getRowType().getFieldCount());
+  return !bits.intersects(rightBits);
+}
+
+@Override public void onMatch(RelOptRuleCall call) {
+  final Project project = call.rel(0);
+  final Join join = call.rel(1);
+  final RelNode left = call.rel(2);
+  final RelNode right = call.rel(3);
+
+  final JoinInfo joinInfo = join.analyzeCondition();
+  final RelOptCluster cluster = join.getCluster();
+  final RelMetadataQuery mq = cluster.getMetadataQuery();
+  final Boolean unique = mq.areColumnsUnique(right, joinInfo.rightSet());
+  if (unique != null && unique) {
+final RelBuilder builder = call.builder();
+switch (join.getJoinType()) {
+case INNER:
+  builder.push(left);
+  builder.push(right);
+  builder.join(JoinRelType.SEMI, join.getCondition());
+  break;
+case LEFT:
+  builder.push(left);
+  break;
+default:
+  throw new AssertionError(join.getJoinType());
+}
+builder.project(project.getProjects());

Review Comment:
   @sdreynolds thanks for catching this, I noticed that you have opened 
CALCITE-5391 to track this, let's go to there for further discussions.



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

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



[GitHub] [calcite] libenchao commented on a diff in pull request #2848: [CALCITE-5201] Improve SemiJoinRule to match Join's right input which is unique for Join keys

2022-08-11 Thread GitBox


libenchao commented on code in PR #2848:
URL: https://github.com/apache/calcite/pull/2848#discussion_r943331872


##
core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java:
##
@@ -232,6 +234,87 @@ default JoinToSemiJoinRuleConfig 
withOperandFor(Class joinClass,
 }
   }
 
+  /**
+   * SemiJoinRule that matches a Project on top of a Join with a RelNode
+   * which is unique for Join's right keys.
+   *
+   * @see CoreRules#JOIN_ON_UNIQUE_TO_SEMI_JOIN */
+  public static class JoinOnUniqueToSemiJoinRule extends SemiJoinRule {
+
+/** Creates a JoinOnUniqueToSemiJoinRule. */
+protected JoinOnUniqueToSemiJoinRule(JoinOnUniqueToSemiJoinRuleConfig 
config) {
+  super(config);
+}
+
+@Override public boolean matches(RelOptRuleCall call) {
+  final Project project = call.rel(0);
+  final Join join = call.rel(1);
+  final RelNode left = call.rel(2);
+
+  final ImmutableBitSet bits =
+  RelOptUtil.InputFinder.bits(project.getProjects(), null);
+  final ImmutableBitSet rightBits =
+  ImmutableBitSet.range(left.getRowType().getFieldCount(),
+  join.getRowType().getFieldCount());
+  return !bits.intersects(rightBits);
+}
+
+@Override public void onMatch(RelOptRuleCall call) {
+  final Project project = call.rel(0);
+  final Join join = call.rel(1);
+  final RelNode left = call.rel(2);
+  final RelNode right = call.rel(3);
+
+  // Same with ProjectToSemiJoinRule
+  if (right instanceof Aggregate) {

Review Comment:
   sounds good, added.



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

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



[GitHub] [calcite] libenchao commented on a diff in pull request #2848: [CALCITE-5201] Improve SemiJoinRule to match Join's right input which is unique for Join keys

2022-08-10 Thread GitBox


libenchao commented on code in PR #2848:
URL: https://github.com/apache/calcite/pull/2848#discussion_r943103659


##
core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java:
##
@@ -232,6 +234,87 @@ default JoinToSemiJoinRuleConfig 
withOperandFor(Class joinClass,
 }
   }
 
+  /**
+   * SemiJoinRule that matches a Project on top of a Join with a RelNode
+   * which is unique for Join's right keys.
+   *
+   * @see CoreRules#JOIN_ON_UNIQUE_TO_SEMI_JOIN */
+  public static class JoinOnUniqueToSemiJoinRule extends SemiJoinRule {
+
+/** Creates a JoinOnUniqueToSemiJoinRule. */
+protected JoinOnUniqueToSemiJoinRule(JoinOnUniqueToSemiJoinRuleConfig 
config) {
+  super(config);
+}
+
+@Override public boolean matches(RelOptRuleCall call) {
+  final Project project = call.rel(0);
+  final Join join = call.rel(1);
+  final RelNode left = call.rel(2);
+
+  final ImmutableBitSet bits =
+  RelOptUtil.InputFinder.bits(project.getProjects(), null);
+  final ImmutableBitSet rightBits =
+  ImmutableBitSet.range(left.getRowType().getFieldCount(),
+  join.getRowType().getFieldCount());
+  return !bits.intersects(rightBits);
+}
+
+@Override public void onMatch(RelOptRuleCall call) {
+  final Project project = call.rel(0);
+  final Join join = call.rel(1);
+  final RelNode left = call.rel(2);
+  final RelNode right = call.rel(3);
+
+  // Same with ProjectToSemiJoinRule
+  if (right instanceof Aggregate) {

Review Comment:
   Good idea!



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

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



[GitHub] [calcite] libenchao commented on a diff in pull request #2848: [CALCITE-5201] Improve SemiJoinRule to match Join's right input which is unique for Join keys

2022-08-06 Thread GitBox


libenchao commented on code in PR #2848:
URL: https://github.com/apache/calcite/pull/2848#discussion_r939600845


##
core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java:
##
@@ -42,14 +43,18 @@
 /**
  * Planner rule that creates a {@code SemiJoin} from a
  * {@link org.apache.calcite.rel.core.Join} on top of a
- * {@link org.apache.calcite.rel.logical.LogicalAggregate}.
+ * {@link org.apache.calcite.rel.logical.LogicalAggregate} or
+ * on a {@link org.apache.calcite.rel.RelNode} which is
+ * unique for join's right keys.
  */
 public abstract class SemiJoinRule
 extends RelRule
 implements TransformationRule {
   private static boolean isJoinTypeSupported(Join join) {
 final JoinRelType type = join.getJoinType();
-return type == JoinRelType.INNER || type == JoinRelType.LEFT;
+return type == JoinRelType.INNER
+|| type == JoinRelType.LEFT
+|| type == JoinRelType.SEMI;
   }

Review Comment:
   @chunweilei I figured out another way to do this. In 
`JoinOnUniqueToSemiJoinRule`, if we found that right is `Aggregate`, we can 
fallback to `ProjectToSemiJoinRule`. 



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

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



[GitHub] [calcite] libenchao commented on a diff in pull request #2848: [CALCITE-5201] Improve SemiJoinRule to match Join's right input which is unique for Join keys

2022-08-06 Thread GitBox


libenchao commented on code in PR #2848:
URL: https://github.com/apache/calcite/pull/2848#discussion_r939600749


##
core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java:
##
@@ -96,11 +96,7 @@ public static EnumerableLimit create(final RelNode input, 
@Nullable RexNode offs
 final BlockBuilder builder = new BlockBuilder();
 final EnumerableRel child = (EnumerableRel) getInput();
 final Result result = implementor.visitChild(this, 0, child, pref);
-final PhysType physType =
-PhysTypeImpl.of(
-implementor.getTypeFactory(),
-getRowType(),
-result.format);
+final PhysType physType = result.physType;

Review Comment:
   @rubenada I removed these changes in this PR, and logged 
[CALCITE-5229](https://issues.apache.org/jira/browse/CALCITE-5229) as a 
follow-up work.



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

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



[GitHub] [calcite] libenchao commented on a diff in pull request #2848: [CALCITE-5201] Improve SemiJoinRule to match Join's right input which is unique for Join keys

2022-07-06 Thread GitBox


libenchao commented on code in PR #2848:
URL: https://github.com/apache/calcite/pull/2848#discussion_r915379084


##
core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java:
##
@@ -42,14 +43,18 @@
 /**
  * Planner rule that creates a {@code SemiJoin} from a
  * {@link org.apache.calcite.rel.core.Join} on top of a
- * {@link org.apache.calcite.rel.logical.LogicalAggregate}.
+ * {@link org.apache.calcite.rel.logical.LogicalAggregate} or
+ * on a {@link org.apache.calcite.rel.RelNode} which is
+ * unique for join's right keys.
  */
 public abstract class SemiJoinRule
 extends RelRule
 implements TransformationRule {
   private static boolean isJoinTypeSupported(Join join) {
 final JoinRelType type = join.getJoinType();
-return type == JoinRelType.INNER || type == JoinRelType.LEFT;
+return type == JoinRelType.INNER
+|| type == JoinRelType.LEFT
+|| type == JoinRelType.SEMI;
   }

Review Comment:
   It's good to know there is already a discussion about this before in 
CALCITE-4623.
   
   Consider this case, we have a plan below:
   ```xml
   LogicalJoin(condition=[=($7, $9)], joinType=[inner])
 LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 LogicalAggregate(group=[{0}], agg#0=[MIN($1)])
   LogicalProject(DEPTNO=[$0], $f0=[true])
 LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
   ```
   After `JoinOnUniqueToSemiJoinRule`, it would be transformed to:
   ```xml
   LogicalJoin(condition=[=($7, $9)], joinType=[semi])
 LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 LogicalAggregate(group=[{0}], agg#0=[MIN($1)])
   LogicalProject(DEPTNO=[$0], $f0=[true])
 LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
   ```
   And we can still use `JoinToSemiJoinRule` to remove the `Aggregate`:
   ```xml
   LogicalJoin(condition=[=($7, $9)], joinType=[semi])
 LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 LogicalProject(DEPTNO=[$0], $f0=[true])
   LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
   ```
   
   In many cases, we would not expect the rules firing orders, especially in 
`HepPlanner`, hence the above transforming order is valid and should be valid 
IMHO.



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

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



[GitHub] [calcite] libenchao commented on a diff in pull request #2848: [CALCITE-5201] Improve SemiJoinRule to match Join's right input which is unique for Join keys

2022-07-06 Thread GitBox


libenchao commented on code in PR #2848:
URL: https://github.com/apache/calcite/pull/2848#discussion_r915375337


##
core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java:
##
@@ -96,11 +96,7 @@ public static EnumerableLimit create(final RelNode input, 
@Nullable RexNode offs
 final BlockBuilder builder = new BlockBuilder();
 final EnumerableRel child = (EnumerableRel) getInput();
 final Result result = implementor.visitChild(this, 0, child, pref);
-final PhysType physType =
-PhysTypeImpl.of(
-implementor.getTypeFactory(),
-getRowType(),
-result.format);
+final PhysType physType = result.physType;

Review Comment:
   The plan before this PR:
   ```xml
   EnumerableLimit(offset=[?1], fetch=[?0])
 EnumerableSort(sort0=[$3], dir0=[ASC])
   EnumerableCalc(expr#0..6=[{inputs}], empid=[$t2], deptno=[$t3], 
name=[$t4], salary=[$t5], commission=[$t6])
 EnumerableHashJoin(condition=[AND(=($0, $2), =($1, $3))], 
joinType=[inner])
   EnumerableAggregate(group=[{0}], EXPR$0=[MAX($1)])
 EnumerableTableScan(table=[[hr, emps]])
   EnumerableTableScan(table=[[hr, emps]])
   ```
   after this PR:
   ```xml
   EnumerableSort(offset=[?1], fetch=[?0])
 EnumerableSort(sort0=[$3], dir0=[ASC])
   EnumerableHashJoin(condition=[AND(=($0, $5), =($1, $6))], 
joinType=[semi])
 EnumerableTableScan(table=[[hr, emps]])
 EnumerableAggregate(group=[{0}], EXPR$0=[MAX($1)])
   EnumerableTableScan(table=[[hr, emps]])
   ```
   
   `EnumerableHashJoin(joinType=[semi])`'s physical type is `Employee`. 
However, `EnumerableSort` and `EnumerableSort`'s physical type is `Record5_0`. 
That's why after this PR, the test will fail.
   
   > This change seems a bit ad-hoc. Why is it applied only on 
{{EnumerableLimit}} and {{EnumerableSort}}, but we keep the "old code" in other 
operators like {{EnumerableLimitSort}} or {{EnumerableAggregate}}?
   
   You are right, we should check all places, or figure out another way to fix 
this.
   
   This is indeed orthogonal with current PR, maybe we can leave it as a 
follow-up issue. I've considered this way before, there may be two ways to to 
this:
   * Remove `JoinOnUniqueToSemiJoinRule` from 
`RelOptRules#ABSTRACT_RELATIONAL_RULES` and `Prorams#RULE_SET`
   * Disable `JdbcTest#testDynamicParameterInLimitOffset` and link the 
follow-up issue id to it  
   
   What do you think about these two ways?



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

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



[GitHub] [calcite] libenchao commented on a diff in pull request #2848: [CALCITE-5201] Improve SemiJoinRule to match Join's right input which is unique for Join keys

2022-07-05 Thread GitBox


libenchao commented on code in PR #2848:
URL: https://github.com/apache/calcite/pull/2848#discussion_r913762560


##
core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimit.java:
##
@@ -96,11 +96,7 @@ public static EnumerableLimit create(final RelNode input, 
@Nullable RexNode offs
 final BlockBuilder builder = new BlockBuilder();
 final EnumerableRel child = (EnumerableRel) getInput();
 final Result result = implementor.visitChild(this, 0, child, pref);
-final PhysType physType =
-PhysTypeImpl.of(
-implementor.getTypeFactory(),
-getRowType(),
-result.format);
+final PhysType physType = result.physType;

Review Comment:
   Yes, you are right. I also struggled with this change for some time.
   Without this change, `JdbcTest#testDynamicParameterInLimitOffset` will fail. 
The reason is that in this PR, we strengthened the ability to transform to semi 
join, and this test case is the affected one. The physical implementation for 
semi join and inner join is different in `EnumerableHashJoin`, and this leads 
to the test failure.
   
   I also thought about to leave it as a follow-up issue, but I found it's very 
hard to reproduce this error without this PR. Hence I put the fix in this PR.
   What do you think about this?



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

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



[GitHub] [calcite] libenchao commented on a diff in pull request #2848: [CALCITE-5201] Improve SemiJoinRule to match Join's right input which is unique for Join keys

2022-07-05 Thread GitBox


libenchao commented on code in PR #2848:
URL: https://github.com/apache/calcite/pull/2848#discussion_r913754054


##
core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java:
##
@@ -42,14 +43,18 @@
 /**
  * Planner rule that creates a {@code SemiJoin} from a
  * {@link org.apache.calcite.rel.core.Join} on top of a
- * {@link org.apache.calcite.rel.logical.LogicalAggregate}.
+ * {@link org.apache.calcite.rel.logical.LogicalAggregate} or
+ * on a {@link org.apache.calcite.rel.RelNode} which is
+ * unique for join's right keys.
  */
 public abstract class SemiJoinRule
 extends RelRule
 implements TransformationRule {
   private static boolean isJoinTypeSupported(Join join) {
 final JoinRelType type = join.getJoinType();
-return type == JoinRelType.INNER || type == JoinRelType.LEFT;
+return type == JoinRelType.INNER
+|| type == JoinRelType.LEFT
+|| type == JoinRelType.SEMI;
   }

Review Comment:
   I put it in this condition because `JoinToSemiJoinRule` needs it. If it's 
already a semi join, we can still removes the `Aggregate` if we could match. In 
current implementation, we already considered the semi join case, see 
https://github.com/apache/calcite/blob/724eb032d0141c15d17422d50c5235be00ac989f/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java#L81
 and 
https://github.com/apache/calcite/blob/724eb032d0141c15d17422d50c5235be00ac989f/core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java#L99



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

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



[GitHub] [calcite] libenchao commented on a diff in pull request #2848: [CALCITE-5201] Improve SemiJoinRule to match Join's right input which is unique for Join keys

2022-07-05 Thread GitBox


libenchao commented on code in PR #2848:
URL: https://github.com/apache/calcite/pull/2848#discussion_r913747509


##
core/src/main/java/org/apache/calcite/rel/rules/SemiJoinRule.java:
##
@@ -232,6 +237,79 @@ default JoinToSemiJoinRuleConfig 
withOperandFor(Class joinClass,
 }
   }
 
+  /**
+   * SemiJoinRule that matches a Join with a RelNode which is unique
+   * for Join's right keys.
+   *
+   * @see CoreRules#JOIN_ON_UNIQUE_TO_SEMI_JOIN */
+  public static class JoinOnUniqueToSemiJoinRule extends SemiJoinRule {
+
+/** Creates a JoinOnUniqueToSemiJoinRule. */
+protected JoinOnUniqueToSemiJoinRule(JoinOnUniqueToSemiJoinRuleConfig 
config) {
+  super(config);
+}
+
+@Override public void onMatch(RelOptRuleCall call) {
+  final Project project = call.rel(0);
+  final Join join = call.rel(1);
+  final RelNode left = call.rel(2);
+  final RelNode right = call.rel(3);
+
+  // return if right's fields are used.
+  final ImmutableBitSet bits =
+  RelOptUtil.InputFinder.bits(project.getProjects(), null);
+  final ImmutableBitSet rightBits =
+  ImmutableBitSet.range(left.getRowType().getFieldCount(),
+  join.getRowType().getFieldCount());
+  if (bits.intersects(rightBits)) {
+return;
+  }
+
+  final JoinInfo joinInfo = join.analyzeCondition();
+  final RelOptCluster cluster = join.getCluster();
+  final RelMetadataQuery mq = cluster.getMetadataQuery();
+  final Boolean unique = mq.areColumnsUnique(right, joinInfo.rightSet());
+  if (unique != null && unique) {
+final RelBuilder builder = call.builder();
+switch (join.getJoinType()) {
+case INNER:
+case SEMI:
+  builder.push(left);
+  builder.push(right);
+  builder.join(JoinRelType.SEMI, join.getCondition());
+  break;
+case LEFT:
+  builder.push(left);
+  break;
+default:
+  throw new AssertionError(join.getJoinType());
+}
+builder.project(project.getProjects());
+call.transformTo(builder.build());
+  }
+}
+
+/**
+ * Rule configuration.
+ */
+@Value.Immutable
+public interface JoinOnUniqueToSemiJoinRuleConfig extends 
SemiJoinRule.Config {
+  JoinOnUniqueToSemiJoinRuleConfig DEFAULT = 
ImmutableJoinOnUniqueToSemiJoinRuleConfig.of()
+  .withDescription("SemiJoinRule:unique")
+  .withOperandSupplier(b ->
+  b.operand(Project.class).oneInput(
+  b2 -> 
b2.operand(Join.class).predicate(SemiJoinRule::isJoinTypeSupported).inputs(
+  b3 -> b3.operand(RelNode.class).anyInputs(),
+  b4 -> b4.operand(RelNode.class).anyInputs()

Review Comment:
   We need to check whether the right input is unique for Join's keys, hence we 
need at least the right input.



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

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