You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by da...@apache.org on 2020/07/17 01:38:14 UTC

[calcite] branch master updated (551e3f5 -> a4fa054)

This is an automated email from the ASF dual-hosted git repository.

danny0405 pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git.


    omit 551e3f5  [CALCITE-4073] Add a new component RexNormalize for more effect rex nodes normalization (part1)
     new a4fa054  [CALCITE-4073] Add a new component RexNormalize for more effect rex nodes normalization (part1)

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (551e3f5)
            \
             N -- N -- N   refs/heads/master (a4fa054)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../main/java/org/apache/calcite/rex/RexCall.java  | 11 ++---
 .../java/org/apache/calcite/rex/RexNormalize.java  | 56 ++++++++++++++++++++--
 .../org/apache/calcite/test/RelMetadataTest.java   |  8 +++-
 3 files changed, 63 insertions(+), 12 deletions(-)


[calcite] 01/01: [CALCITE-4073] Add a new component RexNormalize for more effect rex nodes normalization (part1)

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

danny0405 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git

commit a4fa05458840cfdd93fb5cba16d102b06197539d
Author: yuzhao.cyz <yu...@gmail.com>
AuthorDate: Tue Jul 14 15:38:58 2020 +0800

    [CALCITE-4073] Add a new component RexNormalize for more effect rex nodes normalization (part1)
    
    * Add a new component named RexNormalize
    * By default, we only normalize the RexCalls during planning phrase, and
    there is no way to normalize it when constructing the calls now
    * Recover the plan diffs
    * Changes the RexCall#equals to be semantic equivalent, which i think is
    not a good design, we should promote to have a digest abstraction just
    like RelNode
---
 .../main/java/org/apache/calcite/rex/RexCall.java  | 107 +++--------
 .../main/java/org/apache/calcite/rex/RexNode.java  |  14 --
 .../java/org/apache/calcite/rex/RexNormalize.java  | 196 +++++++++++++++++++++
 .../main/java/org/apache/calcite/sql/SqlKind.java  |   6 +-
 .../java/org/apache/calcite/sql/SqlOperator.java   |  11 ++
 .../org/apache/calcite/sql/type/SqlTypeUtil.java   |  28 +++
 .../calcite/rex/RexCallNormalizationTest.java      |  85 ---------
 .../org/apache/calcite/rex/RexNormalizeTest.java   | 141 +++++++++++++++
 .../org/apache/calcite/test/JdbcAdapterTest.java   |   4 +-
 .../java/org/apache/calcite/test/LatticeTest.java  |   6 +-
 .../org/apache/calcite/test/RelBuilderTest.java    |   4 +-
 .../org/apache/calcite/test/RelMetadataTest.java   |   8 +-
 .../test/enumerable/EnumerableCorrelateTest.java   |   4 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml    |  74 ++++----
 .../apache/calcite/test/SqlToRelConverterTest.xml  |  24 +--
 .../org/apache/calcite/test/TopDownOptTest.xml     |   4 +-
 core/src/test/resources/sql/blank.iq               |   2 +-
 core/src/test/resources/sql/sub-query.iq           |   4 +-
 18 files changed, 470 insertions(+), 252 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rex/RexCall.java b/core/src/main/java/org/apache/calcite/rex/RexCall.java
