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

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


The following commit(s) were added to refs/heads/master by this push:
     new d48e123d5a3 [Feature](json_functions) support json-remove (#55575)
d48e123d5a3 is described below

commit d48e123d5a3ccb0a9bc90aebb2f25d2b1eca3d1d
Author: lw112 <[email protected]>
AuthorDate: Thu Sep 11 17:54:57 2025 +0800

    [Feature](json_functions) support json-remove (#55575)
    
    ### What problem does this PR solve?
    
    support json_remove
    reference MYSQL:
    
https://dev.mysql.com/doc/refman/8.0/en/json-modification-functions.html#function_json-remove
---
 be/src/vec/functions/function_jsonb.cpp            | 332 ++++++++++++++++++++-
 .../doris/catalog/BuiltinScalarFunctions.java      |   2 +
 .../rules/expression/ExpressionRuleType.java       |   1 +
 .../expression/rules/JsonFunctionRewrite.java      |  23 +-
 .../expressions/functions/scalar/JsonRemove.java   |  71 +++++
 .../expressions/visitor/ScalarFunctionVisitor.java |   5 +
 .../json_functions/test_json_function.out          | Bin 2071 -> 2996 bytes
 .../json_functions/test_json_function.groovy       | 124 +++++++-
 8 files changed, 549 insertions(+), 9 deletions(-)

diff --git a/be/src/vec/functions/function_jsonb.cpp 
b/be/src/vec/functions/function_jsonb.cpp
index 562d0268568..f7ff6c33c0d 100644
--- a/be/src/vec/functions/function_jsonb.cpp
+++ b/be/src/vec/functions/function_jsonb.cpp
@@ -2179,7 +2179,7 @@ public:
                 | {"key": "value"}                         |
                 +------------------------------------------+
                 ```
-                So when inserting an element into `$[1]`, even if '$' does not 
represent an array, 
+                So when inserting an element into `$[1]`, even if '$' does not 
represent an array,
                 it should be converted to an array before insertion:
                 ```
                 select json_insert('123','$[1]', null);
@@ -2690,6 +2690,331 @@ public:
     }
 };
 
