You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by mm...@apache.org on 2018/11/09 16:40:49 UTC

[2/3] calcite git commit: [CALCITE-2266] Implement SQL:2016 JSON functions: JSON_EXISTS, JSON_VALUE, JSON_QUERY, JSON_OBJECT, JSON_OBJECTAGG, JSON_ARRAY, JSON_ARRAYAGG, IS JSON predicate (Hongze Zhang)

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/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
new file mode 100644
index 0000000..919abe9
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectAggAggFunction.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.sql.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;
+
+/**
+ * The <code>JSON_OBJECTAGG</code> aggregation function.
+ */
+public class SqlJsonObjectAggAggFunction extends SqlAggFunction {
+  private final String name;
+  private final SqlJsonConstructorNullClause nullClause;
+
+  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
+    );
+    this.name = name;
+    this.nullClause = nullClause;
+  }
+
+  @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
+    assert call.operandCount() == 2;
+    final SqlWriter.Frame frame = writer.startFunCall("JSON_OBJECTAGG");
+    writer.keyword("KEY");
+    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.endFunCall(frame);
+  }
+
+  private <E extends Enum<E>> E getEnumValue(SqlNode operand) {
+    return (E) ((SqlLiteral) operand).getValue();
+  }
+}
+
+// End SqlJsonObjectAggAggFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/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
new file mode 100644
index 0000000..8ca8eed
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectFunction.java
@@ -0,0 +1,137 @@
+/*
+ * 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.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlFunction;
+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.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlValidator;
+
+import java.util.Locale;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * The <code>JSON_OBJECT</code> function.
+ */
+public class SqlJsonObjectFunction extends SqlFunction {
+  public SqlJsonObjectFunction() {
+    super(
+        "JSON_OBJECT",
+        SqlKind.OTHER_FUNCTION,
+        ReturnTypes.VARCHAR_2000,
+        null,
+        null,
+        SqlFunctionCategory.SYSTEM
+    );
+  }
+
+  @Override public SqlOperandCountRange getOperandCountRange() {
+    return SqlOperandCountRanges.from(1);
+  }
+
+  @Override protected void checkOperandCount(SqlValidator validator, SqlOperandTypeChecker argType,
+                                             SqlCall call) {
+    assert call.operandCount() % 2 == 1;
+  }
+
+  @Override public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
+    int count = callBinding.getOperandCount();
+    for (int i = 1; i < count; i += 2) {
+      RelDataType nameType = callBinding.getOperandType(i);
+      if (!SqlTypeUtil.inCharFamily(nameType)) {
+        if (throwOnFailure) {
+          throw callBinding.newError(RESOURCE.expectedCharacter());
+        }
+        return false;
+      }
+      if (nameType.isNullable()) {
+        if (throwOnFailure) {
+          throw callBinding.newError(
+              RESOURCE.argumentMustNotBeNull(callBinding.operand(i).toString()));
+        }
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos,
+                                      SqlNode... operands) {
+    if (operands[0] == null) {
+      operands[0] = SqlLiteral.createSymbol(SqlJsonConstructorNullClause.NULL_ON_NULL, pos);
+    }
+    return super.createCall(functionQualifier, pos, operands);
+  }
+
+  @Override public String getSignatureTemplate(int operandsCount) {
+    assert operandsCount % 2 == 1;
+    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("{1})");
+    return sb.toString();
+  }
+
+  @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
+    assert call.operandCount() % 2 == 1;
+    final SqlWriter.Frame frame = writer.startFunCall(getName());
+    SqlWriter.Frame listFrame = writer.startList("", "");
+    for (int i = 1; i < call.operandCount(); i += 2) {
+      writer.sep(",");
+      writer.keyword("KEY");
+      call.operand(i).unparse(writer, leftPrec, rightPrec);
+      writer.keyword("VALUE");
+      call.operand(i + 1).unparse(writer, leftPrec, rightPrec);
+    }
+    writer.endList(listFrame);
+
+    SqlJsonConstructorNullClause nullClause = getEnumValue(call.operand(0));
+    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.endFunCall(frame);
+  }
+
+  private <E extends Enum<E>> E getEnumValue(SqlNode operand) {
+    return (E) ((SqlLiteral) operand).getValue();
+  }
+}
+
+// End SqlJsonObjectFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java
new file mode 100644
index 0000000..f650fbb
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlJsonQueryEmptyOrErrorBehavior;
+import org.apache.calcite.sql.SqlJsonQueryWrapperBehavior;
+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.parser.SqlParserPos;
+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.type.SqlTypeTransforms;
+
+/**
+ * The <code>JSON_QUERY</code> function.
+ */
+public class SqlJsonQueryFunction extends SqlFunction {
+  public SqlJsonQueryFunction() {
+    super(
+        "JSON_QUERY",
+        SqlKind.OTHER_FUNCTION,
+        ReturnTypes.cascade(ReturnTypes.VARCHAR_2000, SqlTypeTransforms.FORCE_NULLABLE),
+        null,
+        OperandTypes.family(SqlTypeFamily.ANY,
+            SqlTypeFamily.ANY, SqlTypeFamily.ANY, SqlTypeFamily.ANY),
+        SqlFunctionCategory.SYSTEM
+    );
+  }
+
+  @Override public String getSignatureTemplate(int operandsCount) {
+    return "{0}({1} {2} WRAPPER {3} ON EMPTY {4} ON ERROR)";
+  }
+
+  @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
+    final SqlWriter.Frame frame = writer.startFunCall(getName());
+    call.operand(0).unparse(writer, 0, 0);
+    SqlJsonQueryWrapperBehavior wrapperBehavior = getEnumValue(call.operand(1));
+    switch (wrapperBehavior) {
+    case WITHOUT_ARRAY:
+      writer.keyword("WITHOUT ARRAY");
+      break;
+    case WITH_CONDITIONAL_ARRAY:
+      writer.keyword("WITH CONDITIONAL ARRAY");
+      break;
+    case WITH_UNCONDITIONAL_ARRAY:
+      writer.keyword("WITH UNCONDITIONAL ARRAY");
+      break;
+    default:
+      throw new IllegalStateException("unreachable code");
+    }
+    writer.keyword("WRAPPER");
+    unparseEmptyOrErrorBehavior(writer, getEnumValue(call.operand(2)));
+    writer.keyword("ON EMPTY");
+    unparseEmptyOrErrorBehavior(writer, getEnumValue(call.operand(3)));
+    writer.keyword("ON ERROR");
+    writer.endFunCall(frame);
+  }
+
+  @Override public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos,
+                                      SqlNode... operands) {
+    if (operands[1] == null) {
+      operands[1] = SqlLiteral.createSymbol(SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY, pos);
+    }
+    if (operands[2] == null) {
+      operands[2] = SqlLiteral.createSymbol(SqlJsonQueryEmptyOrErrorBehavior.NULL, pos);
+    }
+    if (operands[3] == null) {
+      operands[3] = SqlLiteral.createSymbol(SqlJsonQueryEmptyOrErrorBehavior.NULL, pos);
+    }
+    return super.createCall(functionQualifier, pos, operands);
+  }
+
+
+  private void unparseEmptyOrErrorBehavior(SqlWriter writer,
+                                           SqlJsonQueryEmptyOrErrorBehavior emptyBehavior) {
+    switch (emptyBehavior) {
+    case NULL:
+      writer.keyword("NULL");
+      break;
+    case ERROR:
+      writer.keyword("ERROR");
+      break;
+    case EMPTY_ARRAY:
+      writer.keyword("EMPTY ARRAY");
+      break;
+    case EMPTY_OBJECT:
+      writer.keyword("EMPTY OBJECT");
+      break;
+    default:
+      throw new IllegalStateException("unreachable code");
+    }
+  }
+
+  private <E extends Enum<E>> E getEnumValue(SqlNode operand) {
+    return (E) ((SqlLiteral) operand).getValue();
+  }
+}
+
+// End SqlJsonQueryFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueExpressionOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueExpressionOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueExpressionOperator.java
new file mode 100644
index 0000000..ac6b53c
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueExpressionOperator.java
@@ -0,0 +1,75 @@
+/*
+ * 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.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * The JSON value expression operator which indicate that the value expression
+ * should be parsed as JSON.
+ */
+public class SqlJsonValueExpressionOperator extends SqlSpecialOperator {
+  private final boolean structured;
+
+  public SqlJsonValueExpressionOperator(String name, boolean structured) {
+    super(
+        name,
+        SqlKind.JSON_VALUE_EXPRESSION,
+        100,
+        true,
+        new SqlReturnTypeInference() {
+          @Override public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+            return opBinding.getTypeFactory().createTypeWithNullability(
+                opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY),
+                true);
+          }
+        },
+        new SqlOperandTypeInference() {
+          @Override public void inferOperandTypes(SqlCallBinding callBinding,
+                                                  RelDataType returnType,
+                                                  RelDataType[] operandTypes) {
+            if (callBinding.isOperandNull(0, false)) {
+              operandTypes[0] = callBinding.getTypeFactory().createTypeWithNullability(
+                  callBinding.getTypeFactory().createSqlType(SqlTypeName.ANY),
+                  true);
+            }
+          }
+        },
+        structured ? OperandTypes.ANY : OperandTypes.STRING
+    );
+    this.structured = structured;
+  }
+
+  @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
+    call.operand(0).unparse(writer, 0, 0);
+    if (!structured) {
+      writer.keyword("FORMAT JSON");
+    }
+  }
+}
+
+// End SqlJsonValueExpressionOperator.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueFunction.java
new file mode 100644
index 0000000..2b16500
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueFunction.java
@@ -0,0 +1,196 @@
+/*
+ * 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.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlJsonValueEmptyOrErrorBehavior;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * The <code>JSON_VALUE</code> function.
+ */
+public class SqlJsonValueFunction extends SqlFunction {
+
+  private final boolean returnAny;
+
+  public SqlJsonValueFunction(String name, boolean returnAny) {
+    super(
+        name,
+        SqlKind.OTHER_FUNCTION,
+        null,
+        new SqlOperandTypeInference() {
+          @Override public void inferOperandTypes(SqlCallBinding callBinding,
+                                                  RelDataType returnType,
+                                                  RelDataType[] operandTypes) {
+            RelDataTypeFactory typeFactory = callBinding.getTypeFactory();
+            for (int i = 0; i < operandTypes.length; ++i) {
+              operandTypes[i] =
+                  typeFactory.createSqlType(SqlTypeName.ANY);
+            }
+          }
+        },
+        null,
+        SqlFunctionCategory.SYSTEM
+    );
+    this.returnAny = returnAny;
+  }
+
+  @Override public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos,
+                                      SqlNode... operands) {
+    List<SqlNode> operandList = new ArrayList<>();
+    operandList.add(operands[0]);
+    if (operands[1] == null) {
+      operandList.add(SqlLiteral.createSymbol(SqlJsonValueEmptyOrErrorBehavior.NULL, pos));
+      operandList.add(SqlLiteral.createNull(pos));
+    } else {
+      operandList.add(operands[1]);
+      operandList.add(operands[2]);
+    }
+    if (operands[3] == null) {
+      operandList.add(SqlLiteral.createSymbol(SqlJsonValueEmptyOrErrorBehavior.NULL, pos));
+      operandList.add(SqlLiteral.createNull(pos));
+    } else {
+      operandList.add(operands[3]);
+      operandList.add(operands[4]);
+    }
+    if (operands.length == 6 && operands[5] != null) {
+      if (returnAny) {
+        throw new IllegalArgumentException("illegal returning clause in json_value_any function");
+      }
+      operandList.add(operands[5]);
+    } else if (!returnAny) {
+      SqlDataTypeSpec defaultTypeSpec = new SqlDataTypeSpec(
+          new SqlIdentifier("VARCHAR", pos), 2000, -1,
+          null, null, pos);
+      operandList.add(
+          defaultTypeSpec);
+    }
+    return super.createCall(functionQualifier, pos,
+        operandList.toArray(SqlNode.EMPTY_ARRAY));
+  }
+
+  @Override public SqlOperandCountRange getOperandCountRange() {
+    return SqlOperandCountRanges.between(5, 6);
+  }
+
+  @Override public boolean checkOperandTypes(SqlCallBinding callBinding,
+                                             boolean throwOnFailure) {
+    RelDataType defaultValueOnEmptyType =
+        callBinding.getValidator().getValidatedNodeType(callBinding.operand(2));
+    RelDataType defaultValueOnErrorType =
+        callBinding.getValidator().getValidatedNodeType(callBinding.operand(4));
+    RelDataType returnType = callBinding.getValidator().deriveType(
+        callBinding.getScope(), callBinding.operand(5));
+    if (!canCastFrom(callBinding, throwOnFailure, defaultValueOnEmptyType, returnType)) {
+      return false;
+    }
+    if (!canCastFrom(callBinding, throwOnFailure, defaultValueOnErrorType, returnType)) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+    assert opBinding.getOperandCount() == 5 || opBinding.getOperandCount() == 6;
+    RelDataType ret;
+    if (opBinding.getOperandCount() == 6) {
+      ret = opBinding.getOperandType(5);
+    } else {
+      ret = opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY);
+    }
+    return opBinding.getTypeFactory().createTypeWithNullability(
+        ret,
+        true);
+  }
+
+  @Override public String getSignatureTemplate(int operandsCount) {
+    assert operandsCount == 5 || operandsCount == 6;
+    if (operandsCount == 6) {
+      return "{0}({1} RETURNING {6} {2} {3} ON EMPTY {4} {5} ON ERROR)";
+    }
+    return "{0}({1} {2} {3} ON EMPTY {4} {5} ON ERROR)";
+  }
+
+  @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
+    assert call.operandCount() == 5 || call.operandCount() == 6;
+    final SqlWriter.Frame frame = writer.startFunCall(getName());
+    call.operand(0).unparse(writer, 0, 0);
+    if (!returnAny) {
+      writer.keyword("RETURNING");
+      call.operand(5).unparse(writer, 0, 0);
+    }
+    unparseEnum(writer, call.operand(1));
+    if (isDefaultLiteral(call.operand(1))) {
+      call.operand(2).unparse(writer, 0, 0);
+    }
+    writer.keyword("ON");
+    writer.keyword("EMPTY");
+    unparseEnum(writer, call.operand(3));
+    if (isDefaultLiteral(call.operand(3))) {
+      call.operand(4).unparse(writer, 0, 0);
+    }
+    writer.keyword("ON");
+    writer.keyword("ERROR");
+    writer.endFunCall(frame);
+  }
+
+  private void unparseEnum(SqlWriter writer, SqlLiteral literal) {
+    writer.keyword(((Enum) literal.getValue()).name());
+  }
+
+  private boolean isDefaultLiteral(SqlLiteral literal) {
+    return literal.getValueAs(SqlJsonValueEmptyOrErrorBehavior.class)
+        == SqlJsonValueEmptyOrErrorBehavior.DEFAULT;
+  }
+
+  private boolean canCastFrom(SqlCallBinding callBinding, boolean throwOnFailure,
+                              RelDataType inType, RelDataType outType) {
+    if (SqlTypeUtil.canCastFrom(outType, inType, true)) {
+      return true;
+    }
+    if (throwOnFailure) {
+      throw callBinding.newError(
+          RESOURCE.cannotCastValue(inType.toString(),
+              outType.toString()));
+    }
+    return false;
+  }
+}
+
+// End SqlJsonValueFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/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 ac09932..99fc91d 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
@@ -27,6 +27,7 @@ import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlGroupedWindowFunction;
 import org.apache.calcite.sql.SqlInternalOperator;