index e604820..107d6e2 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCall.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCall.java
@@ -18,12 +18,11 @@ package org.apache.calcite.rex;
 
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSyntax;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 
@@ -58,11 +57,16 @@ public class RexCall extends RexNode {
   public final ImmutableList<RexNode> operands;
   public final RelDataType type;
   public final int nodeCount;
+
   /**
    * Cache of hash code.
    */
   protected int hash = 0;
 
+  /**
+   * Cache of normalized variables used for #equals and #hashCode.
+   */
+  private Pair<SqlOperator, List<RexNode>> normalized;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -71,11 +75,8 @@ public class RexCall extends RexNode {
       SqlOperator op,
       List<? extends RexNode> operands) {
     this.type = Objects.requireNonNull(type, "type");
-    Objects.requireNonNull(op, "operator");
-    final Pair<SqlOperator, ImmutableList<RexNode>> normalized =
-        normalize(op, operands);
-    this.op = normalized.left;
-    this.operands = normalized.right;
+    this.op = Objects.requireNonNull(op, "operator");
+    this.operands = ImmutableList.copyOf(operands);
     this.nodeCount = RexUtil.nodeCount(1, this.operands);
     assert op.getKind() != null : op;
     assert op.validRexOperands(operands.size(), Litmus.THROW) : this;
@@ -119,7 +120,7 @@ public class RexCall extends RexNode {
         RexNode otherArg = operands.get(1 - i);
         if ((!(otherArg instanceof RexLiteral)
             || ((RexLiteral) otherArg).digestIncludesType() == RexDigestIncludeType.NO_TYPE)
-            && equalSansNullability(operand.getType(), otherArg.getType())) {
+            && SqlTypeUtil.equalSansNullability(operand.getType(), otherArg.getType())) {
           includeType = RexDigestIncludeType.NO_TYPE;
         }
       }
@@ -139,81 +140,6 @@ public class RexCall extends RexNode {
     }
   }
 
-  private Pair<SqlOperator, ImmutableList<RexNode>> normalize(
-      SqlOperator operator,
-      List<? extends RexNode> operands) {
-    final ImmutableList<RexNode> oldOperands = ImmutableList.copyOf(operands);
-    final Pair<SqlOperator, ImmutableList<RexNode>> original = Pair.of(operator, oldOperands);
-    if (!needNormalize()) {
-      return original;
-    }
-    if (operands.size() != 2
-        || !operands.stream().allMatch(operand -> operand.getClass() == RexInputRef.class)) {
-      return original;
-    }
-    final RexInputRef operand0 = (RexInputRef) operands.get(0);
-    final RexInputRef operand1 = (RexInputRef) operands.get(1);
-    final SqlKind kind = operator.getKind();
-    if (operand0.getIndex() < operand1.getIndex()) {
-      return original;
-    }
-    // If arguments are the same, then we normalize < vs >
-    // '<' == 60, '>' == 62, so we prefer <.
-    if (operand0.getIndex() == operand1.getIndex()) {
-      if (kind.reverse().compareTo(kind) < 0) {
-        return Pair.of(SqlStdOperatorTable.reverse(operator), oldOperands);
-      } else {
-        return original;
-      }
-    }
-
-    if (SqlKind.SYMMETRICAL_SAME_ARG_TYPE.contains(kind)) {
-      final RelDataType firstType = operands.get(0).getType();
-      for (int i = 1; i < operands.size(); i++) {
-        if (!equalSansNullability(firstType, operands.get(i).getType())) {
-          // Arguments have different type, thus they must not be sorted
-          return original;
-        }
-      }
-      // fall through: order arguments below
-    } else if (!SqlKind.SYMMETRICAL.contains(kind)
-        && kind == kind.reverse()) {
-      // The operations have to be either symmetrical or reversible
-      // Nothing matched => we skip argument sorting
-      return original;
-    }
-    // $0=$1 is the same as $1=$0, so we make sure the digest is the same for them.
-
-    // When $1 > $0 is normalized, the operation needs to be flipped
-    // So we sort arguments first, then flip the sign.
-    final SqlOperator newOp = SqlStdOperatorTable.reverse(operator);
-    final ImmutableList<RexNode> newOperands = ImmutableList.of(operand1, operand0);
-    return Pair.of(newOp, newOperands);
-  }
-
-  /**
-   * This is a poorman's
-   * {@link org.apache.calcite.sql.type.SqlTypeUtil#equalSansNullability(RelDataTypeFactory, RelDataType, RelDataType)}
-   * <p>{@code SqlTypeUtil} requires {@link RelDataTypeFactory} which we haven't, so we assume that
-   * "not null" is represented in the type's digest as a trailing "NOT NULL" (case sensitive)
-   * @param a first type
-   * @param b second type
-   * @return true if the types are equal or the only difference is nullability
-   */
-  private static boolean equalSansNullability(RelDataType a, RelDataType b) {
-    String x = a.getFullTypeString();
-    String y = b.getFullTypeString();
-    if (x.length() < y.length()) {
-      String c = x;
-      x = y;
-      y = c;
-    }
-
-    return (x.length() == y.length()
-        || x.length() == y.length() + 9 && x.endsWith(" NOT NULL"))
-        && x.startsWith(y);
-  }
-
   protected @Nonnull String computeDigest(boolean withType) {
     final StringBuilder sb = new StringBuilder(op.getName());
     if ((operands.size() == 0)
@@ -318,6 +244,13 @@ public class RexCall extends RexNode {
     return new RexCall(type, op, operands);
   }
 
+  private Pair<SqlOperator, List<RexNode>> getNormalized() {
+    if (this.normalized == null) {
+      this.normalized = RexNormalize.normalize(this.op, this.operands);
+    }
+    return this.normalized;
+  }
+
   @Override public boolean equals(Object o) {
     if (this == o) {
       return true;
@@ -325,15 +258,17 @@ public class RexCall extends RexNode {
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
+    Pair<SqlOperator, List<RexNode>> x = getNormalized();
     RexCall rexCall = (RexCall) o;
-    return op.equals(rexCall.op)
-        && operands.equals(rexCall.operands)
+    Pair<SqlOperator, List<RexNode>> y = rexCall.getNormalized();
+    return x.left.equals(y.left)
+        && x.right.equals(y.right)
         && type.equals(rexCall.type);
   }
 
   @Override public int hashCode() {
     if (hash == 0) {
-      hash = Objects.hash(op, operands);
+      hash = RexNormalize.hashCode(this.op, this.operands);
     }
     return hash;
   }
diff --git a/core/src/main/java/org/apache/calcite/rex/RexNode.java b/core/src/main/java/org/apache/calcite/rex/RexNode.java
index 88332a2..8a42b9d 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexNode.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexNode.java
@@ -16,12 +16,9 @@
  */
 package org.apache.calcite.rex;
 
-import org.apache.calcite.config.CalciteSystemProperty;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlKind;
 
-import org.apiguardian.api.API;
-
 import java.util.Collection;
 
 /**
@@ -40,17 +37,6 @@ import java.util.Collection;
  */
 public abstract class RexNode {
 
-  /**
-   * The digest of {@code RexNode} is normalized by default, however, sometimes a non-normalized
-   * representation is required.
-   * This method enables subclasses to identify if normalization is required.
-   * @return true if the digest needs to be normalized
-   */
-  @API(since = "1.22", status = API.Status.EXPERIMENTAL)
-  protected static boolean needNormalize() {
-    return CalciteSystemProperty.ENABLE_REX_DIGEST_NORMALIZE.value();
-  }
-
   //~ Instance fields --------------------------------------------------------
 
   // Effectively final. Set in each sub-class constructor, and never re-set.
diff --git a/core/src/main/java/org/apache/calcite/rex/RexNormalize.java b/core/src/main/java/org/apache/calcite/rex/RexNormalize.java
new file mode 100644
index 0000000..fb7af07
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rex/RexNormalize.java
@@ -0,0 +1,196 @@
+/*
+ * 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.rex;
+
+import org.apache.calcite.config.CalciteSystemProperty;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.Pair;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apiguardian.api.API;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Context required to normalize a row-expression.
+ *
+ * <p>Currently, only simple normalization is supported, such as:
+ *
+ * <ul>
+ *   <li>$2 = $1 &rarr; $1 = $2</li>
+ *   <li>$2 &gt; $1 &rarr; $1 &lt; $2</li>
+ *   <li>1.23 = $1 &rarr; $1 = 1.23</li>
+ *   <li>OR(OR(udf($1), $2), $3) &rarr; OR($3, OR($2, udf($1)))</li>
+ * </ul>
+ *
+ * <p>In the future, this component may extend to support more normalization cases
+ * for general promotion. e.g. the strategy to decide which operand is more complex
+ * should be more smart.
+ *
+ * <p>There is no one normalization strategy that works for all cases, and no consensus about what
+ * the desired strategies should be. So by default, the normalization is disabled. We do force
+ * normalization when computing the digest of {@link RexCall}s during planner planning.
+ */
+public class RexNormalize {
+
+  private RexNormalize() {}
+
+  /**
+   * Normalizes the variables of a rex call.
+   *
+   * @param operator The operator
+   * @param operands The operands
+   *
+   * @return normalized variables of the call or the original
+   * if there is no need to normalize
+   */
+  @API(since = "1.24", status = API.Status.EXPERIMENTAL)
+  public static Pair<SqlOperator, List<RexNode>> normalize(
+      SqlOperator operator,
+      List<RexNode> operands) {
+    final Pair<SqlOperator, List<RexNode>> original = Pair.of(operator, operands);
+    if (!allowsNormalize() || operands.size() != 2) {
+      return original;
+    }
+
+    final RexNode operand0 = operands.get(0);
+    final RexNode operand1 = operands.get(1);
+
+    // If arguments are the same, then we normalize < vs >
+    // '<' == 60, '>' == 62, so we prefer <.
+    final SqlKind kind = operator.getKind();
+    final SqlKind reversedKind = kind.reverse();
+    final int x = reversedKind.compareTo(kind);
+    if (x < 0) {
+      return Pair.of(
+          SqlStdOperatorTable.reverse(operator),
+          ImmutableList.of(operand1, operand0));
+    }
+    if (x > 0) {
+      return original;
+    }
+
+    if (!isSymmetricalCall(operator, operand0, operand1)) {
+      return original;
+    }
+
+    if (reorderOperands(operand0, operand1) < 0) {
+      // $0=$1 is the same as $1=$0, so we make sure the digest is the same for them.
+
+      // When $1 > $0 is normalized, the operation needs to be flipped
+      // so we sort arguments first, then flip the sign.
+      return Pair.of(
+          SqlStdOperatorTable.reverse(operator),
+          ImmutableList.of(operand1, operand0));
+    }
+    return original;
+  }
+
+  /**
+   * Computes the hashCode of a rex call. We ignore the operands sequence when the call is
+   * symmetrical.
+   *
+   * <p>Note that the logic to decide whether operands need reordering
+   * should be strictly same with {@link #normalize}.
+   */
+  public static int hashCode(
+      SqlOperator operator,
+      List<RexNode> operands) {
+    if (!allowsNormalize() || operands.size() != 2) {
+      return Objects.hash(operator, operands);
+    }
+    // If arguments are the same, then we normalize < vs >
+    // '<' == 60, '>' == 62, so we prefer <.
+    final SqlKind kind = operator.getKind();
+    final SqlKind reversedKind = kind.reverse();
+    final int x = reversedKind.compareTo(kind);
+    if (x < 0) {
+      return Objects.hash(
+          SqlStdOperatorTable.reverse(operator),
+          Arrays.asList(operands.get(1), operands.get(0)));
+    }
+    if (isSymmetricalCall(operator, operands.get(0), operands.get(1))) {
+      return Objects.hash(operator, unorderedHash(operands));
+    }
+    return Objects.hash(operator, operands);
+  }
+
+  /**
+   * Compares two operands to see which one should be normalized to be in front of the other.
+   *
+   * <p>We can always use the #hashCode to reorder the operands, do it as a fallback to keep
+   * good readability.
+   *
+   * @param operand0  First operand
+   * @param operand1  Second operand
+   *
+   * @return non-negative (>=0) if {@code operand0} should be in the front,
+   * negative if {@code operand1} should be in the front
+   */
+  private static int reorderOperands(RexNode operand0, RexNode operand1) {
+    // Reorder the operands based on the SqlKind enumeration sequence,
+    // smaller is in the behind, e.g. the literal is behind of input ref and AND, OR.
+    int x = operand0.getKind().compareTo(operand1.getKind());
+    // If the operands are same kind, use the hashcode to reorder.
+    // Note: the RexInputRef's hash code is its index.
+    return x != 0 ? x : operand1.hashCode() - operand0.hashCode();
+  }
+
+  /** Returns whether a call is symmetrical. **/
+  private static boolean isSymmetricalCall(
+      SqlOperator operator,
+      RexNode operand0,
+      RexNode operand1) {
+    return operator.isSymmetrical()
+        || SqlKind.SYMMETRICAL_SAME_ARG_TYPE.contains(operator.getKind())
+            && SqlTypeUtil.equalSansNullability(operand0.getType(), operand1.getType());
+  }
+
+  /** Compute a hash that is symmetric in its arguments - that is a hash
+   *  where the order of appearance of elements does not matter.
+   *  This is useful for hashing symmetrical rex calls, for example.
+   */
+  private static int unorderedHash(List<?> xs) {
+    int a = 0;
+    int b = 0;
+    int c = 1;
+    for (Object x : xs) {
+      int h = x.hashCode();
+      a += h;
+      b ^= h;
+      if (h != 0) {
+        c *= h;
+      }
+    }
+    return (a * 17 + b) * 17 + c;
+  }
+
+  /**
+   * The digest of {@code RexNode} is normalized by default.
+   *
+   * @return true if the digest allows normalization
+   */
+  private static boolean allowsNormalize() {
+    return CalciteSystemProperty.ENABLE_REX_DIGEST_NORMALIZE.value();
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index f1cfb08..247c985 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -1387,9 +1387,9 @@ public enum SqlKind {
   public static final Set<SqlKind> SIMPLE_BINARY_OPS;
 
   static {
-    EnumSet<SqlKind> kinds = EnumSet.of(SqlKind.PLUS, SqlKind.MINUS, SqlKind.TIMES, SqlKind.DIVIDE);
-    kinds.addAll(SqlKind.COMPARISON);
-    kinds.remove(SqlKind.IN);
+    EnumSet<SqlKind> kinds = EnumSet.copyOf(SqlKind.BINARY_ARITHMETIC);
+    kinds.remove(SqlKind.MOD);
+    kinds.addAll(SqlKind.BINARY_COMPARISON);
     SIMPLE_BINARY_OPS = Sets.immutableEnumSet(kinds);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
index 199681c..14d9edd 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
@@ -942,6 +942,17 @@ public abstract class SqlOperator {
   }
 
   /**
+   * Returns whether a call to this operator is not sensitive to the operands input order.
+   * An operator is symmetrical if the call returns the same result when
+   * the operands are shuffled.
+   *
+   * <p>By default, returns true for {@link SqlKind#SYMMETRICAL}.
+   */
+  public boolean isSymmetrical() {
+    return SqlKind.SYMMETRICAL.contains(kind);
+  }
+
+  /**
    * Returns whether it is unsafe to cache query plans referencing this
    * operator; false is assumed by default
    */
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
index 9a093c9..bbad1ca 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
@@ -1188,6 +1188,34 @@ public abstract class SqlTypeUtil {
   }
 
   /**
+   * This is a poorman's
+   * {@link #equalSansNullability(RelDataTypeFactory, RelDataType, RelDataType)}.
+   *
+   * <p>We assume that "not null" is represented in the type's digest
+   * as a trailing "NOT NULL" (case sensitive).
+   *
+   * <p>If you got a type factory, {@link #equalSansNullability(RelDataTypeFactory, RelDataType, RelDataType)}
+   * is preferred.
+   *
+   * @param type1 First type
+   * @param type2 Second type
+   * @return true if the types are equal or the only difference is nullability
+   */
+  public static boolean equalSansNullability(RelDataType type1, RelDataType type2) {
+    String x = type1.getFullTypeString();
+    String y = type2.getFullTypeString();
+    if (x.length() < y.length()) {
+      String c = x;
+      x = y;
+      y = c;
+    }
+
+    return (x.length() == y.length()
+        || x.length() == y.length() + 9 && x.endsWith(" NOT NULL"))
+        && x.startsWith(y);
+  }
+
+  /**
    * Returns whether two collection types are equal, ignoring nullability.
    *
    * <p>They need not come from the same factory.
diff --git a/core/src/test/java/org/apache/calcite/rex/RexCallNormalizationTest.java b/core/src/test/java/org/apache/calcite/rex/RexCallNormalizationTest.java
deleted file mode 100644
index 38d08c1..0000000
--- a/core/src/test/java/org/apache/calcite/rex/RexCallNormalizationTest.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.rex;
-
-import org.junit.jupiter.api.Test;
-
-class RexCallNormalizationTest extends RexProgramTestBase {
-  @Test void digestIsNormalized() {
-    final RexNode node = and(or(input(tBool(), 1), input(tBool(), 0)), input(tBool(), 0));
-    checkDigest(node, "AND(OR($0, $1), $0)");
-
-    checkDigest(eq(input(tVarchar(), 0), literal("0123456789012345")),
-        "=($0, '0123456789012345')");
-    checkDigest(eq(input(tVarchar(), 0), literal("01")), "=($0, '01')");
-  }
-
-  @Test void reversibleSameArgOpsNormalizedToLess() {
-    checkDigest(lt(input(tBool(), 0), input(tBool(), 0)), "<($0, $0)");
-    checkDigest(gt(input(tBool(), 0), input(tBool(), 0)), "<($0, $0)");
-    checkDigest(le(input(tBool(), 0), input(tBool(), 0)), "<=($0, $0)");
-    checkDigest(ge(input(tBool(), 0), input(tBool(), 0)), "<=($0, $0)");
-  }
-
-  @Test void reversibleDifferentArgTypesShouldNotBeShuffled() {
-    checkDigest(plus(input(tSmallInt(), 0), input(tInt(), 1)), "+($0, $1)");
-    checkDigest(plus(input(tInt(), 0), input(tSmallInt(), 1)), "+($0, $1)");
-    checkDigest(mul(input(tSmallInt(), 0), input(tInt(), 1)), "*($0, $1)");
-    checkDigest(mul(input(tInt(), 0), input(tSmallInt(), 1)), "*($0, $1)");
-  }
-
-  @Test void reversibleDifferentNullabilityArgsAreNormalized() {
-    checkDigest(plus(input(tInt(false), 1), input(tInt(), 0)), "+($0, $1)");
-    checkDigest(plus(input(tInt(), 1), input(tInt(false), 0)), "+($0, $1)");
-    checkDigest(mul(input(tInt(false), 1), input(tInt(), 0)), "*($0, $1)");
-    checkDigest(mul(input(tInt(), 1), input(tInt(false), 0)), "*($0, $1)");
-  }
-
-  @Test void symmetricalDifferentArgOps() {
-    for (int i = 0; i < 2; i++) {
-      int j = 1 - i;
-      checkDigest(eq(input(tBool(), i), input(tBool(), j)), "=($0, $1)");
-      checkDigest(ne(input(tBool(), i), input(tBool(), j)), "<>($0, $1)");
-    }
-  }
-
-  @Test void reversibleDifferentArgOps() {
-    for (int i = 0; i < 2; i++) {
-      int j = 1 - i;
-      checkDigest(
-          lt(input(tBool(), i), input(tBool(), j)),
-          i < j
-              ? "<($0, $1)"
-              : ">($0, $1)");
-      checkDigest(
-          le(input(tBool(), i), input(tBool(), j)),
-          i < j
-              ? "<=($0, $1)"
-              : ">=($0, $1)");
-      checkDigest(
-          gt(input(tBool(), i), input(tBool(), j)),
-          i < j
-              ? ">($0, $1)"
-              : "<($0, $1)");
-      checkDigest(
-          ge(input(tBool(), i), input(tBool(), j)),
-          i < j
-              ? ">=($0, $1)"
-              : "<=($0, $1)");
-    }
-  }
-}
diff --git a/core/src/test/java/org/apache/calcite/rex/RexNormalizeTest.java b/core/src/test/java/org/apache/calcite/rex/RexNormalizeTest.java
new file mode 100644
index 0000000..03bb8ee
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/rex/RexNormalizeTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.rex;
+
+import org.hamcrest.CoreMatchers;
+import org.junit.jupiter.api.Test;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+/** Test cases for {@link RexNormalize}. */
+class RexNormalizeTest extends RexProgramTestBase {
+
+  @Test void digestIsNormalized() {
+    assertNodeEquals(
+        and(or(vBool(1), vBool(0)), vBool(0)),
+        and(vBool(0), or(vBool(0), vBool(1))));
+
+    assertNodeEquals(
+        and(or(vBool(1), vBool(0)), vBool(0)),
+        and(or(vBool(0), vBool(1)), vBool(0)));
+
+    assertNodeEquals(
+        eq(vVarchar(0), literal("0123456789012345")),
+        eq(literal("0123456789012345"), vVarchar(0)));
+
+    assertNodeEquals(
+        eq(vVarchar(0), literal("01")),
+        eq(literal("01"), vVarchar(0)));
+  }
+
+  @Test void reversibleNormalizedToLess() {
+    // Same type operands.
+    assertNodeEquals(
+        lt(vBool(0), vBool(0)),
+        gt(vBool(0), vBool(0)));
+
+    assertNodeEquals(
+        le(vBool(0), vBool(0)),
+        ge(vBool(0), vBool(0)));
+
+    // Different type operands.
+    assertNodeEquals(
+        lt(vSmallInt(0), vInt(1)),
+        gt(vInt(1), vSmallInt(0)));
+
+    assertNodeEquals(
+        le(vSmallInt(0), vInt(1)),
+        ge(vInt(1), vSmallInt(0)));
+  }
+
+  @Test void reversibleDifferentArgTypesShouldNotBeShuffled() {
+    assertNodeNotEqual(
+        plus(vSmallInt(1), vInt(0)),
+        plus(vInt(0), vSmallInt(1)));
+
+    assertNodeNotEqual(
+        mul(vSmallInt(0), vInt(1)),
+        mul(vInt(1), vSmallInt(0)));
+  }
+
+  @Test void reversibleDifferentNullabilityArgsAreNormalized() {
+    assertNodeEquals(
+        plus(vIntNotNull(0), vInt(1)),
+        plus(vInt(1), vIntNotNull(0)));
+
+    assertNodeEquals(
+        mul(vIntNotNull(1), vInt(0)),
+        mul(vInt(0), vIntNotNull(1)));
+  }
+
+  @Test void symmetricalDifferentArgOps() {
+    assertNodeEquals(
+        eq(vBool(0), vBool(1)),
+        eq(vBool(1), vBool(0)));
+
+    assertNodeEquals(
+        ne(vBool(0), vBool(1)),
+        ne(vBool(1), vBool(0)));
+  }
+
+  @Test void reversibleDifferentArgOps() {
+    assertNodeNotEqual(
+        lt(vBool(0), vBool(1)),
+        lt(vBool(1), vBool(0)));
+
+    assertNodeNotEqual(
+        le(vBool(0), vBool(1)),
+        le(vBool(1), vBool(0)));
+
+    assertNodeNotEqual(
+        gt(vBool(0), vBool(1)),
+        gt(vBool(1), vBool(0)));
+
+    assertNodeNotEqual(
+        ge(vBool(0), vBool(1)),
+        ge(vBool(1), vBool(0)));
+  }
+
+  /** Asserts two rex nodes are equal. */
+  private static void assertNodeEquals(RexNode node1, RexNode node2) {
+    final String reason = getReason(node1, node2, true);
+    assertThat(reason, node1, equalTo(node2));
+    assertThat(reason, node1.hashCode(), equalTo(node2.hashCode()));
+  }
+
+  /** Asserts two rex nodes are not equal. */
+  private static void assertNodeNotEqual(RexNode node1, RexNode node2) {
+    final String reason = getReason(node1, node2, false);
+    assertThat(reason, node1, CoreMatchers.not(equalTo(node2)));
+    assertThat(reason, node1.hashCode(), CoreMatchers.not(equalTo(node2.hashCode())));
+  }
+
+  /** Returns the assertion reason. */
+  private static String getReason(RexNode node1, RexNode node2, boolean equal) {
+    StringBuilder reason = new StringBuilder("Rex nodes [");
+    reason.append(node1);
+    reason.append("] and [");
+    reason.append(node2);
+    reason.append("] expect to be ");
+    if (!equal) {
+      reason.append("not ");
+    }
+    reason.append("equal");
+    return reason.toString();
+  }
+}
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
index 2b32b52..67b3938 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
@@ -252,7 +252,7 @@ class JdbcAdapterTest {
             + "on s.losal <= e.sal and s.hisal >= e.sal")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
             + "  JdbcProject(EMPNO=[$0], ENAME=[$1], GRADE=[$3])\n"
-            + "    JdbcJoin(condition=[AND(>=($2, $4), <=($2, $5))], joinType=[inner])\n"
+            + "    JdbcJoin(condition=[AND(<=($4, $2), >=($5, $2))], joinType=[inner])\n"
             + "      JdbcProject(EMPNO=[$0], ENAME=[$1], SAL=[$5])\n"
             + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
             + "      JdbcTableScan(table=[[SCOTT, SALGRADE]])")
@@ -295,7 +295,7 @@ class JdbcAdapterTest {
             + "e.mgr = m.empno and (e.sal > m.sal or m.hiredate > e.hiredate)")
         .explainContains("PLAN=JdbcToEnumerableConverter\n"
             + "  JdbcProject(EMPNO=[$0], ENAME=[$1], EMPNO0=[$0], ENAME0=[$1])\n"
-            + "    JdbcJoin(condition=[AND(=($2, $5), OR(>($4, $7), <($3, $6)))], joinType=[inner])\n"
+            + "    JdbcJoin(condition=[AND(=($2, $5), OR(>($4, $7), >($6, $3)))], joinType=[inner])\n"
             + "      JdbcProject(EMPNO=[$0], ENAME=[$1], MGR=[$3], HIREDATE=[$4], SAL=[$5])\n"
             + "        JdbcTableScan(table=[[SCOTT, EMP]])\n"
             + "      JdbcProject(EMPNO=[$0], HIREDATE=[$4], SAL=[$5])\n"
diff --git a/core/src/test/java/org/apache/calcite/test/LatticeTest.java b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
index 9d672ab..987081a 100644
--- a/core/src/test/java/org/apache/calcite/test/LatticeTest.java
+++ b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
@@ -234,9 +234,9 @@ class LatticeTest {
           final Map.Entry<String, CalciteSchema.LatticeEntry> entry =
               adhoc.unwrap(CalciteSchema.class).getLatticeMap().firstEntry();
           final Lattice lattice = entry.getValue().getLattice();
-          assertThat(lattice.firstColumn("S"), is(0));
-          assertThat(lattice.firstColumn("P"), is(8));
-          assertThat(lattice.firstColumn("T"), is(23));
+          assertThat(lattice.firstColumn("S"), is(10));
+          assertThat(lattice.firstColumn("P"), is(18));
+          assertThat(lattice.firstColumn("T"), is(0));
           assertThat(lattice.firstColumn("PC"), is(-1));
           assertThat(lattice.defaultMeasures.size(), is(1));
           assertThat(lattice.rootNode.descendants.size(), is(3));
diff --git a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
index d00228f..dd1463d 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -1978,7 +1978,7 @@ public class RelBuilderTest {
         + "  LogicalTableScan(table=[[scott, DEPT]])\n";
     final String expectedWithoutSimplify = ""
         + "LogicalJoin(condition=[OR(null:NULL, "
-        + "AND(=($7, 1), =($7, 2), =($7, $8)))], joinType=[inner])\n"
+        + "AND(=($7, 1), =($7, 2), =($8, $7)))], joinType=[inner])\n"
         + "  LogicalTableScan(table=[[scott, EMP]])\n"
         + "  LogicalTableScan(table=[[scott, DEPT]])\n";
     assertThat(f.apply(createBuilder()), hasTree(expected));
@@ -2151,7 +2151,7 @@ public class RelBuilderTest {
                     builder.field("e", "MGR")))
             .build();
     final String expected = ""
-        + "LogicalFilter(condition=[AND(=($7, $16), =($3, $8))])\n"
+        + "LogicalFilter(condition=[AND(=($7, $16), =($8, $3))])\n"
         + "  LogicalJoin(condition=[true], joinType=[inner])\n"
         + "    LogicalTableScan(table=[[scott, EMP]])\n"
         + "    LogicalJoin(condition=[true], joinType=[inner])\n"
diff --git a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
index 07e59bd..44c707d 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -2044,7 +2044,13 @@ public class RelMetadataTest extends SqlToRelTestBase {
         + "select empno, comm, deptno from emp where empno=1 and comm=4");
     final RelMetadataQuery mq = rel.getCluster().getMetadataQuery();
     assertThat(mq.getPulledUpPredicates(rel).pulledUpPredicates,
-        sortsAs("[=($0, 1), OR(AND(=($1, 2), =($2, 3)), =($1, 4))]"));
+        // Because the hashCode for
+        // OR(AND(=($1, 2), =($2, 3)) and
+        // OR(AND(=($2, 3), =($1, 2)) are the same, the result is flipped and not stable,
+        // but they both are correct.
+        CoreMatchers.anyOf(
+            sortsAs("[=($0, 1), OR(AND(=($1, 2), =($2, 3)), =($1, 4))]"),
+            sortsAs("[=($0, 1), OR(AND(=($2, 3), =($1, 2)), =($1, 4))]")));
 
   }
 
diff --git a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
index dce7015..1432d0e 100644
--- a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
+++ b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
@@ -99,7 +99,7 @@ class EnumerableCorrelateTest {
             + "  EnumerableCorrelate(correlation=[$cor1], joinType=[inner], requiredColumns=[{0}])\n"
             + "    EnumerableAggregate(group=[{0}])\n"
             + "      EnumerableTableScan(table=[[s, depts]])\n"
-            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[$cor1], expr#6=[$t5.deptno], expr#7=[=($t6, $t1)], proj#0..2=[{exprs}], $condition=[$t7])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[$cor1], expr#6=[$t5.deptno], expr#7=[=($t1, $t6)], proj#0..2=[{exprs}], $condition=[$t7])\n"
             + "      EnumerableTableScan(table=[[s, emps]])")
         .returnsUnordered(
             "empid=100; name=Bill",
@@ -129,7 +129,7 @@ class EnumerableCorrelateTest {
             + "  EnumerableCorrelate(correlation=[$cor1], joinType=[inner], requiredColumns=[{0}])\n"
             + "    EnumerableAggregate(group=[{0}])\n"
             + "      EnumerableTableScan(table=[[s, depts]])\n"
-            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[$cor1], expr#6=[$t5.deptno], expr#7=[=($t6, $t1)], expr#8=[100], expr#9=[>($t0, $t8)], expr#10=[AND($t7, $t9)], proj#0..2=[{exprs}], $condition=[$t10])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[$cor1], expr#6=[$t5.deptno], expr#7=[=($t1, $t6)], expr#8=[100], expr#9=[>($t0, $t8)], expr#10=[AND($t7, $t9)], proj#0..2=[{exprs}], $condition=[$t10])\n"
             + "      EnumerableTableScan(table=[[s, emps]])")
         .returnsUnordered(
             "empid=110; name=Theodore",
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index e8c29b6..57fd8c4 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -638,7 +638,7 @@ LogicalProject(MGR=[$3])
         </Resource>
         <Resource name="planMid">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], EXPR$1=[OR(AND(IS NOT NULL($5), <>($2, 0)), AND(>($2, $3), null, <>($2, 0), IS NULL($5)))])
+LogicalProject(DEPTNO=[$0], EXPR$1=[OR(AND(IS NOT NULL($5), <>($2, 0)), AND(<($3, $2), null, <>($2, 0), IS NULL($5)))])
   LogicalJoin(condition=[=($1, $4)], joinType=[left])
     LogicalJoin(condition=[true], joinType=[inner])
       LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
@@ -652,7 +652,7 @@ LogicalProject(DEPTNO=[$0], EXPR$1=[OR(AND(IS NOT NULL($5), <>($2, 0)), AND(>($2
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(DEPTNO=[$0], EXPR$1=[OR(AND(IS NOT NULL($5), <>($2, 0)), AND(>($2, $3), null, <>($2, 0), IS NULL($5)))])
+LogicalProject(DEPTNO=[$0], EXPR$1=[OR(AND(IS NOT NULL($5), <>($2, 0)), AND(<($3, $2), null, <>($2, 0), IS NULL($5)))])
   LogicalJoin(condition=[=($1, $4)], joinType=[left])
     LogicalJoin(condition=[true], joinType=[inner])
       LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
@@ -1837,7 +1837,7 @@ group by case when e.sal < 11 then 11 else -1 * e.sal end]]>
 LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
   LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
     LogicalProject(EXPR$1=[CASE(<($9, 11), 11, *(-1, $9))])
-      LogicalJoin(condition=[=($0, $5)], joinType=[left])
+      LogicalJoin(condition=[=($5, $0)], joinType=[left])
         LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -1847,7 +1847,7 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
 LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
   LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
     LogicalProject(EXPR$1=[CASE($2, 11, $3)])
-      LogicalJoin(condition=[=($0, $1)], joinType=[left])
+      LogicalJoin(condition=[=($1, $0)], joinType=[left])
         LogicalProject(ENAME=[$0])
           LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
         LogicalProject(ENAME=[$1], <=[<($5, 11)], *=[*(-1, $5)])
@@ -1866,7 +1866,7 @@ group by case when e.sal < 11 then -1 * e.sal else e.sal end]]>
 LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
   LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
     LogicalProject(EXPR$1=[CASE(<($9, 11), *(-1, $9), $9)])
-      LogicalJoin(condition=[=($0, $5)], joinType=[left])
+      LogicalJoin(condition=[=($5, $0)], joinType=[left])
         LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -1876,7 +1876,7 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
 LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
   LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
     LogicalProject(EXPR$1=[$2])
-      LogicalJoin(condition=[=($0, $1)], joinType=[left])
+      LogicalJoin(condition=[=($1, $0)], joinType=[left])
         LogicalProject(ENAME=[$0])
           LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
         LogicalProject(ENAME=[$1], CASE=[CASE(<($5, 11), *(-1, $5), $5)])
@@ -1983,7 +1983,7 @@ group by case when e.sal < 11 then 11 else -1 * e.sal end]]>
 LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
   LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
     LogicalProject(EXPR$1=[CASE(<($9, 11), 11, *(-1, $9))])
-      LogicalJoin(condition=[=($0, $5)], joinType=[right])
+      LogicalJoin(condition=[=($5, $0)], joinType=[right])
         LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -1993,7 +1993,7 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
 LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
   LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
     LogicalProject(EXPR$1=[$2])
-      LogicalJoin(condition=[=($0, $1)], joinType=[right])
+      LogicalJoin(condition=[=($1, $0)], joinType=[right])
         LogicalProject(ENAME=[$0])
           LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
         LogicalProject(ENAME=[$1], CASE=[CASE(<($5, 11), 11, *(-1, $5))])
@@ -2012,7 +2012,7 @@ group by case when e.sal < 11 then -1 * e.sal else e.sal end]]>
 LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
   LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
     LogicalProject(EXPR$1=[CASE(<($9, 11), *(-1, $9), $9)])
-      LogicalJoin(condition=[=($0, $5)], joinType=[right])
+      LogicalJoin(condition=[=($5, $0)], joinType=[right])
         LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -2022,7 +2022,7 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
 LogicalProject(EXPR$0=[$1], EXPR$1=[$0])
   LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
     LogicalProject(EXPR$1=[$2])
-      LogicalJoin(condition=[=($0, $1)], joinType=[right])
+      LogicalJoin(condition=[=($1, $0)], joinType=[right])
         LogicalProject(ENAME=[$0])
           LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
         LogicalProject(ENAME=[$1], CASE=[CASE(<($5, 11), *(-1, $5), $5)])
@@ -3582,7 +3582,7 @@ join sales.emp e on e.deptno = d.deptno and d.deptno not in (4, 6)]]>
         <Resource name="planBefore">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], EMPNO0=[$9], ENAME0=[$10], JOB0=[$11], MGR0=[$12], HIREDATE0=[$13], SAL0=[$14], COMM0=[$15], DEPTNO0=[$16], SLACKER0=[$17])
-  LogicalJoin(condition=[AND(=($7, $16), NOT(OR(=($7, 4), =($7, 6))))], joinType=[inner])
+  LogicalJoin(condition=[AND(=($16, $7), NOT(OR(=($7, 4), =($7, 6))))], joinType=[inner])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -3590,7 +3590,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], EMPNO0=[$9], ENAME0=[$10], JOB0=[$11], MGR0=[$12], HIREDATE0=[$13], SAL0=[$14], COMM0=[$15], DEPTNO0=[$16], SLACKER0=[$17])
-  LogicalJoin(condition=[=($7, $16)], joinType=[inner])
+  LogicalJoin(condition=[=($16, $7)], joinType=[inner])
     LogicalFilter(condition=[AND(<>($7, 4), <>($7, 6))])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalFilter(condition=[AND(<>($7, 4), <>($7, 6))])