+struct DocumentBuffer {
+    std::unique_ptr<char[]> ptr;
+    size_t size = 0;
+    size_t capacity = 0;
+};
+
+class FunctionJsonbRemove : public IFunction {
+public:
+    static constexpr auto name = "jsonb_remove";
+    static constexpr auto alias = "json_remove";
+
+    static FunctionPtr create() { return 
std::make_shared<FunctionJsonbRemove>(); }
+
+    String get_name() const override { return name; }
+
+    size_t get_number_of_arguments() const override { return 0; }
+    bool is_variadic() const override { return true; }
+
+    bool use_default_implementation_for_nulls() const override { return false; 
}
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const 
override {
+        return make_nullable(std::make_shared<DataTypeJsonb>());
+    }
+
+    Status execute_impl(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
+                        uint32_t result, size_t input_rows_count) const 
override {
+        DCHECK_GE(arguments.size(), 2);
+
+        // Check if arguments count is valid (json_doc + at least one path)
+        if (arguments.size() < 2) {
+            return Status::InvalidArgument("json_remove requires at least 2 
arguments");
+        }
+
+        auto return_data_type = 
make_nullable(std::make_shared<DataTypeJsonb>());
+        auto result_column = return_data_type->create_column();
+        auto& nullable_column = assert_cast<ColumnNullable&>(*result_column);
+        auto& res_chars =
+                
assert_cast<ColumnString&>(nullable_column.get_nested_column()).get_chars();
+        auto& res_offsets =
+                
assert_cast<ColumnString&>(nullable_column.get_nested_column()).get_offsets();
+        auto& null_map = nullable_column.get_null_map_data();
+
+        res_chars.reserve(input_rows_count * 64);
+        res_offsets.resize(input_rows_count);
+        null_map.resize(input_rows_count);
+
+        // Get JSON document column
+        auto [json_column, json_const] =
+                unpack_if_const(block.get_by_position(arguments[0]).column);
+        const auto* json_nullable = 
check_and_get_column<ColumnNullable>(json_column.get());
+        const ColumnString* json_data_column = nullptr;
+        const NullMap* json_null_map = nullptr;
+
+        if (json_nullable) {
+            json_null_map = &json_nullable->get_null_map_data();
+            json_data_column =
+                    
check_and_get_column<ColumnString>(&json_nullable->get_nested_column());
+        } else {
+            json_data_column = 
check_and_get_column<ColumnString>(json_column.get());
+        }
+
+        if (!json_data_column) {
+            return Status::InvalidArgument("First argument must be a JSON 
document");
+        }
+
+        // Parse paths
+        std::vector<const ColumnString*> path_columns;
+        std::vector<const NullMap*> path_null_maps;
+        std::vector<bool> path_constants;
+
+        for (size_t i = 1; i < arguments.size(); ++i) {
+            auto [path_column, path_const] =
+                    
unpack_if_const(block.get_by_position(arguments[i]).column);
+            const auto* path_nullable = 
check_and_get_column<ColumnNullable>(path_column.get());
+
+            if (path_nullable) {
+                path_null_maps.push_back(&path_nullable->get_null_map_data());
+                path_columns.push_back(
+                        
check_and_get_column<ColumnString>(&path_nullable->get_nested_column()));
+            } else {
+                path_null_maps.push_back(nullptr);
+                
path_columns.push_back(check_and_get_column<ColumnString>(path_column.get()));
+            }
+
+            if (!path_columns.back()) {
+                return Status::InvalidArgument(
+                        fmt::format("Argument {} must be a string path", i + 
1));
+            }
+
+            path_constants.push_back(path_const);
+        }
+
+        // Reusable JsonbWriter for performance
+        JsonbWriter writer;
+
+        for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) {
+            size_t json_idx = index_check_const(row_idx, json_const);
+
+            // Check if JSON document is null
+            if (json_null_map && (*json_null_map)[json_idx]) {
+                null_map[row_idx] = 1;
+                res_offsets[row_idx] = static_cast<uint32_t>(res_chars.size());
+                continue;
+            }
+
+            // Parse JSON document
+            const auto& json_data = json_data_column->get_data_at(json_idx);
+            JsonbDocument* json_doc = nullptr;
+            Status parse_status = 
JsonbDocument::checkAndCreateDocument(json_data.data,
+                                                                        
json_data.size, &json_doc);
+
+            if (!parse_status.ok() || !json_doc) {
+                null_map[row_idx] = 1;
+                res_offsets[row_idx] = static_cast<uint32_t>(res_chars.size());
+                continue;
+            }
+
+            // Check if any path is null
+            bool has_null_path = false;
+            for (size_t path_idx = 0; path_idx < path_columns.size(); 
++path_idx) {
+                size_t idx = index_check_const(row_idx, 
path_constants[path_idx]);
+                if (path_null_maps[path_idx] && 
(*path_null_maps[path_idx])[idx]) {
+                    has_null_path = true;
+                    break;
+                }
+            }
+
+            if (has_null_path) {
+                null_map[row_idx] = 1;
+                res_offsets[row_idx] = static_cast<uint32_t>(res_chars.size());
+                continue;
+            }
+
+            std::vector<JsonbPath> paths;
+            std::vector<bool> path_constants_vec;
+
+            for (size_t path_idx = 0; path_idx < path_columns.size(); 
++path_idx) {
+                size_t idx = index_check_const(row_idx, 
path_constants[path_idx]);
+                const auto& path_data = 
path_columns[path_idx]->get_data_at(idx);
+
+                JsonbPath path;
+                if (!path.seek(path_data.data, path_data.size)) {
+                    return Status::InvalidArgument(
+                            "Json path error: Invalid Json Path for value: {} 
at row: {}",
+                            std::string_view(path_data.data, path_data.size), 
row_idx);
+                }
+
+                if (path.is_wildcard() || path.is_supper_wildcard()) {
+                    return Status::InvalidArgument(
+                            "In this situation, path expressions may not 
contain the * and ** "
+                            "tokens or an array range, argument index: {}, row 
index: {}",
+                            path_idx + 1, row_idx);
+                }
+
+                paths.push_back(std::move(path));
+                path_constants_vec.push_back(path_constants[path_idx]);
+            }
+
+            const JsonbValue* current_value = json_doc->getValue();
+
+            DocumentBuffer tmp_buffer;
+
+            for (size_t path_idx = 0; path_idx < paths.size(); ++path_idx) {
+                writer.reset();
+
+                auto find_result = current_value->findValue(paths[path_idx]);
+
+                if (find_result.is_wildcard) {
+                    continue;
+                }
+
+                if (find_result.value) {
+                    RETURN_IF_ERROR(clone_without_path(current_value, 
paths[path_idx], writer));
+
+                    auto* writer_output = writer.getOutput();
+                    if (writer_output->getSize() > tmp_buffer.capacity) {
+                        tmp_buffer.capacity =
+                                ((size_t(writer_output->getSize()) + 1024 - 1) 
/ 1024) * 1024;
+                        tmp_buffer.ptr = 
std::make_unique<char[]>(tmp_buffer.capacity);
+                        DCHECK_LE(writer_output->getSize(), 
tmp_buffer.capacity);
+                    }
+
+                    memcpy(tmp_buffer.ptr.get(), writer_output->getBuffer(),
+                           writer_output->getSize());
+                    tmp_buffer.size = writer_output->getSize();
+
+                    JsonbDocument* new_doc = nullptr;
+                    RETURN_IF_ERROR(JsonbDocument::checkAndCreateDocument(
+                            tmp_buffer.ptr.get(), tmp_buffer.size, &new_doc));
+
+                    current_value = new_doc->getValue();
+                }
+            }
+
+            JsonbDocument* modified_doc = nullptr;
+            if (current_value != json_doc->getValue()) {
+                RETURN_IF_ERROR(JsonbDocument::checkAndCreateDocument(
+                        tmp_buffer.ptr.get(), tmp_buffer.size, &modified_doc));
+            } else {
+                modified_doc = json_doc;
+            }
+
+            // Write the final result
+            const auto size = modified_doc->numPackedBytes();
+            res_chars.insert(reinterpret_cast<const char*>(modified_doc),
+                             reinterpret_cast<const char*>(modified_doc) + 
size);
+            res_offsets[row_idx] = static_cast<uint32_t>(res_chars.size());
+        }
+
+        block.get_by_position(result).column = std::move(result_column);
+        return Status::OK();
+    }
+
+private:
+    Status clone_without_path(const JsonbValue* root, const JsonbPath& path,
+                              JsonbWriter& writer) const {
+        // Start writing at the root level
+        if (root->isObject()) {
+            writer.writeStartObject();
+            RETURN_IF_ERROR(clone_object_without_path(root, path, 0, writer));
+            writer.writeEndObject();
+        } else if (root->isArray()) {
+            writer.writeStartArray();
+            RETURN_IF_ERROR(clone_array_without_path(root, path, 0, writer));
+            writer.writeEndArray();
+        } else {
+            // Primitive value - can't remove anything from it
+            writer.writeValue(root);
+        }
+        return Status::OK();
+    }
+
+    Status clone_object_without_path(const JsonbValue* obj_value, const 
JsonbPath& path,
+                                     size_t depth, JsonbWriter& writer) const {
+        const auto* obj = obj_value->unpack<ObjectVal>();
+
+        for (const auto& kv : *obj) {
+            std::string key(kv.getKeyStr(), kv.klen());
+
+            if (depth < path.get_leg_vector_size()) {
+                const auto* leg = path.get_leg_from_leg_vector(depth);
+                if (leg->type == MEMBER_CODE) {
+                    std::string target_key(leg->leg_ptr, leg->leg_len);
+
+                    if (key == target_key) {
+                        if (depth == path.get_leg_vector_size() - 1) {
+                            continue;
+                        } else {
+                            writer.writeKey(kv.getKeyStr(), kv.klen());
+                            if (kv.value()->isObject()) {
+                                writer.writeStartObject();
+                                
RETURN_IF_ERROR(clone_object_without_path(kv.value(), path,
+                                                                          
depth + 1, writer));
+                                writer.writeEndObject();
+                            } else if (kv.value()->isArray()) {
+                                writer.writeStartArray();
+                                
RETURN_IF_ERROR(clone_array_without_path(kv.value(), path,
+                                                                         depth 
+ 1, writer));
+                                writer.writeEndArray();
+                            } else {
+                                writer.writeValue(kv.value());
+                            }
+                        }
+                    } else {
+                        writer.writeKey(kv.getKeyStr(), kv.klen());
+                        writer.writeValue(kv.value());
+                    }
+                } else {
+                    writer.writeKey(kv.getKeyStr(), kv.klen());
+                    writer.writeValue(kv.value());
+                }
+            } else {
+                writer.writeKey(kv.getKeyStr(), kv.klen());
+                writer.writeValue(kv.value());
+            }
+        }
+
+        return Status::OK();
+    }
+
+    Status clone_array_without_path(const JsonbValue* arr_value, const 
JsonbPath& path,
+                                    size_t depth, JsonbWriter& writer) const {
+        const auto* arr = arr_value->unpack<ArrayVal>();
+
+        int index = 0;
+        for (const auto& element : *arr) {
+            if (depth < path.get_leg_vector_size()) {
+                const auto* leg = path.get_leg_from_leg_vector(depth);
+                if (leg->type == ARRAY_CODE) {
+                    int target_index = leg->array_index;
+
+                    if (index == target_index) {
+                        if (depth == path.get_leg_vector_size() - 1) {
+                            // This is the target element to remove - skip it
+                        } else {
+                            if (element.isObject()) {
+                                writer.writeStartObject();
+                                
RETURN_IF_ERROR(clone_object_without_path(&element, path, depth + 1,
+                                                                          
writer));
+                                writer.writeEndObject();
+                            } else if (element.isArray()) {
+                                writer.writeStartArray();
+                                
RETURN_IF_ERROR(clone_array_without_path(&element, path, depth + 1,
+                                                                         
writer));
+                                writer.writeEndArray();
+                            } else {
+                                writer.writeValue(&element);
+                            }
+                        }
+                    } else {
+                        writer.writeValue(&element);
+                    }
+                } else {
+                    writer.writeValue(&element);
+                }
+            } else {
+                writer.writeValue(&element);
+            }
+            index++;
+        }
+
+        return Status::OK();
+    }
+};
+
 void register_function_jsonb(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionJsonbParse>(FunctionJsonbParse::name);
     factory.register_alias(FunctionJsonbParse::name, 
FunctionJsonbParse::alias);
@@ -2739,6 +3064,9 @@ void register_function_jsonb(SimpleFunctionFactory& 
factory) {
     factory.register_function<FunctionJsonbModify<JsonbModifyType::Replace>>();
     factory.register_alias(FunctionJsonbModify<JsonbModifyType::Replace>::name,
                            
FunctionJsonbModify<JsonbModifyType::Replace>::alias);
+
+    factory.register_function<FunctionJsonbRemove>();
+    factory.register_alias(FunctionJsonbRemove::name, 
FunctionJsonbRemove::alias);
 }
 
-} // namespace doris::vectorized
+} // namespace doris::vectorized
\ No newline at end of file
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
index 278d5c42252..bf6b4e8d445 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
@@ -268,6 +268,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.JsonKeys;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonLength;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonObject;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonQuote;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonRemove;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonReplace;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonSearch;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonSet;
@@ -764,6 +765,7 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(JsonInsert.class, "json_insert", "jsonb_insert"),
             scalar(JsonReplace.class, "json_replace", "jsonb_replace"),
             scalar(JsonSet.class, "json_set", "jsonb_set"),
