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 2018/12/06 08:51:50 UTC

[3/3] calcite git commit: [CALCITE-2670] Combine similar JSON aggregate functions in operator table

[CALCITE-2670] Combine similar JSON aggregate functions in operator table

Convert parameters from Enum to more specific types;
include varying flag values in JSON aggregate functions.

Some cosmetic stuff.

Close apache/calcite#916


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

Branch: refs/heads/master
Commit: f3655e15a11a9fb266af290cb390e690d4301c09
Parents: 40d12b7
Author: hongzezhang <ho...@tencent.com>
Authored: Wed Nov 14 19:48:49 2018 +0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Dec 5 16:59:02 2018 -0800

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |  26 ++---
 .../calcite/adapter/enumerable/RexImpTable.java |  34 +++---
 .../calcite/jdbc/JavaTypeFactoryImpl.java       |   2 +
 .../apache/calcite/rel/core/AggregateCall.java  |   2 +-
 .../apache/calcite/runtime/SqlFunctions.java    |  18 +---
 .../sql/SqlJsonConstructorNullClause.java       |   9 +-
 .../apache/calcite/sql/SqlJsonEmptyOrError.java |   5 +-
 .../calcite/sql/SqlJsonExistsErrorBehavior.java |   5 +-
 .../sql/SqlJsonQueryEmptyOrErrorBehavior.java   |   2 +-
 .../sql/SqlJsonValueEmptyOrErrorBehavior.java   |   2 +-
 .../calcite/sql/dialect/MssqlSqlDialect.java    |   4 +-
 .../sql/fun/SqlJsonArrayAggAggFunction.java     |  51 ++++++---
 .../calcite/sql/fun/SqlJsonArrayFunction.java   |   2 +-
 .../sql/fun/SqlJsonObjectAggAggFunction.java    |  47 +++++---
 .../calcite/sql/fun/SqlJsonObjectFunction.java  |   2 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  16 +--
 .../org/apache/calcite/util/BuiltInMethod.java  |  21 ++--
 .../calcite/test/SqlToRelConverterTest.java     |  58 ++++++++++
 .../calcite/test/SqlToRelConverterTest.xml      | 108 +++++++++++++++++++
 core/src/test/resources/sql/misc.iq             |   6 ++
 .../java/org/apache/calcite/test/CsvTest.java   |   8 +-
 21 files changed, 304 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/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 cf6b3f2..6cecc56 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -5267,14 +5267,8 @@ SqlCall JsonObjectAggFunctionCall() :
         }
     ]
     <RPAREN> {
-        switch (nullClause) {
-        case ABSENT_ON_NULL:
-          return SqlStdOperatorTable.JSON_OBJECTAGG_ABSENT_ON_NULL.createCall(span.end(this), args);
-        case NULL_ON_NULL:
-          return SqlStdOperatorTable.JSON_OBJECTAGG_NULL_ON_NULL.createCall(span.end(this), args);
-        default:
-          return SqlStdOperatorTable.JSON_OBJECTAGG_NULL_ON_NULL.createCall(span.end(this), args);
-        }
+        return SqlStdOperatorTable.JSON_OBJECTAGG.with(nullClause)
+            .createCall(span.end(this), args);
     }
 }
 