@@ -3750,10 +3750,10 @@ LogicalProject(SAL=[$5])
     LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$8])
       LogicalFilter(condition=[AND(=($7, 20), >($5, 1000))])
         LogicalTableScan(table=[[CATALOG, SALES, EMPNULLABLES]])
-    LogicalFilter(condition=[=($0, $1)])
+    LogicalFilter(condition=[=($1, $0)])
       LogicalAggregate(group=[{0, 1}])
         LogicalProject(SAL=[$5], SAL0=[$8])
-          LogicalJoin(condition=[OR(=($5, $8), =($8, 4))], joinType=[inner])
+          LogicalJoin(condition=[OR(=($8, $5), =($8, 4))], joinType=[inner])
             LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], SLACKER=[$8])
               LogicalFilter(condition=[AND(=($7, 20), >($5, 1000))])
                 LogicalTableScan(table=[[CATALOG, SALES, EMPNULLABLES]])
@@ -6414,7 +6414,7 @@ group by e.job]]>
             <![CDATA[
 LogicalAggregate(group=[{2}])
   LogicalJoin(condition=[AND(=($2, $11), =($9, $12))], joinType=[full])
-    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($0, $7)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, $0)])
       LogicalFilter(condition=[=($1, 'A')])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalProject(DEPTNO=[$0], NAME=[$1], $f2=[+($0, 5)])
