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/16 14:26:08 UTC

[calcite] branch master updated: [CALCITE-2975] Implement JSON_REMOVE function (xuqianjin)

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 999c41d  [CALCITE-2975] Implement JSON_REMOVE function (xuqianjin)
999c41d is described below

commit 999c41d5ff893a30b2622442a448102c72fc475e
Author: XuQianJin-Stars <x1...@163.com>
AuthorDate: Wed Apr 3 23:32:08 2019 +0800

    [CALCITE-2975] Implement JSON_REMOVE function (xuqianjin)
    
    Close apache/calcite#1146
---
 .../calcite/adapter/enumerable/RexImpTable.java    |  2 +
 .../apache/calcite/runtime/CalciteResource.java    |  5 +-
 .../org/apache/calcite/runtime/JsonFunctions.java  | 26 ++++++++
 .../calcite/sql/fun/SqlJsonRemoveFunction.java     | 73 ++++++++++++++++++++++
 .../calcite/sql/fun/SqlStdOperatorTable.java       |  2 +
 .../org/apache/calcite/util/BuiltInMethod.java     |  1 +
 .../calcite/runtime/CalciteResource.properties     |  3 +-
 .../calcite/rel/rel2sql/RelToSqlConverterTest.java |  7 +++
 .../apache/calcite/sql/parser/SqlParserTest.java   |  7 +++
 .../calcite/sql/test/SqlOperatorBaseTest.java      | 20 ++++++
 .../java/org/apache/calcite/test/JdbcTest.java     |  8 +++
 .../apache/calcite/test/SqlJsonFunctionsTest.java  | 21 ++++++-
 .../org/apache/calcite/test/SqlValidatorTest.java  |  7 +++
 site/_docs/reference.md                            | 20 +++++-
 14 files changed, 197 insertions(+), 5 deletions(-)

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 5a9f835..c5237ee 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
@@ -172,6 +172,7 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_EXISTS;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECT;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECTAGG;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_QUERY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_REMOVE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_VALUE_ANY;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_VALUE_EXPRESSION;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LAG;
@@ -462,6 +463,7 @@ public class RexImpTable {
     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_REMOVE, BuiltInMethod.JSON_REMOVE.method, NullPolicy.ARG0);
     defineMethod(JSON_OBJECT, BuiltInMethod.JSON_OBJECT.method, NullPolicy.NONE);
     defineMethod(JSON_ARRAY, BuiltInMethod.JSON_ARRAY.method, NullPolicy.NONE);
     aggMap.put(JSON_OBJECTAGG.with(SqlJsonConstructorNullClause.ABSENT_ON_NULL),
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 3d12aee..3a4033b 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -874,7 +874,7 @@ public interface CalciteResource {
   @BaseMessage("Not a valid input for JSON_DEPTH: ''{0}''")
   ExInst<CalciteException> invalidInputForJsonDepth(String value);
 
-  @BaseMessage("Cannot serialize object to JSON, and the object is: ''{0}''")
+  @BaseMessage("Cannot serialize object to JSON: ''{0}''")
   ExInst<CalciteException> exceptionWhileSerializingToJson(String value);
 
   @BaseMessage("Not a valid input for JSON_LENGTH: ''{0}''")
@@ -882,6 +882,9 @@ public interface CalciteResource {
 
   @BaseMessage("Not a valid input for JSON_KEYS: ''{0}''")
   ExInst<CalciteException> invalidInputForJsonKeys(String value);
+
+  @BaseMessage("Invalid input for JSON_REMOVE: document: ''{0}'', jsonpath expressions: ''{1}''")
+  ExInst<CalciteException> invalidInputForJsonRemove(String value, String pathSpecs);
 }
 
 // End CalciteResource.java
diff --git a/core/src/main/java/org/apache/calcite/runtime/JsonFunctions.java b/core/src/main/java/org/apache/calcite/runtime/JsonFunctions.java
index c268708..cf38f8c 100644
--- a/core/src/main/java/org/apache/calcite/runtime/JsonFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/JsonFunctions.java
@@ -36,6 +36,7 @@ import com.jayway.jsonpath.spi.mapper.JacksonMappingProvider;
 import com.jayway.jsonpath.spi.mapper.MappingProvider;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
@@ -605,6 +606,31 @@ public class JsonFunctions {
     return jsonize(list);
   }
 
+  public static String jsonRemove(String input, String... pathSpecs) {
+    return jsonRemove(jsonValueExpression(input), pathSpecs);
+  }
+
+  public static String jsonRemove(JsonValueContext input, String... pathSpecs) {
+    try {
+      DocumentContext ctx = JsonPath.parse(input.obj,
+          Configuration
+              .builder()
+              .options(Option.SUPPRESS_EXCEPTIONS)
+              .jsonProvider(JSON_PATH_JSON_PROVIDER)
+              .mappingProvider(JSON_PATH_MAPPING_PROVIDER)
+              .build());
+      for (String pathSpec : pathSpecs) {
+        if ((pathSpec != null) && (ctx.read(pathSpec) != null)) {
+          ctx.delete(pathSpec);
+        }
+      }
+      return ctx.jsonString();
+    } catch (Exception ex) {
+      throw RESOURCE.invalidInputForJsonRemove(
+          input.toString(), Arrays.toString(pathSpecs)).ex();
+    }
+  }
+
   public static boolean isJsonValue(String input) {
     try {
       dejsonize(input);
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonRemoveFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonRemoveFunction.java
new file mode 100644
index 0000000..3e7cb85
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonRemoveFunction.java
@@ -0,0 +1,73 @@
+/*
+ * 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.SqlCallBinding;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperandCountRange;
+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.SqlTypeTransforms;
+
+import java.util.Locale;
+
+/**
+ * The <code>JSON_REMOVE</code> function.
+ */
+public class SqlJsonRemoveFunction extends SqlFunction {
+
+  public SqlJsonRemoveFunction() {
+    super("JSON_REMOVE",
+        SqlKind.OTHER_FUNCTION,
+        ReturnTypes.cascade(ReturnTypes.VARCHAR_2000,
+            SqlTypeTransforms.FORCE_NULLABLE),
+        null,
+        null,
+        SqlFunctionCategory.SYSTEM);
+  }
+
+  @Override public SqlOperandCountRange getOperandCountRange() {
+    return SqlOperandCountRanges.from(2);
+  }
+
+  @Override public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
+    final int operandCount = callBinding.getOperandCount();
+    assert operandCount >= 2;
+    if (!OperandTypes.ANY.checkSingleOperandType(
+        callBinding, callBinding.operand(0), 0, throwOnFailure)) {
+      return false;
+    }
+    for (int i = 1; i < operandCount; i++) {
+      if (!OperandTypes.CHARACTER.checkSingleOperandType(
+          callBinding, callBinding.operand(i), 0, throwOnFailure)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override public String getAllowedSignatures(String opNameToUse) {
+    return String.format(Locale.ROOT, "'%s(<%s>, <%s>, <%s>...)'", getName(), SqlTypeFamily.ANY,
+        SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER);
+  }
+}
+
+// End SqlJsonRemoveFunction.java
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index 3bb0640..2c6fe37 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
@@ -1317,6 +1317,8 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
 
   public static final SqlFunction JSON_LENGTH = new SqlJsonLengthFunction();
 
+  public static final SqlFunction JSON_REMOVE = new SqlJsonRemoveFunction();
+
   public static final SqlJsonObjectAggAggFunction JSON_OBJECTAGG =
       new SqlJsonObjectAggAggFunction(SqlKind.JSON_OBJECTAGG,
           SqlJsonConstructorNullClause.NULL_ON_NULL);
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 04d936c..ac5f9bd 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -291,6 +291,7 @@ public enum BuiltInMethod {
   JSON_KEYS(JsonFunctions.class, "jsonKeys", String.class),
   JSON_PRETTY(JsonFunctions.class, "jsonPretty", String.class),
   JSON_LENGTH(JsonFunctions.class, "jsonLength", String.class),
+  JSON_REMOVE(JsonFunctions.class, "jsonRemove", String.class),
   JSON_OBJECTAGG_ADD(JsonFunctions.class, "jsonObjectAggAdd", Map.class,
       String.class, Object.class, SqlJsonConstructorNullClause.class),
   JSON_ARRAY(JsonFunctions.class, "jsonArray",
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index d9ebc4b..bd33f6f 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -284,7 +284,8 @@ AmbiguousSortOrderInJsonArrayAggFunc=Including both WITHIN GROUP(...) and inside
 ExceptionWhilePerformingQueryOnJdbcSubSchema = While executing SQL [{0}] on JDBC sub-schema
 InvalidInputForJsonType=Not a valid input for JSON_TYPE: ''{0}''
 InvalidInputForJsonDepth=Not a valid input for JSON_DEPTH: ''{0}''
-ExceptionWhileSerializingToJson=Cannot serialize object to JSON, and the object is: ''{0}''
+ExceptionWhileSerializingToJson=Cannot serialize object to JSON: ''{0}''
 InvalidInputForJsonLength=Not a valid input for JSON_LENGTH: ''{0}''
 InvalidInputForJsonKeys=Not a valid input for JSON_KEYS: ''{0}''
+InvalidInputForJsonRemove=Invalid input for JSON_REMOVE: document: ''{0}'', jsonpath expressions: ''{1}''
 # End CalciteResource.properties
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 e4fe971..2ebfadc 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
@@ -3297,6 +3297,13 @@ public class RelToSqlConverterTest {
     sql(query).ok(expected);
   }
 
+  @Test public void testJsonRemove() {
+    String query = "select json_remove(\"product_name\", '$[0]') from \"product\"";
+    final String expected = "SELECT JSON_REMOVE(\"product_name\", '$[0]')\n"
+           + "FROM \"foodmart\".\"product\"";
+    sql(query).ok(expected);
+  }
+
   /** Fluid interface to run tests. */
   static class Sql {
     private final SchemaPlus schema;
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index cd973c6..f2af797 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -8634,6 +8634,13 @@ public class SqlParserTest {
             "JSON_KEYS('{\"foo\": \"bar\"}', 'invalid $')");
   }
 
+  @Test public void testJsonRemove() {
+    checkExp("json_remove('[\"a\", [\"b\", \"c\"], \"d\"]', '$')",
+            "JSON_REMOVE('[\"a\", [\"b\", \"c\"], \"d\"]', '$')");
+    checkExp("json_remove('[\"a\", [\"b\", \"c\"], \"d\"]', '$[1]', '$[0]')",
+            "JSON_REMOVE('[\"a\", [\"b\", \"c\"], \"d\"]', '$[1]', '$[0]')");
+  }
+
   @Test public void testJsonObjectAgg() {
     checkExp("json_objectagg(k_column: v_column)",
         "JSON_OBJECTAGG(KEY `K_COLUMN` VALUE `V_COLUMN` NULL ON NULL)");
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 ecca87d..dedb9a9 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
@@ -4664,6 +4664,26 @@ public abstract class SqlOperatorBaseTest {
     tester.checkNull("json_keys(cast(null as varchar))");
   }
 
+  @Test public void testJsonRemove() {
+    tester.checkString("json_remove('{\"foo\":100}', '$.foo')",
+        "{}", "VARCHAR(2000)");
+    tester.checkString("json_remove('{\"foo\":100, \"foo1\":100}', '$.foo')",
+        "{\"foo1\":100}", "VARCHAR(2000)");
+    tester.checkString("json_remove('[\"a\", [\"b\", \"c\"], \"d\"]', '$[1][0]')",
+        "[\"a\",[\"c\"],\"d\"]", "VARCHAR(2000)");
+    tester.checkString("json_remove('[\"a\", [\"b\", \"c\"], \"d\"]', '$[1]')",
+        "[\"a\",\"d\"]", "VARCHAR(2000)");
+    tester.checkString("json_remove('[\"a\", [\"b\", \"c\"], \"d\"]', '$[0]', '$[0]')",
+        "[\"d\"]", "VARCHAR(2000)");
+    tester.checkFails("json_remove('[\"a\", [\"b\", \"c\"], \"d\"]', '$')",
+        "(?s).*Invalid input for.*", true);
+
+    // nulls
+    tester.checkFails("json_remove(^null^, '$')",
+        "(?s).*Illegal use of 'NULL'.*", false);
+    tester.checkNull("json_remove(cast(null as varchar), '$')");
+  }
+
   @Test public void testJsonObject() {
     tester.checkString("json_object()", "{}", "VARCHAR(2000) NOT NULL");
     tester.checkString("json_object('foo': 'bar')",
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index b2e31ff..2d5c78e 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -6891,6 +6891,14 @@ public class JdbcTest {
         .returns("C1=[\"a\",\"b\"]; C2=null; C3=[\"c\"]; C4=null; C5=null\n");
   }
 
+  @Test public void testJsonRemove() {
+    CalciteAssert.that()
+        .query("SELECT JSON_REMOVE(v, '$[1]') AS c1\n"
+            + "FROM (VALUES ('[\"a\", [\"b\", \"c\"], \"d\"]')) AS t(v)\n"
+            + "limit 10")
+        .returns("C1=[\"a\",\"d\"]\n");
+  }
+
   /**
    * Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-2609">[CALCITE-2609]
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 3126ff1..7c85f70 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlJsonFunctionsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlJsonFunctionsTest.java
@@ -430,7 +430,7 @@ public class SqlJsonFunctionsTest {
       private final Object self = this;
     };
     CalciteException expected = new CalciteException(
-        "Cannot serialize object to JSON, and the object is: '" + input + "'", null);
+        "Cannot serialize object to JSON: '" + input + "'", null);
     assertJsonPrettyFailed(
         JsonFunctions.JsonValueContext.withJavaObj(input), errorMatches(expected));
   }
@@ -532,6 +532,18 @@ public class SqlJsonFunctionsTest {
   }
 
   @Test
+  public void testJsonRemove() {
+    assertJsonRemove(
+        JsonFunctions.jsonValueExpression("{\"a\": 1, \"b\": [2]}"),
+        new String[]{"$.a"},
+        is("{\"b\":[2]}"));
+    assertJsonRemove(
+        JsonFunctions.jsonValueExpression("{\"a\": 1, \"b\": [2]}"),
+        new String[]{"$.a", "$.b"},
+        is("{}"));
+  }
+
+  @Test
   public void testJsonObjectAggAdd() {
     Map<String, Object> map = new HashMap<>();
     Map<String, Object> expected = new HashMap<>();
@@ -736,6 +748,13 @@ public class SqlJsonFunctionsTest {
         matcher);
   }
 
+  private void assertJsonRemove(JsonFunctions.JsonValueContext input, String[] pathSpecs,
+      Matcher<? super String> matcher) {
+    assertThat(invocationDesc(BuiltInMethod.JSON_REMOVE.getMethodName(), input, pathSpecs),
+             JsonFunctions.jsonRemove(input, pathSpecs),
+             matcher);
+  }
+
   private void assertDejsonize(String input,
       Matcher<Object> matcher) {
     assertThat(invocationDesc(BuiltInMethod.DEJSONIZE.getMethodName(), input),
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 aa7ea14..ca428b4 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -11049,6 +11049,13 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkExpType("json_keys('{\"foo\":\"bar\"}', 'strict $')", "VARCHAR(2000)");
   }
 
+  @Test public void testJsonRemove() {
+    checkExp("json_remove('{\"foo\":\"bar\"}', '$')");
+    checkExpType("json_remove('{\"foo\":\"bar\"}', '$')", "VARCHAR(2000)");
+    checkFails("select ^json_remove('{\"foo\":\"bar\"}')^",
+            "(?s).*Invalid number of arguments.*");
+  }
+
   @Test public void testJsonObjectAgg() {
     check("select json_objectagg(ename: empno) from emp");
     checkFails("select ^json_objectagg(empno: ename)^ from emp",
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 76d5f4b..17c9588 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -2100,6 +2100,7 @@ semantics.
 | m | JSON_PRETTY(jsonValue)                         | Returns a pretty-printing of *jsonValue*
 | m | JSON_LENGTH(jsonValue [, path ])               | Returns a integer indicating the length of *jsonValue*
 | m | JSON_KEYS(jsonValue [, path ])                 | Returns a string indicating the keys of a JSON *jsonValue*
+| m | JSON_REMOVE(jsonValue, path[, path])           | Removes data from *jsonValue* using a series of *path* expressions and returns the result
 | o | DECODE(value, value1, result1 [, valueN, resultN ]* [, default ]) | Compares *value* to each *valueN* value one by one; if *value* is equal to a *valueN*, returns the corresponding *resultN*, else returns *default*, or NULL if *default* is not specified
 | o | NVL(value1, value2)                            | Returns *value1* if *value1* is not null, otherwise *value2*
 | o | LTRIM(string)                                  | Returns *string* with all blanks removed from the start
@@ -2113,7 +2114,7 @@ Note:
 
 * `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_LENGTH` / `JSON_KEYS` / `JSON_REMOVE` 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
@@ -2212,6 +2213,22 @@ LIMIT 10;
 | ---------- | ---- | ----- | ---- | ---- |
 | ["a", "b"] | NULL | ["c"] | NULL | NULL |
 
+##### JSON_REMOVE example
+
+SQL
+
+ ```SQL
+SELECT JSON_REMOVE(v, '$[1]') AS c1
+FROM (VALUES ('["a", ["b", "c"], "d"]')) AS t(v)
+LIMIT 10;
+```
+
+ Result
+
+| c1         |
+| ---------- |
+| ["a", "d"] |
+
 #### DECODE example
 
 SQL
@@ -2254,7 +2271,6 @@ Not implemented:
 * JSON_INSERT
 * JSON_SET
 * JSON_REPLACE
-* JSON_REMOVE
 
 ## User-defined functions