+import org.apache.calcite.sql.SqlJsonConstructorNullClause;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLateralOperator;
 import org.apache.calcite.sql.SqlLiteral;
@@ -732,6 +733,79 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
           null,
           OperandTypes.COLLECTION_OR_MAP);
 
+  public static final SqlPostfixOperator IS_JSON_VALUE =
+      new SqlPostfixOperator(
+          "IS JSON VALUE",
+          SqlKind.OTHER,
+          28,
+          ReturnTypes.BOOLEAN,
+          null,
+          OperandTypes.CHARACTER);
+
+  public static final SqlPostfixOperator IS_NOT_JSON_VALUE =
+      new SqlPostfixOperator(
+          "IS NOT JSON VALUE",
+          SqlKind.OTHER,
+          28,
+          ReturnTypes.BOOLEAN,
+          null,
+          OperandTypes.CHARACTER);
+
+  public static final SqlPostfixOperator IS_JSON_OBJECT =
+      new SqlPostfixOperator(
+          "IS JSON OBJECT",
+          SqlKind.OTHER,
+          28,
+          ReturnTypes.BOOLEAN,
+          null,
+          OperandTypes.CHARACTER);
+
+  public static final SqlPostfixOperator IS_NOT_JSON_OBJECT =
+      new SqlPostfixOperator(
+          "IS NOT JSON OBJECT",
+          SqlKind.OTHER,
+          28,
+          ReturnTypes.BOOLEAN,
+          null,
+          OperandTypes.CHARACTER);
+
+  public static final SqlPostfixOperator IS_JSON_ARRAY =
+      new SqlPostfixOperator(
+          "IS JSON ARRAY",
+          SqlKind.OTHER,
+          28,
+          ReturnTypes.BOOLEAN,
+          null,
+          OperandTypes.CHARACTER);
+
+  public static final SqlPostfixOperator IS_NOT_JSON_ARRAY =
+      new SqlPostfixOperator(
+          "IS NOT JSON ARRAY",
+          SqlKind.OTHER,
+          28,
+          ReturnTypes.BOOLEAN,
+          null,
+          OperandTypes.CHARACTER);
+
+  public static final SqlPostfixOperator IS_JSON_SCALAR =
+      new SqlPostfixOperator(
+          "IS JSON SCALAR",
+          SqlKind.OTHER,
+          28,
+          ReturnTypes.BOOLEAN,
+          null,
+          OperandTypes.CHARACTER);
+
+  public static final SqlPostfixOperator IS_NOT_JSON_SCALAR =
+      new SqlPostfixOperator(
+          "IS NOT JSON SCALAR",
+          SqlKind.OTHER,
+          28,
+          ReturnTypes.BOOLEAN,
+          null,
+          OperandTypes.CHARACTER);
+
+
   //-------------------------------------------------------------
   //                   PREFIX OPERATORS
   //-------------------------------------------------------------
