You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/06/21 15:07:25 UTC

[GitHub] [doris] freesinger opened a new pull request, #10322: [feature](JSON datatype)Support JSON datatype

freesinger opened a new pull request, #10322:
URL: https://github.com/apache/doris/pull/10322

   # Proposed changes
   
   Add JSON datatype, following features are implemented by this PR:
   1. `CREATE` tables with `JSON` type columns
   2. `INSERT` values containing `JSON` type value which stored in `String`
   3. `SELECT` JSON columns
   
   Detail design refers [DSIP-016: Support JSON type](https://cwiki.apache.org/confluence/display/DORIS/DSIP-016%3A+Support+JSON+type)
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: (Yes/No/I Don't know)
   4. Has unit tests been added: (Yes/No/No Need)
   5. Has document been added or modified: (Yes/No/No Need)
   6. Does it need to update dependencies: (Yes/No)
   7. Are there any changes that cannot be rolled back: (Yes/No)
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto:dev@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc...
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928253643


##########
be/src/vec/columns/column_json.h:
##########
@@ -0,0 +1,299 @@
+// 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.
+
+#pragma once
+
+#include <cassert>
+#include <cstring>
+
+#include "vec/columns/column.h"
+#include "vec/columns/column_impl.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/memcmp_small.h"
+#include "vec/common/memcpy_small.h"
+#include "vec/common/pod_array.h"
+#include "vec/common/sip_hash.h"
+#include "vec/core/field.h"
+
+namespace doris::vectorized {
+class ColumnJson final : public COWHelper<IColumn, ColumnJson> {
+public:
+    using Char = UInt8;
+    using Chars = PaddedPODArray<UInt8>;
+
+private:
+    friend class COWHelper<IColumn, ColumnJson>;
+
+    Offsets offsets;
+
+    Chars chars;
+
+    size_t ALWAYS_INLINE offset_at(ssize_t i) const { return offsets[i - 1]; }
+
+    size_t ALWAYS_INLINE size_at(ssize_t i) const { return offsets[i] - offsets[i - 1]; }
+
+    template <bool positive>
+    struct less;
+
+    template <bool positive>
+    struct lessWithCollation;
+
+    ColumnJson() = default;
+
+    ColumnJson(const ColumnJson& src)
+            : offsets(src.offsets.begin(), src.offsets.end()),
+              chars(src.chars.begin(), src.chars.end()) {}
+
+public:
+    const char* get_family_name() const override { return "JSON"; }
+
+    size_t size() const override { return offsets.size(); }
+
+    size_t byte_size() const override { return chars.size() + offsets.size() * sizeof(offsets[0]); }
+
+    size_t allocated_bytes() const override {
+        return chars.allocated_bytes() + offsets.allocated_bytes();
+    }
+
+    void protect() override;
+
+    MutableColumnPtr clone_resized(size_t to_size) const override;
+
+    Field operator[](size_t n) const override {
+        assert(n < size());
+        return Field(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    void get(size_t n, Field& res) const override {
+        assert(n < size());
+        res.assign_json(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    StringRef get_data_at(size_t n) const override {
+        assert(n < size());
+        return StringRef(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+/// Suppress gcc 7.3.1 warning: '*((void*)&<anonymous> +8)' may be used uninitialized in this function
+#if !__clang__
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
+#endif
+
+    void insert(const Field& x) override {
+        const JsonField& s = doris::vectorized::get<const JsonField&>(x);
+
+        const size_t old_size = chars.size();
+        const size_t size_to_append = s.get_size() + 1;
+        const size_t new_size = old_size + size_to_append;
+
+        chars.resize(new_size);
+        memcpy(chars.data() + old_size, s.get_value(), size_to_append);
+        offsets.push_back(new_size);
+    }
+
+#if !__clang__
+#pragma GCC diagnostic pop
+#endif
+
+    void insert_from(const IColumn& src_, size_t n) override {
+        const ColumnJson& src = assert_cast<const ColumnJson&>(src_);
+        const size_t size_to_append =
+                src.offsets[n] - src.offsets[n - 1]; /// -1th index is Ok, see PaddedPODArray.
+
+        if (size_to_append == 1) {
+            /// shortcut for empty json
+            chars.push_back(0);
+            offsets.push_back(chars.size());
+        } else {
+            const size_t old_size = chars.size();
+            const size_t offset = src.offsets[n - 1];
+            const size_t new_size = old_size + size_to_append;
+
+            chars.resize(new_size);
+            memcpy_small_allow_read_write_overflow15(chars.data() + old_size, &src.chars[offset],
+                                                     size_to_append);
+            offsets.push_back(new_size);
+        }
+    }
+
+    void insert_data(const char* pos, size_t length) override {
+        const size_t old_size = chars.size();
+        const size_t new_size = old_size + length + 1;
+
+        chars.resize(new_size);
+        if (length) memcpy(chars.data() + old_size, pos, length);
+        chars[old_size + length] = 0;
+        offsets.push_back(new_size);
+    }
+
+    void insert_many_binary_data(char* data_array, uint32_t* len_array,
+                                 uint32_t* start_offset_array, size_t num) override {
+        for (size_t i = 0; i < num; i++) {
+            uint32_t len = len_array[i];
+            uint32_t start_offset = start_offset_array[i];
+            insert_data(data_array + start_offset, len);
+        }
+    };
+
+    void insert_many_dict_data(const int32_t* data_array, size_t start_index, const StringRef* dict,
+                               size_t num, uint32_t /*dict_num*/) {
+        for (size_t end_index = start_index + num; start_index < end_index; ++start_index) {
+            int32_t codeword = data_array[start_index];
+            insert_data(dict[codeword].data, dict[codeword].size);
+        }
+    }
+
+    void pop_back(size_t n) override {
+        size_t nested_n = offsets.back() - offset_at(offsets.size() - n);
+        chars.resize(chars.size() - nested_n);
+        offsets.resize_assume_reserved(offsets.size() - n);
+    }
+
+    StringRef serialize_value_into_arena(size_t n, Arena& arena, char const*& begin) const override;
+
+    const char* deserialize_and_insert_from_arena(const char* pos) override;
+
+    void update_hash_with_value(size_t n, SipHash& hash) const override {
+        size_t string_size = size_at(n);
+        size_t offset = offset_at(n);
+
+        hash.update(reinterpret_cast<const char*>(&string_size), sizeof(string_size));
+        hash.update(reinterpret_cast<const char*>(&chars[offset]), string_size);
+    }
+
+    void insert_range_from(const IColumn& src, size_t start, size_t length) override;
+
+    void insert_indices_from(const IColumn& src, const int* indices_begin,
+                             const int* indices_end) override;
+
+    ColumnPtr filter(const Filter& filt, ssize_t result_size_hint) const override;
+
+    ColumnPtr permute(const Permutation& perm, size_t limit) const override;
+
+    //    ColumnPtr index(const IColumn & indexes, size_t limit) const override;
+
+    template <typename Type>
+    ColumnPtr index_impl(const PaddedPODArray<Type>& indexes, size_t limit) const;
+
+    void insert_default() override {
+        chars.push_back(0);
+        offsets.push_back(offsets.back() + 1);
+    }
+
+    void insert_many_defaults(size_t length) override {

Review Comment:
   same porblem as insert_default



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] morningman merged pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
morningman merged PR #10322:
URL: https://github.com/apache/doris/pull/10322


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r921818624


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/JsonLiteral.java:
##########
@@ -0,0 +1,197 @@
+// 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.analysis;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Objects;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.JsonParser;
+import com.google.gson.JsonSyntaxException;
+
+import org.apache.doris.common.io.Text;
+import org.apache.doris.thrift.TExprNode;
+import org.apache.doris.thrift.TExprNodeType;
+import org.apache.doris.thrift.TJsonLiteral;
+import org.apache.logging.log4j.LogManager;
+import org.apache.doris.catalog.Type;
+import org.apache.logging.log4j.Logger;
+
+import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.qe.VariableVarConverters;
+
+public class JsonLiteral extends LiteralExpr {
+    private static final Logger LOG = LogManager.getLogger(JsonLiteral.class);
+    private JsonParser parser = new JsonParser();
+    private String value;
+    // Means the converted session variable need to be cast to int, such as "cast 'STRICT_TRANS_TABLES' to Integer".
+    private String beConverted = "";
+
+    public JsonLiteral() {
+        super();
+        type = Type.JSON;
+    }
+
+    public JsonLiteral(String value) throws AnalysisException {
+        try {
+            parser.parse(value);
+        } catch (JsonSyntaxException e) {
+            throw new AnalysisException("Invalid json literal: " + e.getMessage());
+        }
+        this.value = value;
+        type = Type.JSON;
+        analysisDone();
+    }
+
+    protected JsonLiteral(JsonLiteral other) {
+        super(other);
+        value = other.value;
+    }
+
+    public void setBeConverted(String val) {
+        this.beConverted = val;
+    }
+
+    @Override
+    public Expr clone() {
+        return new JsonLiteral(this);
+    }
+
+    @Override
+    public int compareLiteral(LiteralExpr expr) {
+        if (expr instanceof NullLiteral) {
+            return 1;
+        }
+        if (expr == MaxLiteral.MAX_VALUE) {
+            return -1;
+        }
+        // compare string with utf-8 byte array, same with DM,BE,StorageEngine
+        byte[] thisBytes = null;
+        byte[] otherBytes = null;
+        try {
+            thisBytes = value.getBytes("UTF-8");
+            otherBytes = expr.getStringValue().getBytes("UTF-8");
+        } catch (UnsupportedEncodingException e) {
+            Preconditions.checkState(false);
+        }
+
+        int minLength = Math.min(thisBytes.length, otherBytes.length);
+        int i = 0;
+        for (i = 0; i < minLength; i++) {
+            if (thisBytes[i] < otherBytes[i]) {
+                return -1;
+            } else if (thisBytes[i] > otherBytes[i]) {
+                return 1;
+            }
+        }
+        if (thisBytes.length > otherBytes.length) {
+            if (thisBytes[i] == 0x00) {
+                return 0;
+            } else {
+                return 1;
+            }
+        } else if (thisBytes.length < otherBytes.length) {
+            if (otherBytes[i] == 0x00) {
+                return 0;
+            } else {
+                return -1;
+            }
+        } else {
+            return 0;
+        }
+    }
+
+    public String getValue() {
+        return value;
+    }
+
+    @Override
+    public boolean isMinValue() {
+        return false;
+    }
+
+    @Override
+    public String toSqlImpl() {
+        return "'" + value.replaceAll("'", "''") + "'";
+    }
+
+    @Override
+    protected void toThrift(TExprNode msg) {
+        msg.node_type = TExprNodeType.JSON_LITERAL;
+        msg.json_literal = new TJsonLiteral(getUnescapedValue());
+    }
+
+    public String getUnescapedValue() {
+        // Unescape string exactly like Hive does. Hive's method assumes
+        // quotes so we add them here to reuse Hive's code.
+        return value;
+    }
+
+    public String getJsonValue() {
+        return value;
+    }
+
+    @Override
+    public long getLongValue() {
+        return 0;

Review Comment:
   Since `getLongValue()` and `getDoubleValue()` methods of  JsonLiteral are extended from class LiteralExpr, which means it's impossible to override a function cannot throw expection. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928256575


##########
be/src/olap/types.h:
##########
@@ -1454,6 +1460,66 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSON> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSON values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        auto jdoc = JsonbDocument::createDocument(scan_key.c_str(), scan_key.size());
+        size_t value_len = jdoc->numPackedBytes();
+        if (value_len > config::json_type_length_soft_limit_bytes) {
+            LOG(WARNING) << "the len of value json is too long, len=" << value_len
+                         << ", max_len=" << config::json_type_length_soft_limit_bytes;
+            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        }
+
+        auto slice = reinterpret_cast<Slice*>(buf);
+        memory_copy(slice->data, reinterpret_cast<const char*>(jdoc->getValue()), value_len);
+        slice->size = value_len;
+        return Status::OK();
+    }
+
+    static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
+                               MemPool* mem_pool, size_t variable_len = 0) {
+        JsonbToJson toStr;
+        switch (src_type->type()) {
+        case OLAP_FIELD_TYPE_TINYINT:
+        case OLAP_FIELD_TYPE_SMALLINT:
+        case OLAP_FIELD_TYPE_INT:
+        case OLAP_FIELD_TYPE_BIGINT:
+        case OLAP_FIELD_TYPE_LARGEINT:
+        case OLAP_FIELD_TYPE_FLOAT:
+        case OLAP_FIELD_TYPE_DOUBLE:
+        case OLAP_FIELD_TYPE_DECIMAL: {
+            auto s = src_type->to_string(src);
+            std::string result =
+                    toStr.json(JsonbDocument::createDocument(s.c_str(), s.size())->getValue());
+            auto slice = reinterpret_cast<Slice*>(dest);
+            slice->data = reinterpret_cast<char*>(mem_pool->allocate(result.size()));
+            memcpy(slice->data, result.c_str(), result.size());
+            slice->size = result.size();
+            return Status::OK();
+        }
+        default:

Review Comment:
   no branch for string varchar



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on PR #10322:
URL: https://github.com/apache/doris/pull/10322#issuecomment-1257129704

   Thanks for @xiaokang 's carefully review and tremendous aids.
   Cheers~


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSONB datatype)Support JSONB datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r967566758


##########
fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlColType.java:
##########
@@ -51,7 +51,8 @@ public enum MysqlColType {
     MYSQL_TYPE_BLOB(252, "BLOB"),
     MYSQL_TYPE_VARSTRING(253, "VAR STRING"),
     MYSQL_TYPE_STRING(254, "STRING"),
-    MYSQL_TYPE_GEOMETRY(255, "GEOMETRY");
+    MYSQL_TYPE_GEOMETRY(255, "GEOMETRY"),
+    MYSQL_TYPE_JSONB(256, "JSONB");

Review Comment:
   refer to the mysql link above



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java:
##########
@@ -148,6 +149,19 @@ public abstract class Type {
         supportedTypes.add(HLL);
         supportedTypes.add(BITMAP);
         supportedTypes.add(QUANTILE_STATE);
+        supportedTypes.add(CHAR);

Review Comment:
   why we add these types here?



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/PrimitiveType.java:
##########
@@ -49,6 +49,7 @@ public enum PrimitiveType {
     // 8-byte pointer and 4-byte length indicator (12 bytes total).
     // Aligning to 8 bytes so 16 total.
     VARCHAR("VARCHAR", 16, TPrimitiveType.VARCHAR),
+    JSONB("JSONB", 16, TPrimitiveType.JSONB),

Review Comment:
   we need to consider it carefully



##########
be/test/runtime/jsonb_value_test.cpp:
##########
@@ -0,0 +1,61 @@
+// 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/jsonb_value.h"
+
+#include <gtest/gtest.h>
+
+#include <string>
+
+#include "util/cpu_info.h"
+
+using std::string;
+
+namespace doris {
+
+JsonBinaryValue FromStdString(const string& str) {
+    char* ptr = const_cast<char*>(str.c_str());
+    int len = str.size();
+    return JsonBinaryValue(ptr, len);
+}
+
+TEST(JsonBinaryValueTest, TestValidation) {
+    JsonbErrType err;
+    JsonBinaryValue json_val;
+
+    // single value not wrapped as an arrar or object is invalid
+    std::vector<string> invalid_strs = {"", "1", "null", "false", "abc"};
+    for (size_t i = 0; i < invalid_strs.size(); i++) {
+        err = json_val.from_json_str(invalid_strs[i].c_str(), invalid_strs[i].size());
+        EXPECT_NE(err, JsonbErrType::E_NONE);
+        std::cout << JsonbErrMsg::getErrMsg(err) << std::endl;

Review Comment:
   not necessary



##########
gensrc/thrift/Exprs.thrift:
##########
@@ -40,6 +40,7 @@ enum TExprNodeType {
   NULL_LITERAL,
   SLOT_REF,
   STRING_LITERAL,
+  JSONB_LITERAL,

Review Comment:
   json_literal is better, since literal is json format



##########
be/src/vec/exprs/vliteral.cpp:
##########
@@ -231,6 +240,11 @@ std::string VLiteral::debug_string() const {
                 out << ref;
                 break;
             }
+            case TYPE_JSONB: {
+                JsonBinaryValue value(ref.data, ref.size);

Review Comment:
   not necessary, just out << ref



##########
be/src/runtime/jsonb_value.h:
##########
@@ -0,0 +1,122 @@
+// 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.
+
+#ifndef DORIS_BE_RUNTIME_JSON_VALUE_H
+#define DORIS_BE_RUNTIME_JSON_VALUE_H
+
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/jsonb_error.h"
+#include "util/jsonb_parser.h"
+#include "util/jsonb_utils.h"
+#include "vec/common/string_ref.h"
+
+#ifdef __SSE4_2__
+#include "util/sse_util.hpp"
+#endif
+
+namespace doris {
+
+struct JsonBinaryValue {

Review Comment:
   JsonbValue is enough



##########
be/src/vec/sink/vmysql_result_writer.cpp:
##########
@@ -106,6 +106,21 @@ Status VMysqlResultWriter::_add_one_column(const ColumnPtr& column_ptr,
                     buf_ret = _buffer.push_string(string_val.data, string_val.size);
                 }
             }
+            if constexpr (type == TYPE_JSONB) {
+                const auto json_val = column->get_data_at(i);
+
+                if (json_val.data == nullptr) {
+                    if (json_val.size == 0) {
+                        // 0x01 is a magic num, not useful actually, just for present ""
+                        char* tmp_val = reinterpret_cast<char*>(0x01);
+                        buf_ret = _buffer.push_json_string(tmp_val, json_val.size);

Review Comment:
   push_json_string is very simple, can we just call json_val.to_string() and _buffer.push_string



##########
be/src/vec/data_types/data_type_jsonb.h:
##########
@@ -0,0 +1,61 @@
+// 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.
+
+#pragma once
+
+#include <ostream>
+
+#include "vec/columns/column_jsonb.h"
+#include "vec/data_types/data_type.h"
+
+namespace doris::vectorized {
+class DataTypeJsonb final : public IDataType {
+public:
+    using ColumnType = ColumnJsonb;
+    using FieldType = JsonbField;
+    static constexpr bool is_parametric = false;
+
+    const char* get_family_name() const override { return "JSONB"; }
+    TypeIndex get_type_id() const override { return TypeIndex::JSONB; }
+
+    int64_t get_uncompressed_serialized_bytes(const IColumn& column) const override;
+    char* serialize(const IColumn& column, char* buf) const override;
+    const char* deserialize(const char* buf, IColumn* column) const override;
+
+    MutableColumnPtr create_column() const override;
+
+    virtual Field get_default() const override {
+        std::string default_json = "{}";

Review Comment:
   should create a JsonbValue first to parse json string to binary json format.



##########
be/src/olap/types.h:
##########
@@ -1559,6 +1565,62 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSONB> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSONB values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        auto jdoc = JsonbDocument::createDocument(scan_key.c_str(), scan_key.size());

Review Comment:
   what does scan_key mean in this function?



##########
be/src/olap/olap_define.h:
##########
@@ -57,6 +57,9 @@ static const uint16_t OLAP_VARCHAR_MAX_LENGTH = 65535;
 // the max length supported for string type 2GB
 static const uint32_t OLAP_STRING_MAX_LENGTH = 2147483647;
 
+// the max length supported for jsonb type 2G
+static const uint32_t OLAP_JSONB_MAX_LENGTH = 2147483647;

Review Comment:
   we should change max length of jsonb library



##########
be/src/olap/types.h:
##########
@@ -1559,6 +1565,62 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSONB> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSONB values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        auto jdoc = JsonbDocument::createDocument(scan_key.c_str(), scan_key.size());
+        size_t value_len = jdoc->numPackedBytes();
+        if (value_len > config::jsonb_type_length_soft_limit_bytes) {
+            LOG(WARNING) << "the len of value json is too long, len=" << value_len
+                         << ", max_len=" << config::jsonb_type_length_soft_limit_bytes;
+            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        }
+
+        auto slice = reinterpret_cast<Slice*>(buf);
+        memory_copy(slice->data, reinterpret_cast<const char*>(jdoc->getValue()), value_len);
+        slice->size = value_len;
+        return Status::OK();
+    }
+
+    static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
+                               MemPool* mem_pool, size_t variable_len = 0) {
+        JsonbToJson toStr;
+        switch (src_type->type()) {
+        // TODO(wzy): JSONB should support all numerics
+        case OLAP_FIELD_TYPE_CHAR:
+        case OLAP_FIELD_TYPE_VARCHAR:
+        case OLAP_FIELD_TYPE_STRING: {
+            auto s = src_type->to_string(src);
+            std::string result = toStr.jsonb_to_string(
+                    JsonbDocument::createDocument(s.c_str(), s.size())->getValue());

Review Comment:
   is s json text or binary? If it's text, it can not be passed to createDocument.



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java:
##########
@@ -521,6 +521,8 @@ public static String getUdfTypeName(PrimitiveType t) {
             case QUANTILE_STATE:
             case STRING:
                 return "string_val";
+            case JSONB:
+                return "json_val";

Review Comment:
   jsonb_val



##########
be/src/olap/types.h:
##########
@@ -1559,6 +1565,62 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSONB> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSONB values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        auto jdoc = JsonbDocument::createDocument(scan_key.c_str(), scan_key.size());

Review Comment:
   createDocument is used to create a JsonbDocument from jsonb binary string.



##########
be/src/olap/types.h:
##########
@@ -1559,6 +1565,62 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSONB> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSONB values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        auto jdoc = JsonbDocument::createDocument(scan_key.c_str(), scan_key.size());
+        size_t value_len = jdoc->numPackedBytes();
+        if (value_len > config::jsonb_type_length_soft_limit_bytes) {
+            LOG(WARNING) << "the len of value json is too long, len=" << value_len
+                         << ", max_len=" << config::jsonb_type_length_soft_limit_bytes;
+            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        }
+
+        auto slice = reinterpret_cast<Slice*>(buf);
+        memory_copy(slice->data, reinterpret_cast<const char*>(jdoc->getValue()), value_len);

Review Comment:
   It seems to be wrong to copy jdoc->getValue() since it drops jsonb header



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java:
##########
@@ -571,6 +573,8 @@ public static String getUdfType(PrimitiveType t) {
             case QUANTILE_STATE:
             case STRING:
                 return "StringVal";
+            case JSONB:
+                return "JsonVal";

Review Comment:
   JsonbVal



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/PrimitiveType.java:
##########
@@ -136,6 +138,7 @@ public enum PrimitiveType {
         builder.put(BOOLEAN, DECIMAL64);
         builder.put(BOOLEAN, DECIMAL128);
         builder.put(BOOLEAN, VARCHAR);
+        builder.put(BOOLEAN, JSONB);

Review Comment:
   builder is for implicitCastMap. currently casting from types other than string/varchar is not supported



##########
be/src/vec/data_types/data_type_jsonb.cpp:
##########
@@ -0,0 +1,132 @@
+// 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 "data_type_jsonb.h"
+
+#include "gen_cpp/data.pb.h"
+#include "vec/columns/column_const.h"
+#include "vec/columns/column_jsonb.h"
+#include "vec/common/assert_cast.h"
+#include "vec/core/field.h"
+#include "vec/io/io_helper.h"
+
+#ifdef __SSE2__
+#include <emmintrin.h>
+#endif
+
+namespace doris::vectorized {
+
+template <typename Reader>
+static inline void read(IColumn& column, Reader&& reader) {
+    ColumnJsonb& column_json = assert_cast<ColumnJsonb&>(column);
+    ColumnJsonb::Chars& data = column_json.get_chars();
+    ColumnJsonb::Offsets& offsets = column_json.get_offsets();
+    size_t old_chars_size = data.size();
+    size_t old_offsets_size = offsets.size();
+    try {
+        reader(data);
+        data.push_back(0);
+        offsets.push_back(data.size());
+    } catch (...) {
+        offsets.resize_assume_reserved(old_offsets_size);
+        data.resize_assume_reserved(old_chars_size);
+        throw;
+    }
+}
+
+std::string jsonb_to_string(const StringRef& s) {

Review Comment:
   can be defined in util class



##########
be/src/vec/io/io_helper.h:
##########
@@ -194,6 +199,24 @@ inline StringRef read_string_binary_into(Arena& arena, BufferReadable& buf) {
     return StringRef(data, size);
 }
 
+inline void read_json_binary(StringRef& s, BufferReadable& buf,
+                             size_t MAX_JSON_SIZE = DEFAULT_MAX_JSON_SIZE) {
+    size_t size = 0;
+    read_var_uint(size, buf);
+
+    if (size > MAX_JSON_SIZE) {
+        throw Exception("Too large jsonb size.", TStatusCode::VEC_EXCEPTION);
+    }
+
+    s = buf.read(size);
+}
+
+inline void read_json_binary(JsonbField val, BufferReadable& buf,
+                             size_t MAX_JSON_SIZE = DEFAULT_MAX_JSON_SIZE) {
+    StringRef jrf = StringRef {val.get_value(), val.get_size()};
+    read_json_binary(jrf, buf);

Review Comment:
   just call read_string_binary



##########
be/test/vec/core/column_jsonb_test.cpp:
##########
@@ -0,0 +1,61 @@
+// 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 "vec/columns/column_jsonb.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <string>
+
+#include "vec/columns/column.h"
+#include "vec/columns/column_string.h"
+#include "vec/columns/column_vector.h"
+
+namespace doris::vectorized {
+
+JsonBinaryValue FromStdString(const std::string& str) {
+    char* ptr = const_cast<char*>(str.c_str());
+    int len = str.size();
+    return JsonBinaryValue(ptr, len);
+}
+
+TEST(ColumnJsonbTest, SingleValueTest) {
+    auto off_column = ColumnVector<IColumn::Offset>::create();

Review Comment:
   not necessary



##########
be/src/util/mysql_row_buffer.cpp:
##########
@@ -351,6 +353,15 @@ int MysqlRowBuffer::push_string(const char* str, int64_t length) {
     return 0;
 }
 
+int MysqlRowBuffer::push_json_string(const char* data, int64_t length) {

Review Comment:
   maybe not necessary



##########
be/src/runtime/jsonb_value.h:
##########
@@ -0,0 +1,122 @@
+// 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.
+
+#ifndef DORIS_BE_RUNTIME_JSON_VALUE_H
+#define DORIS_BE_RUNTIME_JSON_VALUE_H
+
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/jsonb_error.h"
+#include "util/jsonb_parser.h"
+#include "util/jsonb_utils.h"
+#include "vec/common/string_ref.h"
+
+#ifdef __SSE4_2__
+#include "util/sse_util.hpp"
+#endif
+
+namespace doris {
+
+struct JsonBinaryValue {
+    static const int MAX_LENGTH = (1 << 30);
+
+    const char* ptr;
+    size_t len;
+    JsonbParser parser;
+
+    JsonBinaryValue() : ptr(nullptr), len(0) {}
+    JsonBinaryValue(char* ptr, int len) { from_json_str(const_cast<const char*>(ptr), len); }
+    JsonBinaryValue(const std::string& s) { from_json_str(s.c_str(), s.length()); }
+    JsonBinaryValue(const char* ptr, int len) { from_json_str(ptr, len); }
+
+    const char* value() { return ptr; }
+
+    size_t size() { return len; }
+
+    void replace(char* ptr, int len) {
+        this->ptr = ptr;
+        this->len = len;
+    }
+
+    bool operator==(const JsonBinaryValue& other) const {
+        LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
+    }
+    // !=
+    bool ne(const JsonBinaryValue& other) const {
+        LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
+    }
+    // <=
+    bool le(const JsonBinaryValue& other) const {
+        LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
+    }
+    // >=
+    bool ge(const JsonBinaryValue& other) const {
+        LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
+    }
+    // <
+    bool lt(const JsonBinaryValue& other) const {
+        LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
+    }
+    // >
+    bool gt(const JsonBinaryValue& other) const {
+        LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
+    }
+
+    bool operator!=(const JsonBinaryValue& other) const {
+        LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
+    }
+
+    bool operator<=(const JsonBinaryValue& other) const {
+        LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
+    }
+
+    bool operator>=(const JsonBinaryValue& other) const {
+        LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
+    }
+
+    bool operator<(const JsonBinaryValue& other) const {
+        LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
+    }
+
+    bool operator>(const JsonBinaryValue& other) const {
+        LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
+    }
+
+    JsonbErrType from_json_str(const char* s, int len);

Review Comment:
   It's better to make the function names symmetric, such as from_string/to_string or from_json_string/to_json_string



##########
be/src/olap/types.h:
##########
@@ -1559,6 +1565,62 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSONB> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSONB values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        auto jdoc = JsonbDocument::createDocument(scan_key.c_str(), scan_key.size());
+        size_t value_len = jdoc->numPackedBytes();
+        if (value_len > config::jsonb_type_length_soft_limit_bytes) {
+            LOG(WARNING) << "the len of value json is too long, len=" << value_len
+                         << ", max_len=" << config::jsonb_type_length_soft_limit_bytes;
+            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        }
+
+        auto slice = reinterpret_cast<Slice*>(buf);
+        memory_copy(slice->data, reinterpret_cast<const char*>(jdoc->getValue()), value_len);
+        slice->size = value_len;
+        return Status::OK();
+    }
+
+    static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
+                               MemPool* mem_pool, size_t variable_len = 0) {
+        JsonbToJson toStr;
+        switch (src_type->type()) {
+        // TODO(wzy): JSONB should support all numerics
+        case OLAP_FIELD_TYPE_CHAR:
+        case OLAP_FIELD_TYPE_VARCHAR:
+        case OLAP_FIELD_TYPE_STRING: {
+            auto s = src_type->to_string(src);
+            std::string result = toStr.jsonb_to_string(
+                    JsonbDocument::createDocument(s.c_str(), s.size())->getValue());
+            auto slice = reinterpret_cast<Slice*>(dest);
+            slice->data = reinterpret_cast<char*>(mem_pool->allocate(result.size()));
+            memcpy(slice->data, result.c_str(), result.size());
+            slice->size = result.size();
+            return Status::OK();
+        }
+        default:
+            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        }
+    }
+
+    static void set_to_min(void* buf) {
+        auto slice = reinterpret_cast<Slice*>(buf);
+        slice->size = 0;
+    }
+
+    static void set_to_max(void* buf) {
+        auto slice = reinterpret_cast<Slice*>(buf);
+        slice->size = OLAP_JSONB_MAX_LENGTH; // 2G

Review Comment:
   what about slice->data?



##########
fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlColType.java:
##########
@@ -51,7 +51,8 @@ public enum MysqlColType {
     MYSQL_TYPE_BLOB(252, "BLOB"),
     MYSQL_TYPE_VARSTRING(253, "VAR STRING"),
     MYSQL_TYPE_STRING(254, "STRING"),
-    MYSQL_TYPE_GEOMETRY(255, "GEOMETRY");
+    MYSQL_TYPE_GEOMETRY(255, "GEOMETRY"),
+    MYSQL_TYPE_JSONB(256, "JSONB");

Review Comment:
   mysql type may be JSON



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r933266400


##########
be/src/vec/columns/column_json.h:
##########
@@ -0,0 +1,299 @@
+// 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.
+
+#pragma once
+
+#include <cassert>
+#include <cstring>
+
+#include "vec/columns/column.h"
+#include "vec/columns/column_impl.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/memcmp_small.h"
+#include "vec/common/memcpy_small.h"
+#include "vec/common/pod_array.h"
+#include "vec/common/sip_hash.h"
+#include "vec/core/field.h"
+
+namespace doris::vectorized {
+class ColumnJson final : public COWHelper<IColumn, ColumnJson> {
+public:
+    using Char = UInt8;
+    using Chars = PaddedPODArray<UInt8>;
+
+private:
+    friend class COWHelper<IColumn, ColumnJson>;
+
+    Offsets offsets;
+
+    Chars chars;
+
+    size_t ALWAYS_INLINE offset_at(ssize_t i) const { return offsets[i - 1]; }
+
+    size_t ALWAYS_INLINE size_at(ssize_t i) const { return offsets[i] - offsets[i - 1]; }
+
+    template <bool positive>
+    struct less;
+
+    template <bool positive>
+    struct lessWithCollation;
+
+    ColumnJson() = default;
+
+    ColumnJson(const ColumnJson& src)
+            : offsets(src.offsets.begin(), src.offsets.end()),
+              chars(src.chars.begin(), src.chars.end()) {}
+
+public:
+    const char* get_family_name() const override { return "JSON"; }
+
+    size_t size() const override { return offsets.size(); }
+
+    size_t byte_size() const override { return chars.size() + offsets.size() * sizeof(offsets[0]); }
+
+    size_t allocated_bytes() const override {
+        return chars.allocated_bytes() + offsets.allocated_bytes();
+    }
+
+    void protect() override;
+
+    MutableColumnPtr clone_resized(size_t to_size) const override;
+
+    Field operator[](size_t n) const override {
+        assert(n < size());
+        return Field(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    void get(size_t n, Field& res) const override {
+        assert(n < size());
+        res.assign_json(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    StringRef get_data_at(size_t n) const override {
+        assert(n < size());
+        return StringRef(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+/// Suppress gcc 7.3.1 warning: '*((void*)&<anonymous> +8)' may be used uninitialized in this function
+#if !__clang__
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
+#endif
+
+    void insert(const Field& x) override {
+        const JsonField& s = doris::vectorized::get<const JsonField&>(x);
+
+        const size_t old_size = chars.size();
+        const size_t size_to_append = s.get_size() + 1;
+        const size_t new_size = old_size + size_to_append;
+
+        chars.resize(new_size);
+        memcpy(chars.data() + old_size, s.get_value(), size_to_append);
+        offsets.push_back(new_size);
+    }
+
+#if !__clang__
+#pragma GCC diagnostic pop
+#endif
+
+    void insert_from(const IColumn& src_, size_t n) override {
+        const ColumnJson& src = assert_cast<const ColumnJson&>(src_);
+        const size_t size_to_append =
+                src.offsets[n] - src.offsets[n - 1]; /// -1th index is Ok, see PaddedPODArray.
+
+        if (size_to_append == 1) {
+            /// shortcut for empty json
+            chars.push_back(0);
+            offsets.push_back(chars.size());
+        } else {
+            const size_t old_size = chars.size();
+            const size_t offset = src.offsets[n - 1];
+            const size_t new_size = old_size + size_to_append;
+
+            chars.resize(new_size);
+            memcpy_small_allow_read_write_overflow15(chars.data() + old_size, &src.chars[offset],
+                                                     size_to_append);
+            offsets.push_back(new_size);
+        }
+    }
+
+    void insert_data(const char* pos, size_t length) override {
+        const size_t old_size = chars.size();
+        const size_t new_size = old_size + length + 1;
+
+        chars.resize(new_size);
+        if (length) memcpy(chars.data() + old_size, pos, length);
+        chars[old_size + length] = 0;
+        offsets.push_back(new_size);
+    }
+
+    void insert_many_binary_data(char* data_array, uint32_t* len_array,
+                                 uint32_t* start_offset_array, size_t num) override {
+        for (size_t i = 0; i < num; i++) {
+            uint32_t len = len_array[i];
+            uint32_t start_offset = start_offset_array[i];
+            insert_data(data_array + start_offset, len);
+        }
+    };
+
+    void insert_many_dict_data(const int32_t* data_array, size_t start_index, const StringRef* dict,
+                               size_t num, uint32_t /*dict_num*/) {
+        for (size_t end_index = start_index + num; start_index < end_index; ++start_index) {
+            int32_t codeword = data_array[start_index];
+            insert_data(dict[codeword].data, dict[codeword].size);
+        }
+    }
+
+    void pop_back(size_t n) override {
+        size_t nested_n = offsets.back() - offset_at(offsets.size() - n);
+        chars.resize(chars.size() - nested_n);
+        offsets.resize_assume_reserved(offsets.size() - n);
+    }
+
+    StringRef serialize_value_into_arena(size_t n, Arena& arena, char const*& begin) const override;
+
+    const char* deserialize_and_insert_from_arena(const char* pos) override;
+
+    void update_hash_with_value(size_t n, SipHash& hash) const override {
+        size_t string_size = size_at(n);
+        size_t offset = offset_at(n);
+
+        hash.update(reinterpret_cast<const char*>(&string_size), sizeof(string_size));
+        hash.update(reinterpret_cast<const char*>(&chars[offset]), string_size);
+    }
+
+    void insert_range_from(const IColumn& src, size_t start, size_t length) override;
+
+    void insert_indices_from(const IColumn& src, const int* indices_begin,
+                             const int* indices_end) override;
+
+    ColumnPtr filter(const Filter& filt, ssize_t result_size_hint) const override;
+
+    ColumnPtr permute(const Permutation& perm, size_t limit) const override;
+
+    //    ColumnPtr index(const IColumn & indexes, size_t limit) const override;
+
+    template <typename Type>
+    ColumnPtr index_impl(const PaddedPODArray<Type>& indexes, size_t limit) const;
+
+    void insert_default() override {
+        chars.push_back(0);

Review Comment:
   shoudl insert empty json `{}`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r956849411


##########
be/src/olap/olap_define.h:
##########
@@ -57,6 +57,9 @@ static const uint16_t OLAP_VARCHAR_MAX_LENGTH = 65535;
 // the max length supported for string type 2GB
 static const uint32_t OLAP_STRING_MAX_LENGTH = 2147483647;
 
+// the max length supported for json type 2GB
+static const uint32_t OLAP_JSON_MAX_LENGTH = 10 * 1024 * 1024;

Review Comment:
   nice catch, change 2G to 10M



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r975637134


##########
be/src/olap/types.h:
##########
@@ -1559,6 +1566,65 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSONB> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSONB values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        JsonBinaryValue binary_val(scan_key.c_str(), scan_key.size());
+        auto jdoc = JsonbDocument::createDocument(binary_val.value(), binary_val.size());
+        size_t value_len = jdoc->numPackedBytes();
+        if (value_len > config::jsonb_type_length_soft_limit_bytes) {
+            LOG(WARNING) << "the len of value json is too long, len=" << value_len
+                         << ", max_len=" << config::jsonb_type_length_soft_limit_bytes;
+            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        }
+
+        auto slice = reinterpret_cast<Slice*>(buf);
+        memory_copy(slice->data, reinterpret_cast<const char*>(jdoc->getValue()), value_len);

Review Comment:
   jdoc has header



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928252070


##########
be/src/vec/data_types/data_type_json.h:
##########
@@ -0,0 +1,62 @@
+// 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.
+
+#pragma once
+
+#include <ostream>
+
+#include "vec/columns/column_json.h"
+#include "vec/data_types/data_type.h"
+
+namespace doris::vectorized {
+class DataTypeJson final : public IDataType {
+public:
+    using ColumnType = ColumnJson;
+    using FieldType = JsonField;
+    static constexpr bool is_parametric = false;
+
+    const char* get_family_name() const override { return "JSON"; }
+    TypeIndex get_type_id() const override { return TypeIndex::JSON; }
+
+    int64_t get_uncompressed_serialized_bytes(const IColumn& column) const override;
+    char* serialize(const IColumn& column, char* buf) const override;
+    const char* deserialize(const char* buf, IColumn* column) const override;
+
+    MutableColumnPtr create_column() const override;
+
+    virtual Field get_default() const override {
+        LOG(FATAL) << "Method get_default() is not implemented for data type " << get_name();
+        // unreachable
+        return String();
+    }
+
+    bool equals(const IDataType& rhs) const override;
+
+    bool get_is_parametric() const override { return false; }
+    bool have_subtypes() const override { return false; }
+    bool is_comparable() const override { return false; }
+    bool can_be_compared_with_collation() const override { return false; }
+    bool is_value_unambiguously_represented_in_contiguous_memory_region() const override {
+        return true;
+    }
+    bool is_categorial() const override { return true; }

Review Comment:
   return false may be better, since json can contain any value



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r918517597


##########
be/src/common/config.h:
##########
@@ -744,6 +744,11 @@ CONF_mInt32(string_type_length_soft_limit_bytes, "1048576");
 CONF_Validator(string_type_length_soft_limit_bytes,
                [](const int config) -> bool { return config > 0 && config <= 2147483643; });
 
+CONF_mInt32(json_type_length_soft_limit_bytes, "1048576");

Review Comment:
   Similar to String type, soft limit for JSON here is used to set max length for binary JSON data, which means the size of each row in ColumnJson should not > 1MB
   Usages:
   - [olap/row_block2.cpp](https://github.com/freesinger/incubator-doris/blob/6e23f46b48a78ea620d891b0184133ec97c3742b/be/src/olap/row_block2.cpp#L292)
   - [olap/types.h](https://github.com/freesinger/incubator-doris/blob/6e23f46b48a78ea620d891b0184133ec97c3742b/be/src/olap/types.h#L1357)
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r920722912


##########
be/src/vec/common/string_ref.h:
##########
@@ -73,6 +75,18 @@ struct StringRef {
     static StringRef from_string_val(StringVal sv) {
         return StringRef(reinterpret_cast<char*>(sv.ptr), sv.len);
     }
+
+    doris::JsonbDocument& to_jsonb_doc() const {

Review Comment:
   fixed, make it as a normal function in data_type_json.cpp



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r921819208


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/JsonLiteral.java:
##########
@@ -0,0 +1,197 @@
+// 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.analysis;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Objects;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.JsonParser;
+import com.google.gson.JsonSyntaxException;
+
+import org.apache.doris.common.io.Text;
+import org.apache.doris.thrift.TExprNode;
+import org.apache.doris.thrift.TExprNodeType;
+import org.apache.doris.thrift.TJsonLiteral;
+import org.apache.logging.log4j.LogManager;
+import org.apache.doris.catalog.Type;
+import org.apache.logging.log4j.Logger;
+
+import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.qe.VariableVarConverters;
+
+public class JsonLiteral extends LiteralExpr {
+    private static final Logger LOG = LogManager.getLogger(JsonLiteral.class);
+    private JsonParser parser = new JsonParser();
+    private String value;
+    // Means the converted session variable need to be cast to int, such as "cast 'STRICT_TRANS_TABLES' to Integer".
+    private String beConverted = "";
+
+    public JsonLiteral() {
+        super();
+        type = Type.JSON;
+    }
+
+    public JsonLiteral(String value) throws AnalysisException {
+        try {
+            parser.parse(value);

Review Comment:
   Actually yes



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r975507274


##########
be/src/vec/data_types/data_type_jsonb.cpp:
##########
@@ -0,0 +1,181 @@
+// 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 "data_type_jsonb.h"
+
+#include "gen_cpp/data.pb.h"
+#include "vec/columns/column_const.h"
+#include "vec/columns/column_jsonb.h"
+#include "vec/common/assert_cast.h"
+#include "vec/core/field.h"
+#include "vec/io/io_helper.h"
+
+#ifdef __SSE2__
+#include <emmintrin.h>
+#endif
+
+namespace doris::vectorized {
+
+template <typename Reader>
+static inline void read(IColumn& column, Reader&& reader) {
+    ColumnJsonb& column_json = assert_cast<ColumnJsonb&>(column);
+    ColumnJsonb::Chars& data = column_json.get_chars();
+    ColumnJsonb::Offsets& offsets = column_json.get_offsets();
+    size_t old_chars_size = data.size();
+    size_t old_offsets_size = offsets.size();
+    try {
+        reader(data);
+        data.push_back(0);
+        offsets.push_back(data.size());
+    } catch (...) {
+        offsets.resize_assume_reserved(old_offsets_size);
+        data.resize_assume_reserved(old_chars_size);
+        throw;
+    }
+}
+
+std::string DataTypeJsonb::to_string(const IColumn& column, size_t row_num) const {
+    JsonbToJson toStr;
+    const StringRef& s =
+            reinterpret_cast<const ColumnJsonb&>(*column.convert_to_full_column_if_const().get())
+                    .get_data_at(row_num);
+    return toStr.jsonb_to_string(s.data, s.size);
+}
+
+void DataTypeJsonb::to_string(const class doris::vectorized::IColumn& column, size_t row_num,
+                              class doris::vectorized::BufferWritable& ostr) const {
+    JsonbToJson toStr;
+    const StringRef& s =
+            reinterpret_cast<const ColumnJsonb&>(*column.convert_to_full_column_if_const().get())
+                    .get_data_at(row_num);
+    std::string str = toStr.jsonb_to_string(s.data, s.size);
+    ostr.write(str.c_str(), str.size());
+}
+
+MutableColumnPtr DataTypeJsonb::create_column() const {
+    return ColumnJsonb::create();
+}
+
+bool DataTypeJsonb::equals(const IDataType& rhs) const {
+    return typeid(rhs) == typeid(*this);
+}
+
+int64_t DataTypeJsonb::get_uncompressed_serialized_bytes(const IColumn& column,
+                                                         int data_version) const {
+    auto ptr = column.convert_to_full_column_if_const();
+    const auto& data_column = assert_cast<const ColumnJsonb&>(*ptr.get());
+
+    if (data_version == -1) {
+        return sizeof(IColumn::Offset) * (column.size() + 1) + sizeof(uint64_t) +
+               data_column.get_chars().size() + column.size();
+    }
+
+    return sizeof(IColumn::Offset) * (column.size() + 1) + sizeof(uint64_t) +
+           data_column.get_chars().size();
+}
+
+char* DataTypeJsonb::serialize(const IColumn& column, char* buf, int data_version) const {
+    auto ptr = column.convert_to_full_column_if_const();
+    const auto& data_column = assert_cast<const ColumnJsonb&>(*ptr.get());
+
+    if (data_version == -1) {
+        // row num
+        *reinterpret_cast<IColumn::Offset*>(buf) = column.size();
+        buf += sizeof(IColumn::Offset);
+        // offsets
+        for (int i = 0; i < column.size(); i++) {
+            *reinterpret_cast<IColumn::Offset*>(buf) = data_column.get_offsets()[i] + i + 1;
+            buf += sizeof(IColumn::Offset);
+        }
+        // total length
+        *reinterpret_cast<uint64_t*>(buf) = data_column.get_chars().size() + column.size();
+        buf += sizeof(uint64_t);
+        // values
+        for (int i = 0; i < column.size(); i++) {
+            auto data = data_column.get_data_at(i);
+            memcpy(buf, data.data, data.size);
+            buf += data.size;
+            *buf = '\0';
+            buf++;
+        }
+        return buf;
+    }
+
+    // row num
+    *reinterpret_cast<IColumn::Offset*>(buf) = column.size();
+    buf += sizeof(IColumn::Offset);
+    // offsets
+    memcpy(buf, data_column.get_offsets().data(), column.size() * sizeof(IColumn::Offset));
+    buf += column.size() * sizeof(IColumn::Offset);
+    // total length
+    uint64_t value_len = data_column.get_chars().size();
+    *reinterpret_cast<uint64_t*>(buf) = value_len;
+    buf += sizeof(uint64_t);
+    // values
+    memcpy(buf, data_column.get_chars().data(), value_len);
+    buf += value_len;
+
+    return buf;
+}
+
+const char* DataTypeJsonb::deserialize(const char* buf, IColumn* column, int data_version) const {
+    ColumnJsonb* column_string = assert_cast<ColumnJsonb*>(column);
+    ColumnJsonb::Chars& data = column_string->get_chars();
+    ColumnJsonb::Offsets& offsets = column_string->get_offsets();
+
+    if (data_version == -1) {
+        // row num
+        IColumn::Offset row_num = *reinterpret_cast<const IColumn::Offset*>(buf);
+        buf += sizeof(IColumn::Offset);
+        // offsets
+        offsets.resize(row_num);
+        for (int i = 0; i < row_num; i++) {
+            offsets[i] = *reinterpret_cast<const IColumn::Offset*>(buf) - i - 1;
+            buf += sizeof(IColumn::Offset);
+        }
+        // total length
+        uint64_t value_len = *reinterpret_cast<const uint64_t*>(buf);
+        buf += sizeof(uint64_t);
+        // values
+        data.resize(value_len - row_num);
+        for (int i = 0; i < row_num; i++) {
+            memcpy(data.data() + offsets[i - 1], buf, offsets[i] - offsets[i - 1]);
+            buf += offsets[i] - offsets[i - 1] + 1;

Review Comment:
   follow ColumnString to drop \0 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on PR #10322:
URL: https://github.com/apache/doris/pull/10322#issuecomment-1252523697

   LGTM


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r923966756


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/JsonLiteral.java:
##########
@@ -0,0 +1,197 @@
+// 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.analysis;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Objects;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.JsonParser;
+import com.google.gson.JsonSyntaxException;
+
+import org.apache.doris.common.io.Text;
+import org.apache.doris.thrift.TExprNode;
+import org.apache.doris.thrift.TExprNodeType;
+import org.apache.doris.thrift.TJsonLiteral;
+import org.apache.logging.log4j.LogManager;
+import org.apache.doris.catalog.Type;
+import org.apache.logging.log4j.Logger;
+
+import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.qe.VariableVarConverters;
+
+public class JsonLiteral extends LiteralExpr {
+    private static final Logger LOG = LogManager.getLogger(JsonLiteral.class);
+    private JsonParser parser = new JsonParser();
+    private String value;
+    // Means the converted session variable need to be cast to int, such as "cast 'STRICT_TRANS_TABLES' to Integer".
+    private String beConverted = "";
+
+    public JsonLiteral() {
+        super();
+        type = Type.JSON;
+    }
+
+    public JsonLiteral(String value) throws AnalysisException {
+        try {
+            parser.parse(value);

Review Comment:
   we can just accept the value and do parse json work in be.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r933274534


##########
be/src/olap/types.h:
##########
@@ -1454,6 +1460,66 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSON> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSON values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        auto jdoc = JsonbDocument::createDocument(scan_key.c_str(), scan_key.size());
+        size_t value_len = jdoc->numPackedBytes();
+        if (value_len > config::json_type_length_soft_limit_bytes) {
+            LOG(WARNING) << "the len of value json is too long, len=" << value_len
+                         << ", max_len=" << config::json_type_length_soft_limit_bytes;
+            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        }
+
+        auto slice = reinterpret_cast<Slice*>(buf);
+        memory_copy(slice->data, reinterpret_cast<const char*>(jdoc->getValue()), value_len);
+        slice->size = value_len;
+        return Status::OK();
+    }
+
+    static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
+                               MemPool* mem_pool, size_t variable_len = 0) {
+        JsonbToJson toStr;
+        switch (src_type->type()) {
+        case OLAP_FIELD_TYPE_TINYINT:
+        case OLAP_FIELD_TYPE_SMALLINT:
+        case OLAP_FIELD_TYPE_INT:
+        case OLAP_FIELD_TYPE_BIGINT:
+        case OLAP_FIELD_TYPE_LARGEINT:
+        case OLAP_FIELD_TYPE_FLOAT:
+        case OLAP_FIELD_TYPE_DOUBLE:
+        case OLAP_FIELD_TYPE_DECIMAL: {
+            auto s = src_type->to_string(src);
+            std::string result =
+                    toStr.json(JsonbDocument::createDocument(s.c_str(), s.size())->getValue());

Review Comment:
   no, this should be optimized to the level if industry SQL standard



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928215954


##########
be/src/vec/core/field.h:
##########
@@ -89,6 +89,89 @@ struct AggregateFunctionStateData {
     }
 };
 
+class JsonField {
+public:
+    JsonField() = default;
+
+    JsonField(const char* ptr, uint len) : size(len) {

Review Comment:
   uint32_t



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928225882


##########
be/src/vec/columns/column_json.h:
##########
@@ -0,0 +1,299 @@
+// 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.
+
+#pragma once
+
+#include <cassert>
+#include <cstring>
+
+#include "vec/columns/column.h"
+#include "vec/columns/column_impl.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/memcmp_small.h"
+#include "vec/common/memcpy_small.h"
+#include "vec/common/pod_array.h"
+#include "vec/common/sip_hash.h"
+#include "vec/core/field.h"
+
+namespace doris::vectorized {
+class ColumnJson final : public COWHelper<IColumn, ColumnJson> {
+public:
+    using Char = UInt8;
+    using Chars = PaddedPODArray<UInt8>;
+
+private:
+    friend class COWHelper<IColumn, ColumnJson>;
+
+    Offsets offsets;
+
+    Chars chars;
+
+    size_t ALWAYS_INLINE offset_at(ssize_t i) const { return offsets[i - 1]; }
+
+    size_t ALWAYS_INLINE size_at(ssize_t i) const { return offsets[i] - offsets[i - 1]; }
+
+    template <bool positive>
+    struct less;
+
+    template <bool positive>
+    struct lessWithCollation;
+
+    ColumnJson() = default;
+
+    ColumnJson(const ColumnJson& src)
+            : offsets(src.offsets.begin(), src.offsets.end()),
+              chars(src.chars.begin(), src.chars.end()) {}
+
+public:
+    const char* get_family_name() const override { return "JSON"; }
+
+    size_t size() const override { return offsets.size(); }
+
+    size_t byte_size() const override { return chars.size() + offsets.size() * sizeof(offsets[0]); }
+
+    size_t allocated_bytes() const override {
+        return chars.allocated_bytes() + offsets.allocated_bytes();
+    }
+
+    void protect() override;
+
+    MutableColumnPtr clone_resized(size_t to_size) const override;
+
+    Field operator[](size_t n) const override {
+        assert(n < size());
+        return Field(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    void get(size_t n, Field& res) const override {
+        assert(n < size());
+        res.assign_json(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    StringRef get_data_at(size_t n) const override {
+        assert(n < size());
+        return StringRef(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+/// Suppress gcc 7.3.1 warning: '*((void*)&<anonymous> +8)' may be used uninitialized in this function
+#if !__clang__
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
+#endif
+
+    void insert(const Field& x) override {
+        const JsonField& s = doris::vectorized::get<const JsonField&>(x);
+
+        const size_t old_size = chars.size();
+        const size_t size_to_append = s.get_size() + 1;

Review Comment:
   is extra 1 byte necessary for ColumnJSON?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928255341


##########
be/src/vec/columns/column_json.h:
##########
@@ -0,0 +1,299 @@
+// 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.
+
+#pragma once
+
+#include <cassert>
+#include <cstring>
+
+#include "vec/columns/column.h"
+#include "vec/columns/column_impl.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/memcmp_small.h"
+#include "vec/common/memcpy_small.h"
+#include "vec/common/pod_array.h"
+#include "vec/common/sip_hash.h"
+#include "vec/core/field.h"
+
+namespace doris::vectorized {
+class ColumnJson final : public COWHelper<IColumn, ColumnJson> {

Review Comment:
   ColumnJSON is almost the same as ColumnString, can we implement ColumnJSON based on rather than copy ColumnString, eg. extend or compose ColumnString.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928234058


##########
be/src/vec/olap/olap_data_convertor.cpp:
##########
@@ -577,6 +580,77 @@ Status OlapBlockDataConvertor::OlapColumnDataConvertorDate::convert_to_olap() {
     }
 }
 
+// class OlapBlockDataConvertor::OlapColumnDataConvertorJson
+void OlapBlockDataConvertor::OlapColumnDataConvertorJson::set_source_column(
+        const ColumnWithTypeAndName& typed_column, size_t row_pos, size_t num_rows) {
+    OlapBlockDataConvertor::OlapColumnDataConvertorBase::set_source_column(typed_column, row_pos,
+                                                                           num_rows);
+    _slice.resize(num_rows);
+}
+
+const void* OlapBlockDataConvertor::OlapColumnDataConvertorJson::get_data() const {
+    return _slice.data();
+}
+
+const void* OlapBlockDataConvertor::OlapColumnDataConvertorJson::get_data_at(size_t offset) const {
+    assert(offset < _num_rows && _num_rows == _slice.size());
+    UInt8 null_flag = 0;
+    if (_nullmap) {
+        null_flag = _nullmap[offset];

Review Comment:
   + _row_pos



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on PR #10322:
URL: https://github.com/apache/doris/pull/10322#issuecomment-1162660289

   > No UT is included in this PR?
   
   yep, I may add this later since this is really a heavy work


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r923960417


##########
be/src/runtime/json_value.cpp:
##########
@@ -0,0 +1,54 @@
+// 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/json_value.h"
+
+#include <cstring>
+
+#include "util/jsonb_error.h"
+
+namespace doris {
+
+JsonbErrType JsonValue::from_json_str(std::string s) {
+    JsonbErrType error = JsonbErrType::E_NONE;
+    if (!parser.parse(s)) {
+        error = parser.getErrorCode();
+        // TODO(wzy): document must be an object or an array,
+        // rune, pure-string, numeirc are valid JSON but get parse error here
+        // should return error gracefully to client
+        LOG(FATAL) << "invalid json value: " << JsonbErrMsg::getErrMsg(error);
+    }
+    ptr = parser.getWriter().getOutput()->getBuffer();
+    len = (unsigned)parser.getWriter().getOutput()->getSize();
+    DCHECK_LE(len, MAX_LENGTH);
+    return error;
+}
+
+std::string JsonValue::to_string() const {
+    JsonbToJson toStr;
+    return toStr.json(JsonbDocument::createDocument(ptr, len)->getValue());
+}
+
+std::ostream& operator<<(std::ostream& os, const JsonValue& json_value) {
+    return os << json_value.to_string();
+}
+
+std::size_t operator-(const JsonValue& v1, const JsonValue& v2) {

Review Comment:
   operator- is not accurate. Is it necessary? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r923961127


##########
be/src/runtime/json_value.h:
##########
@@ -0,0 +1,135 @@
+// 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.
+
+#ifndef DORIS_BE_RUNTIME_JSON_VALUE_H
+#define DORIS_BE_RUNTIME_JSON_VALUE_H
+
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/jsonb_error.h"
+#include "util/jsonb_parser.h"
+#include "util/jsonb_utils.h"
+#include "vec/common/string_ref.h"
+
+#ifdef __SSE4_2__
+#include "util/sse_util.hpp"
+#endif
+
+namespace doris {
+
+struct JsonValue {
+    static const int MAX_LENGTH = (1 << 30);
+
+    const char* ptr;
+    size_t len;
+    JsonbParser parser;
+
+    JsonValue() : ptr(nullptr), len(0) {}
+
+    JsonValue(char* ptr, int len) {
+        DCHECK_LE(len, MAX_LENGTH);
+        DCHECK(parser.parse(const_cast<const char*>(ptr), len));
+        this->ptr = parser.getWriter().getOutput()->getBuffer();
+        this->len = (unsigned)parser.getWriter().getOutput()->getSize();
+    }
+
+    JsonValue(const char* ptr, int len) {

Review Comment:
   we should extract it to a common function accept const char*, and call it in char* and string version.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928216650


##########
be/src/vec/core/field.h:
##########
@@ -572,13 +671,21 @@ class Field {
         create(reinterpret_cast<const char*>(data), size);
     }
 
+    void create_json(const unsigned char* data, size_t size) {
+        new (&storage) JsonField(reinterpret_cast<const char*>(data), size);
+        which = Types::JSON;
+    }
+
     ALWAYS_INLINE void destroy() {
         if (which < Types::MIN_NON_POD) return;
 
         switch (which) {
         case Types::String:
             destroy<String>();
             break;
+        case Types::JSON:
+            destroy<JsonField>();

Review Comment:
   where is the implementation of destroy<JsonField>() ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928253550


##########
be/src/vec/columns/column_json.h:
##########
@@ -0,0 +1,299 @@
+// 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.
+
+#pragma once
+
+#include <cassert>
+#include <cstring>
+
+#include "vec/columns/column.h"
+#include "vec/columns/column_impl.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/memcmp_small.h"
+#include "vec/common/memcpy_small.h"
+#include "vec/common/pod_array.h"
+#include "vec/common/sip_hash.h"
+#include "vec/core/field.h"
+
+namespace doris::vectorized {
+class ColumnJson final : public COWHelper<IColumn, ColumnJson> {
+public:
+    using Char = UInt8;
+    using Chars = PaddedPODArray<UInt8>;
+
+private:
+    friend class COWHelper<IColumn, ColumnJson>;
+
+    Offsets offsets;
+
+    Chars chars;
+
+    size_t ALWAYS_INLINE offset_at(ssize_t i) const { return offsets[i - 1]; }
+
+    size_t ALWAYS_INLINE size_at(ssize_t i) const { return offsets[i] - offsets[i - 1]; }
+
+    template <bool positive>
+    struct less;
+
+    template <bool positive>
+    struct lessWithCollation;
+
+    ColumnJson() = default;
+
+    ColumnJson(const ColumnJson& src)
+            : offsets(src.offsets.begin(), src.offsets.end()),
+              chars(src.chars.begin(), src.chars.end()) {}
+
+public:
+    const char* get_family_name() const override { return "JSON"; }
+
+    size_t size() const override { return offsets.size(); }
+
+    size_t byte_size() const override { return chars.size() + offsets.size() * sizeof(offsets[0]); }
+
+    size_t allocated_bytes() const override {
+        return chars.allocated_bytes() + offsets.allocated_bytes();
+    }
+
+    void protect() override;
+
+    MutableColumnPtr clone_resized(size_t to_size) const override;
+
+    Field operator[](size_t n) const override {
+        assert(n < size());
+        return Field(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    void get(size_t n, Field& res) const override {
+        assert(n < size());
+        res.assign_json(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    StringRef get_data_at(size_t n) const override {
+        assert(n < size());
+        return StringRef(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+/// Suppress gcc 7.3.1 warning: '*((void*)&<anonymous> +8)' may be used uninitialized in this function
+#if !__clang__
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
+#endif
+
+    void insert(const Field& x) override {
+        const JsonField& s = doris::vectorized::get<const JsonField&>(x);
+
+        const size_t old_size = chars.size();
+        const size_t size_to_append = s.get_size() + 1;
+        const size_t new_size = old_size + size_to_append;
+
+        chars.resize(new_size);
+        memcpy(chars.data() + old_size, s.get_value(), size_to_append);
+        offsets.push_back(new_size);
+    }
+
+#if !__clang__
+#pragma GCC diagnostic pop
+#endif
+
+    void insert_from(const IColumn& src_, size_t n) override {
+        const ColumnJson& src = assert_cast<const ColumnJson&>(src_);
+        const size_t size_to_append =
+                src.offsets[n] - src.offsets[n - 1]; /// -1th index is Ok, see PaddedPODArray.
+
+        if (size_to_append == 1) {
+            /// shortcut for empty json
+            chars.push_back(0);
+            offsets.push_back(chars.size());
+        } else {
+            const size_t old_size = chars.size();
+            const size_t offset = src.offsets[n - 1];
+            const size_t new_size = old_size + size_to_append;
+
+            chars.resize(new_size);
+            memcpy_small_allow_read_write_overflow15(chars.data() + old_size, &src.chars[offset],
+                                                     size_to_append);
+            offsets.push_back(new_size);
+        }
+    }
+
+    void insert_data(const char* pos, size_t length) override {
+        const size_t old_size = chars.size();
+        const size_t new_size = old_size + length + 1;
+
+        chars.resize(new_size);
+        if (length) memcpy(chars.data() + old_size, pos, length);
+        chars[old_size + length] = 0;
+        offsets.push_back(new_size);
+    }
+
+    void insert_many_binary_data(char* data_array, uint32_t* len_array,
+                                 uint32_t* start_offset_array, size_t num) override {
+        for (size_t i = 0; i < num; i++) {
+            uint32_t len = len_array[i];
+            uint32_t start_offset = start_offset_array[i];
+            insert_data(data_array + start_offset, len);
+        }
+    };
+
+    void insert_many_dict_data(const int32_t* data_array, size_t start_index, const StringRef* dict,
+                               size_t num, uint32_t /*dict_num*/) {
+        for (size_t end_index = start_index + num; start_index < end_index; ++start_index) {
+            int32_t codeword = data_array[start_index];
+            insert_data(dict[codeword].data, dict[codeword].size);
+        }
+    }
+
+    void pop_back(size_t n) override {
+        size_t nested_n = offsets.back() - offset_at(offsets.size() - n);
+        chars.resize(chars.size() - nested_n);
+        offsets.resize_assume_reserved(offsets.size() - n);
+    }
+
+    StringRef serialize_value_into_arena(size_t n, Arena& arena, char const*& begin) const override;
+
+    const char* deserialize_and_insert_from_arena(const char* pos) override;
+
+    void update_hash_with_value(size_t n, SipHash& hash) const override {
+        size_t string_size = size_at(n);
+        size_t offset = offset_at(n);
+
+        hash.update(reinterpret_cast<const char*>(&string_size), sizeof(string_size));
+        hash.update(reinterpret_cast<const char*>(&chars[offset]), string_size);
+    }
+
+    void insert_range_from(const IColumn& src, size_t start, size_t length) override;
+
+    void insert_indices_from(const IColumn& src, const int* indices_begin,
+                             const int* indices_end) override;
+
+    ColumnPtr filter(const Filter& filt, ssize_t result_size_hint) const override;
+
+    ColumnPtr permute(const Permutation& perm, size_t limit) const override;
+
+    //    ColumnPtr index(const IColumn & indexes, size_t limit) const override;
+
+    template <typename Type>
+    ColumnPtr index_impl(const PaddedPODArray<Type>& indexes, size_t limit) const;
+
+    void insert_default() override {
+        chars.push_back(0);

Review Comment:
   can default value "" be processed by later function, such as deserialize?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928256700


##########
be/src/olap/types.h:
##########
@@ -1454,6 +1460,66 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSON> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSON values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        auto jdoc = JsonbDocument::createDocument(scan_key.c_str(), scan_key.size());
+        size_t value_len = jdoc->numPackedBytes();
+        if (value_len > config::json_type_length_soft_limit_bytes) {
+            LOG(WARNING) << "the len of value json is too long, len=" << value_len
+                         << ", max_len=" << config::json_type_length_soft_limit_bytes;
+            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        }
+
+        auto slice = reinterpret_cast<Slice*>(buf);
+        memory_copy(slice->data, reinterpret_cast<const char*>(jdoc->getValue()), value_len);
+        slice->size = value_len;
+        return Status::OK();
+    }
+
+    static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
+                               MemPool* mem_pool, size_t variable_len = 0) {
+        JsonbToJson toStr;
+        switch (src_type->type()) {
+        case OLAP_FIELD_TYPE_TINYINT:
+        case OLAP_FIELD_TYPE_SMALLINT:
+        case OLAP_FIELD_TYPE_INT:
+        case OLAP_FIELD_TYPE_BIGINT:
+        case OLAP_FIELD_TYPE_LARGEINT:
+        case OLAP_FIELD_TYPE_FLOAT:
+        case OLAP_FIELD_TYPE_DOUBLE:
+        case OLAP_FIELD_TYPE_DECIMAL: {
+            auto s = src_type->to_string(src);
+            std::string result =
+                    toStr.json(JsonbDocument::createDocument(s.c_str(), s.size())->getValue());
+            auto slice = reinterpret_cast<Slice*>(dest);
+            slice->data = reinterpret_cast<char*>(mem_pool->allocate(result.size()));
+            memcpy(slice->data, result.c_str(), result.size());
+            slice->size = result.size();
+            return Status::OK();
+        }
+        default:
+            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        }
+    }
+
+    static void set_to_min(void* buf) {
+        auto slice = reinterpret_cast<Slice*>(buf);
+        slice->size = 0;
+    }
+
+    static void set_to_max(void* buf) {
+        auto slice = reinterpret_cast<Slice*>(buf);
+        slice->size = OLAP_JSON_MAX_LENGTH; // 2G

Review Comment:
   it may cause big memory allocation problem



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r933120201


##########
be/src/vec/columns/column_json.h:
##########
@@ -0,0 +1,299 @@
+// 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.
+
+#pragma once
+
+#include <cassert>
+#include <cstring>
+
+#include "vec/columns/column.h"
+#include "vec/columns/column_impl.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/memcmp_small.h"
+#include "vec/common/memcpy_small.h"
+#include "vec/common/pod_array.h"
+#include "vec/common/sip_hash.h"
+#include "vec/core/field.h"
+
+namespace doris::vectorized {
+class ColumnJson final : public COWHelper<IColumn, ColumnJson> {
+public:
+    using Char = UInt8;
+    using Chars = PaddedPODArray<UInt8>;
+
+private:
+    friend class COWHelper<IColumn, ColumnJson>;
+
+    Offsets offsets;
+
+    Chars chars;
+
+    size_t ALWAYS_INLINE offset_at(ssize_t i) const { return offsets[i - 1]; }
+
+    size_t ALWAYS_INLINE size_at(ssize_t i) const { return offsets[i] - offsets[i - 1]; }
+
+    template <bool positive>
+    struct less;
+
+    template <bool positive>
+    struct lessWithCollation;
+
+    ColumnJson() = default;
+
+    ColumnJson(const ColumnJson& src)
+            : offsets(src.offsets.begin(), src.offsets.end()),
+              chars(src.chars.begin(), src.chars.end()) {}
+
+public:
+    const char* get_family_name() const override { return "JSON"; }
+
+    size_t size() const override { return offsets.size(); }
+
+    size_t byte_size() const override { return chars.size() + offsets.size() * sizeof(offsets[0]); }
+
+    size_t allocated_bytes() const override {
+        return chars.allocated_bytes() + offsets.allocated_bytes();
+    }
+
+    void protect() override;
+
+    MutableColumnPtr clone_resized(size_t to_size) const override;
+
+    Field operator[](size_t n) const override {
+        assert(n < size());
+        return Field(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    void get(size_t n, Field& res) const override {
+        assert(n < size());
+        res.assign_json(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    StringRef get_data_at(size_t n) const override {
+        assert(n < size());
+        return StringRef(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+/// Suppress gcc 7.3.1 warning: '*((void*)&<anonymous> +8)' may be used uninitialized in this function
+#if !__clang__
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
+#endif
+
+    void insert(const Field& x) override {
+        const JsonField& s = doris::vectorized::get<const JsonField&>(x);
+
+        const size_t old_size = chars.size();
+        const size_t size_to_append = s.get_size() + 1;
+        const size_t new_size = old_size + size_to_append;
+
+        chars.resize(new_size);
+        memcpy(chars.data() + old_size, s.get_value(), size_to_append);
+        offsets.push_back(new_size);
+    }
+
+#if !__clang__
+#pragma GCC diagnostic pop
+#endif
+
+    void insert_from(const IColumn& src_, size_t n) override {
+        const ColumnJson& src = assert_cast<const ColumnJson&>(src_);
+        const size_t size_to_append =
+                src.offsets[n] - src.offsets[n - 1]; /// -1th index is Ok, see PaddedPODArray.
+
+        if (size_to_append == 1) {
+            /// shortcut for empty json
+            chars.push_back(0);
+            offsets.push_back(chars.size());
+        } else {
+            const size_t old_size = chars.size();
+            const size_t offset = src.offsets[n - 1];
+            const size_t new_size = old_size + size_to_append;
+
+            chars.resize(new_size);
+            memcpy_small_allow_read_write_overflow15(chars.data() + old_size, &src.chars[offset],
+                                                     size_to_append);
+            offsets.push_back(new_size);
+        }
+    }
+
+    void insert_data(const char* pos, size_t length) override {
+        const size_t old_size = chars.size();
+        const size_t new_size = old_size + length + 1;
+
+        chars.resize(new_size);
+        if (length) memcpy(chars.data() + old_size, pos, length);
+        chars[old_size + length] = 0;
+        offsets.push_back(new_size);
+    }
+
+    void insert_many_binary_data(char* data_array, uint32_t* len_array,
+                                 uint32_t* start_offset_array, size_t num) override {
+        for (size_t i = 0; i < num; i++) {
+            uint32_t len = len_array[i];
+            uint32_t start_offset = start_offset_array[i];
+            insert_data(data_array + start_offset, len);
+        }
+    };
+
+    void insert_many_dict_data(const int32_t* data_array, size_t start_index, const StringRef* dict,
+                               size_t num, uint32_t /*dict_num*/) {
+        for (size_t end_index = start_index + num; start_index < end_index; ++start_index) {
+            int32_t codeword = data_array[start_index];
+            insert_data(dict[codeword].data, dict[codeword].size);
+        }
+    }
+
+    void pop_back(size_t n) override {
+        size_t nested_n = offsets.back() - offset_at(offsets.size() - n);
+        chars.resize(chars.size() - nested_n);
+        offsets.resize_assume_reserved(offsets.size() - n);
+    }
+
+    StringRef serialize_value_into_arena(size_t n, Arena& arena, char const*& begin) const override;
+
+    const char* deserialize_and_insert_from_arena(const char* pos) override;
+
+    void update_hash_with_value(size_t n, SipHash& hash) const override {
+        size_t string_size = size_at(n);
+        size_t offset = offset_at(n);
+
+        hash.update(reinterpret_cast<const char*>(&string_size), sizeof(string_size));
+        hash.update(reinterpret_cast<const char*>(&chars[offset]), string_size);
+    }
+
+    void insert_range_from(const IColumn& src, size_t start, size_t length) override;
+
+    void insert_indices_from(const IColumn& src, const int* indices_begin,
+                             const int* indices_end) override;
+
+    ColumnPtr filter(const Filter& filt, ssize_t result_size_hint) const override;
+
+    ColumnPtr permute(const Permutation& perm, size_t limit) const override;
+
+    //    ColumnPtr index(const IColumn & indexes, size_t limit) const override;
+
+    template <typename Type>
+    ColumnPtr index_impl(const PaddedPODArray<Type>& indexes, size_t limit) const;
+
+    void insert_default() override {
+        chars.push_back(0);
+        offsets.push_back(offsets.back() + 1);
+    }
+
+    void insert_many_defaults(size_t length) override {
+        size_t chars_old_size = chars.size();
+        chars.resize(chars_old_size + length);
+        memset(chars.data() + chars_old_size, 0, length);
+
+        const size_t old_size = offsets.size();
+        const size_t new_size = old_size + length;
+        const auto num = offsets.back() + 1;
+        offsets.resize_fill(new_size, num);
+        for (size_t i = old_size, j = 0; i < new_size; i++, j++) {
+            offsets[i] += j;
+        }
+    }
+
+    int compare_at(size_t n, size_t m, const IColumn& rhs_,
+                   int /*nan_direction_hint*/) const override {
+        const ColumnJson& rhs = assert_cast<const ColumnJson&>(rhs_);
+        return memcmp_small_allow_overflow15(chars.data() + offset_at(n), size_at(n) - 1,

Review Comment:
   LOG(FATAL) instead



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r933283403


##########
be/src/olap/types.h:
##########
@@ -1454,6 +1460,66 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSON> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSON values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        auto jdoc = JsonbDocument::createDocument(scan_key.c_str(), scan_key.size());
+        size_t value_len = jdoc->numPackedBytes();
+        if (value_len > config::json_type_length_soft_limit_bytes) {
+            LOG(WARNING) << "the len of value json is too long, len=" << value_len
+                         << ", max_len=" << config::json_type_length_soft_limit_bytes;
+            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        }
+
+        auto slice = reinterpret_cast<Slice*>(buf);
+        memory_copy(slice->data, reinterpret_cast<const char*>(jdoc->getValue()), value_len);
+        slice->size = value_len;
+        return Status::OK();
+    }
+
+    static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
+                               MemPool* mem_pool, size_t variable_len = 0) {
+        JsonbToJson toStr;
+        switch (src_type->type()) {
+        case OLAP_FIELD_TYPE_TINYINT:
+        case OLAP_FIELD_TYPE_SMALLINT:
+        case OLAP_FIELD_TYPE_INT:
+        case OLAP_FIELD_TYPE_BIGINT:
+        case OLAP_FIELD_TYPE_LARGEINT:
+        case OLAP_FIELD_TYPE_FLOAT:
+        case OLAP_FIELD_TYPE_DOUBLE:
+        case OLAP_FIELD_TYPE_DECIMAL: {
+            auto s = src_type->to_string(src);
+            std::string result =
+                    toStr.json(JsonbDocument::createDocument(s.c_str(), s.size())->getValue());
+            auto slice = reinterpret_cast<Slice*>(dest);
+            slice->data = reinterpret_cast<char*>(mem_pool->allocate(result.size()));
+            memcpy(slice->data, result.c_str(), result.size());
+            slice->size = result.size();
+            return Status::OK();
+        }
+        default:
+            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        }
+    }
+
+    static void set_to_min(void* buf) {
+        auto slice = reinterpret_cast<Slice*>(buf);
+        slice->size = 0;
+    }
+
+    static void set_to_max(void* buf) {
+        auto slice = reinterpret_cast<Slice*>(buf);
+        slice->size = OLAP_JSON_MAX_LENGTH; // 2G

Review Comment:
   change to 10M 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r918518686


##########
be/src/common/env_config.h.in:
##########
@@ -19,6 +19,6 @@
 
 namespace doris {
 
-#cmakedefine HAVE_SCHED_GETCPU @HAVE_SCHED_GETCPU@
+#cmakedefine HAVE_SCHED_GETCPU @HAVE_SCHED_GETCPU @

Review Comment:
   Caused by Clang-Format plugin, I will fix it 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r933135588


##########
be/src/vec/olap/olap_data_convertor.cpp:
##########
@@ -577,6 +580,77 @@ Status OlapBlockDataConvertor::OlapColumnDataConvertorDate::convert_to_olap() {
     }
 }
 
+// class OlapBlockDataConvertor::OlapColumnDataConvertorJson
+void OlapBlockDataConvertor::OlapColumnDataConvertorJson::set_source_column(
+        const ColumnWithTypeAndName& typed_column, size_t row_pos, size_t num_rows) {
+    OlapBlockDataConvertor::OlapColumnDataConvertorBase::set_source_column(typed_column, row_pos,
+                                                                           num_rows);
+    _slice.resize(num_rows);
+}
+
+const void* OlapBlockDataConvertor::OlapColumnDataConvertorJson::get_data() const {
+    return _slice.data();
+}
+
+const void* OlapBlockDataConvertor::OlapColumnDataConvertorJson::get_data_at(size_t offset) const {
+    assert(offset < _num_rows && _num_rows == _slice.size());
+    UInt8 null_flag = 0;
+    if (_nullmap) {
+        null_flag = _nullmap[offset];

Review Comment:
   no need for _row_pos here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r925374511


##########
be/src/runtime/json_value.cpp:
##########
@@ -0,0 +1,54 @@
+// 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/json_value.h"
+
+#include <cstring>
+
+#include "util/jsonb_error.h"
+
+namespace doris {
+
+JsonbErrType JsonValue::from_json_str(std::string s) {
+    JsonbErrType error = JsonbErrType::E_NONE;
+    if (!parser.parse(s)) {
+        error = parser.getErrorCode();
+        // TODO(wzy): document must be an object or an array,
+        // rune, pure-string, numeirc are valid JSON but get parse error here
+        // should return error gracefully to client
+        LOG(FATAL) << "invalid json value: " << JsonbErrMsg::getErrMsg(error);

Review Comment:
   No proper solutions here for now since there's no good way to return error.
   Best way is to make JSONB support official JSON standard, which validation result can be consistent with that from FE. So Doris may only need to validate JSON data once in FE and return error as early as possible.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r924035717


##########
be/src/runtime/json_value.cpp:
##########
@@ -0,0 +1,54 @@
+// 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/json_value.h"
+
+#include <cstring>
+
+#include "util/jsonb_error.h"
+
+namespace doris {
+
+JsonbErrType JsonValue::from_json_str(std::string s) {
+    JsonbErrType error = JsonbErrType::E_NONE;
+    if (!parser.parse(s)) {
+        error = parser.getErrorCode();
+        // TODO(wzy): document must be an object or an array,
+        // rune, pure-string, numeirc are valid JSON but get parse error here
+        // should return error gracefully to client
+        LOG(FATAL) << "invalid json value: " << JsonbErrMsg::getErrMsg(error);
+    }
+    ptr = parser.getWriter().getOutput()->getBuffer();
+    len = (unsigned)parser.getWriter().getOutput()->getSize();
+    DCHECK_LE(len, MAX_LENGTH);
+    return error;
+}
+
+std::string JsonValue::to_string() const {
+    JsonbToJson toStr;
+    return toStr.json(JsonbDocument::createDocument(ptr, len)->getValue());
+}
+
+std::ostream& operator<<(std::ostream& os, const JsonValue& json_value) {
+    return os << json_value.to_string();
+}
+
+std::size_t operator-(const JsonValue& v1, const JsonValue& v2) {

Review Comment:
   removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928239578


##########
be/src/vec/io/io_helper.h:
##########
@@ -122,6 +123,19 @@ inline void write_string_binary(const char* s, BufferWritable& buf) {
     write_string_binary(StringRef {s}, buf);
 }
 
+inline void write_json_binary(const StringRef& s, BufferWritable& buf) {
+    write_var_uint(s.size, buf);
+    buf.write(s.data, s.size);
+}
+
+inline void write_json_binary(const char* s, BufferWritable& buf) {
+    write_json_binary(StringRef {s}, buf);
+}
+
+inline void write_json_binary(JsonField s, BufferWritable& buf) {

Review Comment:
   just call write_string_binary



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928252969


##########
be/src/vec/columns/column_json.h:
##########
@@ -0,0 +1,299 @@
+// 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.
+
+#pragma once
+
+#include <cassert>
+#include <cstring>
+
+#include "vec/columns/column.h"
+#include "vec/columns/column_impl.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/memcmp_small.h"
+#include "vec/common/memcpy_small.h"
+#include "vec/common/pod_array.h"
+#include "vec/common/sip_hash.h"
+#include "vec/core/field.h"
+
+namespace doris::vectorized {
+class ColumnJson final : public COWHelper<IColumn, ColumnJson> {
+public:
+    using Char = UInt8;
+    using Chars = PaddedPODArray<UInt8>;
+
+private:
+    friend class COWHelper<IColumn, ColumnJson>;
+
+    Offsets offsets;
+
+    Chars chars;
+
+    size_t ALWAYS_INLINE offset_at(ssize_t i) const { return offsets[i - 1]; }
+
+    size_t ALWAYS_INLINE size_at(ssize_t i) const { return offsets[i] - offsets[i - 1]; }
+
+    template <bool positive>
+    struct less;
+
+    template <bool positive>
+    struct lessWithCollation;
+
+    ColumnJson() = default;
+
+    ColumnJson(const ColumnJson& src)
+            : offsets(src.offsets.begin(), src.offsets.end()),
+              chars(src.chars.begin(), src.chars.end()) {}
+
+public:
+    const char* get_family_name() const override { return "JSON"; }
+
+    size_t size() const override { return offsets.size(); }
+
+    size_t byte_size() const override { return chars.size() + offsets.size() * sizeof(offsets[0]); }
+
+    size_t allocated_bytes() const override {
+        return chars.allocated_bytes() + offsets.allocated_bytes();
+    }
+
+    void protect() override;
+
+    MutableColumnPtr clone_resized(size_t to_size) const override;
+
+    Field operator[](size_t n) const override {
+        assert(n < size());
+        return Field(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    void get(size_t n, Field& res) const override {
+        assert(n < size());
+        res.assign_json(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    StringRef get_data_at(size_t n) const override {
+        assert(n < size());
+        return StringRef(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+/// Suppress gcc 7.3.1 warning: '*((void*)&<anonymous> +8)' may be used uninitialized in this function
+#if !__clang__
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
+#endif
+
+    void insert(const Field& x) override {
+        const JsonField& s = doris::vectorized::get<const JsonField&>(x);
+
+        const size_t old_size = chars.size();
+        const size_t size_to_append = s.get_size() + 1;
+        const size_t new_size = old_size + size_to_append;
+
+        chars.resize(new_size);
+        memcpy(chars.data() + old_size, s.get_value(), size_to_append);
+        offsets.push_back(new_size);
+    }
+
+#if !__clang__
+#pragma GCC diagnostic pop
+#endif
+
+    void insert_from(const IColumn& src_, size_t n) override {
+        const ColumnJson& src = assert_cast<const ColumnJson&>(src_);
+        const size_t size_to_append =
+                src.offsets[n] - src.offsets[n - 1]; /// -1th index is Ok, see PaddedPODArray.
+
+        if (size_to_append == 1) {
+            /// shortcut for empty json
+            chars.push_back(0);
+            offsets.push_back(chars.size());
+        } else {
+            const size_t old_size = chars.size();
+            const size_t offset = src.offsets[n - 1];
+            const size_t new_size = old_size + size_to_append;
+
+            chars.resize(new_size);
+            memcpy_small_allow_read_write_overflow15(chars.data() + old_size, &src.chars[offset],
+                                                     size_to_append);
+            offsets.push_back(new_size);
+        }
+    }
+
+    void insert_data(const char* pos, size_t length) override {
+        const size_t old_size = chars.size();
+        const size_t new_size = old_size + length + 1;
+
+        chars.resize(new_size);
+        if (length) memcpy(chars.data() + old_size, pos, length);
+        chars[old_size + length] = 0;
+        offsets.push_back(new_size);
+    }
+
+    void insert_many_binary_data(char* data_array, uint32_t* len_array,
+                                 uint32_t* start_offset_array, size_t num) override {
+        for (size_t i = 0; i < num; i++) {
+            uint32_t len = len_array[i];
+            uint32_t start_offset = start_offset_array[i];
+            insert_data(data_array + start_offset, len);

Review Comment:
   call insert_data for each row is right but poor performance, just follow ColumnString's batch way.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928256518


##########
be/src/olap/types.h:
##########
@@ -1454,6 +1460,66 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSON> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSON values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        auto jdoc = JsonbDocument::createDocument(scan_key.c_str(), scan_key.size());
+        size_t value_len = jdoc->numPackedBytes();
+        if (value_len > config::json_type_length_soft_limit_bytes) {
+            LOG(WARNING) << "the len of value json is too long, len=" << value_len
+                         << ", max_len=" << config::json_type_length_soft_limit_bytes;
+            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        }
+
+        auto slice = reinterpret_cast<Slice*>(buf);
+        memory_copy(slice->data, reinterpret_cast<const char*>(jdoc->getValue()), value_len);
+        slice->size = value_len;
+        return Status::OK();
+    }
+
+    static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
+                               MemPool* mem_pool, size_t variable_len = 0) {
+        JsonbToJson toStr;
+        switch (src_type->type()) {
+        case OLAP_FIELD_TYPE_TINYINT:
+        case OLAP_FIELD_TYPE_SMALLINT:
+        case OLAP_FIELD_TYPE_INT:
+        case OLAP_FIELD_TYPE_BIGINT:
+        case OLAP_FIELD_TYPE_LARGEINT:
+        case OLAP_FIELD_TYPE_FLOAT:
+        case OLAP_FIELD_TYPE_DOUBLE:
+        case OLAP_FIELD_TYPE_DECIMAL: {
+            auto s = src_type->to_string(src);
+            std::string result =
+                    toStr.json(JsonbDocument::createDocument(s.c_str(), s.size())->getValue());

Review Comment:
   can int string be parsed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] kpfly commented on pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
kpfly commented on PR #10322:
URL: https://github.com/apache/doris/pull/10322#issuecomment-1162602436

   Wonderful work


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r933111884


##########
be/src/vec/core/field.h:
##########
@@ -572,13 +671,21 @@ class Field {
         create(reinterpret_cast<const char*>(data), size);
     }
 
+    void create_json(const unsigned char* data, size_t size) {
+        new (&storage) JsonField(reinterpret_cast<const char*>(data), size);
+        which = Types::JSON;
+    }
+
     ALWAYS_INLINE void destroy() {
         if (which < Types::MIN_NON_POD) return;
 
         switch (which) {
         case Types::String:
             destroy<String>();
             break;
+        case Types::JSON:
+            destroy<JsonField>();

Review Comment:
   ```C++
       template <typename T>
       void destroy() {
           T* MAY_ALIAS ptr = reinterpret_cast<T*>(&storage);
           ptr->~T();
       }
   ```
   [Line707](https://github.com/freesinger/incubator-doris/blob/878d392fb89991c9e80f9fff0936fe3ce97350de/be/src/vec/core/field.h#L707)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] cambyzju commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
cambyzju commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r956848313


##########
be/src/olap/olap_define.h:
##########
@@ -57,6 +57,9 @@ static const uint16_t OLAP_VARCHAR_MAX_LENGTH = 65535;
 // the max length supported for string type 2GB
 static const uint32_t OLAP_STRING_MAX_LENGTH = 2147483647;
 
+// the max length supported for json type 2GB
+static const uint32_t OLAP_JSON_MAX_LENGTH = 10 * 1024 * 1024;

Review Comment:
   It seems `10 * 1024 * 1024` do not equal `2GB`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r975505646


##########
be/src/util/mysql_row_buffer.cpp:
##########
@@ -25,6 +25,8 @@
 #include "date_func.h"
 #include "gutil/strings/numbers.h"
 #include "runtime/large_int_value.h"
+#include "util/jsonb_document.h"

Review Comment:
   not necessary any more



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r975509813


##########
be/src/olap/types.h:
##########
@@ -1559,6 +1566,65 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_JSONB> : public FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> {
+    static int cmp(const void* left, const void* right) {
+        LOG(WARNING) << "can not compare JSONB values";
+        return -1; // always update ?
+    }
+
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        JsonBinaryValue binary_val(scan_key.c_str(), scan_key.size());
+        auto jdoc = JsonbDocument::createDocument(binary_val.value(), binary_val.size());
+        size_t value_len = jdoc->numPackedBytes();
+        if (value_len > config::jsonb_type_length_soft_limit_bytes) {
+            LOG(WARNING) << "the len of value json is too long, len=" << value_len
+                         << ", max_len=" << config::jsonb_type_length_soft_limit_bytes;
+            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        }
+
+        auto slice = reinterpret_cast<Slice*>(buf);
+        memory_copy(slice->data, reinterpret_cast<const char*>(jdoc->getValue()), value_len);

Review Comment:
   just copy binary_val.value(), binary_val.size()



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928215324


##########
be/src/vec/exprs/vliteral.cpp:
##########
@@ -118,6 +120,13 @@ void VLiteral::init(const TExprNode& node) {
             field = node.string_literal.value;
             break;
         }
+        case TYPE_JSON: {
+            DCHECK_EQ(node.node_type, TExprNodeType::JSON_LITERAL);
+            DCHECK(node.__isset.json_literal);
+            JsonValue value(node.json_literal.value);

Review Comment:
   JsonValue is only used here, is it worth to create a class? maybe include json parser in JsonField is better.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928228269


##########
be/src/vec/exprs/vliteral.cpp:
##########
@@ -118,6 +120,13 @@ void VLiteral::init(const TExprNode& node) {
             field = node.string_literal.value;
             break;
         }
+        case TYPE_JSON: {
+            DCHECK_EQ(node.node_type, TExprNodeType::JSON_LITERAL);
+            DCHECK(node.__isset.json_literal);
+            JsonValue value(node.json_literal.value);

Review Comment:
   I do considered sole class `JsonField` is enough here,  but I think it's better to keep `JsonValue` and `JsonField` two different concepts, for that 
   - `to_string` functions is needed in `JsonValue` and used in `VLiteral::debug_string()`
   - `JsonField` is in memory representation of `JsonValue`, they have different semantics in logic
   -  Consistence with implementation such as `Decimal` is good, which seems better for me



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928252341


##########
be/src/vec/data_types/data_type_json.cpp:
##########
@@ -0,0 +1,132 @@
+// 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 "data_type_json.h"
+
+#include "gen_cpp/data.pb.h"
+#include "vec/columns/column_const.h"
+#include "vec/columns/column_json.h"
+#include "vec/common/assert_cast.h"
+#include "vec/core/field.h"
+#include "vec/io/io_helper.h"
+
+#ifdef __SSE2__
+#include <emmintrin.h>
+#endif
+
+namespace doris::vectorized {
+
+template <typename Reader>
+static inline void read(IColumn& column, Reader&& reader) {
+    ColumnJson& column_json = assert_cast<ColumnJson&>(column);
+    ColumnJson::Chars& data = column_json.get_chars();
+    ColumnJson::Offsets& offsets = column_json.get_offsets();
+    size_t old_chars_size = data.size();
+    size_t old_offsets_size = offsets.size();
+    try {
+        reader(data);
+        data.push_back(0);
+        offsets.push_back(data.size());
+    } catch (...) {
+        offsets.resize_assume_reserved(old_offsets_size);
+        data.resize_assume_reserved(old_chars_size);
+        throw;
+    }
+}
+
+std::string jsonb_to_string(const StringRef& s) {
+    doris::JsonbToJson toStr;
+    doris::JsonbValue* val = doris::JsonbDocument::createDocument(s.data, s.size)->getValue();
+    return toStr.json(val);
+}
+
+std::string DataTypeJson::to_string(const IColumn& column, size_t row_num) const {
+    const StringRef& s =
+            reinterpret_cast<const ColumnJson&>(*column.convert_to_full_column_if_const().get())
+                    .get_data_at(row_num);
+    return jsonb_to_string(s);
+}
+
+void DataTypeJson::to_string(const class doris::vectorized::IColumn& column, size_t row_num,
+                             class doris::vectorized::BufferWritable& ostr) const {
+    const StringRef& s =

Review Comment:
   call to_string above to avoid duplicated code



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928256067


##########
be/src/vec/data_types/data_type_json.h:
##########
@@ -0,0 +1,62 @@
+// 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.
+
+#pragma once
+
+#include <ostream>
+
+#include "vec/columns/column_json.h"
+#include "vec/data_types/data_type.h"
+
+namespace doris::vectorized {
+class DataTypeJson final : public IDataType {

Review Comment:
   DataTypeJson can be implemented based on DataTypeString



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928224089


##########
be/src/vec/columns/column_json.h:
##########
@@ -0,0 +1,299 @@
+// 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.
+
+#pragma once
+
+#include <cassert>
+#include <cstring>
+
+#include "vec/columns/column.h"
+#include "vec/columns/column_impl.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/memcmp_small.h"
+#include "vec/common/memcpy_small.h"
+#include "vec/common/pod_array.h"
+#include "vec/common/sip_hash.h"
+#include "vec/core/field.h"
+
+namespace doris::vectorized {
+class ColumnJson final : public COWHelper<IColumn, ColumnJson> {
+public:
+    using Char = UInt8;
+    using Chars = PaddedPODArray<UInt8>;
+
+private:
+    friend class COWHelper<IColumn, ColumnJson>;
+
+    Offsets offsets;
+
+    Chars chars;
+
+    size_t ALWAYS_INLINE offset_at(ssize_t i) const { return offsets[i - 1]; }
+
+    size_t ALWAYS_INLINE size_at(ssize_t i) const { return offsets[i] - offsets[i - 1]; }
+
+    template <bool positive>
+    struct less;
+
+    template <bool positive>
+    struct lessWithCollation;
+
+    ColumnJson() = default;
+
+    ColumnJson(const ColumnJson& src)
+            : offsets(src.offsets.begin(), src.offsets.end()),
+              chars(src.chars.begin(), src.chars.end()) {}
+
+public:
+    const char* get_family_name() const override { return "JSON"; }
+
+    size_t size() const override { return offsets.size(); }
+
+    size_t byte_size() const override { return chars.size() + offsets.size() * sizeof(offsets[0]); }
+
+    size_t allocated_bytes() const override {
+        return chars.allocated_bytes() + offsets.allocated_bytes();
+    }
+
+    void protect() override;
+
+    MutableColumnPtr clone_resized(size_t to_size) const override;
+
+    Field operator[](size_t n) const override {
+        assert(n < size());
+        return Field(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    void get(size_t n, Field& res) const override {
+        assert(n < size());
+        res.assign_json(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    StringRef get_data_at(size_t n) const override {
+        assert(n < size());
+        return StringRef(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+/// Suppress gcc 7.3.1 warning: '*((void*)&<anonymous> +8)' may be used uninitialized in this function
+#if !__clang__
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
+#endif
+
+    void insert(const Field& x) override {
+        const JsonField& s = doris::vectorized::get<const JsonField&>(x);
+
+        const size_t old_size = chars.size();
+        const size_t size_to_append = s.get_size() + 1;
+        const size_t new_size = old_size + size_to_append;
+
+        chars.resize(new_size);
+        memcpy(chars.data() + old_size, s.get_value(), size_to_append);
+        offsets.push_back(new_size);
+    }
+
+#if !__clang__
+#pragma GCC diagnostic pop
+#endif
+
+    void insert_from(const IColumn& src_, size_t n) override {
+        const ColumnJson& src = assert_cast<const ColumnJson&>(src_);
+        const size_t size_to_append =
+                src.offsets[n] - src.offsets[n - 1]; /// -1th index is Ok, see PaddedPODArray.
+
+        if (size_to_append == 1) {
+            /// shortcut for empty json
+            chars.push_back(0);
+            offsets.push_back(chars.size());
+        } else {
+            const size_t old_size = chars.size();
+            const size_t offset = src.offsets[n - 1];
+            const size_t new_size = old_size + size_to_append;
+
+            chars.resize(new_size);
+            memcpy_small_allow_read_write_overflow15(chars.data() + old_size, &src.chars[offset],
+                                                     size_to_append);
+            offsets.push_back(new_size);
+        }
+    }
+
+    void insert_data(const char* pos, size_t length) override {
+        const size_t old_size = chars.size();
+        const size_t new_size = old_size + length + 1;
+
+        chars.resize(new_size);
+        if (length) memcpy(chars.data() + old_size, pos, length);
+        chars[old_size + length] = 0;
+        offsets.push_back(new_size);
+    }
+
+    void insert_many_binary_data(char* data_array, uint32_t* len_array,
+                                 uint32_t* start_offset_array, size_t num) override {
+        for (size_t i = 0; i < num; i++) {
+            uint32_t len = len_array[i];
+            uint32_t start_offset = start_offset_array[i];
+            insert_data(data_array + start_offset, len);
+        }
+    };
+
+    void insert_many_dict_data(const int32_t* data_array, size_t start_index, const StringRef* dict,
+                               size_t num, uint32_t /*dict_num*/) {
+        for (size_t end_index = start_index + num; start_index < end_index; ++start_index) {
+            int32_t codeword = data_array[start_index];
+            insert_data(dict[codeword].data, dict[codeword].size);
+        }
+    }
+
+    void pop_back(size_t n) override {
+        size_t nested_n = offsets.back() - offset_at(offsets.size() - n);
+        chars.resize(chars.size() - nested_n);
+        offsets.resize_assume_reserved(offsets.size() - n);
+    }
+
+    StringRef serialize_value_into_arena(size_t n, Arena& arena, char const*& begin) const override;
+
+    const char* deserialize_and_insert_from_arena(const char* pos) override;
+
+    void update_hash_with_value(size_t n, SipHash& hash) const override {
+        size_t string_size = size_at(n);
+        size_t offset = offset_at(n);
+
+        hash.update(reinterpret_cast<const char*>(&string_size), sizeof(string_size));
+        hash.update(reinterpret_cast<const char*>(&chars[offset]), string_size);
+    }
+
+    void insert_range_from(const IColumn& src, size_t start, size_t length) override;
+
+    void insert_indices_from(const IColumn& src, const int* indices_begin,
+                             const int* indices_end) override;
+
+    ColumnPtr filter(const Filter& filt, ssize_t result_size_hint) const override;
+
+    ColumnPtr permute(const Permutation& perm, size_t limit) const override;
+
+    //    ColumnPtr index(const IColumn & indexes, size_t limit) const override;
+
+    template <typename Type>
+    ColumnPtr index_impl(const PaddedPODArray<Type>& indexes, size_t limit) const;
+
+    void insert_default() override {
+        chars.push_back(0);
+        offsets.push_back(offsets.back() + 1);
+    }
+
+    void insert_many_defaults(size_t length) override {
+        size_t chars_old_size = chars.size();
+        chars.resize(chars_old_size + length);
+        memset(chars.data() + chars_old_size, 0, length);
+
+        const size_t old_size = offsets.size();
+        const size_t new_size = old_size + length;
+        const auto num = offsets.back() + 1;
+        offsets.resize_fill(new_size, num);
+        for (size_t i = old_size, j = 0; i < new_size; i++, j++) {
+            offsets[i] += j;
+        }
+    }
+
+    int compare_at(size_t n, size_t m, const IColumn& rhs_,
+                   int /*nan_direction_hint*/) const override {
+        const ColumnJson& rhs = assert_cast<const ColumnJson&>(rhs_);
+        return memcmp_small_allow_overflow15(chars.data() + offset_at(n), size_at(n) - 1,

Review Comment:
   is it meaningful for json compare?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928251583


##########
be/src/vec/data_types/data_type_json.h:
##########
@@ -0,0 +1,62 @@
+// 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.
+
+#pragma once
+
+#include <ostream>
+
+#include "vec/columns/column_json.h"
+#include "vec/data_types/data_type.h"
+
+namespace doris::vectorized {
+class DataTypeJson final : public IDataType {
+public:
+    using ColumnType = ColumnJson;
+    using FieldType = JsonField;
+    static constexpr bool is_parametric = false;
+
+    const char* get_family_name() const override { return "JSON"; }
+    TypeIndex get_type_id() const override { return TypeIndex::JSON; }
+
+    int64_t get_uncompressed_serialized_bytes(const IColumn& column) const override;
+    char* serialize(const IColumn& column, char* buf) const override;
+    const char* deserialize(const char* buf, IColumn* column) const override;
+
+    MutableColumnPtr create_column() const override;
+
+    virtual Field get_default() const override {

Review Comment:
   return a empty json {} for default



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r918439471


##########
be/src/common/config.h:
##########
@@ -744,6 +744,11 @@ CONF_mInt32(string_type_length_soft_limit_bytes, "1048576");
 CONF_Validator(string_type_length_soft_limit_bytes,
                [](const int config) -> bool { return config > 0 && config <= 2147483643; });
 
+CONF_mInt32(json_type_length_soft_limit_bytes, "1048576");

Review Comment:
   how is soft limit used?



##########
be/src/common/env_config.h.in:
##########
@@ -19,6 +19,6 @@
 
 namespace doris {
 
-#cmakedefine HAVE_SCHED_GETCPU @HAVE_SCHED_GETCPU@
+#cmakedefine HAVE_SCHED_GETCPU @HAVE_SCHED_GETCPU @

Review Comment:
   a mistake?



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/PrimitiveType.java:
##########
@@ -648,22 +679,29 @@ public static boolean isImplicitCast(PrimitiveType type, PrimitiveType target) {
 
         compatibilityMatrix[CHAR.ordinal()][CHAR.ordinal()] = CHAR;
         compatibilityMatrix[CHAR.ordinal()][VARCHAR.ordinal()] = VARCHAR;
+        compatibilityMatrix[CHAR.ordinal()][JSON.ordinal()] = JSON;  
         compatibilityMatrix[CHAR.ordinal()][STRING.ordinal()] = STRING;
         compatibilityMatrix[CHAR.ordinal()][DECIMALV2.ordinal()] = INVALID_TYPE;
         compatibilityMatrix[CHAR.ordinal()][TIME.ordinal()] = INVALID_TYPE;
         compatibilityMatrix[CHAR.ordinal()][TIMEV2.ordinal()] = INVALID_TYPE;
 
         compatibilityMatrix[VARCHAR.ordinal()][VARCHAR.ordinal()] = VARCHAR;
         compatibilityMatrix[VARCHAR.ordinal()][STRING.ordinal()] = STRING;
+        compatibilityMatrix[VARCHAR.ordinal()][JSON.ordinal()] = JSON;  
         compatibilityMatrix[VARCHAR.ordinal()][DECIMALV2.ordinal()] = INVALID_TYPE;
         compatibilityMatrix[VARCHAR.ordinal()][TIME.ordinal()] = INVALID_TYPE;
         compatibilityMatrix[VARCHAR.ordinal()][TIMEV2.ordinal()] = INVALID_TYPE;
 
         compatibilityMatrix[STRING.ordinal()][STRING.ordinal()] = STRING;
         compatibilityMatrix[STRING.ordinal()][DECIMALV2.ordinal()] = INVALID_TYPE;
+        compatibilityMatrix[STRING.ordinal()][JSON.ordinal()] = STRING; 
         compatibilityMatrix[STRING.ordinal()][TIME.ordinal()] = INVALID_TYPE;
         compatibilityMatrix[STRING.ordinal()][TIMEV2.ordinal()] = INVALID_TYPE;
 
+        compatibilityMatrix[JSON.ordinal()][JSON.ordinal()] = JSON;
+        compatibilityMatrix[JSON.ordinal()][STRING.ordinal()] = STRING;
+        compatibilityMatrix[JSON.ordinal()][VARCHAR.ordinal()] = VARCHAR;

Review Comment:
   missed CHAR



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/PrimitiveType.java:
##########
@@ -648,22 +679,29 @@ public static boolean isImplicitCast(PrimitiveType type, PrimitiveType target) {
 
         compatibilityMatrix[CHAR.ordinal()][CHAR.ordinal()] = CHAR;
         compatibilityMatrix[CHAR.ordinal()][VARCHAR.ordinal()] = VARCHAR;
+        compatibilityMatrix[CHAR.ordinal()][JSON.ordinal()] = JSON;  

Review Comment:
   According to the definition of compatibilityMatrix: 'A value of any of the two types could be assigned to a slot of the assignment-compatible type without loss of precision', the slot value of CHAR and JSON should be CHAR.



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java:
##########
@@ -666,6 +666,9 @@ public String getSignatureString(Map<PrimitiveType, String> typeStringMap) {
             case VARCHAR:
                 sb.append(String.format(typeStringMap.get(dataType), getStrLen()));
                 break;
+            case JSON:
+                sb.append(String.format(typeStringMap.get(dataType), getStrLen()));

Review Comment:
   format is for VARCHAR(length) but not necessary for JSON



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java:
##########
@@ -1270,6 +1300,18 @@ public static Type getCmpType(Type t1, Type t2) {
                 || (t1ResultType == PrimitiveType.VARCHAR && t2ResultType == PrimitiveType.STRING)) {
             return Type.STRING;
         }
+        // TODO(wzy): support NUMERIC/CHAR cast to JSON
+        if (t1ResultType == PrimitiveType.JSON && t2ResultType == PrimitiveType.JSON) {
+            return Type.JSON;
+        }
+        if ((t1ResultType == PrimitiveType.JSON && t2ResultType == PrimitiveType.VARCHAR) 
+                || (t1ResultType == PrimitiveType.VARCHAR && t2ResultType == PrimitiveType.JSON)){
+            return Type.VARCHAR;
+        }
+        if ((t1ResultType == PrimitiveType.JSON && t2ResultType == PrimitiveType.STRING) 
+                || (t1ResultType == PrimitiveType.STRING && t2ResultType == PrimitiveType.JSON)){
+            return Type.SMALLINT;

Review Comment:
   should be STRING?



##########
be/src/vec/common/string_ref.h:
##########
@@ -73,6 +75,18 @@ struct StringRef {
     static StringRef from_string_val(StringVal sv) {
         return StringRef(reinterpret_cast<char*>(sv.ptr), sv.len);
     }
+
+    doris::JsonbDocument& to_jsonb_doc() const {

Review Comment:
   json related util function should be in a json util class instead of StringRef.



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/PrimitiveType.java:
##########
@@ -256,6 +266,7 @@ public enum PrimitiveType {
         builder.put(DATETIME, DATETIMEV2);
         builder.put(DATETIME, DECIMALV2);
         builder.put(DATETIME, VARCHAR);
+        builder.put(DATETIME, JSON);

Review Comment:
   DATEV2, DATETIMEV2 should also be handled



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java:
##########
@@ -75,7 +75,7 @@ public abstract class Type {
     public static final ScalarType DEFAULT_DATETIMEV2 = ScalarType.createDatetimeV2Type(0);
     public static final ScalarType DATETIMEV2 = DEFAULT_DATETIMEV2;
     public static final ScalarType DEFAULT_TIMEV2 = ScalarType.createTimeV2Type(0);
-    public static final ScalarType DECIMALV2 = DEFAULT_DECIMALV2;

Review Comment:
   shoule not delete DECIMALV2



##########
gensrc/thrift/Types.thrift:
##########
@@ -74,6 +74,7 @@ enum TPrimitiveType {
   CHAR,
   LARGEINT,
   VARCHAR,
+  JSON,

Review Comment:
   it's better to add new type to tail of enum.



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java:
##########
@@ -1124,6 +1139,7 @@ public Integer getNumPrecRadix() {
         compatibilityMatrix[CHAR.ordinal()][TIMEV2.ordinal()] = PrimitiveType.INVALID_TYPE;
         compatibilityMatrix[CHAR.ordinal()][BITMAP.ordinal()] = PrimitiveType.INVALID_TYPE;
         compatibilityMatrix[CHAR.ordinal()][STRING.ordinal()] = PrimitiveType.STRING;
+        compatibilityMatrix[CHAR.ordinal()][JSON.ordinal()] = PrimitiveType.INVALID_TYPE;

Review Comment:
   it's not consistent with TPrimitiveType



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/PrimitiveType.java:
##########
@@ -648,22 +679,29 @@ public static boolean isImplicitCast(PrimitiveType type, PrimitiveType target) {
 
         compatibilityMatrix[CHAR.ordinal()][CHAR.ordinal()] = CHAR;
         compatibilityMatrix[CHAR.ordinal()][VARCHAR.ordinal()] = VARCHAR;
+        compatibilityMatrix[CHAR.ordinal()][JSON.ordinal()] = JSON;  
         compatibilityMatrix[CHAR.ordinal()][STRING.ordinal()] = STRING;
         compatibilityMatrix[CHAR.ordinal()][DECIMALV2.ordinal()] = INVALID_TYPE;
         compatibilityMatrix[CHAR.ordinal()][TIME.ordinal()] = INVALID_TYPE;
         compatibilityMatrix[CHAR.ordinal()][TIMEV2.ordinal()] = INVALID_TYPE;
 
         compatibilityMatrix[VARCHAR.ordinal()][VARCHAR.ordinal()] = VARCHAR;
         compatibilityMatrix[VARCHAR.ordinal()][STRING.ordinal()] = STRING;
+        compatibilityMatrix[VARCHAR.ordinal()][JSON.ordinal()] = JSON;  

Review Comment:
   just as CHAR



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/PrimitiveType.java:
##########
@@ -496,6 +517,7 @@ public static boolean isImplicitCast(PrimitiveType type, PrimitiveType target) {
         compatibilityMatrix[BOOLEAN.ordinal()][DATETIMEV2.ordinal()] = INVALID_TYPE;
         compatibilityMatrix[BOOLEAN.ordinal()][CHAR.ordinal()] = INVALID_TYPE;
         compatibilityMatrix[BOOLEAN.ordinal()][VARCHAR.ordinal()] = INVALID_TYPE;
+        compatibilityMatrix[BOOLEAN.ordinal()][JSON.ordinal()] = INVALID_TYPE;  

Review Comment:
   it's not consistent with implicitCastMap



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java:
##########
@@ -1098,6 +1112,7 @@ public Integer getNumPrecRadix() {
         compatibilityMatrix[DATETIME.ordinal()][TIMEV2.ordinal()] = PrimitiveType.INVALID_TYPE;
         compatibilityMatrix[DATETIME.ordinal()][BITMAP.ordinal()] = PrimitiveType.INVALID_TYPE;
         compatibilityMatrix[DATETIME.ordinal()][STRING.ordinal()] = PrimitiveType.INVALID_TYPE;
+        compatibilityMatrix[DATETIME.ordinal()][JSON.ordinal()] = PrimitiveType.INVALID_TYPE;

Review Comment:
   DATEV2, DATETIMEV2 is missed



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarFunction.java:
##########
@@ -164,6 +164,8 @@ public static ScalarFunction createBuiltinOperator(
                 case QUANTILE_STATE:
                     beFn += "_string_val";
                     break;
+                case JSON:
+                    beFn += "json_val";

Review Comment:
   non-vectorized function can be ignored



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/JsonLiteral.java:
##########
@@ -0,0 +1,197 @@
+// 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.analysis;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Objects;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.JsonParser;
+import com.google.gson.JsonSyntaxException;
+
+import org.apache.doris.common.io.Text;
+import org.apache.doris.thrift.TExprNode;
+import org.apache.doris.thrift.TExprNodeType;
+import org.apache.doris.thrift.TJsonLiteral;
+import org.apache.logging.log4j.LogManager;
+import org.apache.doris.catalog.Type;
+import org.apache.logging.log4j.Logger;
+
+import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.qe.VariableVarConverters;
+
+public class JsonLiteral extends LiteralExpr {
+    private static final Logger LOG = LogManager.getLogger(JsonLiteral.class);
+    private JsonParser parser = new JsonParser();
+    private String value;
+    // Means the converted session variable need to be cast to int, such as "cast 'STRICT_TRANS_TABLES' to Integer".
+    private String beConverted = "";
+
+    public JsonLiteral() {
+        super();
+        type = Type.JSON;
+    }
+
+    public JsonLiteral(String value) throws AnalysisException {
+        try {
+            parser.parse(value);

Review Comment:
   poor performance to parse json in fe



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarFunction.java:
##########
@@ -164,6 +164,8 @@ public static ScalarFunction createBuiltinOperator(
                 case QUANTILE_STATE:
                     beFn += "_string_val";
                     break;
+                case JSON:
+                    beFn += "json_val";

Review Comment:
   _json_val



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java:
##########
@@ -1145,7 +1162,14 @@ public Integer getNumPrecRadix() {
         compatibilityMatrix[STRING.ordinal()][DATETIMEV2.ordinal()] = PrimitiveType.INVALID_TYPE;
         compatibilityMatrix[STRING.ordinal()][BITMAP.ordinal()] = PrimitiveType.INVALID_TYPE;
         compatibilityMatrix[STRING.ordinal()][QUANTILE_STATE.ordinal()] = PrimitiveType.INVALID_TYPE;
+        compatibilityMatrix[STRING.ordinal()][JSON.ordinal()] = PrimitiveType.STRING;
 
+        //JSON
+        compatibilityMatrix[JSON.ordinal()][HLL.ordinal()] = PrimitiveType.INVALID_TYPE;
+        compatibilityMatrix[JSON.ordinal()][TIME.ordinal()] = PrimitiveType.INVALID_TYPE;
+        compatibilityMatrix[JSON.ordinal()][BITMAP.ordinal()] = PrimitiveType.INVALID_TYPE;
+        compatibilityMatrix[JSON.ordinal()][QUANTILE_STATE.ordinal()] = PrimitiveType.INVALID_TYPE;
+        compatibilityMatrix[JSON.ordinal()][STRING.ordinal()] = PrimitiveType.STRING;

Review Comment:
   what about CHAR VARCHAR and many other types?



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/ColumnType.java:
##########
@@ -94,6 +94,7 @@ public abstract class ColumnType {
         schemaChangeMatrix[PrimitiveType.VARCHAR.ordinal()][PrimitiveType.DOUBLE.ordinal()] = true;
         schemaChangeMatrix[PrimitiveType.VARCHAR.ordinal()][PrimitiveType.DATE.ordinal()] = true;
         schemaChangeMatrix[PrimitiveType.VARCHAR.ordinal()][PrimitiveType.STRING.ordinal()] = true;
+        schemaChangeMatrix[PrimitiveType.VARCHAR.ordinal()][PrimitiveType.JSON.ordinal()] = true;

Review Comment:
   if VARCHAR is allowed, STRING should also be allowed



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/JsonLiteral.java:
##########
@@ -0,0 +1,197 @@
+// 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.analysis;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Objects;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.JsonParser;
+import com.google.gson.JsonSyntaxException;
+
+import org.apache.doris.common.io.Text;
+import org.apache.doris.thrift.TExprNode;
+import org.apache.doris.thrift.TExprNodeType;
+import org.apache.doris.thrift.TJsonLiteral;
+import org.apache.logging.log4j.LogManager;
+import org.apache.doris.catalog.Type;
+import org.apache.logging.log4j.Logger;
+
+import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.qe.VariableVarConverters;
+
+public class JsonLiteral extends LiteralExpr {
+    private static final Logger LOG = LogManager.getLogger(JsonLiteral.class);
+    private JsonParser parser = new JsonParser();
+    private String value;
+    // Means the converted session variable need to be cast to int, such as "cast 'STRICT_TRANS_TABLES' to Integer".
+    private String beConverted = "";
+
+    public JsonLiteral() {
+        super();
+        type = Type.JSON;
+    }
+
+    public JsonLiteral(String value) throws AnalysisException {
+        try {
+            parser.parse(value);
+        } catch (JsonSyntaxException e) {
+            throw new AnalysisException("Invalid json literal: " + e.getMessage());
+        }
+        this.value = value;
+        type = Type.JSON;
+        analysisDone();
+    }
+
+    protected JsonLiteral(JsonLiteral other) {
+        super(other);
+        value = other.value;
+    }
+
+    public void setBeConverted(String val) {
+        this.beConverted = val;
+    }
+
+    @Override
+    public Expr clone() {
+        return new JsonLiteral(this);
+    }
+
+    @Override
+    public int compareLiteral(LiteralExpr expr) {
+        if (expr instanceof NullLiteral) {
+            return 1;
+        }
+        if (expr == MaxLiteral.MAX_VALUE) {
+            return -1;
+        }
+        // compare string with utf-8 byte array, same with DM,BE,StorageEngine
+        byte[] thisBytes = null;
+        byte[] otherBytes = null;
+        try {
+            thisBytes = value.getBytes("UTF-8");
+            otherBytes = expr.getStringValue().getBytes("UTF-8");
+        } catch (UnsupportedEncodingException e) {
+            Preconditions.checkState(false);
+        }
+
+        int minLength = Math.min(thisBytes.length, otherBytes.length);
+        int i = 0;
+        for (i = 0; i < minLength; i++) {
+            if (thisBytes[i] < otherBytes[i]) {
+                return -1;
+            } else if (thisBytes[i] > otherBytes[i]) {
+                return 1;
+            }
+        }
+        if (thisBytes.length > otherBytes.length) {
+            if (thisBytes[i] == 0x00) {
+                return 0;
+            } else {
+                return 1;
+            }
+        } else if (thisBytes.length < otherBytes.length) {
+            if (otherBytes[i] == 0x00) {
+                return 0;
+            } else {
+                return -1;
+            }
+        } else {
+            return 0;
+        }
+    }
+
+    public String getValue() {
+        return value;
+    }
+
+    @Override
+    public boolean isMinValue() {
+        return false;
+    }
+
+    @Override
+    public String toSqlImpl() {
+        return "'" + value.replaceAll("'", "''") + "'";
+    }
+
+    @Override
+    protected void toThrift(TExprNode msg) {
+        msg.node_type = TExprNodeType.JSON_LITERAL;
+        msg.json_literal = new TJsonLiteral(getUnescapedValue());
+    }
+
+    public String getUnescapedValue() {
+        // Unescape string exactly like Hive does. Hive's method assumes
+        // quotes so we add them here to reuse Hive's code.
+        return value;
+    }
+
+    public String getJsonValue() {
+        return value;
+    }
+
+    @Override
+    public long getLongValue() {
+        return 0;

Review Comment:
   should throw AnalysisException if not supported at present



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r923961127


##########
be/src/runtime/json_value.h:
##########
@@ -0,0 +1,135 @@
+// 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.
+
+#ifndef DORIS_BE_RUNTIME_JSON_VALUE_H
+#define DORIS_BE_RUNTIME_JSON_VALUE_H
+
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/jsonb_error.h"
+#include "util/jsonb_parser.h"
+#include "util/jsonb_utils.h"
+#include "vec/common/string_ref.h"
+
+#ifdef __SSE4_2__
+#include "util/sse_util.hpp"
+#endif
+
+namespace doris {
+
+struct JsonValue {
+    static const int MAX_LENGTH = (1 << 30);
+
+    const char* ptr;
+    size_t len;
+    JsonbParser parser;
+
+    JsonValue() : ptr(nullptr), len(0) {}
+
+    JsonValue(char* ptr, int len) {
+        DCHECK_LE(len, MAX_LENGTH);
+        DCHECK(parser.parse(const_cast<const char*>(ptr), len));
+        this->ptr = parser.getWriter().getOutput()->getBuffer();
+        this->len = (unsigned)parser.getWriter().getOutput()->getSize();
+    }
+
+    JsonValue(const char* ptr, int len) {

Review Comment:
   we should extract it to a common method accept const char*, and call it in char* and string version.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r923960702


##########
be/src/runtime/json_value.cpp:
##########
@@ -0,0 +1,54 @@
+// 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/json_value.h"
+
+#include <cstring>
+
+#include "util/jsonb_error.h"
+
+namespace doris {
+
+JsonbErrType JsonValue::from_json_str(std::string s) {
+    JsonbErrType error = JsonbErrType::E_NONE;
+    if (!parser.parse(s)) {
+        error = parser.getErrorCode();
+        // TODO(wzy): document must be an object or an array,
+        // rune, pure-string, numeirc are valid JSON but get parse error here
+        // should return error gracefully to client
+        LOG(FATAL) << "invalid json value: " << JsonbErrMsg::getErrMsg(error);

Review Comment:
   an invalid JSON will cause be coredump, it's to ragid.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r923967565


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/JsonLiteral.java:
##########
@@ -0,0 +1,197 @@
+// 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.analysis;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Objects;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.JsonParser;
+import com.google.gson.JsonSyntaxException;
+
+import org.apache.doris.common.io.Text;
+import org.apache.doris.thrift.TExprNode;
+import org.apache.doris.thrift.TExprNodeType;
+import org.apache.doris.thrift.TJsonLiteral;
+import org.apache.logging.log4j.LogManager;
+import org.apache.doris.catalog.Type;
+import org.apache.logging.log4j.Logger;
+
+import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.qe.VariableVarConverters;
+
+public class JsonLiteral extends LiteralExpr {
+    private static final Logger LOG = LogManager.getLogger(JsonLiteral.class);
+    private JsonParser parser = new JsonParser();
+    private String value;
+    // Means the converted session variable need to be cast to int, such as "cast 'STRICT_TRANS_TABLES' to Integer".
+    private String beConverted = "";
+
+    public JsonLiteral() {
+        super();
+        type = Type.JSON;
+    }
+
+    public JsonLiteral(String value) throws AnalysisException {
+        try {
+            parser.parse(value);
+        } catch (JsonSyntaxException e) {
+            throw new AnalysisException("Invalid json literal: " + e.getMessage());
+        }
+        this.value = value;
+        type = Type.JSON;
+        analysisDone();
+    }
+
+    protected JsonLiteral(JsonLiteral other) {
+        super(other);
+        value = other.value;
+    }
+
+    public void setBeConverted(String val) {
+        this.beConverted = val;
+    }
+
+    @Override
+    public Expr clone() {
+        return new JsonLiteral(this);
+    }
+
+    @Override
+    public int compareLiteral(LiteralExpr expr) {
+        if (expr instanceof NullLiteral) {
+            return 1;
+        }
+        if (expr == MaxLiteral.MAX_VALUE) {
+            return -1;
+        }
+        // compare string with utf-8 byte array, same with DM,BE,StorageEngine
+        byte[] thisBytes = null;
+        byte[] otherBytes = null;
+        try {
+            thisBytes = value.getBytes("UTF-8");
+            otherBytes = expr.getStringValue().getBytes("UTF-8");
+        } catch (UnsupportedEncodingException e) {
+            Preconditions.checkState(false);
+        }
+
+        int minLength = Math.min(thisBytes.length, otherBytes.length);
+        int i = 0;
+        for (i = 0; i < minLength; i++) {
+            if (thisBytes[i] < otherBytes[i]) {
+                return -1;
+            } else if (thisBytes[i] > otherBytes[i]) {
+                return 1;
+            }
+        }
+        if (thisBytes.length > otherBytes.length) {
+            if (thisBytes[i] == 0x00) {
+                return 0;
+            } else {
+                return 1;
+            }
+        } else if (thisBytes.length < otherBytes.length) {
+            if (otherBytes[i] == 0x00) {
+                return 0;
+            } else {
+                return -1;
+            }
+        } else {
+            return 0;
+        }
+    }
+
+    public String getValue() {
+        return value;
+    }
+
+    @Override
+    public boolean isMinValue() {
+        return false;
+    }
+
+    @Override
+    public String toSqlImpl() {
+        return "'" + value.replaceAll("'", "''") + "'";
+    }
+
+    @Override
+    protected void toThrift(TExprNode msg) {
+        msg.node_type = TExprNodeType.JSON_LITERAL;
+        msg.json_literal = new TJsonLiteral(getUnescapedValue());
+    }
+
+    public String getUnescapedValue() {
+        // Unescape string exactly like Hive does. Hive's method assumes
+        // quotes so we add them here to reuse Hive's code.
+        return value;
+    }
+
+    public String getJsonValue() {
+        return value;
+    }
+
+    @Override
+    public long getLongValue() {
+        return 0;

Review Comment:
   always return 0 is a hidden bug that is not easy to find, just throw a RuntimeException



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r923956619


##########
fe/fe-core/src/main/java/org/apache/doris/catalog/ColumnType.java:
##########
@@ -94,6 +94,9 @@ public abstract class ColumnType {
         schemaChangeMatrix[PrimitiveType.VARCHAR.ordinal()][PrimitiveType.DOUBLE.ordinal()] = true;
         schemaChangeMatrix[PrimitiveType.VARCHAR.ordinal()][PrimitiveType.DATE.ordinal()] = true;
         schemaChangeMatrix[PrimitiveType.VARCHAR.ordinal()][PrimitiveType.STRING.ordinal()] = true;
+        schemaChangeMatrix[PrimitiveType.VARCHAR.ordinal()][PrimitiveType.JSON.ordinal()] = true;
+
+        schemaChangeMatrix[PrimitiveType.STRING.ordinal()][PrimitiveType.JSON.ordinal()] = true;

Review Comment:
   is JSON to STRING/VARCHAR allowed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928253412


##########
be/src/vec/columns/column_json.h:
##########
@@ -0,0 +1,299 @@
+// 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.
+
+#pragma once
+
+#include <cassert>
+#include <cstring>
+
+#include "vec/columns/column.h"
+#include "vec/columns/column_impl.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/memcmp_small.h"
+#include "vec/common/memcpy_small.h"
+#include "vec/common/pod_array.h"
+#include "vec/common/sip_hash.h"
+#include "vec/core/field.h"
+
+namespace doris::vectorized {
+class ColumnJson final : public COWHelper<IColumn, ColumnJson> {
+public:
+    using Char = UInt8;
+    using Chars = PaddedPODArray<UInt8>;
+
+private:
+    friend class COWHelper<IColumn, ColumnJson>;
+
+    Offsets offsets;
+
+    Chars chars;
+
+    size_t ALWAYS_INLINE offset_at(ssize_t i) const { return offsets[i - 1]; }
+
+    size_t ALWAYS_INLINE size_at(ssize_t i) const { return offsets[i] - offsets[i - 1]; }
+
+    template <bool positive>
+    struct less;
+
+    template <bool positive>
+    struct lessWithCollation;
+
+    ColumnJson() = default;
+
+    ColumnJson(const ColumnJson& src)
+            : offsets(src.offsets.begin(), src.offsets.end()),
+              chars(src.chars.begin(), src.chars.end()) {}
+
+public:
+    const char* get_family_name() const override { return "JSON"; }
+
+    size_t size() const override { return offsets.size(); }
+
+    size_t byte_size() const override { return chars.size() + offsets.size() * sizeof(offsets[0]); }
+
+    size_t allocated_bytes() const override {
+        return chars.allocated_bytes() + offsets.allocated_bytes();
+    }
+
+    void protect() override;
+
+    MutableColumnPtr clone_resized(size_t to_size) const override;
+
+    Field operator[](size_t n) const override {
+        assert(n < size());
+        return Field(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    void get(size_t n, Field& res) const override {
+        assert(n < size());
+        res.assign_json(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+    StringRef get_data_at(size_t n) const override {
+        assert(n < size());
+        return StringRef(&chars[offset_at(n)], size_at(n) - 1);
+    }
+
+/// Suppress gcc 7.3.1 warning: '*((void*)&<anonymous> +8)' may be used uninitialized in this function
+#if !__clang__
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
+#endif
+
+    void insert(const Field& x) override {
+        const JsonField& s = doris::vectorized::get<const JsonField&>(x);
+
+        const size_t old_size = chars.size();
+        const size_t size_to_append = s.get_size() + 1;
+        const size_t new_size = old_size + size_to_append;
+
+        chars.resize(new_size);
+        memcpy(chars.data() + old_size, s.get_value(), size_to_append);
+        offsets.push_back(new_size);
+    }
+
+#if !__clang__
+#pragma GCC diagnostic pop
+#endif
+
+    void insert_from(const IColumn& src_, size_t n) override {
+        const ColumnJson& src = assert_cast<const ColumnJson&>(src_);
+        const size_t size_to_append =
+                src.offsets[n] - src.offsets[n - 1]; /// -1th index is Ok, see PaddedPODArray.
+
+        if (size_to_append == 1) {
+            /// shortcut for empty json
+            chars.push_back(0);
+            offsets.push_back(chars.size());
+        } else {
+            const size_t old_size = chars.size();
+            const size_t offset = src.offsets[n - 1];
+            const size_t new_size = old_size + size_to_append;
+
+            chars.resize(new_size);
+            memcpy_small_allow_read_write_overflow15(chars.data() + old_size, &src.chars[offset],
+                                                     size_to_append);
+            offsets.push_back(new_size);
+        }
+    }
+
+    void insert_data(const char* pos, size_t length) override {
+        const size_t old_size = chars.size();
+        const size_t new_size = old_size + length + 1;
+
+        chars.resize(new_size);
+        if (length) memcpy(chars.data() + old_size, pos, length);
+        chars[old_size + length] = 0;
+        offsets.push_back(new_size);
+    }
+
+    void insert_many_binary_data(char* data_array, uint32_t* len_array,
+                                 uint32_t* start_offset_array, size_t num) override {
+        for (size_t i = 0; i < num; i++) {
+            uint32_t len = len_array[i];
+            uint32_t start_offset = start_offset_array[i];
+            insert_data(data_array + start_offset, len);
+        }
+    };
+
+    void insert_many_dict_data(const int32_t* data_array, size_t start_index, const StringRef* dict,
+                               size_t num, uint32_t /*dict_num*/) {
+        for (size_t end_index = start_index + num; start_index < end_index; ++start_index) {
+            int32_t codeword = data_array[start_index];
+            insert_data(dict[codeword].data, dict[codeword].size);
+        }
+    }
+
+    void pop_back(size_t n) override {
+        size_t nested_n = offsets.back() - offset_at(offsets.size() - n);
+        chars.resize(chars.size() - nested_n);
+        offsets.resize_assume_reserved(offsets.size() - n);
+    }
+
+    StringRef serialize_value_into_arena(size_t n, Arena& arena, char const*& begin) const override;
+
+    const char* deserialize_and_insert_from_arena(const char* pos) override;
+
+    void update_hash_with_value(size_t n, SipHash& hash) const override {
+        size_t string_size = size_at(n);
+        size_t offset = offset_at(n);
+
+        hash.update(reinterpret_cast<const char*>(&string_size), sizeof(string_size));
+        hash.update(reinterpret_cast<const char*>(&chars[offset]), string_size);
+    }
+

Review Comment:
   get_max_row_byte_size, serialize_vec, serialize_vec_with_null_map not implemented as ColumnString



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928216332


##########
be/src/vec/core/field.h:
##########
@@ -269,6 +355,11 @@ class Field {
         create(data, size);
     }
 
+    void assign_json(const unsigned char* data, size_t size) {

Review Comment:
   char* and unsinged char* is both needed as assign_string?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] xiaokang commented on a diff in pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
xiaokang commented on code in PR #10322:
URL: https://github.com/apache/doris/pull/10322#discussion_r928239348


##########
be/src/vec/common/string_ref.h:
##########
@@ -29,6 +29,8 @@
 #include "gutil/hash/city.h"
 #include "gutil/hash/hash128to64.h"
 #include "udf/udf.h"
+#include "util/jsonb_document.h"

Review Comment:
   not used include for json



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] github-actions[bot] commented on pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #10322:
URL: https://github.com/apache/doris/pull/10322#issuecomment-1257124505

   PR approved by at least one committer and no changes requested.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] github-actions[bot] commented on pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #10322:
URL: https://github.com/apache/doris/pull/10322#issuecomment-1257124517

   PR approved by anyone and no changes requested.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [doris] freesinger commented on pull request #10322: [feature](JSON datatype)Support JSON datatype

Posted by GitBox <gi...@apache.org>.
freesinger commented on PR #10322:
URL: https://github.com/apache/doris/pull/10322#issuecomment-1240574927

   Sample records are as follows:
   ![image](https://user-images.githubusercontent.com/20896930/189108039-044fcb02-f513-42e7-b2d5-c11b9d95031a.png)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org