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 2017/07/04 23:14:32 UTC

[3/3] calcite git commit: [CALCITE-1845] Quantified comparison predicates (SOME, ANY, ALL)

[CALCITE-1845] Quantified comparison predicates (SOME, ANY, ALL)

Add SqlKind.NOT_IN, and deprecate SqlInOperator.isNotIn()

Add a test case for [DRILL-5644] (already worked).


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/f4746523
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/f4746523
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/f4746523

Branch: refs/heads/master
Commit: f47465236b7650f2280092b708fa39062fe79ffd
Parents: ccaa700
Author: Julian Hyde <jh...@apache.org>
Authored: Fri Jun 16 00:51:44 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Jul 3 21:32:04 2017 -0700

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |  46 +-
 .../calcite/rel/rules/SubQueryRemoveRule.java   |  56 +-
 .../org/apache/calcite/rex/RexSubQuery.java     |  27 +-
 .../java/org/apache/calcite/sql/SqlKind.java    |  31 +-
 .../apache/calcite/sql/fun/SqlInOperator.java   |  29 +-
 .../calcite/sql/fun/SqlQuantifyOperator.java    |  62 ++
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  86 ++-
 .../calcite/sql/validate/SqlValidatorImpl.java  |   4 +-
 .../calcite/sql2rel/SqlToRelConverter.java      | 113 ++--
 .../calcite/sql/parser/SqlParserTest.java       |  48 ++
 .../apache/calcite/sql/test/SqlAdvisorTest.java |  12 +-
 .../org/apache/calcite/test/QuidemTest.java     |   5 +
 .../apache/calcite/test/RelOptRulesTest.java    |  12 +
 .../calcite/test/SqlToRelConverterTest.java     |  17 +
 .../apache/calcite/test/SqlValidatorTest.java   |  61 ++
 .../org/apache/calcite/test/RelOptRulesTest.xml |  80 +++
 .../calcite/test/SqlToRelConverterTest.xml      |  40 ++
 core/src/test/resources/sql/some.iq             | 577 +++++++++++++++++++
 core/src/test/resources/sql/sub-query.iq        |  18 +
 site/_docs/reference.md                         |  13 +
 20 files changed, 1253 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index 7ee3f1e..67d1a1b 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -2963,16 +2963,21 @@ List<Object> Expression2(ExprContext exprContext) :
                     checkNonQueryExpression(exprContext);
                 }
                 (
-                    <NOT> <IN> {
-                        op = SqlStdOperatorTable.NOT_IN;
-                        s.clear().add(this);
-                    }
+                    <NOT> <IN> { op = SqlStdOperatorTable.NOT_IN; }
                 |
-                    <IN> {
-                        op = SqlStdOperatorTable.IN;
-                        s.clear().add(this);
-                    }
+                    <IN> { op = SqlStdOperatorTable.IN; }
+                |
+                    { final SqlKind k; }
+                    k = comp()
+                    (
+                        <SOME> { op = SqlStdOperatorTable.some(k); }
+                    |
+                        <ANY> { op = SqlStdOperatorTable.some(k); }
+                    |
+                        <ALL> { op = SqlStdOperatorTable.all(k); }
+                    )
                 )
+                { s.clear().add(this); }
                 nodeList = ParenthesizedQueryOrCommaList(ExprContext.ACCEPT_NONCURSOR)
                 {
                     list.add(new SqlParserUtil.ToTreeListItem(op, s.pos()));
@@ -3084,6 +3089,31 @@ List<Object> Expression2(ExprContext exprContext) :
     )
 }
 
+/** Parses a comparison operator inside a SOME / ALL predicate. */
+SqlKind comp() :
+{
+}
+{
+    <LT> { return SqlKind.LESS_THAN; }
+|
+    <LE> { return SqlKind.LESS_THAN_OR_EQUAL; }
+|
+    <GT> { return SqlKind.GREATER_THAN; }
+|
+    <GE> { return SqlKind.GREATER_THAN_OR_EQUAL; }
+|
+    <EQ> { return SqlKind.EQUALS; }
+|
+    <NE> { return SqlKind.NOT_EQUALS; }
+|
+    <NE2> {
+        if (!this.conformance.isBangEqualAllowed()) {
+            throw new ParseException(RESOURCE.bangEqualNotAllowed().str());
+        }
+        return SqlKind.NOT_EQUALS;
+    }
+}
+
 /**
  * Parses a unary row expression, or a parenthesized expression of any
  * kind.

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
index bc886b6..f0f57a7 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
@@ -37,6 +37,7 @@ import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlQuantifyOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.tools.RelBuilder;
@@ -168,12 +169,59 @@ public abstract class SubQueryRemoveRule extends RelOptRule {
       builder.join(JoinRelType.LEFT, builder.literal(true), variablesSet);
       return field(builder, inputCount, offset);
 
+    case SOME:
+      // Most general case, where the left and right keys might have nulls, and
+      // caller requires 3-valued logic return.
+      //
+      // select e.deptno, e.deptno < some (select deptno from emp) as v
+      // from emp as e
+      //
+      // becomes
+      //
+      // select e.deptno,
+      //   case
+      //   when q.c = 0 then false // sub-query is empty
+      //   when (e.deptno < q.m) is true then true
+      //   when q.c > q.d then unknown // sub-query has at least one null
+      //   else e.deptno < q.m
+      //   end as v
+      // from emp as e
+      // cross join (
+      //   select max(deptno) as m, count(*) as c, count(deptno) as d
+      //   from emp) as q
+      //
+      final SqlQuantifyOperator op = (SqlQuantifyOperator) e.op;
+      builder.push(e.rel)
+          .aggregate(builder.groupKey(),
+              op.comparisonKind == SqlKind.GREATER_THAN
+                  || op.comparisonKind == SqlKind.GREATER_THAN_OR_EQUAL
+                  ? builder.min("m", builder.field(0))
+                  : builder.max("m", builder.field(0)),
+              builder.count(false, "c"),
+              builder.count(false, "d", builder.field(0)))
+          .as("q")
+          .join(JoinRelType.INNER);
+      return builder.call(SqlStdOperatorTable.CASE,
+          builder.call(SqlStdOperatorTable.EQUALS,
+              builder.field("q", "c"), builder.literal(0)),
+          builder.literal(false),
+          builder.call(SqlStdOperatorTable.IS_TRUE,
+              builder.call(RelOptUtil.op(op.comparisonKind, null),
+                  e.operands.get(0), builder.field("q", "m"))),
+          builder.literal(true),
+          builder.call(SqlStdOperatorTable.GREATER_THAN,
+              builder.field("q", "c"), builder.field("q", "d")),
+          builder.literal(null),
+          builder.call(RelOptUtil.op(op.comparisonKind, null),
+              e.operands.get(0), builder.field("q", "m")));
+
     case IN:
     case EXISTS:
       // Most general case, where the left and right keys might have nulls, and
       // caller requires 3-valued logic return.
       //
       // select e.deptno, e.deptno in (select deptno from emp)
+      // from emp as e
       //
       // becomes
       //
@@ -185,7 +233,7 @@ public abstract class SubQueryRemoveRule extends RelOptRule {
       //   when ct.ck < ct.c then null
       //   else false
       //   end
-      // from e
+      // from emp as e
       // left join (
       //   (select count(*) as c, count(deptno) as ck from emp) as ct
       //   cross join (select distinct deptno, true as i from emp)) as dt
@@ -198,7 +246,7 @@ public abstract class SubQueryRemoveRule extends RelOptRule {
       //   when dt.i is not null then true
       //   else false
       //   end
-      // from e
+      // from emp as e
       // left join (select distinct deptno, true as i from emp) as dt
       //   on e.deptno = dt.deptno
       //
@@ -206,7 +254,7 @@ public abstract class SubQueryRemoveRule extends RelOptRule {
       //
       // select e.deptno,
       //   dt.i is not null
-      // from e
+      // from emp as e
       // left join (select distinct deptno, true as i from emp) as dt
       //   on e.deptno = dt.deptno
       //
@@ -218,7 +266,7 @@ public abstract class SubQueryRemoveRule extends RelOptRule {
       //
       // select e.deptno,
       //   true
-      // from e
+      // from emp as e
       // inner join (select distinct deptno from emp) as dt
       //   on e.deptno = dt.deptno
       //

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/main/java/org/apache/calcite/rex/RexSubQuery.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSubQuery.java b/core/src/main/java/org/apache/calcite/rex/RexSubQuery.java
index d8fff4e..dc0d405 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSubQuery.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSubQuery.java
@@ -21,7 +21,9 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlQuantifyOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 
@@ -39,10 +41,29 @@ public class RexSubQuery extends RexCall {
       ImmutableList<RexNode> operands, RelNode rel) {
     super(type, op, operands);
     this.rel = rel;
+    this.digest = computeDigest(false);
   }
 
   /** Creates an IN sub-query. */
   public static RexSubQuery in(RelNode rel, ImmutableList<RexNode> nodes) {
+    final RelDataType type = type(rel, nodes);
+    return new RexSubQuery(type, SqlStdOperatorTable.IN, nodes, rel);
+  }
+
+  /** Creates a SOME sub-query.
+   *
+   * <p>There is no ALL. For {@code x comparison ALL (sub-query)} use instead
+   * {@code NOT (x inverse-comparison SOME (sub-query))}.
+   * If {@code comparison} is {@code >}
+   * then {@code negated-comparison} is {@code <=}, and so forth. */
+  public static RexSubQuery some(RelNode rel, ImmutableList<RexNode> nodes,
+      SqlQuantifyOperator op) {
+    assert op.kind == SqlKind.SOME;
+    final RelDataType type = type(rel, nodes);
+    return new RexSubQuery(type, op, nodes, rel);
+  }
+
+  static RelDataType type(RelNode rel, ImmutableList<RexNode> nodes) {
     assert rel.getRowType().getFieldCount() == nodes.size();
     final RelDataTypeFactory typeFactory = rel.getCluster().getTypeFactory();
     boolean nullable = false;
@@ -56,10 +77,8 @@ public class RexSubQuery extends RexCall {
         nullable = true;
       }
     }