@@ -1195,6 +1269,43 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
 
   public static final SqlThrowOperator THROW = new SqlThrowOperator();
 
+  public static final SqlJsonValueExpressionOperator JSON_VALUE_EXPRESSION =
+      new SqlJsonValueExpressionOperator("JSON_VALUE_EXPRESSION", false);
+
+  public static final SqlJsonValueExpressionOperator JSON_STRUCTURED_VALUE_EXPRESSION =
+      new SqlJsonValueExpressionOperator("JSON_STRUCTURED_VALUE_EXPRESSION", true);
+
+  public static final SqlJsonApiCommonSyntaxOperator JSON_API_COMMON_SYNTAX =
+      new SqlJsonApiCommonSyntaxOperator();
+
+  public static final SqlFunction JSON_EXISTS = new SqlJsonExistsFunction();
+
+  public static final SqlFunction JSON_VALUE = new SqlJsonValueFunction("JSON_VALUE", false);
+
+  public static final SqlFunction JSON_VALUE_ANY = new SqlJsonValueFunction("JSON_VALUE_ANY", true);
+
+  public static final SqlFunction JSON_QUERY = new SqlJsonQueryFunction();
+
+  public static final SqlFunction JSON_OBJECT = new SqlJsonObjectFunction();
+
+  public static final SqlAggFunction JSON_OBJECTAGG_NULL_ON_NULL
+      = new SqlJsonObjectAggAggFunction("JSON_OBJECTAGG_NULL_ON_NULL",
+      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",
+      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/8e557d26/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
index fc0022d..fe77119 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
@@ -203,6 +203,12 @@ public abstract class ReturnTypes {
       };
 
   /**
+   * Type-inference strategy whereby the result type of a call is a nullable Boolean.
+   */
+  public static final SqlReturnTypeInference BOOLEAN_FORCE_NULLABLE =
+      cascade(BOOLEAN, SqlTypeTransforms.FORCE_NULLABLE);
+
+  /**
    * Type-inference strategy whereby the result type of a call is Boolean
    * not null.
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/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 d51627b..e5d5557 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -209,6 +209,30 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
                 call.operand(0),
                 SqlLiteral.createExactNumeric("0.5", SqlParserPos.ZERO))));
 
+    // Convert json_value('{"foo":"bar"}', 'lax $.foo', returning varchar(2000))
+    // to cast(json_value('{"foo":"bar"}', 'lax $.foo') as varchar(2000))
+    registerOp(
+        SqlStdOperatorTable.JSON_VALUE,
+        new SqlRexConvertlet() {
+          @Override public RexNode convertCall(SqlRexContext cx, SqlCall call) {
+            SqlNode expanded = SqlStdOperatorTable.CAST.createCall(
+                SqlParserPos.ZERO,
+                SqlStdOperatorTable.JSON_VALUE_ANY.createCall(
+                    SqlParserPos.ZERO,
+                    call.operand(0),
+                    call.operand(1),
+                    call.operand(2),
+                    call.operand(3),
+                    call.operand(4),
+                    null
+                ),
+                call.operand(5)
+            );
+            return cx.convertExpression(expanded);
+          }
+        }
+    );
+
     // REVIEW jvs 24-Apr-2006: This only seems to be working from within a
     // windowed agg.  I have added an optimizer rule
     // org.apache.calcite.rel.rules.AggregateReduceFunctionsRule which handles

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/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 8462d95..79efaac 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -92,6 +92,10 @@ import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlJsonConstructorNullClause;
+import org.apache.calcite.sql.SqlJsonQueryEmptyOrErrorBehavior;
+import org.apache.calcite.sql.SqlJsonQueryWrapperBehavior;
+import org.apache.calcite.sql.SqlJsonValueEmptyOrErrorBehavior;
 
 import com.google.common.collect.ImmutableMap;
 
@@ -254,6 +258,36 @@ public enum BuiltInMethod {
   ANY_ITEM(SqlFunctions.class, "itemOptional", Object.class, Object.class),
   UPPER(SqlFunctions.class, "upper", String.class),
   LOWER(SqlFunctions.class, "lower", String.class),
+  JSONIZE(SqlFunctions.class, "jsonize", Object.class),
+  JSON_VALUE_EXPRESSION(SqlFunctions.class, "jsonValueExpression", String.class),
+  JSON_STRUCTURED_VALUE_EXPRESSION(SqlFunctions.class, "jsonStructuredValueExpression",
+      Object.class),
+  JSON_API_COMMON_SYNTAX(SqlFunctions.class, "jsonApiCommonSyntax", 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,
+      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_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),
+  IS_JSON_VALUE(SqlFunctions.class, "isJsonValue", String.class),
+  IS_JSON_OBJECT(SqlFunctions.class, "isJsonObject", String.class),
+  IS_JSON_ARRAY(SqlFunctions.class, "isJsonArray", String.class),
+  IS_JSON_SCALAR(SqlFunctions.class, "isJsonScalar", String.class),
   INITCAP(SqlFunctions.class, "initcap", String.class),
   SUBSTRING(SqlFunctions.class, "substring", String.class, int.class,
       int.class),

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index edb93b9..da48650 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -36,6 +36,7 @@ IllegalOrderBy=ORDER BY unexpected
 IllegalBinaryString=Illegal binary string {0}
 IllegalFromEmpty=''FROM'' without operands preceding it is illegal
 IllegalRowExpression=ROW expression encountered in illegal context
+IllegalColon=Illegal identifier '':''. Was expecting ''VALUE''
 InvalidSampleSize=TABLESAMPLE percentage must be between 0 and 100, inclusive
 UnknownCharacterSet=Unknown character set ''{0}''
 CharsetEncoding=Failed to encode ''{0}'' in character set ''{1}''
@@ -77,6 +78,7 @@ NeedQueryOp=Operand {0} must be a query
 NeedSameTypeParameter=Parameters must be of the same type
 CanNotApplyOp2Type=Cannot apply ''{0}'' to arguments of type {1}. Supported form(s): {2}
 ExpectedBoolean=Expected a boolean type
+ExpectedCharacter=Expected a character type
 MustNotNullInElse=ELSE clause or at least one THEN clause must be non-NULL
 FunctionUndefined=Function ''{0}'' is not defined
 WrongNumberOfParam=Encountered {0} with {1,number} parameter(s); was expecting {2}
@@ -250,4 +252,27 @@ SchemaNotFound=Schema ''{0}'' not found
 ViewNotFound=View ''{0}'' not found
 TypeNotFound=Type ''{0}'' not found
 DialectDoesNotSupportFeature=Dialect does not support feature: ''{0}''
+IllegalNegativeSubstringLength=Substring error: negative substring length not allowed
+TrimError=Trim error: trim character must be exactly 1 character
+InvalidTypesForArithmetic=Invalid types for arithmetic: {0} {1} {2}
+InvalidTypesForComparison=Invalid types for comparison: {0} {1} {2}
+CannotConvert=Cannot convert {0} to {1}
+InvalidCharacterForCast=Invalid character for cast: {0}
+MoreThanOneValueInList=More than one value in list: {0}
+FailedToAccessField=Failed to access field ''{0}'' of object of type {1}
+IllegalJsonPathSpec=Illegal jsonpath spec ''{0}'', format of the spec should be: ''<lax|strict> $'{'expr'}'''
+IllegalJsonPathMode=Illegal jsonpath mode ''{0}''
+IllegalJsonPathModeInPathSpec=Illegal jsonpath mode ''{0}'' in jsonpath spec: ''{1}''
+StrictPathModeRequiresNonEmptyValue=Strict jsonpath mode requires a non empty returned value, but is null
+IllegalErrorBehaviorInJsonExistsFunc=Illegal error behavior ''{0}'' specified in JSON_EXISTS function
+EmptyResultOfJsonValueFuncNotAllowed=Empty result of JSON_VALUE function is not allowed
+IllegalEmptyBehaviorInJsonValueFunc=Illegal empty behavior ''{0}'' specified in JSON_VALUE function
+IllegalErrorBehaviorInJsonValueFunc=Illegal error behavior ''{0}'' specified in JSON_VALUE function
+ScalarValueRequiredInStrictModeOfJsonValueFunc=Strict jsonpath mode requires scalar value, and the actual value is: ''{0}''
+IllegalWrapperBehaviorInJsonQueryFunc=Illegal wrapper behavior ''{0}'' specified in JSON_QUERY function
+EmptyResultOfJsonQueryFuncNotAllowed=Empty result of JSON_QUERY function is not allowed
+IllegalEmptyBehaviorInJsonQueryFunc=Illegal empty behavior ''{0}'' specified in JSON_VALUE function
+ArrayOrObjectValueRequiredInStrictModeOfJsonQueryFunc=Strict jsonpath mode requires array or object value, and the actual value is: ''{0}''
+IllegalErrorBehaviorInJsonQueryFunc=Illegal error behavior ''{0}'' specified in JSON_VALUE function
+NullKeyOfJsonObjectNotAllowed=Null key of JSON object is not allowed
 # End CalciteResource.properties

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/test/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/core/src/test/codegen/config.fmpp b/core/src/test/codegen/config.fmpp
index 317656f..db3518b 100644
--- a/core/src/test/codegen/config.fmpp
+++ b/core/src/test/codegen/config.fmpp
@@ -35,6 +35,7 @@ data: {
       # List of keywords from "keywords" section that are not reserved.
       nonReservedKeywords: [
         "A"
+        "ABSENT"
         "ABSOLUTE"
         "ACTION"
         "ADA"
@@ -73,6 +74,7 @@ data: {
         "COMMAND_FUNCTION_CODE"
         "COMMITTED"
         "CONDITION_NUMBER"
+        "CONDITIONAL"
         "CONNECTION"
         "CONNECTION_NAME"
         "CONSTRAINT_CATALOG"
@@ -105,13 +107,16 @@ data: {
         "DOY"
         "DYNAMIC_FUNCTION"
         "DYNAMIC_FUNCTION_CODE"
+        "ENCODING"
         "EPOCH"
+        "ERROR"
         "EXCEPTION"
         "EXCLUDE"
         "EXCLUDING"
         "FINAL"
         "FIRST"
         "FOLLOWING"
+        "FORMAT"
         "FORTRAN"
         "FOUND"
         "FRAC_SECOND"
@@ -187,6 +192,7 @@ data: {
         "PARAMETER_SPECIFIC_SCHEMA"
         "PARTIAL"
         "PASCAL"
+        "PASSING"
         "PASSTHROUGH"
         "PAST"
         "PATH"
@@ -209,12 +215,14 @@ data: {
         "RETURNED_LENGTH"
         "RETURNED_OCTET_LENGTH"
         "RETURNED_SQLSTATE"
+        "RETURNING"
         "ROLE"
         "ROUTINE"
         "ROUTINE_CATALOG"
         "ROUTINE_NAME"
         "ROUTINE_SCHEMA"
         "ROW_COUNT"
+        "SCALAR"
         "SCALE"
         "SCHEMA"
         "SCHEMA_NAME"
@@ -308,6 +316,7 @@ data: {
         "TYPE"
         "UNBOUNDED"
         "UNCOMMITTED"
+        "UNCONDITIONAL"
         "UNDER"
         "UNNAMED"
         "USAGE"
@@ -315,6 +324,9 @@ data: {
         "USER_DEFINED_TYPE_CODE"
         "USER_DEFINED_TYPE_NAME"
         "USER_DEFINED_TYPE_SCHEMA"
+        "UTF8"
+        "UTF16"
+        "UTF32"
         "VERSION"
         "VIEW"
         "WEEK"

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
index 351550a..189c33f 100644
--- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
@@ -2822,6 +2822,87 @@ public class RelToSqlConverterTest {
     sql(query).ok(expected);
   }
 
+  @Test public void testJsonExists() {
+    String query = "select json_exists(\"product_name\", 'lax $') from \"product\"";
+    final String expected = "SELECT JSON_EXISTS(\"product_name\" FORMAT JSON, 'lax $')\n"
+        + "FROM \"foodmart\".\"product\"";
+    sql(query).ok(expected);
+  }
+
+  @Test public void testJsonValue() {
+    String query = "select json_value(\"product_name\", 'lax $') from \"product\"";
+    // todo translate to JSON_VALUE rather than CAST
+    final String expected = "SELECT CAST(JSON_VALUE_ANY(\"product_name\" FORMAT JSON, "
+        + "'lax $' NULL ON EMPTY NULL ON ERROR) AS VARCHAR(2000) CHARACTER SET \"ISO-8859-1\")\n"
+        + "FROM \"foodmart\".\"product\"";
+    sql(query).ok(expected);
+  }
+
+  @Test public void testJsonQuery() {
+    String query = "select json_query(\"product_name\", 'lax $') from \"product\"";
+    final String expected = "SELECT JSON_QUERY(\"product_name\" FORMAT JSON, 'lax $' "
+        + "WITHOUT ARRAY WRAPPER NULL ON EMPTY NULL ON ERROR)\n"
+        + "FROM \"foodmart\".\"product\"";
+    sql(query).ok(expected);
+  }
+
+  @Test public void testJsonArray() {
+    String query = "select json_array(\"product_name\", \"product_name\") from \"product\"";
+    final String expected = "SELECT JSON_ARRAY(\"product_name\", \"product_name\" ABSENT ON NULL)\n"
+        + "FROM \"foodmart\".\"product\"";
+    sql(query).ok(expected);
+  }
+
+  @Test public void testJsonArrayAgg() {
+    String query = "select json_arrayagg(\"product_name\") from \"product\"";
+    final String expected = "SELECT JSON_ARRAYAGG(\"product_name\" ABSENT ON NULL)\n"
+        + "FROM \"foodmart\".\"product\"";
+    sql(query).ok(expected);
+  }
+
+  @Test public void testJsonObject() {
+    String query = "select json_object(\"product_name\": \"product_id\") from \"product\"";
+    final String expected = "SELECT "
+        + "JSON_OBJECT(KEY \"product_name\" VALUE \"product_id\" NULL ON NULL)\n"
+        + "FROM \"foodmart\".\"product\"";
+    sql(query).ok(expected);
+  }
+
+  @Test public void testJsonObjectAgg() {
+    String query = "select json_objectagg(\"product_name\": \"product_id\") from \"product\"";
+    final String expected = "SELECT "
+        + "JSON_OBJECTAGG(KEY \"product_name\" VALUE \"product_id\" NULL ON NULL)\n"
+        + "FROM \"foodmart\".\"product\"";
+    sql(query).ok(expected);
+  }
+
+  @Test public void testJsonPredicate() {
+    String query = "select "
+        + "\"product_name\" is json, "
+        + "\"product_name\" is json value, "
+        + "\"product_name\" is json object, "
+        + "\"product_name\" is json array, "
+        + "\"product_name\" is json scalar, "
+        + "\"product_name\" is not json, "
+        + "\"product_name\" is not json value, "
+        + "\"product_name\" is not json object, "
+        + "\"product_name\" is not json array, "
+        + "\"product_name\" is not json scalar "
+        + "from \"product\"";
+    final String expected = "SELECT "
+        + "\"product_name\" IS JSON VALUE, "
+        + "\"product_name\" IS JSON VALUE, "
+        + "\"product_name\" IS JSON OBJECT, "
+        + "\"product_name\" IS JSON ARRAY, "
+        + "\"product_name\" IS JSON SCALAR, "
+        + "\"product_name\" IS NOT JSON VALUE, "
+        + "\"product_name\" IS NOT JSON VALUE, "
+        + "\"product_name\" IS NOT JSON OBJECT, "
+        + "\"product_name\" IS NOT JSON ARRAY, "
+        + "\"product_name\" IS NOT JSON SCALAR\n"
+        + "FROM \"foodmart\".\"product\"";
+    sql(query).ok(expected);
+  }
 
   /** Fluid interface to run tests. */
   static class Sql {

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index d8dddc3..ad2856a 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -291,6 +291,13 @@ public class SqlParserTest {
       "ISOLATION",                     "92", "99",
       "ITERATE",                             "99", "2003",
       "JOIN",                          "92", "99", "2003", "2011", "2014", "c",
+      "JSON_ARRAY",                                                        "c",
+      "JSON_ARRAYAGG",                                                     "c",
+      "JSON_EXISTS",                                                       "c",
+      "JSON_OBJECT",                                                       "c",
+      "JSON_OBJECTAGG",                                                    "c",
+      "JSON_QUERY",                                                        "c",
+      "JSON_VALUE",                                                        "c",
       "KEEP",                                              "2011",
       "KEY",                           "92", "99",
       "LAG",                                               "2011", "2014", "c",
@@ -8252,6 +8259,146 @@ public class SqlParserTest {
     sql(sql).ok(expected);
   }
 
+  @Test public void testJsonExists() {
+    checkExp("json_exists('{\"foo\": \"bar\"}', 'lax $.foo')",
+        "JSON_EXISTS('{\"foo\": \"bar\"}' FORMAT JSON, 'lax $.foo')");
+    checkExp("json_exists('{\"foo\": \"bar\"}', 'lax $.foo' error on error)",
+        "JSON_EXISTS('{\"foo\": \"bar\"}' FORMAT JSON, 'lax $.foo' ERROR ON ERROR)");
+  }
+
+  @Test public void testJsonValue() {
+    checkExp("json_value('{\"foo\": \"100\"}', 'lax $.foo' "
+            + "returning integer)",
+        "JSON_VALUE('{\"foo\": \"100\"}' FORMAT JSON, 'lax $.foo' "
+            + "RETURNING INTEGER NULL ON EMPTY NULL ON ERROR)");
+    checkExp("json_value('{\"foo\": \"100\"}', 'lax $.foo' "
+            + "returning integer default 10 on empty error on error)",
+        "JSON_VALUE('{\"foo\": \"100\"}' FORMAT JSON, 'lax $.foo' "
+            + "RETURNING INTEGER DEFAULT 10 ON EMPTY ERROR ON ERROR)");
+  }
+
+  @Test public void testJsonQuery() {
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' WITHOUT ARRAY WRAPPER)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITHOUT ARRAY WRAPPER NULL ON EMPTY NULL ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' WITH WRAPPER)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITH UNCONDITIONAL ARRAY WRAPPER NULL ON EMPTY NULL ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' WITH UNCONDITIONAL WRAPPER)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITH UNCONDITIONAL ARRAY WRAPPER NULL ON EMPTY NULL ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' WITH CONDITIONAL WRAPPER)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITH CONDITIONAL ARRAY WRAPPER NULL ON EMPTY NULL ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' NULL ON EMPTY)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITHOUT ARRAY WRAPPER NULL ON EMPTY NULL ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' ERROR ON EMPTY)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITHOUT ARRAY WRAPPER ERROR ON EMPTY NULL ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' EMPTY ARRAY ON EMPTY)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITHOUT ARRAY WRAPPER EMPTY ARRAY ON EMPTY NULL ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' EMPTY OBJECT ON EMPTY)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITHOUT ARRAY WRAPPER EMPTY OBJECT ON EMPTY NULL ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' NULL ON ERROR)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITHOUT ARRAY WRAPPER NULL ON EMPTY NULL ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' ERROR ON ERROR)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITHOUT ARRAY WRAPPER NULL ON EMPTY ERROR ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' EMPTY ARRAY ON ERROR)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITHOUT ARRAY WRAPPER NULL ON EMPTY EMPTY ARRAY ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' EMPTY OBJECT ON ERROR)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITHOUT ARRAY WRAPPER NULL ON EMPTY EMPTY OBJECT ON ERROR)");
+    checkExp("json_query('{\"foo\": \"bar\"}', 'lax $' EMPTY ARRAY ON EMPTY "
+            + "EMPTY OBJECT ON ERROR)",
+        "JSON_QUERY('{\"foo\": \"bar\"}' FORMAT JSON, "
+            + "'lax $' WITHOUT ARRAY WRAPPER EMPTY ARRAY ON EMPTY EMPTY OBJECT ON ERROR)");
+  }
+
+  @Test public void testJsonObject() {
+    checkExp("json_object('foo': 'bar')",
+        "JSON_OBJECT(KEY 'foo' VALUE 'bar' NULL ON NULL)");
+    checkExp("json_object('foo': 'bar', 'foo2': 'bar2')",
+        "JSON_OBJECT(KEY 'foo' VALUE 'bar', KEY 'foo2' VALUE 'bar2' NULL ON NULL)");
+    checkExp("json_object('foo' value 'bar')",
+        "JSON_OBJECT(KEY 'foo' VALUE 'bar' NULL ON NULL)");
+    checkExp("json_object(key 'foo' value 'bar')",
+        "JSON_OBJECT(KEY 'foo' VALUE 'bar' NULL ON NULL)");
+    checkExp("json_object('foo': null)",
+        "JSON_OBJECT(KEY 'foo' VALUE NULL NULL ON NULL)");
+    checkExp("json_object('foo': null absent on null)",
+        "JSON_OBJECT(KEY 'foo' VALUE NULL ABSENT ON NULL)");
+    checkExp("json_object('foo': json_object('foo': 'bar') format json)",
+        "JSON_OBJECT(KEY 'foo' VALUE "
+            + "JSON_OBJECT(KEY 'foo' VALUE 'bar' NULL ON NULL) "
+            + "FORMAT JSON NULL ON NULL)");
+  }
+
+  @Test public void testJsonObjectAgg() {
+    checkExp("json_objectagg(k_column: v_column)",
+        "JSON_OBJECTAGG(KEY `K_COLUMN` VALUE `V_COLUMN` NULL ON NULL)");
+    checkExp("json_objectagg(k_column value v_column)",
+        "JSON_OBJECTAGG(KEY `K_COLUMN` VALUE `V_COLUMN` NULL ON NULL)");
+    checkExp("json_objectagg(key k_column value v_column)",
+        "JSON_OBJECTAGG(KEY `K_COLUMN` VALUE `V_COLUMN` NULL ON NULL)");
+    checkExp("json_objectagg(k_column: null)",
+        "JSON_OBJECTAGG(KEY `K_COLUMN` VALUE NULL NULL ON NULL)");
+    checkExp("json_objectagg(k_column: null absent on null)",
+        "JSON_OBJECTAGG(KEY `K_COLUMN` VALUE NULL ABSENT ON NULL)");
+    checkExp("json_objectagg(k_column: json_object(k_column: v_column) format json)",
+        "JSON_OBJECTAGG(KEY `K_COLUMN` VALUE "
+            + "JSON_OBJECT(KEY `K_COLUMN` VALUE `V_COLUMN` NULL ON NULL) "
+            + "FORMAT JSON NULL ON NULL)");
+  }
+
+  @Test public void testJsonArray() {
+    checkExp("json_array('foo')",
+        "JSON_ARRAY('foo' ABSENT ON NULL)");
+    checkExp("json_array(null)",
+        "JSON_ARRAY(NULL ABSENT ON NULL)");
+    checkExp("json_array(null null on null)",
+        "JSON_ARRAY(NULL NULL ON NULL)");
+    checkExp("json_array(json_array('foo', 'bar') format json)",
+        "JSON_ARRAY(JSON_ARRAY('foo', 'bar' ABSENT ON NULL) FORMAT JSON ABSENT ON NULL)");
+  }
+
+  @Test public void testJsonArrayAgg() {
+    checkExp("json_arrayagg(\"column\")",
+        "JSON_ARRAYAGG(`column` ABSENT ON NULL)");
+    checkExp("json_arrayagg(\"column\" null on null)",
+        "JSON_ARRAYAGG(`column` NULL ON NULL)");
+    checkExp("json_arrayagg(json_array(\"column\") format json)",
+        "JSON_ARRAYAGG(JSON_ARRAY(`column` ABSENT ON NULL) FORMAT JSON ABSENT ON NULL)");
+  }
+
+  @Test public void testJsonPredicate() {
+    checkExp("'{}' is json",
+        "('{}' IS JSON VALUE)");
+    checkExp("'{}' is json value",
+        "('{}' IS JSON VALUE)");
+    checkExp("'{}' is json object",
+        "('{}' IS JSON OBJECT)");
+    checkExp("'[]' is json array",
+        "('[]' IS JSON ARRAY)");
+    checkExp("'100' is json scalar",
+        "('100' IS JSON SCALAR)");
+    checkExp("'{}' is not json",
+        "('{}' IS NOT JSON VALUE)");
+    checkExp("'{}' is not json value",
+        "('{}' IS NOT JSON VALUE)");
+    checkExp("'{}' is not json object",
+        "('{}' IS NOT JSON OBJECT)");
+    checkExp("'[]' is not json array",
+        "('[]' IS NOT JSON ARRAY)");
+    checkExp("'100' is not json scalar",
+        "('100' IS NOT JSON SCALAR)");
+  }
+
   //~ Inner Interfaces -------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java b/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
