You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by ho...@apache.org on 2019/05/15 04:57:42 UTC

[calcite] branch master updated: [CALCITE-3017] Improve null handling of JsonValueExpressionOperator

This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git


The following commit(s) were added to refs/heads/master by this push:
     new e98c779  [CALCITE-3017] Improve null handling of JsonValueExpressionOperator
e98c779 is described below

commit e98c779d1ec0bc87c81a72b974c89a41a7222a07
Author: Hongze Zhang <ho...@apache.org>
AuthorDate: Thu Apr 25 02:17:35 2019 +0800

    [CALCITE-3017] Improve null handling of JsonValueExpressionOperator
    
    * Remove SqlJsonApiCommonSyntaxOperator;
    * Move implementation methods of JSON functions from SqlFunctions to JsonFunctions;
    * Add new null policy NullPolicy.ARG0 (currently used by some JSON functions only);
    * Change return type to nullable for some operators of JSON functions;
    * In code generation, perform regular null semantic check on operands of a call when NullAs.IS_NULL/IS_NOT_NULL is used.
---
 core/src/main/codegen/templates/Parser.jj          |  59 +-
 .../calcite/adapter/enumerable/NullPolicy.java     |   2 +
 .../calcite/adapter/enumerable/RexImpTable.java    |  49 +-
 .../org/apache/calcite/runtime/JsonFunctions.java  | 757 +++++++++++++++++++++
 .../org/apache/calcite/runtime/SqlFunctions.java   | 580 +---------------
 .../main/java/org/apache/calcite/sql/SqlKind.java  |   7 +-
 .../sql/fun/SqlJsonApiCommonSyntaxOperator.java    |  64 --
 .../calcite/sql/fun/SqlJsonExistsFunction.java     |  18 +-
 .../calcite/sql/fun/SqlJsonKeysFunction.java       |   3 +-
 .../calcite/sql/fun/SqlJsonPrettyFunction.java     |   3 +-
 .../calcite/sql/fun/SqlJsonQueryFunction.java      |  20 +-
 .../calcite/sql/fun/SqlJsonTypeFunction.java       |   5 +-
 .../calcite/sql/fun/SqlJsonValueFunction.java      |  88 +--
 .../calcite/sql/fun/SqlStdOperatorTable.java       |   3 -
 .../calcite/sql2rel/StandardConvertletTable.java   |  73 +-
 .../org/apache/calcite/util/BuiltInMethod.java     |  45 +-
 .../main/java/org/apache/calcite/util/Util.java    |  12 +
 .../calcite/rel/rel2sql/RelToSqlConverterTest.java |  18 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java      | 112 ++-
 .../apache/calcite/test/SqlJsonFunctionsTest.java  | 320 ++++-----
 .../org/apache/calcite/test/SqlValidatorTest.java  |  11 +-
 .../apache/calcite/test/SqlToRelConverterTest.xml  |  16 +-
 site/_docs/reference.md                            |   3 +-
 23 files changed, 1212 insertions(+), 1056 deletions(-)

diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index ab706f1..192f65b 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -4988,48 +4988,41 @@ SqlNode JsonPathSpec() :
     }
 }
 
-SqlNode JsonApiCommonSyntax(boolean acceptNonPath) :
+List<SqlNode> JsonApiCommonSyntax() :
 {
     SqlNode e;
     List<SqlNode> args = new ArrayList<SqlNode>();
-    Span span;
 }
 {
     e = Expression(ExprContext.ACCEPT_NON_QUERY) {
         args.add(e);
-        span = Span.of(e);
     }
     (
         <COMMA>
         e = Expression(ExprContext.ACCEPT_NON_QUERY) {
             args.add(e);
         }
-    |
-        {
-            if (!acceptNonPath) {
-              throw new ParseException(RESOURCE.jsonPathMustBeSpecified().str());
-            }
-        }
     )
     [
+        // We currently don't support JSON passing clause, leave the java code blocks no-op
         <PASSING> e = Expression(ExprContext.ACCEPT_NON_QUERY) {
-            args.add(e);
+            // no-op
         }
         <AS> e = SimpleIdentifier() {
-            args.add(e);
+            // no-op
         }
         (
             <COMMA>
             e = Expression(ExprContext.ACCEPT_NON_QUERY) {
-                        args.add(e);
+                // no-op
             }
             <AS> e = SimpleIdentifier() {
-                        args.add(e);
+                // no-op
             }
         )*
     ]
     {
-        return SqlStdOperatorTable.JSON_API_COMMON_SYNTAX.createCall(span.end(this), args);
+        return args;
     }
 }
 
