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/03/05 04:05:19 UTC

[1/4] calcite git commit: [CALCITE-1066] Add Oracle function table, and functions DECODE, NVL, LTRIM, RTRIM, GREATEST, LEAST

Repository: calcite
Updated Branches:
  refs/heads/master c7db615fa -> 5897dcd9d


[CALCITE-1066] Add Oracle function table, and functions DECODE, NVL, LTRIM, RTRIM, GREATEST, LEAST


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

Branch: refs/heads/master
Commit: 707e72d5c2a03f4ec5994b58109d7421ed938d9f
Parents: c7db615
Author: Julian Hyde <jh...@apache.org>
Authored: Sun Jan 24 18:05:20 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Mar 4 15:25:57 2016 -0800

----------------------------------------------------------------------
 .../calcite/config/CalciteConnectionConfig.java |   2 +
 .../config/CalciteConnectionConfigImpl.java     |  33 ++++
 .../config/CalciteConnectionProperty.java       |   6 +-
 .../calcite/prepare/CalcitePrepareImpl.java     |  21 +--
 .../rel/rules/AggregateReduceFunctionsRule.java |   8 +-
 .../java/org/apache/calcite/sql/SqlKind.java    |  42 ++++-
 .../calcite/sql/fun/OracleSqlOperatorTable.java | 120 +++++++++++++++
 .../calcite/sql/fun/SqlStdOperatorTable.java    |   2 +-
 .../apache/calcite/sql/fun/SqlTrimFunction.java |  51 +++---
 .../calcite/sql/type/SqlTypeTransforms.java     |  16 ++
 .../apache/calcite/sql/type/SqlTypeUtil.java    |  12 ++
 .../sql2rel/StandardConvertletTable.java        | 154 +++++++++++++++++--
 .../main/java/org/apache/calcite/util/Util.java |  23 +++
 .../calcite/sql/test/DefaultSqlTestFactory.java |  29 ++--
 .../sql/test/DelegatingSqlTestFactory.java      |   4 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java   | 136 ++++++++++++----
 .../apache/calcite/sql/test/SqlTestFactory.java |   2 +-
 .../org/apache/calcite/sql/test/SqlTester.java  |   9 ++
 .../apache/calcite/sql/test/SqlTesterImpl.java  |  15 +-
 .../org/apache/calcite/test/CalciteAssert.java  |   2 +-
 .../calcite/test/CalciteSqlOperatorTest.java    |  21 +--
 .../java/org/apache/calcite/test/JdbcTest.java  |  15 ++
 .../calcite/test/SqlValidatorFeatureTest.java   |   2 +-
 .../java/org/apache/calcite/util/UtilTest.java  |  12 ++
 site/_docs/adapter.md                           |  22 +++
 25 files changed, 640 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/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 d04e76d..522783f 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
@@ -32,6 +32,8 @@ public interface CalciteConnectionConfig extends ConnectionConfig {
   boolean createMaterializations();
   /** @see CalciteConnectionProperty#DEFAULT_NULL_COLLATION */
   NullCollation defaultNullCollation();
+  /** @see CalciteConnectionProperty#FUN */
+  <T> T fun(Class<T> operatorTableClass, T defaultOperatorTable);
   /** @see CalciteConnectionProperty#MODEL */
   String model();
   /** @see CalciteConnectionProperty#LEX */

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/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 fefacaa..b8e44ed 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
@@ -19,7 +19,13 @@ package org.apache.calcite.config;
 import org.apache.calcite.avatica.ConnectionConfigImpl;
 import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.avatica.util.Quoting;
+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 java.util.ArrayList;
+import java.util.List;
 import java.util.Properties;
 
 /** Implementation of {@link CalciteConnectionConfig}. */
@@ -56,6 +62,33 @@ public class CalciteConnectionConfigImpl extends ConnectionConfigImpl
         .getEnum(NullCollation.class, NullCollation.HIGH);
   }
 