index f9f252e..95b183e 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
@@ -346,6 +346,16 @@ public abstract class AbstractSqlTester implements SqlTester, AutoCloseable {
     check(query, SqlTests.ANY_TYPE_CHECKER, result, delta);
   }
 
+  public void checkAggWithMultipleArgs(
+      String expr,
+      String[][] inputValues,
+      Object result,
+      double delta) {
+    String query =
+        SqlTests.generateAggQueryWithMultipleArgs(expr, inputValues);
+    check(query, SqlTests.ANY_TYPE_CHECKER, result, delta);
+  }
+
   public void checkWinAgg(
       String expr,
       String[] inputValues,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
index d7376c1..c79454f 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
@@ -165,6 +165,13 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
           "KEYWORD(GROUPING)",
           "KEYWORD(HOUR)",
           "KEYWORD(INTERVAL)",
+          "KEYWORD(JSON_ARRAY)",
+          "KEYWORD(JSON_ARRAYAGG)",
+          "KEYWORD(JSON_EXISTS)",
+          "KEYWORD(JSON_OBJECT)",
+          "KEYWORD(JSON_OBJECTAGG)",
+          "KEYWORD(JSON_QUERY)",
+          "KEYWORD(JSON_VALUE)",
           "KEYWORD(LAG)",
           "KEYWORD(LAST_VALUE)",
           "KEYWORD(LEAD)",

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/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 da7d099..c1cb089 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
@@ -4254,6 +4254,309 @@ public abstract class SqlOperatorBaseTest {
     tester.checkNull("upper(cast(null as varchar(1)))");
   }
 
+
+  @Test public void testJsonExists() {
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'strict $.foo' false on error)", Boolean.TRUE);
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'strict $.foo' true on error)", Boolean.TRUE);
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'strict $.foo' unknown on error)", Boolean.TRUE);
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'lax $.foo' false on error)", Boolean.TRUE);
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'lax $.foo' true on error)", Boolean.TRUE);
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'lax $.foo' unknown on error)", Boolean.TRUE);
+    tester.checkBoolean("json_exists('{}', "
+        + "'invalid $.foo' false on error)", Boolean.FALSE);
+    tester.checkBoolean("json_exists('{}', "
+        + "'invalid $.foo' true on error)", Boolean.TRUE);
+    tester.checkBoolean("json_exists('{}', "
+        + "'invalid $.foo' unknown on error)", null);
+
+    // not exists
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'strict $.foo1' false on error)", Boolean.FALSE);
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'strict $.foo1' true on error)", Boolean.TRUE);
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'strict $.foo1' unknown on error)", null);
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'lax $.foo1' true on error)", Boolean.FALSE);
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'lax $.foo1' false on error)", Boolean.FALSE);
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'lax $.foo1' error on error)", Boolean.FALSE);
+    tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
+        + "'lax $.foo1' unknown on error)", Boolean.FALSE);
+  }
+
+  @Test public void testJsonValue() {
+    // type casting test
+    tester.checkString("json_value('{\"foo\":100}', 'strict $.foo')",
+        "100", "VARCHAR(2000)");
+    tester.checkScalar("json_value('{\"foo\":100}', 'strict $.foo' returning integer)",
+        100, "INTEGER");
+    tester.checkFails("json_value('{\"foo\":\"100\"}', 'strict $.foo' returning boolean)",
+        INVALID_CHAR_MESSAGE, true);
+    tester.checkScalar("json_value('{\"foo\":100}', 'lax $.foo1' returning integer "
+            + "null on empty)", null, "INTEGER");
+    tester.checkScalar("json_value('{\"foo\":\"100\"}', 'strict $.foo1' returning boolean "
+        + "null on error)", null, "BOOLEAN");
+
+    // lax test
+    tester.checkString("json_value('{\"foo\":100}', 'lax $.foo' null on empty)",
+        "100", "VARCHAR(2000)");
+    tester.checkString("json_value('{\"foo\":100}', 'lax $.foo' error on empty)",
+        "100", "VARCHAR(2000)");
+    tester.checkString("json_value('{\"foo\":100}', 'lax $.foo' default 'empty' on empty)",
+        "100", "VARCHAR(2000)");
+    tester.checkString("json_value('{\"foo\":100}', 'lax $.foo1' null on empty)",
+        null, "VARCHAR(2000)");
+    tester.checkFails("json_value('{\"foo\":100}', 'lax $.foo1' error on empty)",
+        "(?s).*Empty result of JSON_VALUE function is not allowed.*", true);
+    tester.checkString("json_value('{\"foo\":100}', 'lax $.foo1' default 'empty' on empty)",
+        "empty", "VARCHAR(2000)");
+    tester.checkString("json_value('{\"foo\":{}}', 'lax $.foo' null on empty)",
+        null, "VARCHAR(2000)");
+    tester.checkFails("json_value('{\"foo\":{}}', 'lax $.foo' error on empty)",
+        "(?s).*Empty result of JSON_VALUE function is not allowed.*", true);
+    tester.checkString("json_value('{\"foo\":{}}', 'lax $.foo' default 'empty' on empty)",
+        "empty", "VARCHAR(2000)");
+    tester.checkString("json_value('{\"foo\":100}', 'lax $.foo' null on error)",
+        "100", "VARCHAR(2000)");
+    tester.checkString("json_value('{\"foo\":100}', 'lax $.foo' error on error)",
+        "100", "VARCHAR(2000)");
+    tester.checkString("json_value('{\"foo\":100}', 'lax $.foo' default 'empty' on error)",
+        "100", "VARCHAR(2000)");
+
+    // path error test
+    tester.checkString("json_value('{\"foo\":100}', 'invalid $.foo' null on error)",
+        null, "VARCHAR(2000)");
+    tester.checkFails("json_value('{\"foo\":100}', 'invalid $.foo' error on error)",
+        "(?s).*Illegal jsonpath spec.*", true);
+    tester.checkString("json_value('{\"foo\":100}', "
+            + "'invalid $.foo' default 'empty' on error)",
+        "empty", "VARCHAR(2000)");
+
+    // strict test
+    tester.checkString("json_value('{\"foo\":100}', 'strict $.foo' null on empty)",
+        "100", "VARCHAR(2000)");
+    tester.checkString("json_value('{\"foo\":100}', 'strict $.foo' error on empty)",
+        "100", "VARCHAR(2000)");
+    tester.checkString("json_value('{\"foo\":100}', "
+            + "'strict $.foo' default 'empty' on empty)",
+        "100", "VARCHAR(2000)");
+    tester.checkString("json_value('{\"foo\":100}', 'strict $.foo1' null on error)",
+        null, "VARCHAR(2000)");
+    tester.checkFails("json_value('{\"foo\":100}', 'strict $.foo1' error on error)",
+        "(?s).*No results for path: \\$\\['foo1'\\].*", true);
+    tester.checkString("json_value('{\"foo\":100}', "
+            + "'strict $.foo1' default 'empty' on error)",
+        "empty", "VARCHAR(2000)");
+    tester.checkString("json_value('{\"foo\":{}}', 'strict $.foo' null on error)",
+        null, "VARCHAR(2000)");
+    tester.checkFails("json_value('{\"foo\":{}}', 'strict $.foo' error on error)",
+        "(?s).*Strict jsonpath mode requires scalar value, "
+            + "and the actual value is: '\\{\\}'.*", true);
+    tester.checkString("json_value('{\"foo\":{}}', "
+            + "'strict $.foo' default 'empty' on error)",
+        "empty", "VARCHAR(2000)");
+  }
+
+  @Test public void testJsonQuery() {
+    // lax test
+    tester.checkString("json_query('{\"foo\":100}', 'lax $' null on empty)",
+        "{\"foo\":100}", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'lax $' error on empty)",
+        "{\"foo\":100}", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'lax $' empty array on empty)",
+        "{\"foo\":100}", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'lax $' empty object on empty)",
+        "{\"foo\":100}", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'lax $.foo' null on empty)",
+        null, "VARCHAR(2000)");
+    tester.checkFails("json_query('{\"foo\":100}', 'lax $.foo' error on empty)",
+        "(?s).*Empty result of JSON_QUERY function is not allowed.*", true);
+    tester.checkString("json_query('{\"foo\":100}', 'lax $.foo' empty array on empty)",
+        "[]", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'lax $.foo' empty object on empty)",
+        "{}", "VARCHAR(2000)");
+
+    // path error test
+    tester.checkString("json_query('{\"foo\":100}', 'invalid $.foo' null on error)",
+        null, "VARCHAR(2000)");
+    tester.checkFails("json_query('{\"foo\":100}', 'invalid $.foo' error on error)",
+        "(?s).*Illegal jsonpath spec.*", true);
+    tester.checkString("json_query('{\"foo\":100}', "
+            + "'invalid $.foo' empty array on error)",
+        "[]", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', "
+            + "'invalid $.foo' empty object on error)",
+        "{}", "VARCHAR(2000)");
+
+    // strict test
+    tester.checkString("json_query('{\"foo\":100}', 'strict $' null on empty)",
+        "{\"foo\":100}", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'strict $' error on empty)",
+        "{\"foo\":100}", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'strict $' empty array on error)",
+        "{\"foo\":100}", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'strict $' empty object on error)",
+        "{\"foo\":100}", "VARCHAR(2000)");
+
+    tester.checkString("json_query('{\"foo\":100}', 'strict $.foo1' null on error)",
+        null, "VARCHAR(2000)");
+    tester.checkFails("json_query('{\"foo\":100}', 'strict $.foo1' error on error)",
+        "(?s).*No results for path: \\$\\['foo1'\\].*", true);
+    tester.checkString("json_query('{\"foo\":100}', 'strict $.foo1' empty array on error)",
+        "[]", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'strict $.foo1' empty object on error)",
+        "{}", "VARCHAR(2000)");
+
+    tester.checkString("json_query('{\"foo\":100}', 'strict $.foo' null on error)",
+        null, "VARCHAR(2000)");
+    tester.checkFails("json_query('{\"foo\":100}', 'strict $.foo' error on error)",
+        "(?s).*Strict jsonpath mode requires array or object value, "
+            + "and the actual value is: '100'.*", true);
+    tester.checkString("json_query('{\"foo\":100}', 'strict $.foo' empty array on error)",
+        "[]", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'strict $.foo' empty object on error)",
+        "{}", "VARCHAR(2000)");
+
+    // array wrapper test
+    tester.checkString("json_query('{\"foo\":100}', 'strict $.foo' without wrapper)",
+        null, "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'strict $.foo' without array wrapper)",
+        null, "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'strict $.foo' with wrapper)",
+        "[100]", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'strict $.foo' "
+            + "with unconditional wrapper)",
+        "[100]", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":100}', 'strict $.foo' "
+            + "with conditional wrapper)",
+        "[100]", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":[100]}', 'strict $.foo' without wrapper)",
+        "[100]", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":[100]}', 'strict $.foo' without array wrapper)",
+        "[100]", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":[100]}', 'strict $.foo' with wrapper)",
+        "[[100]]", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":[100]}', 'strict $.foo' "
+            + "with unconditional wrapper)",
+        "[[100]]", "VARCHAR(2000)");
+    tester.checkString("json_query('{\"foo\":[100]}', 'strict $.foo' "
+            + "with conditional wrapper)",
+        "[100]", "VARCHAR(2000)");
+
+
+  }
+
+  @Test public void testJsonObject() {
+    tester.checkString("json_object()", "{}", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_object('foo': 'bar')",
+        "{\"foo\":\"bar\"}", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_object('foo': 'bar', 'foo2': 'bar2')",
+        "{\"foo\":\"bar\",\"foo2\":\"bar2\"}", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_object('foo': null)",
+        "{\"foo\":null}", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_object('foo': null null on null)",
+        "{\"foo\":null}", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_object('foo': null absent on null)",
+        "{}", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_object('foo': 100)",
+        "{\"foo\":100}", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_object('foo': json_object('foo': 'bar'))",
+        "{\"foo\":\"{\\\"foo\\\":\\\"bar\\\"}\"}", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_object('foo': json_object('foo': 'bar') format json)",
+        "{\"foo\":{\"foo\":\"bar\"}}", "VARCHAR(2000) NOT NULL");
+  }
+
+  @Test public void testJsonObjectAgg() {
+    checkAggType(tester, "json_objectagg('foo': 'bar')", "VARCHAR(2000) NOT NULL");
+    tester.checkFails("^json_objectagg(100: 'bar')^",
+        "(?s).*Cannot apply.*", false);
+    final String[][] values = {
+        {"'foo'", "'bar'"},
+        {"'foo2'", "cast(null as varchar(2000))"},
+        {"'foo3'", "'bar3'"}
+    };
+    tester.checkAggWithMultipleArgs("json_objectagg(x: x2)",
+        values,
+        "{\"foo\":\"bar\",\"foo2\":null,\"foo3\":\"bar3\"}",
+        0.0D);
+    tester.checkAggWithMultipleArgs("json_objectagg(x: x2 null on null)",
+        values,
+        "{\"foo\":\"bar\",\"foo2\":null,\"foo3\":\"bar3\"}",
+        0.0D);
+    tester.checkAggWithMultipleArgs("json_objectagg(x: x2 absent on null)",
+        values,
+        "{\"foo\":\"bar\",\"foo3\":\"bar3\"}",
+        0.0D);
+  }
+
+  @Test public void testJsonArray() {
+    tester.checkString("json_array()", "[]", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_array('foo')",
+        "[\"foo\"]", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_array('foo', 'bar')",
+        "[\"foo\",\"bar\"]", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_array(null)",
+        "[]", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_array(null null on null)",
+        "[null]", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_array(null absent on null)",
+        "[]", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_array(100)",
+        "[100]", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_array(json_array('foo'))",
+        "[\"[\\\"foo\\\"]\"]", "VARCHAR(2000) NOT NULL");
+    tester.checkString("json_array(json_array('foo') format json)",
+        "[[\"foo\"]]", "VARCHAR(2000) NOT NULL");
+  }
+
+  @Test public void testJsonArrayAgg() {
+    checkAggType(tester, "json_arrayagg('foo')", "VARCHAR(2000) NOT NULL");
+    final String[] values = {
+        "'foo'",
+        "cast(null as varchar(2000))",
+        "'foo3'"
+    };
+    tester.checkAgg("json_arrayagg(x)",
+        values,
+        "[\"foo\",\"foo3\"]",
+        0.0D);
+    tester.checkAgg("json_arrayagg(x null on null)",
+        values,
+        "[\"foo\",null,\"foo3\"]",
+        0.0D);
+    tester.checkAgg("json_arrayagg(x absent on null)",
+        values,
+        "[\"foo\",\"foo3\"]",
+        0.0D);
+  }
+
+  @Test public void testJsonPredicate() {
+    tester.checkBoolean("'{}' is json value", true);
+    tester.checkBoolean("'{]' is json value", false);
+    tester.checkBoolean("'{}' is json object", true);
+    tester.checkBoolean("'[]' is json object", false);
+    tester.checkBoolean("'{}' is json array", false);
+    tester.checkBoolean("'[]' is json array", true);
+    tester.checkBoolean("'100' is json scalar", true);
+    tester.checkBoolean("'[]' is json scalar", false);
+    tester.checkBoolean("'{}' is not json value", false);
+    tester.checkBoolean("'{]' is not json value", true);
+    tester.checkBoolean("'{}' is not json object", false);
+    tester.checkBoolean("'[]' is not json object", true);
+    tester.checkBoolean("'{}' is not json array", true);
+    tester.checkBoolean("'[]' is not json array", false);
+    tester.checkBoolean("'100' is not json scalar", false);
+    tester.checkBoolean("'[]' is not json scalar", true);
+  }
+
   @Test public void testLowerFunc() {
     tester.setFor(SqlStdOperatorTable.LOWER);
 
@@ -5226,7 +5529,7 @@ public abstract class SqlOperatorBaseTest {
         "VARCHAR(3) NOT NULL");
     tester.checkFails(
         "substring('abc' from 1 for -1)",
-        "substring error: negative substring length not allowed",
+        "Substring error: negative substring length not allowed",
         true);
     tester.checkString(
         "substring('abc' from 2)", "bc", "VARCHAR(3) NOT NULL");
@@ -5259,7 +5562,7 @@ public abstract class SqlOperatorBaseTest {
         "VARBINARY(3) NOT NULL");
     tester.checkFails(
         "substring(x'aabbcc' from 1 for -1)",
-        "substring error: negative substring length not allowed",
+        "Substring error: negative substring length not allowed",
         true);
     tester.checkString(
         "substring(x'aabbcc' from 2)", "bbcc", "VARBINARY(3) NOT NULL");
@@ -5310,11 +5613,11 @@ public abstract class SqlOperatorBaseTest {
     // have the SQL error code 22027.
     tester.checkFails(
         "trim('xy' from 'abcde')",
-        "trim error: trim character must be exactly 1 character",
+        "Trim error: trim character must be exactly 1 character",
         true);
     tester.checkFails(
         "trim('' from 'abcde')",
-        "trim error: trim character must be exactly 1 character",
+        "Trim error: trim character must be exactly 1 character",
         true);
 
     final SqlTester tester1 = tester.withConformance(SqlConformanceEnum.MYSQL_5);

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/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 196911b..ef63253 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
@@ -318,6 +318,23 @@ public interface SqlTester extends AutoCloseable, SqlValidatorTestCase.Tester {
       double delta);
 
   /**
+   * Checks that an aggregate expression with multiple args returns the expected result.
+   *
+   * @param expr        Aggregate expression, e.g. <code>AGG_FUNC(x, x2, x3)</code>
+   * @param inputValues Nested array of input values, e.g. <code>[
+   *                    ["1", null, "2"]
+   *                    ["3", "4", null]
+   *                    ]</code>.
+   * @param result      Expected result
+   * @param delta       Allowable variance from expected result
+   */
+  void checkAggWithMultipleArgs(
+      String expr,
+      String[][] inputValues,
+      Object result,
+      double delta);
+
+  /**
    * Checks that a windowed aggregate expression returns the expected result.
    *
    * <p>For example, <code>checkWinAgg("FIRST_VALUE(x)", new String[] {"2",

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
index 22e3d95..a46a002 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
@@ -27,6 +27,7 @@ import org.apache.calcite.util.Util;
 
 import java.sql.ResultSet;
 import java.sql.Types;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
@@ -136,6 +137,43 @@ public abstract class SqlTests {
     return buf.toString();
   }
 
+  public static String generateAggQueryWithMultipleArgs(String expr, String[][] inputValues) {
+    int argCount = -1;
+    for (String[] row : inputValues) {
+      if (argCount == -1) {
+        argCount = row.length;
+      } else if (argCount != row.length) {
+        throw new IllegalArgumentException("invalid test input: " + Arrays.toString(row));
+      }
+    }
+    StringBuilder buf = new StringBuilder();
+    buf.append("SELECT ").append(expr).append(" FROM ");
+    if (inputValues.length == 0) {
+      buf.append("(VALUES 1) AS t(x) WHERE false");
+    } else {
+      buf.append("(");
+      for (int i = 0; i < inputValues.length; i++) {
+        if (i > 0) {
+          buf.append(" UNION ALL ");
+        }
+        buf.append("SELECT ");
+        for (int j = 0; j < argCount; j++) {
+          if (j != 0) {
+            buf.append(", ");
+          }
+          String inputValue = inputValues[i][j];
+          buf.append(inputValue).append(" AS x");
+          if (j != 0) {
+            buf.append(j + 1);
+          }
+        }
+        buf.append(" FROM (VALUES (1))");
+      }
+      buf.append(")");
+    }
+    return buf.toString();
+  }
+
   public static String generateWinAggQuery(
       String expr,
       String windowSpec,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
index 058d813..4c6f620 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -100,6 +100,7 @@ import org.junit.runners.Suite;
     FilteratorTest.class,
     PermutationTestCase.class,
     SqlFunctionsTest.class,
+    SqlJsonFunctionsTest.class,
     SqlTypeNameTest.class,
     ModelTest.class,
     SqlValidatorFeatureTest.class,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8e557d26/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
index 10aa51b..55d8811 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
@@ -18,6 +18,7 @@ package org.apache.calcite.test;
 
 import org.apache.calcite.avatica.util.ByteString;
 import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.runtime.Utilities;
 
@@ -530,7 +531,7 @@ public class SqlFunctionsTest {
     try {
       assertThat(SqlFunctions.ltAny("1", 2L), is(false));
       fail("'lt' on non-numeric different type is not possible");
-    } catch (IllegalArgumentException e) {
+    } catch (CalciteException e) {
       assertThat(e.getMessage(),
           is("Invalid types for comparison: class java.lang.String < "
               + "class java.lang.Long"));
@@ -565,7 +566,7 @@ public class SqlFunctionsTest {
     try {
       assertThat(SqlFunctions.leAny("2", 2L), is(false));
       fail("'le' on non-numeric different type is not possible");
-    } catch (IllegalArgumentException e) {
+    } catch (CalciteException e) {
       assertThat(e.getMessage(),
           is("Invalid types for comparison: class java.lang.String <= "
               + "class java.lang.Long"));
@@ -591,7 +592,7 @@ public class SqlFunctionsTest {
     try {
       assertThat(SqlFunctions.gtAny("2", 1L), is(false));
       fail("'gt' on non-numeric different type is not possible");
-    } catch (IllegalArgumentException e) {
+    } catch (CalciteException e) {
       assertThat(e.getMessage(),
           is("Invalid types for comparison: class java.lang.String > "
               + "class java.lang.Long"));
@@ -626,7 +627,7 @@ public class SqlFunctionsTest {
     try {
       assertThat(SqlFunctions.geAny("2", 2L), is(false));
       fail("'ge' on non-numeric different type is not possible");
-    } catch (IllegalArgumentException e) {
+    } catch (CalciteException e) {
       assertThat(e.getMessage(),
           is("Invalid types for comparison: class java.lang.String >= "
               + "class java.lang.Long"));
@@ -656,7 +657,7 @@ public class SqlFunctionsTest {
     try {
       SqlFunctions.plusAny("2", 2L);
       fail("'plus' on non-numeric type is not possible");
-    } catch (IllegalArgumentException e) {
+    } catch (CalciteException e) {
       assertThat(e.getMessage(),
           is("Invalid types for arithmetic: class java.lang.String + "
               + "class java.lang.Long"));
@@ -686,7 +687,7 @@ public class SqlFunctionsTest {
     try {
       SqlFunctions.minusAny("2", 2L);
       fail("'minus' on non-numeric type is not possible");
-    } catch (IllegalArgumentException e) {
+    } catch (CalciteException e) {
       assertThat(e.getMessage(),
           is("Invalid types for arithmetic: class java.lang.String - "
               + "class java.lang.Long"));
@@ -718,7 +719,7 @@ public class SqlFunctionsTest {
     try {
       SqlFunctions.multiplyAny("2", 2L);
       fail("'multiply' on non-numeric type is not possible");
-    } catch (IllegalArgumentException e) {
+    } catch (CalciteException e) {
       assertThat(e.getMessage(),
           is("Invalid types for arithmetic: class java.lang.String * "
               + "class java.lang.Long"));
@@ -751,7 +752,7 @@ public class SqlFunctionsTest {
     try {
       SqlFunctions.divideAny("5", 2L);
       fail("'divide' on non-numeric type is not possible");
-    } catch (IllegalArgumentException e) {
+    } catch (CalciteException e) {
       assertThat(e.getMessage(),
           is("Invalid types for arithmetic: class java.lang.String / "
               + "class java.lang.Long"));