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

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


The following commit(s) were added to refs/heads/branch-4.0 by this push:
     new 1d35aa5cc21 branch-4.0: [feature](function) support json_hash function 
 #56962 (#57254)
1d35aa5cc21 is described below

commit 1d35aa5cc21a7315d6c4d7d8a7916e943b183332
Author: github-actions[bot] 
<41898282+github-actions[bot]@users.noreply.github.com>
AuthorDate: Thu Oct 23 15:00:59 2025 +0800

    branch-4.0: [feature](function) support json_hash function  #56962 (#57254)
    
    Cherry-picked from #56962
    
    Co-authored-by: Mryange <[email protected]>
---
 be/src/util/jsonb_document.cpp                     |  20 ++++
 be/src/util/jsonb_document.h                       |   3 +
 be/src/vec/functions/function_json_hash.cpp        | 104 ++++++++++++++++++
 be/src/vec/functions/simple_function_factory.h     |   2 +
 .../doris/catalog/BuiltinScalarFunctions.java      |   3 +
 .../expressions/functions/scalar/JsonHash.java     |  73 +++++++++++++
 .../expressions/visitor/ScalarFunctionVisitor.java |   5 +
 regression-test/data/jsonb_p0/test_json_hash.out   |  49 +++++++++
 .../suites/jsonb_p0/test_json_hash.groovy          | 116 +++++++++++++++++++++
 9 files changed, 375 insertions(+)

diff --git a/be/src/util/jsonb_document.cpp b/be/src/util/jsonb_document.cpp
index e76189eff85..2ffa8f129e1 100644
--- a/be/src/util/jsonb_document.cpp
+++ b/be/src/util/jsonb_document.cpp
@@ -159,4 +159,24 @@ JsonbFindResult JsonbValue::findValue(JsonbPath& path) 
const {
 
     return result;
 }
+
+std::vector<std::pair<StringRef, const JsonbValue*>> 
ObjectVal::get_ordered_key_value_pairs()
+        const {
+    std::vector<std::pair<StringRef, const JsonbValue*>> kvs;
+    const auto* obj_val = this;
+    for (auto it = obj_val->begin(); it != obj_val->end(); ++it) {
+        kvs.emplace_back(StringRef(it->getKeyStr(), it->klen()), it->value());
+    }
+    // sort by key
+    std::sort(kvs.begin(), kvs.end(),
+              [](const auto& left, const auto& right) { return left.first < 
right.first; });
+    // unique by key
+    kvs.erase(std::unique(kvs.begin(), kvs.end(),
+                          [](const auto& left, const auto& right) {
+                              return left.first == right.first;
+                          }),
+              kvs.end());
+    return kvs;
+}
+
 } // namespace doris
\ No newline at end of file
diff --git a/be/src/util/jsonb_document.h b/be/src/util/jsonb_document.h
index 1e8078594bc..30f3ea040b9 100644
--- a/be/src/util/jsonb_document.h
+++ b/be/src/util/jsonb_document.h
@@ -79,6 +79,7 @@
 #include "common/status.h"
 #include "runtime/define_primitive_type.h"
 #include "util/string_util.h"
+#include "vec/common/string_ref.h"
 #include "vec/core/types.h"
 
 // #include "util/string_parser.hpp"