-    final RelDataType type =
-        typeFactory.createTypeWithNullability(
-            typeFactory.createSqlType(SqlTypeName.BOOLEAN), nullable);
-    return new RexSubQuery(type, SqlStdOperatorTable.IN, nodes, rel);
+    return typeFactory.createTypeWithNullability(
+        typeFactory.createSqlType(SqlTypeName.BOOLEAN), nullable);
   }
 
   /** Creates an EXISTS sub-query. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
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 4147c9a..c4f3bf1 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -275,34 +275,43 @@ public enum SqlKind {
   IN,
 
   /**
+   * The "NOT IN" operator.
+   *
+   * <p>Only occurs in SqlNode trees. Is expanded to NOT(IN ...) before
+   * entering RelNode land.
+   */
+  NOT_IN("NOT IN"),
+
+  /**
    * The less-than operator, "&lt;".
    */
-  LESS_THAN,
+  LESS_THAN("<"),
 
   /**
    * The greater-than operator, "&gt;".
    */
-  GREATER_THAN,
+  GREATER_THAN(">"),
 
   /**
    * The less-than-or-equal operator, "&lt;=".
    */
-  LESS_THAN_OR_EQUAL,
+  LESS_THAN_OR_EQUAL("<="),
 
   /**
    * The greater-than-or-equal operator, "&gt;=".
    */
-  GREATER_THAN_OR_EQUAL,
+  GREATER_THAN_OR_EQUAL(">="),
 
   /**
    * The equals operator, "=".
    */
-  EQUALS,
+  EQUALS("="),
 
   /**
    * The not-equals operator, "&#33;=" or "&lt;&gt;".
+   * The latter is standard, and preferred.
    */