@@ -6426,7 +6426,7 @@ LogicalAggregate(group=[{2}])
 LogicalAggregate(group=[{0}])
   LogicalJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[full])
     LogicalAggregate(group=[{2, 9}])
-      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($0, $7)])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, $0)])
         LogicalFilter(condition=[=($1, 'A')])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalAggregate(group=[{1, 2}])
@@ -6596,7 +6596,7 @@ group by e.job]]>
             <![CDATA[
 LogicalAggregate(group=[{2}])
   LogicalJoin(condition=[AND(=($2, $11), =($9, $12))], joinType=[left])
-    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($0, $7)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, $0)])
       LogicalFilter(condition=[=($1, 'A')])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalProject(DEPTNO=[$0], NAME=[$1], $f2=[+($0, 5)])
@@ -6608,7 +6608,7 @@ LogicalAggregate(group=[{2}])
 LogicalAggregate(group=[{0}])
   LogicalJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[left])
     LogicalAggregate(group=[{2, 9}])
-      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($0, $7)])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, $0)])
         LogicalFilter(condition=[=($1, 'A')])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalAggregate(group=[{1, 2}])
@@ -6629,7 +6629,7 @@ group by d.name]]>
             <![CDATA[
 LogicalAggregate(group=[{11}])
   LogicalJoin(condition=[AND(=($2, $11), =($9, $12))], joinType=[left])
-    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($0, $7)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, $0)])
       LogicalFilter(condition=[=($1, 'A')])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalProject(DEPTNO=[$0], NAME=[$1], $f2=[+($0, 5)])