+  public <T> T fun(Class<T> operatorTableClass, T defaultOperatorTable) {
+    final String fun =
+        CalciteConnectionProperty.FUN.wrap(properties).getString();
+    if (fun == null || fun.equals("") || fun.equals("standard")) {
+      return defaultOperatorTable;
+    }
+    final List<SqlOperatorTable> tables = new ArrayList<>();
+    for (String s : fun.split(",")) {
+      tables.add(operatorTable(s));
+    }
+    return operatorTableClass.cast(
+        ChainedSqlOperatorTable.of(
+            tables.toArray(new SqlOperatorTable[tables.size()])));
+  }
+
+  private static SqlOperatorTable operatorTable(String s) {
+    switch (s) {
+    case "standard":
+      return SqlStdOperatorTable.instance();
+    case "oracle":
+      return ChainedSqlOperatorTable.of(OracleSqlOperatorTable.instance(),
+          SqlStdOperatorTable.instance());
+    default:
+      throw new IllegalArgumentException("Unknown operator table: " + s);
+    }
+  }
+
   public String model() {
     return CalciteConnectionProperty.MODEL.wrap(properties).getString();
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/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 8cbac97..64b3d53 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
@@ -40,7 +40,7 @@ public enum CalciteConnectionProperty implements ConnectionProperty {
   CREATE_MATERIALIZATIONS("createMaterializations", Type.BOOLEAN, true, false),
 
   /** How NULL values should be sorted if neither NULLS FIRST nor NULLS LAST are
-   * specified. The defult, HIGH, sorts NULL values the same as Oracle. */
+   * specified. The default, HIGH, sorts NULL values the same as Oracle. */
   DEFAULT_NULL_COLLATION("defaultNullCollation", Type.ENUM, NullCollation.HIGH,
       true),
 
@@ -50,6 +50,10 @@ public enum CalciteConnectionProperty implements ConnectionProperty {
   /** Lexical policy. */
   LEX("lex", Type.ENUM, Lex.ORACLE, false),
 
+  /** Collection of built-in functions and operators. Valid values include
+   * "standard" and "oracle". */
+  FUN("fun", Type.STRING, "standard", true),
+
   /** How identifiers are quoted.
    *  If not specified, value from {@link #LEX} is used. */
   QUOTING("quoting", Type.ENUM, null, false),

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/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 dc17ddd..177cd0b 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -276,8 +276,7 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     } catch (SqlParseException e) {
       throw new RuntimeException("parse failed", e);
     }
-    final SqlValidator validator =
-        createSqlValidator(catalogReader, typeFactory);
+    final SqlValidator validator = createSqlValidator(context, catalogReader);
     SqlNode sqlNode1 = validator.validate(sqlNode);
     if (convert) {
       return convert_(
@@ -714,7 +713,7 @@ public class CalcitePrepareImpl implements CalcitePrepare {
       }
 
       final SqlValidator validator =
-          createSqlValidator(catalogReader, typeFactory);
+          createSqlValidator(context, catalogReader);
       validator.setIdentifierExpansion(true);
       validator.setDefaultNullCollation(config.defaultNullCollation());
 
@@ -783,11 +782,14 @@ public class CalcitePrepareImpl implements CalcitePrepare {
         statementType);
   }
 
-  private SqlValidator createSqlValidator(CalciteCatalogReader catalogReader,
-      JavaTypeFactory typeFactory) {
+  private SqlValidator createSqlValidator(Context context,
+      CalciteCatalogReader catalogReader) {
+    final SqlOperatorTable opTab0 =
+        context.config().fun(SqlOperatorTable.class,
+            SqlStdOperatorTable.instance());
     final SqlOperatorTable opTab =
-        ChainedSqlOperatorTable.of(SqlStdOperatorTable.instance(),
-            catalogReader);
+        ChainedSqlOperatorTable.of(opTab0, catalogReader);
+    final JavaTypeFactory typeFactory = context.getTypeFactory();
     return new CalciteSqlValidator(opTab, catalogReader, typeFactory);
   }
 
@@ -1125,9 +1127,8 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     }
 
     protected SqlValidator createSqlValidator(CatalogReader catalogReader) {
-      return prepare.createSqlValidator(
-          (CalciteCatalogReader) catalogReader,
-          (JavaTypeFactory) typeFactory);
+      return prepare.createSqlValidator(context,
+          (CalciteCatalogReader) catalogReader);
     }
 
     @Override protected SqlValidator getSqlValidator() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
index fca05ed..b99c061 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
@@ -31,6 +31,7 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlAvgAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.fun.SqlSumAggFunction;
@@ -192,9 +193,8 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
       return reduceSum(oldAggRel, oldCall, newCalls, aggCallMapping);
     }
     if (oldCall.getAggregation() instanceof SqlAvgAggFunction) {
-      final SqlAvgAggFunction.Subtype subtype =
-          ((SqlAvgAggFunction) oldCall.getAggregation()).getSubtype();
-      switch (subtype) {
+      final SqlKind kind = oldCall.getAggregation().getKind();
+      switch (kind) {
       case AVG:
         // replace original AVG(x) with SUM(x) / COUNT(x)
         return reduceAvg(oldAggRel, oldCall, newCalls, aggCallMapping);
@@ -225,7 +225,7 @@ public class AggregateReduceFunctionsRule extends RelOptRule {
         return reduceStddev(oldAggRel, oldCall, false, false, newCalls,
             aggCallMapping, inputExprs);
       default:
-        throw Util.unexpected(subtype);
+        throw Util.unexpected(kind);
       }
     } else {
       // anything else:  preserve original call

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index bbc7e7e..7f34162 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -331,6 +331,36 @@ public enum SqlKind {
    */
   CASE,
 
+  /**
+   * The "NULLIF" operator.
+   */
+  NULLIF,
+
+  /**
+   * The "COALESCE" operator.
+   */
+  COALESCE,
+
+  /**
+   * The "DECODE" function (Oracle).
+   */
+  DECODE,
+
+  /**
+   * The "NVL" function (Oracle).
+   */
+  NVL,
+
+  /**
+   * The "GREATEST" function (Oracle).
+   */
+  GREATEST,
+
+  /**
+   * The "LEAST" function (Oracle).
+   */
+  LEAST,
+
   // prefix operators
 
   /**
@@ -523,6 +553,16 @@ public enum SqlKind {
   TRIM,
 
   /**
+   * The "LTRIM" function (Oracle).
+   */
+  LTRIM,
+
+  /**
+   * The "RTRIM" function (Oracle).
+   */
+  RTRIM,
+
+  /**
    * Call to a function using JDBC function syntax.
    */
   JDBC_FN,
@@ -882,7 +922,7 @@ public enum SqlKind {
    * functions {@link #ROW}, {@link #TRIM}, {@link #CAST}, {@link #JDBC_FN}.
    */
   public static final Set<SqlKind> FUNCTION =
-      EnumSet.of(OTHER_FUNCTION, ROW, TRIM, CAST, JDBC_FN);
+      EnumSet.of(OTHER_FUNCTION, ROW, TRIM, LTRIM, RTRIM, CAST, JDBC_FN);
 
   /**
    * Category of comparison operators.

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/main/java/org/apache/calcite/sql/fun/OracleSqlOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/OracleSqlOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/OracleSqlOperatorTable.java
new file mode 100644
index 0000000..1fc1fd5
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/OracleSqlOperatorTable.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeTransforms;
+import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Operator table that contains only Oracle-specific functions and operators.
+ */
+public class OracleSqlOperatorTable extends ReflectiveSqlOperatorTable {
+  //~ Static fields/initializers ---------------------------------------------
+
+  /**
+   * The table of contains Oracle-specific operators.
+   */
+  private static OracleSqlOperatorTable instance;
+
+  /** Return type inference for {@code DECODE}. */
+  protected static final SqlReturnTypeInference DECODE_RETURN_TYPE =
+      new SqlReturnTypeInference() {
+        public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+          final List<RelDataType> list = new ArrayList<>();
+          for (int i = 1, n = opBinding.getOperandCount(); i < n; i++) {
+            if (i < n - 1) {
+              ++i;
+            }
+            list.add(opBinding.getOperandType(i));
+          }
+          final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
+          RelDataType type = typeFactory.leastRestrictive(list);
+          if (opBinding.getOperandCount() % 2 == 1) {
+            type = typeFactory.createTypeWithNullability(type, true);
+          }
+          return type;
+        }
+      };
+
+  /** The "DECODE(v, v1, result1, [v2, result2, ...], resultN)" function. */
+  public static final SqlFunction DECODE =
+      new SqlFunction("DECODE", SqlKind.DECODE, DECODE_RETURN_TYPE, null,
+          OperandTypes.VARIADIC, SqlFunctionCategory.SYSTEM);
+
+  /** The "NVL(value, value)" function. */
+  public static final SqlFunction NVL =
+      new SqlFunction("NVL", SqlKind.NVL,
+          ReturnTypes.cascade(ReturnTypes.LEAST_RESTRICTIVE,
+              SqlTypeTransforms.TO_NULLABLE_ALL),
+          null, OperandTypes.SAME_SAME, SqlFunctionCategory.SYSTEM);
+
+  /** The "LTRIM(string)" function. */
+  public static final SqlFunction LTRIM =
+      new SqlFunction("LTRIM", SqlKind.LTRIM,
+          ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NULLABLE,
+              SqlTypeTransforms.TO_VARYING), null,
+          OperandTypes.STRING, SqlFunctionCategory.STRING);
+
+  /** The "RTRIM(string)" function. */
+  public static final SqlFunction RTRIM =
+      new SqlFunction("RTRIM", SqlKind.RTRIM,
+          ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NULLABLE,
+              SqlTypeTransforms.TO_VARYING), null,
+          OperandTypes.STRING, SqlFunctionCategory.STRING);
+
+  /** The "GREATEST(value, value)" function. */
+  public static final SqlFunction GREATEST =
+      new SqlFunction("GREATEST", SqlKind.GREATEST,
+          ReturnTypes.cascade(ReturnTypes.LEAST_RESTRICTIVE,
+              SqlTypeTransforms.TO_NULLABLE), null,
+          OperandTypes.SAME_VARIADIC, SqlFunctionCategory.SYSTEM);
+
+  /** The "LEAST(value, value)" function. */
+  public static final SqlFunction LEAST =
+      new SqlFunction("LEAST", SqlKind.LEAST,
+          ReturnTypes.cascade(ReturnTypes.LEAST_RESTRICTIVE,
+              SqlTypeTransforms.TO_NULLABLE), null,
+          OperandTypes.SAME_VARIADIC, SqlFunctionCategory.SYSTEM);
+
+  /**
+   * Returns the Oracle operator table, creating it if necessary.
+   */
+  public static synchronized OracleSqlOperatorTable instance() {
+    if (instance == null) {
+      // Creates and initializes the standard operator table.
+      // Uses two-phase construction, because we can't initialize the
+      // table until the constructor of the sub-class has completed.
+      instance = new OracleSqlOperatorTable();
+      instance.init();
+    }
+    return instance;
+  }
+}
+
+// End OracleSqlOperatorTable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index dce373f..ece34a5 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -1098,7 +1098,7 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
   public static final SqlFunction OVERLAY = new SqlOverlayFunction();
 
   /** The "TRIM" function. */
-  public static final SqlFunction TRIM = new SqlTrimFunction();
+  public static final SqlFunction TRIM = SqlTrimFunction.INSTANCE;
 
   public static final SqlFunction POSITION = new SqlPositionFunction();
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java
index 0ba461b..606c21e 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java
@@ -28,7 +28,9 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SameOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeTransformCascade;
 import org.apache.calcite.sql.type.SqlTypeTransforms;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 
@@ -41,6 +43,21 @@ import java.util.List;
  * Definition of the "TRIM" builtin SQL function.
  */
 public class SqlTrimFunction extends SqlFunction {
+  protected static final SqlTrimFunction INSTANCE =
+      new SqlTrimFunction("TRIM", SqlKind.TRIM,
+          ReturnTypes.cascade(ReturnTypes.ARG2, SqlTypeTransforms.TO_NULLABLE,
+              SqlTypeTransforms.TO_VARYING),
+          OperandTypes.and(
+              OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.STRING,
+                  SqlTypeFamily.STRING),
+              // Arguments 1 and 2 must have same type
+              new SameOperandTypeChecker(3) {
+                @Override protected List<Integer>
+                getOperandList(int operandCount) {
+                  return ImmutableList.of(1, 2);
+                }
+              }));
+
   //~ Enums ------------------------------------------------------------------
 
   /**
@@ -76,25 +93,10 @@ public class SqlTrimFunction extends SqlFunction {
 
   //~ Constructors -----------------------------------------------------------
 
-  public SqlTrimFunction() {
-    super(
-        "TRIM",
-        SqlKind.TRIM,
-        ReturnTypes.cascade(
-            ReturnTypes.ARG2,
-            SqlTypeTransforms.TO_NULLABLE,
-            SqlTypeTransforms.TO_VARYING),
-        null,
-        OperandTypes.and(
-            OperandTypes.family(
-                SqlTypeFamily.ANY, SqlTypeFamily.STRING, SqlTypeFamily.STRING),
-            // Arguments 1 and 2 must have same type
-            new SameOperandTypeChecker(3) {
-              @Override protected List<Integer>
-              getOperandList(int operandCount) {
-                return ImmutableList.of(1, 2);
-              }
-            }),
+  public SqlTrimFunction(String name, SqlKind kind,
+      SqlTypeTransformCascade returnTypeInference,
+      SqlSingleOperandTypeChecker operandTypeChecker) {
+    super(name, kind, returnTypeInference, null, operandTypeChecker,
         SqlFunctionCategory.STRING);
   }
 
@@ -158,9 +160,14 @@ public class SqlTrimFunction extends SqlFunction {
     if (!super.checkOperandTypes(callBinding, throwOnFailure)) {
       return false;
     }
-    return SqlTypeUtil.isCharTypeComparable(callBinding,
-        ImmutableList.of(callBinding.operand(1), callBinding.operand(2)),
-        throwOnFailure);
+    switch (kind) {
+    case TRIM:
+      return SqlTypeUtil.isCharTypeComparable(callBinding,
+          ImmutableList.of(callBinding.operand(1), callBinding.operand(2)),
+          throwOnFailure);
+    default:
+      return true;
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
index 2f4668f..1eea720 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.sql.type;
 
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.util.Util;
@@ -56,6 +57,21 @@ public abstract class SqlTypeTransforms {
 
   /**
    * Parameter type-inference transform strategy where a derived type is
+   * transformed into the same type, but nullable if and only if all of a call's
+   * operands are nullable.
+   */
+  public static final SqlTypeTransform TO_NULLABLE_ALL =
+      new SqlTypeTransform() {
+        public RelDataType transformType(SqlOperatorBinding opBinding,
+            RelDataType type) {
+          final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
+          return typeFactory.createTypeWithNullability(type,
+              SqlTypeUtil.allNullable(opBinding.collectOperandTypes()));
+        }
+      };
+
+  /**
+   * Parameter type-inference transform strategy where a derived type is
    * transformed into the same type but not nullable.
    */
   public static final SqlTypeTransform TO_NOT_NULLABLE =

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
index d8a437d..203392a 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
@@ -207,6 +207,18 @@ public abstract class SqlTypeUtil {
   }
 
   /**
+   * Returns whether all of array of types are nullable.
+   */
+  public static boolean allNullable(List<RelDataType> types) {
+    for (RelDataType type : types) {
+      if (!containsNullable(type)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
    * Returns whether one or more of an array of types is nullable.
    */
   public static boolean containsNullable(List<RelDataType> types) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
index af34603..99452bf 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -33,6 +33,7 @@ import org.apache.calcite.rex.RexRangeRef;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlBinaryOperator;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlFunction;
@@ -48,8 +49,8 @@ import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlNumericLiteral;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.fun.OracleSqlOperatorTable;
 import org.apache.calcite.sql.fun.SqlArrayValueConstructor;
-import org.apache.calcite.sql.fun.SqlAvgAggFunction;
 import org.apache.calcite.sql.fun.SqlBetweenOperator;
 import org.apache.calcite.sql.fun.SqlCase;
 import org.apache.calcite.sql.fun.SqlDatetimeSubtractionOperator;
@@ -63,6 +64,7 @@ import org.apache.calcite.sql.fun.SqlQuarterFunction;
 import org.apache.calcite.sql.fun.SqlRowOperator;
 import org.apache.calcite.sql.fun.SqlSequenceValueOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlTypeFamily;
@@ -136,6 +138,57 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
           }
         });
 
+    registerOp(OracleSqlOperatorTable.LTRIM,
+        new TrimConvertlet(SqlTrimFunction.Flag.LEADING));
+    registerOp(OracleSqlOperatorTable.RTRIM,
+        new TrimConvertlet(SqlTrimFunction.Flag.TRAILING));
+
+    registerOp(OracleSqlOperatorTable.GREATEST, new GreatestConvertlet());
+    registerOp(OracleSqlOperatorTable.LEAST, new GreatestConvertlet());
+
+    registerOp(OracleSqlOperatorTable.NVL,
+        new SqlRexConvertlet() {
+          public RexNode convertCall(SqlRexContext cx, SqlCall call) {
+            final RexBuilder rexBuilder = cx.getRexBuilder();
+            final RexNode operand0 =
+                cx.convertExpression(call.getOperandList().get(0));
+            final RexNode operand1 =
+                cx.convertExpression(call.getOperandList().get(1));
+            final RelDataType type =
+                cx.getValidator().getValidatedNodeType(call);
+            return rexBuilder.makeCall(type, SqlStdOperatorTable.CASE,
+                ImmutableList.of(
+                    rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL,
+                        operand0),
+                    rexBuilder.makeCast(type, operand0),
+                    rexBuilder.makeCast(type, operand1)));
+          }
+        });
+
+    registerOp(OracleSqlOperatorTable.DECODE,
+        new SqlRexConvertlet() {
+          public RexNode convertCall(SqlRexContext cx, SqlCall call) {
+            final RexBuilder rexBuilder = cx.getRexBuilder();
+            final List<RexNode> operands = convertExpressionList(cx,
+                call.getOperandList(), SqlOperandTypeChecker.Consistency.NONE);
+            final RelDataType type =
+                cx.getValidator().getValidatedNodeType(call);
+            final List<RexNode> exprs = new ArrayList<RexNode>();
+            for (int i = 1; i < operands.size() - 1; i += 2) {
+              exprs.add(
+                  RelOptUtil.isDistinctFrom(rexBuilder, operands.get(0),
+                      operands.get(i), true));
+              exprs.add(operands.get(i + 1));
+            }
+            if (operands.size() % 2 == 0) {
+              exprs.add(Util.last(operands));
+            } else {
+              exprs.add(rexBuilder.makeNullLiteral(type.getSqlTypeName()));
+            }
+            return rexBuilder.makeCall(type, SqlStdOperatorTable.CASE, exprs);
+          }
+        });
+
     // Expand "x NOT LIKE y" into "NOT (x LIKE y)"
     registerOp(
         SqlStdOperatorTable.NOT_LIKE,
@@ -219,19 +272,19 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     // Similarly STDDEV_POP and STDDEV_SAMP, VAR_POP and VAR_SAMP.
     registerOp(
         SqlStdOperatorTable.AVG,
-        new AvgVarianceConvertlet(SqlAvgAggFunction.Subtype.AVG));
+        new AvgVarianceConvertlet(SqlKind.AVG));
     registerOp(
         SqlStdOperatorTable.STDDEV_POP,
-        new AvgVarianceConvertlet(SqlAvgAggFunction.Subtype.STDDEV_POP));
+        new AvgVarianceConvertlet(SqlKind.STDDEV_POP));
     registerOp(
         SqlStdOperatorTable.STDDEV_SAMP,
-        new AvgVarianceConvertlet(SqlAvgAggFunction.Subtype.STDDEV_SAMP));
+        new AvgVarianceConvertlet(SqlKind.STDDEV_SAMP));
     registerOp(
         SqlStdOperatorTable.VAR_POP,
-        new AvgVarianceConvertlet(SqlAvgAggFunction.Subtype.VAR_POP));
+        new AvgVarianceConvertlet(SqlKind.VAR_POP));
     registerOp(
         SqlStdOperatorTable.VAR_SAMP,
-        new AvgVarianceConvertlet(SqlAvgAggFunction.Subtype.VAR_SAMP));
+        new AvgVarianceConvertlet(SqlKind.VAR_SAMP));
 
     registerOp(
         SqlStdOperatorTable.FLOOR, new FloorCeilConvertlet(true));
