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/05/15 22:28:32 UTC

[2/2] calcite git commit: Further work on [CALCITE-1120]

Further work on [CALCITE-1120]

Document new syntax in SQL reference;
WHERE, GROUP BY and HAVING are illegal if SELECT has no FROM.

Add "conformance" connect-string parameter, and enable SELECT without
FROM if conformance is not strict. It is enabled by default.


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

Branch: refs/heads/master
Commit: f8f2af3e16364745e228570a51cf2fa5decbe686
Parents: 40576e7
Author: Julian Hyde <jh...@apache.org>
Authored: Sat May 14 17:55:50 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun May 15 15:28:12 2016 -0700

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       | 54 +++++++-----
 .../calcite/config/CalciteConnectionConfig.java |  5 +-
 .../config/CalciteConnectionConfigImpl.java     |  6 +-
 .../config/CalciteConnectionProperty.java       |  5 +-
 .../calcite/prepare/CalcitePrepareImpl.java     |  5 +-
 .../calcite/prepare/CalciteSqlValidator.java    |  9 +-
 .../org/apache/calcite/prepare/PlannerImpl.java | 27 +++++-
 .../calcite/sql/validate/SqlConformance.java    | 28 +++++--
 .../calcite/sql/validate/SqlValidatorUtil.java  | 18 +++-
 .../calcite/sql/parser/SqlParserTest.java       | 29 ++++++-
 .../calcite/sql/test/DefaultSqlTestFactory.java |  4 +-
 .../java/org/apache/calcite/test/JdbcTest.java  |  5 +-
 .../apache/calcite/test/SqlValidatorTest.java   | 17 ++--
 .../calcite/test/SqlValidatorTestCase.java      | 12 ++-
 core/src/test/resources/sql/misc.iq             | 88 ++++++++++++++++++++
 site/_docs/adapter.md                           |  1 +
 site/_docs/reference.md                         |  9 ++
 17 files changed, 257 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/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 124e436..bad7622 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -980,14 +980,12 @@ SqlSelect SqlSelect() :
 {
     final List<SqlLiteral> keywords = Lists.newArrayList();
     List<SqlNode> selectList;
-    SqlNode fromClause;
-    SqlNode where;
-    SqlNodeList groupBy;
-    SqlNode having;
-    SqlNodeList windowDecls;
+    final SqlNode fromClause;
+    final SqlNode where;
+    final SqlNodeList groupBy;
+    final SqlNode having;
+    final SqlNodeList windowDecls;
     SqlParserPos pos;
-    SqlParserPos selectListPos;
-    SqlNode selectItem;
 }
 {
     <SELECT>
@@ -1004,19 +1002,29 @@ SqlSelect SqlSelect() :
         <DISTINCT> {
             keywords.add(SqlSelectKeyword.DISTINCT.symbol(getPos()));
         }
-        |   <ALL> {
+    |   <ALL> {
             keywords.add(SqlSelectKeyword.ALL.symbol(getPos()));
         }
     )?
     selectList = SelectList()
-    fromClause = FromClauseOpt()
-    where = WhereOpt()
-    groupBy = GroupByOpt()
-    having = HavingOpt()
-    windowDecls = WindowOpt()
-    {
-        selectItem = (SqlNode)selectList.get(0);
-        selectListPos = selectItem.getParserPosition();
+    (
+        <FROM> fromClause = FromClause()
+        where = WhereOpt()
+        groupBy = GroupByOpt()
+        having = HavingOpt()
+        windowDecls = WindowOpt()
+    |
+        E() {
+            fromClause = null;
+            where = null;
+            groupBy = null;
+            having = null;
+            windowDecls = null;
+        }
+    )
+    {
+        final SqlNode selectItem = (SqlNode)selectList.get(0);
+        final SqlParserPos selectListPos = selectItem.getParserPosition();
         return new SqlSelect(pos.plus(getPos()),
             new SqlNodeList(keywords, pos),
             new SqlNodeList(selectList, selectListPos.plusAll(selectList)),
@@ -1644,9 +1652,13 @@ SqlNode JoinTable(SqlNode e) :
 // We allow CROSS JOIN to have a join condition, even though that is not valid
 // SQL; the validator will catch it.
 /**
- * Parses the optional FROM clause for a SELECT.
+ * Parses the FROM clause for a SELECT.
+ *
+ * <p>FROM is mandatory in standard SQL, optional in dialects such as MySQL,
+ * PostgreSQL. The parser allows SELECT without FROM, but the validator fails
+ * if conformance is, say, STRICT_2003.
  */
-SqlNode FromClauseOpt() :
+SqlNode FromClause() :
 {
     SqlNode e, e2, condition;
     SqlLiteral natural, joinType;
@@ -1654,7 +1666,7 @@ SqlNode FromClauseOpt() :
     SqlParserPos pos;
 }
 {
-    <FROM> e = TableRef()
+    e = TableRef()
     (
         // Decide whether to read a JOIN clause or a comma, or to quit having
         // seen a single entry FROM clause like 'FROM emps'. See comments
@@ -1718,10 +1730,6 @@ SqlNode FromClauseOpt() :
     {
         return e;
     }
-    |
-    {
-        return null;
-    }
 }
 
 // TODO jvs 15-Nov-2003: SQL standard allows column aliases on table

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
index fa1226f..24906d3 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
@@ -19,6 +19,7 @@ package org.apache.calcite.config;
 import org.apache.calcite.avatica.ConnectionConfig;
 import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.avatica.util.Quoting;
+import org.apache.calcite.sql.validate.SqlConformance;
 
 /** Interface for reading connection properties within Calcite code. There is
  * a method for every property. At some point there will be similar config
@@ -52,8 +53,8 @@ public interface CalciteConnectionConfig extends ConnectionConfig {
   boolean forceDecorrelate();
   /** @see CalciteConnectionProperty#TYPE_SYSTEM */
   <T> T typeSystem(Class<T> typeSystemClass, T defaultTypeSystem);
-  /** @see CalciteConnectionProperty#FROM_REQUIRED */
-  boolean fromRequired();
+  /** @see CalciteConnectionProperty#CONFORMANCE */
+  SqlConformance conformance();
 }
 
 // End CalciteConnectionConfig.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
index 9e352a9..1f810e2 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
@@ -23,6 +23,7 @@ import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.fun.OracleSqlOperatorTable;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -131,8 +132,9 @@ public class CalciteConnectionConfigImpl extends ConnectionConfigImpl
         .getPlugin(typeSystemClass, defaultTypeSystem);
   }
 
-  public boolean fromRequired() {
-    return CalciteConnectionProperty.FROM_REQUIRED.wrap(properties).getBoolean();
+  public SqlConformance conformance() {
+    return CalciteConnectionProperty.CONFORMANCE.wrap(properties)
+        .getEnum(SqlConformance.class);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
index ff46edb..7bf157d 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.config;
 
 import org.apache.calcite.avatica.ConnectionProperty;
+import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.util.Bug;
 
 import java.util.HashMap;
@@ -96,8 +97,8 @@ public enum CalciteConnectionProperty implements ConnectionProperty {
    * default constructor or an {@code INSTANCE} constant. */
   TYPE_SYSTEM("typeSystem", Type.PLUGIN, null, false),
 
-  /** Whether to support SELECT without FROM, e.g. SELECT 2+2. */
-  FROM_REQUIRED("fromRequired", Type.BOOLEAN, false, false);
+  /** SQL conformance level. */
+  CONFORMANCE("conformance", Type.ENUM, SqlConformance.DEFAULT, false);
 
   private final String camelName;
   private final Type type;

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index e766fb0..bd53b7b 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -121,6 +121,7 @@ import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.sql2rel.StandardConvertletTable;
@@ -793,7 +794,9 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     final SqlOperatorTable opTab =
         ChainedSqlOperatorTable.of(opTab0, catalogReader);
     final JavaTypeFactory typeFactory = context.getTypeFactory();
-    return new CalciteSqlValidator(opTab, catalogReader, typeFactory);
+    final SqlConformance conformance = context.config().conformance();
+    return new CalciteSqlValidator(opTab, catalogReader, typeFactory,
+        conformance);
   }
 
   private List<ColumnMetaData> getColumnMetaDataList(

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java b/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java
index 63670ad..a7fc2fe 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java
@@ -25,11 +25,10 @@ import org.apache.calcite.sql.validate.SqlValidatorImpl;
 
 /** Validator. */
 class CalciteSqlValidator extends SqlValidatorImpl {
-  public CalciteSqlValidator(
-      SqlOperatorTable opTab,
-      CalciteCatalogReader catalogReader,
-      JavaTypeFactory typeFactory) {
-    super(opTab, catalogReader, typeFactory, SqlConformance.DEFAULT);
+  public CalciteSqlValidator(SqlOperatorTable opTab,
+      CalciteCatalogReader catalogReader, JavaTypeFactory typeFactory,
+      SqlConformance conformance) {
+    super(opTab, catalogReader, typeFactory, conformance);
   }
 
   @Override protected RelDataType getLogicalSourceRowType(

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
index fa6ea30..a472683 100644
--- a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
@@ -17,7 +17,9 @@
 package org.apache.calcite.prepare;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptSchema;
@@ -34,6 +36,7 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
@@ -171,9 +174,11 @@ public class PlannerImpl implements Planner {
 
   public SqlNode validate(SqlNode sqlNode) throws ValidationException {
     ensure(State.STATE_3_PARSED);
+    final SqlConformance conformance = conformance();
+    final CalciteCatalogReader catalogReader = createCatalogReader();
     this.validator =
-        new CalciteSqlValidator(
-            operatorTable, createCatalogReader(), typeFactory);
+        new CalciteSqlValidator(operatorTable, catalogReader, typeFactory,
+            conformance);
     this.validator.setIdentifierExpansion(true);
     try {
       validatedSqlNode = validator.validate(sqlNode);
@@ -184,6 +189,18 @@ public class PlannerImpl implements Planner {
     return validatedSqlNode;
   }
 
+  private SqlConformance conformance() {
+    final Context context = config.getContext();
+    if (context != null) {
+      final CalciteConnectionConfig connectionConfig =
+          context.unwrap(CalciteConnectionConfig.class);
+      if (connectionConfig != null) {
+        return connectionConfig.conformance();
+      }
+    }
+    return SqlConformance.DEFAULT;
+  }
+
   public Pair<SqlNode, RelDataType> validateAndGetType(SqlNode sqlNode)
       throws ValidationException {
     final SqlNode validatedNode = this.validate(sqlNode);
@@ -227,10 +244,12 @@ public class PlannerImpl implements Planner {
         throw new RuntimeException("parse failed", e);
       }
 
+      final SqlConformance conformance = conformance();
       final CalciteCatalogReader catalogReader =
           createCatalogReader().withSchemaPath(schemaPath);
-      final SqlValidator validator = new CalciteSqlValidator(operatorTable,
-          catalogReader, typeFactory);
+      final SqlValidator validator =
+          new CalciteSqlValidator(operatorTable, catalogReader, typeFactory,
+              conformance);
       validator.setIdentifierExpansion(true);
       final SqlNode validatedSqlNode = validator.validate(sqlNode);
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
index 8198de8..4dec2cd 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
@@ -17,7 +17,7 @@
 package org.apache.calcite.sql.validate;
 
 /**
- * Enumeration of valid SQL compatiblity modes.
+ * Enumeration of valid SQL compatibility modes.
  */
 public enum SqlConformance {
   DEFAULT, STRICT_92, STRICT_99, PRAGMATIC_99, ORACLE_10, STRICT_2003,
@@ -26,6 +26,10 @@ public enum SqlConformance {
   /**
    * Whether 'order by 2' is interpreted to mean 'sort by the 2nd column in
    * the select list'.
+   *
+   * <p>True in {@link #DEFAULT}, {@link #ORACLE_10}, {@link #STRICT_92},
+   * {@link #PRAGMATIC_99}, {@link #PRAGMATIC_2003};
+   * false otherwise.
    */
   public boolean isSortByOrdinal() {
     switch (this) {
@@ -43,6 +47,9 @@ public enum SqlConformance {
   /**
    * Whether 'order by x' is interpreted to mean 'sort by the select list item
    * whose alias is x' even if there is a column called x.
+   *
+   * <p>True in {@link #DEFAULT}, {@link #ORACLE_10}, {@link #STRICT_92};
+   * false otherwise.
    */
   public boolean isSortByAlias() {
     switch (this) {
@@ -58,21 +65,30 @@ public enum SqlConformance {
   /**
    * Whether "empno" is invalid in "select empno as x from emp order by empno"
    * because the alias "x" obscures it.
+   *
+   * <p>True in {@link #STRICT_92};
+   * false otherwise.
    */
   public boolean isSortByAliasObscures() {
     return this == SqlConformance.STRICT_92;
   }
 
   /**
-   * Whether from clause is required for any select statement.
+   * Whether FROM clause is required in a SELECT statement.
+   *
+   * <p>True in {@link #ORACLE_10}, {@link #STRICT_92}, {@link #STRICT_99},
+   * {@link #STRICT_2003};
+   * false otherwise.
    */
   public boolean isFromRequired() {
     switch (this) {
-    case DEFAULT:
-    case PRAGMATIC_2003:
-      return false;
-    default:
+    case ORACLE_10:
+    case STRICT_92:
+    case STRICT_99:
+    case STRICT_2003:
       return true;
+    default:
+      return false;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
index ac03caa..922512b 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
@@ -219,11 +219,21 @@ public class SqlValidatorUtil {
   public static SqlValidatorWithHints newValidator(
       SqlOperatorTable opTab,
       SqlValidatorCatalogReader catalogReader,
+      RelDataTypeFactory typeFactory,
+      SqlConformance conformance) {
+    return new SqlValidatorImpl(opTab, catalogReader, typeFactory,
+        conformance);
+  }
+
+  /**
+   * Factory method for {@link SqlValidator}, with default conformance.
+   */
+  @Deprecated // to be removed before 2.0
+  public static SqlValidatorWithHints newValidator(
+      SqlOperatorTable opTab,
+      SqlValidatorCatalogReader catalogReader,
       RelDataTypeFactory typeFactory) {
-    return new SqlValidatorImpl(
-        opTab,
-        catalogReader,
-        typeFactory,
+    return newValidator(opTab, catalogReader, typeFactory,
         SqlConformance.DEFAULT);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/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 ea9c192..26334bd 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
@@ -1936,9 +1936,32 @@ public class SqlParserTest {
   }
 
   @Test public void testSelectWithoutFrom() {
-    check(
-        "select 2+2",
-        "SELECT (2 + 2)");
+    sql("select 2+2").ok("SELECT (2 + 2)");
+  }
+
+  @Test public void testSelectWithoutFrom2() {
+    sql("select 2+2 as x, 'a' as y")
+        .ok("SELECT (2 + 2) AS `X`, 'a' AS `Y`");
+  }
+
+  @Test public void testSelectDistinctWithoutFrom() {
+    sql("select distinct 2+2 as x, 'a' as y")
+        .ok("SELECT DISTINCT (2 + 2) AS `X`, 'a' AS `Y`");
+  }
+
+  @Test public void testSelectWithoutFromWhereFails() {
+    sql("select 2+2 as x ^where^ 1 > 2")
+        .fails("(?s).*Encountered \"where\" at line .*");
+  }
+
+  @Test public void testSelectWithoutFromGroupByFails() {
+    sql("select 2+2 as x ^group^ by 1, 2")
+        .fails("(?s).*Encountered \"group\" at line .*");
+  }
+
+  @Test public void testSelectWithoutFromHavingFails() {
+    sql("select 2+2 as x ^having^ 1 > 2")
+        .fails("(?s).*Encountered \"having\" at line .*");
   }
 
   @Test public void testSelectList3() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
index 51670aa..4ea62e3 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
@@ -86,11 +86,13 @@ public class DefaultSqlTestFactory implements SqlTestFactory {
   public SqlValidator getValidator(SqlTestFactory factory) {
     final SqlOperatorTable operatorTable = factory.createOperatorTable(factory);
     final boolean caseSensitive = (Boolean) factory.get("caseSensitive");
+    final SqlConformance conformance =
+        (SqlConformance) factory.get("conformance");
     final RelDataTypeFactory typeFactory =
         new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
     return SqlValidatorUtil.newValidator(operatorTable,
         new MockCatalogReader(typeFactory, caseSensitive).init(),
-        typeFactory);
+        typeFactory, conformance);
   }
 
   public SqlAdvisor createAdvisor(SqlValidatorWithHints validator) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/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 894b992..06a6854 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -2615,7 +2615,10 @@ public class JdbcTest {
         .returns("EXPR$0=-3\n");
   }
 
-  @Test public void testValuesWithoutFrom() {
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1120">[CALCITE-1120]
+   * Support SELECT without FROM</a>. */
+  @Test public void testSelectWithoutFrom() {
     CalciteAssert.that()
         .query("select 2+2")
         .returns("EXPR$0=4\n");

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/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 dedcffc..20f9ec7 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -6842,14 +6842,15 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testSelectWithoutFrom() {
-    final SqlConformance conformance = tester.getConformance();
-    checkFails(
-        "^select 2+2^",
-
-        conformance.isFromRequired()
-            ? "SELECT must have a FROM clause"
-            // otherwise valid
-            : null);
+    sql("^select 2+2^")
+        .tester(tester.withConformance(SqlConformance.DEFAULT))
+        .ok();
+    sql("^select 2+2^")
+        .tester(tester.withConformance(SqlConformance.ORACLE_10))
+        .fails("SELECT must have a FROM clause");
+    sql("^select 2+2^")
+        .tester(tester.withConformance(SqlConformance.STRICT_2003))
+        .fails("SELECT must have a FROM clause");
   }
 
   @Test public void testTableExtend() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
index 358bb2a..6860851 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
@@ -82,7 +82,7 @@ public class SqlValidatorTestCase {
   }
 
   public final Sql sql(String sql) {
-    return new Sql(sql);
+    return new Sql(tester, sql);
   }
 
   public final Sql winSql(String sql) {
@@ -525,13 +525,19 @@ public class SqlValidatorTestCase {
   }
 
   /** Fluent testing API. */
-  class Sql {
+  static class Sql {
+    private final SqlTester tester;
     private final String sql;
 
-    Sql(String sql) {
+    Sql(SqlTester tester, String sql) {
+      this.tester = tester;
       this.sql = sql;
     }
 
+    Sql tester(SqlTester tester) {
+      return new Sql(tester, sql);
+    }
+
     Sql ok() {
       tester.assertExceptionIsThrown(sql, null);
       return this;

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/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 7612f22..129965e 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -1195,4 +1195,92 @@ from "scott".dept;
 
 !ok
 
+# [CALCITE-1120] Support SELECT without FROM
+select 2 + 2;
++--------+
+| EXPR$0 |
++--------+
+|      4 |
++--------+
+(1 row)
+
+!ok
+
+select 2 + 2 as a, 'b' as b;
++---+---+
+| A | B |
++---+---+
+| 4 | b |
++---+---+
+(1 row)
+
+!ok
+
+select distinct 2 as a, 'b' as b;
++---+---+
+| A | B |
++---+---+
+| 2 | b |
++---+---+
+(1 row)
+
+!ok
+
+select distinct 2 as a, 'b' as b
+order by 2;
++---+---+
+| A | B |
++---+---+
+| 2 | b |
++---+---+
+(1 row)
+
+!ok
+
+select distinct 2 as a, 'b' as b
+union all
+select 3, 'c';
++---+---+
+| A | B |
++---+---+
+| 2 | b |
+| 3 | c |
++---+---+
+(2 rows)
+
+!ok
+
+select * from (
+  select 2 as a, 3 as b)
+where a < b;
++---+---+
+| A | B |
++---+---+
+| 2 | 3 |
++---+---+
+(1 row)
+
+!ok
+
+select * from (
+  select distinct 2 as a, 3 as b)
+where a > b;
++---+---+
+| A | B |
++---+---+
++---+---+
+(0 rows)
+
+!ok
+
+select count(*) as c from "scott".dept where exists (select 1);
++---+
+| C |
++---+
+| 4 |
++---+
+(1 row)
+
+!ok
+
 # End misc.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/site/_docs/adapter.md
----------------------------------------------------------------------
diff --git a/site/_docs/adapter.md b/site/_docs/adapter.md
index 7805333..b003e81 100644
--- a/site/_docs/adapter.md
+++ b/site/_docs/adapter.md
@@ -68,6 +68,7 @@ JDBC connect string parameters
 | Property | Description |
 |:-------- |:------------|
 | <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#CASE_SENSITIVE">caseSensitive</a> | Whether identifiers are matched case-sensitively. If not specified, value from `lex` is used.
+| <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#CONFORMANCE">conformance</a> | SQL conformance level. Values: "default" (the default, similar to "pragmatic_2003"), "oracle_10", "pragmatic_99", "pragmatic_2003", "strict_92", "strict_99", "strict_2003".
 | <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#CREATE_MATERIALIZATIONS">createMaterializations</a> | Whether Calcite should create materializations. Default false.
 | <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#DEFAULT_NULL_COLLATION">materializationsEnabled</a> | How NULL values should be sorted if neither NULLS FIRST nor NULLS LAST are specified in a query. The default, HIGH, sorts NULL values the same as Oracle.
 | <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#FORCE_DECORRELATE">forceDecorrelate</a> | Whether the planner should try de-correlating as much as possible. Default true.

http://git-wip-us.apache.org/repos/asf/calcite/blob/f8f2af3e/site/_docs/reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 26ccbaf..a6ce287 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -90,6 +90,7 @@ query:
   |   WITH withItem [ , withItem ]* query
   |   {
           select
+      |   selectWithoutFrom
       |   query UNION [ ALL ] query
       |   query EXCEPT query
       |   query INTERSECT query
@@ -116,6 +117,10 @@ select:
       [ HAVING booleanExpression ]
       [ WINDOW windowName AS windowSpec [, windowName AS windowSpec ]* ]
 
+selectWithoutFrom:
+      SELECT [ ALL | DISTINCT ]
+          { * | projectItem [, projectItem ]* }
+
 projectItem:
       expression [ [ AS ] columnAlias ]
   |   tableAlias . *
@@ -192,6 +197,10 @@ function).
 An IN, EXISTS or scalar sub-query may be correlated; that is, it
 may refer to tables in the FROM clause of an enclosing query.
 
+*selectWithoutFrom* is equivalent to VALUES,
+but is not standard SQL and is only allowed in certain
+[conformance levels]({{ site.apiRoot }}/org/apache/calcite/sql/validate/SqlConformance.html#isFromRequired--).
+
 ## Keywords
 
 The following is a list of SQL keywords.