@@ -6641,7 +6641,7 @@ LogicalAggregate(group=[{11}])
 LogicalAggregate(group=[{2}])
   LogicalJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[left])
     LogicalAggregate(group=[{2, 9}])
-      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($0, $7)])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, $0)])
         LogicalFilter(condition=[=($1, 'A')])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalAggregate(group=[{1, 2}])
@@ -6662,7 +6662,7 @@ group by e.job, d.name]]>
             <![CDATA[
 LogicalAggregate(group=[{2, 11}])
   LogicalJoin(condition=[AND(=($2, $11), =($9, $12))], joinType=[left])
-    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($0, $7)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, $0)])
       LogicalFilter(condition=[=($1, 'A')])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalProject(DEPTNO=[$0], NAME=[$1], $f2=[+($0, 5)])
@@ -6674,7 +6674,7 @@ LogicalAggregate(group=[{2, 11}])
 LogicalAggregate(group=[{0, 2}])
   LogicalJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[left])
     LogicalAggregate(group=[{2, 9}])
-      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($0, $7)])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, $0)])
         LogicalFilter(condition=[=($1, 'A')])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalAggregate(group=[{1, 2}])
@@ -6743,7 +6743,7 @@ join sales.dept as d on e.empno = d.deptno and e.deptno + e.empno = d.deptno + 5
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$10], NAME=[$11])
   LogicalJoin(condition=[AND(=($0, $10), =($9, $12))], joinType=[inner])