@@ -297,7 +350,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     SqlNodeList thenList = call.getThenOperands();
     assert whenList.size() == thenList.size();
 
-    final List<RexNode> exprList = new ArrayList<RexNode>();
+    final List<RexNode> exprList = new ArrayList<>();
     for (int i = 0; i < whenList.size(); i++) {
       exprList.add(cx.convertExpression(whenList.get(i)));
       exprList.add(cx.convertExpression(thenList.get(i)));
@@ -792,7 +845,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
   private List<Integer> elseArgs(int count) {
     // If list is odd, e.g. [0, 1, 2, 3, 4] we get [1, 3, 4]
     // If list is even, e.g. [0, 1, 2, 3, 4, 5] we get [2, 4, 5]
-    List<Integer> list = new ArrayList<Integer>();
+    final List<Integer> list = new ArrayList<>();
     for (int i = count % 2;;) {
       list.add(i);
       i += 2;
@@ -988,7 +1041,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       return convertCall(cx, call);
     }
     final RexBuilder rexBuilder = cx.getRexBuilder();
-    final List<RexNode> columns = new ArrayList<RexNode>();
+    final List<RexNode> columns = new ArrayList<>();
     for (SqlNode operand : call.getOperandList()) {
       columns.add(
           rexBuilder.makeLiteral(
@@ -1088,17 +1141,17 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
   /** Convertlet that handles {@code AVG} and {@code VARIANCE}
    * windowed aggregate functions. */
   private static class AvgVarianceConvertlet implements SqlRexConvertlet {
-    private final SqlAvgAggFunction.Subtype subtype;
+    private final SqlKind kind;
 
-    public AvgVarianceConvertlet(SqlAvgAggFunction.Subtype subtype) {
-      this.subtype = subtype;
+    public AvgVarianceConvertlet(SqlKind kind) {
+      this.kind = kind;
     }
 
     public RexNode convertCall(SqlRexContext cx, SqlCall call) {
       assert call.operandCount() == 1;
       final SqlNode arg = call.operand(0);
       final SqlNode expr;
-      switch (subtype) {
+      switch (kind) {
       case AVG:
         expr = expandAvg(arg);
         break;
@@ -1115,7 +1168,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
         expr = expandVariance(arg, false, false);
         break;
       default:
-        throw Util.unexpected(subtype);
+        throw Util.unexpected(kind);
       }
       RelDataType type =
           cx.getValidator().getValidatedNodeType(call);
@@ -1198,6 +1251,79 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     }
   }
 
+  /** Convertlet that converts {@code LTRIM} and {@code RTRIM} to
+   * {@code TRIM}. */
+  private static class TrimConvertlet implements SqlRexConvertlet {
+    private final SqlTrimFunction.Flag flag;
+
+    public TrimConvertlet(SqlTrimFunction.Flag flag) {
+      this.flag = flag;
+    }
+
+    public RexNode convertCall(SqlRexContext cx, SqlCall call) {
+      final RexBuilder rexBuilder = cx.getRexBuilder();
+      final RexNode operand =
+          cx.convertExpression(call.getOperandList().get(0));
+      return rexBuilder.makeCall(SqlStdOperatorTable.TRIM,
+          rexBuilder.makeFlag(flag), rexBuilder.makeLiteral(" "), operand);
+    }
+  }
+
+  /** Convertlet that converts {@code GREATEST} and {@code LEAST}. */
+  private static class GreatestConvertlet implements SqlRexConvertlet {
+    public RexNode convertCall(SqlRexContext cx, SqlCall call) {
+      // Translate
+      //   GREATEST(a, b, c, d)
+      // to
+      //   CASE
+      //   WHEN a IS NULL OR b IS NULL OR c IS NULL OR d IS NULL
+      //   THEN NULL
+      //   WHEN a > b AND a > c AND a > d
+      //   THEN a
+      //   WHEN b > c AND b > d
+      //   THEN b
+      //   WHEN c > d
+      //   THEN c
+      //   ELSE d
+      //   END
+      final RexBuilder rexBuilder = cx.getRexBuilder();
+      final RelDataType type =
+          cx.getValidator().getValidatedNodeType(call);
+      final SqlBinaryOperator op;
+      switch (call.getKind()) {
+      case GREATEST:
+        op = SqlStdOperatorTable.GREATER_THAN;
+        break;
+      case LEAST:
+        op = SqlStdOperatorTable.LESS_THAN;
+        break;
+      default:
+        throw new AssertionError();
+      }
+      final List<RexNode> exprs = convertExpressionList(cx,
+          call.getOperandList(), SqlOperandTypeChecker.Consistency.NONE);
+      final List<RexNode> list = new ArrayList<>();
+      final List<RexNode> orList = new ArrayList<>();
+      for (RexNode expr : exprs) {
+        orList.add(rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, expr));
+      }
+      list.add(RexUtil.composeDisjunction(rexBuilder, orList, false));
+      list.add(rexBuilder.makeNullLiteral(type.getSqlTypeName()));
+      for (int i = 0; i < exprs.size() - 1; i++) {
+        RexNode expr = exprs.get(i);
+        final List<RexNode> andList = new ArrayList<>();
+        for (int j = i + 1; j < exprs.size(); j++) {
+          final RexNode expr2 = exprs.get(j);
+          andList.add(rexBuilder.makeCall(op, expr, expr2));
+        }
+        list.add(RexUtil.composeConjunction(rexBuilder, andList, false));
+        list.add(expr);
+      }
+      list.add(exprs.get(exprs.size() - 1));
+      return rexBuilder.makeCall(type, SqlStdOperatorTable.CASE, list);
+    }
+  }
+
   /** Convertlet that handles {@code FLOOR} and {@code CEIL} functions. */
   private class FloorCeilConvertlet implements SqlRexConvertlet {
     private final boolean floor;

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 7ec4fd5..f972db8 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -81,6 +81,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Properties;
 import java.util.Set;
+import java.util.StringTokenizer;
 import java.util.TimeZone;
 import java.util.jar.JarFile;
 import java.util.regex.Matcher;
@@ -1222,6 +1223,28 @@ public class Util {
     return toString(strings, "", "\n", "");
   }
 
+  /** Converts a string into tokens. */
+  public static Iterable<String> tokenize(final String s, final String delim) {
+    return new Iterable<String>() {
+      final StringTokenizer t = new StringTokenizer(s, delim);
+      public Iterator<String> iterator() {
+        return new Iterator<String>() {
+          public boolean hasNext() {
+            return t.hasMoreTokens();
+          }
+
+          public String next() {
+            return t.nextToken();
+          }
+
+          public void remove() {
+            throw new UnsupportedOperationException("remove");
+          }
+        };
+      }
+    };
+  }
+
   /**
    * Converts a Java timezone to POSIX format, so that the boost C++ library
    * can instantiate timezone objects.

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/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 a6b671e..51670aa 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
@@ -29,6 +29,7 @@ import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.sql.validate.SqlValidatorWithHints;
+import org.apache.calcite.test.CalciteAssert;
 import org.apache.calcite.test.MockCatalogReader;
 import org.apache.calcite.test.MockSqlOperatorTable;
 
@@ -45,12 +46,19 @@ import com.google.common.collect.ImmutableMap;
 */
 public class DefaultSqlTestFactory implements SqlTestFactory {
   public static final ImmutableMap<String, Object> DEFAULT_OPTIONS =
-      ImmutableMap.of(
-          "quoting", (Object) Quoting.DOUBLE_QUOTE,
-          "quotedCasing", Casing.UNCHANGED,
-          "unquotedCasing", Casing.TO_UPPER,
-          "caseSensitive", true,
-          "conformance", SqlConformance.DEFAULT);
+      ImmutableMap.<String, Object>builder()
+          .put("quoting", Quoting.DOUBLE_QUOTE)
+          .put("quotedCasing", Casing.UNCHANGED)
+          .put("unquotedCasing", Casing.TO_UPPER)
+          .put("caseSensitive", true)
+          .put("conformance", SqlConformance.DEFAULT)
+          .put("operatorTable", SqlStdOperatorTable.instance())
+          .put("connectionFactory",
+              CalciteAssert.EMPTY_CONNECTION_FACTORY
+                  .with(
+                      new CalciteAssert.AddSchemaSpecPostProcessor(
+                          CalciteAssert.SchemaSpec.HR)))
+          .build();
 
   public static final DefaultSqlTestFactory INSTANCE =
       new DefaultSqlTestFactory();
@@ -58,9 +66,10 @@ public class DefaultSqlTestFactory implements SqlTestFactory {
   private DefaultSqlTestFactory() {
   }
 
-  public SqlOperatorTable createOperatorTable() {
-    MockSqlOperatorTable opTab =
-        new MockSqlOperatorTable(SqlStdOperatorTable.instance());
+  public SqlOperatorTable createOperatorTable(SqlTestFactory factory) {
+    final SqlOperatorTable opTab0 =
+        (SqlOperatorTable) factory.get("operatorTable");
+    MockSqlOperatorTable opTab = new MockSqlOperatorTable(opTab0);
     MockSqlOperatorTable.addRamp(opTab);
     return opTab;
   }
@@ -75,7 +84,7 @@ public class DefaultSqlTestFactory implements SqlTestFactory {
   }
 
   public SqlValidator getValidator(SqlTestFactory factory) {
-    final SqlOperatorTable operatorTable = factory.createOperatorTable();
+    final SqlOperatorTable operatorTable = factory.createOperatorTable(factory);
     final boolean caseSensitive = (Boolean) factory.get("caseSensitive");
     final RelDataTypeFactory typeFactory =
         new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/test/java/org/apache/calcite/sql/test/DelegatingSqlTestFactory.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/DelegatingSqlTestFactory.java b/core/src/test/java/org/apache/calcite/sql/test/DelegatingSqlTestFactory.java
index 211f1f4..395c036 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/DelegatingSqlTestFactory.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/DelegatingSqlTestFactory.java
@@ -46,8 +46,8 @@ public class DelegatingSqlTestFactory implements SqlTestFactory {
     return factory.get(name);
   }
 
-  public SqlOperatorTable createOperatorTable() {
-    return factory.createOperatorTable();
+  public SqlOperatorTable createOperatorTable(SqlTestFactory factory) {
+    return this.factory.createOperatorTable(factory);
   }
 
   public SqlAdvisor createAdvisor(SqlValidatorWithHints validator) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/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 c11571a..81bfeed 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
@@ -33,12 +33,14 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlOperandCountRange;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.OracleSqlOperatorTable;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
 import org.apache.calcite.sql.type.BasicSqlType;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
 import org.apache.calcite.sql.util.SqlString;
 import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
@@ -292,6 +294,17 @@ public abstract class SqlOperatorBaseTest {
     tester.setFor(null);
   }
 
+  protected SqlTester oracleTester() {
+    return tester.withOperatorTable(
+        ChainedSqlOperatorTable.of(OracleSqlOperatorTable.instance(),
+            SqlStdOperatorTable.instance()))
+        .withConnectionFactory(
+            CalciteAssert.EMPTY_CONNECTION_FACTORY
+                .with(new CalciteAssert
+                    .AddSchemaSpecPostProcessor(CalciteAssert.SchemaSpec.HR))
+                .with("fun", "oracle"));
+  }
+
   //--- Tests -----------------------------------------------------------
 
   /**
@@ -4065,6 +4078,79 @@ public abstract class SqlOperatorBaseTest {
     }
   }
 
+  @Test public void testRtrimFunc() {
+    tester.setFor(OracleSqlOperatorTable.RTRIM);
+    final SqlTester tester1 = oracleTester();
+    tester1.checkString("rtrim(' aAa  ')", " aAa", "VARCHAR(6) NOT NULL");
+    tester1.checkNull("rtrim(CAST(NULL AS VARCHAR(6)))");
+  }
+
+  @Test public void testLtrimFunc() {
+    tester.setFor(OracleSqlOperatorTable.LTRIM);
+    final SqlTester tester1 = oracleTester();
+    tester1.checkString("ltrim(' aAa  ')", "aAa  ", "VARCHAR(6) NOT NULL");
+    tester1.checkNull("ltrim(CAST(NULL AS VARCHAR(6)))");
+  }
+
+  @Test public void testGreatestFunc() {
+    tester.setFor(OracleSqlOperatorTable.GREATEST);
+    final SqlTester tester1 = oracleTester();
+    tester1.checkString("greatest('on', 'earth')", "on   ", "CHAR(5) NOT NULL");
+    tester1.checkString("greatest('show', 'on', 'earth')", "show ",
+        "CHAR(5) NOT NULL");
+    tester1.checkScalar("greatest(12, CAST(NULL AS INTEGER), 3)", null, "INTEGER");
+    tester1.checkScalar("greatest(false, true)", true, "BOOLEAN NOT NULL");
+  }
+
+  @Test public void testLeastFunc() {
+    tester.setFor(OracleSqlOperatorTable.LEAST);
+    final SqlTester tester1 = oracleTester();
+    tester1.checkString("least('on', 'earth')", "earth", "CHAR(5) NOT NULL");
+    tester1.checkString("least('show', 'on', 'earth')", "earth",
+        "CHAR(5) NOT NULL");
+    tester1.checkScalar("least(12, CAST(NULL AS INTEGER), 3)", null, "INTEGER");
+    tester1.checkScalar("least(false, true)", false, "BOOLEAN NOT NULL");
+  }
+
+  @Test public void testNvlFunc() {
+    tester.setFor(OracleSqlOperatorTable.NVL);
+    final SqlTester tester1 = oracleTester();
+    tester1.checkScalar("nvl(1, 2)", "1", "INTEGER NOT NULL");
+    tester1.checkFails("^nvl(1, true)^", "Parameters must be of the same type",
+        false);
+    tester1.checkScalar("nvl(true, false)", true, "BOOLEAN NOT NULL");
+    tester1.checkScalar("nvl(false, true)", false, "BOOLEAN NOT NULL");
+    tester1.checkString("nvl('abc', 'de')", "abc", "CHAR(3) NOT NULL");
+    tester1.checkString("nvl('abc', 'defg')", "abc ", "CHAR(4) NOT NULL");
+    tester1.checkString("nvl('abc', CAST(NULL AS VARCHAR(20)))", "abc",
+        "VARCHAR(20) NOT NULL");
+    tester1.checkString("nvl(CAST(NULL AS VARCHAR(20)), 'abc')", "abc",
+        "VARCHAR(20) NOT NULL");
+    tester1.checkNull(
+        "nvl(CAST(NULL AS VARCHAR(6)), cast(NULL AS VARCHAR(4)))");
+  }
+
+  @Test public void testDecodeFunc() {
+    tester.setFor(OracleSqlOperatorTable.DECODE);
+    final SqlTester tester1 = oracleTester();
+    tester1.checkScalar("decode(0, 0, 'a', 1, 'b', 2, 'c')", "a", "CHAR(1)");
+    tester1.checkScalar("decode(1, 0, 'a', 1, 'b', 2, 'c')", "b", "CHAR(1)");
+    // if there are duplicates, take the first match
+    tester1.checkScalar("decode(1, 0, 'a', 1, 'b', 1, 'z', 2, 'c')", "b",
+        "CHAR(1)");
+    // if there's no match, and no "else", return null
+    tester1.checkScalar("decode(3, 0, 'a', 1, 'b', 2, 'c')", null, "CHAR(1)");
+    // if there's no match, return the "else" value
+    tester1.checkScalar("decode(3, 0, 'a', 1, 'b', 2, 'c', 'd')", "d",
+        "CHAR(1) NOT NULL");
+    tester1.checkScalar("decode(1, 0, 'a', 1, 'b', 2, 'c', 'd')", "b",
+        "CHAR(1) NOT NULL");
+    // nulls match
+    tester1.checkScalar("decode(cast(null as integer), 0, 'a',\n"
+        + " cast(null as integer), 'b', 2, 'c', 'd')", "b",
+        "CHAR(1) NOT NULL");
+  }
+
   @Test public void testWindow() {
     if (!enable) {
       return;
@@ -5353,12 +5439,8 @@ public abstract class SqlOperatorBaseTest {
     }
   }
 
-  /**
-   * Creates a {@link org.apache.calcite.sql.test.SqlTester} based on a JDBC
-   * connection.
-   */
-  public static SqlTester tester(Connection connection) {
-    return new TesterImpl(connection);
+  public static SqlTester tester() {
+    return new TesterImpl(DefaultSqlTestFactory.INSTANCE);
   }
 
   /**
@@ -5366,11 +5448,8 @@ public abstract class SqlOperatorBaseTest {
    * JDBC connection.
    */
   protected static class TesterImpl extends SqlTesterImpl {
-    final Connection connection;
-
-    public TesterImpl(Connection connection) {
-      super(DefaultSqlTestFactory.INSTANCE);
-      this.connection = connection;
+    public TesterImpl(SqlTestFactory testFactory) {
+      super(testFactory);
     }
 
     @Override public void check(
@@ -5381,31 +5460,30 @@ public abstract class SqlOperatorBaseTest {
           query,
           typeChecker,
           resultChecker);
-      Statement statement = null;
-      try {
-        statement = connection.createStatement();
+      //noinspection unchecked
+      final CalciteAssert.ConnectionFactory connectionFactory =
+          (CalciteAssert.ConnectionFactory)
+              getFactory().get("connectionFactory");
+      try (Connection connection = connectionFactory.createConnection();
+           Statement statement = connection.createStatement()) {
         final ResultSet resultSet =
             statement.executeQuery(query);
         resultChecker.checkResult(resultSet);
       } catch (Exception e) {
-        throw new RuntimeException(e);
-      } finally {
-        if (statement != null) {
-          try {
-            statement.close();
-          } catch (SQLException e) {
-            // ignore
-          }
-        }
+        throw Throwables.propagate(e);
       }
     }
 
-    public void close() {
-      try {
-        connection.close();
-      } catch (SQLException e) {
-        throw new RuntimeException(e);
-      }
+    @Override protected TesterImpl with(final String name2, final Object value) {
+      return new TesterImpl(
+          new DelegatingSqlTestFactory(factory) {
+            @Override public Object get(String name) {
+              if (name.equals(name2)) {
+                return value;
+              }
+              return super.get(name);
+            }
+          });
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/test/java/org/apache/calcite/sql/test/SqlTestFactory.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTestFactory.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTestFactory.java
index 7aadf37..fabae76 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTestFactory.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTestFactory.java
@@ -28,7 +28,7 @@ import org.apache.calcite.sql.validate.SqlValidatorWithHints;
  * @see org.apache.calcite.sql.test.SqlTester
 */
 public interface SqlTestFactory {
-  SqlOperatorTable createOperatorTable();
+  SqlOperatorTable createOperatorTable(SqlTestFactory factory);
   SqlParser createParser(SqlTestFactory factory, String sql);
   SqlValidator getValidator(SqlTestFactory factory);
   SqlAdvisor createAdvisor(SqlValidatorWithHints validator);

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
index 4144eb1..2873c6d 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
@@ -21,8 +21,10 @@ import org.apache.calcite.avatica.util.Quoting;
 import org.apache.calcite.config.Lex;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.test.CalciteAssert;
 import org.apache.calcite.test.SqlValidatorTestCase;
 
 import java.sql.ResultSet;
@@ -76,6 +78,13 @@ public interface SqlTester extends AutoCloseable, SqlValidatorTestCase.Tester {
    * version. */
   SqlTester withConformance(SqlConformance conformance);
 
+  /** Returns a tester that gets connections from a given factory. */
+  SqlTester withConnectionFactory(
+      CalciteAssert.ConnectionFactory connectionFactory);
+
+  /** Returns a tester that uses a given operator table. */
+  SqlTester withOperatorTable(SqlOperatorTable operatorTable);
+
   /**
    * Tests that a scalar SQL expression returns the expected result and the
    * expected type. For example,

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
index 98741f4..4770344 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
@@ -29,6 +29,7 @@ import org.apache.calcite.sql.SqlIntervalLiteral;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
@@ -42,6 +43,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorNamespace;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.test.CalciteAssert;
 import org.apache.calcite.test.SqlValidatorTestCase;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.TestUtil;
@@ -72,7 +74,7 @@ import static org.junit.Assert.fail;
  * that talks to a mock catalog.
  */
 public class SqlTesterImpl implements SqlTester {
-  private final SqlTestFactory factory;
+  protected final SqlTestFactory factory;
 
   public SqlTesterImpl(SqlTestFactory factory) {
     this.factory = factory;
@@ -303,7 +305,16 @@ public class SqlTesterImpl implements SqlTester {
     return with("conformance", conformance);
   }
 
-  private SqlTesterImpl with(final String name2, final Object value) {
+  public SqlTester withOperatorTable(SqlOperatorTable operatorTable) {
+    return with("operatorTable", operatorTable);
+  }
+
+  public SqlTester withConnectionFactory(
+      CalciteAssert.ConnectionFactory connectionFactory) {
+    return with("connectionFactory", connectionFactory);
+  }
+
+  protected SqlTesterImpl with(final String name2, final Object value) {
     return new SqlTesterImpl(
         new DelegatingSqlTestFactory(factory) {
           @Override public Object get(String name) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index e15664f..729326a 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -128,7 +128,7 @@ public class CalciteAssert {
     UTC_TIMESTAMP_FORMAT.setTimeZone(utc);
   }
 
-  private static final ConnectionFactory EMPTY_CONNECTION_FACTORY =
+  public static final ConnectionFactory EMPTY_CONNECTION_FACTORY =
       new MapConnectionFactory(ImmutableMap.<String, String>of(),
           ImmutableList.<ConnectionPostProcessor>of());
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/test/java/org/apache/calcite/test/CalciteSqlOperatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSqlOperatorTest.java b/core/src/test/java/org/apache/calcite/test/CalciteSqlOperatorTest.java
index 7373628..080a45b 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSqlOperatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSqlOperatorTest.java
@@ -17,33 +17,14 @@
 package org.apache.calcite.test;
 
 import org.apache.calcite.sql.test.SqlOperatorBaseTest;
-import org.apache.calcite.sql.test.SqlTester;
-
-import java.sql.Connection;
 
 /**
  * Embodiment of {@link org.apache.calcite.sql.test.SqlOperatorBaseTest}
  * that generates SQL statements and executes them using Calcite.
  */
 public class CalciteSqlOperatorTest extends SqlOperatorBaseTest {
-  private static final ThreadLocal<Connection> LOCAL =
-      new ThreadLocal<Connection>() {
-        @Override protected Connection initialValue() {
-          try {
-            return CalciteAssert.that().with(
-                CalciteAssert.SchemaSpec.HR).connect();
-          } catch (Exception e) {
-            throw new RuntimeException(e);
-          }
-        }
-      };
-
-  private static SqlTester getHrTester() {
-    return tester(LOCAL.get());
-  }
-
   public CalciteSqlOperatorTest() {
-    super(false, getHrTester());
+    super(false, tester());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/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 d7b635c..b57feee 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -5960,6 +5960,21 @@ public class JdbcTest {
         .returnsUnordered("empno=1");
   }
 
+  @Test public void testFunOracle() {
+    CalciteAssert.that(CalciteAssert.Config.REGULAR)
+        .with("fun", "oracle")
+        .query("select nvl(\"commission\", -99) as c from \"hr\".\"emps\"")
+        .returnsUnordered("C=-99",
+            "C=1000",
+            "C=250",
+            "C=500");
+
+    // NVL is not present in the default operator table
+    CalciteAssert.that(CalciteAssert.Config.REGULAR)
+        .query("select nvl(\"commission\", -99) as c from \"hr\".\"emps\"")
+        .throws_("No match found for function signature NVL(<NUMERIC>, <NUMERIC>)");
+  }
+
   /** Tests that {@link Hook#PARSE_TREE} works. */
   @Test public void testHook() {
     final int[] callCount = {0};

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
index 46a2fb4..0e67640 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
@@ -137,7 +137,7 @@ public class SqlValidatorFeatureTest extends SqlValidatorTestCase {
       SqlConformance conformance = (SqlConformance) get("conformance");
       final boolean caseSensitive = (Boolean) get("caseSensitive");
       return new FeatureValidator(
-          factory.createOperatorTable(),
+          factory.createOperatorTable(factory),
           new MockCatalogReader(typeFactory, caseSensitive).init(),
           typeFactory,
           conformance);

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 5a9d0b7..285cbc9 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -228,6 +228,18 @@ public class UtilTest {
   }
 
   /**
+   * Unit-test for {@link Util#tokenize(String, String)}.
+   */
+  @Test public void testTokenize() {
+    final List<String> list = new ArrayList<>();
+    for (String s : Util.tokenize("abc,de,f", ",")) {
+      list.add(s);
+    }
+    assertThat(list.size(), is(3));
+    assertThat(list.toString(), is("[abc, de, f]"));
+  }
+
+  /**
    * Unit-test for {@link BitString}.
    */
   @Test public void testBitString() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/707e72d5/site/_docs/adapter.md
----------------------------------------------------------------------
diff --git a/site/_docs/adapter.md b/site/_docs/adapter.md
index fa8eb21..f40c5b8 100644
--- a/site/_docs/adapter.md
+++ b/site/_docs/adapter.md
@@ -36,3 +36,25 @@ limitations under the License.
 
 * <a href="{{ site.apiRoot }}/org/apache/calcite/jdbc/package-summary.html">JDBC driver</a>
 
+The basic form of the JDBC connect string is
+
+  "jdbc:calcite:?prop=value&prop2=value2&prop3=value3"
+
+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#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.
+| <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#FUN">fun</a> | Collection of built-in functions and operators. Valid values: "standard" (the default), "oracle".
+| <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#LEX">lex</a> | Lexical policy. Values are ORACLE (default), MYSQL, MYSQL_ANSI, SQL_SERVER, JAVA.
+| <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#MATERIALIZATIONS_ENABLED">materializationsEnabled</a> | Whether Calcite should use materializations. Default false.
+| <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#MODEL">model</a> | URI of the JSON model file.
+| <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#QUOTING">quoting</a> | How identifiers are quoted. Values are DOUBLE_QUOTE, BACK_QUOTE, BRACKET. If not specified, value from `lex` is used.
+| <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#SCHEMA">schema</a> | Name of initial schema.
+| <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#SPARK">spark</a> | Specifies whether Spark should be used as the engine for processing that cannot be pushed to the source system. If false (the default), Calcite generates code that implements the Enumerable interface.
+| <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#TIME_ZONE">timeZone</a> | Time zone, for example "gmt-3". Default is the JVM's time zone.
+| <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#TYPE_SYSTEM">typeSystem</a> | Type system. The name of a class that implements <a href="{{ site.apiRoot }}/org/apache/calcite/rel/type/RelDataTypeSystem.html">RelDataTypeSystem</a> and has a public default constructor or an `INSTANCE` constant.
+| <a href="{{ site.apiRoot }}/org/apache/calcite/config/CalciteConnectionProperty.html#UNQUOTED_CASING">unquotedCasing</a> | How identifiers are stored if they are not quoted. Values are UNCHANGED, TO_UPPER, TO_LOWER. If not specified, value from `lex` is used.


[3/4] calcite git commit: Add SQL-Gremlin to Adapters page

Posted by jh...@apache.org.
Add SQL-Gremlin to Adapters page

Add description of JDBC connect string


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

Branch: refs/heads/master
Commit: 9cd2f40267d7e4ad385f86cb59e9dc39deefc86d
Parents: 558e442
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Feb 25 15:49:57 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Mar 4 15:25:59 2016 -0800

----------------------------------------------------------------------
 .../calcite/avatica/ConnectStringParser.java    | 11 +++---
 site/_docs/adapter.md                           | 37 ++++++++++++++++++--
 2 files changed, 40 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/9cd2f402/avatica/src/main/java/org/apache/calcite/avatica/ConnectStringParser.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/ConnectStringParser.java b/avatica/src/main/java/org/apache/calcite/avatica/ConnectStringParser.java
index 61de10a..0145024 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ConnectStringParser.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ConnectStringParser.java
@@ -22,8 +22,8 @@ import java.util.Properties;
 
 /**
  * ConnectStringParser is a utility class that parses or creates a JDBC connect
- * string according to the OLE DB connect string syntax described at <a
- * href="http://msdn.microsoft.com/library/default.asp?url=/library/en-us/oledb/htm/oledbconnectionstringsyntax.asp">
+ * string according to the
+ * <a href="https://msdn.microsoft.com/en-us/library/windows/desktop/ms722656(v=vs.85).aspx">
  * OLE DB Connection String Syntax</a>.
  *
  * <p>This code was adapted from Mondrian's mondrian.olap.Util class.
@@ -32,7 +32,7 @@ import java.util.Properties;
  * <ul>
  * <li>use of regular {@link Properties} for compatibility with the JDBC API
  * (replaces Mondrian's use of its own order-preserving and case-insensitive
- * PropertyList, found in Util.java at link above)</li>
+ * PropertyList)</li>
  *
  * <li>ability to pass to {@link #parse} a pre-existing Properties object into
  * which properties are to be parsed, possibly overriding prior values</li>
@@ -217,7 +217,7 @@ public class ConnectStringParser {
       String value = parseQuoted(c);
 
       // skip over trailing white space
-      while ((i < n) && ((c = s.charAt(i)) == ' ')) {
+      while (i < n && s.charAt(i) == ' ') {
         i++;
       }
       if (i >= n) {
@@ -327,7 +327,7 @@ public class ConnectStringParser {
       // write parameter value
       len = value.length();
       boolean hasSemi = value.indexOf(';') >= 0;
-      boolean hasSQ = value.indexOf("'") >= 0;
+      boolean hasSQ = value.indexOf('\'') >= 0;
       boolean hasDQ = value.indexOf('"') >= 0;
       if (value.startsWith(" ") || value.endsWith(" ")) {
         quote = "'";
@@ -383,6 +383,7 @@ public class ConnectStringParser {
    */
   public static Map<String, String> toMap(
       final Properties properties) {
+    //noinspection unchecked
     return (Map) properties;
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/9cd2f402/site/_docs/adapter.md
----------------------------------------------------------------------
diff --git a/site/_docs/adapter.md b/site/_docs/adapter.md
index f40c5b8..09c5904 100644
--- a/site/_docs/adapter.md
+++ b/site/_docs/adapter.md
@@ -22,8 +22,6 @@ limitations under the License.
 {% endcomment %}
 -->
 
-* <a href="https://github.com/apache/drill">Apache Drill adapter</a>
-* Cascading adapter (<a href="https://github.com/Cascading/lingual">Lingual</a>)
 * Cassandra adapter (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/cassandra/package-summary.html">calcite-cassandra</a>)
 * CSV adapter (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/csv/package-summary.html">example/csv</a>)
 * JDBC adapter (part of <a href="{{ site.apiRoot }}/org/apache/calcite/adapter/jdbc/package-summary.html">calcite-core</a>)
@@ -32,13 +30,46 @@ limitations under the License.
 * Splunk adapter (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/splunk/package-summary.html">calcite-splunk</a>)
 * Eclipse Memory Analyzer (MAT) adapter (<a href="https://github.com/vlsi/mat-calcite-plugin">mat-calcite-plugin</a>)
 
+## Engines
+
+The following standalone engines are powered by Apache Calcite.
+
+* <a href="https://drill.apache.org">Apache Drill</a>
+  uses Calcite for SQL parsing and query optimization
+* <a href="https://flink.apache.org">Apache Flink</a>
+  uses Calcite for parsing both regular and streaming SQL,
+  and for query optimization
+* <a href="https://hive.apache.org">Apache Hive</a>
+  uses Calcite for query optimization
+* <a href="https://kylin.apache.org">Apache Kylin</a>
+  uses Calcite for SQL parsing and query optimization
+* <a href="https://phoenix.apache.org">Apache Phoenix</a>
+  uses Calcite for SQL parsing and query optimization (under development),
+  and also uses Avatica for its remote JDBC driver
+* <a href="https://github.com/milinda/samza-sql">SamzaSQL</a>,
+  an extension to
+  <a href="https://samza.apache.org">Apache Samza</a>,
+  uses Calcite for parsing streaming SQL and query optimization
+* <a href="https://storm.apache.org">Apache Storm</a>
+  uses Calcite for parsing streaming SQL and query optimization
+* Cascading adapter (<a href="https://github.com/Cascading/lingual">Lingual</a>)
+* <a href="https://github.com/twilmes/sql-gremlin">SQL-Gremlin</a>,
+  a SQL interface to a
+  <a href="http://tinkerpop.incubator.apache.org/">Apache TinkerPop</a>-enabled
+  graph database
+
 ## Drivers
 
 * <a href="{{ site.apiRoot }}/org/apache/calcite/jdbc/package-summary.html">JDBC driver</a>
 
 The basic form of the JDBC connect string is
 
-  "jdbc:calcite:?prop=value&prop2=value2&prop3=value3"
+  jdbc:calcite:property=value;property2=value2
+
+where `property`, `property2` are properties as described below.
+(Connect strings are compliant with OLE DB Connect String syntax,
+as implemented by Avatica's
+<a href="{{ site.apiRoot }}/org/apache/calcite/avatica/ConnectStringParser.html">ConnectStringParser</a>.)
 
 JDBC connect string parameters
 


[4/4] calcite git commit: [CALCITE-1111] Upgrade Guava, and test on a range of Guava versions

Posted by jh...@apache.org.
[CALCITE-1111] Upgrade Guava, and test on a range of Guava versions

Add compatibility note to release notes.

Add some history.


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

Branch: refs/heads/master
Commit: 5897dcd9d59d260a7d2b5c2dde257291fba5f9d1
Parents: 9cd2f40
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Mar 2 11:24:49 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Mar 4 15:26:00 2016 -0800

----------------------------------------------------------------------
 pom.xml               |   7 +--
 site/_docs/history.md | 104 ++++++++++++++++++++++++++++++++++++++++++++-
 site/_docs/howto.md   |  17 ++++++--
 3 files changed, 121 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/5897dcd9/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 365cb6e..cd92cd7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -68,9 +68,10 @@ limitations under the License.
     <freemarker.version>2.3.19</freemarker.version>
     <git-commit-id-plugin.version>2.1.9</git-commit-id-plugin.version>
 
-    <!-- We support guava versions as old as 14.0.1 (the version used by Hive)
-         but prefer more recent versions. -->
-    <guava.version>14.0.1</guava.version>
+    <!-- We support (and test against) Guava versions between
+         12.0.1 (HDFS, HBase) and 19.0 (most recent).
+         Override with your preferred version and it should work. -->
+    <guava.version>18.0</guava.version>
     <h2.version>1.4.185</h2.version>
     <hadoop.version>2.6.0</hadoop.version>
     <hamcrest.version>1.3</hamcrest.version>

http://git-wip-us.apache.org/repos/asf/calcite/blob/5897dcd9/site/_docs/history.md
----------------------------------------------------------------------
diff --git a/site/_docs/history.md b/site/_docs/history.md
index d46bfc8..073440f 100644
--- a/site/_docs/history.md
+++ b/site/_docs/history.md
@@ -35,13 +35,110 @@ One notable change is that the use of JUL (java.util.logging) has been replaced
 with [SLF4J](http://slf4j.org/). SLF4J provides an API which Calcite can use
 independent of the logging implementation. This ultimately provides additional
 flexibility to users, allowing them to configure Calcite's logging within their
-own chosen logging framework. This work was done in [CALCITE-669](https://issues.apache.org/jira/browse/CALCITE-669).
+own chosen logging framework. This work was done in
+[[CALCITE-669](https://issues.apache.org/jira/browse/CALCITE-669)].
 
 For users experienced with configuring JUL in Calcite previously, there are some
 differences as some the JUL logging levels do not exist in SLF4J: `FINE`,
 `FINER`, and `FINEST`, specifically. To deal with this, `FINE` was mapped
 to SLF4J's `DEBUG` level, while `FINER` and `FINEST` were mapped to SLF4J's `TRACE`.
 
+Compatibility: This release is tested
+on Linux, Mac OS X, Microsoft Windows;
+using Oracle JDK 1.7, 1.8;
+Guava versions 12.0.1 to 19.0;
+other software versions as specified in `pom.xml`.
+
+New features
+
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1066">CALCITE-1066</a>]
+  Add Oracle function table, and functions `DECODE`, `NVL`, `LTRIM`, `RTRIM`,
+  `GREATEST`, `LEAST`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1080">CALCITE-1080</a>]
+  Cassandra adapter (Michael Mior)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1062">CALCITE-1062</a>]
+  In validation, lookup a (possibly overloaded) operator from an operator
+  table (Hsuan-Yi Chu)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-551">CALCITE-551</a>]
+  Sub-query inside aggregate function
+
+Avatica features and bug fixes
+
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-642">CALCITE-642</a>]
+  Add an avatica-metrics API
+  * [<a href="https://issues.apache.org/jira/browse/CALCITE-1085">CALCITE-1085</a>]
+    Use a NoopContext singleton in NoopTimer
+
+Planner rules
+
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1109">CALCITE-1109</a>]
+  Fix up condition when pushing `Filter` through `Aggregate` (Amogh Margoor)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1100">CALCITE-1100</a>]
+  If constant reduction no-ops, don't create a new `RelNode` (Hsuan-Yi Chu)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1076">CALCITE-1076</a>]
+  Update `RelMdDistribution` to match other metadata APIs (Ted Xu)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1056">CALCITE-1056</a>]
+  In `RelBuilder`, simplify predicates, and optimize away `WHERE FALSE`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1059">CALCITE-1059</a>]
+  Not valid to convert `Aggregate` on empty to empty if its `GROUP BY` key is empty
+
+Bug fixes, API changes and minor enhancements
+
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1111">CALCITE-1111</a>]
+  Upgrade Guava, and test on a range of Guava versions
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1054">CALCITE-1054</a>]
+  Wrong code generation for `TIMESTAMP` values that may be `NULL`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-604">CALCITE-604</a>]
+  Tune metadata by generating a dispatcher at runtime
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1063">CALCITE-1063</a>]
+  Flat lists for 4, 5, 6 elements
+* Add Orinoco schema (streaming retail data), accessible from Quidem scripts
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1097">CALCITE-1097</a>]
+  Exception when executing query with too many aggregation columns (chenzhifa)
+* Add tests for leap days
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-553">CALCITE-553</a>]
+  In maven, enable compiler profiles by default
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1031">CALCITE-1031</a>]
+  In prepared statement, `CsvScannableTable.scan` is called twice
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1046">CALCITE-1046</a>]
+  Matchers for testing SQL query results
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1083">CALCITE-1083</a>]
+  `SqlNode.equalsDeep` has O(n ^ 2) performance
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-998">CALCITE-998</a>]
+  Exception when calling `STDDEV_SAMP`, `STDDEV_POP` (Matthew Shaer)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1071">CALCITE-1071</a>]
+  Improve hash functions
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1072">CALCITE-1072</a>]
+  CSV adapter incorrectly parses `TIMESTAMP` values after noon (Chris Albright)
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-669">CALCITE-669</a>]
+  Mass removal of Java Logging for SLF4J
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1068">CALCITE-1068</a>]
+  Deprecate `Stacks`
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1067">CALCITE-1067</a>]
+  Test failures due to clashing temporary table names
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-864">CALCITE-864</a>]
+  Correlation variable has incorrect row type if it is populated by right side
+  of a Join
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1021">CALCITE-1021</a>]
+  Upgrade Jackson
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-999">CALCITE-999</a>]
+  Clean up maven POM files
+
+Web site and documentation
+
+* Add SQL-Gremlin to Adapters page
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1090">CALCITE-1090</a>]
+  Revise Streaming SQL specification
+* Appoint Josh Elser to PMC
+* Add "Streaming SQL" talk
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-623">CALCITE-623</a>]
+  Add a small blurb to the site about Jenkins for CI
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1070">CALCITE-1070</a>]
+  Upgrade to new Apache logo
+* Document how to announce a release
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-1074">CALCITE-1074</a>]
+  Delete old releases from mirroring system
+
 ## <a href="https://github.com/apache/calcite/releases/tag/calcite-1.6.0">1.6.0</a> / 2016-01-22
 {: #v1-6-0}
 
@@ -75,6 +172,11 @@ cyclic metadata calls and produce a safe answer to the metadata
 request. It will also allow us to add finer-grained caching and
 further tune the metadata layer.
 
+Compatibility: This release is tested
+on Linux, Mac OS X, Microsoft Windows;
+using Oracle JDK 1.7, 1.8;
+other software versions as specified in `pom.xml`.
+
 New features
 
 * [<a href="https://issues.apache.org/jira/browse/CALCITE-816">CALCITE-816</a>]

http://git-wip-us.apache.org/repos/asf/calcite/blob/5897dcd9/site/_docs/howto.md
----------------------------------------------------------------------
diff --git a/site/_docs/howto.md b/site/_docs/howto.md
index 13f2e72..5954c33 100644
--- a/site/_docs/howto.md
+++ b/site/_docs/howto.md
@@ -428,13 +428,24 @@ Before you start:
 * Make sure you are using JDK 1.7 (not 1.8).
 * Check that `README` and `site/_docs/howto.md` have the correct version number.
 * Set `version.major` and `version.minor` in `pom.xml`.
-* Make sure build and tests succeed, including with
-  -Dcalcite.test.db={mysql,hsqldb}, -Dcalcite.test.slow,
-  -Dcalcite.test.mongodb, -Dcalcite.test.splunk.
+* Make sure build and tests succeed, including with `-P it,it-oracle`.
+  Supported configurations are:
+  * JDK 1.7, 1.8;
+  * Linux, Mac OS X, and Windows;
+  * Guava versions 12.0.1, 18.0 (the default) and 19.0 (specify `-Dguava.version=x.y`)
+* Optional extra tests:
+  * `-Dcalcite.test.db=mysql`
+  * `-Dcalcite.test.db=hsqldb`
+  * `-Dcalcite.test.slow`
+  * `-Dcalcite.test.mongodb`
+  * `-Dcalcite.test.splunk`
 * Trigger a
   <a href="https://scan.coverity.com/projects/2966">Coverity scan</a>
   by merging the latest code into the `julianhyde/coverity_scan` branch,
   and when it completes, make sure that there are no important issues.
+* Add release notes to `site/_docs/history.md`. Include the commit history,
+  and say which versions of Java, Guava and operating systems the release is
+  tested against.
 * Make sure that
   <a href="https://issues.apache.org/jira/issues/?jql=project%20%3D%20CALCITE%20AND%20status%20%3D%20Resolved%20and%20fixVersion%20is%20null">
   every "resolved" JIRA case</a> (including duplicates) has


[2/4] calcite git commit: Update copyright year

Posted by jh...@apache.org.
Update copyright year


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

Branch: refs/heads/master
Commit: 558e442459215aeb0c4ce9106472679217c27e7c
Parents: 707e72d
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Feb 4 21:14:24 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Mar 4 15:25:59 2016 -0800

----------------------------------------------------------------------
 NOTICE | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/558e4424/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index 12e3133..0dab09d 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,5 +1,5 @@
 Apache Calcite
-Copyright 2012-2015 The Apache Software Foundation
+Copyright 2012-2016 The Apache Software Foundation
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).