You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by "macroguo-ghy (via GitHub)" <gi...@apache.org> on 2024/01/04 06:17:35 UTC

Re: [PR] [CALCITE-6162] Add rule(s) to remove joins with constant single tuple… [calcite]

macroguo-ghy commented on code in PR #3597:
URL: https://github.com/apache/calcite/pull/3597#discussion_r1441266692


##########
core/src/main/java/org/apache/calcite/rel/rules/SingleValueOptimizationRules.java:
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+/**
+ * Collection of rules which simplify sections of query plan which are known to
+ * produce single row.
+ *
+ * <p>Conventionally, the way to represent a single row relational expression is
+ * with a {@link Values} that has one tuple.
+ *
+ * @see LogicalValues#createOneRow
+ */
+public abstract class SingleValueOptimizationRules {
+
+  public static final RelOptRule JOIN_LEFT_INSTANCE =
+      SingleValueOptimizationRules.JoinLeftSingleRuleConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_RIGHT_INSTANCE =
+      SingleValueOptimizationRules.JoinRightSingleRuleConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_LEFT_PROJECT_INSTANCE =
+      SingleValueOptimizationRules.JoinLeftSingleValueRuleWithExprConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_RIGHT_PROJECT_INSTANCE =
+      SingleValueOptimizationRules.JoinRightSingleValueRuleWithExprConfig.DEFAULT.toRule();
+
+  /**
+   * Transformer class to transform a single value nodes on either side of the join.
+   * This transformer contains the common code for all the SingleValueJoin rules.
+   */
+  private static class SingleValueRelTransformer {
+
+    private Join join;
+    private RelNode relNode;
+    private Predicate<Join> cannotTransform;
+    private BiFunction<RexNode, List<RexNode>, List<RexNode>> litTransformer;
+    private boolean valuesAsLeftChild;
+    private List<RexNode> literals;
+
+    protected SingleValueRelTransformer(
+        Join join, List<RexNode> rexNodes, RelNode otherNode,
+        Predicate<Join> nonTransformable, boolean isValuesLeftChild,
+        BiFunction<RexNode, List<RexNode>, List<RexNode>> litTransformer) {
+      this.relNode = otherNode;

Review Comment:
   Ensure that the names and order of the member variables and parameters match.
   



##########
core/src/main/java/org/apache/calcite/rel/rules/SingleValueOptimizationRules.java:
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+/**
+ * Collection of rules which simplify sections of query plan which are known to
+ * produce single row.
+ *
+ * <p>Conventionally, the way to represent a single row relational expression is
+ * with a {@link Values} that has one tuple.
+ *
+ * @see LogicalValues#createOneRow
+ */
+public abstract class SingleValueOptimizationRules {
+
+  public static final RelOptRule JOIN_LEFT_INSTANCE =
+      SingleValueOptimizationRules.JoinLeftSingleRuleConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_RIGHT_INSTANCE =
+      SingleValueOptimizationRules.JoinRightSingleRuleConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_LEFT_PROJECT_INSTANCE =
+      SingleValueOptimizationRules.JoinLeftSingleValueRuleWithExprConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_RIGHT_PROJECT_INSTANCE =
+      SingleValueOptimizationRules.JoinRightSingleValueRuleWithExprConfig.DEFAULT.toRule();
+
+  /**
+   * Transformer class to transform a single value nodes on either side of the join.
+   * This transformer contains the common code for all the SingleValueJoin rules.
+   */
+  private static class SingleValueRelTransformer {
+
+    private Join join;
+    private RelNode relNode;
+    private Predicate<Join> cannotTransform;
+    private BiFunction<RexNode, List<RexNode>, List<RexNode>> litTransformer;
+    private boolean valuesAsLeftChild;
+    private List<RexNode> literals;
+
+    protected SingleValueRelTransformer(
+        Join join, List<RexNode> rexNodes, RelNode otherNode,
+        Predicate<Join> nonTransformable, boolean isValuesLeftChild,
+        BiFunction<RexNode, List<RexNode>, List<RexNode>> litTransformer) {
+      this.relNode = otherNode;
+      this.join = join;
+      this.cannotTransform = nonTransformable;
+      this.litTransformer = litTransformer;
+      this.valuesAsLeftChild = isValuesLeftChild;
+      this.literals = rexNodes;
+    }
+
+    public @Nullable RelNode transform(RelBuilder relBuilder) {
+      if (cannotTransform.test(join)) {
+        return null;
+      }
+      BitSet bitSet = new BitSet(join.getRowType().getFieldCount());
+      int end = valuesAsLeftChild
+          ? join.getLeft().getRowType().getFieldCount()
+          : join.getRowType().getFieldCount();
+
+      int start = valuesAsLeftChild
+          ? 0
+          : join.getLeft().getRowType().getFieldCount();
+
+      bitSet.set(start, end);
+      RexNode trueNode = relBuilder.getRexBuilder().makeLiteral(true);
+      final RexNode filterCondition =
+          new ReplaceExprWithConstValue(bitSet,
+              literals,
+              (valuesAsLeftChild ? 0 : -1) * join.getLeft().getRowType().getFieldCount())
+              .go(join.getCondition());
+
+      RexNode fixedCondition =
+          valuesAsLeftChild
+              ? RexUtil.shift(filterCondition,
+              -1 * join.getLeft().getRowType().getFieldCount())
+              : filterCondition;
+
+      List<RexNode> rexLiterals = litTransformer.apply(fixedCondition, literals);
+      relBuilder.push(relNode)
+          .filter(join.getJoinType().isOuterJoin() ? trueNode : fixedCondition);
+
+      List<RexNode> rexNodes = relNode
+          .getRowType()
+          .getFieldList()
+          .stream()
+          .map(fld -> relBuilder.field(fld.getIndex()))
+          .collect(Collectors.toList());
+
+      List<RexNode> projects = new ArrayList<>();
+      projects.addAll(valuesAsLeftChild ? rexLiterals : rexNodes);
+      projects.addAll(valuesAsLeftChild ? rexNodes : rexLiterals);
+      return relBuilder.project(projects).build();
+    }
+  }
+
+  /**
+   * A rex shuttle to replace field refs with constants from a {@link Values} row.
+   */
+  private static class ReplaceExprWithConstValue extends RexShuttle {
+
+    private final BitSet bitSet;
+    private final List<RexNode> fieldValues;
+    private final int offset;
+    ReplaceExprWithConstValue(BitSet bitSet, List<RexNode> values, int offset) {
+      this.bitSet = bitSet;
+      this.fieldValues = values;
+      this.offset = offset;
+    }
+    @Override public RexNode visitInputRef(RexInputRef inputRef) {
+      if (bitSet.get(inputRef.getIndex())) {
+        return this.fieldValues.get(inputRef.getIndex() + offset);
+      }
+      return super.visitInputRef(inputRef);
+    }
+
+    public RexNode go(RexNode condition) {
+      return condition.accept(this);
+    }
+  }
+
+  /**
+   * Abstract prune single value rule that implements SubstitutionRule interface.
+   */
+  protected abstract static class PruneSingleValueRule
+      extends RelRule<PruneSingleValueRule.Config>
+      implements SubstitutionRule {
+    protected PruneSingleValueRule(PruneSingleValueRule.Config config) {
+      super(config);
+    }
+
+    protected BiFunction<RexNode, List<RexNode>, List<RexNode>>
+        getRexTransformer(RexBuilder rexBuilder,
+        JoinRelType joinRelType) {
+      switch (joinRelType) {
+      case LEFT:
+      case RIGHT:
+        return (condition, rexLiterals) -> rexLiterals.stream().map(lit ->
+            rexBuilder.makeCall(SqlStdOperatorTable.CASE, condition,
+                lit, rexBuilder.makeNullLiteral(lit.getType()))).collect(Collectors.toList());
+      default:
+        return (condition, rexLiterals) -> rexLiterals.stream().collect(Collectors.toList());
+      }
+    }
+
+    static final Predicate<Join> elibilityPredicate(boolean isLeft) {
+      if (isLeft) {
+        return jn -> jn.getJoinType() == JoinRelType.LEFT
+            || jn.getJoinType() == JoinRelType.FULL
+            || jn.getJoinType() == JoinRelType.ANTI;
+      } else {
+        return jn -> jn.getJoinType() == JoinRelType.RIGHT
+            || jn.getJoinType() == JoinRelType.FULL
+            || jn.getJoinType() == JoinRelType.ANTI;
+      }
+    }
+
+    @Override public boolean autoPruneOld() {
+      return true;
+    }
+
+    /** Rule configuration. */
+    public interface Config extends RelRule.Config {
+      @Override PruneSingleValueRule toRule();
+    }
+  }
+
+  /** Configuration for a rule that simplifies join node with constant row on its right input. */
+  @Value.Immutable
+  interface JoinRightSingleRuleConfig extends PruneSingleValueRule.Config {
+    JoinRightSingleRuleConfig DEFAULT = ImmutableJoinRightSingleRuleConfig.of()
+        .withOperandSupplier(b0 ->
+            b0.operand(Join.class).inputs(
+                b1 -> b1.operand(RelNode.class).anyInputs(),
+                b2 -> b2.operand(Values.class).predicate(Values::isSingleValue).noInputs()))
+        .withDescription("PruneJoinSingleValue(right)");
+
+    @Override default SingleValueOptimizationRules.PruneSingleValueRule toRule() {
+      return new SingleValueOptimizationRules.PruneSingleValueRule(this) {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Predicate<Join> predicate = elibilityPredicate(false);
+          final Join join = call.rel(0);
+          final Values values = call.rel(2);
+          final RelNode right = call.rel(1);
+          final RelBuilder relBuilder = call.builder();
+          RelNode transformed =
+              new SingleValueRelTransformer(join,
+                  values.tuples.get(0).stream().collect(Collectors.toList()), right,
+                  predicate, false, getRexTransformer(relBuilder.getRexBuilder(),
+                  join.getJoinType())).transform(relBuilder);
+          if (transformed != null) {
+            call.transformTo(transformed);
+          }
+        }
+      };
+    }
+  }
+
+  /** Configuration for a rule that simplifies join node with constant row on its left input. */
+  @Value.Immutable
+  interface JoinLeftSingleRuleConfig extends PruneSingleValueRule.Config {
+    JoinLeftSingleRuleConfig DEFAULT = ImmutableJoinLeftSingleRuleConfig.of()
+        .withOperandSupplier(b0 ->
+            b0.operand(Join.class).inputs(
+                b1 -> b1.operand(Values.class).predicate(Values::isSingleValue).noInputs(),
+                b2 -> b2.operand(RelNode.class).anyInputs()))
+        .withDescription("PruneJoinSingleValueRule(left)");
+
+    @Override default SingleValueOptimizationRules.PruneSingleValueRule toRule() {
+      return new SingleValueOptimizationRules.PruneSingleValueRule(this) {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Predicate<Join> predicate = elibilityPredicate(true);
+          final Join join = call.rel(0);
+          final Values values = call.rel(1);
+          final RelNode right = call.rel(2);
+          final RelBuilder relBuilder = call.builder();
+          RelNode transformed =
+              new SingleValueRelTransformer(join,
+                  values.tuples.get(0).stream().collect(Collectors.toList()), right,
+                  predicate, true, getRexTransformer(relBuilder.getRexBuilder(),
+                  join.getJoinType())).transform(relBuilder);
+          if (transformed != null) {
+            call.transformTo(transformed);
+          }
+        }
+      };
+    }
+  }
+
+  /** Configuration for a rule that simplifies join node with a project on a constant row
+   *  on its left input. */
+  @Value.Immutable
+  interface JoinLeftSingleValueRuleWithExprConfig extends PruneSingleValueRule.Config {
+    JoinLeftSingleValueRuleWithExprConfig DEFAULT =
+        ImmutableJoinLeftSingleValueRuleWithExprConfig.of().withOperandSupplier(b0 ->
+            b0.operand(Join.class).inputs(
+                b1 -> b1.operand(Project.class).inputs(
+                    b11 -> b11.operand(Values.class).predicate(Values::isSingleValue).noInputs()),
+                b2 -> b2.operand(RelNode.class).anyInputs()))
+        .withDescription("PruneJoinSingleValueRuleWithExpr(left)");
+
+    @Override default SingleValueOptimizationRules.PruneSingleValueRule toRule() {
+      return new SingleValueOptimizationRules.PruneSingleValueRule(this) {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Predicate<Join> predicate = elibilityPredicate(true);
+          Join join = call.rel(0);
+          Project project = call.rel(1);
+          Values values = call.rel(2);
+          RelNode other = call.rel(3);
+          BitSet bitSet = new BitSet();
+          bitSet.set(0, values.getRowType().getFieldCount());
+          RexShuttle shuttle =
+              new ReplaceExprWithConstValue(bitSet,
+                  values.getTuples().get(0).stream().collect(Collectors.toList()),
+                  0);
+
+          List<RexNode> rexNodes =
+              project.getProjects().stream()
+                  .map(proj -> shuttle.apply(proj))
+                  .collect(Collectors.toList());
+
+          RelBuilder relBuilder = call.builder();
+          RelNode transformed =
+              new SingleValueRelTransformer(join, rexNodes, other,
+                  predicate, true, getRexTransformer(relBuilder.getRexBuilder(),
+                  join.getJoinType())).transform(relBuilder);
+          if (transformed != null) {
+            call.transformTo(transformed);
+          }
+        }
+      };
+    }
+  }
+
+  /** Configuration for a rule that simplifies join node with a project on a constant row
+   *  on its right input. */
+  @Value.Immutable
+  interface JoinRightSingleValueRuleWithExprConfig extends PruneSingleValueRule.Config {
+    JoinRightSingleValueRuleWithExprConfig DEFAULT =
+        ImmutableJoinRightSingleValueRuleWithExprConfig.of().withOperandSupplier(b0 ->
+            b0.operand(Join.class).inputs(
+                b1 -> b1.operand(RelNode.class).anyInputs(),
+                b2 -> b2.operand(Project.class).inputs(
+                    b21 -> b21.operand(Values.class).predicate(Values::isSingleValue).noInputs())))
+        .withDescription("PruneJoinSingleValueRuleWithExpr(right)");
+
+    @Override default SingleValueOptimizationRules.PruneSingleValueRule toRule() {
+      return new SingleValueOptimizationRules.PruneSingleValueRule(this) {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Predicate<Join> predicate = elibilityPredicate(false);
+          Join join = call.rel(0);
+          RelNode other = call.rel(1);
+          Project project = call.rel(2);
+          Values values = call.rel(3);
+          RelBuilder relBuilder = call.builder();
+          BitSet bitSet = new BitSet();

Review Comment:
   Use `ImmutableBitSet`?



##########
core/src/main/java/org/apache/calcite/rel/rules/SingleValueOptimizationRules.java:
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+/**
+ * Collection of rules which simplify sections of query plan which are known to
+ * produce single row.
+ *
+ * <p>Conventionally, the way to represent a single row relational expression is
+ * with a {@link Values} that has one tuple.
+ *
+ * @see LogicalValues#createOneRow
+ */
+public abstract class SingleValueOptimizationRules {
+
+  public static final RelOptRule JOIN_LEFT_INSTANCE =
+      SingleValueOptimizationRules.JoinLeftSingleRuleConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_RIGHT_INSTANCE =
+      SingleValueOptimizationRules.JoinRightSingleRuleConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_LEFT_PROJECT_INSTANCE =
+      SingleValueOptimizationRules.JoinLeftSingleValueRuleWithExprConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_RIGHT_PROJECT_INSTANCE =
+      SingleValueOptimizationRules.JoinRightSingleValueRuleWithExprConfig.DEFAULT.toRule();
+
+  /**
+   * Transformer class to transform a single value nodes on either side of the join.
+   * This transformer contains the common code for all the SingleValueJoin rules.
+   */
+  private static class SingleValueRelTransformer {
+
+    private Join join;
+    private RelNode relNode;
+    private Predicate<Join> cannotTransform;
+    private BiFunction<RexNode, List<RexNode>, List<RexNode>> litTransformer;
+    private boolean valuesAsLeftChild;
+    private List<RexNode> literals;
+
+    protected SingleValueRelTransformer(
+        Join join, List<RexNode> rexNodes, RelNode otherNode,
+        Predicate<Join> nonTransformable, boolean isValuesLeftChild,
+        BiFunction<RexNode, List<RexNode>, List<RexNode>> litTransformer) {
+      this.relNode = otherNode;
+      this.join = join;
+      this.cannotTransform = nonTransformable;
+      this.litTransformer = litTransformer;
+      this.valuesAsLeftChild = isValuesLeftChild;
+      this.literals = rexNodes;
+    }
+
+    public @Nullable RelNode transform(RelBuilder relBuilder) {
+      if (cannotTransform.test(join)) {
+        return null;
+      }
+      BitSet bitSet = new BitSet(join.getRowType().getFieldCount());
+      int end = valuesAsLeftChild
+          ? join.getLeft().getRowType().getFieldCount()
+          : join.getRowType().getFieldCount();
+
+      int start = valuesAsLeftChild
+          ? 0
+          : join.getLeft().getRowType().getFieldCount();
+
+      bitSet.set(start, end);
+      RexNode trueNode = relBuilder.getRexBuilder().makeLiteral(true);
+      final RexNode filterCondition =
+          new ReplaceExprWithConstValue(bitSet,
+              literals,
+              (valuesAsLeftChild ? 0 : -1) * join.getLeft().getRowType().getFieldCount())
+              .go(join.getCondition());
+
+      RexNode fixedCondition =
+          valuesAsLeftChild
+              ? RexUtil.shift(filterCondition,
+              -1 * join.getLeft().getRowType().getFieldCount())
+              : filterCondition;
+
+      List<RexNode> rexLiterals = litTransformer.apply(fixedCondition, literals);
+      relBuilder.push(relNode)
+          .filter(join.getJoinType().isOuterJoin() ? trueNode : fixedCondition);
+
+      List<RexNode> rexNodes = relNode
+          .getRowType()
+          .getFieldList()
+          .stream()
+          .map(fld -> relBuilder.field(fld.getIndex()))
+          .collect(Collectors.toList());
+
+      List<RexNode> projects = new ArrayList<>();
+      projects.addAll(valuesAsLeftChild ? rexLiterals : rexNodes);
+      projects.addAll(valuesAsLeftChild ? rexNodes : rexLiterals);
+      return relBuilder.project(projects).build();
+    }
+  }
+
+  /**
+   * A rex shuttle to replace field refs with constants from a {@link Values} row.
+   */
+  private static class ReplaceExprWithConstValue extends RexShuttle {
+
+    private final BitSet bitSet;
+    private final List<RexNode> fieldValues;
+    private final int offset;
+    ReplaceExprWithConstValue(BitSet bitSet, List<RexNode> values, int offset) {
+      this.bitSet = bitSet;
+      this.fieldValues = values;
+      this.offset = offset;
+    }
+    @Override public RexNode visitInputRef(RexInputRef inputRef) {
+      if (bitSet.get(inputRef.getIndex())) {
+        return this.fieldValues.get(inputRef.getIndex() + offset);
+      }
+      return super.visitInputRef(inputRef);
+    }
+
+    public RexNode go(RexNode condition) {
+      return condition.accept(this);
+    }
+  }
+
+  /**
+   * Abstract prune single value rule that implements SubstitutionRule interface.
+   */
+  protected abstract static class PruneSingleValueRule
+      extends RelRule<PruneSingleValueRule.Config>
+      implements SubstitutionRule {
+    protected PruneSingleValueRule(PruneSingleValueRule.Config config) {
+      super(config);
+    }
+
+    protected BiFunction<RexNode, List<RexNode>, List<RexNode>>
+        getRexTransformer(RexBuilder rexBuilder,
+        JoinRelType joinRelType) {
+      switch (joinRelType) {
+      case LEFT:
+      case RIGHT:
+        return (condition, rexLiterals) -> rexLiterals.stream().map(lit ->
+            rexBuilder.makeCall(SqlStdOperatorTable.CASE, condition,
+                lit, rexBuilder.makeNullLiteral(lit.getType()))).collect(Collectors.toList());
+      default:
+        return (condition, rexLiterals) -> rexLiterals.stream().collect(Collectors.toList());
+      }
+    }
+
+    static final Predicate<Join> elibilityPredicate(boolean isLeft) {

Review Comment:
   Can this rule be applied to semi join?



##########
core/src/main/java/org/apache/calcite/rel/rules/SingleValueOptimizationRules.java:
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+/**
+ * Collection of rules which simplify sections of query plan which are known to
+ * produce single row.
+ *
+ * <p>Conventionally, the way to represent a single row relational expression is
+ * with a {@link Values} that has one tuple.
+ *
+ * @see LogicalValues#createOneRow
+ */
+public abstract class SingleValueOptimizationRules {
+
+  public static final RelOptRule JOIN_LEFT_INSTANCE =
+      SingleValueOptimizationRules.JoinLeftSingleRuleConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_RIGHT_INSTANCE =
+      SingleValueOptimizationRules.JoinRightSingleRuleConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_LEFT_PROJECT_INSTANCE =
+      SingleValueOptimizationRules.JoinLeftSingleValueRuleWithExprConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_RIGHT_PROJECT_INSTANCE =
+      SingleValueOptimizationRules.JoinRightSingleValueRuleWithExprConfig.DEFAULT.toRule();
+
+  /**
+   * Transformer class to transform a single value nodes on either side of the join.
+   * This transformer contains the common code for all the SingleValueJoin rules.
+   */
+  private static class SingleValueRelTransformer {
+
+    private Join join;
+    private RelNode relNode;
+    private Predicate<Join> cannotTransform;
+    private BiFunction<RexNode, List<RexNode>, List<RexNode>> litTransformer;
+    private boolean valuesAsLeftChild;
+    private List<RexNode> literals;
+
+    protected SingleValueRelTransformer(
+        Join join, List<RexNode> rexNodes, RelNode otherNode,
+        Predicate<Join> nonTransformable, boolean isValuesLeftChild,
+        BiFunction<RexNode, List<RexNode>, List<RexNode>> litTransformer) {
+      this.relNode = otherNode;
+      this.join = join;
+      this.cannotTransform = nonTransformable;
+      this.litTransformer = litTransformer;
+      this.valuesAsLeftChild = isValuesLeftChild;
+      this.literals = rexNodes;
+    }
+
+    public @Nullable RelNode transform(RelBuilder relBuilder) {
+      if (cannotTransform.test(join)) {
+        return null;
+      }
+      BitSet bitSet = new BitSet(join.getRowType().getFieldCount());
+      int end = valuesAsLeftChild
+          ? join.getLeft().getRowType().getFieldCount()
+          : join.getRowType().getFieldCount();
+
+      int start = valuesAsLeftChild
+          ? 0
+          : join.getLeft().getRowType().getFieldCount();
+
+      bitSet.set(start, end);
+      RexNode trueNode = relBuilder.getRexBuilder().makeLiteral(true);
+      final RexNode filterCondition =
+          new ReplaceExprWithConstValue(bitSet,
+              literals,
+              (valuesAsLeftChild ? 0 : -1) * join.getLeft().getRowType().getFieldCount())
+              .go(join.getCondition());
+
+      RexNode fixedCondition =
+          valuesAsLeftChild
+              ? RexUtil.shift(filterCondition,
+              -1 * join.getLeft().getRowType().getFieldCount())
+              : filterCondition;
+
+      List<RexNode> rexLiterals = litTransformer.apply(fixedCondition, literals);
+      relBuilder.push(relNode)
+          .filter(join.getJoinType().isOuterJoin() ? trueNode : fixedCondition);
+
+      List<RexNode> rexNodes = relNode
+          .getRowType()
+          .getFieldList()
+          .stream()
+          .map(fld -> relBuilder.field(fld.getIndex()))
+          .collect(Collectors.toList());
+
+      List<RexNode> projects = new ArrayList<>();
+      projects.addAll(valuesAsLeftChild ? rexLiterals : rexNodes);
+      projects.addAll(valuesAsLeftChild ? rexNodes : rexLiterals);
+      return relBuilder.project(projects).build();
+    }
+  }
+
+  /**
+   * A rex shuttle to replace field refs with constants from a {@link Values} row.
+   */
+  private static class ReplaceExprWithConstValue extends RexShuttle {
+
+    private final BitSet bitSet;
+    private final List<RexNode> fieldValues;
+    private final int offset;
+    ReplaceExprWithConstValue(BitSet bitSet, List<RexNode> values, int offset) {
+      this.bitSet = bitSet;
+      this.fieldValues = values;
+      this.offset = offset;
+    }
+    @Override public RexNode visitInputRef(RexInputRef inputRef) {
+      if (bitSet.get(inputRef.getIndex())) {
+        return this.fieldValues.get(inputRef.getIndex() + offset);
+      }
+      return super.visitInputRef(inputRef);
+    }
+
+    public RexNode go(RexNode condition) {
+      return condition.accept(this);
+    }
+  }
+
+  /**
+   * Abstract prune single value rule that implements SubstitutionRule interface.
+   */
+  protected abstract static class PruneSingleValueRule
+      extends RelRule<PruneSingleValueRule.Config>
+      implements SubstitutionRule {
+    protected PruneSingleValueRule(PruneSingleValueRule.Config config) {
+      super(config);
+    }
+
+    protected BiFunction<RexNode, List<RexNode>, List<RexNode>>
+        getRexTransformer(RexBuilder rexBuilder,
+        JoinRelType joinRelType) {
+      switch (joinRelType) {
+      case LEFT:
+      case RIGHT:
+        return (condition, rexLiterals) -> rexLiterals.stream().map(lit ->
+            rexBuilder.makeCall(SqlStdOperatorTable.CASE, condition,
+                lit, rexBuilder.makeNullLiteral(lit.getType()))).collect(Collectors.toList());
+      default:
+        return (condition, rexLiterals) -> rexLiterals.stream().collect(Collectors.toList());
+      }
+    }
+
+    static final Predicate<Join> elibilityPredicate(boolean isLeft) {
+      if (isLeft) {
+        return jn -> jn.getJoinType() == JoinRelType.LEFT
+            || jn.getJoinType() == JoinRelType.FULL
+            || jn.getJoinType() == JoinRelType.ANTI;
+      } else {
+        return jn -> jn.getJoinType() == JoinRelType.RIGHT
+            || jn.getJoinType() == JoinRelType.FULL
+            || jn.getJoinType() == JoinRelType.ANTI;
+      }
+    }
+
+    @Override public boolean autoPruneOld() {
+      return true;
+    }
+
+    /** Rule configuration. */
+    public interface Config extends RelRule.Config {
+      @Override PruneSingleValueRule toRule();
+    }
+  }
+
+  /** Configuration for a rule that simplifies join node with constant row on its right input. */
+  @Value.Immutable
+  interface JoinRightSingleRuleConfig extends PruneSingleValueRule.Config {
+    JoinRightSingleRuleConfig DEFAULT = ImmutableJoinRightSingleRuleConfig.of()
+        .withOperandSupplier(b0 ->
+            b0.operand(Join.class).inputs(
+                b1 -> b1.operand(RelNode.class).anyInputs(),
+                b2 -> b2.operand(Values.class).predicate(Values::isSingleValue).noInputs()))
+        .withDescription("PruneJoinSingleValue(right)");
+
+    @Override default SingleValueOptimizationRules.PruneSingleValueRule toRule() {
+      return new SingleValueOptimizationRules.PruneSingleValueRule(this) {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Predicate<Join> predicate = elibilityPredicate(false);
+          final Join join = call.rel(0);
+          final Values values = call.rel(2);
+          final RelNode right = call.rel(1);
+          final RelBuilder relBuilder = call.builder();
+          RelNode transformed =
+              new SingleValueRelTransformer(join,
+                  values.tuples.get(0).stream().collect(Collectors.toList()), right,
+                  predicate, false, getRexTransformer(relBuilder.getRexBuilder(),
+                  join.getJoinType())).transform(relBuilder);
+          if (transformed != null) {
+            call.transformTo(transformed);
+          }
+        }
+      };
+    }
+  }
+
+  /** Configuration for a rule that simplifies join node with constant row on its left input. */
+  @Value.Immutable
+  interface JoinLeftSingleRuleConfig extends PruneSingleValueRule.Config {
+    JoinLeftSingleRuleConfig DEFAULT = ImmutableJoinLeftSingleRuleConfig.of()
+        .withOperandSupplier(b0 ->
+            b0.operand(Join.class).inputs(
+                b1 -> b1.operand(Values.class).predicate(Values::isSingleValue).noInputs(),
+                b2 -> b2.operand(RelNode.class).anyInputs()))
+        .withDescription("PruneJoinSingleValueRule(left)");
+
+    @Override default SingleValueOptimizationRules.PruneSingleValueRule toRule() {
+      return new SingleValueOptimizationRules.PruneSingleValueRule(this) {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Predicate<Join> predicate = elibilityPredicate(true);
+          final Join join = call.rel(0);
+          final Values values = call.rel(1);
+          final RelNode right = call.rel(2);
+          final RelBuilder relBuilder = call.builder();
+          RelNode transformed =
+              new SingleValueRelTransformer(join,
+                  values.tuples.get(0).stream().collect(Collectors.toList()), right,
+                  predicate, true, getRexTransformer(relBuilder.getRexBuilder(),
+                  join.getJoinType())).transform(relBuilder);
+          if (transformed != null) {
+            call.transformTo(transformed);
+          }
+        }
+      };
+    }
+  }
+
+  /** Configuration for a rule that simplifies join node with a project on a constant row
+   *  on its left input. */
+  @Value.Immutable
+  interface JoinLeftSingleValueRuleWithExprConfig extends PruneSingleValueRule.Config {
+    JoinLeftSingleValueRuleWithExprConfig DEFAULT =
+        ImmutableJoinLeftSingleValueRuleWithExprConfig.of().withOperandSupplier(b0 ->
+            b0.operand(Join.class).inputs(
+                b1 -> b1.operand(Project.class).inputs(
+                    b11 -> b11.operand(Values.class).predicate(Values::isSingleValue).noInputs()),
+                b2 -> b2.operand(RelNode.class).anyInputs()))
+        .withDescription("PruneJoinSingleValueRuleWithExpr(left)");
+
+    @Override default SingleValueOptimizationRules.PruneSingleValueRule toRule() {
+      return new SingleValueOptimizationRules.PruneSingleValueRule(this) {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Predicate<Join> predicate = elibilityPredicate(true);
+          Join join = call.rel(0);
+          Project project = call.rel(1);
+          Values values = call.rel(2);
+          RelNode other = call.rel(3);
+          BitSet bitSet = new BitSet();
+          bitSet.set(0, values.getRowType().getFieldCount());
+          RexShuttle shuttle =
+              new ReplaceExprWithConstValue(bitSet,
+                  values.getTuples().get(0).stream().collect(Collectors.toList()),
+                  0);
+
+          List<RexNode> rexNodes =
+              project.getProjects().stream()
+                  .map(proj -> shuttle.apply(proj))
+                  .collect(Collectors.toList());
+
+          RelBuilder relBuilder = call.builder();
+          RelNode transformed =
+              new SingleValueRelTransformer(join, rexNodes, other,
+                  predicate, true, getRexTransformer(relBuilder.getRexBuilder(),
+                  join.getJoinType())).transform(relBuilder);
+          if (transformed != null) {
+            call.transformTo(transformed);
+          }
+        }
+      };
+    }
+  }
+
+  /** Configuration for a rule that simplifies join node with a project on a constant row
+   *  on its right input. */
+  @Value.Immutable
+  interface JoinRightSingleValueRuleWithExprConfig extends PruneSingleValueRule.Config {
+    JoinRightSingleValueRuleWithExprConfig DEFAULT =
+        ImmutableJoinRightSingleValueRuleWithExprConfig.of().withOperandSupplier(b0 ->
+            b0.operand(Join.class).inputs(
+                b1 -> b1.operand(RelNode.class).anyInputs(),
+                b2 -> b2.operand(Project.class).inputs(
+                    b21 -> b21.operand(Values.class).predicate(Values::isSingleValue).noInputs())))
+        .withDescription("PruneJoinSingleValueRuleWithExpr(right)");
+
+    @Override default SingleValueOptimizationRules.PruneSingleValueRule toRule() {
+      return new SingleValueOptimizationRules.PruneSingleValueRule(this) {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Predicate<Join> predicate = elibilityPredicate(false);
+          Join join = call.rel(0);

Review Comment:
   Use `final` modifier



##########
core/src/main/java/org/apache/calcite/rel/rules/SingleValueOptimizationRules.java:
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.immutables.value.Value;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+/**
+ * Collection of rules which simplify sections of query plan which are known to
+ * produce single row.
+ *
+ * <p>Conventionally, the way to represent a single row relational expression is
+ * with a {@link Values} that has one tuple.
+ *
+ * @see LogicalValues#createOneRow
+ */
+public abstract class SingleValueOptimizationRules {
+
+  public static final RelOptRule JOIN_LEFT_INSTANCE =
+      SingleValueOptimizationRules.JoinLeftSingleRuleConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_RIGHT_INSTANCE =
+      SingleValueOptimizationRules.JoinRightSingleRuleConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_LEFT_PROJECT_INSTANCE =
+      SingleValueOptimizationRules.JoinLeftSingleValueRuleWithExprConfig.DEFAULT.toRule();
+
+  public static final RelOptRule JOIN_RIGHT_PROJECT_INSTANCE =
+      SingleValueOptimizationRules.JoinRightSingleValueRuleWithExprConfig.DEFAULT.toRule();
+
+  /**
+   * Transformer class to transform a single value nodes on either side of the join.
+   * This transformer contains the common code for all the SingleValueJoin rules.
+   */
+  private static class SingleValueRelTransformer {
+
+    private Join join;
+    private RelNode relNode;
+    private Predicate<Join> cannotTransform;
+    private BiFunction<RexNode, List<RexNode>, List<RexNode>> litTransformer;
+    private boolean valuesAsLeftChild;
+    private List<RexNode> literals;
+
+    protected SingleValueRelTransformer(
+        Join join, List<RexNode> rexNodes, RelNode otherNode,
+        Predicate<Join> nonTransformable, boolean isValuesLeftChild,
+        BiFunction<RexNode, List<RexNode>, List<RexNode>> litTransformer) {
+      this.relNode = otherNode;
+      this.join = join;
+      this.cannotTransform = nonTransformable;
+      this.litTransformer = litTransformer;
+      this.valuesAsLeftChild = isValuesLeftChild;
+      this.literals = rexNodes;
+    }
+
+    public @Nullable RelNode transform(RelBuilder relBuilder) {
+      if (cannotTransform.test(join)) {
+        return null;
+      }
+      BitSet bitSet = new BitSet(join.getRowType().getFieldCount());
+      int end = valuesAsLeftChild
+          ? join.getLeft().getRowType().getFieldCount()
+          : join.getRowType().getFieldCount();
+
+      int start = valuesAsLeftChild
+          ? 0
+          : join.getLeft().getRowType().getFieldCount();
+
+      bitSet.set(start, end);
+      RexNode trueNode = relBuilder.getRexBuilder().makeLiteral(true);
+      final RexNode filterCondition =
+          new ReplaceExprWithConstValue(bitSet,
+              literals,
+              (valuesAsLeftChild ? 0 : -1) * join.getLeft().getRowType().getFieldCount())
+              .go(join.getCondition());
+
+      RexNode fixedCondition =
+          valuesAsLeftChild
+              ? RexUtil.shift(filterCondition,
+              -1 * join.getLeft().getRowType().getFieldCount())
+              : filterCondition;
+
+      List<RexNode> rexLiterals = litTransformer.apply(fixedCondition, literals);
+      relBuilder.push(relNode)
+          .filter(join.getJoinType().isOuterJoin() ? trueNode : fixedCondition);
+
+      List<RexNode> rexNodes = relNode
+          .getRowType()
+          .getFieldList()
+          .stream()
+          .map(fld -> relBuilder.field(fld.getIndex()))
+          .collect(Collectors.toList());
+
+      List<RexNode> projects = new ArrayList<>();
+      projects.addAll(valuesAsLeftChild ? rexLiterals : rexNodes);
+      projects.addAll(valuesAsLeftChild ? rexNodes : rexLiterals);
+      return relBuilder.project(projects).build();
+    }
+  }
+
+  /**
+   * A rex shuttle to replace field refs with constants from a {@link Values} row.
+   */
+  private static class ReplaceExprWithConstValue extends RexShuttle {
+
+    private final BitSet bitSet;
+    private final List<RexNode> fieldValues;
+    private final int offset;
+    ReplaceExprWithConstValue(BitSet bitSet, List<RexNode> values, int offset) {
+      this.bitSet = bitSet;
+      this.fieldValues = values;
+      this.offset = offset;
+    }
+    @Override public RexNode visitInputRef(RexInputRef inputRef) {
+      if (bitSet.get(inputRef.getIndex())) {
+        return this.fieldValues.get(inputRef.getIndex() + offset);
+      }
+      return super.visitInputRef(inputRef);
+    }
+
+    public RexNode go(RexNode condition) {
+      return condition.accept(this);
+    }
+  }
+
+  /**
+   * Abstract prune single value rule that implements SubstitutionRule interface.
+   */
+  protected abstract static class PruneSingleValueRule
+      extends RelRule<PruneSingleValueRule.Config>
+      implements SubstitutionRule {
+    protected PruneSingleValueRule(PruneSingleValueRule.Config config) {
+      super(config);
+    }
+
+    protected BiFunction<RexNode, List<RexNode>, List<RexNode>>
+        getRexTransformer(RexBuilder rexBuilder,
+        JoinRelType joinRelType) {
+      switch (joinRelType) {
+      case LEFT:
+      case RIGHT:
+        return (condition, rexLiterals) -> rexLiterals.stream().map(lit ->
+            rexBuilder.makeCall(SqlStdOperatorTable.CASE, condition,
+                lit, rexBuilder.makeNullLiteral(lit.getType()))).collect(Collectors.toList());
+      default:
+        return (condition, rexLiterals) -> rexLiterals.stream().collect(Collectors.toList());
+      }
+    }
+
+    static final Predicate<Join> elibilityPredicate(boolean isLeft) {
+      if (isLeft) {
+        return jn -> jn.getJoinType() == JoinRelType.LEFT
+            || jn.getJoinType() == JoinRelType.FULL
+            || jn.getJoinType() == JoinRelType.ANTI;
+      } else {
+        return jn -> jn.getJoinType() == JoinRelType.RIGHT
+            || jn.getJoinType() == JoinRelType.FULL
+            || jn.getJoinType() == JoinRelType.ANTI;
+      }
+    }
+
+    @Override public boolean autoPruneOld() {
+      return true;
+    }
+
+    /** Rule configuration. */
+    public interface Config extends RelRule.Config {
+      @Override PruneSingleValueRule toRule();
+    }
+  }
+
+  /** Configuration for a rule that simplifies join node with constant row on its right input. */
+  @Value.Immutable
+  interface JoinRightSingleRuleConfig extends PruneSingleValueRule.Config {
+    JoinRightSingleRuleConfig DEFAULT = ImmutableJoinRightSingleRuleConfig.of()
+        .withOperandSupplier(b0 ->
+            b0.operand(Join.class).inputs(
+                b1 -> b1.operand(RelNode.class).anyInputs(),
+                b2 -> b2.operand(Values.class).predicate(Values::isSingleValue).noInputs()))
+        .withDescription("PruneJoinSingleValue(right)");
+
+    @Override default SingleValueOptimizationRules.PruneSingleValueRule toRule() {
+      return new SingleValueOptimizationRules.PruneSingleValueRule(this) {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Predicate<Join> predicate = elibilityPredicate(false);
+          final Join join = call.rel(0);
+          final Values values = call.rel(2);
+          final RelNode right = call.rel(1);
+          final RelBuilder relBuilder = call.builder();
+          RelNode transformed =
+              new SingleValueRelTransformer(join,
+                  values.tuples.get(0).stream().collect(Collectors.toList()), right,
+                  predicate, false, getRexTransformer(relBuilder.getRexBuilder(),
+                  join.getJoinType())).transform(relBuilder);
+          if (transformed != null) {
+            call.transformTo(transformed);
+          }
+        }
+      };
+    }
+  }
+
+  /** Configuration for a rule that simplifies join node with constant row on its left input. */
+  @Value.Immutable
+  interface JoinLeftSingleRuleConfig extends PruneSingleValueRule.Config {
+    JoinLeftSingleRuleConfig DEFAULT = ImmutableJoinLeftSingleRuleConfig.of()
+        .withOperandSupplier(b0 ->
+            b0.operand(Join.class).inputs(
+                b1 -> b1.operand(Values.class).predicate(Values::isSingleValue).noInputs(),
+                b2 -> b2.operand(RelNode.class).anyInputs()))
+        .withDescription("PruneJoinSingleValueRule(left)");
+
+    @Override default SingleValueOptimizationRules.PruneSingleValueRule toRule() {
+      return new SingleValueOptimizationRules.PruneSingleValueRule(this) {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Predicate<Join> predicate = elibilityPredicate(true);
+          final Join join = call.rel(0);
+          final Values values = call.rel(1);
+          final RelNode right = call.rel(2);
+          final RelBuilder relBuilder = call.builder();
+          RelNode transformed =
+              new SingleValueRelTransformer(join,
+                  values.tuples.get(0).stream().collect(Collectors.toList()), right,
+                  predicate, true, getRexTransformer(relBuilder.getRexBuilder(),
+                  join.getJoinType())).transform(relBuilder);
+          if (transformed != null) {
+            call.transformTo(transformed);
+          }
+        }
+      };
+    }
+  }
+
+  /** Configuration for a rule that simplifies join node with a project on a constant row
+   *  on its left input. */
+  @Value.Immutable
+  interface JoinLeftSingleValueRuleWithExprConfig extends PruneSingleValueRule.Config {

Review Comment:
   Agree.



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

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