@@ -5313,7 +5307,7 @@ SqlCall JsonArrayFunctionCall() :
 
 SqlCall JsonArrayAggFunctionCall() :
 {
-    final SqlNode arg;
+    final SqlNode[] args = new SqlNode[1];
     List<SqlNode> list;
     final Span span;
     SqlJsonConstructorNullClause nullClause =
@@ -5322,21 +5316,17 @@ SqlCall JsonArrayAggFunctionCall() :
 }
 {
     <JSON_ARRAYAGG> { span = span(); }
-    <LPAREN> arg = JsonValueExpression(false)
+    <LPAREN> e = JsonValueExpression(false) {
+        args[0] = e;
+    }
     [
         e = JsonConstructorNullClause() {
             nullClause = (SqlJsonConstructorNullClause) ((SqlLiteral) e).getValue();
         }
     ]
     <RPAREN> {
-        switch (nullClause) {
-        case ABSENT_ON_NULL:
-          return SqlStdOperatorTable.JSON_ARRAYAGG_ABSENT_ON_NULL.createCall(span.end(this), arg);
-        case NULL_ON_NULL:
-          return SqlStdOperatorTable.JSON_ARRAYAGG_NULL_ON_NULL.createCall(span.end(this), arg);
-        default:
-          return SqlStdOperatorTable.JSON_ARRAYAGG_ABSENT_ON_NULL.createCall(span.end(this), arg);
-        }
+        return SqlStdOperatorTable.JSON_ARRAYAGG.with(nullClause)
+            .createCall(span.end(this), args);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 7776ac9..dd117ee 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -45,6 +45,8 @@ import org.apache.calcite.schema.impl.AggregateFunctionImpl;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlBinaryOperator;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlJsonArrayAggAggFunction;
+import org.apache.calcite.sql.fun.SqlJsonObjectAggAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.fun.SqlTrimFunction;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -157,12 +159,10 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_TRUE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ITEM;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_API_COMMON_SYNTAX;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_ARRAY;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_ARRAYAGG_ABSENT_ON_NULL;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_ARRAYAGG_NULL_ON_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_ARRAYAGG;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_EXISTS;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECT;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECTAGG_ABSENT_ON_NULL;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECTAGG_NULL_ON_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECTAGG;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_QUERY;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_STRUCTURED_VALUE_EXPRESSION;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_VALUE_ANY;
@@ -448,19 +448,13 @@ public class RexImpTable {
     defineMethod(JSON_VALUE_ANY, BuiltInMethod.JSON_VALUE_ANY.method, NullPolicy.NONE);
     defineMethod(JSON_QUERY, BuiltInMethod.JSON_QUERY.method, NullPolicy.NONE);
     defineMethod(JSON_OBJECT, BuiltInMethod.JSON_OBJECT.method, NullPolicy.NONE);
-    aggMap.put(JSON_OBJECTAGG_NULL_ON_NULL,
+    aggMap.put(JSON_OBJECTAGG,
         JsonObjectAggImplementor
-            .supplierFor(BuiltInMethod.JSON_OBJECTAGG_ADD_NULL_ON_NULL.method));
-    aggMap.put(JSON_OBJECTAGG_ABSENT_ON_NULL,
-        JsonObjectAggImplementor
-            .supplierFor(BuiltInMethod.JSON_OBJECTAGG_ADD_ABSENT_ON_NULL.method));
+            .supplierFor(BuiltInMethod.JSON_OBJECTAGG_ADD.method));
     defineMethod(JSON_ARRAY, BuiltInMethod.JSON_ARRAY.method, NullPolicy.NONE);
-    aggMap.put(JSON_ARRAYAGG_NULL_ON_NULL,
-        JsonArrayAggImplementor
-            .supplierFor(BuiltInMethod.JSON_ARRAYAGG_ADD_NULL_ON_NULL.method));
-    aggMap.put(JSON_ARRAYAGG_ABSENT_ON_NULL,
+    aggMap.put(JSON_ARRAYAGG,
         JsonArrayAggImplementor
-            .supplierFor(BuiltInMethod.JSON_ARRAYAGG_ADD_ABSENT_ON_NULL.method));
+            .supplierFor(BuiltInMethod.JSON_ARRAYAGG_ADD.method));
     defineImplementor(IS_JSON_VALUE, NullPolicy.NONE,
             new MethodImplementor(BuiltInMethod.IS_JSON_VALUE.method), false);
     defineImplementor(IS_JSON_OBJECT, NullPolicy.NONE,
@@ -1813,12 +1807,16 @@ public class RexImpTable {
     }
 
     @Override public void implementAdd(AggContext info, AggAddContext add) {
+      final SqlJsonObjectAggAggFunction function =
+          (SqlJsonObjectAggAggFunction) info.aggregation();
       add.currentBlock().add(
           Expressions.statement(
               Expressions.call(m,
                   Iterables.concat(
                       Collections.singletonList(add.accumulator().get(0)),
-                      add.arguments()))));
+                      add.arguments(),
+                      Collections.singletonList(
+                          Expressions.constant(function.getNullClause()))))));
     }
 
     @Override public Expression implementResult(AggContext info,
@@ -1854,12 +1852,16 @@ public class RexImpTable {
 
     @Override public void implementAdd(AggContext info,
         AggAddContext add) {
+      final SqlJsonArrayAggAggFunction function =
+          (SqlJsonArrayAggAggFunction) info.aggregation();
       add.currentBlock().add(
           Expressions.statement(
               Expressions.call(m,
                   Iterables.concat(
                       Collections.singletonList(add.accumulator().get(0)),
-                      add.arguments()))));
+                      add.arguments(),
+                      Collections.singletonList(
+                          Expressions.constant(function.getNullClause()))))));
     }
 
     @Override public Expression implementResult(AggContext info,

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
index 59805fe..08a7aa8 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
@@ -213,6 +213,8 @@ public class JavaTypeFactoryImpl
         return ByteString.class;
       case GEOMETRY:
         return GeoFunctions.Geom.class;
+      case SYMBOL:
+        return Enum.class;
       case ANY:
         return Object.class;
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java b/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
index fb32cbc..d612ea4 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
@@ -254,7 +254,7 @@ public class AggregateCall {
   }
 
   public String toString() {
-    StringBuilder buf = new StringBuilder(aggFunction.getName());
+    StringBuilder buf = new StringBuilder(aggFunction.toString());
     buf.append("(");
     if (distinct) {
       buf.append((argList.size() == 0) ? "DISTINCT" : "DISTINCT ");

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index af008fb..7644d0a 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -2409,7 +2409,7 @@ public class SqlFunctions {
       if (!matcher.matches()) {
         throw RESOURCE.illegalJsonPathSpec(pathSpec).ex();
       }
-      PathMode mode = PathMode.valueOf(matcher.group(1).toUpperCase(Locale.ENGLISH));
+      PathMode mode = PathMode.valueOf(matcher.group(1).toUpperCase(Locale.ROOT));
       String pathWff = matcher.group(2);
       DocumentContext ctx;
       switch (mode) {
@@ -2636,14 +2636,6 @@ public class SqlFunctions {
     }
   }
 
-  public static void jsonObjectAggAddNullOnNull(Map map, String k, Object v) {
-    jsonObjectAggAdd(map, k, v, SqlJsonConstructorNullClause.NULL_ON_NULL);
-  }
-
-  public static void jsonObjectAggAddAbsentOnNull(Map map, String k, Object v) {
-    jsonObjectAggAdd(map, k, v, SqlJsonConstructorNullClause.ABSENT_ON_NULL);
-  }
-
   public static String jsonArray(SqlJsonConstructorNullClause nullClause,
       Object... elements) {
     List<Object> list = new ArrayList<>();
@@ -2670,14 +2662,6 @@ public class SqlFunctions {
     }
   }
 
-  public static void jsonArrayAggAddNullOnNull(List list, Object element) {
-    jsonArrayAggAdd(list, element, SqlJsonConstructorNullClause.NULL_ON_NULL);
-  }
-
-  public static void jsonArrayAggAddAbsentOnNull(List list, Object element) {
-    jsonArrayAggAdd(list, element, SqlJsonConstructorNullClause.ABSENT_ON_NULL);
-  }
-
   public static boolean isJsonValue(String input) {
     try {
       dejsonize(input);

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java b/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java
index a4a94b8..ef660fa 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java
@@ -20,7 +20,14 @@ package org.apache.calcite.sql;
  * Indicating that how do Json constructors handle null
  */
 public enum SqlJsonConstructorNullClause {
-  NULL_ON_NULL, ABSENT_ON_NULL
+  NULL_ON_NULL("NULL ON NULL"),
+  ABSENT_ON_NULL("ABSENT ON NULL");
+
+  public final String sql;
+
+  SqlJsonConstructorNullClause(String sql) {
+    this.sql = sql;
+  }
 }
 
 // End SqlJsonConstructorNullClause.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/SqlJsonEmptyOrError.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJsonEmptyOrError.java b/core/src/main/java/org/apache/calcite/sql/SqlJsonEmptyOrError.java
index 82491cd..07d847a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJsonEmptyOrError.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJsonEmptyOrError.java
@@ -23,10 +23,11 @@ import java.util.Locale;
  * EmptyOrErrorBehavior is invoked.
  */
 public enum SqlJsonEmptyOrError {
-  EMPTY, ERROR;
+  EMPTY,
+  ERROR;
 
   @Override public String toString() {
-    return String.format(Locale.ENGLISH, "SqlJsonEmptyOrError[%s]", name());
+    return String.format(Locale.ROOT, "SqlJsonEmptyOrError[%s]", name());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/SqlJsonExistsErrorBehavior.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJsonExistsErrorBehavior.java b/core/src/main/java/org/apache/calcite/sql/SqlJsonExistsErrorBehavior.java
index 4da8a4f..1f9e9d8 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJsonExistsErrorBehavior.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJsonExistsErrorBehavior.java
@@ -20,7 +20,10 @@ package org.apache.calcite.sql;
  * Categorizing Json exists error behaviors.
  */
 public enum SqlJsonExistsErrorBehavior {
-  TRUE, FALSE, UNKNOWN, ERROR
+  TRUE,
+  FALSE,
+  UNKNOWN,
+  ERROR
 }
 
 // End SqlJsonExistsErrorBehavior.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/SqlJsonQueryEmptyOrErrorBehavior.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJsonQueryEmptyOrErrorBehavior.java b/core/src/main/java/org/apache/calcite/sql/SqlJsonQueryEmptyOrErrorBehavior.java
index 2dd2e34..1893552 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJsonQueryEmptyOrErrorBehavior.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJsonQueryEmptyOrErrorBehavior.java
@@ -28,7 +28,7 @@ public enum SqlJsonQueryEmptyOrErrorBehavior {
   EMPTY_OBJECT;
 
   @Override public String toString() {
-    return String.format(Locale.ENGLISH,
+    return String.format(Locale.ROOT,
         "SqlJsonQueryEmptyOrErrorBehavior[%s]", name());
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/SqlJsonValueEmptyOrErrorBehavior.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJsonValueEmptyOrErrorBehavior.java b/core/src/main/java/org/apache/calcite/sql/SqlJsonValueEmptyOrErrorBehavior.java
index cb39279..0650fb7 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJsonValueEmptyOrErrorBehavior.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJsonValueEmptyOrErrorBehavior.java
@@ -27,7 +27,7 @@ public enum SqlJsonValueEmptyOrErrorBehavior {
   DEFAULT;
 
   @Override public String toString() {
-    return String.format(Locale.ENGLISH,
+    return String.format(Locale.ROOT,
         "SqlJsonValueEmptyOrErrorBehavior[%s]", name());
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
index 644a39b..89de617 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
@@ -180,8 +180,8 @@ public class MssqlSqlDialect extends SqlDialect {
 
   private void unparseSqlIntervalLiteralMssql(
       SqlWriter writer, SqlIntervalLiteral literal, int sign) {
-    SqlIntervalLiteral.IntervalValue interval
-        = (SqlIntervalLiteral.IntervalValue) literal.getValue();
+    final SqlIntervalLiteral.IntervalValue interval =
+        (SqlIntervalLiteral.IntervalValue) literal.getValue();
     unparseSqlIntervalQualifier(writer, interval.getIntervalQualifier(),
         RelDataTypeSystem.DEFAULT);
     writer.sep(",", true);

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayAggAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayAggAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayAggAggFunction.java
index 4791e67..a5d1520 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayAggAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayAggAggFunction.java
@@ -16,20 +16,27 @@
  */
 package org.apache.calcite.sql.fun;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlJsonConstructorNullClause;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.Optionality;
 
+import java.util.Locale;
+import java.util.Objects;
+
 /**
- * The <code>JSON_OBJECTAGG</code> aggregation function.
+ * The <code>JSON_OBJECTAGG</code> aggregate function.
  */
 public class SqlJsonArrayAggAggFunction extends SqlAggFunction {
   private final SqlJsonConstructorNullClause nullClause;
@@ -37,9 +44,9 @@ public class SqlJsonArrayAggAggFunction extends SqlAggFunction {
   public SqlJsonArrayAggAggFunction(String name,
       SqlJsonConstructorNullClause nullClause) {
     super(name, null, SqlKind.JSON_ARRAYAGG, ReturnTypes.VARCHAR_2000, null,
-        OperandTypes.ANY, SqlFunctionCategory.SYSTEM, false, false,
-        Optionality.FORBIDDEN);
-    this.nullClause = nullClause;
+        OperandTypes.family(SqlTypeFamily.ANY), SqlFunctionCategory.SYSTEM,
+        false, false, Optionality.FORBIDDEN);
+    this.nullClause = Objects.requireNonNull(nullClause);
   }
 
   @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec,
@@ -47,21 +54,31 @@ public class SqlJsonArrayAggAggFunction extends SqlAggFunction {
     assert call.operandCount() == 1;
     final SqlWriter.Frame frame = writer.startFunCall("JSON_ARRAYAGG");
     call.operand(0).unparse(writer, leftPrec, rightPrec);
-    switch (nullClause) {
-    case ABSENT_ON_NULL:
-      writer.keyword("ABSENT ON NULL");
-      break;
-    case NULL_ON_NULL:
-      writer.keyword("NULL ON NULL");
-      break;
-    default:
-      throw new IllegalStateException("unreachable code");
-    }
+    writer.keyword(nullClause.sql);
     writer.endFunCall(frame);
   }
 
-  private <E extends Enum<E>> E getEnumValue(SqlNode operand) {
-    return (E) ((SqlLiteral) operand).getValue();
+  @Override public RelDataType deriveType(SqlValidator validator,
+      SqlValidatorScope scope, SqlCall call) {
+    // To prevent operator rewriting by SqlFunction#deriveType.
+    for (SqlNode operand : call.getOperandList()) {
+      RelDataType nodeType = validator.deriveType(scope, operand);
+      ((SqlValidatorImpl) validator).setValidatedNodeType(operand, nodeType);
+    }
+    return validateOperands(validator, scope, call);
+  }
+
+  @Override public String toString() {
+    return getName() + String.format(Locale.ROOT, "<%s>", nullClause);
+  }
+
+  public SqlJsonArrayAggAggFunction with(SqlJsonConstructorNullClause nullClause) {
+    return this.nullClause == nullClause ? this
+        : new SqlJsonArrayAggAggFunction(getName(), nullClause);
+  }
+
+  public SqlJsonConstructorNullClause getNullClause() {
+    return nullClause;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayFunction.java
index 0306654..29ae0f8 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayFunction.java
@@ -67,7 +67,7 @@ public class SqlJsonArrayFunction extends SqlFunction {
     final StringBuilder sb = new StringBuilder();
     sb.append("{0}(");
     for (int i = 1; i < operandsCount; i++) {
-      sb.append(String.format(Locale.ENGLISH, "{%d} ", i + 1));
+      sb.append(String.format(Locale.ROOT, "{%d} ", i + 1));
     }
     sb.append("{1})");
     return sb.toString();

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectAggAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectAggAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectAggAggFunction.java
index 4b50c5c..d8b85c7 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectAggAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectAggAggFunction.java
@@ -16,31 +16,38 @@
  */
 package org.apache.calcite.sql.fun;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlJsonConstructorNullClause;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.Optionality;
 
+import java.util.Locale;
+import java.util.Objects;
+
 /**
- * The <code>JSON_OBJECTAGG</code> aggregation function.
+ * The <code>JSON_OBJECTAGG</code> aggregate function.
  */
 public class SqlJsonObjectAggAggFunction extends SqlAggFunction {
   private final SqlJsonConstructorNullClause nullClause;
 
+  /** Creates a SqlJsonObjectAggAggFunction. */
   public SqlJsonObjectAggAggFunction(String name,
       SqlJsonConstructorNullClause nullClause) {
     super(name, null, SqlKind.JSON_OBJECTAGG, ReturnTypes.VARCHAR_2000, null,
         OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.ANY),
         SqlFunctionCategory.SYSTEM, false, false, Optionality.FORBIDDEN);
-    this.nullClause = nullClause;
+    this.nullClause = Objects.requireNonNull(nullClause);
   }
 
   @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec,
@@ -51,21 +58,31 @@ public class SqlJsonObjectAggAggFunction extends SqlAggFunction {
     call.operand(0).unparse(writer, leftPrec, rightPrec);
     writer.keyword("VALUE");
     call.operand(1).unparse(writer, leftPrec, rightPrec);
-    switch (nullClause) {
-    case ABSENT_ON_NULL:
-      writer.keyword("ABSENT ON NULL");
-      break;
-    case NULL_ON_NULL:
-      writer.keyword("NULL ON NULL");
-      break;
-    default:
-      throw new IllegalStateException("unreachable code");
-    }
+    writer.keyword(nullClause.sql);
     writer.endFunCall(frame);
   }
 
-  private <E extends Enum<E>> E getEnumValue(SqlNode operand) {
-    return (E) ((SqlLiteral) operand).getValue();
+  @Override public RelDataType deriveType(SqlValidator validator,
+      SqlValidatorScope scope, SqlCall call) {
+    // To prevent operator rewriting by SqlFunction#deriveType.
+    for (SqlNode operand : call.getOperandList()) {
+      RelDataType nodeType = validator.deriveType(scope, operand);
+      ((SqlValidatorImpl) validator).setValidatedNodeType(operand, nodeType);
+    }
+    return validateOperands(validator, scope, call);
+  }
+
+  @Override public String toString() {
+    return getName() + String.format(Locale.ROOT, "<%s>", nullClause);
+  }
+
+  public SqlJsonObjectAggAggFunction with(SqlJsonConstructorNullClause nullClause) {
+    return this.nullClause == nullClause ? this
+        : new SqlJsonObjectAggAggFunction(getName(), nullClause);
+  }
+
+  public SqlJsonConstructorNullClause getNullClause() {
+    return nullClause;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectFunction.java
index 771c823..0240532 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectFunction.java
@@ -93,7 +93,7 @@ public class SqlJsonObjectFunction extends SqlFunction {
     StringBuilder sb = new StringBuilder();
     sb.append("{0}(");
     for (int i = 1; i < operandsCount; i++) {
-      sb.append(String.format(Locale.ENGLISH, "{%d} ", i + 1));
+      sb.append(String.format(Locale.ROOT, "{%d} ", i + 1));
     }
     sb.append("{1})");
     return sb.toString();

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/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 03d4336..6c3c596 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
@@ -1291,24 +1291,16 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
 
   public static final SqlFunction JSON_OBJECT = new SqlJsonObjectFunction();
 
-  public static final SqlAggFunction JSON_OBJECTAGG_NULL_ON_NULL =
-      new SqlJsonObjectAggAggFunction("JSON_OBJECTAGG_NULL_ON_NULL",
+  public static final SqlJsonObjectAggAggFunction JSON_OBJECTAGG =
+      new SqlJsonObjectAggAggFunction("JSON_OBJECTAGG",
           SqlJsonConstructorNullClause.NULL_ON_NULL);
 
-  public static final SqlAggFunction JSON_OBJECTAGG_ABSENT_ON_NULL =
-      new SqlJsonObjectAggAggFunction("JSON_OBJECTAGG_ABSENT_ON_NULL",
-          SqlJsonConstructorNullClause.ABSENT_ON_NULL);
-
   public static final SqlFunction JSON_ARRAY = new SqlJsonArrayFunction();
 
-  public static final SqlAggFunction JSON_ARRAYAGG_NULL_ON_NULL =
-      new SqlJsonArrayAggAggFunction("JSON_ARRAYAGG_NULL_ON_NULL",
+  public static final SqlJsonArrayAggAggFunction JSON_ARRAYAGG =
+      new SqlJsonArrayAggAggFunction("JSON_ARRAYAGG",
           SqlJsonConstructorNullClause.NULL_ON_NULL);
 
-  public static final SqlAggFunction JSON_ARRAYAGG_ABSENT_ON_NULL =
-      new SqlJsonArrayAggAggFunction("JSON_ARRAYAGG_ABSENT_ON_NULL",
-          SqlJsonConstructorNullClause.ABSENT_ON_NULL);
-
   public static final SqlBetweenOperator BETWEEN =
       new SqlBetweenOperator(
           SqlBetweenOperator.Flag.ASYMMETRIC,

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 8160ead..f23a931 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -272,27 +272,20 @@ public enum BuiltInMethod {
       Object.class, String.class),
   JSON_EXISTS(SqlFunctions.class, "jsonExists", Object.class),
   JSON_VALUE_ANY(SqlFunctions.class, "jsonValueAny", Object.class,
-      SqlJsonValueEmptyOrErrorBehavior.class,
-      Object.class,
-      SqlJsonValueEmptyOrErrorBehavior.class,
-      Object.class),
-  JSON_QUERY(SqlFunctions.class, "jsonQuery",
-      Object.class,
+      SqlJsonValueEmptyOrErrorBehavior.class, Object.class,
+      SqlJsonValueEmptyOrErrorBehavior.class, Object.class),
+  JSON_QUERY(SqlFunctions.class, "jsonQuery", Object.class,
       SqlJsonQueryWrapperBehavior.class,
       SqlJsonQueryEmptyOrErrorBehavior.class,
       SqlJsonQueryEmptyOrErrorBehavior.class),
   JSON_OBJECT(SqlFunctions.class, "jsonObject",
       SqlJsonConstructorNullClause.class),
-  JSON_OBJECTAGG_ADD_NULL_ON_NULL(SqlFunctions.class,
-      "jsonObjectAggAddNullOnNull", Map.class, String.class, Object.class),
-  JSON_OBJECTAGG_ADD_ABSENT_ON_NULL(SqlFunctions.class,
-      "jsonObjectAggAddAbsentOnNull", Map.class, String.class, Object.class),
+  JSON_OBJECTAGG_ADD(SqlFunctions.class, "jsonObjectAggAdd", Map.class,
+      String.class, Object.class, SqlJsonConstructorNullClause.class),
   JSON_ARRAY(SqlFunctions.class, "jsonArray",
       SqlJsonConstructorNullClause.class),
-  JSON_ARRAYAGG_ADD_NULL_ON_NULL(SqlFunctions.class,
-      "jsonArrayAggAddNullOnNull", List.class, Object.class),
-  JSON_ARRAYAGG_ADD_ABSENT_ON_NULL(SqlFunctions.class,
-      "jsonArrayAggAddAbsentOnNull", List.class, Object.class),
+  JSON_ARRAYAGG_ADD(SqlFunctions.class, "jsonArrayAggAdd",
+      List.class, Object.class, SqlJsonConstructorNullClause.class),
   IS_JSON_VALUE(SqlFunctions.class, "isJsonValue", String.class),
   IS_JSON_OBJECT(SqlFunctions.class, "isJsonObject", String.class),
   IS_JSON_ARRAY(SqlFunctions.class, "isJsonArray", String.class),

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 1207020..f13894c 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -2882,6 +2882,64 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).with(tester).ok();
   }
 
+  @Test public void testJsonExists() {
+    final String sql = "select json_exists(ename, 'lax $')\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonValue() {
+    final String sql = "select json_value(ename, 'lax $')\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonQuery() {
+    final String sql = "select json_query(ename, 'lax $')\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonArray() {
+    final String sql = "select json_array(ename, ename)\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonArrayAgg() {
+    final String sql = "select json_arrayagg(ename)\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonObject() {
+    final String sql = "select json_object(ename: deptno, ename: deptno)\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonObjectAgg() {
+    final String sql = "select json_objectagg(ename: deptno)\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonPredicate() {
+    final String sql = "select\n"
+        + "ename is json,\n"
+        + "ename is json value,\n"
+        + "ename is json object,\n"
+        + "ename is json array,\n"
+        + "ename is json scalar,\n"
+        + "ename is not json,\n"
+        + "ename is not json value,\n"
+        + "ename is not json object,\n"
+        + "ename is not json array,\n"
+        + "ename is not json scalar\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
   @Test public void testWithinGroup1() {
     final String sql = "select deptno,\n"
         + " collect(empno) within group (order by deptno, hiredate desc)\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 28d2e3f..d8539c7 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -5316,6 +5316,114 @@ LogicalProject(ANYEMPNO=[$1])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testJsonExists">
+        <Resource name="sql">
+            <![CDATA[select json_exists(ename, 'lax $')
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[JSON_EXISTS(JSON_API_COMMON_SYNTAX(JSON_VALUE_EXPRESSION($1), 'lax $'))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonValue">
+        <Resource name="sql">
+            <![CDATA[select json_value(ename, 'lax $')
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[CAST(JSON_VALUE_ANY(JSON_API_COMMON_SYNTAX(JSON_VALUE_EXPRESSION($1), 'lax $'), FLAG(SqlJsonValueEmptyOrErrorBehavior[NULL]), null, FLAG(SqlJsonValueEmptyOrErrorBehavior[NULL]), null)):VARCHAR(2000) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonQuery">
+        <Resource name="sql">
+            <![CDATA[select json_query(ename, 'lax $')
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[JSON_QUERY(JSON_API_COMMON_SYNTAX(JSON_VALUE_EXPRESSION($1), 'lax $'), FLAG(WITHOUT_ARRAY), FLAG(SqlJsonQueryEmptyOrErrorBehavior[NULL]), FLAG(SqlJsonQueryEmptyOrErrorBehavior[NULL]))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonArray">
+        <Resource name="sql">
+            <![CDATA[select json_array(ename, ename)
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[JSON_ARRAY(FLAG(ABSENT_ON_NULL), JSON_STRUCTURED_VALUE_EXPRESSION($1), JSON_STRUCTURED_VALUE_EXPRESSION($1))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonArrayAgg">
+        <Resource name="sql">
+            <![CDATA[select json_arrayagg(ename)
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[JSON_ARRAYAGG<ABSENT_ON_NULL>($0)])
+  LogicalProject($f0=[JSON_STRUCTURED_VALUE_EXPRESSION($1)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonObject">
+        <Resource name="sql">
+            <![CDATA[select select json_object(ename: deptno, ename: deptno)
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[JSON_OBJECT(FLAG(NULL_ON_NULL), $1, JSON_STRUCTURED_VALUE_EXPRESSION($7), $1, JSON_STRUCTURED_VALUE_EXPRESSION($7))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonObjectAgg">
+        <Resource name="sql">
+            <![CDATA[select json_objectagg(ename: deptno)
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[JSON_OBJECTAGG<NULL_ON_NULL>($0, $1)])
+  LogicalProject(ENAME=[$1], $f1=[JSON_STRUCTURED_VALUE_EXPRESSION($7)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonPredicate">
+        <Resource name="sql">
+            <![CDATA[select
+ename is json,
+ename is json value,
+ename is json object,
+ename is json array,
+ename is json scalar,
+ename is not json,
+ename is not json value,
+ename is not json object,
+ename is not json array,
+ename is not json scalar
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[IS JSON VALUE($1)], EXPR$1=[IS JSON VALUE($1)], EXPR$2=[IS JSON OBJECT($1)], EXPR$3=[IS JSON ARRAY($1)], EXPR$4=[IS JSON SCALAR($1)], EXPR$5=[IS NOT JSON VALUE($1)], EXPR$6=[IS NOT JSON VALUE($1)], EXPR$7=[IS NOT JSON OBJECT($1)], EXPR$8=[IS NOT JSON ARRAY($1)], EXPR$9=[IS NOT JSON SCALAR($1)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testWithinGroup1">
         <Resource name="sql">
             <![CDATA[select deptno,

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/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 0862a54..8b771ff 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -2213,4 +2213,10 @@ X
 true
 !ok
 
+# JSON
+values json_exists('{"foo":"bar"}', 'strict $.foo' false on error);
+EXPR$0
+true
+!ok
+
 # End misc.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
----------------------------------------------------------------------
diff --git a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
index 703e813..5789d41 100644
--- a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
+++ b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
@@ -512,8 +512,8 @@ public class CsvTest {
     Properties info = new Properties();
     info.put("model", jsonPath("bug"));
 
-    try (Connection connection
-        = DriverManager.getConnection("jdbc:calcite:", info)) {
+    try (Connection connection =
+        DriverManager.getConnection("jdbc:calcite:", info)) {
       ResultSet res = connection.getMetaData().getColumns(null, null,
           "DATE", "JOINEDAT");
       res.next();
@@ -560,8 +560,8 @@ public class CsvTest {
     Properties info = new Properties();
     info.put("model", jsonPath("bug"));
 
-    try (Connection connection
-        = DriverManager.getConnection("jdbc:calcite:", info)) {
+    try (Connection connection =
+        DriverManager.getConnection("jdbc:calcite:", info)) {
       Statement statement = connection.createStatement();
       final String sql = "select * from \"DATE\"\n"
           + "where EMPNO >= 140 and EMPNO < 200";