-    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($0, $7)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, $0)])
       LogicalFilter(condition=[=($0, 10)])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalProject(DEPTNO=[$0], NAME=[$1], $f2=[+($0, 5)])
@@ -6754,7 +6754,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
             <![CDATA[
 LogicalProject(EMPNO=[10], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[10], NAME=[$11])
   LogicalJoin(condition=[=($9, 15)], joinType=[inner])
-    LogicalProject(EMPNO=[10], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+(10, $7)])
+    LogicalProject(EMPNO=[10], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, 10)])
       LogicalFilter(condition=[=($0, 10)])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalProject(DEPTNO=[10], NAME=[$1], $f2=[15])
@@ -7868,13 +7868,13 @@ from emp]]>
             <![CDATA[
 LogicalProject($0=[$3], $1=[$4])
   LogicalWindow(window#0=[window(partition {1} order by [0] aggs [SUM($1), SUM($2)])])
-    LogicalProject(SAL=[$5], DEPTNO=[$7], $2=[+($5, $7)])
+    LogicalProject(SAL=[$5], DEPTNO=[$7], $2=[+($7, $5)])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
-LogicalProject(SUM1=[SUM($7) OVER (PARTITION BY $7 ORDER BY $5)], SUM2=[SUM(+($5, $7)) OVER (PARTITION BY $7 ORDER BY $5)])
+LogicalProject(SUM1=[SUM($7) OVER (PARTITION BY $7 ORDER BY $5)], SUM2=[SUM(+($7, $5)) OVER (PARTITION BY $7 ORDER BY $5)])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -7920,7 +7920,7 @@ group by e.job,d.name]]>
             <![CDATA[
 LogicalAggregate(group=[{2, 11}])
   LogicalJoin(condition=[AND(=($2, $11), =($9, $12))], joinType=[inner])
-    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($0, $7)])
+    LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, $0)])
       LogicalFilter(condition=[=($1, 'A')])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalProject(DEPTNO=[$0], NAME=[$1], $f2=[+($0, 5)])
@@ -7932,7 +7932,7 @@ LogicalAggregate(group=[{2, 11}])
 LogicalAggregate(group=[{0, 2}])
   LogicalJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[inner])
     LogicalAggregate(group=[{2, 9}])
-      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($0, $7)])
+      LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, $0)])
         LogicalFilter(condition=[=($1, 'A')])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalAggregate(group=[{1, 2}])