+            scalar(JsonRemove.class, "json_remove"),
             scalar(JsonbExistsPath.class, "json_exists_path"),
             scalar(JsonbExistsPath.class, "jsonb_exists_path"),
             scalar(JsonbExtract.class, "jsonb_extract", "json_extract"),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java
index 8094ec5dac0..f32273f3a35 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java
@@ -68,6 +68,7 @@ public enum ExpressionRuleType {
     JSON_FUNCTION_REWRITE_JSON_EXTRACT_LARGEINT,
     JSON_FUNCTION_REWRITE_JSON_EXTRACT_BOOLEAN,
     JSON_FUNCTION_REWRITE_JSON_EXTRACT_DOUBLE,
+    JSON_FUNCTION_REWRITE_JSON_REMOVE,
     JSON_FUNCTION_REWRITE_JSON_EXTRACT_STRING;
 
     public int type() {
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/JsonFunctionRewrite.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/JsonFunctionRewrite.java
index 8ddf423755c..00de1b64760 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/JsonFunctionRewrite.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/JsonFunctionRewrite.java
@@ -26,6 +26,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.JsonArray;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.JsonArrayIgnoreNull;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonInsert;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonObject;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonRemove;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonReplace;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonSet;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtract;
@@ -60,8 +61,7 @@ public class JsonFunctionRewrite implements 
ExpressionPatternRuleFactory {
 
     @Override
     public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
-        return ImmutableList.of(
-                
matchesType(JsonArray.class).then(JsonFunctionRewrite::rewriteJsonArrayArguments)
+        return 
ImmutableList.of(matchesType(JsonArray.class).then(JsonFunctionRewrite::rewriteJsonArrayArguments)
                         
.toRule(ExpressionRuleType.JSON_FUNCTION_REWRITE_JSON_ARRAY),
                 
matchesType(JsonArrayIgnoreNull.class).then(JsonFunctionRewrite::rewriteJsonArrayArguments)
                         
.toRule(ExpressionRuleType.JSON_FUNCTION_REWRITE_JSON_ARRAY_IGNORE_NULL),
@@ -73,6 +73,8 @@ public class JsonFunctionRewrite implements 
ExpressionPatternRuleFactory {
                         
.toRule(ExpressionRuleType.JSON_FUNCTION_REWRITE_JSON_SET),
                 
matchesType(JsonReplace.class).then(JsonFunctionRewrite::rewriteJsonModifyArguments)
                         
.toRule(ExpressionRuleType.JSON_FUNCTION_REWRITE_JSON_REPLACE),
+                
matchesType(JsonRemove.class).then(JsonFunctionRewrite::rewriteJsonRemoveArguments)
+                        
.toRule(ExpressionRuleType.JSON_FUNCTION_REWRITE_JSON_REMOVE),
                 
matchesType(JsonbExtractInt.class).then(JsonFunctionRewrite::rewriteJsonExtractFunctions)
                         
.toRule(ExpressionRuleType.JSON_FUNCTION_REWRITE_JSON_EXTRACT_INT),
                 
matchesType(JsonbExtractBigint.class).then(JsonFunctionRewrite::rewriteJsonExtractFunctions)
@@ -84,8 +86,7 @@ public class JsonFunctionRewrite implements 
ExpressionPatternRuleFactory {
                 
matchesType(JsonbExtractDouble.class).then(JsonFunctionRewrite::rewriteJsonExtractFunctions)
                         
.toRule(ExpressionRuleType.JSON_FUNCTION_REWRITE_JSON_EXTRACT_DOUBLE),
                 
matchesType(JsonbExtractString.class).then(JsonFunctionRewrite::rewriteJsonExtractFunctions)
-                        
.toRule(ExpressionRuleType.JSON_FUNCTION_REWRITE_JSON_EXTRACT_STRING)
-        );
+                        
.toRule(ExpressionRuleType.JSON_FUNCTION_REWRITE_JSON_EXTRACT_STRING));
     }
 
     private static <T extends ScalarFunction> Expression 
rewriteJsonArrayArguments(T function) {
@@ -140,6 +141,20 @@ public class JsonFunctionRewrite implements 
ExpressionPatternRuleFactory {
         });
     }
 
+    private static <T extends ScalarFunction> Expression 
rewriteJsonRemoveArguments(T function) {
+        return MoreFieldsThread.keepFunctionSignature(false, () -> {
+            List<Expression> convectedChildren = new ArrayList<>();
+            List<Expression> children = function.children();
+
+            convectedChildren.add(children.get(0));
+
+            for (int i = 1; i < children.size(); i++) {
+                convectedChildren.add(children.get(i));
+            }
+            return function.withChildren(convectedChildren);
+        });
+    }
+
     private static <T extends ScalarFunction> Expression 
rewriteJsonExtractFunctions(T function) {
         JsonbExtract jsonExtract = new 
JsonbExtract(function.children().get(0), function.children().get(1));
         if (function instanceof JsonbExtractInt) {
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonRemove.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonRemove.java
new file mode 100644
index 00000000000..b2bc873cbc3
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonRemove.java
@@ -0,0 +1,71 @@
+// 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.doris.nereids.trees.expressions.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.CustomSignature;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DataType;
+import org.apache.doris.nereids.types.JsonType;
+import org.apache.doris.nereids.types.VarcharType;
+import org.apache.doris.nereids.util.ExpressionUtils;
+
+import com.google.common.base.Preconditions;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * ScalarFunction 'json_remove'. This class implements JSON_REMOVE function.
+ */
+public class JsonRemove extends ScalarFunction implements AlwaysNullable, 
CustomSignature {
+    /**
+     * constructor with 2 or more arguments.
+     */
+    public JsonRemove(Expression arg0, Expression arg1, Expression... varArgs) 
{
+        super("json_remove", ExpressionUtils.mergeArguments(arg0, arg1, 
varArgs));
+    }
+
+    // constructor for withChildren and reuse signature
+    private JsonRemove(ScalarFunctionParams functionParams) {
+        super(functionParams);
+    }
+
+    @Override
+    public FunctionSignature customSignature() {
+        List<DataType> arguments = new ArrayList<>();
+        arguments.add(JsonType.INSTANCE);
+        for (int i = 1; i < arity(); i++) {
+            arguments.add(VarcharType.SYSTEM_DEFAULT);
+        }
+        return FunctionSignature.of(JsonType.INSTANCE, arguments);
+    }
+
+    @Override
+    public JsonRemove withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() >= 2);
+        return new JsonRemove(getFunctionParams(children));
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitJsonRemove(this, context);
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
index b52c8bda9da..ee327402673 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
@@ -273,6 +273,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.JsonKeys;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonLength;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonObject;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonQuote;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonRemove;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonReplace;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonSearch;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonSet;
@@ -1508,6 +1509,10 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(jsonSet, context);
     }
 
+    default R visitJsonRemove(JsonRemove jsonRemove, C context) {
+        return visitScalarFunction(jsonRemove, context);
+    }
+
     default R visitJsonQuote(JsonQuote jsonQuote, C context) {
         return visitScalarFunction(jsonQuote, context);
     }
diff --git 
a/regression-test/data/query_p0/sql_functions/json_functions/test_json_function.out
 
b/regression-test/data/query_p0/sql_functions/json_functions/test_json_function.out
index a763f1bd6a1..29559063b3e 100644
Binary files 
a/regression-test/data/query_p0/sql_functions/json_functions/test_json_function.out
 and 
b/regression-test/data/query_p0/sql_functions/json_functions/test_json_function.out
 differ
diff --git 
a/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy
 
b/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy
index a86fee5312f..31f9010be3d 100644
--- 
a/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy
+++ 
b/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy
@@ -79,7 +79,7 @@ suite("test_json_function", "arrow_flight_sql") {
     qt_sql """SELECT JSON_CONTAINS("",'1','\$.a')"""
 
     qt_sql """select k6, json_extract_string(cast(k7 as json), "\$.a") as x10 
from test_query_db.baseall group by k6, x10 order by 1,2; """
-    
+
     qt_sql "SELECT json_extract_no_quotes('[1, 2, 3]', '\$.[1]');"
     qt_sql "SELECT json_extract_no_quotes('{\"id\": 123, \"name\": 
\"doris\"}', '\$.name');"
     qt_sql "SELECT json_extract_no_quotes('{\"id\": 123, \"name\": 
\"doris\"}', '\$.id', null);"
@@ -102,7 +102,7 @@ suite("test_json_function", "arrow_flight_sql") {
         distributed BY hash(k1) buckets 3
         properties("replication_num" = "1");
     """
-    sql """insert into d_table values 
+    sql """insert into d_table values
     ('{\"a\": 1, \"b\": 2, \"c\": {\"d\": 4}}', '{\"a\": 1, \"b\": 2, \"c\": 
{\"d\": 4}}'),
     ('{\"a\": 1, \"b\": 2, \"c\": {\"d\": 4}}', '{\"a\": 1, \"b\": 2, \"c\": 
{\"d\": 5}}'),
     ('{\"a\": 1, \"b\": 2, \"c\": {\"d\": 4}}', '{\"a\": 1, \"b\": 2, \"c\": 
{\"d\": 6}}'),
@@ -149,7 +149,7 @@ suite("test_json_function", "arrow_flight_sql") {
 
         exception "In this situation, path expressions may not contain the * 
and ** tokens or an array range."
     }
-    
+
     test {
         sql """
             SELECT JSON_KEYS(k1, '\$.*.c') FROM d_table order by k1;
@@ -165,4 +165,122 @@ suite("test_json_function", "arrow_flight_sql") {
 
         exception "Json path error: Invalid Json Path for value: \$**"
     }
+
+    qt_json_remove1 """
+      SELECT JSON_REMOVE('{"a": 1, "b": 2, "c": 3}', '\$.b');
+    """
+
+    qt_json_remove2 """
+      SELECT JSON_REMOVE('{"Name": "Homer", "Gender": "Male", "Age": 39}', 
'\$.Age');
+    """
+
+    qt_json_remove3 """
+      SELECT JSON_REMOVE('{"Name": "Homer", "Age": 39}', '\$.Gender');
+    """
+
+    qt_json_remove4 """
+      SELECT JSON_REMOVE('[1, 2, 3]', '\$[0]');
+    """
+
+    qt_json_remove5 """
+      SELECT JSON_REMOVE('[1, 2, [3, 4, 5]]', '\$[2][1]');
+    """
+
+    qt_json_remove6 """
+      SELECT JSON_REMOVE('[1, 2, 3, 4, 5]', '\$[1]', '\$[3]');
+    """
+
+    qt_json_remove7 """
+      SELECT JSON_REMOVE('[1, 2, 3, 4, 5]', '\$[3]');
+    """
+
+    qt_json_remove8 """
+      SELECT JSON_REMOVE('[1, 2, [3, 4, 5]]', '\$[0]', '\$[1][1]');
+    """
+
+    qt_json_remove9 """
+      SELECT JSON_REMOVE('[1, 2, [3, 4, 5]]', '\$[2][1]', '\$[0]');
+    """
+
+    qt_json_remove10 """
+      SELECT JSON_REMOVE('{"Person": {"Name": "Homer","Age": 39,"Hobbies": 
["Eating", "Sleeping", "Base Jumping"]}}', '\$.Person.Age', 
'\$.Person.Hobbies[2]');
+    """
+
+    qt_json_remove11 """
+      SELECT JSON_REMOVE(NULL, '\$.a');
+    """
+
+    qt_json_remove12 """
+      SELECT JSON_REMOVE('{"a": 1, "b": 2}', NULL);
+    """
+
+    qt_json_remove13 """
+      SELECT JSON_REMOVE(NULL, NULL);
+    """
+
+    qt_json_remove14 """
+      SELECT JSON_REMOVE('{"a": 1, "b": 2}', '\$.a', NULL, '\$.b');
+    """
+
+    sql "drop table if exists json_remove_test_table;"
+    sql """
+        create table json_remove_test_table (
+            id int,
+            json_data varchar(500) null,
+            path1 varchar(100),
+            path2 varchar(100)
+        )
+        duplicate key (id)
+        distributed BY hash(id) buckets 1
+        properties("replication_num" = "1");
+    """
+
+    sql """insert into json_remove_test_table values
+    (1, '{"name": "Alice", "age": 25, "skills": ["Java", "Python", "SQL"]}', 
'\$.age', '\$.skills[1]'),
+    (2, '{"product": "laptop", "price": 999, "specs": {"cpu": "Intel", "ram": 
"16GB"}}', '\$.price', '\$.specs.ram'),
+    (3, '{"users": [{"id": 1, "name": "Bob"}, {"id": 2, "name": "Charlie"}]}', 
'\$.users[0].id', '\$.users[1]'),
+    (4, '{"empty": {}}', '\$.empty', '\$.nonexistent');
+    """
+
+    qt_json_remove15 """
+      SELECT JSON_REMOVE(json_data, path1) FROM json_remove_test_table ORDER 
BY id;
+    """
+
+    qt_json_remove16 """
+      SELECT JSON_REMOVE(json_data, path1, path2) FROM json_remove_test_table 
ORDER BY id;
+    """
+
+    // returning errors
+    test {
+        sql """
+            SELECT JSON_REMOVE('{"a": 1}', '\$.*');
+        """
+        exception "In this situation, path expressions may not contain the * 
and ** tokens or an array range"
+    }
+
+    test {
+        sql """
+            SELECT JSON_REMOVE('{"a": 1}', '\$**');
+        """
+        exception "Json path error: Invalid Json Path for value: \$**"
+    }
+
+    // edge cases
+    qt_json_remove17 """
+      SELECT JSON_REMOVE('{}', '\$.nonexistent');
+    """
+
+    qt_json_remove18 """
+      SELECT JSON_REMOVE('[]', '\$[0]');
+    """
+
+    qt_json_remove19 """
+      SELECT JSON_REMOVE('123', '\$.key');
+    """
+
+    qt_json_remove20 """
+      SELECT JSON_REMOVE('true', '\$.key');
+    """
+
+    sql "drop table if exists json_remove_test_table;"
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to