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-unsubscr...@doris.apache.org

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


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

Reply via email to