@@ -5050,15 +5043,15 @@ SqlJsonExistsErrorBehavior JsonExistsErrorBehavior() :
 SqlCall JsonExistsFunctionCall() :
 {
     List<SqlNode> args;
-    SqlNode e;
+    List<SqlNode> commonSyntax;
     final Span span;
     SqlJsonExistsErrorBehavior errorBehavior;
 }
 {
     <JSON_EXISTS> { span = span(); }
-    <LPAREN> e = JsonApiCommonSyntax(false) {
+    <LPAREN> commonSyntax = JsonApiCommonSyntax() {
         args = new ArrayList<SqlNode>();
-        args.add(e);
+        args.addAll(commonSyntax);
     }
     [
         errorBehavior = JsonExistsErrorBehavior() { args.add(SqlLiteral.createSymbol(errorBehavior, getPos())); }
@@ -5106,19 +5099,21 @@ List<SqlNode> JsonValueEmptyOrErrorBehavior() :
 
 SqlCall JsonValueFunctionCall() :
 {
-    final SqlNode[] args = new SqlNode[6];
+    final SqlNode[] args = new SqlNode[7];
     SqlNode e;
+    List<SqlNode> commonSyntax;
     final Span span;
     List<SqlNode> behavior;
 }
 {
     <JSON_VALUE> { span = span(); }
-    <LPAREN> e = JsonApiCommonSyntax(false) {
-        args[0] = e;
+    <LPAREN> commonSyntax = JsonApiCommonSyntax() {
+        args[0] = commonSyntax.get(0);
+        args[1] = commonSyntax.get(1);
     }
     [
         e = JsonReturningClause() {
-            args[5] = e;
+            args[6] = e;
         }
     ]
     (
@@ -5127,12 +5122,12 @@ SqlCall JsonValueFunctionCall() :
                 ((SqlLiteral) behavior.get(2)).getValueAs(SqlJsonEmptyOrError.class);
             switch (symbol) {
             case EMPTY:
-                args[1] = behavior.get(0);
-                args[2] = behavior.get(1);
+                args[2] = behavior.get(0);
+                args[3] = behavior.get(1);
                 break;
             case ERROR:
-                args[3] = behavior.get(0);
-                args[4] = behavior.get(1);
+                args[4] = behavior.get(0);
+                args[5] = behavior.get(1);
                 break;
             }
         }
@@ -5200,19 +5195,21 @@ SqlNode JsonQueryWrapperBehavior() :
 
 SqlCall JsonQueryFunctionCall() :
 {
-    final SqlNode[] args = new SqlNode[4];
+    final SqlNode[] args = new SqlNode[5];
     SqlNode e;
+    List<SqlNode> commonSyntax;
     final Span span;
     List<SqlNode> behavior;
 }
 {
     <JSON_QUERY> { span = span(); }
-    <LPAREN> e = JsonApiCommonSyntax(false) {
-        args[0] = e;
+    <LPAREN> commonSyntax = JsonApiCommonSyntax() {
+        args[0] = commonSyntax.get(0);
+        args[1] = commonSyntax.get(1);
     }
     [
         e = JsonQueryWrapperBehavior() <WRAPPER> {
-            args[1] = e;
+            args[2] = e;
         }
     ]
     (
@@ -5221,10 +5218,10 @@ SqlCall JsonQueryFunctionCall() :
                 ((SqlLiteral) behavior.get(1)).getValueAs(SqlJsonEmptyOrError.class);
             switch (symbol) {
             case EMPTY:
-                args[2] = behavior.get(0);
+                args[3] = behavior.get(0);
                 break;
             case ERROR:
-                args[3] = behavior.get(0);
+                args[4] = behavior.get(0);
                 break;
             }
         }
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
index 43c3431..736af5a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
@@ -31,6 +31,8 @@ public enum NullPolicy {
   SEMI_STRICT,
   /** If any of the arguments are null, return null. */
   ANY,
+  /** If the first argument is null, return null. */
+  ARG0,
   /** If any of the arguments are false, result is false; else if any
    * arguments are null, result is null; else true. */
   AND,
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 4f29b2c..5a9f835 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
@@ -166,7 +166,6 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_TRUE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NULL;
 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;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_EXISTS;
@@ -455,18 +454,16 @@ public class RexImpTable {
     // Json Operators
     defineMethod(JSON_VALUE_EXPRESSION,
         BuiltInMethod.JSON_VALUE_EXPRESSION.method, NullPolicy.STRICT);
-    defineMethod(JSON_API_COMMON_SYNTAX, BuiltInMethod.JSON_API_COMMON_SYNTAX.method,
-            NullPolicy.NONE);
-    defineMethod(JSON_EXISTS, BuiltInMethod.JSON_EXISTS.method, NullPolicy.NONE);
-    defineMethod(JSON_VALUE_ANY, BuiltInMethod.JSON_VALUE_ANY.method, NullPolicy.NONE);
-    defineMethod(JSON_QUERY, BuiltInMethod.JSON_QUERY.method, NullPolicy.NONE);
+    defineMethod(JSON_EXISTS, BuiltInMethod.JSON_EXISTS.method, NullPolicy.ARG0);
+    defineMethod(JSON_VALUE_ANY, BuiltInMethod.JSON_VALUE_ANY.method, NullPolicy.ARG0);
+    defineMethod(JSON_QUERY, BuiltInMethod.JSON_QUERY.method, NullPolicy.ARG0);
+    defineMethod(JSON_TYPE, BuiltInMethod.JSON_TYPE.method, NullPolicy.ARG0);
+    defineMethod(JSON_DEPTH, BuiltInMethod.JSON_DEPTH.method, NullPolicy.ARG0);
+    defineMethod(JSON_KEYS, BuiltInMethod.JSON_KEYS.method, NullPolicy.ARG0);
+    defineMethod(JSON_PRETTY, BuiltInMethod.JSON_PRETTY.method, NullPolicy.ARG0);
+    defineMethod(JSON_LENGTH, BuiltInMethod.JSON_LENGTH.method, NullPolicy.ARG0);
     defineMethod(JSON_OBJECT, BuiltInMethod.JSON_OBJECT.method, NullPolicy.NONE);
     defineMethod(JSON_ARRAY, BuiltInMethod.JSON_ARRAY.method, NullPolicy.NONE);
-    defineMethod(JSON_TYPE, BuiltInMethod.JSON_TYPE.method, NullPolicy.NONE);
-    defineMethod(JSON_DEPTH, BuiltInMethod.JSON_DEPTH.method, NullPolicy.NONE);
-    defineMethod(JSON_KEYS, BuiltInMethod.JSON_KEYS.method, NullPolicy.NONE);
-    defineMethod(JSON_PRETTY, BuiltInMethod.JSON_PRETTY.method, NullPolicy.NONE);
-    defineMethod(JSON_LENGTH, BuiltInMethod.JSON_LENGTH.method, NullPolicy.NONE);
     aggMap.put(JSON_OBJECTAGG.with(SqlJsonConstructorNullClause.ABSENT_ON_NULL),
         JsonObjectAggImplementor
             .supplierFor(BuiltInMethod.JSON_OBJECTAGG_ADD.method));
@@ -605,6 +602,7 @@ public class RexImpTable {
     case ANY:
     case STRICT:
     case SEMI_STRICT:
+    case ARG0:
       return (translator, call, nullAs) -> implementNullSemantics0(
           translator, call, nullAs, nullPolicy, harmonize,
           implementor);
@@ -965,8 +963,10 @@ public class RexImpTable {
       case NOT_POSSIBLE:
         throw e;
       case FALSE:
+      case IS_NOT_NULL:
         return FALSE_EXPR;
       case TRUE:
+      case IS_NULL:
         return TRUE_EXPR;
       default:
         return NULL_EXPR;
@@ -981,10 +981,15 @@ public class RexImpTable {
       NullPolicy nullPolicy,
       NotNullImplementor implementor) {
     final List<Expression> list = new ArrayList<>();
+    final List<RexNode> conditionalOps
+        = nullPolicy == NullPolicy.ARG0 ? Collections.singletonList(call.getOperands().get(0))
+        : call.getOperands();
     switch (nullAs) {
     case NULL:
+    case IS_NULL:
+    case IS_NOT_NULL:
       // v0 == null || v1 == null ? null : f(v0, v1)
-      for (Ord<RexNode> operand : Ord.zip(call.getOperands())) {
+      for (Ord<RexNode> operand : Ord.zip(conditionalOps)) {
         if (translator.isNullable(operand.e)) {
           list.add(
               translator.translate(
@@ -995,14 +1000,28 @@ public class RexImpTable {
       final Expression box =
           Expressions.box(
               implementCall(translator, call, implementor, nullAs));
+      final Expression ifTrue;
+      switch (nullAs) {
+      case NULL:
+        ifTrue = Types.castIfNecessary(box.getType(), NULL_EXPR);
+        break;
+      case IS_NULL:
+        ifTrue = TRUE_EXPR;
+        break;
+      case IS_NOT_NULL:
+        ifTrue = FALSE_EXPR;
+        break;
+      default:
+        throw new AssertionError();
+      }
       return optimize(
           Expressions.condition(
               Expressions.foldOr(list),
-              Types.castIfNecessary(box.getType(), NULL_EXPR),
+              ifTrue,
               box));
     case FALSE:
       // v0 != null && v1 != null && f(v0, v1)
-      for (Ord<RexNode> operand : Ord.zip(call.getOperands())) {
+      for (Ord<RexNode> operand : Ord.zip(conditionalOps)) {
         if (translator.isNullable(operand.e)) {
           list.add(
               translator.translate(
@@ -1014,7 +1033,7 @@ public class RexImpTable {
       return Expressions.foldAnd(list);
     case TRUE:
       // v0 == null || v1 == null || f(v0, v1)
-      for (Ord<RexNode> operand : Ord.zip(call.getOperands())) {
+      for (Ord<RexNode> operand : Ord.zip(conditionalOps)) {
         if (translator.isNullable(operand.e)) {
           list.add(
               translator.translate(
diff --git a/core/src/main/java/org/apache/calcite/runtime/JsonFunctions.java b/core/src/main/java/org/apache/calcite/runtime/JsonFunctions.java
new file mode 100644
index 0000000..c268708
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/runtime/JsonFunctions.java
@@ -0,0 +1,757 @@
+/*
+ * 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.runtime;
+
+import org.apache.calcite.sql.SqlJsonConstructorNullClause;
+import org.apache.calcite.sql.SqlJsonExistsErrorBehavior;
+import org.apache.calcite.sql.SqlJsonQueryEmptyOrErrorBehavior;
+import org.apache.calcite.sql.SqlJsonQueryWrapperBehavior;
+import org.apache.calcite.sql.SqlJsonValueEmptyOrErrorBehavior;
+import org.apache.calcite.util.Util;
+
+import com.fasterxml.jackson.annotation.JsonValue;
+import com.fasterxml.jackson.core.PrettyPrinter;
+import com.fasterxml.jackson.core.util.DefaultIndenter;
+import com.fasterxml.jackson.core.util.DefaultPrettyPrinter;
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.DocumentContext;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.Option;
+import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
+import com.jayway.jsonpath.spi.mapper.JacksonMappingProvider;
+import com.jayway.jsonpath.spi.mapper.MappingProvider;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Queue;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * A collection of functions used in JSON processing.
+ */
+public class JsonFunctions {
+
+  private static final Pattern JSON_PATH_BASE =
+      Pattern.compile("^\\s*(?<mode>strict|lax)\\s+(?<spec>.+)$",
+          Pattern.CASE_INSENSITIVE | Pattern.DOTALL | Pattern.MULTILINE);
+
+  private static final JacksonJsonProvider JSON_PATH_JSON_PROVIDER =
+      new JacksonJsonProvider();
+  private static final MappingProvider JSON_PATH_MAPPING_PROVIDER =
+      new JacksonMappingProvider();
+  private static final PrettyPrinter JSON_PRETTY_PRINTER =
+      new DefaultPrettyPrinter().withObjectIndenter(
+          DefaultIndenter.SYSTEM_LINEFEED_INSTANCE.withLinefeed("\n"));
+
+  private JsonFunctions() {
+  }
+
+  private static boolean isScalarObject(Object obj) {
+    if (obj instanceof Collection) {
+      return false;
+    }
+    if (obj instanceof Map) {
+      return false;
+    }
+    return true;
+  }
+
+  public static String jsonize(Object input) {
+    return JSON_PATH_JSON_PROVIDER.toJson(input);
+  }
+
+  public static Object dejsonize(String input) {
+    return JSON_PATH_JSON_PROVIDER.parse(input);
+  }
+
+  public static JsonValueContext jsonValueExpression(String input) {
+    try {
+      return JsonValueContext.withJavaObj(dejsonize(input));
+    } catch (Exception e) {
+      return JsonValueContext.withException(e);
+    }
+  }
+
+  public static JsonPathContext jsonApiCommonSyntax(String input) {
+    return jsonApiCommonSyntax(jsonValueExpression(input));
+  }
+
+  public static JsonPathContext jsonApiCommonSyntax(JsonValueContext input) {
+    return jsonApiCommonSyntax(input, "strict $");
+  }
+
+  public static JsonPathContext jsonApiCommonSyntax(String input, String pathSpec) {
+    return jsonApiCommonSyntax(jsonValueExpression(input), pathSpec);
+  }
+
+  public static JsonPathContext jsonApiCommonSyntax(JsonValueContext input, String pathSpec) {
+    try {
+      Matcher matcher = JSON_PATH_BASE.matcher(pathSpec);
+      if (!matcher.matches()) {
+        throw RESOURCE.illegalJsonPathSpec(pathSpec).ex();
+      }
+      PathMode mode = PathMode.valueOf(matcher.group(1).toUpperCase(Locale.ROOT));
+      String pathWff = matcher.group(2);
+      DocumentContext ctx;
+      switch (mode) {
+      case STRICT:
+        if (input.hasException()) {
+          return JsonPathContext.withStrictException(input.exc);
+        }
+        ctx = JsonPath.parse(input.obj,
+            Configuration
+                .builder()
+                .jsonProvider(JSON_PATH_JSON_PROVIDER)
+                .mappingProvider(JSON_PATH_MAPPING_PROVIDER)
+                .build());
+        break;
+      case LAX:
+        if (input.hasException()) {
+          return JsonPathContext.withJavaObj(PathMode.LAX, null);
+        }
+        ctx = JsonPath.parse(input.obj,
+            Configuration
+                .builder()
+                .options(Option.SUPPRESS_EXCEPTIONS)
+                .jsonProvider(JSON_PATH_JSON_PROVIDER)
+                .mappingProvider(JSON_PATH_MAPPING_PROVIDER)
+                .build());
+        break;
+      default:
+        throw RESOURCE.illegalJsonPathModeInPathSpec(mode.toString(), pathSpec).ex();
+      }
+      try {
+        return JsonPathContext.withJavaObj(mode, ctx.read(pathWff));
+      } catch (Exception e) {
+        return JsonPathContext.withStrictException(e);
+      }
+    } catch (Exception e) {
+      return JsonPathContext.withUnknownException(e);
+    }
+  }
+
+  public static Boolean jsonExists(String input, String pathSpec) {
+    return jsonExists(jsonApiCommonSyntax(input, pathSpec));
+  }
+
+  public static Boolean jsonExists(String input, String pathSpec,
+      SqlJsonExistsErrorBehavior errorBehavior) {
+    return jsonExists(jsonApiCommonSyntax(input, pathSpec), errorBehavior);
+  }
+
+  public static Boolean jsonExists(JsonValueContext input, String pathSpec) {
+    return jsonExists(jsonApiCommonSyntax(input, pathSpec));
+  }
+
+  public static Boolean jsonExists(JsonValueContext input, String pathSpec,
+      SqlJsonExistsErrorBehavior errorBehavior) {
+    return jsonExists(jsonApiCommonSyntax(input, pathSpec), errorBehavior);
+  }
+
+  public static Boolean jsonExists(JsonPathContext context) {
+    return jsonExists(context, SqlJsonExistsErrorBehavior.FALSE);
+  }
+
+  public static Boolean jsonExists(JsonPathContext context,
+      SqlJsonExistsErrorBehavior errorBehavior) {
+    if (context.hasException()) {
+      switch (errorBehavior) {
+      case TRUE:
+        return Boolean.TRUE;
+      case FALSE:
+        return Boolean.FALSE;
+      case ERROR:
+        throw toUnchecked(context.exc);
+      case UNKNOWN:
+        return null;
+      default:
+        throw RESOURCE.illegalErrorBehaviorInJsonExistsFunc(
+            errorBehavior.toString()).ex();
+      }
+    } else {
+      return context.obj != null;
+    }
+  }
+
+  public static Object jsonValueAny(String input,
+      String pathSpec,
+      SqlJsonValueEmptyOrErrorBehavior emptyBehavior,
+      Object defaultValueOnEmpty,
+      SqlJsonValueEmptyOrErrorBehavior errorBehavior,
+      Object defaultValueOnError) {
+    return jsonValueAny(
+        jsonApiCommonSyntax(input, pathSpec),
+        emptyBehavior,
+        defaultValueOnEmpty,
+        errorBehavior,
+        defaultValueOnError);
+  }
+
+  public static Object jsonValueAny(JsonValueContext input,
+      String pathSpec,
+      SqlJsonValueEmptyOrErrorBehavior emptyBehavior,
+      Object defaultValueOnEmpty,
+      SqlJsonValueEmptyOrErrorBehavior errorBehavior,
+      Object defaultValueOnError) {
+    return jsonValueAny(
+        jsonApiCommonSyntax(input, pathSpec),
+        emptyBehavior,
+        defaultValueOnEmpty,
+        errorBehavior,
+        defaultValueOnError);
+  }
+
+  public static Object jsonValueAny(JsonPathContext context,
+      SqlJsonValueEmptyOrErrorBehavior emptyBehavior,
+      Object defaultValueOnEmpty,
+      SqlJsonValueEmptyOrErrorBehavior errorBehavior,
+      Object defaultValueOnError) {
+    final Exception exc;
+    if (context.hasException()) {
+      exc = context.exc;
+    } else {
+      Object value = context.obj;
+      if (value == null || context.mode == PathMode.LAX
+          && !isScalarObject(value)) {
+        switch (emptyBehavior) {
+        case ERROR:
+          throw RESOURCE.emptyResultOfJsonValueFuncNotAllowed().ex();
+        case NULL:
+          return null;
+        case DEFAULT:
+          return defaultValueOnEmpty;
+        default:
+          throw RESOURCE.illegalEmptyBehaviorInJsonValueFunc(
+              emptyBehavior.toString()).ex();
+        }
+      } else if (context.mode == PathMode.STRICT
+          && !isScalarObject(value)) {
+        exc = RESOURCE.scalarValueRequiredInStrictModeOfJsonValueFunc(
+            value.toString()).ex();
+      } else {
+        return value;
+      }
+    }
+    switch (errorBehavior) {
+    case ERROR:
+      throw toUnchecked(exc);
+    case NULL:
+      return null;
+    case DEFAULT:
+      return defaultValueOnError;
+    default:
+      throw RESOURCE.illegalErrorBehaviorInJsonValueFunc(
+          errorBehavior.toString()).ex();
+    }
+  }
+
+  public static String jsonQuery(String input,
+      String pathSpec,
+      SqlJsonQueryWrapperBehavior wrapperBehavior,
+      SqlJsonQueryEmptyOrErrorBehavior emptyBehavior,
+      SqlJsonQueryEmptyOrErrorBehavior errorBehavior) {
+    return jsonQuery(
+        jsonApiCommonSyntax(input, pathSpec),
+        wrapperBehavior, emptyBehavior, errorBehavior);
+  }
+
+  public static String jsonQuery(JsonValueContext input,
+      String pathSpec,
+      SqlJsonQueryWrapperBehavior wrapperBehavior,
+      SqlJsonQueryEmptyOrErrorBehavior emptyBehavior,
+      SqlJsonQueryEmptyOrErrorBehavior errorBehavior) {
+    return jsonQuery(
+        jsonApiCommonSyntax(input, pathSpec),
+        wrapperBehavior, emptyBehavior, errorBehavior);
+  }
+
+  public static String jsonQuery(JsonPathContext context,
+      SqlJsonQueryWrapperBehavior wrapperBehavior,
+      SqlJsonQueryEmptyOrErrorBehavior emptyBehavior,
+      SqlJsonQueryEmptyOrErrorBehavior errorBehavior) {
+    final Exception exc;
+    if (context.hasException()) {
+      exc = context.exc;
+    } else {
+      Object value;
+      if (context.obj == null) {
+        value = null;
+      } else {
+        switch (wrapperBehavior) {
+        case WITHOUT_ARRAY:
+          value = context.obj;
+          break;
+        case WITH_UNCONDITIONAL_ARRAY:
+          value = Collections.singletonList(context.obj);
+          break;
+        case WITH_CONDITIONAL_ARRAY:
+          if (context.obj instanceof Collection) {
+            value = context.obj;
+          } else {
+            value = Collections.singletonList(context.obj);
+          }
+          break;
+        default:
+          throw RESOURCE.illegalWrapperBehaviorInJsonQueryFunc(
+              wrapperBehavior.toString()).ex();
+        }
+      }
+      if (value == null || context.mode == PathMode.LAX
+          && isScalarObject(value)) {
+        switch (emptyBehavior) {
+        case ERROR:
+          throw RESOURCE.emptyResultOfJsonQueryFuncNotAllowed().ex();
+        case NULL:
+          return null;
+        case EMPTY_ARRAY:
+          return "[]";
+        case EMPTY_OBJECT:
+          return "{}";
+        default:
+          throw RESOURCE.illegalEmptyBehaviorInJsonQueryFunc(
+              emptyBehavior.toString()).ex();
+        }
+      } else if (context.mode == PathMode.STRICT && isScalarObject(value)) {
+        exc = RESOURCE.arrayOrObjectValueRequiredInStrictModeOfJsonQueryFunc(
+            value.toString()).ex();
+      } else {
+        try {
+          return jsonize(value);
+        } catch (Exception e) {
+          exc = e;
+        }
+      }
+    }
+    switch (errorBehavior) {
+    case ERROR:
+      throw toUnchecked(exc);
+    case NULL:
+      return null;
+    case EMPTY_ARRAY:
+      return "[]";
+    case EMPTY_OBJECT:
+      return "{}";
+    default:
+      throw RESOURCE.illegalErrorBehaviorInJsonQueryFunc(
+          errorBehavior.toString()).ex();
+    }
+  }
+
+  public static String jsonObject(SqlJsonConstructorNullClause nullClause,
+      Object... kvs) {
+    assert kvs.length % 2 == 0;
+    Map<String, Object> map = new HashMap<>();
+    for (int i = 0; i < kvs.length; i += 2) {
+      String k = (String) kvs[i];
+      Object v = kvs[i + 1];
+      if (k == null) {
+        throw RESOURCE.nullKeyOfJsonObjectNotAllowed().ex();
+      }
+      if (v == null) {
+        if (nullClause == SqlJsonConstructorNullClause.NULL_ON_NULL) {
+          map.put(k, null);
+        }
+      } else {
+        map.put(k, v);
+      }
+    }
+    return jsonize(map);
+  }
+
+  public static void jsonObjectAggAdd(Map map, String k, Object v,
+      SqlJsonConstructorNullClause nullClause) {
+    if (k == null) {
+      throw RESOURCE.nullKeyOfJsonObjectNotAllowed().ex();
+    }
+    if (v == null) {
+      if (nullClause == SqlJsonConstructorNullClause.NULL_ON_NULL) {
+        map.put(k, null);
+      }
+    } else {
+      map.put(k, v);
+    }
+  }
+
+  public static String jsonArray(SqlJsonConstructorNullClause nullClause,
+      Object... elements) {
+    List<Object> list = new ArrayList<>();
+    for (Object element : elements) {
+      if (element == null) {
+        if (nullClause == SqlJsonConstructorNullClause.NULL_ON_NULL) {
+          list.add(null);
+        }
+      } else {
+        list.add(element);
+      }
+    }
+    return jsonize(list);
+  }
+
+  public static void jsonArrayAggAdd(List list, Object element,
+      SqlJsonConstructorNullClause nullClause) {
+    if (element == null) {
+      if (nullClause == SqlJsonConstructorNullClause.NULL_ON_NULL) {
+        list.add(null);
+      }
+    } else {
+      list.add(element);
+    }
+  }
+
+  public static String jsonPretty(String input) {
+    return jsonPretty(jsonValueExpression(input));
+  }
+
+  public static String jsonPretty(JsonValueContext input) {
+    try {
+      return JSON_PATH_JSON_PROVIDER.getObjectMapper().writer(JSON_PRETTY_PRINTER)
+          .writeValueAsString(input.obj);
+    } catch (Exception e) {
+      throw RESOURCE.exceptionWhileSerializingToJson(Objects.toString(input.obj)).ex();
+    }
+  }
+
+  public static String jsonType(String input) {
+    return jsonType(jsonValueExpression(input));
+  }
+
+  public static String jsonType(JsonValueContext input) {
+    final String result;
+    final Object val = input.obj;
+    try {
+      if (val instanceof Integer) {
+        result = "INTEGER";
+      } else if (val instanceof String) {
+        result = "STRING";
+      } else if (val instanceof Float) {
+        result = "FLOAT";
+      } else if (val instanceof Double) {
+        result = "DOUBLE";
+      } else if (val instanceof Long) {
+        result = "LONG";
+      } else if (val instanceof Boolean) {
+        result = "BOOLEAN";
+      } else if (val instanceof Date) {
+        result = "DATE";
+      } else if (val instanceof Map) {
+        result = "OBJECT";
+      } else if (val instanceof Collection) {
+        result = "ARRAY";
+      } else if (val == null) {
+        result = "NULL";
+      } else {
+        throw RESOURCE.invalidInputForJsonType(val.toString()).ex();
+      }
+      return result;
+    } catch (Exception ex) {
+      throw RESOURCE.invalidInputForJsonType(val.toString()).ex();
+    }
+  }
+
+  public static Integer jsonDepth(String input) {
+    return jsonDepth(jsonValueExpression(input));
+  }
+
+  public static Integer jsonDepth(JsonValueContext input) {
+    final Integer result;
+    final Object o = input.obj;
+    try {
+      if (o == null) {
+        result = null;
+      } else {
+        result = calculateDepth(o);
+      }
+      return result;
+    } catch (Exception ex) {
+      throw RESOURCE.invalidInputForJsonDepth(o.toString()).ex();
+    }
+  }
+
+  private static Integer calculateDepth(Object o) {
+    if (isScalarObject(o)) {
+      return 1;
+    }
+    Queue<Object> q = new LinkedList<>();
+    int depth = 0;
+    q.add(o);
+
+    while (!q.isEmpty()) {
+      int size = q.size();
+      for (int i = 0; i < size; ++i) {
+        Object obj = q.poll();
+        if (obj instanceof Map) {
+          for (Object value : ((LinkedHashMap) obj).values()) {
+            q.add(value);
+          }
+        } else if (obj instanceof Collection) {
+          for (Object value : (Collection) obj) {
+            q.add(value);
+          }
+        }
+      }
+      ++depth;
+    }
+    return depth;
+  }
+
+  public static Integer jsonLength(String input) {
+    return jsonLength(jsonApiCommonSyntax(input));
+  }
+
+  public static Integer jsonLength(JsonValueContext input) {
+    return jsonLength(jsonApiCommonSyntax(input));
+  }
+
+  public static Integer jsonLength(String input, String pathSpec) {
+    return jsonLength(jsonApiCommonSyntax(input, pathSpec));
+  }
+
+  public static Integer jsonLength(JsonValueContext input, String pathSpec) {
+    return jsonLength(jsonApiCommonSyntax(input, pathSpec));
+  }
+
+  public static Integer jsonLength(JsonPathContext context) {
+    final Integer result;
+    final Object value;
+    try {
+      if (context.hasException()) {
+        throw toUnchecked(context.exc);
+      }
+      value = context.obj;
+
+      if (value == null) {
+        result = null;
+      } else {
+        if (value instanceof Collection) {
+          result = ((Collection) value).size();
+        } else if (value instanceof Map) {
+          result = ((LinkedHashMap) value).size();
+        } else if (isScalarObject(value)) {
+          result = 1;
+        } else {
+          result = 0;
+        }
+      }
+    } catch (Exception ex) {
+      throw RESOURCE.invalidInputForJsonLength(
+          context.toString()).ex();
+    }
+    return result;
+  }
+
+  public static String jsonKeys(String input) {
+    return jsonKeys(jsonApiCommonSyntax(input));
+  }
+
+  public static String jsonKeys(JsonValueContext input) {
+    return jsonKeys(jsonApiCommonSyntax(input));
+  }
+
+  public static String jsonKeys(String input, String pathSpec) {
+    return jsonKeys(jsonApiCommonSyntax(input, pathSpec));
+  }
+
+  public static String jsonKeys(JsonValueContext input, String pathSpec) {
+    return jsonKeys(jsonApiCommonSyntax(input, pathSpec));
+  }
+
+  public static String jsonKeys(JsonPathContext context) {
+    List<String> list = new ArrayList<>();
+    final Object value;
+    try {
+      if (context.hasException()) {
+        throw toUnchecked(context.exc);
+      }
+      value = context.obj;
+
+      if ((value == null) || (value instanceof Collection)
+          || isScalarObject(value)) {
+        list = null;
+      } else if (value instanceof Map) {
+        for (Object key : ((LinkedHashMap) value).keySet()) {
+          list.add(key.toString());
+        }
+      }
+    } catch (Exception ex) {
+      throw RESOURCE.invalidInputForJsonKeys(
+          context.toString()).ex();
+    }
+    return jsonize(list);
+  }
+
+  public static boolean isJsonValue(String input) {
+    try {
+      dejsonize(input);
+      return true;
+    } catch (Exception e) {
+      return false;
+    }
+  }
+
+  public static boolean isJsonObject(String input) {
+    try {
+      Object o = dejsonize(input);
+      return o instanceof Map;
+    } catch (Exception e) {
+      return false;
+    }
+  }
+
+  public static boolean isJsonArray(String input) {
+    try {
+      Object o = dejsonize(input);
+      return o instanceof Collection;
+    } catch (Exception e) {
+      return false;
+    }
+  }
+
+  public static boolean isJsonScalar(String input) {
+    try {
+      Object o = dejsonize(input);
+      return !(o instanceof Map) && !(o instanceof Collection);
+    } catch (Exception e) {
+      return false;
+    }
+  }
+
+  private static RuntimeException toUnchecked(Exception e) {
+    return Util.toUnchecked(e);
+  }
+
+  /**
+   * Returned path context of JsonApiCommonSyntax, public for testing.
+   */
+  public static class JsonPathContext {
+    public final PathMode mode;
+    public final Object obj;
+    public final Exception exc;
+
+    private JsonPathContext(Object obj, Exception exc) {
+      this(PathMode.NONE, obj, exc);
+    }
+
+    private JsonPathContext(PathMode mode, Object obj, Exception exc) {
+      assert obj == null || exc == null;
+      this.mode = mode;
+      this.obj = obj;
+      this.exc = exc;
+    }
+
+    public boolean hasException() {
+      return exc != null;
+    }
+
+    public static JsonPathContext withUnknownException(Exception exc) {
+      return new JsonPathContext(PathMode.UNKNOWN, null, exc);
+    }
+
+    public static JsonPathContext withStrictException(Exception exc) {
+      return new JsonPathContext(PathMode.STRICT, null, exc);
+    }
+
+    public static JsonPathContext withJavaObj(PathMode mode, Object obj) {
+      if (mode == PathMode.UNKNOWN) {
+        throw RESOURCE.illegalJsonPathMode(mode.toString()).ex();
+      }
+      if (mode == PathMode.STRICT && obj == null) {
+        throw RESOURCE.strictPathModeRequiresNonEmptyValue().ex();
+      }
+      return new JsonPathContext(mode, obj, null);
+    }
+
+    @Override public String toString() {
+      return "JsonPathContext{"
+          + "mode=" + mode
+          + ", obj=" + obj
+          + ", exc=" + exc
+          + '}';
+    }
+  }
+
+  /**
+   * The Java output of {@link org.apache.calcite.sql.fun.SqlJsonValueExpressionOperator}.
+   */
+  public static class JsonValueContext {
+    @JsonValue
+    public final Object obj;
+    public final Exception exc;
+
+    private JsonValueContext(Object obj, Exception exc) {
+      assert obj == null || exc == null;
+      this.obj = obj;
+      this.exc = exc;
+    }
+
+    public static JsonValueContext withJavaObj(Object obj) {
+      return new JsonValueContext(obj, null);
+    }
+
+    public static JsonValueContext withException(Exception exc) {
+      return new JsonValueContext(null, exc);
+    }
+
+    public boolean hasException() {
+      return exc != null;
+    }
+
+    @Override public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      JsonValueContext jsonValueContext = (JsonValueContext) o;
+      return Objects.equals(obj, jsonValueContext.obj);
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(obj);
+    }
+
+    @Override public String toString() {
+      return Objects.toString(obj);
+    }
+  }
+
+  /**
+   * Path spec has two different modes: lax mode and strict mode.
+   * Lax mode suppresses any thrown exception and returns null,
+   * whereas strict mode throws exceptions.
+   */
+  public enum PathMode {
+    LAX,
+    STRICT,
+    UNKNOWN,
+    NONE
+  }
+}
+
+// End JsonFunctions.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 aa3d7d4..99fd3ff 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -33,26 +33,11 @@ import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.function.NonDeterministic;
 import org.apache.calcite.linq4j.tree.Primitive;
 import org.apache.calcite.runtime.FlatLists.ComparableList;
-import org.apache.calcite.sql.SqlJsonConstructorNullClause;
-import org.apache.calcite.sql.SqlJsonExistsErrorBehavior;
-import org.apache.calcite.sql.SqlJsonQueryEmptyOrErrorBehavior;
-import org.apache.calcite.sql.SqlJsonQueryWrapperBehavior;
-import org.apache.calcite.sql.SqlJsonValueEmptyOrErrorBehavior;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.NumberUtil;
 import org.apache.calcite.util.TimeWithTimeZoneString;
 import org.apache.calcite.util.TimestampWithTimeZoneString;
-
-import com.fasterxml.jackson.core.PrettyPrinter;
-import com.fasterxml.jackson.core.util.DefaultIndenter;
-import com.fasterxml.jackson.core.util.DefaultPrettyPrinter;
-import com.jayway.jsonpath.Configuration;
-import com.jayway.jsonpath.DocumentContext;
-import com.jayway.jsonpath.JsonPath;
-import com.jayway.jsonpath.Option;
-import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
-import com.jayway.jsonpath.spi.mapper.JacksonMappingProvider;
-import com.jayway.jsonpath.spi.mapper.MappingProvider;
+import org.apache.calcite.util.Util;
 
 import java.lang.reflect.Field;
 import java.math.BigDecimal;
@@ -65,21 +50,17 @@ import java.text.DecimalFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Queue;
 import java.util.Set;
 import java.util.TimeZone;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -129,18 +110,6 @@ public class SqlFunctions {
   private static final ThreadLocal<Map<String, AtomicLong>> THREAD_SEQUENCES =
       ThreadLocal.withInitial(HashMap::new);
 
-  private static final Pattern JSON_PATH_BASE =
-      Pattern.compile("^\\s*(?<mode>strict|lax)\\s+(?<spec>.+)$",
-          Pattern.CASE_INSENSITIVE | Pattern.DOTALL | Pattern.MULTILINE);
-
-  private static final JacksonJsonProvider JSON_PATH_JSON_PROVIDER =
-      new JacksonJsonProvider();
-  private static final MappingProvider JSON_PATH_MAPPING_PROVIDER =
-      new JacksonMappingProvider();
-  private static final PrettyPrinter JSON_PRETTY_PRINTER =
-      new DefaultPrettyPrinter().withObjectIndenter(
-          DefaultIndenter.SYSTEM_LINEFEED_INSTANCE.withLinefeed("\n"));
-
   private SqlFunctions() {
   }
 
@@ -2101,7 +2070,7 @@ public class SqlFunctions {
     try {
       return Primitive.asList(a.getArray());
     } catch (SQLException e) {
-      throw toUnchecked(e);
+      throw Util.toUnchecked(e);
     }
   }
 
@@ -2437,551 +2406,6 @@ public class SqlFunctions {
     }
   }
 
-
-  private static boolean isScalarObject(Object obj) {
-    if (obj instanceof Collection) {
-      return false;
-    }
-    if (obj instanceof Map) {
-      return false;
-    }
-    return true;
-  }
-
-  public static Object jsonValueExpression(String input) {
-    try {
-      return dejsonize(input);
-    } catch (Exception e) {
-      return e;
-    }
-  }
-
-  public static PathContext jsonApiCommonSyntax(Object input) {
-    return jsonApiCommonSyntax(input, "strict $");
-  }
-
-  public static PathContext jsonApiCommonSyntax(Object input, String pathSpec) {
-    try {
-      Matcher matcher = JSON_PATH_BASE.matcher(pathSpec);
-      if (!matcher.matches()) {
-        throw RESOURCE.illegalJsonPathSpec(pathSpec).ex();
-      }
-      PathMode mode = PathMode.valueOf(matcher.group(1).toUpperCase(Locale.ROOT));
-      String pathWff = matcher.group(2);
-      DocumentContext ctx;
-      switch (mode) {
-      case STRICT:
-        if (input instanceof Exception) {
-          return PathContext.withStrictException((Exception) input);
-        }
-        ctx = JsonPath.parse(input,
-            Configuration
-                .builder()
-                .jsonProvider(JSON_PATH_JSON_PROVIDER)
-                .mappingProvider(JSON_PATH_MAPPING_PROVIDER)
-                .build());
-        break;
-      case LAX:
-        if (input instanceof Exception) {
-          return PathContext.withReturned(PathMode.LAX, null);
-        }
-        ctx = JsonPath.parse(input,
-            Configuration
-                .builder()
-                .options(Option.SUPPRESS_EXCEPTIONS)
-                .jsonProvider(JSON_PATH_JSON_PROVIDER)
-                .mappingProvider(JSON_PATH_MAPPING_PROVIDER)
-                .build());
-        break;
-      default:
-        throw RESOURCE.illegalJsonPathModeInPathSpec(mode.toString(), pathSpec).ex();
-      }
-      try {
-        return PathContext.withReturned(mode, ctx.read(pathWff));
-      } catch (Exception e) {
-        return PathContext.withStrictException(e);
-      }
-    } catch (Exception e) {
-      return PathContext.withUnknownException(e);
-    }
-  }
-
-  public static Boolean jsonExists(Object input) {
-    return jsonExists(input, SqlJsonExistsErrorBehavior.FALSE);
-  }
-
-  public static Boolean jsonExists(Object input,
-      SqlJsonExistsErrorBehavior errorBehavior) {
-    PathContext context = (PathContext) input;
-    if (context.exc != null) {
-      switch (errorBehavior) {
-      case TRUE:
-        return Boolean.TRUE;
-      case FALSE:
-        return Boolean.FALSE;
-      case ERROR:
-        throw toUnchecked(context.exc);
-      case UNKNOWN:
-        return null;
-      default:
-        throw RESOURCE.illegalErrorBehaviorInJsonExistsFunc(
-            errorBehavior.toString()).ex();
-      }
-    } else {
-      return context.pathReturned != null;
-    }
-  }
-
-  public static Object jsonValueAny(Object input,
-      SqlJsonValueEmptyOrErrorBehavior emptyBehavior,
-      Object defaultValueOnEmpty,
-      SqlJsonValueEmptyOrErrorBehavior errorBehavior,
-      Object defaultValueOnError) {
-    final PathContext context = (PathContext) input;
-    final Exception exc;
-    if (context.exc != null) {
-      exc = context.exc;
-    } else {
-      Object value = context.pathReturned;
-      if (value == null || context.mode == PathMode.LAX
-          && !isScalarObject(value)) {
-        switch (emptyBehavior) {
-        case ERROR:
-          throw RESOURCE.emptyResultOfJsonValueFuncNotAllowed().ex();
-        case NULL:
-          return null;
-        case DEFAULT:
-          return defaultValueOnEmpty;
-        default:
-          throw RESOURCE.illegalEmptyBehaviorInJsonValueFunc(
-              emptyBehavior.toString()).ex();
-        }
-      } else if (context.mode == PathMode.STRICT
-          && !isScalarObject(value)) {
-        exc = RESOURCE.scalarValueRequiredInStrictModeOfJsonValueFunc(
-            value.toString()).ex();
-      } else {
-        return value;
-      }
-    }
-    switch (errorBehavior) {
-    case ERROR:
-      throw toUnchecked(exc);
-    case NULL:
-      return null;
-    case DEFAULT:
-      return defaultValueOnError;
-    default:
-      throw RESOURCE.illegalErrorBehaviorInJsonValueFunc(
-          errorBehavior.toString()).ex();
-    }
-  }
-
-  public static String jsonQuery(Object input,
-      SqlJsonQueryWrapperBehavior wrapperBehavior,
-      SqlJsonQueryEmptyOrErrorBehavior emptyBehavior,
-      SqlJsonQueryEmptyOrErrorBehavior errorBehavior) {
-    final PathContext context = (PathContext) input;
-    final Exception exc;
-    if (context.exc != null) {
-      exc = context.exc;
-    } else {
-      Object value;
-      if (context.pathReturned == null) {
-        value = null;
-      } else {
-        switch (wrapperBehavior) {
-        case WITHOUT_ARRAY:
-          value = context.pathReturned;
-          break;
-        case WITH_UNCONDITIONAL_ARRAY:
-          value = Collections.singletonList(context.pathReturned);
-          break;
-        case WITH_CONDITIONAL_ARRAY:
-          if (context.pathReturned instanceof Collection) {
-            value = context.pathReturned;
-          } else {
-            value = Collections.singletonList(context.pathReturned);
-          }
-          break;
-        default:
-          throw RESOURCE.illegalWrapperBehaviorInJsonQueryFunc(
-              wrapperBehavior.toString()).ex();
-        }
-      }
-      if (value == null || context.mode == PathMode.LAX
-          && isScalarObject(value)) {
-        switch (emptyBehavior) {
-        case ERROR:
-          throw RESOURCE.emptyResultOfJsonQueryFuncNotAllowed().ex();
-        case NULL:
-          return null;
-        case EMPTY_ARRAY:
-          return "[]";
-        case EMPTY_OBJECT:
-          return "{}";
-        default:
-          throw RESOURCE.illegalEmptyBehaviorInJsonQueryFunc(
-              emptyBehavior.toString()).ex();
-        }
-      } else if (context.mode == PathMode.STRICT && isScalarObject(value)) {
-        exc = RESOURCE.arrayOrObjectValueRequiredInStrictModeOfJsonQueryFunc(
-            value.toString()).ex();
-      } else {
-        try {
-          return jsonize(value);
-        } catch (Exception e) {
-          exc = e;
-        }
-      }
-    }
-    switch (errorBehavior) {
-    case ERROR:
-      throw toUnchecked(exc);
-    case NULL:
-      return null;
-    case EMPTY_ARRAY:
-      return "[]";
-    case EMPTY_OBJECT:
-      return "{}";
-    default:
-      throw RESOURCE.illegalErrorBehaviorInJsonQueryFunc(
-          errorBehavior.toString()).ex();
-    }
-  }
-
-  public static String jsonize(Object input) {
-    return JSON_PATH_JSON_PROVIDER.toJson(input);
-  }
-
-  public static Object dejsonize(String input) {
-    return JSON_PATH_JSON_PROVIDER.parse(input);
-  }
-
-  public static String jsonObject(SqlJsonConstructorNullClause nullClause,
-      Object... kvs) {
-    assert kvs.length % 2 == 0;
-    Map<String, Object> map = new HashMap<>();
-    for (int i = 0; i < kvs.length; i += 2) {
-      String k = (String) kvs[i];
-      Object v = kvs[i + 1];
-      if (k == null) {
-        throw RESOURCE.nullKeyOfJsonObjectNotAllowed().ex();
-      }
-      if (v == null) {
-        if (nullClause == SqlJsonConstructorNullClause.NULL_ON_NULL) {
-          map.put(k, null);
-        }
-      } else {
-        map.put(k, v);
-      }
-    }
-    return jsonize(map);
-  }
-
-  public static void jsonObjectAggAdd(Map map, String k, Object v,
-      SqlJsonConstructorNullClause nullClause) {
-    if (k == null) {
-      throw RESOURCE.nullKeyOfJsonObjectNotAllowed().ex();
-    }
-    if (v == null) {
-      if (nullClause == SqlJsonConstructorNullClause.NULL_ON_NULL) {
-        map.put(k, null);
-      }
-    } else {
-      map.put(k, v);
-    }
-  }
-
-  public static String jsonArray(SqlJsonConstructorNullClause nullClause,
-      Object... elements) {
-    List<Object> list = new ArrayList<>();
-    for (Object element : elements) {
-      if (element == null) {
-        if (nullClause == SqlJsonConstructorNullClause.NULL_ON_NULL) {
-          list.add(null);
-        }
-      } else {
-        list.add(element);
-      }
-    }
-    return jsonize(list);
-  }
-
-  public static void jsonArrayAggAdd(List list, Object element,
-      SqlJsonConstructorNullClause nullClause) {
-    if (element == null) {
-      if (nullClause == SqlJsonConstructorNullClause.NULL_ON_NULL) {
-        list.add(null);
-      }
-    } else {
-      list.add(element);
-    }
-  }
-
-  public static String jsonPretty(Object input) {
-    try {
-      return JSON_PATH_JSON_PROVIDER.getObjectMapper().writer(JSON_PRETTY_PRINTER)
-          .writeValueAsString(input);
-    } catch (Exception e) {
-      throw RESOURCE.exceptionWhileSerializingToJson(input.toString()).ex();
-    }
-  }
-
-  public static String jsonType(Object o) {
-    final String result;
-    try {
-      if (o instanceof Integer) {
-        result = "INTEGER";
-      } else if (o instanceof String) {
-        result = "STRING";
-      } else if (o instanceof Float) {
-        result = "FLOAT";
-      } else if (o instanceof Double) {
-        result = "DOUBLE";
-      } else if (o instanceof Long) {
-        result = "LONG";
-      } else if (o instanceof Boolean) {
-        result = "BOOLEAN";
-      } else if (o instanceof Date) {
-        result = "DATE";
-      } else if (o instanceof Map) {
-        result = "OBJECT";
-      } else if (o instanceof Collection) {
-        result = "ARRAY";
-      } else if (o == null) {
-        result = "NULL";
-      } else {
-        throw RESOURCE.invalidInputForJsonType(o.toString()).ex();
-      }
-      return result;
-    } catch (Exception ex) {
-      throw RESOURCE.invalidInputForJsonType(o.toString()).ex();
-    }
-  }
-
-  public static Integer jsonDepth(Object o) {
-    final Integer result;
-    try {
-      if (o == null) {
-        result = null;
-      } else {
-        result = getJsonDepth(o);
-      }
-      return result;
-    } catch (Exception ex) {
-      throw RESOURCE.invalidInputForJsonDepth(o.toString()).ex();
-    }
-  }
-
-  private static Integer getJsonDepth(Object o) {
-    if (isScalarObject(o)) {
-      return 1;
-    }
-    Queue<Object> q = new LinkedList<>();
-    int depth = 0;
-    q.add(o);
-
-    while (!q.isEmpty()) {
-      int size = q.size();
-      for (int i = 0; i < size; ++i) {
-        Object obj = q.poll();
-        if (obj instanceof Map) {
-          for (Object value : ((LinkedHashMap) obj).values()) {
-            q.add(value);
-          }
-        } else if (obj instanceof Collection) {
-          for (Object value : (Collection) obj) {
-            q.add(value);
-          }
-        }
-      }
-      ++depth;
-    }
-
-    return depth;
-  }
-
-  public static Integer jsonLength(Object input) {
-    return jsonLength(jsonApiCommonSyntax(input));
-  }
-
-  public static Integer jsonLength(Object input, String pathSpec) {
-    return jsonLength(jsonApiCommonSyntax(input, pathSpec));
-  }
-
-  public static Integer jsonLength(PathContext context) {
-    final Integer result;
-    final Object value;
-    try {
-      if (context.exc != null) {
-        throw toUnchecked(context.exc);
-      }
-      value = context.pathReturned;
-
-      if (value == null) {
-        result = null;
-      } else {
-        if (value instanceof Collection) {
-          result = ((Collection) value).size();
-        } else if (value instanceof Map) {
-          result = ((LinkedHashMap) value).size();
-        } else if (isScalarObject(value)) {
-          result = 1;
-        } else {
-          result = 0;
-        }
-      }
-    } catch (Exception ex) {
-      throw RESOURCE.invalidInputForJsonLength(
-          context.toString()).ex();
-    }
-    return result;
-  }
-
-  public static String jsonKeys(Object input) {
-    return jsonKeys(jsonApiCommonSyntax(input));
-  }
-
-  public static String jsonKeys(Object input, String pathSpec) {
-    return jsonKeys(jsonApiCommonSyntax(input, pathSpec));
-  }
-
-  public static String jsonKeys(PathContext context) {
-    List<String> list = new ArrayList<>();
-    final Object value;
-    try {
-      if (context.exc != null) {
-        throw toUnchecked(context.exc);
-      }
-      value = context.pathReturned;
-
-      if ((value == null) || (value instanceof Collection)
-             || isScalarObject(value)) {
-        list = null;
-      } else if (value instanceof Map) {
-        for (Object key : ((LinkedHashMap) value).keySet()) {
-          list.add(key.toString());
-        }
-      }
-    } catch (Exception ex) {
-      throw RESOURCE.invalidInputForJsonKeys(
-              context.toString()).ex();
-    }
-    return jsonize(list);
-  }
-
-  public static boolean isJsonPathContext(Object input) {
-    try {
-      PathContext context = (PathContext) input;
-      return context != null;
-    } catch (Exception e) {
-      return false;
-    }
-  }
-
-  public static boolean isJsonValue(String input) {
-    try {
-      dejsonize(input);
-      return true;
-    } catch (Exception e) {
-      return false;
-    }
-  }
-
-  public static boolean isJsonObject(String input) {
-    try {
-      Object o = dejsonize(input);
-      return o instanceof Map;
-    } catch (Exception e) {
-      return false;
-    }
-  }
-
-  public static boolean isJsonArray(String input) {
-    try {
-      Object o = dejsonize(input);
-      return o instanceof Collection;
-    } catch (Exception e) {
-      return false;
-    }
-  }
-
-  public static boolean isJsonScalar(String input) {
-    try {
-      Object o = dejsonize(input);
-      return !(o instanceof Map) && !(o instanceof Collection);
-    } catch (Exception e) {
-      return false;
-    }
-  }
-
-  private static RuntimeException toUnchecked(Exception e) {
-    if (e instanceof RuntimeException) {
-      return (RuntimeException) e;
-    }
-    return new RuntimeException(e);
-  }
-
-  /**
-   * Returned path context of JsonApiCommonSyntax, public for testing.
-   */
-  public static class PathContext {
-    public final PathMode mode;
-    public final Object pathReturned;
-    public final Exception exc;
-
-    private PathContext(Object pathReturned, Exception exc) {
-      this.mode = PathMode.NONE;
-      this.pathReturned = pathReturned;
-      this.exc = exc;
-    }
-
-    private PathContext(PathMode mode, Object pathReturned, Exception exc) {
-      this.mode = mode;
-      this.pathReturned = pathReturned;
-      this.exc = exc;
-    }
-
-    public static PathContext withUnknownException(Exception exc) {
-      return new PathContext(PathMode.UNKNOWN, null, exc);
-    }
-
-    public static PathContext withStrictException(Exception exc) {
-      return new PathContext(PathMode.STRICT, null, exc);
-    }
-
-    public static PathContext withReturned(PathMode mode, Object pathReturned) {
-      if (mode == PathMode.UNKNOWN) {
-        throw RESOURCE.illegalJsonPathMode(mode.toString()).ex();
-      }
-      if (mode == PathMode.STRICT && pathReturned == null) {
-        throw RESOURCE.strictPathModeRequiresNonEmptyValue().ex();
-      }
-      return new PathContext(mode, pathReturned, null);
-    }
-
-    @Override public String toString() {
-      return "PathContext{"
-          + "mode=" + mode
-          + ", pathReturned=" + pathReturned
-          + ", exc=" + exc
-          + '}';
-    }
-  }
-
-  /**
-   * Path spec has two different modes: lax mode and strict mode.
-   * Lax mode suppresses any thrown exception and returns null,
-   * whereas strict mode throws exceptions.
-   */
-  public enum PathMode {
-    LAX,
-    STRICT,
-    UNKNOWN,
-    NONE
-  }
-
   /** Enumerates over the cartesian product of the given lists, returning
    * a comparable list for each row.
    *
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 9b836b9..756bd62 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -753,11 +753,6 @@ public enum SqlKind {
   JSON_VALUE_EXPRESSION,
 
   /**
-   * The JSON API common syntax.
-   */
-  JSON_API_COMMON_SYNTAX,
-
-  /**
    * The {@code JSON_ARRAYAGG} aggregate function.
    */
   JSON_ARRAYAGG,
@@ -1214,7 +1209,7 @@ public enum SqlKind {
                   LITERAL_CHAIN, JDBC_FN, PRECEDING, FOLLOWING, ORDER_BY,
                   NULLS_FIRST, NULLS_LAST, COLLECTION_TABLE, TABLESAMPLE,
                   VALUES, WITH, WITH_ITEM, SKIP_TO_FIRST, SKIP_TO_LAST,
-                  JSON_VALUE_EXPRESSION, JSON_API_COMMON_SYNTAX),
+                  JSON_VALUE_EXPRESSION),
               AGGREGATE, DML, DDL));
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonApiCommonSyntaxOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonApiCommonSyntaxOperator.java
deleted file mode 100644
index 7fdfe43..0000000
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonApiCommonSyntaxOperator.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.sql.fun;
-
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlKind;
-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.ReturnTypes;
-import org.apache.calcite.sql.type.SqlTypeFamily;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * The JSON API common syntax including a path specification, which is for
- * JSON querying and processing.
- */
-public class SqlJsonApiCommonSyntaxOperator extends SqlSpecialOperator {
-
-  public SqlJsonApiCommonSyntaxOperator(String name) {
-    super(name, SqlKind.JSON_API_COMMON_SYNTAX, 100, true,
-        ReturnTypes.explicit(SqlTypeName.ANY), null,
-        OperandTypes.or(OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.STRING),
-            OperandTypes.family(SqlTypeFamily.ANY)));
-  }
-
-  @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec,
-      int rightPrec) {
-    final boolean hasPath = call.operandCount() % 2 == 0;
-    SqlWriter.Frame frame = writer.startList(SqlWriter.FrameTypeEnum.SIMPLE);
-    call.operand(0).unparse(writer, 0, 0);
-    if (hasPath) {
-      // the case '{'foo':'bar'}', 'lax $.foo'
-      writer.sep(",", true);
-      call.operand(1).unparse(writer, 0, 0);
-    }
-    if (hasPath && call.operandCount() > 2
-        || !hasPath && call.operandCount() > 1) {
-      writer.keyword("PASSING");
-      for (int i = 2; i < call.getOperandList().size(); i += 2) {
-        call.operand(i).unparse(writer, 0, 0);
-        writer.keyword("AS");
-        call.operand(i + 1).unparse(writer, 0, 0);
-      }
-    }
-    writer.endFunCall(frame);
-  }
-}
-
-// End SqlJsonApiCommonSyntaxOperator.java
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonExistsFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonExistsFunction.java
index 84774b2..f23a897 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonExistsFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonExistsFunction.java
@@ -23,6 +23,8 @@ import org.apache.calcite.sql.SqlKind;
 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.type.SqlTypeTransforms;
 
 /**
  * The <code>JSON_EXISTS</code> function.
@@ -30,25 +32,29 @@ import org.apache.calcite.sql.type.ReturnTypes;
 public class SqlJsonExistsFunction extends SqlFunction {
   public SqlJsonExistsFunction() {
     super("JSON_EXISTS", SqlKind.OTHER_FUNCTION,
-        ReturnTypes.BOOLEAN_FORCE_NULLABLE, null,
-        OperandTypes.or(OperandTypes.ANY, OperandTypes.ANY_ANY),
+        ReturnTypes.cascade(ReturnTypes.BOOLEAN, SqlTypeTransforms.FORCE_NULLABLE), null,
+        OperandTypes.or(
+            OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.CHARACTER),
+            OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.CHARACTER, SqlTypeFamily.ANY)),
         SqlFunctionCategory.SYSTEM);
   }
 
   @Override public String getSignatureTemplate(int operandsCount) {
     assert operandsCount == 1 || operandsCount == 2;
     if (operandsCount == 1) {
-      return "{0}({1})";
+      return "{0}({1} {2})";
     }
-    return "{0}({1} {2} ON ERROR)";
+    return "{0}({1} {2} {3} 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);
-    if (call.operandCount() == 2) {
-      call.operand(1).unparse(writer, 0, 0);
+    writer.sep(",", true);
+    call.operand(1).unparse(writer, 0, 0);
+    if (call.operandCount() == 3) {
+      call.operand(2).unparse(writer, 0, 0);
       writer.keyword("ON ERROR");
     }
     writer.endFunCall(frame);
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonKeysFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonKeysFunction.java
index 17c4507..2d07549 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonKeysFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonKeysFunction.java
@@ -22,6 +22,7 @@ import org.apache.calcite.sql.SqlKind;
 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_KEYS</code> function.
@@ -29,7 +30,7 @@ import org.apache.calcite.sql.type.SqlTypeFamily;
 public class SqlJsonKeysFunction extends SqlFunction {
   public SqlJsonKeysFunction() {
     super("JSON_KEYS", SqlKind.OTHER_FUNCTION,
-          ReturnTypes.VARCHAR_2000,
+          ReturnTypes.cascade(ReturnTypes.VARCHAR_2000, SqlTypeTransforms.FORCE_NULLABLE),
           null,
           OperandTypes.or(OperandTypes.ANY,
               OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.CHARACTER)),
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonPrettyFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonPrettyFunction.java
index 5826eaa..da397cb 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonPrettyFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonPrettyFunction.java
@@ -28,6 +28,7 @@ import org.apache.calcite.sql.type.OperandTypes;
 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.SqlTypeTransforms;
 import org.apache.calcite.sql.validate.SqlValidator;
 
 /**
@@ -38,7 +39,7 @@ public class SqlJsonPrettyFunction extends SqlFunction {
   public SqlJsonPrettyFunction() {
     super("JSON_PRETTY",
         SqlKind.OTHER_FUNCTION,
-        ReturnTypes.VARCHAR_2000,
+        ReturnTypes.cascade(ReturnTypes.VARCHAR_2000, SqlTypeTransforms.FORCE_NULLABLE),
         null,
         OperandTypes.ANY,
         SqlFunctionCategory.SYSTEM);
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
index 91f9c55..fff1b26 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java
@@ -41,20 +41,22 @@ public class SqlJsonQueryFunction extends SqlFunction {
             SqlTypeTransforms.FORCE_NULLABLE),
         null,
         OperandTypes.family(SqlTypeFamily.ANY,
-            SqlTypeFamily.ANY, SqlTypeFamily.ANY, SqlTypeFamily.ANY),
+            SqlTypeFamily.CHARACTER, 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)";
+    return "{0}({1} {2} {3} WRAPPER {4} ON EMPTY {5} 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);
+    writer.sep(",", true);
+    call.operand(1).unparse(writer, 0, 0);
     final SqlJsonQueryWrapperBehavior wrapperBehavior =
-        getEnumValue(call.operand(1));
+        getEnumValue(call.operand(2));
     switch (wrapperBehavior) {
     case WITHOUT_ARRAY:
       writer.keyword("WITHOUT ARRAY");
@@ -69,24 +71,24 @@ public class SqlJsonQueryFunction extends SqlFunction {
       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 EMPTY");
+    unparseEmptyOrErrorBehavior(writer, getEnumValue(call.operand(4)));
     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);
+      operands[2] = SqlLiteral.createSymbol(SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY, pos);
     }
     if (operands[3] == null) {
       operands[3] = SqlLiteral.createSymbol(SqlJsonQueryEmptyOrErrorBehavior.NULL, pos);
     }
+    if (operands[4] == null) {
+      operands[4] = SqlLiteral.createSymbol(SqlJsonQueryEmptyOrErrorBehavior.NULL, pos);
+    }
     return super.createCall(functionQualifier, pos, operands);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonTypeFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonTypeFunction.java
index 1d03378..91d8f60 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonTypeFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonTypeFunction.java
@@ -29,6 +29,7 @@ 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.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeTransforms;
 import org.apache.calcite.sql.validate.SqlValidator;
 
 /**
@@ -38,7 +39,9 @@ public class SqlJsonTypeFunction extends SqlFunction {
   public SqlJsonTypeFunction() {
     super("JSON_TYPE",
         SqlKind.OTHER_FUNCTION,
-        ReturnTypes.explicit(SqlTypeName.VARCHAR, 20),
+        ReturnTypes.cascade(
+            ReturnTypes.explicit(SqlTypeName.VARCHAR, 20),
+            SqlTypeTransforms.FORCE_NULLABLE),
         null,
         OperandTypes.ANY,
         SqlFunctionCategory.SYSTEM);
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
index f99a09c..8d207f8 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueFunction.java
@@ -29,11 +29,14 @@ 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.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeTransforms;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlValidator;
 
@@ -49,14 +52,28 @@ public class SqlJsonValueFunction extends SqlFunction {
   private final boolean returnAny;
 
   public SqlJsonValueFunction(String name, boolean returnAny) {
-    super(name, SqlKind.OTHER_FUNCTION, null,
+    super(name, SqlKind.OTHER_FUNCTION,
+        ReturnTypes.cascade(
+            opBinding -> {
+              assert opBinding.getOperandCount() == 6
+                  || opBinding.getOperandCount() == 7;
+              RelDataType ret;
+              if (opBinding.getOperandCount() == 7) {
+                ret = opBinding.getOperandType(6);
+              } else {
+                ret = opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY);
+              }
+              return opBinding.getTypeFactory().createTypeWithNullability(ret, true);
+            }, SqlTypeTransforms.FORCE_NULLABLE),
         (callBinding, returnType, operandTypes) -> {
           RelDataTypeFactory typeFactory = callBinding.getTypeFactory();
-          for (int i = 0; i < operandTypes.length; ++i) {
-            operandTypes[i] = typeFactory.createSqlType(SqlTypeName.ANY);
-          }
+          operandTypes[3] = typeFactory.createSqlType(SqlTypeName.ANY);
+          operandTypes[5] = typeFactory.createSqlType(SqlTypeName.ANY);
         },
-        null, SqlFunctionCategory.SYSTEM);
+        OperandTypes.family(SqlTypeFamily.ANY,
+            SqlTypeFamily.CHARACTER, SqlTypeFamily.ANY, SqlTypeFamily.ANY, SqlTypeFamily.ANY,
+            SqlTypeFamily.ANY, SqlTypeFamily.ANY),
+        SqlFunctionCategory.SYSTEM);
     this.returnAny = returnAny;
   }
 
@@ -64,28 +81,31 @@ public class SqlJsonValueFunction extends SqlFunction {
       SqlParserPos pos, SqlNode... operands) {
     List<SqlNode> operandList = new ArrayList<>();
     operandList.add(operands[0]);
-    if (operands[1] == null) {
+    operandList.add(operands[1]);
+    if (operands[2] == null) {
+      // empty behavior
       operandList.add(
           SqlLiteral.createSymbol(SqlJsonValueEmptyOrErrorBehavior.NULL, pos));
       operandList.add(SqlLiteral.createNull(pos));
     } else {
-      operandList.add(operands[1]);
       operandList.add(operands[2]);
+      operandList.add(operands[3]);
     }
-    if (operands[3] == null) {
+    if (operands[4] == null) {
+      // error behavior
       operandList.add(
           SqlLiteral.createSymbol(SqlJsonValueEmptyOrErrorBehavior.NULL, pos));
       operandList.add(SqlLiteral.createNull(pos));
     } else {
-      operandList.add(operands[3]);
       operandList.add(operands[4]);
+      operandList.add(operands[5]);
     }
-    if (operands.length == 6 && operands[5] != null) {
+    if (operands.length == 7 && operands[6] != null) {
       if (returnAny) {
         throw new IllegalArgumentException(
             "illegal returning clause in json_value_any function");
       }
-      operandList.add(operands[5]);
+      operandList.add(operands[6]);
     } else if (!returnAny) {
       SqlDataTypeSpec defaultTypeSpec =
           new SqlDataTypeSpec(new SqlIdentifier("VARCHAR", pos), 2000, -1,
@@ -97,18 +117,18 @@ public class SqlJsonValueFunction extends SqlFunction {
   }
 
   @Override public SqlOperandCountRange getOperandCountRange() {
-    return SqlOperandCountRanges.between(5, 6);
+    return SqlOperandCountRanges.between(6, 7);
   }
 
   @Override public boolean checkOperandTypes(SqlCallBinding callBinding,
       boolean throwOnFailure) {
     final SqlValidator validator = callBinding.getValidator();
     RelDataType defaultValueOnEmptyType =
-        validator.getValidatedNodeType(callBinding.operand(2));
+        validator.getValidatedNodeType(callBinding.operand(3));
     RelDataType defaultValueOnErrorType =
-        validator.getValidatedNodeType(callBinding.operand(4));
+        validator.getValidatedNodeType(callBinding.operand(5));
     RelDataType returnType =
-        validator.deriveType(callBinding.getScope(), callBinding.operand(5));
+        validator.deriveType(callBinding.getScope(), callBinding.operand(6));
     if (!canCastFrom(callBinding, throwOnFailure, defaultValueOnEmptyType,
         returnType)) {
       return false;
@@ -117,46 +137,36 @@ public class SqlJsonValueFunction extends SqlFunction {
         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);
+    return super.checkOperandTypes(callBinding, throwOnFailure);
   }
 
   @Override public String getSignatureTemplate(int operandsCount) {
-    assert operandsCount == 5 || operandsCount == 6;
-    if (operandsCount == 6) {
+    assert operandsCount == 6 || operandsCount == 7;
+    if (operandsCount == 7) {
       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;
+    assert call.operandCount() == 6 || call.operandCount() == 7;
     final SqlWriter.Frame frame = writer.startFunCall(getName());
     call.operand(0).unparse(writer, 0, 0);
+    writer.sep(",", true);
+    call.operand(1).unparse(writer, 0, 0);
     if (!returnAny) {
       writer.keyword("RETURNING");
-      call.operand(5).unparse(writer, 0, 0);
+      call.operand(6).unparse(writer, 0, 0);
     }
-    unparseEnum(writer, call.operand(1));
-    if (isDefaultLiteral(call.operand(1))) {
-      call.operand(2).unparse(writer, 0, 0);
+    unparseEnum(writer, call.operand(2));
+    if (isDefaultLiteral(call.operand(2))) {
+      call.operand(3).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);
+    unparseEnum(writer, call.operand(4));
+    if (isDefaultLiteral(call.operand(4))) {
+      call.operand(5).unparse(writer, 0, 0);
     }
     writer.keyword("ON");
     writer.keyword("ERROR");
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 f4b61b6..3bb0640 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
@@ -1294,9 +1294,6 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
 
   public static final SqlThrowOperator THROW = new SqlThrowOperator();
 
-  public static final SqlJsonApiCommonSyntaxOperator JSON_API_COMMON_SYNTAX =
-      new SqlJsonApiCommonSyntaxOperator("JSON_API_COMMON_SYNTAX");
-
   public static final SqlFunction JSON_EXISTS = new SqlJsonExistsFunction();
 
   public static final SqlFunction JSON_VALUE =
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 23359fc..c5b0b7d 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -45,7 +45,6 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlNumericLiteral;
 import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlPostfixOperator;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.fun.SqlArrayValueConstructor;
 import org.apache.calcite.sql.fun.SqlBetweenOperator;
@@ -78,12 +77,9 @@ import com.google.common.collect.Lists;
 import java.math.BigDecimal;
 import java.math.RoundingMode;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 
-import static org.apache.calcite.util.Static.RESOURCE;
-
 /**
  * Standard implementation of {@link SqlRexConvertletTable}.
  */
@@ -213,19 +209,6 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
                 call.operand(0),
                 SqlLiteral.createExactNumeric("0.5", SqlParserPos.ZERO))));
 
-    registerOp(SqlStdOperatorTable.JSON_API_COMMON_SYNTAX,
-        new JsonOperatorValueExprConvertlet(0));
-    registerOp(SqlLibraryOperators.JSON_PRETTY,
-        new JsonOperatorValueExprConvertlet(0));
-    registerOp(SqlLibraryOperators.JSON_TYPE,
-        new JsonOperatorValueExprConvertlet(0));
-    registerOp(SqlLibraryOperators.JSON_DEPTH,
-        new JsonOperatorValueExprConvertlet(0));
-    registerOp(SqlLibraryOperators.JSON_LENGTH,
-        new JsonOperatorValueExprConvertlet(0));
-    registerOp(SqlLibraryOperators.JSON_KEYS,
-        new JsonOperatorValueExprConvertlet(0));
-
     // Convert json_value('{"foo":"bar"}', 'lax $.foo', returning varchar(2000))
     // to cast(json_value('{"foo":"bar"}', 'lax $.foo') as varchar(2000))
     registerOp(
@@ -235,8 +218,8 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
               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));
+                      call.operand(2), call.operand(3), call.operand(4), call.operand(5), null),
+              call.operand(6));
           return cx.convertExpression(expanded);
         });
 
@@ -1446,58 +1429,6 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     }
   }
 
-  /** Convertlet that adds implicit calls to
-   * {@link org.apache.calcite.sql.fun.SqlJsonValueExpressionOperator}
-   * for a series fo JSON operators.
-   *
-   * <p> A call to JSON operator sometimes has a specific expression
-   * argument (it's always the first argument of the call), which implies
-   * a "FORMAT JSON" postfix when the argument is not a explicit call to the
-   * {@link org.apache.calcite.sql.fun.SqlJsonValueExpressionOperator}. */
-  private class JsonOperatorValueExprConvertlet implements SqlRexConvertlet {
-    private final int[] implicitJsonValueExprOrdinals;
-
-    private JsonOperatorValueExprConvertlet(int[] implicitJsonValueExprOrdinals) {
-      this.implicitJsonValueExprOrdinals = implicitJsonValueExprOrdinals;
-    }
-
-    private JsonOperatorValueExprConvertlet(int implicitJsonValueExprOrdinal) {
-      this(new int[]{implicitJsonValueExprOrdinal});
-    }
-
-    public RexNode convertCall(SqlRexContext cx, SqlCall call) {
-      List<SqlNode> newOperands = new ArrayList<>(call.getOperandList());
-      for (int ordinal : implicitJsonValueExprOrdinals) {
-        assert ordinal < newOperands.size();
-        final SqlNode operand = call.operand(ordinal);
-        if (operand.getKind() != SqlKind.JSON_VALUE_EXPRESSION) {
-          final SqlPostfixOperator newOp = SqlStdOperatorTable.JSON_VALUE_EXPRESSION;
-
-          // Validate the original operand using the new operator. JSON value expression
-          // accepts only a character input.
-          //
-          // Once CALCITE-2869 is fixed, we can validate the original operand during SQL
-          // validation using SqlTypeName.JSON.
-          RelDataType validateOperandType = cx.getValidator().getValidatedNodeType(operand);
-          assert validateOperandType != null;
-          if (!SqlTypeUtil.inCharFamily(validateOperandType)) {
-            // must be character type
-            throw cx.getValidator().newValidationError(call,
-                RESOURCE.canNotApplyOp2Type(newOp.getName(),
-                    SqlUtil.getOperatorSignature(newOp,
-                        Collections.singletonList(validateOperandType)),
-                    newOp.getAllowedSignatures()));
-          }
-
-          final SqlNode replacement = newOp
-              .createCall(SqlParserPos.ZERO, operand);
-          newOperands.set(ordinal, replacement);
-        }
-      }
-      return StandardConvertletTable.this.convertCall(cx, call.getOperator(), newOperands);
-    }
-  }
-
   /** Convertlet that handles the {@code TIMESTAMPADD} function. */
   private static class TimestampAddConvertlet implements SqlRexConvertlet {
     public RexNode convertCall(SqlRexContext cx, SqlCall call) {
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 f06e2f5..04d936c 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -78,6 +78,7 @@ import org.apache.calcite.runtime.BinarySearch;
 import org.apache.calcite.runtime.Bindable;
 import org.apache.calcite.runtime.Enumerables;
 import org.apache.calcite.runtime.FlatLists;
+import org.apache.calcite.runtime.JsonFunctions;
 import org.apache.calcite.runtime.RandomFunction;
 import org.apache.calcite.runtime.ResultSetEnumerable;
 import org.apache.calcite.runtime.SortedMultiMap;
@@ -267,37 +268,39 @@ public enum BuiltInMethod {
   UPPER(SqlFunctions.class, "upper", String.class),
   LOWER(SqlFunctions.class, "lower", String.class),
   ASCII(SqlFunctions.class, "ascii", String.class),
-  JSONIZE(SqlFunctions.class, "jsonize", Object.class),
-  DEJSONIZE(SqlFunctions.class, "dejsonize", String.class),
-  JSON_VALUE_EXPRESSION(SqlFunctions.class, "jsonValueExpression",
+  JSONIZE(JsonFunctions.class, "jsonize", Object.class),
+  DEJSONIZE(JsonFunctions.class, "dejsonize", String.class),
+  JSON_VALUE_EXPRESSION(JsonFunctions.class, "jsonValueExpression",
       String.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,
+  JSON_API_COMMON_SYNTAX(JsonFunctions.class, "jsonApiCommonSyntax",
+      String.class, String.class),
+  JSON_EXISTS(JsonFunctions.class, "jsonExists", String.class, String.class),
+  JSON_VALUE_ANY(JsonFunctions.class, "jsonValueAny", String.class,
+      String.class,
       SqlJsonValueEmptyOrErrorBehavior.class, Object.class,
       SqlJsonValueEmptyOrErrorBehavior.class, Object.class),
-  JSON_QUERY(SqlFunctions.class, "jsonQuery", Object.class,
+  JSON_QUERY(JsonFunctions.class, "jsonQuery", String.class,
+      String.class,
       SqlJsonQueryWrapperBehavior.class,
       SqlJsonQueryEmptyOrErrorBehavior.class,
       SqlJsonQueryEmptyOrErrorBehavior.class),
-  JSON_OBJECT(SqlFunctions.class, "jsonObject",
+  JSON_OBJECT(JsonFunctions.class, "jsonObject",
       SqlJsonConstructorNullClause.class),
-  JSON_TYPE(SqlFunctions.class, "jsonType", Object.class),
-  JSON_DEPTH(SqlFunctions.class, "jsonDepth", Object.class),
-  JSON_KEYS(SqlFunctions.class, "jsonKeys", Object.class),
-  JSON_PRETTY(SqlFunctions.class, "jsonPretty", Object.class),
-  JSON_LENGTH(SqlFunctions.class, "jsonLength", Object.class),
-  JSON_OBJECTAGG_ADD(SqlFunctions.class, "jsonObjectAggAdd", Map.class,
+  JSON_TYPE(JsonFunctions.class, "jsonType", String.class),
+  JSON_DEPTH(JsonFunctions.class, "jsonDepth", String.class),
+  JSON_KEYS(JsonFunctions.class, "jsonKeys", String.class),
+  JSON_PRETTY(JsonFunctions.class, "jsonPretty", String.class),
+  JSON_LENGTH(JsonFunctions.class, "jsonLength", String.class),
+  JSON_OBJECTAGG_ADD(JsonFunctions.class, "jsonObjectAggAdd", Map.class,
       String.class, Object.class, SqlJsonConstructorNullClause.class),
-  JSON_ARRAY(SqlFunctions.class, "jsonArray",
+  JSON_ARRAY(JsonFunctions.class, "jsonArray",
       SqlJsonConstructorNullClause.class),
-  JSON_ARRAYAGG_ADD(SqlFunctions.class, "jsonArrayAggAdd",
+  JSON_ARRAYAGG_ADD(JsonFunctions.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),
-  IS_JSON_SCALAR(SqlFunctions.class, "isJsonScalar", String.class),
+  IS_JSON_VALUE(JsonFunctions.class, "isJsonValue", String.class),
+  IS_JSON_OBJECT(JsonFunctions.class, "isJsonObject", String.class),
+  IS_JSON_ARRAY(JsonFunctions.class, "isJsonArray", String.class),
+  IS_JSON_SCALAR(JsonFunctions.class, "isJsonScalar", String.class),
   INITCAP(SqlFunctions.class, "initcap", String.class),
   SUBSTRING(SqlFunctions.class, "substring", String.class, int.class,
       int.class),
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index d5c9920..9a7296d 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -857,6 +857,18 @@ public class Util {
   }
 
   /**
+   * Wraps an exception with {@link RuntimeException} and return it.
+   * If the exception is already an instance of RuntimeException,
+   * returns it directly.
+   */
+  public static RuntimeException toUnchecked(Exception e) {
+    if (e instanceof RuntimeException) {
+      return (RuntimeException) e;
+    }
+    return new RuntimeException(e);
+  }
+
+  /**
    * Retrieves messages in a exception and writes them to a string. In the
    * string returned, each message will appear on a different line.
    *
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 5ae565e..8dc4cc5 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
@@ -3162,14 +3162,14 @@ public class RelToSqlConverterTest {
 
   @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"
+    final String expected = "SELECT JSON_EXISTS(\"product_name\", 'lax $')\n"
         + "FROM \"foodmart\".\"product\"";
     sql(query).ok(expected);
   }
 
   @Test public void testJsonPretty() {
     String query = "select json_pretty(\"product_name\") from \"product\"";
-    final String expected = "SELECT JSON_PRETTY(\"product_name\" FORMAT JSON)\n"
+    final String expected = "SELECT JSON_PRETTY(\"product_name\")\n"
         + "FROM \"foodmart\".\"product\"";
     sql(query).ok(expected);
   }
@@ -3177,7 +3177,7 @@ public class RelToSqlConverterTest {
   @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, "
+    final String expected = "SELECT CAST(JSON_VALUE_ANY(\"product_name\", "
         + "'lax $' NULL ON EMPTY NULL ON ERROR) AS VARCHAR(2000) CHARACTER SET \"ISO-8859-1\")\n"
         + "FROM \"foodmart\".\"product\"";
     sql(query).ok(expected);
@@ -3185,7 +3185,7 @@ public class RelToSqlConverterTest {
 
   @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 $' "
+    final String expected = "SELECT JSON_QUERY(\"product_name\", 'lax $' "
         + "WITHOUT ARRAY WRAPPER NULL ON EMPTY NULL ON ERROR)\n"
         + "FROM \"foodmart\".\"product\"";
     sql(query).ok(expected);
@@ -3260,7 +3260,7 @@ public class RelToSqlConverterTest {
   @Test public void testJsonType() {
     String query = "select json_type(\"product_name\") from \"product\"";
     final String expected = "SELECT "
-        + "JSON_TYPE(\"product_name\" FORMAT JSON)\n"
+        + "JSON_TYPE(\"product_name\")\n"
         + "FROM \"foodmart\".\"product\"";
     sql(query).ok(expected);
   }
@@ -3268,7 +3268,7 @@ public class RelToSqlConverterTest {
   @Test public void testJsonDepth() {
     String query = "select json_depth(\"product_name\") from \"product\"";
     final String expected = "SELECT "
-        + "JSON_DEPTH(\"product_name\" FORMAT JSON)\n"
+        + "JSON_DEPTH(\"product_name\")\n"
         + "FROM \"foodmart\".\"product\"";
     sql(query).ok(expected);
   }
@@ -3276,15 +3276,15 @@ public class RelToSqlConverterTest {
   @Test public void testJsonLength() {
     String query = "select json_length(\"product_name\", 'lax $'), "
         + "json_length(\"product_name\") from \"product\"";
-    final String expected = "SELECT JSON_LENGTH(\"product_name\" FORMAT JSON, 'lax $'), "
-        + "JSON_LENGTH(\"product_name\" FORMAT JSON)\n"
+    final String expected = "SELECT JSON_LENGTH(\"product_name\", 'lax $'), "
+        + "JSON_LENGTH(\"product_name\")\n"
         + "FROM \"foodmart\".\"product\"";
     sql(query).ok(expected);
   }
 
   @Test public void testJsonKeys() {
     String query = "select json_keys(\"product_name\", 'lax $') from \"product\"";
-    final String expected = "SELECT JSON_KEYS(\"product_name\" FORMAT JSON, 'lax $')\n"
+    final String expected = "SELECT JSON_KEYS(\"product_name\", 'lax $')\n"
         + "FROM \"foodmart\".\"product\"";
     sql(query).ok(expected);
   }
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 ef4ba27..ecca87d 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
@@ -4299,6 +4299,13 @@ public abstract class SqlOperatorBaseTest {
         + "'lax $.foo1' error on error)", Boolean.FALSE);
     tester.checkBoolean("json_exists('{\"foo\":\"bar\"}', "
         + "'lax $.foo1' unknown on error)", Boolean.FALSE);
+
+    // nulls
+    tester.checkFails("json_exists(^null^, "
+        + "'lax $' unknown on error)", "(?s).*Illegal use of 'NULL'.*", false);
+    tester.checkNull("json_exists(cast(null as varchar), "
+        + "'lax $.foo1' unknown on error)");
+
   }
 
   @Test public void testJsonValue() {
@@ -4372,6 +4379,11 @@ public abstract class SqlOperatorBaseTest {
     tester.checkString("json_value('{\"foo\":{}}', "
             + "'strict $.foo' default 'empty' on error)",
         "empty", "VARCHAR(2000)");
+
+    // nulls
+    tester.checkFails("json_value(^null^, 'strict $')",
+        "(?s).*Illegal use of 'NULL'.*", false);
+    tester.checkNull("json_value(cast(null as varchar), 'strict $')");
   }
 
   @Test public void testJsonQuery() {
@@ -4461,39 +4473,52 @@ public abstract class SqlOperatorBaseTest {
         "[100]", "VARCHAR(2000)");
 
 
+    // nulls
+    tester.checkFails("json_query(^null^, 'lax $')",
+        "(?s).*Illegal use of 'NULL'.*", false);
+    tester.checkNull("json_query(cast(null as varchar), 'lax $')");
   }
 
   @Test public void testJsonPretty() {
     tester.checkString("json_pretty('{\"foo\":100}')",
-        "{\n  \"foo\" : 100\n}", "VARCHAR(2000) NOT NULL");
+        "{\n  \"foo\" : 100\n}", "VARCHAR(2000)");
     tester.checkString("json_pretty('[1,2,3]')",
-        "[ 1, 2, 3 ]", "VARCHAR(2000) NOT NULL");
+        "[ 1, 2, 3 ]", "VARCHAR(2000)");
     tester.checkString("json_pretty('null')",
-        "null", "VARCHAR(2000) NOT NULL");
+        "null", "VARCHAR(2000)");
+
+    // nulls
+    tester.checkFails("json_pretty(^null^)",
+        "(?s).*Illegal use of 'NULL'.*", false);
+    tester.checkNull("json_pretty(cast(null as varchar))");
   }
 
   @Test public void testJsonType() {
     tester.setFor(SqlLibraryOperators.JSON_TYPE);
     tester.checkString("json_type('\"1\"')",
-            "STRING", "VARCHAR(20) NOT NULL");
+            "STRING", "VARCHAR(20)");
     tester.checkString("json_type('1')",
-            "INTEGER", "VARCHAR(20) NOT NULL");
+            "INTEGER", "VARCHAR(20)");
     tester.checkString("json_type('11.45')",
-            "DOUBLE", "VARCHAR(20) NOT NULL");
+            "DOUBLE", "VARCHAR(20)");
     tester.checkString("json_type('true')",
-            "BOOLEAN", "VARCHAR(20) NOT NULL");
+            "BOOLEAN", "VARCHAR(20)");
     tester.checkString("json_type('null')",
-            "NULL", "VARCHAR(20) NOT NULL");
-    tester.checkString("json_type(cast(null as varchar(1)))",
-            "NULL", "VARCHAR(20) NOT NULL");
+            "NULL", "VARCHAR(20)");
+    tester.checkNull("json_type(cast(null as varchar(1)))");
     tester.checkString("json_type('{\"a\": [10, true]}')",
-            "OBJECT", "VARCHAR(20) NOT NULL");
+            "OBJECT", "VARCHAR(20)");
     tester.checkString("json_type('{}')",
-            "OBJECT", "VARCHAR(20) NOT NULL");
+            "OBJECT", "VARCHAR(20)");
     tester.checkString("json_type('[10, true]')",
-            "ARRAY", "VARCHAR(20) NOT NULL");
+            "ARRAY", "VARCHAR(20)");
     tester.checkString("json_type('\"2019-01-27 21:24:00\"')",
-            "STRING", "VARCHAR(20) NOT NULL");
+            "STRING", "VARCHAR(20)");
+
+    // nulls
+    tester.checkFails("json_type(^null^)",
+        "(?s).*Illegal use of 'NULL'.*", false);
+    tester.checkNull("json_type(cast(null as varchar))");
   }
 
   @Test public void testJsonDepth() {
@@ -4522,6 +4547,11 @@ public abstract class SqlOperatorBaseTest {
             "3", "INTEGER");
     tester.checkString("json_depth('[10, {\"a\": [[1,2]]}]')",
             "5", "INTEGER");
+
+    // nulls
+    tester.checkFails("json_depth(^null^)",
+        "(?s).*Illegal use of 'NULL'.*", false);
+    tester.checkNull("json_depth(cast(null as varchar))");
   }
 
   @Test public void testJsonLength() {
@@ -4572,56 +4602,66 @@ public abstract class SqlOperatorBaseTest {
             "(?s).*Illegal jsonpath spec.*", true);
     tester.checkFails("json_length('{\"foo\":100}', 'strict $.foo1')",
             "(?s).*No results for path.*", true);
+
+    // nulls
+    tester.checkFails("json_length(^null^)",
+        "(?s).*Illegal use of 'NULL'.*", false);
+    tester.checkNull("json_length(cast(null as varchar))");
   }
 
   @Test public void testJsonKeys() {
     // no path context
     tester.checkString("json_keys('{}')",
-            "[]", "VARCHAR(2000) NOT NULL");
+            "[]", "VARCHAR(2000)");
     tester.checkString("json_keys('[]')",
-            "null", "VARCHAR(2000) NOT NULL");
+            "null", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"foo\":100}')",
-            "[\"foo\"]", "VARCHAR(2000) NOT NULL");
+            "[\"foo\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"a\": 1, \"b\": {\"c\": 30}}')",
-            "[\"a\",\"b\"]", "VARCHAR(2000) NOT NULL");
+            "[\"a\",\"b\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('[1, 2, {\"a\": 3}]')",
-            "null", "VARCHAR(2000) NOT NULL");
+            "null", "VARCHAR(2000)");
 
     // lax test
     tester.checkString("json_keys('{}', 'lax $')",
-            "[]", "VARCHAR(2000) NOT NULL");
+            "[]", "VARCHAR(2000)");
     tester.checkString("json_keys('[]', 'lax $')",
-            "null", "VARCHAR(2000) NOT NULL");
+            "null", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"foo\":100}', 'lax $')",
-            "[\"foo\"]", "VARCHAR(2000) NOT NULL");
+            "[\"foo\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"a\": 1, \"b\": {\"c\": 30}}', 'lax $')",
-            "[\"a\",\"b\"]", "VARCHAR(2000) NOT NULL");
+            "[\"a\",\"b\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('[1, 2, {\"a\": 3}]', 'lax $')",
-            "null", "VARCHAR(2000) NOT NULL");
+            "null", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"a\": 1, \"b\": {\"c\": 30}}', 'lax $.b')",
-            "[\"c\"]", "VARCHAR(2000) NOT NULL");
+            "[\"c\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"foo\":100}', 'lax $.foo1')",
-            "null", "VARCHAR(2000) NOT NULL");
+            "null", "VARCHAR(2000)");
 
     // strict test
     tester.checkString("json_keys('{}', 'strict $')",
-            "[]", "VARCHAR(2000) NOT NULL");
+            "[]", "VARCHAR(2000)");
     tester.checkString("json_keys('[]', 'strict $')",
-            "null", "VARCHAR(2000) NOT NULL");
+            "null", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"foo\":100}', 'strict $')",
-            "[\"foo\"]", "VARCHAR(2000) NOT NULL");
+            "[\"foo\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"a\": 1, \"b\": {\"c\": 30}}', 'strict $')",
-            "[\"a\",\"b\"]", "VARCHAR(2000) NOT NULL");
+            "[\"a\",\"b\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('[1, 2, {\"a\": 3}]', 'strict $')",
-            "null", "VARCHAR(2000) NOT NULL");
+            "null", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"a\": 1, \"b\": {\"c\": 30}}', 'strict $.b')",
-            "[\"c\"]", "VARCHAR(2000) NOT NULL");
+            "[\"c\"]", "VARCHAR(2000)");
 
     // catch error test
     tester.checkFails("json_keys('{\"foo\":100}', 'invalid $.foo')",
             "(?s).*Illegal jsonpath spec.*", true);
     tester.checkFails("json_keys('{\"foo\":100}', 'strict $.foo1')",
             "(?s).*No results for path.*", true);
+
+    // nulls
+    tester.checkFails("json_keys(^null^)",
+        "(?s).*Illegal use of 'NULL'.*", false);
+    tester.checkNull("json_keys(cast(null as varchar))");
   }
 
   @Test public void testJsonObject() {
@@ -4668,6 +4708,14 @@ public abstract class SqlOperatorBaseTest {
         0.0D);
   }
 
+  @Test public void testJsonValueExpressionOperator() {
+    tester.checkScalar("'{}' format json", "{}", "ANY NOT NULL");
+    tester.checkScalar("'[1, 2, 3]' format json", "[1, 2, 3]", "ANY NOT NULL");
+    tester.checkNull("cast(null as varchar) format json");
+    tester.checkScalar("'null' format json", "null", "ANY NOT NULL");
+    tester.checkFails("^null^ format json", "(?s).*Illegal use of .NULL.*", false);
+  }
+
   @Test public void testJsonArray() {
     tester.checkString("json_array()", "[]", "VARCHAR(2000) NOT NULL");
     tester.checkString("json_array('foo')",
diff --git a/core/src/test/java/org/apache/calcite/test/SqlJsonFunctionsTest.java b/core/src/test/java/org/apache/calcite/test/SqlJsonFunctionsTest.java
index bc4fbd0..3126ff1 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlJsonFunctionsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlJsonFunctionsTest.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.test;
 
 import org.apache.calcite.runtime.CalciteException;
+import org.apache.calcite.runtime.JsonFunctions;
 import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.sql.SqlJsonConstructorNullClause;
 import org.apache.calcite.sql.SqlJsonExistsErrorBehavior;
@@ -25,7 +26,6 @@ import org.apache.calcite.sql.SqlJsonQueryWrapperBehavior;
 import org.apache.calcite.sql.SqlJsonValueEmptyOrErrorBehavior;
 import org.apache.calcite.util.BuiltInMethod;
 
-import com.google.common.collect.ImmutableMap;
 import com.google.common.primitives.Longs;
 import com.jayway.jsonpath.InvalidJsonException;
 import com.jayway.jsonpath.PathNotFoundException;
@@ -58,88 +58,89 @@ public class SqlJsonFunctionsTest {
 
   @Test
   public void testJsonValueExpression() {
-    assertJsonValueExpression("{}", is(Collections.emptyMap()));
+    assertJsonValueExpression("{}",
+        is(JsonFunctions.JsonValueContext.withJavaObj(Collections.emptyMap())));
   }
 
   @Test
   public void testJsonApiCommonSyntax() {
-    assertJsonApiCommonSyntax(ImmutableMap.of("foo", "bar"), "lax $.foo",
+    assertJsonApiCommonSyntax("{\"foo\": \"bar\"}", "lax $.foo",
         contextMatches(
-            SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.LAX, "bar")));
-    assertJsonApiCommonSyntax(ImmutableMap.of("foo", "bar"), "strict $.foo",
+            JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.LAX, "bar")));
+    assertJsonApiCommonSyntax("{\"foo\": \"bar\"}", "strict $.foo",
         contextMatches(
-            SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.STRICT, "bar")));
-    assertJsonApiCommonSyntax(ImmutableMap.of("foo", "bar"), "lax $.foo1",
+            JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.STRICT, "bar")));
+    assertJsonApiCommonSyntax("{\"foo\": \"bar\"}", "lax $.foo1",
         contextMatches(
-            SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.LAX, null)));
-    assertJsonApiCommonSyntax(ImmutableMap.of("foo", "bar"), "strict $.foo1",
+            JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.LAX, null)));
+    assertJsonApiCommonSyntax("{\"foo\": \"bar\"}", "strict $.foo1",
         contextMatches(
-            SqlFunctions.PathContext.withStrictException(
+            JsonFunctions.JsonPathContext.withStrictException(
                 new PathNotFoundException("No results for path: $['foo1']"))));
-    assertJsonApiCommonSyntax(ImmutableMap.of("foo", 100), "lax $.foo",
+    assertJsonApiCommonSyntax("{\"foo\": 100}", "lax $.foo",
         contextMatches(
-            SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.LAX, 100)));
+            JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.LAX, 100)));
   }
 
   @Test
   public void testJsonExists() {
     assertJsonExists(
-        SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.STRICT, "bar"),
+        JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.STRICT, "bar"),
         SqlJsonExistsErrorBehavior.FALSE,
         is(true));
 
     assertJsonExists(
-        SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.STRICT, "bar"),
+        JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.STRICT, "bar"),
         SqlJsonExistsErrorBehavior.TRUE,
         is(true));
 
     assertJsonExists(
-        SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.STRICT, "bar"),
+        JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.STRICT, "bar"),
         SqlJsonExistsErrorBehavior.UNKNOWN,
         is(true));
 
     assertJsonExists(
-        SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.STRICT, "bar"),
+        JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.STRICT, "bar"),
         SqlJsonExistsErrorBehavior.ERROR,
         is(true));
 
     assertJsonExists(
-        SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.LAX, null),
         SqlJsonExistsErrorBehavior.FALSE,
         is(false));
 
     assertJsonExists(
-        SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.LAX, null),
         SqlJsonExistsErrorBehavior.TRUE,
         is(false));
 
     assertJsonExists(
-        SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.LAX, null),
         SqlJsonExistsErrorBehavior.UNKNOWN,
         is(false));
 
     assertJsonExists(
-        SqlFunctions.PathContext.withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext.withJavaObj(JsonFunctions.PathMode.LAX, null),
         SqlJsonExistsErrorBehavior.ERROR,
         is(false));
 
     assertJsonExists(
-        SqlFunctions.PathContext.withStrictException(new Exception("test message")),
+        JsonFunctions.JsonPathContext.withStrictException(new Exception("test message")),
         SqlJsonExistsErrorBehavior.FALSE,
         is(false));
 
     assertJsonExists(
-        SqlFunctions.PathContext.withStrictException(new Exception("test message")),
+        JsonFunctions.JsonPathContext.withStrictException(new Exception("test message")),
         SqlJsonExistsErrorBehavior.TRUE,
         is(true));
 
     assertJsonExists(
-        SqlFunctions.PathContext.withStrictException(new Exception("test message")),
+        JsonFunctions.JsonPathContext.withStrictException(new Exception("test message")),
         SqlJsonExistsErrorBehavior.UNKNOWN,
         nullValue());
 
     assertJsonExistsFailed(
-        SqlFunctions.PathContext.withStrictException(new Exception("test message")),
+        JsonFunctions.JsonPathContext.withStrictException(new Exception("test message")),
         SqlJsonExistsErrorBehavior.ERROR,
         errorMatches(new RuntimeException("java.lang.Exception: test message")));
   }
@@ -147,32 +148,32 @@ public class SqlJsonFunctionsTest {
   @Test
   public void testJsonValueAny() {
     assertJsonValueAny(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, "bar"),
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         is("bar"));
     assertJsonValueAny(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, null),
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         nullValue());
     assertJsonValueAny(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, null),
         SqlJsonValueEmptyOrErrorBehavior.DEFAULT,
         "empty",
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         is("empty"));
     assertJsonValueAnyFailed(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, null),
         SqlJsonValueEmptyOrErrorBehavior.ERROR,
         null,
         SqlJsonValueEmptyOrErrorBehavior.NULL,
@@ -181,24 +182,24 @@ public class SqlJsonFunctionsTest {
             new CalciteException("Empty result of JSON_VALUE function is not "
                 + "allowed", null)));
     assertJsonValueAny(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, Collections.emptyList()),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, Collections.emptyList()),
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         nullValue());
     assertJsonValueAny(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, Collections.emptyList()),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, Collections.emptyList()),
         SqlJsonValueEmptyOrErrorBehavior.DEFAULT,
         "empty",
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         is("empty"));
     assertJsonValueAnyFailed(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, Collections.emptyList()),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, Collections.emptyList()),
         SqlJsonValueEmptyOrErrorBehavior.ERROR,
         null,
         SqlJsonValueEmptyOrErrorBehavior.NULL,
@@ -207,7 +208,7 @@ public class SqlJsonFunctionsTest {
             new CalciteException("Empty result of JSON_VALUE function is not "
                 + "allowed", null)));
     assertJsonValueAny(
-        SqlFunctions.PathContext
+        JsonFunctions.JsonPathContext
             .withStrictException(new Exception("test message")),
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
@@ -215,7 +216,7 @@ public class SqlJsonFunctionsTest {
         null,
         nullValue());
     assertJsonValueAny(
-        SqlFunctions.PathContext
+        JsonFunctions.JsonPathContext
             .withStrictException(new Exception("test message")),
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
@@ -223,7 +224,7 @@ public class SqlJsonFunctionsTest {
         "empty",
         is("empty"));
     assertJsonValueAnyFailed(
-        SqlFunctions.PathContext
+        JsonFunctions.JsonPathContext
             .withStrictException(new Exception("test message")),
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
@@ -232,24 +233,24 @@ public class SqlJsonFunctionsTest {
         errorMatches(
             new RuntimeException("java.lang.Exception: test message")));
     assertJsonValueAny(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT, Collections.emptyList()),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT, Collections.emptyList()),
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         nullValue());
     assertJsonValueAny(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT, Collections.emptyList()),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT, Collections.emptyList()),
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         SqlJsonValueEmptyOrErrorBehavior.DEFAULT,
         "empty",
         is("empty"));
     assertJsonValueAnyFailed(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT, Collections.emptyList()),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT, Collections.emptyList()),
         SqlJsonValueEmptyOrErrorBehavior.NULL,
         null,
         SqlJsonValueEmptyOrErrorBehavior.ERROR,
@@ -262,36 +263,36 @@ public class SqlJsonFunctionsTest {
   @Test
   public void testJsonQuery() {
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, Collections.singletonList("bar")),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, Collections.singletonList("bar")),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         is("[\"bar\"]"));
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, null),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         nullValue());
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, null),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.EMPTY_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         is("[]"));
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, null),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.EMPTY_OBJECT,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         is("{}"));
     assertJsonQueryFailed(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, null),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.ERROR,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
@@ -300,29 +301,29 @@ public class SqlJsonFunctionsTest {
                 + "allowed", null)));
 
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, "bar"),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         nullValue());
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, "bar"),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.EMPTY_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         is("[]"));
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, "bar"),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.EMPTY_OBJECT,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         is("{}"));
     assertJsonQueryFailed(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, "bar"),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.ERROR,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
@@ -330,21 +331,21 @@ public class SqlJsonFunctionsTest {
             new CalciteException("Empty result of JSON_QUERY function is not "
                 + "allowed", null)));
     assertJsonQuery(
-        SqlFunctions.PathContext
+        JsonFunctions.JsonPathContext
             .withStrictException(new Exception("test message")),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         SqlJsonQueryEmptyOrErrorBehavior.EMPTY_ARRAY,
         is("[]"));
     assertJsonQuery(
-        SqlFunctions.PathContext
+        JsonFunctions.JsonPathContext
             .withStrictException(new Exception("test message")),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         SqlJsonQueryEmptyOrErrorBehavior.EMPTY_OBJECT,
         is("{}"));
     assertJsonQueryFailed(
-        SqlFunctions.PathContext
+        JsonFunctions.JsonPathContext
             .withStrictException(new Exception("test message")),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
@@ -352,22 +353,22 @@ public class SqlJsonFunctionsTest {
         errorMatches(
             new RuntimeException("java.lang.Exception: test message")));
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT, "bar"),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         nullValue());
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT, "bar"),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         SqlJsonQueryEmptyOrErrorBehavior.EMPTY_ARRAY,
         is("[]"));
     assertJsonQueryFailed(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT, "bar"),
         SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         SqlJsonQueryEmptyOrErrorBehavior.ERROR,
@@ -378,24 +379,24 @@ public class SqlJsonFunctionsTest {
     // wrapper behavior test
 
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT, "bar"),
         SqlJsonQueryWrapperBehavior.WITH_UNCONDITIONAL_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         is("[\"bar\"]"));
 
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT, "bar"),
         SqlJsonQueryWrapperBehavior.WITH_CONDITIONAL_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
         is("[\"bar\"]"));
 
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT,
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT,
                 Collections.singletonList("bar")),
         SqlJsonQueryWrapperBehavior.WITH_UNCONDITIONAL_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
@@ -403,8 +404,8 @@ public class SqlJsonFunctionsTest {
         is("[[\"bar\"]]"));
 
     assertJsonQuery(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT,
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT,
                 Collections.singletonList("bar")),
         SqlJsonQueryWrapperBehavior.WITH_CONDITIONAL_ARRAY,
         SqlJsonQueryEmptyOrErrorBehavior.NULL,
@@ -420,15 +421,18 @@ public class SqlJsonFunctionsTest {
 
   @Test
   public void assertJsonPretty() {
-    assertJsonPretty(new HashMap<>(), is("{ }"));
-    assertJsonPretty(Longs.asList(1, 2), is("[ 1, 2 ]"));
+    assertJsonPretty(
+        JsonFunctions.JsonValueContext.withJavaObj(new HashMap<>()), is("{ }"));
+    assertJsonPretty(
+        JsonFunctions.JsonValueContext.withJavaObj(Longs.asList(1, 2)), is("[ 1, 2 ]"));
 
     Object input = new Object() {
       private final Object self = this;
     };
     CalciteException expected = new CalciteException(
         "Cannot serialize object to JSON, and the object is: '" + input + "'", null);
-    assertJsonPrettyFailed(input, errorMatches(expected));
+    assertJsonPrettyFailed(
+        JsonFunctions.JsonValueContext.withJavaObj(input), errorMatches(expected));
   }
 
   @Test
@@ -491,39 +495,39 @@ public class SqlJsonFunctionsTest {
   @Test
   public void testJsonLength() {
     assertJsonLength(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, Collections.singletonList("bar")),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, Collections.singletonList("bar")),
         is(1));
     assertJsonLength(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, null),
         nullValue());
     assertJsonLength(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT, Collections.singletonList("bar")),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT, Collections.singletonList("bar")),
         is(1));
     assertJsonLength(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, "bar"),
         is(1));
   }
 
   public void testJsonKeys() {
     assertJsonKeys(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, Collections.singletonList("bar")),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, Collections.singletonList("bar")),
         is("null"));
     assertJsonKeys(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, null),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, null),
         is("null"));
     assertJsonKeys(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.STRICT, Collections.singletonList("bar")),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.STRICT, Collections.singletonList("bar")),
         is("null"));
     assertJsonKeys(
-        SqlFunctions.PathContext
-            .withReturned(SqlFunctions.PathMode.LAX, "bar"),
+        JsonFunctions.JsonPathContext
+            .withJavaObj(JsonFunctions.PathMode.LAX, "bar"),
         is("null"));
   }
 
@@ -592,47 +596,55 @@ public class SqlJsonFunctionsTest {
   }
 
   private void assertJsonValueExpression(String input,
-      Matcher<Object> matcher) {
+      Matcher<? super JsonFunctions.JsonValueContext> matcher) {
     assertThat(
         invocationDesc(BuiltInMethod.JSON_VALUE_EXPRESSION.getMethodName(), input),
-        SqlFunctions.jsonValueExpression(input), matcher);
+        JsonFunctions.jsonValueExpression(input), matcher);
   }
 
-  private void assertJsonApiCommonSyntax(Object input, String pathSpec,
-      Matcher<? super SqlFunctions.PathContext> matcher) {
+  private void assertJsonApiCommonSyntax(String input, String pathSpec,
+      Matcher<? super JsonFunctions.JsonPathContext> matcher) {
     assertThat(
         invocationDesc(BuiltInMethod.JSON_API_COMMON_SYNTAX.getMethodName(), input, pathSpec),
-        SqlFunctions.jsonApiCommonSyntax(input, pathSpec), matcher);
+        JsonFunctions.jsonApiCommonSyntax(input, pathSpec), matcher);
   }
 
-  private void assertJsonExists(Object input, SqlJsonExistsErrorBehavior errorBehavior,
-      Matcher<? super Boolean> matcher) {
-    assertThat(invocationDesc(BuiltInMethod.JSON_EXISTS.getMethodName(), input, errorBehavior),
-        SqlFunctions.jsonExists(input, errorBehavior), matcher);
+  private void assertJsonApiCommonSyntax(JsonFunctions.JsonValueContext input, String pathSpec,
+      Matcher<? super JsonFunctions.JsonPathContext> matcher) {
+    assertThat(
+        invocationDesc(BuiltInMethod.JSON_API_COMMON_SYNTAX.getMethodName(), input, pathSpec),
+        JsonFunctions.jsonApiCommonSyntax(input, pathSpec), matcher);
   }
 
-  private void assertJsonExistsFailed(Object input,
+  private void assertJsonExists(JsonFunctions.JsonPathContext context,
+      SqlJsonExistsErrorBehavior errorBehavior, Matcher<? super Boolean> matcher) {
+    assertThat(invocationDesc(BuiltInMethod.JSON_EXISTS.getMethodName(), context, errorBehavior),
+        JsonFunctions.jsonExists(context, errorBehavior), matcher);
+  }
+
+  private void assertJsonExistsFailed(JsonFunctions.JsonPathContext context,
       SqlJsonExistsErrorBehavior errorBehavior,
       Matcher<? super Throwable> matcher) {
-    assertFailed(invocationDesc(BuiltInMethod.JSON_EXISTS.getMethodName(), input, errorBehavior),
-        () -> SqlFunctions.jsonExists(input, errorBehavior), matcher);
+    assertFailed(invocationDesc(BuiltInMethod.JSON_EXISTS.getMethodName(), context, errorBehavior),
+        () -> JsonFunctions.jsonExists(
+            context, errorBehavior), matcher);
   }
 
-  private void assertJsonValueAny(Object input,
+  private void assertJsonValueAny(JsonFunctions.JsonPathContext context,
       SqlJsonValueEmptyOrErrorBehavior emptyBehavior,
       Object defaultValueOnEmpty,
       SqlJsonValueEmptyOrErrorBehavior errorBehavior,
       Object defaultValueOnError,
       Matcher<Object> matcher) {
     assertThat(
-        invocationDesc(BuiltInMethod.JSON_VALUE_ANY.getMethodName(), input, emptyBehavior,
+        invocationDesc(BuiltInMethod.JSON_VALUE_ANY.getMethodName(), context, emptyBehavior,
             defaultValueOnEmpty, errorBehavior, defaultValueOnError),
-        SqlFunctions.jsonValueAny(input, emptyBehavior, defaultValueOnEmpty,
+        JsonFunctions.jsonValueAny(context, emptyBehavior, defaultValueOnEmpty,
             errorBehavior, defaultValueOnError),
         matcher);
   }
 
-  private void assertJsonValueAnyFailed(Object input,
+  private void assertJsonValueAnyFailed(JsonFunctions.JsonPathContext input,
       SqlJsonValueEmptyOrErrorBehavior emptyBehavior,
       Object defaultValueOnEmpty,
       SqlJsonValueEmptyOrErrorBehavior errorBehavior,
@@ -641,12 +653,12 @@ public class SqlJsonFunctionsTest {
     assertFailed(
         invocationDesc(BuiltInMethod.JSON_VALUE_ANY.getMethodName(), input, emptyBehavior,
             defaultValueOnEmpty, errorBehavior, defaultValueOnError),
-        () -> SqlFunctions.jsonValueAny(input, emptyBehavior,
+        () -> JsonFunctions.jsonValueAny(input, emptyBehavior,
             defaultValueOnEmpty, errorBehavior, defaultValueOnError),
         matcher);
   }
 
-  private void assertJsonQuery(Object input,
+  private void assertJsonQuery(JsonFunctions.JsonPathContext input,
       SqlJsonQueryWrapperBehavior wrapperBehavior,
       SqlJsonQueryEmptyOrErrorBehavior emptyBehavior,
       SqlJsonQueryEmptyOrErrorBehavior errorBehavior,
@@ -654,12 +666,12 @@ public class SqlJsonFunctionsTest {
     assertThat(
         invocationDesc(BuiltInMethod.JSON_QUERY.getMethodName(), input, wrapperBehavior,
             emptyBehavior, errorBehavior),
-        SqlFunctions.jsonQuery(input, wrapperBehavior, emptyBehavior,
+        JsonFunctions.jsonQuery(input, wrapperBehavior, emptyBehavior,
             errorBehavior),
         matcher);
   }
 
-  private void assertJsonQueryFailed(Object input,
+  private void assertJsonQueryFailed(JsonFunctions.JsonPathContext input,
       SqlJsonQueryWrapperBehavior wrapperBehavior,
       SqlJsonQueryEmptyOrErrorBehavior emptyBehavior,
       SqlJsonQueryEmptyOrErrorBehavior errorBehavior,
@@ -667,7 +679,7 @@ public class SqlJsonFunctionsTest {
     assertFailed(
         invocationDesc(BuiltInMethod.JSON_QUERY.getMethodName(), input, wrapperBehavior,
             emptyBehavior, errorBehavior),
-        () -> SqlFunctions.jsonQuery(input, wrapperBehavior, emptyBehavior,
+        () -> JsonFunctions.jsonQuery(input, wrapperBehavior, emptyBehavior,
             errorBehavior),
         matcher);
   }
@@ -675,66 +687,66 @@ public class SqlJsonFunctionsTest {
   private void assertJsonize(Object input,
       Matcher<? super String> matcher) {
     assertThat(invocationDesc(BuiltInMethod.JSONIZE.getMethodName(), input),
-        SqlFunctions.jsonize(input),
+        JsonFunctions.jsonize(input),
         matcher);
   }
 
-  private void assertJsonPretty(Object input,
+  private void assertJsonPretty(JsonFunctions.JsonValueContext input,
       Matcher<? super String> matcher) {
     assertThat(invocationDesc(BuiltInMethod.JSON_PRETTY.getMethodName(), input),
-        SqlFunctions.jsonPretty(input),
+        JsonFunctions.jsonPretty(input),
         matcher);
   }
 
-  private void assertJsonPrettyFailed(Object input,
+  private void assertJsonPrettyFailed(JsonFunctions.JsonValueContext input,
       Matcher<? super Throwable> matcher) {
     assertFailed(invocationDesc(BuiltInMethod.JSON_PRETTY.getMethodName(), input),
-        () -> SqlFunctions.jsonPretty(input),
+        () -> JsonFunctions.jsonPretty(input),
         matcher);
   }
 
-  private void assertJsonLength(SqlFunctions.PathContext input,
+  private void assertJsonLength(JsonFunctions.JsonPathContext input,
       Matcher<? super Integer> matcher) {
     assertThat(
         invocationDesc(BuiltInMethod.JSON_LENGTH.getMethodName(), input),
-        SqlFunctions.jsonLength(input),
+        JsonFunctions.jsonLength(input),
         matcher);
   }
 
-  private void assertJsonLengthFailed(Object input,
+  private void assertJsonLengthFailed(JsonFunctions.JsonValueContext input,
       Matcher<? super Throwable> matcher) {
     assertFailed(
         invocationDesc(BuiltInMethod.JSON_LENGTH.getMethodName(), input),
-        () -> SqlFunctions.jsonLength(input),
+        () -> JsonFunctions.jsonLength(input),
         matcher);
   }
 
-  private void assertJsonKeys(Object input,
+  private void assertJsonKeys(JsonFunctions.JsonPathContext input,
       Matcher<? super String> matcher) {
     assertThat(
         invocationDesc(BuiltInMethod.JSON_KEYS.getMethodName(), input),
-        SqlFunctions.jsonKeys(input),
+        JsonFunctions.jsonKeys(input),
         matcher);
   }
 
-  private void assertJsonKeysFailed(Object input,
+  private void assertJsonKeysFailed(JsonFunctions.JsonValueContext input,
       Matcher<? super Throwable> matcher) {
     assertFailed(invocationDesc(BuiltInMethod.JSON_KEYS.getMethodName(), input),
-        () -> SqlFunctions.jsonKeys(input),
+        () -> JsonFunctions.jsonKeys(input),
         matcher);
   }
 
   private void assertDejsonize(String input,
       Matcher<Object> matcher) {
     assertThat(invocationDesc(BuiltInMethod.DEJSONIZE.getMethodName(), input),
-        SqlFunctions.dejsonize(input),
+        JsonFunctions.dejsonize(input),
         matcher);
   }
 
   private void assertDejsonizeFailed(String input,
       Matcher<? super Throwable> matcher) {
     assertFailed(invocationDesc(BuiltInMethod.DEJSONIZE.getMethodName(), input),
-        () -> SqlFunctions.dejsonize(input),
+        () -> JsonFunctions.dejsonize(input),
         matcher);
   }
 
@@ -742,7 +754,7 @@ public class SqlJsonFunctionsTest {
       SqlJsonConstructorNullClause nullClause,
       Object... kvs) {
     assertThat(invocationDesc(BuiltInMethod.JSON_OBJECT.getMethodName(), nullClause, kvs),
-        SqlFunctions.jsonObject(nullClause, kvs),
+        JsonFunctions.jsonObject(nullClause, kvs),
         matcher);
   }
 
@@ -750,8 +762,7 @@ public class SqlJsonFunctionsTest {
       String input) {
     assertThat(
         invocationDesc(BuiltInMethod.JSON_TYPE.getMethodName(), input),
-        SqlFunctions.jsonType(
-            SqlFunctions.dejsonize(input)),
+        JsonFunctions.jsonType(input),
         matcher);
   }
 
@@ -759,15 +770,14 @@ public class SqlJsonFunctionsTest {
       String input) {
     assertThat(
         invocationDesc(BuiltInMethod.JSON_DEPTH.getMethodName(), input),
-        SqlFunctions.jsonDepth(
-            SqlFunctions.dejsonize(input)),
+        JsonFunctions.jsonDepth(input),
         matcher);
   }
 
   private void assertJsonObjectAggAdd(Map map, String k, Object v,
       SqlJsonConstructorNullClause nullClause,
       Matcher<? super Map> matcher) {
-    SqlFunctions.jsonObjectAggAdd(map, k, v, nullClause);
+    JsonFunctions.jsonObjectAggAdd(map, k, v, nullClause);
     assertThat(
         invocationDesc(BuiltInMethod.JSON_ARRAYAGG_ADD.getMethodName(), map, k, v, nullClause),
         map, matcher);
@@ -776,14 +786,14 @@ public class SqlJsonFunctionsTest {
   private void assertJsonArray(Matcher<? super String> matcher,
       SqlJsonConstructorNullClause nullClause, Object... elements) {
     assertThat(invocationDesc(BuiltInMethod.JSON_ARRAY.getMethodName(), nullClause, elements),
-        SqlFunctions.jsonArray(nullClause, elements),
+        JsonFunctions.jsonArray(nullClause, elements),
         matcher);
   }
 
   private void assertJsonArrayAggAdd(List list, Object element,
       SqlJsonConstructorNullClause nullClause,
       Matcher<? super List> matcher) {
-    SqlFunctions.jsonArrayAggAdd(list, element, nullClause);
+    JsonFunctions.jsonArrayAggAdd(list, element, nullClause);
     assertThat(
         invocationDesc(BuiltInMethod.JSON_ARRAYAGG_ADD.getMethodName(), list, element,
             nullClause),
@@ -793,28 +803,28 @@ public class SqlJsonFunctionsTest {
   private void assertIsJsonValue(String input,
       Matcher<? super Boolean> matcher) {
     assertThat(invocationDesc(BuiltInMethod.IS_JSON_VALUE.getMethodName(), input),
-        SqlFunctions.isJsonValue(input),
+        JsonFunctions.isJsonValue(input),
         matcher);
   }
 
   private void assertIsJsonScalar(String input,
       Matcher<? super Boolean> matcher) {
     assertThat(invocationDesc(BuiltInMethod.IS_JSON_SCALAR.getMethodName(), input),
-        SqlFunctions.isJsonScalar(input),
+        JsonFunctions.isJsonScalar(input),
         matcher);
   }
 
   private void assertIsJsonArray(String input,
       Matcher<? super Boolean> matcher) {
     assertThat(invocationDesc(BuiltInMethod.IS_JSON_ARRAY.getMethodName(), input),
-        SqlFunctions.isJsonArray(input),
+        JsonFunctions.isJsonArray(input),
         matcher);
   }
 
   private void assertIsJsonObject(String input,
       Matcher<? super Boolean> matcher) {
     assertThat(invocationDesc(BuiltInMethod.IS_JSON_OBJECT.getMethodName(), input),
-        SqlFunctions.isJsonObject(input),
+        JsonFunctions.isJsonObject(input),
         matcher);
   }
 
@@ -853,16 +863,16 @@ public class SqlJsonFunctionsTest {
     };
   }
 
-  @Nonnull private BaseMatcher<SqlFunctions.PathContext> contextMatches(
-      SqlFunctions.PathContext expected) {
-    return new BaseMatcher<SqlFunctions.PathContext>() {
+  @Nonnull private BaseMatcher<JsonFunctions.JsonPathContext> contextMatches(
+      JsonFunctions.JsonPathContext expected) {
+    return new BaseMatcher<JsonFunctions.JsonPathContext>() {
       @Override public boolean matches(Object item) {
-        if (!(item instanceof SqlFunctions.PathContext)) {
+        if (!(item instanceof JsonFunctions.JsonPathContext)) {
           return false;
         }
-        SqlFunctions.PathContext context = (SqlFunctions.PathContext) item;
+        JsonFunctions.JsonPathContext context = (JsonFunctions.JsonPathContext) item;
         if (Objects.equals(context.mode, expected.mode)
-            && Objects.equals(context.pathReturned, expected.pathReturned)) {
+            && Objects.equals(context.obj, expected.obj)) {
           if (context.exc == null && expected.exc == null) {
             return true;
           }
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index ce7e585..e0f9c16 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -8909,7 +8909,6 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         + "DEFAULT -\n"
         + "DOT -\n"
         + "ITEM -\n"
-        + "JSON_API_COMMON_SYNTAX -\n"
         + "NEXT_VALUE -\n"
         + "PATTERN_EXCLUDE -\n"
         + "PATTERN_PERMUTE -\n"
@@ -10930,6 +10929,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkExp("'{}' format json encoding utf16");
     checkExp("'{}' format json encoding utf32");
     checkExpType("'{}' format json", "ANY NOT NULL");
+    checkExpType("'null' format json", "ANY NOT NULL");
+    checkExpType("cast(null as varchar) format json", "ANY");
     checkExpFails("^null^ format json", "(?s).*Illegal use of .NULL.*");
   }
 
@@ -10997,7 +10998,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   @Test public void testJsonPretty() {
     check("select json_pretty(ename) from emp");
     checkExp("json_pretty('{\"foo\":\"bar\"}')");
-    checkExpType("json_pretty('{\"foo\":\"bar\"}')", "VARCHAR(2000) NOT NULL");
+    checkExpType("json_pretty('{\"foo\":\"bar\"}')", "VARCHAR(2000)");
     checkFails("select json_pretty(^NULL^) from emp", "(?s).*Illegal use of .NULL.*");
 
     if (!Bug.CALCITE_2869_FIXED) {
@@ -11014,7 +11015,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   @Test public void testJsonType() {
     check("select json_type(ename) from emp");
     checkExp("json_type('{\"foo\":\"bar\"}')");
-    checkExpType("json_type('{\"foo\":\"bar\"}')", "VARCHAR(20) NOT NULL");
+    checkExpType("json_type('{\"foo\":\"bar\"}')", "VARCHAR(20)");
 
     if (!Bug.CALCITE_2869_FIXED) {
       return;
@@ -11045,8 +11046,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
   @Test public void testJsonKeys() {
     checkExp("json_keys('{\"foo\":\"bar\"}', 'lax $')");
-    checkExpType("json_keys('{\"foo\":\"bar\"}', 'lax $')", "VARCHAR(2000) NOT NULL");
-    checkExpType("json_keys('{\"foo\":\"bar\"}', 'strict $')", "VARCHAR(2000) NOT NULL");
+    checkExpType("json_keys('{\"foo\":\"bar\"}', 'lax $')", "VARCHAR(2000)");
+    checkExpType("json_keys('{\"foo\":\"bar\"}', 'strict $')", "VARCHAR(2000)");
   }
 
   @Test public void testJsonObjectAgg() {
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 80f279b..2e52eb4 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -5630,7 +5630,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EXPR$0=[JSON_EXISTS(JSON_API_COMMON_SYNTAX(FORMAT JSON($1), 'lax $'))])
+LogicalProject(EXPR$0=[JSON_EXISTS($1, 'lax $')])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -5642,7 +5642,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EXPR$0=[CAST(JSON_VALUE_ANY(JSON_API_COMMON_SYNTAX(FORMAT JSON($1), 'lax $'), FLAG(SqlJsonValueEmptyOrErrorBehavior[NULL]), null:ANY, FLAG(SqlJsonValueEmptyOrErrorBehavior[NULL]), null:ANY)):VARCHAR(2000)])
+LogicalProject(EXPR$0=[CAST(JSON_VALUE_ANY($1, 'lax $', FLAG(SqlJsonValueEmptyOrErrorBehavior[NULL]), null:ANY, FLAG(SqlJsonValueEmptyOrErrorBehavior[NULL]), null:ANY)):VARCHAR(2000)])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -5654,7 +5654,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EXPR$0=[JSON_QUERY(JSON_API_COMMON_SYNTAX(FORMAT JSON($1), 'lax $'), FLAG(WITHOUT_ARRAY), FLAG(SqlJsonQueryEmptyOrErrorBehavior[NULL]), FLAG(SqlJsonQueryEmptyOrErrorBehavior[NULL]))])
+LogicalProject(EXPR$0=[JSON_QUERY($1, 'lax $', FLAG(WITHOUT_ARRAY), FLAG(SqlJsonQueryEmptyOrErrorBehavior[NULL]), FLAG(SqlJsonQueryEmptyOrErrorBehavior[NULL]))])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -5666,7 +5666,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EXPR$0=[JSON_TYPE(FORMAT JSON($1))])
+LogicalProject(EXPR$0=[JSON_TYPE($1)])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -5678,7 +5678,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EXPR$0=[JSON_PRETTY(FORMAT JSON($1))])
+LogicalProject(EXPR$0=[JSON_PRETTY($1)])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -5690,7 +5690,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EXPR$0=[JSON_DEPTH(FORMAT JSON($1))])
+LogicalProject(EXPR$0=[JSON_DEPTH($1)])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -5702,7 +5702,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EXPR$0=[JSON_LENGTH(FORMAT JSON($1), 'strict $')])
+LogicalProject(EXPR$0=[JSON_LENGTH($1, 'strict $')])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -5714,7 +5714,7 @@ from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(EXPR$0=[JSON_KEYS(FORMAT JSON($1), 'strict $')])
+LogicalProject(EXPR$0=[JSON_KEYS($1, 'strict $')])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index f8414e2..76d5f4b 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -2111,8 +2111,9 @@ semantics.
 
 Note:
 
-* `JSON_TYPE` / `JSON_DEPTH` return null if the argument is null
+* `JSON_TYPE` / `JSON_DEPTH` / `JSON_PRETTY` return null if the argument is null
 * `JSON_TYPE` / `JSON_DEPTH` / `JSON_PRETTY` throw error if the argument is not a valid JSON value
+* `JSON_LENGTH` / `JSON_KEYS` return null if the first argument is null
 * `JSON_TYPE` generally returns an upper-case string flag indicating the type of the JSON input. Currently supported supported type flags are:
   * INTEGER
   * STRING