@@ -8466,7 +8466,7 @@ LogicalProject(EXPR$0=[1])
             <![CDATA[
 LogicalProject(EXPR$0=[1])
   LogicalProject(DEPTNO=[$9], NAME=[$10], EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO0=[$7], SLACKER=[$8])
-    LogicalJoin(condition=[=($7, $9)], joinType=[right])
+    LogicalJoin(condition=[=($9, $7)], joinType=[right])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
       LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
@@ -9742,7 +9742,7 @@ RIGHT JOIN sales.emp e ON e.deptno = d.deptno]]>
         <Resource name="planBefore">
             <![CDATA[
 LogicalProject(DEPTNO=[$9])
-  LogicalJoin(condition=[=($0, $9)], joinType=[right])
+  LogicalJoin(condition=[=($9, $0)], joinType=[right])
     LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -9786,7 +9786,7 @@ RIGHT JOIN sales.emp e ON e.deptno = d.deptno]]>
         <Resource name="planBefore">
             <![CDATA[
 LogicalProject(DEPTNO=[$9], NAME=[$1])
-  LogicalJoin(condition=[=($0, $9)], joinType=[right])
+  LogicalJoin(condition=[=($9, $0)], joinType=[right])
     LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -11289,7 +11289,7 @@ LogicalProject(DEPTNO=[$1])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(EMPNO=[$0], D=[CASE(=($9, 0), false, IS NULL(CASE(true, CAST($7):INTEGER, null:INTEGER)), null:BOOLEAN, IS NOT NULL($12), true, >($9, $10), null:BOOLEAN, false)])
+LogicalProject(EMPNO=[$0], D=[CASE(=($9, 0), false, IS NULL(CASE(true, CAST($7):INTEGER, null:INTEGER)), null:BOOLEAN, IS NOT NULL($12), true, <($10, $9), null:BOOLEAN, false)])
   LogicalJoin(condition=[=(CASE(true, CAST($7):INTEGER, null:INTEGER), $11)], joinType=[left])
     LogicalJoin(condition=[true], joinType=[inner])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
@@ -11337,7 +11337,7 @@ LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null:INTEGER)])
             <![CDATA[
 LogicalProject(EMPNO=[$0])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[<($0, CASE(OR(AND(IS NOT NULL($12), <>($9, 0)), AND(>($9, $10), null, <>($9, 0), IS NULL($12))), 10, AND(OR(IS NULL($12), =($9, 0)), OR(<=($9, $10), =($9, 0), IS NOT NULL($12))), 20, 30))])
+    LogicalFilter(condition=[<($0, CASE(OR(AND(IS NOT NULL($12), <>($9, 0)), AND(<($10, $9), null, <>($9, 0), IS NULL($12))), 10, AND(OR(IS NULL($12), =($9, 0)), OR(>=($10, $9), =($9, 0), IS NOT NULL($12))), 20, 30))])
       LogicalJoin(condition=[=($7, $11)], joinType=[left])
         LogicalJoin(condition=[true], joinType=[inner])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
@@ -11764,7 +11764,7 @@ LogicalProject(DEPTNO=[$0])
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(SAL=[$5])
-  LogicalFilter(condition=[OR(=($10, 0), IS NOT TRUE(OR(IS NOT NULL($13), >($10, $11))))])
+  LogicalFilter(condition=[OR(=($10, 0), IS NOT TRUE(OR(IS NOT NULL($13), <($11, $10))))])
     LogicalJoin(condition=[AND(=($0, $12), =($2, $14))], joinType=[left])
       LogicalJoin(condition=[=($2, $9)], joinType=[left])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
@@ -11779,7 +11779,7 @@ LogicalProject(SAL=[$5])
             <![CDATA[
 LogicalProject(SAL=[$5])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[OR(=($9, 0), IS NOT TRUE(OR(IS NOT NULL($12), >($9, $10))))])
+    LogicalFilter(condition=[OR(=($9, 0), IS NOT TRUE(OR(IS NOT NULL($12), <($10, $9))))])
       LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{2}])
         LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{2}])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
@@ -11856,7 +11856,7 @@ LogicalProject(EMPNO=[$1])
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[OR(=($9, 0), IS NOT TRUE(OR(IS NOT NULL($12), >($9, $10))))])
+    LogicalFilter(condition=[OR(=($9, 0), IS NOT TRUE(OR(IS NOT NULL($12), <($10, $9))))])
       LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{1}])
         LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{1}])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
@@ -11875,7 +11875,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  LogicalFilter(condition=[OR(=($10, 0), IS NOT TRUE(OR(IS NOT NULL($13), >($10, $11))))])
+  LogicalFilter(condition=[OR(=($10, 0), IS NOT TRUE(OR(IS NOT NULL($13), <($11, $10))))])
     LogicalJoin(condition=[AND(=($0, $12), =($1, $14))], joinType=[left])
       LogicalJoin(condition=[=($1, $9)], joinType=[left])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 1e81638..53ef390 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -104,7 +104,7 @@ LogicalAggregate(group=[{0, 1}])
             <![CDATA[
 LogicalProject(D=[$0], EXPR$1=[+($0, $1)])
   LogicalAggregate(group=[{0, 1}])
-    LogicalProject(D=[+($0, $7)], MGR=[$3])
+    LogicalProject(D=[+($7, $0)], MGR=[$3])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -1565,7 +1565,7 @@ order by sal + empno desc, sal * empno, sal + empno desc]]>
             <![CDATA[
 LogicalProject(EMPNO=[$0])
   LogicalSort(sort0=[$1], sort1=[$2], dir0=[DESC], dir1=[ASC])
-    LogicalProject(EMPNO=[$0], EXPR$1=[+($0, $5)], EXPR$2=[*($0, $5)])
+    LogicalProject(EMPNO=[$0], EXPR$1=[+($5, $0)], EXPR$2=[*($5, $0)])
       LogicalJoin(condition=[true], joinType=[inner])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
         LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
@@ -1879,8 +1879,8 @@ order by c + a]]>
             <![CDATA[
 LogicalProject(A=[$0], B=[$1], C=[$2], DEPTNO=[$3], NAME=[$4])
   LogicalSort(sort0=[$5], dir0=[ASC])
-    LogicalProject(A=[$0], B=[$1], C=[$2], DEPTNO=[$3], NAME=[$4], EXPR$5=[+($0, $2)])
-      LogicalJoin(condition=[=($2, $3)], joinType=[inner])
+    LogicalProject(A=[$0], B=[$1], C=[$2], DEPTNO=[$3], NAME=[$4], EXPR$5=[+($2, $0)])
+      LogicalJoin(condition=[=($3, $2)], joinType=[inner])
         LogicalProject(A=[$2], B=[$1], C=[$0])
           LogicalValues(tuples=[[{ 1, 2, 3 }]])
         LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
@@ -2317,7 +2317,7 @@ JOIN dept on dept.deptno = emp.deptno + 0]]>
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$10], NAME=[$11])
-  LogicalJoin(condition=[=($9, $10)], joinType=[inner])
+  LogicalJoin(condition=[=($10, $9)], joinType=[inner])
     LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f9=[+($7, 0)])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
@@ -2407,7 +2407,7 @@ LogicalProject(D2=[$0], D3=[$1])
                   LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
                 LogicalAggregate(group=[{0, 1}], agg#0=[MIN($2)])
                   LogicalProject(D4=[$0], D6=[$2], $f0=[true])
-                    LogicalFilter(condition=[=($0, $1)])
+                    LogicalFilter(condition=[=($1, $0)])
                       LogicalProject(D4=[+($0, 4)], D5=[+($0, 5)], D6=[+($0, 6)])
                         LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
@@ -2590,7 +2590,7 @@ from dept]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(NAME=[$1], EXPR$1=[OR(AND(IS NOT NULL($6), <>($2, 0)), AND(>($2, $3), null, <>($2, 0), IS NULL($6)))])
+LogicalProject(NAME=[$1], EXPR$1=[OR(AND(IS NOT NULL($6), <>($2, 0)), AND(<($3, $2), null, <>($2, 0), IS NULL($6)))])
   LogicalJoin(condition=[=($4, $5)], joinType=[left])
     LogicalProject(DEPTNO=[$0], NAME=[$1], $f0=[$2], $f1=[$3], DEPTNO0=[$0])
       LogicalJoin(condition=[true], joinType=[inner])
@@ -2612,7 +2612,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EMPNO=[$0], EXPR$1=[OR(=($9, 0), AND(>($9, $10), null, <>($9, 0), IS NULL($13)), AND(<>($9, 0), IS NULL($13), <=($9, $10)))])
+LogicalProject(EMPNO=[$0], EXPR$1=[OR(=($9, 0), AND(<($10, $9), null, <>($9, 0), IS NULL($13)), AND(<>($9, 0), IS NULL($13), >=($10, $9)))])
   LogicalJoin(condition=[=($11, $12)], joinType=[left])
     LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$9], $f1=[$10], DEPTNO0=[$7])
       LogicalJoin(condition=[true], joinType=[inner])