@@ -953,6 +954,8 @@ struct ObjectVal : public ContainerVal {
 
     const_iterator end() const { return const_iterator((pointer)(payload + 
size)); }
 
+    std::vector<std::pair<StringRef, const JsonbValue*>> 
get_ordered_key_value_pairs() const;
+
 private:
     iterator internalSearch(const char* key, unsigned int klen) {
         const char* pch = payload;
diff --git a/be/src/vec/functions/function_json_hash.cpp 
b/be/src/vec/functions/function_json_hash.cpp
new file mode 100644
index 00000000000..fd20a221708
--- /dev/null
+++ b/be/src/vec/functions/function_json_hash.cpp
@@ -0,0 +1,104 @@
+// 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.
+
+#include "runtime/primitive_type.h"
+#include "util/jsonb_document.h"
+#include "vec/data_types/data_type_number.h"
+#include "vec/functions/simple_function_factory.h"
+
+namespace doris::vectorized {
+
+void json_hash(const JsonbValue* jsonb_value, size_t& hash_value) {
+    auto update_hash = [&hash_value](const void* data, size_t size) {
+        hash_value = HashUtil::hash64(data, (uint32_t)size, hash_value);
+    };
+
+    if (jsonb_value->isObject()) {
+        hash_value ^= (size_t)JsonbType::T_Object;
+        const auto* obj_val = jsonb_value->unpack<ObjectVal>();
+        const auto ordered = obj_val->get_ordered_key_value_pairs();
+        for (const auto& [key, value] : ordered) {
+            update_hash(key.data, key.size);
+            json_hash(value, hash_value);
+        }
+    } else if (jsonb_value->isArray()) {
+        hash_value ^= (size_t)JsonbType::T_Array;
+        const auto* array_val = jsonb_value->unpack<ArrayVal>();
+        for (auto it = array_val->begin(); it != array_val->end(); ++it) {
+            json_hash(&*it, hash_value);
+        }
+    } else {
+        // Similar to the code below
+        // bool writeValue(const JsonbValue* value) {
+        // ...
+        //         os_->write((char*)value, value->numPackedBytes());
+        // ...
+        // }
+        // The hash value of the whole structure is directly calculated here, 
and the Type of Jsonb is included.
+        update_hash((const char*)jsonb_value, jsonb_value->numPackedBytes());
+    }
+}
+
+class FunctionJsonHash : public IFunction {
+public:
+    static constexpr auto name = "json_hash";
+
+    static FunctionPtr create() { return std::make_shared<FunctionJsonHash>(); 
}
+
+    String get_name() const override { return name; }
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const 
override {
+        return std::make_shared<DataTypeInt64>();
+    }
+
+    size_t get_number_of_arguments() const override { return 1; }
+
+    Status execute_impl(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
+                        uint32_t result, size_t size) const override {
+        auto input_column = block.get_by_position(arguments[0]).column;
+        auto to_column = ColumnInt64::create(size);
+        auto& to_column_data = to_column->get_data();
+
+        const auto& input_jsonb_column = assert_cast<const 
ColumnString&>(*input_column);
+
+        for (size_t i = 0; i < size; ++i) {
+            StringRef val = input_jsonb_column.get_data_at(i);
+            JsonbDocument* doc = nullptr;
+            auto st = JsonbDocument::checkAndCreateDocument(val.data, 
val.size, &doc);
+            if (!st.ok() || !doc || !doc->getValue()) [[unlikely]] {
+                // mayby be invalid jsonb, just insert default
+                // invalid jsonb value may be caused by the default null 
processing
+                continue;
+            }
+
+            size_t hash_value = 0;
+            json_hash(doc->getValue(), hash_value);
+
+            to_column_data[i] = static_cast<int64_t>(hash_value);
+        }
+        block.get_by_position(result).column = std::move(to_column);
+        return Status::OK();
+    }
+};
+
+void register_function_json_hash(SimpleFunctionFactory& factory) {
+    factory.register_function<FunctionJsonHash>();
+
+    factory.register_alias(FunctionJsonHash::name, "jsonb_hash");
+}
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/functions/simple_function_factory.h 
b/be/src/vec/functions/simple_function_factory.h
index 3e3dabd8f22..7042cf1b8be 100644
--- a/be/src/vec/functions/simple_function_factory.h
+++ b/be/src/vec/functions/simple_function_factory.h
@@ -77,6 +77,7 @@ void register_function_json(SimpleFunctionFactory& factory);
 void register_function_jsonb(SimpleFunctionFactory& factory);
 void register_function_to_json(SimpleFunctionFactory& factory);
 void register_function_json_transform(SimpleFunctionFactory& factory);
+void register_function_json_hash(SimpleFunctionFactory& factory);
 void register_function_hash(SimpleFunctionFactory& factory);
 void register_function_ifnull(SimpleFunctionFactory& factory);
 void register_function_like(SimpleFunctionFactory& factory);
@@ -343,6 +344,7 @@ public:
             register_function_score(instance);
             register_function_soundex(instance);
             register_function_json_transform(instance);
+            register_function_json_hash(instance);
 #if defined(BE_TEST) && !defined(BE_BENCHMARK)
             register_function_throw_exception(instance);
 #endif
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 11821907dd6..976635547e2 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
@@ -263,6 +263,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.JsonContains;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.JsonExtractNoQuotes;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonHash;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonInsert;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonKeys;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonLength;
@@ -768,6 +769,8 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(JsonQuote.class, "json_quote"),
             scalar(JsonUnQuote.class, "json_unquote"),
             scalar(JsonExtractNoQuotes.class, "json_extract_no_quotes"),
+            scalar(JsonHash.class, "json_hash"),
+            scalar(JsonHash.class, "jsonb_hash"),
             scalar(JsonInsert.class, "json_insert", "jsonb_insert"),
             scalar(JsonReplace.class, "json_replace", "jsonb_replace"),
             scalar(JsonSet.class, "json_set", "jsonb_set"),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonHash.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonHash.java
new file mode 100644
index 00000000000..6345f29f036
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonHash.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.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.NullOrIdenticalSignature;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BigIntType;
+import org.apache.doris.nereids.types.JsonType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * to_json convert type to json
+ */
+public class JsonHash extends ScalarFunction
+        implements UnaryExpression, NullOrIdenticalSignature, 
PropagateNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(BigIntType.INSTANCE).args(JsonType.INSTANCE));
+
+    /**
+     * constructor with 1 or more arguments.
+     */
+    public JsonHash(Expression arg) {
+        super("json_hash", arg);
+    }
+
+    /** constructor for withChildren and reuse signature */
+    private JsonHash(ScalarFunctionParams functionParams) {
+        super(functionParams);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public JsonHash withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1, "json_hash should 
have exactly one argument");
+        return new JsonHash(getFunctionParams(children));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitJsonHash(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 d3557f04253..3a7c7a0f00e 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
@@ -268,6 +268,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.JsonContains;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.JsonExtractNoQuotes;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonHash;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonInsert;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonKeys;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonLength;
@@ -1493,6 +1494,10 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(jsonExtract, context);
     }
 
+    default R visitJsonHash(JsonHash jsonhash, C context) {
+        return visitScalarFunction(jsonhash, context);
+    }
+
     default R visitJsonKeys(JsonKeys jsonKeys, C context) {
         return visitScalarFunction(jsonKeys, context);
     }
diff --git a/regression-test/data/jsonb_p0/test_json_hash.out 
b/regression-test/data/jsonb_p0/test_json_hash.out
new file mode 100644
index 00000000000..22d67c72bf7
--- /dev/null
+++ b/regression-test/data/jsonb_p0/test_json_hash.out
@@ -0,0 +1,49 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !sql --
+3175091177065413928    {"b":123,"b":456,"a":789}
+
+-- !sql --
+-6020063208326718567
+
+-- !sql --
+82454694884268544      82454694884268544
+
+-- !sql --
+-7416836614234106918   -3126362109586887012    -7416836614234106918
+
+-- !sql --
+7882559133986259892    5279066513252500087
+
+-- !sql --
+-6243162159286972049
+
+-- !sql --
+1      4696129010515407867
+2      7489107430353059455
+3      5863236529657059913
+4      -3730229429866862049
+5      \N
+6      -7114717423609403981
+
+-- !sql --
+1      {"b":123,"b":456,"a":789}
+2      {"d":123,"c":456,"b":789,"a":1011}
+3      {"x":123,"y":456,"z":789}
+4      [{"b":123,"b":456,"a":789},{"b":123},{"b":456},{"a":789}]
+5      \N
+6      
{"Data":[{"a_1":"value1","B2":{"zZ":123,"A_1":"value2","b2":"should_sort_last","b2":"should_sort_first","__key__":[{"dupKey":"first","Dupkey":"second","dupKey":"third","1dupKey":"fourth"},{"mix_key":"foo","Mix_Key":"bar","mix_key":"baz"}]},"B2":"anotherB2","b2":"duplicateB2","A_1":"anotherA_1"},{"b2":[{"k":1,"K":2,"k":3},{"b2":"array_dup","B2":"array_B2"}],"randomKey_3":{"foo":"test","Foo":"TEST","foo":"again"}}],"meta_9":{"info":"example","Info":"EXAMPLE","info":"duplicate"}}
+
+-- !sql --
+1      4696129010515407867
+2      7489107430353059455
+3      5863236529657059913
+4      -3730229429866862049
+6      -7114717423609403981
+
+-- !sql --
+1      {"b":123,"b":456,"a":789}
+2      {"d":123,"c":456,"b":789,"a":1011}
+3      {"x":123,"y":456,"z":789}
+4      [{"b":123,"b":456,"a":789},{"b":123},{"b":456},{"a":789}]
+6      
{"Data":[{"a_1":"value1","B2":{"zZ":123,"A_1":"value2","b2":"should_sort_last","b2":"should_sort_first","__key__":[{"dupKey":"first","Dupkey":"second","dupKey":"third","1dupKey":"fourth"},{"mix_key":"foo","Mix_Key":"bar","mix_key":"baz"}]},"B2":"anotherB2","b2":"duplicateB2","A_1":"anotherA_1"},{"b2":[{"k":1,"K":2,"k":3},{"b2":"array_dup","B2":"array_B2"}],"randomKey_3":{"foo":"test","Foo":"TEST","foo":"again"}}],"meta_9":{"info":"example","Info":"EXAMPLE","info":"duplicate"}}
+
diff --git a/regression-test/suites/jsonb_p0/test_json_hash.groovy 
b/regression-test/suites/jsonb_p0/test_json_hash.groovy
new file mode 100644
index 00000000000..1cbfa10c469
--- /dev/null
+++ b/regression-test/suites/jsonb_p0/test_json_hash.groovy
@@ -0,0 +1,116 @@
+// 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.
+
+
+suite("test_json_hash") {
+    qt_sql """
+        select json_hash(cast('{"b":1234567890123456789,"b":456,"a":789}' as 
json)) ,   normalize_jsonb_numbers_to_double(cast('{"b":123,"b":456,"a":789}' 
as json));
+    """
+
+    qt_sql """
+        select 
json_hash(cast('[{"b":123,"b":456,"a":1234567890123456789.231231} 
,{"b":1213379817829313213},{"b":456123123123123123},{"a":74124123132189} ]' as 
json));
+    """
+
+    qt_sql """
+        select json_hash(cast('{"a":123, "b":456}' as json)) , 
json_hash(cast('{"b":456, "a":123}' as json));
+    """
+
+    qt_sql """
+        select json_hash(cast('{"a":123}' as json)) , 
json_hash(cast('{"a":456}' as json)) , json_hash(cast('{"a":123, "a":456}' as 
json)) ;
+    """
+
+    qt_sql """
+        select json_hash(to_json(cast('123' as int))) , 
json_hash(to_json(cast('123' as tinyint)));
+    """
+
+
+    qt_sql """
+         select json_hash(to_json(cast('1231231322.23123' as Decimal)) );
+    """
+
+    sql "DROP TABLE IF EXISTS test_json_json_hash_table"
+
+    sql """
+    CREATE TABLE IF NOT EXISTS test_json_json_hash_table (
+        id INT,
+        j JSON
+    )
+    DISTRIBUTED BY HASH(id) BUCKETS 3
+    PROPERTIES (
+        "replication_num" = "1"
+    )
+    """
+
+    sql """
+    INSERT INTO test_json_json_hash_table VALUES
+    (1, '{"b":123,"b":456,"a":789}'),
+    (2, '{"d":123,"c":456,"b":789,"a":1011}'),
+    (3, '{"x":123,"y":456,"z":789}'),
+    (4, '[{"b":123,"b":456,"a":789} ,{"b":123},{"b":456},{"a":789}]'),
+    (5, null),
+    (6, 
'{"Data":[{"a_1":"value1","B2":{"zZ":123,"A_1":"value2","b2":"should_sort_last","b2":"should_sort_first","__key__":[{"dupKey":"first","Dupkey":"second","dupKey":"third","1dupKey":"fourth"},{"mix_key":"foo","Mix_Key":"bar","mix_key":"baz"}]},"B2":"anotherB2","b2":"duplicateB2","A_1":"anotherA_1"},{"b2":[{"k":1,"K":2,"k":3},{"b2":"array_dup","B2":"array_B2"}],"randomKey_3":{"foo":"test","Foo":"TEST","foo":"again"}}],"meta_9":{"info":"example","Info":"EXAMPLE","info":"duplicate"}}')
+    """
+
+    qt_sql """
+    SELECT id, json_hash(j) as sorted_json
+    FROM test_json_json_hash_table
+    ORDER BY id
+    """
+
+    qt_sql """
+    SELECT id, normalize_jsonb_numbers_to_double(cast(j as jsonb)) as 
sorted_jsonb
+    FROM test_json_json_hash_table
+    ORDER BY id
+    """
+
+    
+    
+    sql "DROP TABLE IF EXISTS test_json_json_hash_table"
+
+    sql """
+    CREATE TABLE IF NOT EXISTS test_json_json_hash_table (
+        id INT,
+        j JSON NOT NULL
+    )
+    DISTRIBUTED BY HASH(id) BUCKETS 3
+    PROPERTIES (
+        "replication_num" = "1"
+    )
+    """
+
+    sql """
+    INSERT INTO test_json_json_hash_table VALUES
+    (1, '{"b":123,"b":456,"a":789}'),
+    (2, '{"d":123,"c":456,"b":789,"a":1011}'),
+    (3, '{"x":123,"y":456,"z":789}'),
+    (4, '[{"b":123,"b":456,"a":789} ,{"b":123},{"b":456},{"a":789}]'),
+    (6, 
'{"Data":[{"a_1":"value1","B2":{"zZ":123,"A_1":"value2","b2":"should_sort_last","b2":"should_sort_first","__key__":[{"dupKey":"first","Dupkey":"second","dupKey":"third","1dupKey":"fourth"},{"mix_key":"foo","Mix_Key":"bar","mix_key":"baz"}]},"B2":"anotherB2","b2":"duplicateB2","A_1":"anotherA_1"},{"b2":[{"k":1,"K":2,"k":3},{"b2":"array_dup","B2":"array_B2"}],"randomKey_3":{"foo":"test","Foo":"TEST","foo":"again"}}],"meta_9":{"info":"example","Info":"EXAMPLE","info":"duplicate"}}')
+    """
+
+    qt_sql """
+    SELECT id, json_hash(j) as sorted_json
+    FROM test_json_json_hash_table
+    ORDER BY id
+    """
+
+    qt_sql """
+    SELECT id, normalize_jsonb_numbers_to_double(cast(j as jsonb)) as 
sorted_jsonb
+    FROM test_json_json_hash_table
+    ORDER BY id
+    """
+
+}


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

Reply via email to