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/12/14 16:42:45 UTC

[2/3] calcite git commit: [CALCITE-1519] Standardize on "sub-query" rather than "subquery" in class names and comments

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/core/src/main/java/org/apache/calcite/sql2rel/SubqueryConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SubqueryConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SubqueryConverter.java
deleted file mode 100644
index 6145619..0000000
--- a/core/src/main/java/org/apache/calcite/sql2rel/SubqueryConverter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.sql2rel;
-
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.SqlCall;
-
-/**
- * SubqueryConverter provides the interface for classes that convert subqueries
- * into equivalent expressions.
- */
-public interface SubqueryConverter {
-  //~ Methods ----------------------------------------------------------------
-
-  /**
-   * @return true if the subquery can be converted
-   */
-  boolean canConvertSubquery();
-
-  /**
-   * Converts the subquery to an equivalent expression.
-   *
-   * @param subquery        the SqlNode tree corresponding to a subquery
-   * @param parentConverter sqlToRelConverter of the parent query
-   * @param isExists        whether the subquery is part of an EXISTS expression
-   * @param isExplain       whether the subquery is part of an EXPLAIN PLAN
-   *                        statement
-   * @return the equivalent expression or null if the subquery couldn't be
-   * converted
-   */
-  RexNode convertSubquery(
-      SqlCall subquery,
-      SqlToRelConverter parentConverter,
-      boolean isExists,
-      boolean isExplain);
-}
-
-// End SubqueryConverter.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/core/src/main/java/org/apache/calcite/tools/Programs.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Programs.java b/core/src/main/java/org/apache/calcite/tools/Programs.java
index 5f2f098..6bf65e4 100644
--- a/core/src/main/java/org/apache/calcite/tools/Programs.java
+++ b/core/src/main/java/org/apache/calcite/tools/Programs.java
@@ -106,7 +106,7 @@ public class Programs {
 
   /** Program that expands sub-queries. */
   public static final Program SUB_QUERY_PROGRAM =
-      subquery(DefaultRelMetadataProvider.INSTANCE);
+      subQuery(DefaultRelMetadataProvider.INSTANCE);
 
   public static final ImmutableSet<RelOptRule> RULE_SET =
       ImmutableSet.of(
@@ -261,7 +261,12 @@ public class Programs {
     return hep(CALC_RULES, true, metadataProvider);
   }
 
+  @Deprecated // to be removed before 2.0
   public static Program subquery(RelMetadataProvider metadataProvider) {
+    return subQuery(metadataProvider);
+  }
+
+  public static Program subQuery(RelMetadataProvider metadataProvider) {
     return hep(
         ImmutableList.of((RelOptRule) SubQueryRemoveRule.FILTER,
             SubQueryRemoveRule.PROJECT,
@@ -303,7 +308,7 @@ public class Programs {
           }
         };
 
-    return sequence(subquery(metadataProvider),
+    return sequence(subQuery(metadataProvider),
         new DecorrelateProgram(),
         new TrimFieldsProgram(),
         program1,

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 741ef7f..5a278b6 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -140,7 +140,7 @@ DuplicateArgumentName=Duplicate argument name ''{0}''
 DefaultForOptionalParameter=DEFAULT is only allowed for optional parameters
 AccessNotAllowed=Not allowed to perform {0} on {1}
 MinMaxBadType=The {0} function does not support the {1} data type.
-OnlyScalarSubqueryAllowed=Only scalar subqueries allowed in select list.
+OnlyScalarSubQueryAllowed=Only scalar sub-queries allowed in select list.
 OrderByOrdinalOutOfRange=Ordinal out of range
 WindowHasNegativeSize=Window has negative size
 UnboundedFollowingWindowNotSupported=UNBOUNDED FOLLOWING window not supported

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/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 e7a90a1..b8c0ed7 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
@@ -1281,7 +1281,7 @@ public class SqlParserTest {
         "values a similar to b like c similar to d escape e escape f",
         "VALUES (ROW((`A` SIMILAR TO (`B` LIKE (`C` SIMILAR TO `D` ESCAPE `E`) ESCAPE `F`))))");
 
-    // SIMILAR TO with subquery
+    // SIMILAR TO with sub-query
     check(
         "values a similar to (select * from t where a like b escape c) escape d",
         "VALUES (ROW((`A` SIMILAR TO (SELECT *\n"
@@ -1601,7 +1601,7 @@ public class SqlParserTest {
         "(?s)Encountered \"with\" at .*");
   }
 
-  @Test public void testWithNestedInSubquery() {
+  @Test public void testWithNestedInSubQuery() {
     // SQL standard does not allow sub-query to contain WITH but we do
     check("with emp2 as (select * from emp)\n"
             + "(\n"
@@ -1956,7 +1956,7 @@ public class SqlParserTest {
             + "WHERE (3 = 3)");
   }
 
-  @Test public void testSubqueryInJoin() {
+  @Test public void testSubQueryInJoin() {
     if (!Bug.TODO_FIXED) {
       return;
     }
@@ -3423,7 +3423,7 @@ public class SqlParserTest {
         "case col1 when \n1.2 then 'one' when 2 then 'two' else 'three' end",
         "(CASE WHEN (`COL1` = 1.2) THEN 'one' WHEN (`COL1` = 2) THEN 'two' ELSE 'three' END)");
 
-    // subqueries as case expression operands
+    // sub-queries as case expression operands
     checkExp(
         "case (select * from emp) when 1 then 2 end",
         "(CASE WHEN ((SELECT *\n"
@@ -3716,7 +3716,7 @@ public class SqlParserTest {
         "(?s)Encountered \"w1\" at.*");
   }
 
-  @Test public void testWindowInSubquery() {
+  @Test public void testWindowInSubQuery() {
     check(
         "select * from ( select sum(x) over w, sum(y) over w from s window w as (range interval '1' minute preceding))",
         "SELECT *\n"
@@ -3761,7 +3761,7 @@ public class SqlParserTest {
         "select count(z) over w as foo from Bids window w as (order by x ^partition^ by y)",
         "(?s).*Encountered \"partition\".*");
 
-    // Cannot partition by subquery
+    // Cannot partition by sub-query
     checkFails(
         "select sum(a) over (partition by ^(^select 1 from t), x) from t2",
         "Query expression encountered in illegal context");

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/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 f6d81f4..326a460 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
@@ -816,7 +816,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
   @Test public void testSubQueryInWhere() {
     String sql;
 
-    // Aliases from enclosing subqueries are inherited: hence A from
+    // Aliases from enclosing sub-queries are inherited: hence A from
     // enclosing, B from same scope.
     // The raw columns from dept are suggested (because they can
     // be used unqualified in the inner scope) but the raw
@@ -988,7 +988,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     expected = "SELECT emp.empno FROM sales.emp ORDER BY _suggest_";
     assertSimplify(sql, expected);
 
-    // subquery in from
+    // sub-query in from
     sql =
         "select t.^ from (select 1 as x, 2 as y from sales.emp) as t "
             + "where t.dummy=1";
@@ -1025,10 +1025,10 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
         "SELECT t. _suggest_ FROM ( SELECT 0 AS x , 0 AS y FROM sales )";
     assertSimplify(sql, expected);
 
-    // subquery in where; note that:
-    // 1. removes the SELECT clause of subquery in WHERE clause;
-    // 2. keeps SELECT clause of subquery in FROM clause;
-    // 3. removes GROUP BY clause of subquery in FROM clause;
+    // sub-query in where; note that:
+    // 1. removes the SELECT clause of sub-query in WHERE clause;
+    // 2. keeps SELECT clause of sub-query in FROM clause;
+    // 3. removes GROUP BY clause of sub-query in FROM clause;
     // 4. removes SELECT clause of outer query.
     sql =
         "select x + y + 32 from "
@@ -1057,7 +1057,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
             + "WHERE substring ( a. _suggest_ FROM 3 for 6 ) = '1234'";
     assertSimplify(sql, expected);
 
-    // missing ')' following subquery
+    // missing ')' following sub-query
     sql =
         "select * from sales.emp a where deptno in ("
             + "select * from sales.dept b where ^";

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
index ef60d10..de58609 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
@@ -1834,8 +1834,8 @@ public abstract class SqlOperatorBaseTest {
         "1",
         0);
 
-    // Check return type on scalar subquery in select list.  Note return
-    // type is always nullable even if subquery select value is NOT NULL.
+    // Check return type on scalar sub-query in select list.  Note return
+    // type is always nullable even if sub-query select value is NOT NULL.
     // Bug FRG-189 causes this test to fail only in SqlOperatorTest; not
     // in subtypes.
     if (Bug.FRG189_FIXED

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
index f66fd86..e414b0e 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
@@ -207,12 +207,12 @@ public class SqlPrettyWriterTest {
     checkSimple(prettyWriter, "${desc}", "${formatted}");
   }
 
-  @Test public void testDamiansSubqueryStyle() throws Exception {
+  @Test public void testDamiansSubQueryStyle() throws Exception {
     // Note that ( is at the indent, SELECT is on the same line, and ) is
     // below it.
     final SqlPrettyWriter prettyWriter =
         new SqlPrettyWriter(SqlDialect.DUMMY);
-    prettyWriter.setSubqueryStyle(SqlWriter.SubqueryStyle.BLACK);
+    prettyWriter.setSubQueryStyle(SqlWriter.SubQueryStyle.BLACK);
     checkSimple(prettyWriter, "${desc}", "${formatted}");
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
index ccff1c8..a2efb1e 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
@@ -539,7 +539,8 @@ public class JdbcAdapterTest {
     final String expected;
     switch (CalciteAssert.DB) {
     case MYSQL:
-      expected = "Subquery returns more than 1 row";
+      expected = "Sub"
+          + "query returns more than 1 row";
       break;
     default:
       expected = "more than one value in agg SINGLE_VALUE";
@@ -581,7 +582,7 @@ public class JdbcAdapterTest {
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-666">[CALCITE-666]
    * Anti-semi-joins against JDBC adapter give wrong results</a>. */
-  @Test public void testScalarSubquery() {
+  @Test public void testScalarSubQuery() {
     CalciteAssert.model(JdbcTest.SCOTT_MODEL)
         .query("SELECT COUNT(empno) AS cEmpNo FROM \"SCOTT\".\"EMP\" "
             + "WHERE DEPTNO <> (SELECT * FROM (VALUES 1))")

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 643eb09..b8bbe12 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -4723,7 +4723,7 @@ public class JdbcTest {
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-313">[CALCITE-313]
    * Query decorrelation fails</a>. */
-  @Test public void testJoinInCorrelatedSubquery() {
+  @Test public void testJoinInCorrelatedSubQuery() {
     CalciteAssert.hr()
         .query("select *\n"
             + "from \"hr\".\"depts\" as d\n"
@@ -4762,8 +4762,8 @@ public class JdbcTest {
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-559">[CALCITE-559]
-   * Correlated scalar subquery in WHERE gives error</a>. */
-  @Test public void testJoinCorrelatedScalarSubquery() throws SQLException {
+   * Correlated scalar sub-query in WHERE gives error</a>. */
+  @Test public void testJoinCorrelatedScalarSubQuery() throws SQLException {
     final String sql = "select e.employee_id, d.department_id "
         + " from employee e, department d "
         + " where e.department_id = d.department_id "

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/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 2749b7f..7aa6b33 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -179,7 +179,8 @@ public class RelOptRulesTest extends RelOptTestBase {
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-1479">[CALCITE-1479]
-   * AssertionError in ReduceExpressionsRule on multi-column IN subquery</a>. */
+   * AssertionError in ReduceExpressionsRule on multi-column IN
+   * sub-query</a>. */
   @Test public void testReduceCompositeInSubQuery() {
     final HepProgram hepProgram = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/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 4317b7d..14faa67 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -1927,11 +1927,12 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     // No conversion to join since less than IN-list size threshold 10
     SqlToRelConverter.Config noConvertConfig = SqlToRelConverter.configBuilder().
 
-        withInSubqueryThreshold(10).build();
+
+        withInSubQueryThreshold(10).build();
     sql(sql).withConfig(noConvertConfig).convertsTo("${planNotConverted}");
     // Conversion to join since greater than IN-list size threshold 2
     SqlToRelConverter.Config convertConfig = SqlToRelConverter.configBuilder().
-        withInSubqueryThreshold(2).build();
+        withInSubQueryThreshold(2).build();
     sql(sql).withConfig(convertConfig).convertsTo("${planConverted}");
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/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 78377f8..02d242b 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -4400,7 +4400,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
             "Duplicate window specification not allowed in the same window clause");
   }
 
-  @Test public void testWindowClauseWithSubquery() {
+  @Test public void testWindowClauseWithSubQuery() {
     check("select * from\n"
         + "( select sum(empno) over w, sum(deptno) over w from emp\n"
         + "window w as (order by hiredate range interval '1' minute preceding))");
@@ -4748,7 +4748,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         ERR_IN_OPERANDS_INCOMPATIBLE);
   }
 
-  @Test public void testInSubquery() {
+  @Test public void testInSubQuery() {
     check("select * from emp where deptno in (select deptno from dept)");
     check("select * from emp where (empno,deptno)"
         + " in (select deptno,deptno from dept)");
@@ -4800,7 +4800,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "select 1 from emp, dept, emp as e, ^dept as emp^, emp",
         "Duplicate relation name 'EMP' in FROM clause");
 
-    // alias applied to subquery
+    // alias applied to sub-query
     checkFails(
         "select 1 from emp, (^select 1 as x from (values (true))) as emp^",
         "Duplicate relation name 'EMP' in FROM clause");
@@ -5428,8 +5428,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         + "join (select 1 as job from (true)) using (job)", "ambig");
   }
 
-  @Ignore("bug: should fail if subquery does not have alias")
-  @Test public void testJoinSubquery() {
+  @Ignore("bug: should fail if sub-query does not have alias")
+  @Test public void testJoinSubQuery() {
     // Sub-queries require alias
     checkFails("select * from (select 1 as one from emp)\n"
             + "join (values (1), (2)) on true",
@@ -5616,7 +5616,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   /** Tests the {@code WITH} clause in sub-queries. */
-  @Test public void testWithSubquery() {
+  @Test public void testWithSubQuery() {
     // nested WITH (parentheses required - and even with parentheses SQL
     // standard doesn't allow sub-query to have WITH)
     checkResultType("with emp2 as (select * from emp)\n"
@@ -5863,7 +5863,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         // invalid in oracle and pre-99
         conformance.isSortByOrdinal() ? "Ordinal out of range" : null);
 
-    // Sort by scalar subquery
+    // Sort by scalar sub-query
     check("select * from emp\n"
         + "order by (select name from dept where deptno = emp.deptno)");
     checkFails(
@@ -6155,7 +6155,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     check("select * from (select empno,deptno from emp) group by deptno,empno");
 
     // This query tries to reference an agg expression from within a
-    // subquery as a correlating expression, but the SQL syntax rules say
+    // sub-query as a correlating expression, but the SQL syntax rules say
     // that the agg function SUM always applies to the current scope.
     // As it happens, the query is valid.
     check("select deptno\n"
@@ -6163,7 +6163,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         + "group by deptno\n"
         + "having exists (select sum(emp.sal) > 10 from (values(true)))");
 
-    // if you reference a column from a subquery, it must be a group col
+    // if you reference a column from a sub-query, it must be a group col
     check("select deptno "
         + "from emp "
         + "group by deptno "
@@ -6283,7 +6283,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
   // todo: enable when correlating variables work
   public void _testGroupExpressionEquivalenceCorrelated() {
-    // dname comes from dept, so it is constant within the subquery, and
+    // dname comes from dept, so it is constant within the sub-query, and
     // is so is a valid expr in a group-by query
     check("select * from dept where exists ("
         + "select dname from emp group by empno)");
@@ -7187,11 +7187,12 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "SELECT  ename, 1 + (select deptno from dept where deptno=1) as X FROM emp",
         "RecordType(VARCHAR(20) NOT NULL ENAME, INTEGER X) NOT NULL");
 
-    // scalar subquery inside WHERE
+    // scalar sub-query inside WHERE
     check("select * from emp where (select true from dept)");
   }
 
-  public void _testSubqueryInOnClause() {
+  @Ignore("not supported")
+  @Test public void testSubQueryInOnClause() {
     // Currently not supported. Should give validator error, but gives
     // internal error.
     check("select * from emp as emps left outer join dept as depts\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/core/src/test/resources/org/apache/calcite/sql/test/SqlPrettyWriterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/sql/test/SqlPrettyWriterTest.xml b/core/src/test/resources/org/apache/calcite/sql/test/SqlPrettyWriterTest.xml
index 38d4240..472061d 100644
--- a/core/src/test/resources/org/apache/calcite/sql/test/SqlPrettyWriterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/sql/test/SqlPrettyWriterTest.xml
@@ -154,7 +154,7 @@ FROM (SELECT *
 ORDER BY GG]]>
     </Resource>
   </TestCase>
-  <TestCase name="testDamiansSubqueryStyle">
+  <TestCase name="testDamiansSubQueryStyle">
     <Resource name="desc"/>
     <Resource name="formatted">
       <![CDATA[SELECT `X` AS `A`, `B` AS `B`, `C` AS `C`, `D`, 'mixed-Case string', `UNQUOTEDCAMELCASEID`, `quoted id`

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/core/src/test/resources/sql/misc.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.iq b/core/src/test/resources/sql/misc.iq
index 972b73c..841966c 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -626,7 +626,7 @@ EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[=($t0, $t2)], proj#0..1
   EnumerableTableScan(table=[[foodmart2, days]])
 !plan
 
-# [HIVE-5873] Semi-join to count subquery
+# [HIVE-5873] Semi-join to count sub-query
 # [CALCITE-365] AssertionError while translating query with WITH and correlated sub-query
 !if (false) {
 with parts (PNum, OrderOnHand)
@@ -704,8 +704,8 @@ group by case when a=1 then 1 else 2 end;
 
 !ok
 
-# [DERBY-4450] GROUP BY in an IN-subquery inside HAVING clause whose select list
-# is subset of group by columns.
+# [DERBY-4450] GROUP BY in an IN-sub-query inside HAVING clause whose select
+# list is subset of group by columns.
 select sum("day") from "days" group by "week_day" having "week_day" in (
   select "week_day" from "days" group by "week_day", "day");
 +--------+

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/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
new file mode 100644
index 0000000..ec42c0c
--- /dev/null
+++ b/core/src/test/resources/sql/sub-query.iq
@@ -0,0 +1,513 @@
+# sub-query.iq - Queries involving IN and EXISTS sub-queries
+#
+# 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 post
+!set outputformat psql
+
+# [CALCITE-373]
+# the following should return no rows, because the IN list has a null.
+# for details on this: see HIVE-784, Dayal's paper from VLDB-87
+with
+t1(x) as (select * from  (values 1,2, case when 1 = 1 then null else 3 end)),
+t2(x) as (select * from  (values 1,case when 1 = 1 then null else 3 end))
+select *
+from t1
+where t1.x not in (select t2.x from t2);
+ X
+---
+(0 rows)
+
+!ok
+EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t1, $t5)], expr#7=[false], expr#8=[IS NOT NULL($t4)], expr#9=[true], expr#10=[IS NULL($t0)], expr#11=[null], expr#12=[<($t2, $t1)], expr#13=[CASE($t6, $t7, $t8, $t9, $t10, $t11, $t12, $t9, $t7)], expr#14=[NOT($t13)], EXPR$0=[$t0], $condition=[$t14])
+  EnumerableJoin(condition=[=($0, $3)], joinType=[left])
+    EnumerableJoin(condition=[true], joinType=[inner])
+      EnumerableUnion(all=[true])
+        EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
+          EnumerableValues(tuples=[[{ 0 }]])
+        EnumerableCalc(expr#0=[{inputs}], expr#1=[2], EXPR$0=[$t1])
+          EnumerableValues(tuples=[[{ 0 }]])
+        EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
+          EnumerableValues(tuples=[[{ 0 }]])
+      EnumerableAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)])
+        EnumerableUnion(all=[true])
+          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
+            EnumerableValues(tuples=[[{ 0 }]])
+          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
+            EnumerableValues(tuples=[[{ 0 }]])
+    EnumerableAggregate(group=[{0, 1}])
+      EnumerableCalc(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])
+        EnumerableUnion(all=[true])
+          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
+            EnumerableValues(tuples=[[{ 0 }]])
+          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
+            EnumerableValues(tuples=[[{ 0 }]])
+!plan
+
+# Use of case is to get around issue with directly specifying null in values
+# list. Postgres gives 0 rows.
+with
+t1(x) as (select * from  (values (1),(2),(case when 1 = 1 then null else 3 end)) as t1),
+t2(x) as (select * from  (values (1),(case when 1 = 1 then null else 3 end)) as t2)
+select *
+from t1
+where t1.x not in (select t2.x from t2);
+
+ X
+---
+(0 rows)
+
+!ok
+
+# RHS has a mixture of NULL and NOT NULL keys
+select * from dept where deptno not in (select deptno from emp);
+ DEPTNO | DNAME
+--------+-------
+(0 rows)
+
+!ok
+select deptno, deptno     in (select deptno from emp) from dept;
+ DEPTNO | EXPR$1
+--------+--------
+     10 | true
+     20 | true
+     30 | true
+     40 | null
+(4 rows)
+
+!ok
+select deptno, deptno not in (select deptno from emp) from dept;
+ DEPTNO | EXPR$1
+--------+--------
+     10 | false
+     20 | false
+     30 | false
+     40 | null
+(4 rows)
+
+!ok
+
+# RHS has only NULL keys
+select * from dept where deptno not in (select deptno from emp where deptno is null);
+ DEPTNO | DNAME
+--------+-------
+(0 rows)
+
+!ok
+select deptno, deptno     in (select deptno from emp where deptno is null) from dept;
+ DEPTNO | EXPR$1
+--------+--------
+     10 | null
+     20 | null
+     30 | null
+     40 | null
+(4 rows)
+
+!ok
+select deptno, deptno not in (select deptno from emp where deptno is null) from dept;
+ DEPTNO | EXPR$1
+--------+--------
+     10 | null
+     20 | null
+     30 | null
+     40 | null
+(4 rows)
+
+!ok
+
+!set outputformat mysql
+
+# RHS has only NOT NULL keys
+select * from dept where deptno not in (select deptno from emp where deptno is not null);
++--------+-------------+
+| DEPTNO | DNAME       |
++--------+-------------+
+|     40 | Empty       |
++--------+-------------+
+(1 row)
+
+!ok
+select deptno, deptno     in (select deptno from emp where deptno is not null) from dept;
++--------+--------+
+| DEPTNO | EXPR$1 |
++--------+--------+
+|     10 | true   |
+|     20 | true   |
+|     30 | true   |
+|     40 | false  |
++--------+--------+
+(4 rows)
+
+!ok
+select deptno, deptno not in (select deptno from emp where deptno is not null) from dept;
++--------+--------+
+| DEPTNO | EXPR$1 |
++--------+--------+
+|     10 | false  |
+|     20 | false  |
+|     30 | false  |
+|     40 | true   |
++--------+--------+
+(4 rows)
+
+!ok
+
+# RHS has no rows
+# Even 'NULL NOT IN ...' is TRUE.
+select * from dept where deptno not in (select deptno from emp where false);
++--------+-------------+
+| DEPTNO | DNAME       |
++--------+-------------+
+|     10 | Sales       |
+|     20 | Marketing   |
+|     30 | Engineering |
+|     40 | Empty       |
++--------+-------------+
+(4 rows)
+
+!ok
+select deptno, deptno     in (select deptno from emp where false) from dept;
++--------+--------+
+| DEPTNO | EXPR$1 |
++--------+--------+
+|     10 | false  |
+|     20 | false  |
+|     30 | false  |
+|     40 | false  |
++--------+--------+
+(4 rows)
+
+!ok
+select deptno, deptno not in (select deptno from emp where false) from dept;
++--------+--------+
+| DEPTNO | EXPR$1 |
++--------+--------+
+|     10 | true   |
+|     20 | true   |
+|     30 | true   |
+|     40 | true   |
++--------+--------+
+(4 rows)
+
+!ok
+
+# Multiple IN, connected by OR
+select * from dept
+where deptno in (select deptno from emp where gender = 'F')
+or deptno in (select deptno from emp where gender = 'M');
++--------+-------------+
+| DEPTNO | DNAME       |
++--------+-------------+
+|     10 | Sales       |
+|     20 | Marketing   |
+|     30 | Engineering |
++--------+-------------+
+(3 rows)
+
+!ok
+
+# Mix IN and EXISTS
+select * from dept
+where deptno in (select deptno from emp where gender = 'F')
+or exists (select 99, 101 from emp where gender = 'X');
++--------+-------------+
+| DEPTNO | DNAME       |
++--------+-------------+
+|     10 | Sales       |
+|     30 | Engineering |
++--------+-------------+
+(2 rows)
+
+!ok
+
+# Composite key
+select * from dept
+where (deptno, deptno) in (select deptno * 2 - deptno, deptno from emp where gender = 'F');
+
+# Composite key, part literal
+select * from emp
+where (gender, deptno) in (select gender, 10 from emp where gender = 'F');
++-------+--------+--------+
+| ENAME | DEPTNO | GENDER |
++-------+--------+--------+
+| Jane  |     10 | F      |
++-------+--------+--------+
+(1 row)
+
+!ok
+
+!use scott
+
+# [CALCITE-1155] Support columns for IN list
+SELECT empno, ename, mgr FROM "scott".emp WHERE 7782 IN (empno, mgr);
++-------+--------+------+
+| EMPNO | ENAME  | MGR  |
++-------+--------+------+
+|  7782 | CLARK  | 7839 |
+|  7934 | MILLER | 7782 |
++-------+--------+------+
+(2 rows)
+
+!ok
+
+# [CALCITE-694] Scan HAVING clause for sub-queries and IN-lists
+SELECT count(*) AS c
+FROM "scott".emp
+GROUP BY emp.deptno
+HAVING sum(case when emp.empno in (7369, 7839, 7902) then emp.sal else 0 end)
+     BETWEEN 5000.0 AND 10000.0;
++---+
+| C |
++---+
+| 3 |
++---+
+(1 row)
+
+!ok
+
+# [CALCITE-716] Scalar sub-query and aggregate function in SELECT or HAVING
+# clause gives AssertionError
+SELECT emp.deptno
+FROM "scott".emp
+GROUP BY emp.deptno
+HAVING max(emp.empno) > (SELECT min(emp.empno) FROM "scott".emp);
++--------+
+| DEPTNO |
++--------+
+|     10 |
+|     20 |
+|     30 |
++--------+
+(3 rows)
+
+!ok
+
+# [CALCITE-716] Scalar sub-query and aggregate function in SELECT or HAVING
+# clause gives AssertionError
+SELECT emp.deptno,
+  max(emp.empno) > (SELECT min(emp.empno) FROM "scott".emp) as bbbb
+FROM "scott".emp
+GROUP BY emp.deptno;
++--------+------+
+| DEPTNO | BBBB |
++--------+------+
+|     10 | true |
+|     20 | true |
+|     30 | true |
++--------+------+
+(3 rows)
+
+!ok
+
+# [DRILL-4407] Group by sub-query causes Java NPE
+select count(*) as c
+from "scott".emp
+group by (select deptno from "scott".emp where empno = 10);
++----+
+| C  |
++----+
+| 14 |
++----+
+(1 row)
+
+!ok
+
+!if (fixed.calcite1045) {
+# Correlated IN sub-query in WHERE clause of JOIN
+select empno from "scott".emp as e
+join "scott".dept as d using (deptno)
+where e.job in (
+  select e2.job from "scott".emp as e2 where e2.deptno > e.deptno);
+ EMPNO
+-------
+  7369
+  7566
+  7782
+  7876
+  7934
+(5 rows)
+
+!ok
+EnumerableCalc(expr#0..5=[{inputs}], EMPNO=[$t0])
+  EnumerableJoin(condition=[=($2, $5)], joinType=[inner])
+    EnumerableCalc(expr#0..4=[{inputs}], EMPNO=[$t2], JOB=[$t3], DEPTNO=[$t4], JOB0=[$t0], DEPTNO0=[$t1])
+      EnumerableJoin(condition=[AND(=($1, $4), =($0, $3))], joinType=[inner])
+        EnumerableCalc(expr#0..1=[{inputs}], JOB=[$t1], DEPTNO=[$t0])
+          EnumerableAggregate(group=[{0, 2}])
+            EnumerableCalc(expr#0..3=[{inputs}], expr#4=[>($t3, $t0)], proj#0..3=[{exprs}], $condition=[$t4])
+              EnumerableJoin(condition=[true], joinType=[inner])
+                EnumerableAggregate(group=[{7}])
+                  EnumerableTableScan(table=[[scott, EMP]])
+                EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], JOB=[$t2], DEPTNO=[$t7])
+                  EnumerableTableScan(table=[[scott, EMP]])
+        EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], JOB=[$t2], DEPTNO=[$t7])
+          EnumerableTableScan(table=[[scott, EMP]])
+    EnumerableCalc(expr#0..2=[{inputs}], DEPTNO=[$t0])
+      EnumerableTableScan(table=[[scott, DEPT]])
+!plan
+!}
+
+!if (fixed.calcite1045) {
+# Correlated NOT IN sub-query in WHERE clause of JOIN
+select empno from "scott".emp as e
+join "scott".dept as d using (deptno)
+where e.job not in (
+  select e2.job from "scott".emp as e2 where e2.deptno > e.deptno);
+!ok
+!plan
+!}
+
+# Condition that returns a NULL key.
+# Tested on Oracle.
+select count(*) as c
+from "scott".emp
+where sal + 100 not in (
+  select comm
+  from "scott".emp);
++---+
+| C |
++---+
+| 0 |
++---+
+(1 row)
+
+!ok
+
+# Condition that happens to eliminate all NULL keys.
+# The one missing row has {ename: 'MARTIN', comm: 1400}
+# Tested on Oracle.
+select count(*) as c
+from "scott".emp
+where sal + 100 not in (
+  select comm from "scott".emp
+  where job = 'SALESMAN');
++----+
+| C  |
++----+
+| 13 |
++----+
+(1 row)
+
+!ok
+
+# Condition that provably eliminates all NULL keys.
+# Tested on Oracle.
+select count(*) as c
+from "scott".emp
+where sal + 100 not in (
+  select comm
+  from "scott".emp
+  where comm < 1000);
++----+
+| C  |
++----+
+| 14 |
++----+
+(1 row)
+
+!ok
+
+# Correlated condition in NOT IN.
+# Tested on Oracle.
+!if (fixed.calcite1513) {
+select count(*) as c
+from "scott".emp as e
+where sal + 100 not in (
+  select comm
+  from "scott".emp
+  where job = e.job);
+     EMPNO ENAME      JOB              MGR HIREDATE         SAL       COMM     DEPTNO
+---------- ---------- --------- ---------- --------- ---------- ---------- ----------
+      7499 ALLEN      SALESMAN        7698 20-FEB-81       1600        300         30
+      7521 WARD       SALESMAN        7698 22-FEB-81       1250        500         30
+      7654 MARTIN     SALESMAN        7698 28-SEP-81       1250       1400         30
+      7844 TURNER     SALESMAN        7698 08-SEP-81       1500          0         30
+!ok
+!}
+
+# [CALCITE-864] Correlation variable has incorrect row type if it is populated
+# by right side of a Join
+select *
+from "scott".emp as e
+join "scott".dept as d using (deptno)
+where sal = (
+  select max(sal)
+  from "scott".emp as e2
+  join "scott".dept as d2 using (deptno)
+  where d2.deptno = d.deptno);
++-------+-------+-----------+------+------------+---------+------+--------+---------+------------+----------+
+| EMPNO | ENAME | JOB       | MGR  | HIREDATE   | SAL     | COMM | DEPTNO | DEPTNO0 | DNAME      | LOC      |
++-------+-------+-----------+------+------------+---------+------+--------+---------+------------+----------+
+|  7698 | BLAKE | MANAGER   | 7839 | 1981-01-05 | 2850.00 |      |     30 |      30 | SALES      | CHICAGO  |
+|  7788 | SCOTT | ANALYST   | 7566 | 1987-04-19 | 3000.00 |      |     20 |      20 | RESEARCH   | DALLAS   |
+|  7839 | KING  | PRESIDENT |      | 1981-11-17 | 5000.00 |      |     10 |      10 | ACCOUNTING | NEW YORK |
+|  7902 | FORD  | ANALYST   | 7566 | 1981-12-03 | 3000.00 |      |     20 |      20 | RESEARCH   | DALLAS   |
++-------+-------+-----------+------+------------+---------+------+--------+---------+------------+----------+
+(4 rows)
+
+!ok
+
+# Simpler test case for [CALCITE-864]
+select empno, ename, sal, e.deptno, loc
+from "scott".emp as e
+join "scott".dept as d using (deptno)
+where e.sal = (
+  select max(sal)
+  from "scott".emp as e2
+  where e2.deptno = e.deptno);
++-------+-------+---------+--------+----------+
+| EMPNO | ENAME | SAL     | DEPTNO | LOC      |
++-------+-------+---------+--------+----------+
+|  7698 | BLAKE | 2850.00 |     30 | CHICAGO  |
+|  7788 | SCOTT | 3000.00 |     20 | DALLAS   |
+|  7839 | KING  | 5000.00 |     10 | NEW YORK |
+|  7902 | FORD  | 3000.00 |     20 | DALLAS   |
++-------+-------+---------+--------+----------+
+(4 rows)
+
+!ok
+
+# Simpler test case for [CALCITE-864]
+select *
+from "scott".emp as e
+join "scott".dept as d using (deptno)
+where d.dname = (
+  select max(dname)
+  from "scott".dept as d2
+  where d2.deptno = d.deptno);
++-------+--------+-----------+------+------------+---------+---------+--------+---------+------------+----------+
+| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO | DEPTNO0 | DNAME      | LOC      |
++-------+--------+-----------+------+------------+---------+---------+--------+---------+------------+----------+
+|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 |      20 | RESEARCH   | DALLAS   |
+|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |      30 | SALES      | CHICAGO  |
+|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |      30 | SALES      | CHICAGO  |
+|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |      20 | RESEARCH   | DALLAS   |
+|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |      30 | SALES      | CHICAGO  |
+|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |      30 | SALES      | CHICAGO  |
+|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |      10 | ACCOUNTING | NEW YORK |
+|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |      20 | RESEARCH   | DALLAS   |
+|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |      10 | ACCOUNTING | NEW YORK |
+|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |      30 | SALES      | CHICAGO  |
+|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 |      20 | RESEARCH   | DALLAS   |
+|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 |      30 | SALES      | CHICAGO  |
+|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |      20 | RESEARCH   | DALLAS   |
+|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 |      10 | ACCOUNTING | NEW YORK |
++-------+--------+-----------+------+------------+---------+---------+--------+---------+------------+----------+
+(14 rows)
+
+!ok
+
+# End sub-query.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/core/src/test/resources/sql/subquery.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/subquery.iq b/core/src/test/resources/sql/subquery.iq
deleted file mode 100644
index 73ee2f9..0000000
--- a/core/src/test/resources/sql/subquery.iq
+++ /dev/null
@@ -1,513 +0,0 @@
-# subquery.iq - Queries involving IN and EXISTS sub-queries
-#
-# 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 post
-!set outputformat psql
-
-# [CALCITE-373]
-# the following should return no rows, because the IN list has a null.
-# for details on this: see HIVE-784, Dayal's paper from VLDB-87
-with
-t1(x) as (select * from  (values 1,2, case when 1 = 1 then null else 3 end)),
-t2(x) as (select * from  (values 1,case when 1 = 1 then null else 3 end))
-select *
-from t1
-where t1.x not in (select t2.x from t2);
- X
----
-(0 rows)
-
-!ok
-EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t1, $t5)], expr#7=[false], expr#8=[IS NOT NULL($t4)], expr#9=[true], expr#10=[IS NULL($t0)], expr#11=[null], expr#12=[<($t2, $t1)], expr#13=[CASE($t6, $t7, $t8, $t9, $t10, $t11, $t12, $t9, $t7)], expr#14=[NOT($t13)], EXPR$0=[$t0], $condition=[$t14])
-  EnumerableJoin(condition=[=($0, $3)], joinType=[left])
-    EnumerableJoin(condition=[true], joinType=[inner])
-      EnumerableUnion(all=[true])
-        EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
-          EnumerableValues(tuples=[[{ 0 }]])
-        EnumerableCalc(expr#0=[{inputs}], expr#1=[2], EXPR$0=[$t1])
-          EnumerableValues(tuples=[[{ 0 }]])
-        EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
-          EnumerableValues(tuples=[[{ 0 }]])
-      EnumerableAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)])
-        EnumerableUnion(all=[true])
-          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
-            EnumerableValues(tuples=[[{ 0 }]])
-          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
-            EnumerableValues(tuples=[[{ 0 }]])
-    EnumerableAggregate(group=[{0, 1}])
-      EnumerableCalc(expr#0=[{inputs}], expr#1=[true], proj#0..1=[{exprs}])
-        EnumerableUnion(all=[true])
-          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
-            EnumerableValues(tuples=[[{ 0 }]])
-          EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[=($t1, $t1)], expr#3=[null], expr#4=[3], expr#5=[CASE($t2, $t3, $t4)], EXPR$0=[$t5])
-            EnumerableValues(tuples=[[{ 0 }]])
-!plan
-
-# Use of case is to get around issue with directly specifying null in values
-# list. Postgres gives 0 rows.
-with
-t1(x) as (select * from  (values (1),(2),(case when 1 = 1 then null else 3 end)) as t1),
-t2(x) as (select * from  (values (1),(case when 1 = 1 then null else 3 end)) as t2)
-select *
-from t1
-where t1.x not in (select t2.x from t2);
-
- X
----
-(0 rows)
-
-!ok
-
-# RHS has a mixture of NULL and NOT NULL keys
-select * from dept where deptno not in (select deptno from emp);
- DEPTNO | DNAME
---------+-------
-(0 rows)
-
-!ok
-select deptno, deptno     in (select deptno from emp) from dept;
- DEPTNO | EXPR$1
---------+--------
-     10 | true
-     20 | true
-     30 | true
-     40 | null
-(4 rows)
-
-!ok
-select deptno, deptno not in (select deptno from emp) from dept;
- DEPTNO | EXPR$1
---------+--------
-     10 | false
-     20 | false
-     30 | false
-     40 | null
-(4 rows)
-
-!ok
-
-# RHS has only NULL keys
-select * from dept where deptno not in (select deptno from emp where deptno is null);
- DEPTNO | DNAME
---------+-------
-(0 rows)
-
-!ok
-select deptno, deptno     in (select deptno from emp where deptno is null) from dept;
- DEPTNO | EXPR$1
---------+--------
-     10 | null
-     20 | null
-     30 | null
-     40 | null
-(4 rows)
-
-!ok
-select deptno, deptno not in (select deptno from emp where deptno is null) from dept;
- DEPTNO | EXPR$1
---------+--------
-     10 | null
-     20 | null
-     30 | null
-     40 | null
-(4 rows)
-
-!ok
-
-!set outputformat mysql
-
-# RHS has only NOT NULL keys
-select * from dept where deptno not in (select deptno from emp where deptno is not null);
-+--------+-------------+
-| DEPTNO | DNAME       |
-+--------+-------------+
-|     40 | Empty       |
-+--------+-------------+
-(1 row)
-
-!ok
-select deptno, deptno     in (select deptno from emp where deptno is not null) from dept;
-+--------+--------+
-| DEPTNO | EXPR$1 |
-+--------+--------+
-|     10 | true   |
-|     20 | true   |
-|     30 | true   |
-|     40 | false  |
-+--------+--------+
-(4 rows)
-
-!ok
-select deptno, deptno not in (select deptno from emp where deptno is not null) from dept;
-+--------+--------+
-| DEPTNO | EXPR$1 |
-+--------+--------+
-|     10 | false  |
-|     20 | false  |
-|     30 | false  |
-|     40 | true   |
-+--------+--------+
-(4 rows)
-
-!ok
-
-# RHS has no rows
-# Even 'NULL NOT IN ...' is TRUE.
-select * from dept where deptno not in (select deptno from emp where false);
-+--------+-------------+
-| DEPTNO | DNAME       |
-+--------+-------------+
-|     10 | Sales       |
-|     20 | Marketing   |
-|     30 | Engineering |
-|     40 | Empty       |
-+--------+-------------+
-(4 rows)
-
-!ok
-select deptno, deptno     in (select deptno from emp where false) from dept;
-+--------+--------+
-| DEPTNO | EXPR$1 |
-+--------+--------+
-|     10 | false  |
-|     20 | false  |
-|     30 | false  |
-|     40 | false  |
-+--------+--------+
-(4 rows)
-
-!ok
-select deptno, deptno not in (select deptno from emp where false) from dept;
-+--------+--------+
-| DEPTNO | EXPR$1 |
-+--------+--------+
-|     10 | true   |
-|     20 | true   |
-|     30 | true   |
-|     40 | true   |
-+--------+--------+
-(4 rows)
-
-!ok
-
-# Multiple IN, connected by OR
-select * from dept
-where deptno in (select deptno from emp where gender = 'F')
-or deptno in (select deptno from emp where gender = 'M');
-+--------+-------------+
-| DEPTNO | DNAME       |
-+--------+-------------+
-|     10 | Sales       |
-|     20 | Marketing   |
-|     30 | Engineering |
-+--------+-------------+
-(3 rows)
-
-!ok
-
-# Mix IN and EXISTS
-select * from dept
-where deptno in (select deptno from emp where gender = 'F')
-or exists (select 99, 101 from emp where gender = 'X');
-+--------+-------------+
-| DEPTNO | DNAME       |
-+--------+-------------+
-|     10 | Sales       |
-|     30 | Engineering |
-+--------+-------------+
-(2 rows)
-
-!ok
-
-# Composite key
-select * from dept
-where (deptno, deptno) in (select deptno * 2 - deptno, deptno from emp where gender = 'F');
-
-# Composite key, part literal
-select * from emp
-where (gender, deptno) in (select gender, 10 from emp where gender = 'F');
-+-------+--------+--------+
-| ENAME | DEPTNO | GENDER |
-+-------+--------+--------+
-| Jane  |     10 | F      |
-+-------+--------+--------+
-(1 row)
-
-!ok
-
-!use scott
-
-# [CALCITE-1155] Support columns for IN list
-SELECT empno, ename, mgr FROM "scott".emp WHERE 7782 IN (empno, mgr);
-+-------+--------+------+
-| EMPNO | ENAME  | MGR  |
-+-------+--------+------+
-|  7782 | CLARK  | 7839 |
-|  7934 | MILLER | 7782 |
-+-------+--------+------+
-(2 rows)
-
-!ok
-
-# [CALCITE-694] Scan HAVING clause for sub-queries and IN-lists
-SELECT count(*) AS c
-FROM "scott".emp
-GROUP BY emp.deptno
-HAVING sum(case when emp.empno in (7369, 7839, 7902) then emp.sal else 0 end)
-     BETWEEN 5000.0 AND 10000.0;
-+---+
-| C |
-+---+
-| 3 |
-+---+
-(1 row)
-
-!ok
-
-# [CALCITE-716] Scalar sub-query and aggregate function in SELECT or HAVING
-# clause gives AssertionError
-SELECT emp.deptno
-FROM "scott".emp
-GROUP BY emp.deptno
-HAVING max(emp.empno) > (SELECT min(emp.empno) FROM "scott".emp);
-+--------+
-| DEPTNO |
-+--------+
-|     10 |
-|     20 |
-|     30 |
-+--------+
-(3 rows)
-
-!ok
-
-# [CALCITE-716] Scalar sub-query and aggregate function in SELECT or HAVING
-# clause gives AssertionError
-SELECT emp.deptno,
-  max(emp.empno) > (SELECT min(emp.empno) FROM "scott".emp) as bbbb
-FROM "scott".emp
-GROUP BY emp.deptno;
-+--------+------+
-| DEPTNO | BBBB |
-+--------+------+
-|     10 | true |
-|     20 | true |
-|     30 | true |
-+--------+------+
-(3 rows)
-
-!ok
-
-# [DRILL-4407] Group by subquery causes Java NPE
-select count(*) as c
-from "scott".emp
-group by (select deptno from "scott".emp where empno = 10);
-+----+
-| C  |
-+----+
-| 14 |
-+----+
-(1 row)
-
-!ok
-
-!if (fixed.calcite1045) {
-# Correlated IN sub-query in WHERE clause of JOIN
-select empno from "scott".emp as e
-join "scott".dept as d using (deptno)
-where e.job in (
-  select e2.job from "scott".emp as e2 where e2.deptno > e.deptno);
- EMPNO
--------
-  7369
-  7566
-  7782
-  7876
-  7934
-(5 rows)
-
-!ok
-EnumerableCalc(expr#0..5=[{inputs}], EMPNO=[$t0])
-  EnumerableJoin(condition=[=($2, $5)], joinType=[inner])
-    EnumerableCalc(expr#0..4=[{inputs}], EMPNO=[$t2], JOB=[$t3], DEPTNO=[$t4], JOB0=[$t0], DEPTNO0=[$t1])
-      EnumerableJoin(condition=[AND(=($1, $4), =($0, $3))], joinType=[inner])
-        EnumerableCalc(expr#0..1=[{inputs}], JOB=[$t1], DEPTNO=[$t0])
-          EnumerableAggregate(group=[{0, 2}])
-            EnumerableCalc(expr#0..3=[{inputs}], expr#4=[>($t3, $t0)], proj#0..3=[{exprs}], $condition=[$t4])
-              EnumerableJoin(condition=[true], joinType=[inner])
-                EnumerableAggregate(group=[{7}])
-                  EnumerableTableScan(table=[[scott, EMP]])
-                EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], JOB=[$t2], DEPTNO=[$t7])
-                  EnumerableTableScan(table=[[scott, EMP]])
-        EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], JOB=[$t2], DEPTNO=[$t7])
-          EnumerableTableScan(table=[[scott, EMP]])
-    EnumerableCalc(expr#0..2=[{inputs}], DEPTNO=[$t0])
-      EnumerableTableScan(table=[[scott, DEPT]])
-!plan
-!}
-
-!if (fixed.calcite1045) {
-# Correlated NOT IN sub-query in WHERE clause of JOIN
-select empno from "scott".emp as e
-join "scott".dept as d using (deptno)
-where e.job not in (
-  select e2.job from "scott".emp as e2 where e2.deptno > e.deptno);
-!ok
-!plan
-!}
-
-# Condition that returns a NULL key.
-# Tested on Oracle.
-select count(*) as c
-from "scott".emp
-where sal + 100 not in (
-  select comm
-  from "scott".emp);
-+---+
-| C |
-+---+
-| 0 |
-+---+
-(1 row)
-
-!ok
-
-# Condition that happens to eliminate all NULL keys.
-# The one missing row has {ename: 'MARTIN', comm: 1400}
-# Tested on Oracle.
-select count(*) as c
-from "scott".emp
-where sal + 100 not in (
-  select comm from "scott".emp
-  where job = 'SALESMAN');
-+----+
-| C  |
-+----+
-| 13 |
-+----+
-(1 row)
-
-!ok
-
-# Condition that provably eliminates all NULL keys.
-# Tested on Oracle.
-select count(*) as c
-from "scott".emp
-where sal + 100 not in (
-  select comm
-  from "scott".emp
-  where comm < 1000);
-+----+
-| C  |
-+----+
-| 14 |
-+----+
-(1 row)
-
-!ok
-
-# Correlated condition in NOT IN.
-# Tested on Oracle.
-!if (fixed.calcite1513) {
-select count(*) as c
-from "scott".emp as e
-where sal + 100 not in (
-  select comm
-  from "scott".emp
-  where job = e.job);
-     EMPNO ENAME      JOB              MGR HIREDATE         SAL       COMM     DEPTNO
----------- ---------- --------- ---------- --------- ---------- ---------- ----------
-      7499 ALLEN      SALESMAN        7698 20-FEB-81       1600        300         30
-      7521 WARD       SALESMAN        7698 22-FEB-81       1250        500         30
-      7654 MARTIN     SALESMAN        7698 28-SEP-81       1250       1400         30
-      7844 TURNER     SALESMAN        7698 08-SEP-81       1500          0         30
-!ok
-!}
-
-# [CALCITE-864] Correlation variable has incorrect row type if it is populated
-# by right side of a Join
-select *
-from "scott".emp as e
-join "scott".dept as d using (deptno)
-where sal = (
-  select max(sal)
-  from "scott".emp as e2
-  join "scott".dept as d2 using (deptno)
-  where d2.deptno = d.deptno);
-+-------+-------+-----------+------+------------+---------+------+--------+---------+------------+----------+
-| EMPNO | ENAME | JOB       | MGR  | HIREDATE   | SAL     | COMM | DEPTNO | DEPTNO0 | DNAME      | LOC      |
-+-------+-------+-----------+------+------------+---------+------+--------+---------+------------+----------+
-|  7698 | BLAKE | MANAGER   | 7839 | 1981-01-05 | 2850.00 |      |     30 |      30 | SALES      | CHICAGO  |
-|  7788 | SCOTT | ANALYST   | 7566 | 1987-04-19 | 3000.00 |      |     20 |      20 | RESEARCH   | DALLAS   |
-|  7839 | KING  | PRESIDENT |      | 1981-11-17 | 5000.00 |      |     10 |      10 | ACCOUNTING | NEW YORK |
-|  7902 | FORD  | ANALYST   | 7566 | 1981-12-03 | 3000.00 |      |     20 |      20 | RESEARCH   | DALLAS   |
-+-------+-------+-----------+------+------------+---------+------+--------+---------+------------+----------+
-(4 rows)
-
-!ok
-
-# Simpler test case for [CALCITE-864]
-select empno, ename, sal, e.deptno, loc
-from "scott".emp as e
-join "scott".dept as d using (deptno)
-where e.sal = (
-  select max(sal)
-  from "scott".emp as e2
-  where e2.deptno = e.deptno);
-+-------+-------+---------+--------+----------+
-| EMPNO | ENAME | SAL     | DEPTNO | LOC      |
-+-------+-------+---------+--------+----------+
-|  7698 | BLAKE | 2850.00 |     30 | CHICAGO  |
-|  7788 | SCOTT | 3000.00 |     20 | DALLAS   |
-|  7839 | KING  | 5000.00 |     10 | NEW YORK |
-|  7902 | FORD  | 3000.00 |     20 | DALLAS   |
-+-------+-------+---------+--------+----------+
-(4 rows)
-
-!ok
-
-# Simpler test case for [CALCITE-864]
-select *
-from "scott".emp as e
-join "scott".dept as d using (deptno)
-where d.dname = (
-  select max(dname)
-  from "scott".dept as d2
-  where d2.deptno = d.deptno);
-+-------+--------+-----------+------+------------+---------+---------+--------+---------+------------+----------+
-| EMPNO | ENAME  | JOB       | MGR  | HIREDATE   | SAL     | COMM    | DEPTNO | DEPTNO0 | DNAME      | LOC      |
-+-------+--------+-----------+------+------------+---------+---------+--------+---------+------------+----------+
-|  7369 | SMITH  | CLERK     | 7902 | 1980-12-17 |  800.00 |         |     20 |      20 | RESEARCH   | DALLAS   |
-|  7499 | ALLEN  | SALESMAN  | 7698 | 1981-02-20 | 1600.00 |  300.00 |     30 |      30 | SALES      | CHICAGO  |
-|  7521 | WARD   | SALESMAN  | 7698 | 1981-02-22 | 1250.00 |  500.00 |     30 |      30 | SALES      | CHICAGO  |
-|  7566 | JONES  | MANAGER   | 7839 | 1981-02-04 | 2975.00 |         |     20 |      20 | RESEARCH   | DALLAS   |
-|  7654 | MARTIN | SALESMAN  | 7698 | 1981-09-28 | 1250.00 | 1400.00 |     30 |      30 | SALES      | CHICAGO  |
-|  7698 | BLAKE  | MANAGER   | 7839 | 1981-01-05 | 2850.00 |         |     30 |      30 | SALES      | CHICAGO  |
-|  7782 | CLARK  | MANAGER   | 7839 | 1981-06-09 | 2450.00 |         |     10 |      10 | ACCOUNTING | NEW YORK |
-|  7788 | SCOTT  | ANALYST   | 7566 | 1987-04-19 | 3000.00 |         |     20 |      20 | RESEARCH   | DALLAS   |
-|  7839 | KING   | PRESIDENT |      | 1981-11-17 | 5000.00 |         |     10 |      10 | ACCOUNTING | NEW YORK |
-|  7844 | TURNER | SALESMAN  | 7698 | 1981-09-08 | 1500.00 |    0.00 |     30 |      30 | SALES      | CHICAGO  |
-|  7876 | ADAMS  | CLERK     | 7788 | 1987-05-23 | 1100.00 |         |     20 |      20 | RESEARCH   | DALLAS   |
-|  7900 | JAMES  | CLERK     | 7698 | 1981-12-03 |  950.00 |         |     30 |      30 | SALES      | CHICAGO  |
-|  7902 | FORD   | ANALYST   | 7566 | 1981-12-03 | 3000.00 |         |     20 |      20 | RESEARCH   | DALLAS   |
-|  7934 | MILLER | CLERK     | 7782 | 1982-01-23 | 1300.00 |         |     10 |      10 | ACCOUNTING | NEW YORK |
-+-------+--------+-----------+------+------------+---------+---------+--------+---------+------------+----------+
-(14 rows)
-
-!ok
-
-# End subquery.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
----------------------------------------------------------------------
diff --git a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
index 9f65fc5..6c381e5 100644
--- a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
+++ b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
@@ -408,12 +408,11 @@ public class CsvTest {
     final String sql = "SELECT e.name\n"
         + "FROM emps AS e\n"
         + "WHERE cast(e.empno as bigint) in ";
-    checkSql(sql + range(130, SqlToRelConverter.DEFAULT_IN_SUBQUERY_THRESHOLD - 5),
-        "smart", expect("NAME=Alice"));
-    checkSql(sql + range(130, SqlToRelConverter.DEFAULT_IN_SUBQUERY_THRESHOLD),
-        "smart", expect("NAME=Alice"));
-    checkSql(sql + range(130, SqlToRelConverter.DEFAULT_IN_SUBQUERY_THRESHOLD + 1000),
-        "smart", expect("NAME=Alice"));
+    final int threshold = SqlToRelConverter.DEFAULT_IN_SUB_QUERY_THRESHOLD;
+    checkSql(sql + range(130, threshold - 5), "smart", expect("NAME=Alice"));
+    checkSql(sql + range(130, threshold), "smart", expect("NAME=Alice"));
+    checkSql(sql + range(130, threshold + 1000), "smart",
+        expect("NAME=Alice"));
   }
 
   /** Test case for
@@ -423,7 +422,7 @@ public class CsvTest {
     final String sql = "SELECT e.name\n"
         + "FROM emps AS e\n"
         + "WHERE e.empno in "
-        + range(130, SqlToRelConverter.DEFAULT_IN_SUBQUERY_THRESHOLD);
+        + range(130, SqlToRelConverter.DEFAULT_IN_SUB_QUERY_THRESHOLD);
     checkSql(sql, "smart", expect("NAME=Alice"));
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
index 2f4bbef..70c3cde 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRules.java
@@ -430,7 +430,7 @@ public class MongoRules {
       }
       implementor.newline(buf)
           .append("FROM ");
-      implementor.subquery(buf, 0, getChild(), "t");
+      implementor.subQuery(buf, 0, getChild(), "t");
       if (program.getCondition() != null) {
         implementor.newline(buf);
         buf.append("WHERE ");

http://git-wip-us.apache.org/repos/asf/calcite/blob/e38d51e8/site/_docs/history.md
----------------------------------------------------------------------
diff --git a/site/_docs/history.md b/site/_docs/history.md
index 138ef45..51f11da 100644
--- a/site/_docs/history.md
+++ b/site/_docs/history.md
@@ -1011,7 +1011,7 @@ Bug fixes, API changes and minor enhancements
   Type of 'Java<Long> * `INTEGER`' should be `BIGINT`
 * [<a href="https://issues.apache.org/jira/browse/CALCITE-894">CALCITE-894</a>]
   Do not generate redundant column alias for the left relation when
-  translating `IN` subquery (Maryann Xue)
+  translating `IN` sub-query (Maryann Xue)
 * [<a href="https://issues.apache.org/jira/browse/CALCITE-897">CALCITE-897</a>]
   Enable debugging using "-Dcalcite.debug"
 * [<a href="https://issues.apache.org/jira/browse/CALCITE-885">CALCITE-885</a>]
@@ -1137,7 +1137,7 @@ Bug fixes, API changes and minor enhancements
 * [<a href="https://issues.apache.org/jira/browse/CALCITE-813">CALCITE-813</a>]
       Upgrade `updateCount`, `maxRows` from int to long
 * [<a href="https://issues.apache.org/jira/browse/CALCITE-714">CALCITE-714</a>]
-      When de-correlating, push join condition into subquery
+      When de-correlating, push join condition into sub-query
 * [<a href="https://issues.apache.org/jira/browse/CALCITE-751">CALCITE-751</a>]
       Push aggregate with aggregate functions through join
 * Add `RelBuilder.avg`