@@ -2756,7 +2756,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EMPNO=[$0], EXPR$1=[OR(=($9, 0), AND(>($9, $10), null, <>($9, 0), IS NULL($13)), AND(<>($9, 0), IS NULL($13), <=($9, $10)))])
+LogicalProject(EMPNO=[$0], EXPR$1=[OR(=($9, 0), AND(<($10, $9), null, <>($9, 0), IS NULL($13)), AND(<>($9, 0), IS NULL($13), >=($10, $9)))])
   LogicalJoin(condition=[=($11, $12)], joinType=[left])
     LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], $f0=[$9], $f1=[$10], DEPTNO0=[$7])
       LogicalJoin(condition=[true], joinType=[inner])
@@ -5759,7 +5759,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
   LogicalJoin(condition=[=($7, $11)], joinType=[inner])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalProject(DEPTNO=[$0], NAME=[$1], DEPTNO0=[$2])
-      LogicalJoin(condition=[>($0, $2)], joinType=[inner])
+      LogicalJoin(condition=[<($2, $0)], joinType=[inner])
         LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
         LogicalAggregate(group=[{0}])
           LogicalProject(DEPTNO=[$7])
@@ -5807,7 +5807,7 @@ LogicalProject(D2=[$0], D3=[$1])
                   LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
                 LogicalAggregate(group=[{0, 1}], agg#0=[MIN($2)])
                   LogicalProject(D4=[$0], D6=[$2], $f0=[true])
-                    LogicalFilter(condition=[=($0, $1)])
+                    LogicalFilter(condition=[=($1, $0)])
                       LogicalProject(D4=[+($0, 4)], D5=[+($0, 5)], D6=[+($0, 6)])
                         LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
@@ -5931,7 +5931,7 @@ LogicalProject(EMPNO=[$0])
           LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
       LogicalAggregate(group=[{0}], agg#0=[MIN($1)])
         LogicalProject(EMPNO0=[$9], $f0=[true])
-          LogicalJoin(condition=[>($0, $9)], joinType=[inner])
+          LogicalJoin(condition=[<($9, $0)], joinType=[inner])
             LogicalTableScan(table=[[CATALOG, SALES, EMP]])
             LogicalAggregate(group=[{0}])
               LogicalProject(EMPNO=[$0])
diff --git a/core/src/test/resources/org/apache/calcite/test/TopDownOptTest.xml b/core/src/test/resources/org/apache/calcite/test/TopDownOptTest.xml
index 0be66a5..9e6a6e9 100644
--- a/core/src/test/resources/org/apache/calcite/test/TopDownOptTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/TopDownOptTest.xml
@@ -579,7 +579,7 @@ on r.job=s.job and r.ename=s.ename]]>
     <Resource name="planBefore">
       <![CDATA[
 LogicalProject(ENAME=[$0], JOB=[$1], SAL=[$2], COMM=[$3], ENAME0=[$4], JOB0=[$5], EXPR$2=[$6])
-  LogicalJoin(condition=[AND(=($1, $5), =($0, $4))], joinType=[inner])
+  LogicalJoin(condition=[AND(=($5, $1), =($4, $0))], joinType=[inner])
     LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
     LogicalAggregate(group=[{0, 1}], EXPR$2=[MAX($2)])
       LogicalProject(ENAME=[$1], JOB=[$2], SAL=[$5])
@@ -606,7 +606,7 @@ on r.job=s.job and r.ename=s.ename]]>
     <Resource name="planBefore">
       <![CDATA[
 LogicalProject(ENAME=[$0], JOB=[$1], SAL=[$2], COMM=[$3], ENAME0=[$4], JOB0=[$5], MGR=[$6], EXPR$3=[$7])
-  LogicalJoin(condition=[AND(=($1, $5), =($0, $4))], joinType=[inner])
+  LogicalJoin(condition=[AND(=($5, $1), =($4, $0))], joinType=[inner])
     LogicalTableScan(table=[[CATALOG, SALES, BONUS]])
     LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[MAX($3)])
       LogicalProject(ENAME=[$1], JOB=[$2], MGR=[$3], SAL=[$5])
diff --git a/core/src/test/resources/sql/blank.iq b/core/src/test/resources/sql/blank.iq
index 32d3bcb..2d69aaf 100644
--- a/core/src/test/resources/sql/blank.iq
+++ b/core/src/test/resources/sql/blank.iq
@@ -89,7 +89,7 @@ insert into table2 values (NULL, 1), (2, 1);
 # Checked on Oracle
 !set lateDecorrelate true
 select i, j from table1 where table1.j NOT IN (select i from table2 where table1.i=table2.j);
-EnumerableCalc(expr#0..7=[{inputs}], expr#8=[0], expr#9=[=($t3, $t8)], expr#10=[IS NULL($t1)], expr#11=[IS NOT NULL($t7)], expr#12=[>($t3, $t4)], expr#13=[OR($t10, $t11, $t12)], expr#14=[IS NOT TRUE($t13)], expr#15=[OR($t9, $t14)], proj#0..1=[{exprs}], $condition=[$t15])
+EnumerableCalc(expr#0..7=[{inputs}], expr#8=[0], expr#9=[=($t3, $t8)], expr#10=[IS NULL($t1)], expr#11=[IS NOT NULL($t7)], expr#12=[<($t4, $t3)], expr#13=[OR($t10, $t11, $t12)], expr#14=[IS NOT TRUE($t13)], expr#15=[OR($t9, $t14)], proj#0..1=[{exprs}], $condition=[$t15])
   EnumerableHashJoin(condition=[AND(=($0, $6), =($1, $5))], joinType=[left])
     EnumerableHashJoin(condition=[=($0, $2)], joinType=[left])
       EnumerableTableScan(table=[[BLANK, TABLE1]])
diff --git a/core/src/test/resources/sql/sub-query.iq b/core/src/test/resources/sql/sub-query.iq
index 254738d..9a9df2f 100644
--- a/core/src/test/resources/sql/sub-query.iq
+++ b/core/src/test/resources/sql/sub-query.iq
@@ -32,7 +32,7 @@ where t1.x not in (select t2.x from t2);
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t1, $t5)], expr#7=[IS NULL($t4)], expr#8=[<=($t1, $t2)], expr#9=[IS NOT NULL($t0)], expr#10=[AND($t7, $t8, $t9)], expr#11=[OR($t6, $t10)], X=[$t0], $condition=[$t11])
+EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t1, $t5)], expr#7=[IS NULL($t4)], expr#8=[>=($t2, $t1)], expr#9=[IS NOT NULL($t0)], expr#10=[AND($t7, $t8, $t9)], expr#11=[OR($t6, $t10)], X=[$t0], $condition=[$t11])
   EnumerableHashJoin(condition=[=($0, $3)], joinType=[left])
     EnumerableNestedLoopJoin(condition=[true], joinType=[inner])
       EnumerableUnion(all=[true])
@@ -2048,7 +2048,7 @@ where sal + 100 not in (
 
 !ok
 EnumerableAggregate(group=[{}], C=[COUNT()])
-  EnumerableCalc(expr#0..9=[{inputs}], expr#10=[0], expr#11=[=($t4, $t10)], expr#12=[IS NULL($t2)], expr#13=[IS NOT NULL($t7)], expr#14=[>($t4, $t5)], expr#15=[OR($t12, $t13, $t14)], expr#16=[IS NOT TRUE($t15)], expr#17=[OR($t11, $t16)], proj#0..9=[{exprs}], $condition=[$t17])
+  EnumerableCalc(expr#0..9=[{inputs}], expr#10=[0], expr#11=[=($t4, $t10)], expr#12=[IS NULL($t2)], expr#13=[IS NOT NULL($t7)], expr#14=[<($t5, $t4)], expr#15=[OR($t12, $t13, $t14)], expr#16=[IS NOT TRUE($t15)], expr#17=[OR($t11, $t16)], proj#0..9=[{exprs}], $condition=[$t17])
     EnumerableHashJoin(condition=[AND(=($1, $8), =($2, $9))], joinType=[left])
       EnumerableHashJoin(condition=[=($1, $3)], joinType=[left])
         EnumerableCalc(expr#0..7=[{inputs}], proj#0..1=[{exprs}], SAL=[$t5])