-  NOT_EQUALS,
+  NOT_EQUALS("<>"),
 
   /**
    * The is-distinct-from operator.
@@ -453,6 +462,16 @@ public enum SqlKind {
   EXISTS,
 
   /**
+   * The "SOME" quantification operator (also called "ANY").
+   */
+  SOME,
+
+  /**
+   * The "ALL" quantification operator.
+   */
+  ALL,
+
+  /**
    * The "VALUES" operator.
    */
   VALUES,

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/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 4639fbc..5eb5dff 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
@@ -51,39 +51,32 @@ import static org.apache.calcite.util.Static.RESOURCE;
 public class SqlInOperator extends SqlBinaryOperator {
   //~ Instance fields --------------------------------------------------------
 
-  /**
-   * If true the call represents 'NOT IN'.
-   */
-  private final boolean isNotIn;
-
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a SqlInOperator
+   * Creates a SqlInOperator.
    *
-   * @param isNotIn Whether this is the 'NOT IN' operator
+   * @param kind IN or NOT IN
    */
-  SqlInOperator(boolean isNotIn) {
-    super(
-        isNotIn ? "NOT IN" : "IN",
-        SqlKind.IN,
+  SqlInOperator(SqlKind kind) {
+    this(kind.sql, kind);
+    assert kind == SqlKind.IN || kind == SqlKind.NOT_IN;
+  }
+
+  protected SqlInOperator(String name, SqlKind kind) {
+    super(name, kind,
         32,
         true,
         ReturnTypes.BOOLEAN_NULLABLE,
         InferTypes.FIRST_KNOWN,
         null);
-    this.isNotIn = isNotIn;
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * Returns whether this is the 'NOT IN' operator
-   *
-   * @return whether this is the 'NOT IN' operator
-   */
+  @Deprecated // to be removed before 2.0
   public boolean isNotIn() {
-    return isNotIn;
+    return kind == SqlKind.NOT_IN;
   }
 
   @Override public boolean validRexOperands(int count, Litmus litmus) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/main/java/org/apache/calcite/sql/fun/SqlQuantifyOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlQuantifyOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlQuantifyOperator.java
new file mode 100644
index 0000000..6ee5347
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlQuantifyOperator.java
@@ -0,0 +1,62 @@
+/*
+ * 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.sql.fun;
+
+import org.apache.calcite.sql.SqlKind;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Definition of the SQL <code>ALL</code> and <code>SOME</code>operators.
+ *
+ * <p>Each is used in combination with a relational operator:
+ * <code>&lt;</code>, <code>&le;</code>,
+ * <code>&gt;</code>, <code>&ge;</code>,
+ * <code>=</code>, <code>&lt;&gt;</code>.
+ *
+ * <p><code>ANY</code> is a synonym for <code>SOME</code>.
+ */
+public class SqlQuantifyOperator extends SqlInOperator {
+  //~ Instance fields --------------------------------------------------------
+
+  public final SqlKind comparisonKind;
+
+  //~ Constructors -----------------------------------------------------------
+
+  /**
+   * Creates a SqlQuantifyOperator.
+   *
+   * @param kind Either ALL or SOME
+   * @param comparisonKind Either <code>&lt;</code>, <code>&le;</code>,
+   *   <code>&gt;</code>, <code>&ge;</code>,
+   *   <code>=</code> or <code>&lt;&gt;</code>.
+   */
+  SqlQuantifyOperator(SqlKind kind, SqlKind comparisonKind) {
+    super(comparisonKind.sql + " " + kind, kind);
+    this.comparisonKind = Preconditions.checkNotNull(comparisonKind);
+    Preconditions.checkArgument(comparisonKind == SqlKind.EQUALS
+        || comparisonKind == SqlKind.NOT_EQUALS
+        || comparisonKind == SqlKind.LESS_THAN_OR_EQUAL
+        || comparisonKind == SqlKind.LESS_THAN
+        || comparisonKind == SqlKind.GREATER_THAN_OR_EQUAL
+        || comparisonKind == SqlKind.GREATER_THAN);
+    Preconditions.checkArgument(kind == SqlKind.SOME
+        || kind == SqlKind.ALL);
+  }
+}
+
+// End SqlQuantifyOperator.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/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 13b94b9..2ba8b25 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
@@ -371,14 +371,57 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * <code>IN</code> operator tests for a value's membership in a sub-query or
    * a list of values.
    */
-  public static final SqlBinaryOperator IN = new SqlInOperator(false);
+  public static final SqlBinaryOperator IN = new SqlInOperator(SqlKind.IN);
 
   /**
    * <code>NOT IN</code> operator tests for a value's membership in a sub-query
    * or a list of values.
    */
   public static final SqlBinaryOperator NOT_IN =
-      new SqlInOperator(true);
+      new SqlInOperator(SqlKind.NOT_IN);
+
+  /**
+   * The <code>&lt; SOME</code> operator (synonymous with
+   * <code>&lt; ANY</code>).
+   */
+  public static final SqlQuantifyOperator SOME_LT =
+      new SqlQuantifyOperator(SqlKind.SOME, SqlKind.LESS_THAN);
+
+  public static final SqlQuantifyOperator SOME_LE =
+      new SqlQuantifyOperator(SqlKind.SOME, SqlKind.LESS_THAN_OR_EQUAL);
+
+  public static final SqlQuantifyOperator SOME_GT =
+      new SqlQuantifyOperator(SqlKind.SOME, SqlKind.GREATER_THAN);
+
+  public static final SqlQuantifyOperator SOME_GE =
+      new SqlQuantifyOperator(SqlKind.SOME, SqlKind.GREATER_THAN_OR_EQUAL);
+
+  public static final SqlQuantifyOperator SOME_EQ =
+      new SqlQuantifyOperator(SqlKind.SOME, SqlKind.EQUALS);
+
+  public static final SqlQuantifyOperator SOME_NE =
+      new SqlQuantifyOperator(SqlKind.SOME, SqlKind.NOT_EQUALS);
+
+  /**
+   * The <code>&lt; ALL</code> operator.
+   */
+  public static final SqlQuantifyOperator ALL_LT =
+      new SqlQuantifyOperator(SqlKind.ALL, SqlKind.LESS_THAN);
+
+  public static final SqlQuantifyOperator ALL_LE =
+      new SqlQuantifyOperator(SqlKind.ALL, SqlKind.LESS_THAN_OR_EQUAL);
+
+  public static final SqlQuantifyOperator ALL_GT =
+      new SqlQuantifyOperator(SqlKind.ALL, SqlKind.GREATER_THAN);
+
+  public static final SqlQuantifyOperator ALL_GE =
+      new SqlQuantifyOperator(SqlKind.ALL, SqlKind.GREATER_THAN_OR_EQUAL);
+
+  public static final SqlQuantifyOperator ALL_EQ =
+      new SqlQuantifyOperator(SqlKind.ALL, SqlKind.EQUALS);
+
+  public static final SqlQuantifyOperator ALL_NE =
+      new SqlQuantifyOperator(SqlKind.ALL, SqlKind.NOT_EQUALS);
 
   /**
    * Logical less-than operator, '<code>&lt;</code>'.
@@ -2162,6 +2205,45 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
         call.getParserPosition());
   }
 
+  /** Returns the operator for {@code SOME comparisonKind}. */
+  public static SqlQuantifyOperator some(SqlKind comparisonKind) {
+    switch (comparisonKind) {
+    case EQUALS:
+      return SOME_EQ;
+    case NOT_EQUALS:
+      return SOME_NE;
+    case LESS_THAN:
+      return SOME_LT;
+    case LESS_THAN_OR_EQUAL:
+      return SOME_LE;
+    case GREATER_THAN:
+      return SOME_GT;
+    case GREATER_THAN_OR_EQUAL:
+      return SOME_GE;
+    default:
+      throw new AssertionError(comparisonKind);
+    }
+  }
+
+  /** Returns the operator for {@code ALL comparisonKind}. */
+  public static SqlQuantifyOperator all(SqlKind comparisonKind) {
+    switch (comparisonKind) {
+    case EQUALS:
+      return ALL_EQ;
+    case NOT_EQUALS:
+      return ALL_NE;
+    case LESS_THAN:
+      return ALL_LT;
+    case LESS_THAN_OR_EQUAL:
+      return ALL_LE;
+    case GREATER_THAN:
+      return ALL_GT;
+    case GREATER_THAN_OR_EQUAL:
+      return ALL_GE;
+    default:
+      throw new AssertionError(comparisonKind);
+    }
+  }
 }
 
 // End SqlStdOperatorTable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index d445050..84a54cf 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -888,7 +888,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     SqlNode outermostNode = performUnconditionalRewrites(topNode, false);
     cursorSet.add(outermostNode);
     top = outermostNode;
-    TRACER.trace("After unconditional rewrite: " + outermostNode.toString());
+    TRACER.trace("After unconditional rewrite: {}", outermostNode);
     if (outermostNode.isA(SqlKind.TOP_LEVEL)) {
       registerQuery(scope, null, outermostNode, outermostNode, null, false);
     }
@@ -898,7 +898,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       // caller later without needing the scope
       deriveType(scope, outermostNode);
     }
-    TRACER.trace("After validation: " + outermostNode.toString());
+    TRACER.trace("After validation: {}", outermostNode);
     return outermostNode;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/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 955f151..7f17cbb 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -130,6 +130,7 @@ import org.apache.calcite.sql.SqlWith;
 import org.apache.calcite.sql.SqlWithItem;
 import org.apache.calcite.sql.fun.SqlCountAggFunction;
 import org.apache.calcite.sql.fun.SqlInOperator;
+import org.apache.calcite.sql.fun.SqlQuantifyOperator;
 import org.apache.calcite.sql.fun.SqlRowOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
@@ -914,7 +915,7 @@ public class SqlToRelConverter {
           SqlNode[] inOperands = childSqlCall.getOperands();
           SqlInOperator inOp =
               (SqlInOperator) childSqlCall.getOperator();
-          if (inOp.isNotIn()) {
+          if (inOp.kind == SqlKind.NOT_IN) {
             return reg(scope,
                 SqlStdOperatorTable.IN.createCall(SqlParserPos.ZERO,
                     inOperands[0], inOperands[1]));
@@ -1018,6 +1019,9 @@ public class SqlToRelConverter {
       return;
 
     case IN:
+    case NOT_IN:
+    case SOME:
+    case ALL:
       call = (SqlBasicCall) subQuery.node;
       query = call.operand(1);
       if (!config.isExpand() && !(query instanceof SqlNodeList)) {
@@ -1037,7 +1041,6 @@ public class SqlToRelConverter {
         leftKeys = ImmutableList.of(bb.convertExpression(leftKeyNode));
       }
 
-      final boolean notIn = ((SqlInOperator) call.getOperator()).isNotIn();
       if (query instanceof SqlNodeList) {
         SqlNodeList valueList = (SqlNodeList) query;
         if (!containsNullLiteral(valueList)
@@ -1048,7 +1051,7 @@ public class SqlToRelConverter {
                   bb,
                   leftKeys,
                   valueList,
-                  notIn);
+                  (SqlInOperator) call.getOperator());
           return;
         }
 
@@ -1082,6 +1085,7 @@ public class SqlToRelConverter {
       final RelDataType targetRowType =
           SqlTypeUtil.promoteToRowType(typeFactory,
               validator.getValidatedNodeType(leftKeyNode), null);
+      final boolean notIn = call.getOperator().kind == SqlKind.NOT_IN;
       converted =
           convertExists(query, RelOptUtil.SubQueryType.IN, subQuery.logic,
               notIn, targetRowType);
@@ -1383,20 +1387,27 @@ public class SqlToRelConverter {
    *
    * @param leftKeys   LHS
    * @param valuesList RHS
-   * @param isNotIn    is this a NOT IN operator
+   * @param op         The operator (IN, NOT IN, &gt; SOME, ...)
    * @return converted expression
    */
   private RexNode convertInToOr(
       final Blackboard bb,
       final List<RexNode> leftKeys,
       SqlNodeList valuesList,
-      boolean isNotIn) {
+      SqlInOperator op) {
     final List<RexNode> comparisons = new ArrayList<>();
     for (SqlNode rightVals : valuesList) {
       RexNode rexComparison;
+      final SqlOperator comparisonOp;
+      if (op instanceof SqlQuantifyOperator) {
+        comparisonOp = RelOptUtil.op(((SqlQuantifyOperator) op).comparisonKind,
+            SqlStdOperatorTable.EQUALS);
+      } else {
+        comparisonOp = SqlStdOperatorTable.EQUALS;
+      }
       if (leftKeys.size() == 1) {
         rexComparison =
-            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+            rexBuilder.makeCall(comparisonOp,
                 leftKeys.get(0),
                 ensureSqlType(leftKeys.get(0).getType(),
                     bb.convertExpression(rightVals)));
@@ -1412,7 +1423,7 @@ public class SqlToRelConverter {
                     Pair.zip(leftKeys, call.getOperandList()),
                     new Function<Pair<RexNode, SqlNode>, RexNode>() {
                       public RexNode apply(Pair<RexNode, SqlNode> pair) {
-                        return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                        return rexBuilder.makeCall(comparisonOp,
                             pair.left,
                             ensureSqlType(pair.left.getType(),
                                 bb.convertExpression(pair.right)));
@@ -1423,18 +1434,18 @@ public class SqlToRelConverter {
       comparisons.add(rexComparison);
     }
 
-    RexNode result =
-        RexUtil.composeDisjunction(rexBuilder, comparisons, true);
-    assert result != null;
-
-    if (isNotIn) {
-      result =
-          rexBuilder.makeCall(
-              SqlStdOperatorTable.NOT,
-              result);
+    switch (op.kind) {
+    case ALL:
+      return RexUtil.composeConjunction(rexBuilder, comparisons, true);
+    case NOT_IN:
+      return rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+          RexUtil.composeDisjunction(rexBuilder, comparisons, true));
+    case IN:
+    case SOME:
+      return RexUtil.composeDisjunction(rexBuilder, comparisons, true);
+    default:
+      throw new AssertionError();
     }
-
-    return result;
   }
 
   /** Ensures that an expression has a given {@link SqlTypeName}, applying a
@@ -1710,10 +1721,8 @@ public class SqlToRelConverter {
       }
       return;
     case IN:
-      if (((SqlCall) node).getOperator() == SqlStdOperatorTable.NOT_IN) {
-        logic = logic.negate();
-      }
       break;
+    case NOT_IN:
     case NOT:
       logic = logic.negate();
       break;
@@ -1723,20 +1732,18 @@ public class SqlToRelConverter {
         if (operand != null) {
           // In the case of an IN expression, locate scalar
           // sub-queries so we can convert them to constants
-          findSubQueries(
-              bb,
-              operand,
-              logic,
-              kind == SqlKind.IN || registerOnlyScalarSubQueries);
+          findSubQueries(bb, operand, logic,
+              kind == SqlKind.IN || kind == SqlKind.NOT_IN
+                  || kind == SqlKind.SOME || kind == SqlKind.ALL
+                  || registerOnlyScalarSubQueries);
         }
       }
     } else if (node instanceof SqlNodeList) {
       for (SqlNode child : (SqlNodeList) node) {
-        findSubQueries(
-            bb,
-            child,
-            logic,
-            kind == SqlKind.IN || registerOnlyScalarSubQueries);
+        findSubQueries(bb, child, logic,
+            kind == SqlKind.IN || kind == SqlKind.NOT_IN
+                || kind == SqlKind.SOME || kind == SqlKind.ALL
+                || registerOnlyScalarSubQueries);
       }
     }
 
@@ -1744,7 +1751,11 @@ public class SqlToRelConverter {
     // expression, register the IN expression itself.  We need to
     // register the scalar sub-queries first so they can be converted
     // before the IN expression is converted.
-    if (kind == SqlKind.IN) {
+    switch (kind) {
+    case IN:
+    case NOT_IN:
+    case SOME:
+    case ALL:
       switch (logic) {
       case TRUE_FALSE_UNKNOWN:
         if (validator.getValidatedNodeType(node).isNullable()) {
@@ -1757,6 +1768,7 @@ public class SqlToRelConverter {
         logic = RelOptUtil.Logic.TRUE;
       }
       bb.registerSubQuery(node, logic);
+      break;
     }
   }
 
@@ -4338,10 +4350,12 @@ public class SqlToRelConverter {
         final RelRoot root;
         switch (kind) {
         case IN:
+        case NOT_IN:
+        case SOME:
+        case ALL:
           call = (SqlCall) expr;
           query = call.operand(1);
           if (!(query instanceof SqlNodeList)) {
-            final SqlInOperator op = (SqlInOperator) call.getOperator();
             root = convertQueryRecursive(query, false, null);
             final SqlNode operand = call.operand(0);
             List<SqlNode> nodes;
@@ -4357,10 +4371,23 @@ public class SqlToRelConverter {
             for (SqlNode node : nodes) {
               builder.add(convertExpression(node));
             }
-            final RexSubQuery in = RexSubQuery.in(root.rel, builder.build());
-            return op.isNotIn()
-                ? rexBuilder.makeCall(SqlStdOperatorTable.NOT, in)
-                : in;
+            final ImmutableList<RexNode> list = builder.build();
+            switch (kind) {
+            case IN:
+              return RexSubQuery.in(root.rel, list);
+            case NOT_IN:
+              return rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+                  RexSubQuery.in(root.rel, list));
+            case SOME:
+              return RexSubQuery.some(root.rel, list,
+                  (SqlQuantifyOperator) call.getOperator());
+            case ALL:
+              return rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+                  RexSubQuery.some(root.rel, list,
+                      negate((SqlQuantifyOperator) call.getOperator())));
+            default:
+              throw new AssertionError(kind);
+            }
           }
           break;
 
@@ -4386,8 +4413,15 @@ public class SqlToRelConverter {
       }
 
       switch (kind) {
+      case SOME:
+      case ALL:
+        if (config.isExpand()) {
+          throw new RuntimeException(kind
+              + " is only supported if expand = false");
+        }
       case CURSOR:
       case IN:
+      case NOT_IN:
         subQuery = Preconditions.checkNotNull(getSubQuery(expr));
         rex = Preconditions.checkNotNull(subQuery.expr);
         return StandardConvertletTable.castToValidatedType(expr, rex,
@@ -4563,6 +4597,11 @@ public class SqlToRelConverter {
 
   }
 
+  private SqlQuantifyOperator negate(SqlQuantifyOperator operator) {
+    assert operator.kind == SqlKind.ALL;
+    return SqlStdOperatorTable.some(operator.comparisonKind.negateNullSafe());
+  }
+
   /** Deferred lookup. */
   private static class DeferredLookup {
     Blackboard bb;

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index a3e7a5a..e70e291 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -1790,6 +1790,54 @@ public class SqlParserTest {
             + "FROM `DEPT`)) AND FALSE)");
   }
 
+  @Test public void testSome() {
+    final String sql = "select * from emp\n"
+        + "where sal > some (select comm from emp)";
+    final String expected = "SELECT *\n"
+        + "FROM `EMP`\n"
+        + "WHERE (`SAL` > SOME (SELECT `COMM`\n"
+        + "FROM `EMP`))";
+    sql(sql).ok(expected);
+
+    // ANY is a synonym for SOME
+    final String sql2 = "select * from emp\n"
+        + "where sal > any (select comm from emp)";
+    sql(sql2).ok(expected);
+
+    final String sql3 = "select * from emp\n"
+        + "where name like (select ^some^ name from emp)";
+    sql(sql3).fails("(?s).*Encountered \"some\" at .*");
+
+    final String sql4 = "select * from emp\n"
+        + "where name ^like^ some (select name from emp)";
+    sql(sql4).fails("(?s).*Encountered \"like some\" at .*");
+
+    final String sql5 = "select * from emp where empno = any (10,20)";
+    final String expected5 = "SELECT *\n"
+        + "FROM `EMP`\n"
+        + "WHERE (`EMPNO` = SOME (10, 20))";
+    sql(sql5).ok(expected5);
+  }
+
+  @Test public void testAll() {
+    final String sql = "select * from emp\n"
+        + "where sal <= all (select comm from emp) or sal > 10";
+    final String expected = "SELECT *\n"
+        + "FROM `EMP`\n"
+        + "WHERE ((`SAL` <= ALL (SELECT `COMM`\n"
+        + "FROM `EMP`)) OR (`SAL` > 10))";
+    sql(sql).ok(expected);
+  }
+
+  @Test public void testAllList() {
+    final String sql = "select * from emp\n"
+        + "where sal <= all (12, 20, 30)";
+    final String expected = "SELECT *\n"
+        + "FROM `EMP`\n"
+        + "WHERE (`SAL` <= ALL (12, 20, 30))";
+    sql(sql).ok(expected);
+  }
+
   @Test public void testUnion() {
     check(
         "select * from a union select * from a",

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
index d080773..dd867ed 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
@@ -210,6 +210,12 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
           "KEYWORD(VAR_SAMP)",
           "KEYWORD(YEAR)");
 
+  protected static final List<String> QUANTIFIERS =
+      Arrays.asList(
+          "KEYWORD(ALL)",
+          "KEYWORD(ANY)",
+          "KEYWORD(SOME)");
+
   protected static final List<String> SELECT_KEYWORDS =
       Arrays.asList(
           "KEYWORD(ALL)",
@@ -589,7 +595,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
 
     // unfortunately cannot complete this case: syntax is too broken
     sql = "select a.empno, b.deptno from dummy a join sales.^ on a.deptno=";
-    assertComplete(sql, EXPR_KEYWORDS); // join
+    assertComplete(sql, QUANTIFIERS, EXPR_KEYWORDS); // join
   }
 
   @Test public void testJoinKeywords() {
@@ -616,7 +622,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
             + "on a.deptno=^b.dummy where empno=1";
-    assertHint(sql, EXPR_KEYWORDS, AB_TABLES); // on right
+    assertHint(sql, EXPR_KEYWORDS, QUANTIFIERS, AB_TABLES); // on right
 
     sql =
         "select a.empno, b.deptno from sales.emp a join sales.dept b "
@@ -635,7 +641,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     sql =
         "select a.empno, b.deptno from sales.emp a, sales.dept b "
             + "where b.deptno=^a.dummy";
-    assertHint(sql, AB_TABLES, EXPR_KEYWORDS); // where list
+    assertHint(sql, AB_TABLES, EXPR_KEYWORDS, QUANTIFIERS); // where list
 
     sql =
         "select a.empno, b.deptno from sales.emp a, sales.dept b "

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/test/java/org/apache/calcite/test/QuidemTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/QuidemTest.java b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
index 40a3266..28994a2 100644
--- a/core/src/test/java/org/apache/calcite/test/QuidemTest.java
+++ b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
@@ -158,6 +158,11 @@ public class QuidemTest {
                     && (Boolean) value;
                 closer.add(Hook.ENABLE_BINDABLE.addThread(Hook.property(b)));
               }
+              if (propertyName.equals("expand")) {
+                final boolean b = value instanceof Boolean
+                    && (Boolean) value;
+                closer.add(Prepare.THREAD_EXPAND.push(b));
+              }
             }
           })
           .execute();

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 6c973f0..8f5e27d 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -3086,6 +3086,18 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkSubQuery(sql).withLateDecorrelation(true).check();
   }
 
+  @Test public void testAll() {
+    final String sql = "select * from emp e1\n"
+        + "  where e1.empno > ALL (select deptno from dept)";
+    checkSubQuery(sql).withLateDecorrelation(true).check();
+  }
+
+  @Test public void testSome() {
+    final String sql = "select * from emp e1\n"
+        + "  where e1.empno > SOME (select deptno from dept)";
+    checkSubQuery(sql).withLateDecorrelation(true).check();
+  }
+
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-1546">[CALCITE-1546]
    * Sub-queries connected by OR</a>. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/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 2717f48..29797c6 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -1135,6 +1135,7 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
   @Test public void testInValueListShort() {
     final String sql = "select empno from emp where deptno in (10, 20)";
     sql(sql).ok();
+    sql(sql).expand(false).ok();
   }
 
   @Test public void testInValueListLong() {
@@ -1170,6 +1171,22 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
+  @Test public void testAllValueList() {
+    final String sql = "select empno from emp where deptno > all (10, 20)";
+    sql(sql).expand(false).ok();
+  }
+
+  @Test public void testSomeValueList() {
+    final String sql = "select empno from emp where deptno > some (10, 20)";
+    sql(sql).expand(false).ok();
+  }
+
+  @Test public void testSome() {
+    final String sql = "select empno from emp where deptno > some (\n"
+        + "  select deptno from dept)";
+    sql(sql).expand(false).ok();
+  }
+
   @Test public void testNotInUncorrelatedSubQueryRex() {
     final String sql = "select empno from emp where deptno not in"
         + " (select deptno from dept)";

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index a61e7e8..41acc82 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -4797,6 +4797,47 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Values passed to IN operator must have compatible types");
   }
 
+  @Test public void testAnyList() {
+    check("select * from emp where empno = any (10,20)");
+
+    check("select * from emp\n"
+        + "where empno < any (10 + deptno, cast(null as integer))");
+    checkFails(
+        "select * from emp where empno < any ^(10, '20')^",
+        ERR_IN_VALUES_INCOMPATIBLE);
+
+    checkExpType("1 < all (2, 3, 4)", "BOOLEAN NOT NULL");
+    checkExpType("cast(null as integer) < all (2, 3, 4)", "BOOLEAN");
+    checkExpType("1 > some (2, cast(null as integer) , 4)", "BOOLEAN");
+    checkExpType("1 > any (2.5, 3.14)", "BOOLEAN NOT NULL");
+    checkExpType("true = any (false, unknown)", "BOOLEAN");
+    checkExpType("true = any (false, false or unknown)", "BOOLEAN");
+    checkExpType("true <> any (false, true)", "BOOLEAN NOT NULL");
+    checkExpType("(1,2) = any ((1,2), (3,4))", "BOOLEAN NOT NULL");
+    checkExpType("(1,2) < any ((1,2), (3,4))", "BOOLEAN NOT NULL");
+    checkExpType("'abc' < any (cast(null as varchar(10)), 'bc')",
+        "BOOLEAN");
+
+    // nullability depends on nullability of both sides
+    checkColumnType("select empno < any (1, 2) from emp", "BOOLEAN NOT NULL");
+    checkColumnType(
+        "select nullif(empno,empno) > all (1, 2) from emp",
+        "BOOLEAN");
+    checkColumnType(
+        "select empno in (1, nullif(empno,empno), 2) from emp",
+        "BOOLEAN");
+
+    checkExpFails("1 = any ^(2, 'c')^",
+        ERR_IN_VALUES_INCOMPATIBLE);
+    checkExpFails("1 > all ^((2), (3,4))^",
+        ERR_IN_VALUES_INCOMPATIBLE);
+    checkExp("false and 1 = any ('b', 'c')");
+    checkExpFails("false and ^1 = any (date '2012-01-02', date '2012-01-04')^",
+        ERR_IN_OPERANDS_INCOMPATIBLE);
+    checkExpFails("1 > 5 or ^(1, 2) < any (3, 4)^",
+        ERR_IN_OPERANDS_INCOMPATIBLE);
+  }
+
   @Test public void testDoubleNoAlias() {
     check("select * from emp join dept on true");
     check("select * from emp, dept");
@@ -5860,6 +5901,14 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         .fails("Bang equal '!=' is not allowed under the current SQL conformance level")
         .tester(oracleTester)
         .ok();
+
+    sql("select * from (values 1) where 1 != any (2, 3)")
+        .tester(customTester)
+        .ok()
+        .tester(defaultTester)
+        .fails("Bang equal '!=' is not allowed under the current SQL conformance level")
+        .tester(oracleTester)
+        .ok();
   }
 
   @Test public void testOrder() {
@@ -8369,6 +8418,18 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         + "DATETIME_PLUS -\n"
         + "EXISTS pre\n"
         + "\n"
+        + "< ALL left\n"
+        + "< SOME left\n"
+        + "<= ALL left\n"
+        + "<= SOME left\n"
+        + "<> ALL left\n"
+        + "<> SOME left\n"
+        + "= ALL left\n"
+        + "= SOME left\n"
+        + "> ALL left\n"
+        + "> SOME left\n"
+        + ">= ALL left\n"
+        + ">= SOME left\n"
         + "BETWEEN ASYMMETRIC -\n"
         + "BETWEEN SYMMETRIC -\n"
         + "IN left\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
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 ea684e5..e208866 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -321,6 +321,46 @@ LogicalProject(EXPR$0=[1])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testAll">
+        <Resource name="sql">
+            <![CDATA[select * from emp e1
+  where e1.empno > ALL (select deptno from dept)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[NOT(<= SOME($0, {
+LogicalProject(DEPTNO=[$0])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+}))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planMid">
+            <![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=[NOT(CASE(=($10, 0), false, IS TRUE(<=($0, $9)), true, >($10, $11), null, <=($0, $9)))])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{}], m=[MAX($0)], c=[COUNT()], d=[COUNT($0)])
+          LogicalProject(DEPTNO=[$0])
+            LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![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=[NOT(CASE(=($10, 0), false, IS TRUE(<=($0, $9)), true, >($10, $11), null, <=($0, $9)))])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{}], m=[MAX($0)], c=[COUNT()], d=[COUNT($0)])
+          LogicalProject(DEPTNO=[$0])
+            LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testDecorrelateExists">
         <Resource name="sql">
             <![CDATA[select * from sales.emp
@@ -6137,6 +6177,46 @@ LogicalProject(EMPNO=[$0], D=[$9])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testSome">
+        <Resource name="sql">
+            <![CDATA[select * from emp e1
+  where e1.empno > SOME (select deptno from dept)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[> SOME($0, {
+LogicalProject(DEPTNO=[$0])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+})])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planMid">
+            <![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=[CASE(=($10, 0), false, IS TRUE(>($0, $9)), true, >($10, $11), null, >($0, $9))])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{}], m=[MIN($0)], c=[COUNT()], d=[COUNT($0)])
+          LogicalProject(DEPTNO=[$0])
+            LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![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=[CASE(=($10, 0), false, IS TRUE(>($0, $9)), true, >($10, $11), null, >($0, $9))])
+      LogicalJoin(condition=[true], joinType=[inner])
+        LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+        LogicalAggregate(group=[{}], m=[MIN($0)], c=[COUNT()], d=[COUNT($0)])
+          LogicalProject(DEPTNO=[$0])
+            LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testSortJoinTranspose2">
         <Resource name="sql">
             <![CDATA[select * from sales.emp e right join (

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
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 131ea6e..321b6a7 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -118,6 +118,18 @@ LogicalFilter(condition=[>($0, 1)])
             <![CDATA[select count(empno) as e from emp having e > 1]]>
         </Resource>
     </TestCase>
+    <TestCase name="testAllValueList">
+        <Resource name="sql">
+            <![CDATA[select empno from emp where deptno > all (10, 20)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[AND(>($7, 10), >($7, 20))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testGroupJustOneAgg">
         <Resource name="plan">
             <![CDATA[
@@ -3624,6 +3636,34 @@ LogicalProject(EXPR$0=[+(2, 2)])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testSome">
+        <Resource name="sql">
+            <![CDATA[select empno from emp where deptno > some (
+  select deptno from dept)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[> SOME($7, {
+LogicalProject(DEPTNO=[$0])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+})])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testSomeValueList">
+        <Resource name="sql">
+            <![CDATA[select empno from emp where deptno > some (10, 20)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[OR(>($7, 10), >($7, 20))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testUpdateWhere">
         <Resource name="sql">
             <![CDATA[update emp set empno = empno + 1 where deptno = 10]]>

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/test/resources/sql/some.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/some.iq b/core/src/test/resources/sql/some.iq
new file mode 100644
index 0000000..bde7a74
--- /dev/null
+++ b/core/src/test/resources/sql/some.iq
@@ -0,0 +1,577 @@
+# some.iq - Queries with quantifiers SOME (aka ANY) and ALL
+#
+# 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.
+#
+!use scott
+!set expand false
+!set outputformat mysql
+
+# Both sides NOT NULL
+select * from "scott".emp
+where empno > any (select deptno from "scott".dept);
+
++-------+--------+-----------+------+------------+---------+---------+--------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO |
++-------+--------+-----------+------+------------+---------+---------+--------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 |
++-------+--------+-----------+------+------------+---------+---------+--------+
+(14 rows)
+
+!ok
+
+# ANY; left side NOT NULL, right side nullable.
+select * from "scott".emp
+where sal > any (select comm from "scott".emp);
+
++-------+--------+-----------+------+------------+---------+---------+--------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO |
++-------+--------+-----------+------+------------+---------+---------+--------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 |
++-------+--------+-----------+------+------------+---------+---------+--------+
+(14 rows)
+
+!ok
+
+# ALL; left side NOT NULL, right side nullable.
+select * from "scott".emp
+where sal > all (select comm from "scott".emp);
+
++-------+-------+-----+-----+----------+-----+------+--------+
+| EMPNO | ENAME | JOB | MGR | HIREDATE | SAL | COMM | DEPTNO |
++-------+-------+-----+-----+----------+-----+------+--------+
++-------+-------+-----+-----+----------+-----+------+--------+
+(0 rows)
+
+!ok
+
+# Previous, as scalar sub-query
+select *, sal > all (select comm from "scott".emp) as x
+from "scott".emp;
+
++-------+--------+-----------+------+------------+---------+---------+--------+-------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO | X     |
++-------+--------+-----------+------+------------+---------+---------+--------+-------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 | false |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 | false |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 | false |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 | false |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 | false |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 | false |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |       |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |       |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |       |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |       |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |       |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |       |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |       |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |       |
++-------+--------+-----------+------+------------+---------+---------+--------+-------+
+(14 rows)
+
+!ok
+EnumerableCalc(expr#0..10=[{inputs}], expr#11=[0], expr#12=[=($t1, $t11)], expr#13=[false], expr#14=[<=($t8, $t0)], expr#15=[IS TRUE($t14)], expr#16=[true], expr#17=[>($t1, $t2)], expr#18=[null], expr#19=[CASE($t12, $t13, $t15, $t16, $t17, $t18, $t14)], expr#20=[NOT($t19)], EMPNO=[$t3], ENAME=[$t4], JOB=[$t5], MGR=[$t6], HIREDATE=[$t7], SAL=[$t8], COMM=[$t9], DEPTNO=[$t10], X=[$t20])
+  EnumerableJoin(condition=[true], joinType=[inner])
+    EnumerableAggregate(group=[{}], m=[MAX($6)], c=[COUNT()], d=[COUNT($6)])
+      EnumerableTableScan(table=[[scott, EMP]])
+    EnumerableTableScan(table=[[scott, EMP]])
+!plan
+
+# NOT SOME; left side NOT NULL, right side nullable; converse of previous query.
+select * from "scott".emp
+where not sal <= some (select comm from "scott".emp);
+
++-------+-------+-----+-----+----------+-----+------+--------+
+| EMPNO | ENAME | JOB | MGR | HIREDATE | SAL | COMM | DEPTNO |
++-------+-------+-----+-----+----------+-----+------+--------+
++-------+-------+-----+-----+----------+-----+------+--------+
+(0 rows)
+
+!ok
+
+# Similar, as scalar sub-query.
+select *, sal <= some (select comm from "scott".emp) as x
+from "scott".emp;
+
++-------+--------+-----------+------+------------+---------+---------+--------+------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO | X    |
++-------+--------+-----------+------+------------+---------+---------+--------+------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 | true |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |      |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 | true |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |      |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 | true |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |      |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |      |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |      |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |      |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |      |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 | true |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 | true |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |      |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 | true |
++-------+--------+-----------+------+------------+---------+---------+--------+------+
+(14 rows)
+
+!ok
+
+# Sub-query is empty, so "< all" is trivially true. Even for null comm.
+select * from "scott".emp
+where comm < all (select comm from "scott".emp where 1 = 0)
+order by empno;
+
++-------+--------+-----------+------+------------+---------+---------+--------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO |
++-------+--------+-----------+------+------------+---------+---------+--------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 |
++-------+--------+-----------+------+------------+---------+---------+--------+
+(14 rows)
+
+!ok
+
+# If sub-query is empty, "< some" is trivially false. Even for null comm.
+select * from "scott".emp
+where comm < some (select comm from "scott".emp where 1 = 0);
+
++-------+-------+-----+-----+----------+-----+------+--------+
+| EMPNO | ENAME | JOB | MGR | HIREDATE | SAL | COMM | DEPTNO |
++-------+-------+-----+-----+----------+-----+------+--------+
++-------+-------+-----+-----+----------+-----+------+--------+
+(0 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > all (select comm from "scott".emp where comm <> null);
+
++-------+--------+-----------+------+------------+---------+---------+--------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO |
++-------+--------+-----------+------+------------+---------+---------+--------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 |
++-------+--------+-----------+------+------------+---------+---------+--------+
+(14 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > all(500, 2000);
+
++-------+-------+-----------+------+------------+---------+------+--------+
+| EMPNO | ENAME | JOB       | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+-----------+------+------------+---------+------+--------+
+|  7566 | JONES | MANAGER   | 7839 | 1981-02-04 | 2975.00 |      |     20 |
+|  7698 | BLAKE | MANAGER   | 7839 | 1981-01-05 | 2850.00 |      |     30 |
+|  7782 | CLARK | MANAGER   | 7839 | 1981-06-09 | 2450.00 |      |     10 |
+|  7788 | SCOTT | ANALYST   | 7566 | 1987-04-19 | 3000.00 |      |     20 |
+|  7839 | KING  | PRESIDENT |      | 1981-11-17 | 5000.00 |      |     10 |
+|  7902 | FORD  | ANALYST   | 7566 | 1981-12-03 | 3000.00 |      |     20 |
++-------+-------+-----------+------+------------+---------+------+--------+
+(6 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > all (4000, 2000);
+
++-------+-------+-----------+-----+------------+---------+------+--------+
+| EMPNO | ENAME | JOB       | MGR | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+-----------+-----+------------+---------+------+--------+
+|  7839 | KING  | PRESIDENT |     | 1981-11-17 | 5000.00 |      |     10 |
++-------+-------+-----------+-----+------------+---------+------+--------+
+(1 row)
+
+!ok
+
+select * from "scott".emp
+where sal > some (4000, 2000);
+
++-------+-------+-----------+------+------------+---------+------+--------+
+| EMPNO | ENAME | JOB       | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+-----------+------+------------+---------+------+--------+
+|  7566 | JONES | MANAGER   | 7839 | 1981-02-04 | 2975.00 |      |     20 |
+|  7698 | BLAKE | MANAGER   | 7839 | 1981-01-05 | 2850.00 |      |     30 |
+|  7782 | CLARK | MANAGER   | 7839 | 1981-06-09 | 2450.00 |      |     10 |
+|  7788 | SCOTT | ANALYST   | 7566 | 1987-04-19 | 3000.00 |      |     20 |
+|  7839 | KING  | PRESIDENT |      | 1981-11-17 | 5000.00 |      |     10 |
+|  7902 | FORD  | ANALYST   | 7566 | 1981-12-03 | 3000.00 |      |     20 |
++-------+-------+-----------+------+------------+---------+------+--------+
+(6 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > any (4000, 2000);
+
++-------+-------+-----------+------+------------+---------+------+--------+
+| EMPNO | ENAME | JOB       | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+-----------+------+------------+---------+------+--------+
+|  7566 | JONES | MANAGER   | 7839 | 1981-02-04 | 2975.00 |      |     20 |
+|  7698 | BLAKE | MANAGER   | 7839 | 1981-01-05 | 2850.00 |      |     30 |
+|  7782 | CLARK | MANAGER   | 7839 | 1981-06-09 | 2450.00 |      |     10 |
+|  7788 | SCOTT | ANALYST   | 7566 | 1987-04-19 | 3000.00 |      |     20 |
+|  7839 | KING  | PRESIDENT |      | 1981-11-17 | 5000.00 |      |     10 |
+|  7902 | FORD  | ANALYST   | 7566 | 1981-12-03 | 3000.00 |      |     20 |
++-------+-------+-----------+------+------------+---------+------+--------+
+(6 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > (select sal * 2 from "scott".emp);
+more than one value in agg SINGLE_VALUE
+!error
+
+select * from "scott".emp
+where sal > any (select sal * 2 from "scott".emp)
+order by sal desc;
+
++-------+-------+-----------+------+------------+---------+------+--------+
+| EMPNO | ENAME | JOB       | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+-----------+------+------------+---------+------+--------+
+|  7839 | KING  | PRESIDENT |      | 1981-11-17 | 5000.00 |      |     10 |
+|  7788 | SCOTT | ANALYST   | 7566 | 1987-04-19 | 3000.00 |      |     20 |
+|  7902 | FORD  | ANALYST   | 7566 | 1981-12-03 | 3000.00 |      |     20 |
+|  7566 | JONES | MANAGER   | 7839 | 1981-02-04 | 2975.00 |      |     20 |
+|  7698 | BLAKE | MANAGER   | 7839 | 1981-01-05 | 2850.00 |      |     30 |
+|  7782 | CLARK | MANAGER   | 7839 | 1981-06-09 | 2450.00 |      |     10 |
++-------+-------+-----------+------+------------+---------+------+--------+
+(6 rows)
+
+!ok
+
+select * from "scott".emp
+where sal < all (select sal * 2 from "scott".emp)
+order by sal desc;
+
++-------+--------+----------+------+------------+---------+---------+--------+
+| EMPNO | ENAME  | JOB      | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO |
++-------+--------+----------+------+------------+---------+---------+--------+
+|  7844 | TURNER | SALESMAN | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |
+|  7934 | MILLER | CLERK    | 7782 | 1982-01-23 | 1300.00 |         |     10 |
+|  7521 | WARD   | SALESMAN | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |
+|  7654 | MARTIN | SALESMAN | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |
+|  7876 | ADAMS  | CLERK    | 7788 | 1987-05-23 | 1100.00 |         |     20 |
+|  7900 | JAMES  | CLERK    | 7698 | 1981-12-03 |  950.00 |         |     30 |
+|  7369 | SMITH  | CLERK    | 7902 | 1980-12-17 |  800.00 |         |     20 |
++-------+--------+----------+------+------------+---------+---------+--------+
+(7 rows)
+
+!ok
+
+# Equivalent to previous
+select r.*,
+  not case when m is not null then r.sal >= m
+      when c > 0 then null
+      else false end as c
+from (select min(sal * 2) as m, count(*) as c from "scott".emp)
+cross join "scott".emp as r
+order by sal desc;
++-------+--------+-----------+------+------------+---------+---------+--------+-------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO | C     |
++-------+--------+-----------+------+------------+---------+---------+--------+-------+
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 | false |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 | false |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 | false |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 | false |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 | false |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 | false |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 | false |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 | true  |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 | true  |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 | true  |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 | true  |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 | true  |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 | true  |
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 | true  |
++-------+--------+-----------+------+------------+---------+---------+--------+-------+
+(14 rows)
+
+!ok
+
+select * from "scott".emp
+where sal < all (select comm * 2 from "scott".emp);
+
++-------+-------+-----+-----+----------+-----+------+--------+
+| EMPNO | ENAME | JOB | MGR | HIREDATE | SAL | COMM | DEPTNO |
++-------+-------+-----+-----+----------+-----+------+--------+
++-------+-------+-----+-----+----------+-----+------+--------+
+(0 rows)
+
+!ok
+
+select * from "scott".emp
+where sal < any (select comm * 2 from "scott".emp)
+order by empno;
+
++-------+--------+----------+------+------------+---------+---------+--------+
+| EMPNO | ENAME  | JOB      | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO |
++-------+--------+----------+------+------------+---------+---------+--------+
+|  7369 | SMITH  | CLERK    | 7902 | 1980-12-17 |  800.00 |         |     20 |
+|  7499 | ALLEN  | SALESMAN | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |
+|  7521 | WARD   | SALESMAN | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |
+|  7654 | MARTIN | SALESMAN | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |
+|  7782 | CLARK  | MANAGER  | 7839 | 1981-06-09 | 2450.00 |         |     10 |
+|  7844 | TURNER | SALESMAN | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |
+|  7876 | ADAMS  | CLERK    | 7788 | 1987-05-23 | 1100.00 |         |     20 |
+|  7900 | JAMES  | CLERK    | 7698 | 1981-12-03 |  950.00 |         |     30 |
+|  7934 | MILLER | CLERK    | 7782 | 1982-01-23 | 1300.00 |         |     10 |
++-------+--------+----------+------+------------+---------+---------+--------+
+(9 rows)
+
+!ok
+
+# Oracle gives error, but I believe the statement is valid.
+# It should return all rows.
+select * from "scott".emp
+where sal < any (select comm * 2 from "scott".emp) is unknown;
+
++-------+-------+-----------+------+------------+---------+------+--------+
+| EMPNO | ENAME | JOB       | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+-----------+------+------------+---------+------+--------+
+|  7566 | JONES | MANAGER   | 7839 | 1981-02-04 | 2975.00 |      |     20 |
+|  7698 | BLAKE | MANAGER   | 7839 | 1981-01-05 | 2850.00 |      |     30 |
+|  7788 | SCOTT | ANALYST   | 7566 | 1987-04-19 | 3000.00 |      |     20 |
+|  7839 | KING  | PRESIDENT |      | 1981-11-17 | 5000.00 |      |     10 |
+|  7902 | FORD  | ANALYST   | 7566 | 1981-12-03 | 3000.00 |      |     20 |
++-------+-------+-----------+------+------------+---------+------+--------+
+(5 rows)
+
+!ok
+
+# Oracle gives error, but I believe the statement is valid and result is correct.
+select *, sal > all(select comm from "scott".emp) as x from "scott".emp;
+
++-------+--------+-----------+------+------------+---------+---------+--------+-------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO | X     |
++-------+--------+-----------+------+------------+---------+---------+--------+-------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 | false |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |       |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 | false |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |       |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 | false |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |       |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |       |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |       |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |       |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |       |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 | false |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 | false |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |       |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 | false |
++-------+--------+-----------+------+------------+---------+---------+--------+-------+
+(14 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > all (select comm from "scott".emp);
+
++-------+-------+-----+-----+----------+-----+------+--------+
+| EMPNO | ENAME | JOB | MGR | HIREDATE | SAL | COMM | DEPTNO |
++-------+-------+-----+-----+----------+-----+------+--------+
++-------+-------+-----+-----+----------+-----+------+--------+
+(0 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > any (select comm from "scott".emp);
+
++-------+--------+-----------+------+------------+---------+---------+--------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO |
++-------+--------+-----------+------+------------+---------+---------+--------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 |
++-------+--------+-----------+------+------------+---------+---------+--------+
+(14 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > any (select comm from "scott".emp where comm < 1000);
+
++-------+--------+-----------+------+------------+---------+---------+--------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO |
++-------+--------+-----------+------+------------+---------+---------+--------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 |
++-------+--------+-----------+------+------------+---------+---------+--------+
+(14 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > any (select comm from "scott".emp where comm < 2000);
+
++-------+--------+-----------+------+------------+---------+---------+--------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO |
++-------+--------+-----------+------+------------+---------+---------+--------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 |
++-------+--------+-----------+------+------------+---------+---------+--------+
+(14 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > any (select comm * 2 from "scott".emp where comm < 2000);
+
++-------+--------+-----------+------+------------+---------+---------+--------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO |
++-------+--------+-----------+------+------------+---------+---------+--------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 |
++-------+--------+-----------+------+------------+---------+---------+--------+
+(14 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > all (select comm * 2 from "scott".emp where comm < 2000);
+
++-------+-------+-----------+------+------------+---------+------+--------+
+| EMPNO | ENAME | JOB       | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+-----------+------+------------+---------+------+--------+
+|  7566 | JONES | MANAGER   | 7839 | 1981-02-04 | 2975.00 |      |     20 |
+|  7698 | BLAKE | MANAGER   | 7839 | 1981-01-05 | 2850.00 |      |     30 |
+|  7788 | SCOTT | ANALYST   | 7566 | 1987-04-19 | 3000.00 |      |     20 |
+|  7839 | KING  | PRESIDENT |      | 1981-11-17 | 5000.00 |      |     10 |
+|  7902 | FORD  | ANALYST   | 7566 | 1981-12-03 | 3000.00 |      |     20 |
++-------+-------+-----------+------+------------+---------+------+--------+
+(5 rows)
+
+!ok
+
+select * from "scott".emp
+where sal > all (select comm from "scott".emp where comm is not null);
+
++-------+--------+-----------+------+------------+---------+--------+--------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM   | DEPTNO |
++-------+--------+-----------+------+------------+---------+--------+--------+
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 | 300.00 |     30 |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |        |     20 |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |        |     30 |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |        |     10 |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |        |     20 |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |        |     10 |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |   0.00 |     30 |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |        |     20 |
++-------+--------+-----------+------+------------+---------+--------+--------+
+(8 rows)
+
+!ok
+
+# End some.iq
+

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/core/src/test/resources/sql/sub-query.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/sub-query.iq b/core/src/test/resources/sql/sub-query.iq
index f61ffc6..e31319e 100644
--- a/core/src/test/resources/sql/sub-query.iq
+++ b/core/src/test/resources/sql/sub-query.iq
@@ -627,4 +627,22 @@ EnumerableSemiJoin(condition=[=($0, $10)], joinType=[inner])
     EnumerableTableScan(table=[[scott, EMP]])
 !plan
 
+# [DRILL-5644]
+select TJOIN1.RNUM, TJOIN1.C1,
+  case when 10 in ( select C1 from ( values (1) ) T(C1) ) then 'yes' else 'no' end C3
+from (
+  values (0, 10, 15),
+    (1, 20, 25),
+    (2, cast(NULL as integer), 50)) TJOIN1 (RNUM, C1, C2);
++------+----+-----+
+| RNUM | C1 | C3  |
++------+----+-----+
+|    0 | 10 | no  |
+|    1 | 20 | no  |
+|    2 |    | no  |
++------+----+-----+
+(3 rows)
+
+!ok
+
 # End sub-query.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/f4746523/site/_docs/reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 5ff91ea..4756468 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -1045,8 +1045,21 @@ The operator precedence and associativity, highest to lowest.
 | value NOT IN (value [, value]*)                   | Whether *value* is not equal to every value in a list
 | value IN (sub-query)                              | Whether *value* is equal to a row returned by *sub-query*
 | value NOT IN (sub-query)                          | Whether *value* is not equal to every row returned by *sub-query*
+| value comparison SOME (sub-query)                 | Whether *value* *comparison* at least one row returned by *sub-query*
+| value comparison ANY (sub-query)                  | Synonym for SOME
+| value comparison ALL (sub-query)                  | Whether *value* *comparison* every row returned by *sub-query*
 | EXISTS (sub-query)                                | Whether *sub-query* returns at least one row
 
+{% highlight sql %}
+comp:
+      =
+  |   <>
+  |   >
+  |   >=
+  |   <
+  |   <=
+{% endhighlight %}
+
 ### Logical operators
 
 | Operator syntax        | Description