You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2016/01/23 01:30:28 UTC

[10/50] [abbrv] calcite git commit: [CALCITE-955] Litmus (continuation-passing style for methods that check invariants)

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWindow.java b/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
index ced121c..a4150c4 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
@@ -29,6 +29,7 @@ import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ControlFlowException;
 import org.apache.calcite.util.ImmutableNullableList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -478,10 +479,11 @@ public class SqlWindow extends SqlCall {
    * definition.
    *
    * @param node The SqlWindow to compare to "this" window
-   * @param fail Whether to throw if not equal
+   * @param litmus What to do if an error is detected (nodes are not equal)
+   *
    * @return boolean true if all nodes in the subtree are equal
    */
-  @Override public boolean equalsDeep(SqlNode node, boolean fail) {
+  @Override public boolean equalsDeep(SqlNode node, Litmus litmus) {
     // This is the difference over super.equalsDeep.  It skips
     // operands[0] the declared name fo this window.  We only want
     // to check the window components.
@@ -489,8 +491,7 @@ public class SqlWindow extends SqlCall {
         || node instanceof SqlWindow
         && SqlNode.equalDeep(
             Util.skip(getOperandList()),
-            Util.skip(((SqlWindow) node).getOperandList()),
-          fail);
+            Util.skip(((SqlWindow) node).getOperandList()), litmus);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
index 547e0e7..9e8105a 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlInOperator.java
@@ -34,6 +34,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 
 import com.google.common.collect.ImmutableList;
 
@@ -84,6 +85,13 @@ public class SqlInOperator extends SqlBinaryOperator {
     return isNotIn;
   }
 
+  @Override public boolean validRexOperands(int count, Litmus litmus) {
+    if (count == 0) {
+      return litmus.fail("wrong operand count " + count + " for " + this);
+    }
+    return litmus.succeed();
+  }
+
   public RelDataType deriveType(
       SqlValidator validator,
       SqlValidatorScope scope,
@@ -100,7 +108,7 @@ public class SqlInOperator extends SqlBinaryOperator {
     // Derive type for RHS.
     if (right instanceof SqlNodeList) {
       // Handle the 'IN (expr, ...)' form.
-      List<RelDataType> rightTypeList = new ArrayList<RelDataType>();
+      List<RelDataType> rightTypeList = new ArrayList<>();
       SqlNodeList nodeList = (SqlNodeList) right;
       for (int i = 0; i < nodeList.size(); i++) {
         SqlNode node = nodeList.get(i);

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index 3311f50..ec26b7c 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -49,6 +49,7 @@ import org.apache.calcite.sql.type.SqlOperandCountRanges;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlModality;
+import org.apache.calcite.util.Litmus;
 
 /**
  * Implementation of {@link org.apache.calcite.sql.SqlOperatorTable} containing
@@ -613,6 +614,13 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
         public boolean argumentMustBeScalar(int ordinal) {
           return false;
         }
+
+        @Override public boolean validRexOperands(int count, Litmus litmus) {
+          if (count != 0) {
+            return litmus.fail("wrong operand count " + count + " for " + this);
+          }
+          return litmus.succeed();
+        }
       };
 
   public static final SqlPrefixOperator NOT =

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 3b402af..e4e5edf 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql2rel;
 
+import org.apache.calcite.avatica.util.Spaces;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptCluster;
@@ -146,6 +147,7 @@ import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.NumberUtil;
 import org.apache.calcite.util.Pair;
@@ -453,8 +455,8 @@ public class SqlToRelConverter {
         validator.getTypeFactory().createStructType(convertedFields);
 
     if (!RelOptUtil.equal("validated row type", validatedRowType,
-        "converted row type", convertedRowType, false)) {
-      throw Util.newInternal("Conversion to relational algebra failed to "
+        "converted row type", convertedRowType, Litmus.IGNORE)) {
+      throw new AssertionError("Conversion to relational algebra failed to "
           + "preserve datatypes:\n"
           + "validated type:\n"
           + validatedRowType.getFullTypeString()
@@ -1643,7 +1645,7 @@ public class SqlToRelConverter {
       NlsString unpadded = (NlsString) value;
       return rexBuilder.makeCharLiteral(
           new NlsString(
-              Util.rpad(unpadded.getValue(), type.getPrecision()),
+              Spaces.padRight(unpadded.getValue(), type.getPrecision()),
               unpadded.getCharsetName(),
               unpadded.getCollation()));
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/util/Litmus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Litmus.java b/core/src/main/java/org/apache/calcite/util/Litmus.java
index e7e2bbd..1a74e6c 100644
--- a/core/src/main/java/org/apache/calcite/util/Litmus.java
+++ b/core/src/main/java/org/apache/calcite/util/Litmus.java
@@ -23,13 +23,37 @@ public interface Litmus {
   /** Implementation of {@link org.apache.calcite.util.Litmus} that throws
    * an {@link java.lang.AssertionError} on failure. */
   Litmus THROW = new Litmus() {
-    @Override public boolean fail(String message) {
+    public boolean fail(String message) {
       throw new AssertionError(message);
     }
 
-    @Override public boolean succeed() {
+    public boolean succeed() {
       return true;
     }
+
+    public boolean check(boolean condition, Object info) {
+      if (condition) {
+        return succeed();
+      } else {
+        return fail(info == null ? null : info.toString());
+      }
+    }
+  };
+
+  /** Implementation of {@link org.apache.calcite.util.Litmus} that returns
+   * a status code but does not throw. */
+  Litmus IGNORE = new Litmus() {
+    public boolean fail(String message) {
+      return false;
+    }
+
+    public boolean succeed() {
+      return true;
+    }
+
+    public boolean check(boolean condition, Object info) {
+      return condition;
+    }
   };
 
   /** Called when test fails. Returns false or throws. */
@@ -37,6 +61,14 @@ public interface Litmus {
 
   /** Called when test succeeds. Returns true. */
   boolean succeed();
+
+  /** Checks a condition.
+   *
+   * <p>If the condition is true, calls {@link #succeed};
+   * if the condition is false, calls {@link #fail},
+   * converting {@code info} into a string message.
+   */
+  boolean check(boolean condition, Object info);
 }
 
 // End Litmus.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 4dc2fdc..ed39af0 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -1183,6 +1183,8 @@ public class Util {
    * @param s   string to be padded
    * @param len desired length
    * @return padded string
+   *
+   * @deprecated Use {@link Spaces#padRight(String, int)}
    */
   @Deprecated // to be removed before 2.0
   public static String rpad(String s, int len) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 314446e..1c67bc8 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -21,6 +21,7 @@ import org.apache.calcite.rel.RelVisitor;
 import org.apache.calcite.rel.externalize.RelXmlWriter;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
 
@@ -1457,13 +1458,13 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   /**
    * Visitor that checks that every {@link RelNode} in a tree is valid.
    *
-   * @see RelNode#isValid(boolean)
+   * @see RelNode#isValid(org.apache.calcite.util.Litmus)
    */
   public static class RelValidityChecker extends RelVisitor {
     int invalidCount;
 
     public void visit(RelNode node, int ordinal, RelNode parent) {
-      if (!node.isValid(true)) {
+      if (!node.isValid(Litmus.THROW)) {
         ++invalidCount;
       }
       super.visit(node, ordinal, parent);