diff --git a/be/src/vec/columns/column_variant.cpp b/be/src/vec/columns/column_variant.cpp index fff84b51802f82..f41c066c256b47 100644 --- a/be/src/vec/columns/column_variant.cpp +++ b/be/src/vec/columns/column_variant.cpp @@ -78,12 +78,6 @@ namespace doris::vectorized { namespace { DataTypePtr create_array_of_type(PrimitiveType type, size_t num_dimensions, bool is_nullable) { - if (type == ColumnVariant::MOST_COMMON_TYPE_ID) { - // JSONB type MUST NOT wrapped in ARRAY column, it should be top level. - // So we ignored num_dimensions. - return is_nullable ? make_nullable(std::make_shared()) - : std::make_shared(); - } DataTypePtr result = type == PrimitiveType::INVALID_TYPE ? is_nullable ? make_nullable(std::make_shared()) : std::dynamic_pointer_cast( @@ -897,10 +891,25 @@ void ColumnVariant::Subcolumn::get(size_t n, Field& res) const { return; } if (is_finalized()) { + // TODO(hangyu) : we should use data type to get the field value + // here is a special case for Array + if (least_common_type.get_type_id() == PrimitiveType::TYPE_ARRAY && + least_common_type.get_base_type_id() == PrimitiveType::TYPE_JSONB) { + // Array of JsonbField is special case + get_finalized_column().get(n, res); + // here we will get a Array Field or NULL, if it is Array, we need to convert it to Array + convert_array_string_to_array_jsonb(res); + return; + } + + // here is a special case for JsonbField if (least_common_type.get_base_type_id() == PrimitiveType::TYPE_JSONB) { - // JsonbFiled is special case res = Field::create_field(JsonbField()); + get_finalized_column().get(n, res); + return; } + + // common type to get the field value get_finalized_column().get(n, res); return; } @@ -2027,4 +2036,23 @@ bool ColumnVariant::try_insert_default_from_nested(const Subcolumns::NodePtr& en return true; } +void ColumnVariant::Subcolumn::convert_array_string_to_array_jsonb(Field& array_field) { + if (array_field.is_null()) { + return; + } + if (array_field.get_type() != PrimitiveType::TYPE_ARRAY) { + return; + } + Field converted_res = Field::create_field(Array()); + for (auto& item : array_field.get()) { + Field jsonb_item; + DCHECK(item.get_type() == PrimitiveType::TYPE_STRING); + auto& string_item = item.get(); + jsonb_item = Field::create_field( + JsonbField(string_item.c_str(), string_item.size())); + converted_res.get().emplace_back(std::move(jsonb_item)); + } + array_field = std::move(converted_res); +} + } // namespace doris::vectorized diff --git a/be/src/vec/columns/column_variant.h b/be/src/vec/columns/column_variant.h index f12ade0d64f853..506d5bd589ab40 100644 --- a/be/src/vec/columns/column_variant.h +++ b/be/src/vec/columns/column_variant.h @@ -96,6 +96,7 @@ class ColumnVariant final : public COWHelper { constexpr static PrimitiveType MOST_COMMON_TYPE_ID = PrimitiveType::TYPE_JSONB; // Nullable(Array(Nullable(Object))) const static DataTypePtr NESTED_TYPE; + // Finlize mode for subcolumns, write mode will estimate which subcolumns are sparse columns(too many null values inside column), // merge and encode them into a shared column in root column. Only affects in flush block to segments. // Otherwise read mode should be as default mode. @@ -178,6 +179,9 @@ class ColumnVariant final : public COWHelper { void add_new_column_part(DataTypePtr type); + /// Converts Array to Array for special case handling + static void convert_array_string_to_array_jsonb(Field& array_field); + friend class ColumnVariant; private: diff --git a/be/src/vec/data_types/convert_field_to_type.cpp b/be/src/vec/data_types/convert_field_to_type.cpp index ad253317277277..a4ec0138fbf0a1 100644 --- a/be/src/vec/data_types/convert_field_to_type.cpp +++ b/be/src/vec/data_types/convert_field_to_type.cpp @@ -34,6 +34,7 @@ #include "common/exception.h" #include "common/status.h" #include "util/bitmap_value.h" +#include "util/jsonb_document.h" #include "util/jsonb_writer.h" #include "vec/common/field_visitors.h" #include "vec/common/typeid_cast.h" @@ -90,6 +91,11 @@ class FieldVisitorToJsonb : public StaticVisitor { writer->writeString(x); writer->writeEndString(); } + void operator()(const JsonbField& x, JsonbWriter* writer) const { + JsonbDocument* doc; + THROW_IF_ERROR(JsonbDocument::checkAndCreateDocument(x.get_value(), x.get_size(), &doc)); + writer->writeValue(doc->getValue()); + } void operator()(const Array& x, JsonbWriter* writer) const; void operator()(const Tuple& x, JsonbWriter* writer) const { @@ -125,9 +131,6 @@ class FieldVisitorToJsonb : public StaticVisitor { void operator()(const Map& x, JsonbWriter* writer) const { throw doris::Exception(doris::ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); } - void operator()(const JsonbField& x, JsonbWriter* writer) const { - throw doris::Exception(doris::ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); - } }; void FieldVisitorToJsonb::operator()(const Array& x, JsonbWriter* writer) const { @@ -143,7 +146,7 @@ void FieldVisitorToJsonb::operator()(const Array& x, JsonbWriter* writer) const namespace { template Field convert_numeric_type_impl(const Field& from) { - typename PrimitiveTypeTraits::CppType result; + typename PrimitiveTypeTraits::ColumnItemType result; if (!accurate::convertNumeric(from.get(), result)) { return {}; } @@ -153,11 +156,11 @@ Field convert_numeric_type_impl(const Field& from) { template void convert_numric_type(const Field& from, const IDataType& type, Field* to) { if (from.get_type() == PrimitiveType::TYPE_BIGINT) { - *to = convert_numeric_type_impl(from); + *to = convert_numeric_type_impl(from); } else if (from.get_type() == PrimitiveType::TYPE_DOUBLE) { - *to = convert_numeric_type_impl(from); + *to = convert_numeric_type_impl(from); } else if (from.get_type() == PrimitiveType::TYPE_LARGEINT) { - *to = convert_numeric_type_impl(from); + *to = convert_numeric_type_impl(from); } else { throw doris::Exception(ErrorCode::INVALID_ARGUMENT, "Type mismatch in IN or VALUES section. Expected: {}. Got: {}", @@ -280,4 +283,5 @@ void convert_field_to_type(const Field& from_value, const IDataType& to_type, Fi return convert_field_to_typeImpl(from_value, to_type, from_type_hint, to); } } +#include "common/compile_check_end.h" } // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index 5b398ebd8e79bc..743e21d7226f0c 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -752,7 +752,7 @@ struct ConvertNothingToJsonb { } }; -template +template struct ConvertImplFromJsonb { static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments, const uint32_t result, size_t input_rows_count) { @@ -799,6 +799,18 @@ struct ConvertImplFromJsonb { res[i] = 0; continue; } + + // if value is string, convert by parse, otherwise the result is null if ToDataType is not string + if (value->isString()) { + const auto* blob = value->unpack(); + const auto& data = blob->getBlob(); + size_t len = blob->getBlobLen(); + ReadBuffer rb((char*)(data), len); + bool parsed = try_parse_impl(res[i], rb, context); + null_map[i] = !parsed; + continue; + } + if constexpr (type == PrimitiveType::TYPE_BOOLEAN) { // cast from json value to boolean type if (value->isTrue()) { @@ -1690,19 +1702,26 @@ class FunctionCast final : public IFunctionBase { bool jsonb_string_as_string) const { switch (to_type->get_primitive_type()) { case PrimitiveType::TYPE_BOOLEAN: - return &ConvertImplFromJsonb::execute; + return &ConvertImplFromJsonb::execute; case PrimitiveType::TYPE_TINYINT: - return &ConvertImplFromJsonb::execute; + return &ConvertImplFromJsonb::execute; case PrimitiveType::TYPE_SMALLINT: - return &ConvertImplFromJsonb::execute; + return &ConvertImplFromJsonb::execute; case PrimitiveType::TYPE_INT: - return &ConvertImplFromJsonb::execute; + return &ConvertImplFromJsonb::execute; case PrimitiveType::TYPE_BIGINT: - return &ConvertImplFromJsonb::execute; + return &ConvertImplFromJsonb::execute; case PrimitiveType::TYPE_LARGEINT: - return &ConvertImplFromJsonb::execute; + return &ConvertImplFromJsonb::execute; case PrimitiveType::TYPE_DOUBLE: - return &ConvertImplFromJsonb::execute; + return &ConvertImplFromJsonb::execute; case PrimitiveType::TYPE_STRING: case PrimitiveType::TYPE_CHAR: case PrimitiveType::TYPE_VARCHAR: diff --git a/be/src/vec/json/json_parser.cpp b/be/src/vec/json/json_parser.cpp index cd89a5bbd79b7a..be3f69e7fb4798 100644 --- a/be/src/vec/json/json_parser.cpp +++ b/be/src/vec/json/json_parser.cpp @@ -59,8 +59,14 @@ void JSONDataParser::traverse(const Element& element, ParseContext& if (element.isObject()) { traverseObject(element.getObject(), ctx); } else if (element.isArray()) { + if (ctx.has_nested_in_flatten) { + throw doris::Exception(doris::ErrorCode::INVALID_ARGUMENT, + "Nesting of array in Nested array within variant subcolumns is " + "currently not supported."); + } has_nested = false; check_has_nested_object(element); + ctx.has_nested_in_flatten = has_nested && ctx.enable_flatten_nested; if (has_nested && !ctx.enable_flatten_nested) { // Parse nested arrays to JsonbField JsonbWriter writer; @@ -71,6 +77,8 @@ void JSONDataParser::traverse(const Element& element, ParseContext& } else { traverseArray(element.getArray(), ctx); } + // we should set has_nested_in_flatten to false when traverse array finished for next array otherwise it will be true for next array + ctx.has_nested_in_flatten = false; } else { ctx.paths.push_back(ctx.builder.get_parts()); ctx.values.push_back(getValueAsField(element)); @@ -137,6 +145,7 @@ template void JSONDataParser::traverseArray(const JSONArray& array, ParseContext& ctx) { /// Traverse elements of array and collect an array of fields by each path. ParseArrayContext array_ctx; + array_ctx.has_nested_in_flatten = ctx.has_nested_in_flatten; array_ctx.total_size = array.size(); for (auto it = array.begin(); it != array.end(); ++it) { traverseArrayElement(*it, array_ctx); @@ -162,8 +171,9 @@ template void JSONDataParser::traverseArrayElement(const Element& element, ParseArrayContext& ctx) { ParseContext element_ctx; + element_ctx.has_nested_in_flatten = ctx.has_nested_in_flatten; traverse(element, element_ctx); - auto& [_, paths, values, flatten_nested] = element_ctx; + auto& [_, paths, values, flatten_nested, has_nested] = element_ctx; size_t size = paths.size(); size_t keys_to_update = ctx.arrays_by_path.size(); for (size_t i = 0; i < size; ++i) { diff --git a/be/src/vec/json/json_parser.h b/be/src/vec/json/json_parser.h index 845aaea178de2d..ee71994ac0879f 100644 --- a/be/src/vec/json/json_parser.h +++ b/be/src/vec/json/json_parser.h @@ -146,6 +146,7 @@ class JSONDataParser { std::vector paths; std::vector values; bool enable_flatten_nested = false; + bool has_nested_in_flatten = false; }; using PathPartsWithArray = std::pair; using PathToArray = phmap::flat_hash_map; @@ -155,6 +156,7 @@ class JSONDataParser { size_t total_size = 0; PathToArray arrays_by_path; KeyToSizes nested_sizes_by_key; + bool has_nested_in_flatten = false; }; void traverse(const Element& element, ParseContext& ctx); void traverseObject(const JSONObject& object, ParseContext& ctx); diff --git a/be/test/vec/function/cast/function_variant_cast_test.cpp b/be/test/vec/function/cast/function_variant_cast_test.cpp new file mode 100644 index 00000000000000..db0ccc8144b51e --- /dev/null +++ b/be/test/vec/function/cast/function_variant_cast_test.cpp @@ -0,0 +1,468 @@ +// 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 + +#include "common/status.h" +#include "gtest/gtest_pred_impl.h" +#include "olap/field.h" +#include "runtime/define_primitive_type.h" +#include "runtime/primitive_type.h" +#include "runtime/runtime_state.h" +#include "vec/columns/column_array.h" +#include "vec/columns/column_variant.h" +#include "vec/core/field.h" +#include "vec/data_types/data_type_array.h" +#include "vec/data_types/data_type_nullable.h" +#include "vec/data_types/data_type_number.h" +#include "vec/data_types/data_type_string.h" +#include "vec/data_types/data_type_variant.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { +static doris::vectorized::Field construct_variant_map( + const std::vector>& key_and_values) { + doris::vectorized::Field res = Field::create_field(VariantMap {}); + auto& object = res.get(); + for (const auto& [k, v] : key_and_values) { + PathInData path(k); + object.try_emplace(path, v); + } + return res; +} + +static auto construct_basic_varint_column() { + // 1. create an empty variant column + auto variant = ColumnVariant::create(5); + + std::vector> data; + + // 2. subcolumn path + data.emplace_back("v.a", Field::create_field(20)); + data.emplace_back("v.b", Field::create_field("20")); + data.emplace_back("v.c", Field::create_field(20)); + data.emplace_back("v.f", Field::create_field(20)); + data.emplace_back("v.e", Field::create_field("50")); + for (int i = 0; i < 5; ++i) { + auto field = construct_variant_map(data); + variant->try_insert(field); + } + + return variant; +} + +TEST(FunctionVariantCast, CastToVariant) { + // Test casting from basic types to variant + { + // Test Int32 to variant + auto int32_type = std::make_shared(); + auto variant_type = std::make_shared(); + auto int32_col = ColumnInt32::create(); + int32_col->insert(Field::create_field(42)); + int32_col->insert(Field::create_field(100)); + int32_col->insert(Field::create_field(-1)); + + ColumnsWithTypeAndName arguments {{int32_col->get_ptr(), int32_type, "int32_col"}, + {nullptr, variant_type, "variant_type"}}; + + auto function = + SimpleFunctionFactory::instance().get_function("CAST", arguments, variant_type); + ASSERT_NE(function, nullptr); + + Block block {arguments}; + size_t result_column = block.columns(); + block.insert({nullptr, variant_type, "result"}); + + RuntimeState state; + auto ctx = FunctionContext::create_context(&state, {}, {}); + ASSERT_TRUE(function->execute(ctx.get(), block, {0}, result_column, 3).ok()); + + auto result_col = block.get_by_position(result_column).column; + ASSERT_NE(result_col.get(), nullptr); + const auto* variant_col = assert_cast(result_col.get()); + ASSERT_EQ(variant_col->size(), 3); + } + + // Test casting from string to variant + { + auto string_type = std::make_shared(); + auto variant_type = std::make_shared(); + auto string_col = ColumnString::create(); + string_col->insert_data("hello", 5); + string_col->insert_data("world", 5); + + ColumnsWithTypeAndName arguments {{string_col->get_ptr(), string_type, "string_col"}, + {nullptr, variant_type, "variant_type"}}; + + auto function = SimpleFunctionFactory::instance().get_function("CAST", arguments, + make_nullable(variant_type)); + ASSERT_NE(function, nullptr); + + Block block {arguments}; + size_t result_column = block.columns(); + block.insert({nullptr, variant_type, "result"}); + + RuntimeState state; + auto ctx = FunctionContext::create_context(&state, {}, {}); + ASSERT_TRUE(function->execute(ctx.get(), block, {0}, result_column, 2).ok()); + + auto result_col = block.get_by_position(result_column).column; + ASSERT_NE(result_col.get(), nullptr); + const auto* variant_col = + assert_cast(remove_nullable(result_col).get()); + ASSERT_EQ(variant_col->size(), 2); + } + + // Test casting from array to variant + { + auto array_type = std::make_shared(std::make_shared()); + auto variant_type = std::make_shared(); + auto array_col = + ColumnArray::create(ColumnInt32::create(), ColumnArray::ColumnOffsets::create()); + auto& data = assert_cast(array_col->get_data()); + auto& offsets = array_col->get_offsets(); + + data.insert(Field::create_field(1)); + data.insert(Field::create_field(2)); + data.insert(Field::create_field(3)); + offsets.push_back(3); + + ColumnsWithTypeAndName arguments {{array_col->get_ptr(), array_type, "array_col"}, + {nullptr, variant_type, "variant_type"}}; + + auto function = + SimpleFunctionFactory::instance().get_function("CAST", arguments, variant_type); + ASSERT_NE(function, nullptr); + + Block block {arguments}; + size_t result_column = block.columns(); + block.insert({nullptr, variant_type, "result"}); + + RuntimeState state; + auto ctx = FunctionContext::create_context(&state, {}, {}); + ASSERT_TRUE(function->execute(ctx.get(), block, {0}, result_column, 1).ok()); + + auto result_col = block.get_by_position(result_column).column; + ASSERT_NE(result_col.get(), nullptr); + const auto* variant_col = + assert_cast(remove_nullable(result_col).get()); + ASSERT_EQ(variant_col->size(), 1); + } +} + +TEST(FunctionVariantCast, CastFromVariant) { + // Test casting from variant to basic types + { + auto variant_type = std::make_shared(); + auto int32_type = std::make_shared(); + auto variant_col = ColumnVariant::create(true); + + // Create a variant column with integer values + variant_col->create_root(int32_type, ColumnInt32::create()); + MutableColumnPtr data = variant_col->get_root(); + data->insert(Field::create_field(42)); + data->insert(Field::create_field(100)); + data->insert(Field::create_field(-1)); + + ColumnsWithTypeAndName arguments {{variant_col->get_ptr(), variant_type, "variant_col"}, + {nullptr, int32_type, "int32_type"}}; + + auto function = + SimpleFunctionFactory::instance().get_function("CAST", arguments, int32_type); + ASSERT_NE(function, nullptr); + + Block block {arguments}; + size_t result_column = block.columns(); + block.insert({nullptr, int32_type, "result"}); + + RuntimeState state; + auto ctx = FunctionContext::create_context(&state, {}, {}); + ASSERT_TRUE(function->execute(ctx.get(), block, {0}, result_column, 3).ok()); + + auto result_col = block.get_by_position(result_column).column; + ASSERT_NE(result_col.get(), nullptr); + // always nullable + const auto* int32_result = + assert_cast(remove_nullable(result_col).get()); + ASSERT_EQ(int32_result->size(), 3); + ASSERT_EQ(int32_result->get_element(0), 42); + ASSERT_EQ(int32_result->get_element(1), 100); + ASSERT_EQ(int32_result->get_element(2), -1); + } + + // Test casting from variant to string + { + auto variant_type = std::make_shared(); + auto string_type = std::make_shared(); + auto variant_col = ColumnVariant::create(true); + + // Create a variant column with string values + variant_col->create_root(string_type, ColumnString::create()); + MutableColumnPtr data = variant_col->get_root(); + data->insert_data("hello", 5); + data->insert_data("world", 5); + + ColumnsWithTypeAndName arguments {{variant_col->get_ptr(), variant_type, "variant_col"}, + {nullptr, string_type, "string_type"}}; + + auto function = + SimpleFunctionFactory::instance().get_function("CAST", arguments, string_type); + ASSERT_NE(function, nullptr); + + Block block {arguments}; + size_t result_column = block.columns(); + block.insert({nullptr, string_type, "result"}); + + RuntimeState state; + auto ctx = FunctionContext::create_context(&state, {}, {}); + ASSERT_TRUE(function->execute(ctx.get(), block, {0}, result_column, 2).ok()); + + auto result_col = block.get_by_position(result_column).column; + ASSERT_NE(result_col.get(), nullptr); + const auto* string_result = + assert_cast(remove_nullable(result_col).get()); + ASSERT_EQ(string_result->size(), 2); + ASSERT_EQ(string_result->get_data_at(0).to_string(), "hello"); + ASSERT_EQ(string_result->get_data_at(1).to_string(), "world"); + } + + // Test casting from variant to array + { + auto variant_type = std::make_shared(); + auto array_type = std::make_shared(std::make_shared()); + auto variant_col = ColumnVariant::create(true); + + // Create a variant column with array values + variant_col->create_root( + array_type, + ColumnArray::create(ColumnInt32::create(), ColumnArray::ColumnOffsets::create())); + MutableColumnPtr data = variant_col->get_root(); + + Field a = Field::create_field(Array {Field::create_field(1), + Field::create_field(2), + Field::create_field(3)}); + + data->insert(a); + + ColumnsWithTypeAndName arguments {{variant_col->get_ptr(), variant_type, "variant_col"}, + {nullptr, array_type, "array_type"}}; + + auto function = + SimpleFunctionFactory::instance().get_function("CAST", arguments, array_type); + ASSERT_NE(function, nullptr); + + Block block {arguments}; + size_t result_column = block.columns(); + block.insert({nullptr, array_type, "result"}); + + RuntimeState state; + auto ctx = FunctionContext::create_context(&state, {}, {}); + ASSERT_TRUE(function->execute(ctx.get(), block, {0}, result_column, 1).ok()); + + auto result_col = block.get_by_position(result_column).column; + ASSERT_NE(result_col.get(), nullptr); + const auto* array_result = + assert_cast(remove_nullable(result_col).get()); + ASSERT_EQ(array_result->size(), 1); + const auto& result_data = assert_cast(array_result->get_data()); + ASSERT_EQ(result_data.size(), 3); + ASSERT_EQ(result_data.get_element(0), 1); + ASSERT_EQ(result_data.get_element(1), 2); + ASSERT_EQ(result_data.get_element(2), 3); + } +} + +TEST(FunctionVariantCast, CastVariantWithNull) { + auto variant_type = std::make_shared(); + auto int32_type = std::make_shared(); + auto nullable_int32_type = std::make_shared(int32_type); + + // Create a variant column with nullable integer values + auto variant_col = ColumnVariant::create(true); + variant_col->create_root(nullable_int32_type, + ColumnNullable::create(ColumnInt32::create(), ColumnUInt8::create())); + MutableColumnPtr data = variant_col->get_root(); + + data->insert(Field::create_field(42)); + data->insert(Field::create_field(Null())); + data->insert(Field::create_field(100)); + + ColumnsWithTypeAndName arguments {{variant_col->get_ptr(), variant_type, "variant_col"}, + {nullptr, nullable_int32_type, "nullable_int32_type"}}; + + variant_col->finalize(); + auto function = + SimpleFunctionFactory::instance().get_function("CAST", arguments, nullable_int32_type); + ASSERT_NE(function, nullptr); + + Block block {arguments}; + size_t result_column = block.columns(); + block.insert({nullptr, nullable_int32_type, "result"}); + + RuntimeState state; + auto ctx = FunctionContext::create_context(&state, {}, {}); + ASSERT_TRUE(function->execute(ctx.get(), block, {0}, result_column, 3).ok()); + + auto result_col = block.get_by_position(result_column).column; + ASSERT_NE(result_col.get(), nullptr); + const auto* nullable_result = assert_cast(result_col.get()); + ASSERT_EQ(nullable_result->size(), 3); + + const auto& result_data = assert_cast(nullable_result->get_nested_column()); + const auto& result_null_map = nullable_result->get_null_map_data(); + + ASSERT_EQ(result_data.get_element(0), 42); + ASSERT_EQ(result_null_map[0], 0); + ASSERT_EQ(result_null_map[1], 1); + ASSERT_EQ(result_data.get_element(2), 100); +} + +TEST(FunctionVariantCast, CastFromVariantWithEmptyRoot) { + // Test case 1: variant.empty() branch + { + auto variant_type = std::make_shared(); + auto int32_type = std::make_shared(); + MutableColumnPtr root = ColumnInt32::create(); + root->insert(Field::create_field(42)); + vectorized::ColumnVariant::Subcolumns dynamic_subcolumns; + dynamic_subcolumns.add( + vectorized::PathInData(ColumnVariant::COLUMN_NAME_DUMMY), + vectorized::ColumnVariant::Subcolumn {root->get_ptr(), int32_type, true, true}); + auto variant_col = ColumnVariant::create(std::move(dynamic_subcolumns), true); + + variant_col->finalize(); + ColumnsWithTypeAndName arguments {{variant_col->get_ptr(), variant_type, "variant_col"}, + {nullptr, int32_type, "int32_type"}}; + + auto function = + SimpleFunctionFactory::instance().get_function("CAST", arguments, int32_type); + ASSERT_NE(function, nullptr); + + Block block {arguments}; + size_t result_column = block.columns(); + block.insert({nullptr, int32_type, "result"}); + + RuntimeState state; + auto ctx = FunctionContext::create_context(&state, {}, {}); + ASSERT_TRUE(function->execute(ctx.get(), block, {0}, result_column, 1).ok()); + + auto result_col = block.get_by_position(result_column).column; + ASSERT_NE(result_col.get(), nullptr); + // always nullable + const auto* int32_result = + assert_cast(remove_nullable(result_col).get()); + ASSERT_EQ(int32_result->size(), 1); + // because of variant.empty() we insert_default with data_type_to + ASSERT_EQ(int32_result->get_element(0), 0); + } + + // Test case 2: !data_type_to->is_nullable() && !WhichDataType(data_type_to).is_string() branch + { + // object has sparse column + auto int32_type = std::make_shared(); + auto variant_col = construct_basic_varint_column(); + auto variant_type = std::make_shared(); + + ColumnsWithTypeAndName arguments {{variant_col->get_ptr(), variant_type, "variant_col"}, + {nullptr, int32_type, "int32_type"}}; + + variant_col->finalize(); + auto function = + SimpleFunctionFactory::instance().get_function("CAST", arguments, int32_type); + ASSERT_NE(function, nullptr); + + Block block {arguments}; + size_t result_column = block.columns(); + block.insert({nullptr, int32_type, "result"}); + RuntimeState state; + auto ctx = FunctionContext::create_context(&state, {}, {}); + ASSERT_TRUE(function->execute(ctx.get(), block, {0}, result_column, 1).ok()); + + auto result_col = block.get_by_position(result_column).column; + ASSERT_NE(result_col.get(), nullptr); + const auto* nullable_result = assert_cast(result_col.get()); + ASSERT_EQ(nullable_result->size(), 1); + ASSERT_TRUE(nullable_result->is_null_at(0)); + } + + // Test case 3: WhichDataType(data_type_to).is_string() branch + { + // variant has sparse column + auto int32_type = std::make_shared(); + auto variant_col = construct_basic_varint_column(); + + auto string_type = std::make_shared(); + auto variant_type = std::make_shared(); + + ColumnsWithTypeAndName arguments {{variant_col->get_ptr(), variant_type, "variant_col"}, + {nullptr, string_type, "string_type"}}; + + variant_col->finalize(); + auto function = + SimpleFunctionFactory::instance().get_function("CAST", arguments, string_type); + ASSERT_NE(function, nullptr); + + Block block {arguments}; + size_t result_column = block.columns(); + block.insert({nullptr, string_type, "result"}); + RuntimeState state; + auto ctx = FunctionContext::create_context(&state, {}, {}); + ASSERT_TRUE(function->execute(ctx.get(), block, {0}, result_column, 1).ok()); + + auto result_col = block.get_by_position(result_column).column; + ASSERT_NE(result_col.get(), nullptr); + const auto* string_result = assert_cast(result_col.get()); + // just call ConvertImplGenericToString which will insert all source column data to ColumnString + ASSERT_EQ(string_result->size(), variant_col->size()); + ASSERT_EQ(string_result->get_data_at(0).to_string(), + "{\"v\":{\"a\":20,\"b\":\"20\",\"c\":20,\"e\":\"50\",\"f\":20}}"); + } + + // Test case 4: else branch (nullable type) + { + auto variant_col = construct_basic_varint_column(); + variant_col->finalize(); + auto nullable_variant_col = make_nullable(variant_col->get_ptr()); + + auto nullable_string_type = make_nullable(std::make_shared()); + auto variant_type = std::make_shared(); + auto nullable_variant_type = make_nullable(variant_type); + + ColumnsWithTypeAndName arguments { + {nullable_variant_col->get_ptr(), nullable_variant_type, "variant_col"}, + {nullptr, nullable_string_type, "nullable_string_type"}}; + + auto function = SimpleFunctionFactory::instance().get_function("CAST", arguments, + nullable_string_type); + ASSERT_NE(function, nullptr); + + Block block {arguments}; + size_t result_column = block.columns(); + block.insert({nullptr, nullable_string_type, "result"}); + RuntimeState state; + auto ctx = FunctionContext::create_context(&state, {}, {}); + ASSERT_TRUE(function->execute(ctx.get(), block, {0}, result_column, 1).ok()); + + auto result_col = block.get_by_position(result_column).column; + ASSERT_NE(result_col.get(), nullptr); + const auto* nullable_result = assert_cast(result_col.get()); + ASSERT_EQ(nullable_result->size(), 1); + ASSERT_TRUE(nullable_result->is_null_at(1)); + } +} + +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/test/vec/jsonb/convert_field_to_type_test.cpp b/be/test/vec/jsonb/convert_field_to_type_test.cpp new file mode 100644 index 00000000000000..a848f7165ab99d --- /dev/null +++ b/be/test/vec/jsonb/convert_field_to_type_test.cpp @@ -0,0 +1,522 @@ +// 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/data_types/convert_field_to_type.cpp" + +#include + +#include +#include + +#include "runtime/jsonb_value.h" +#include "util/jsonb_document.h" +#include "util/jsonb_writer.h" +#include "vec/core/field.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_array.h" +#include "vec/data_types/data_type_jsonb.h" +#include "vec/data_types/data_type_nullable.h" + +namespace doris::vectorized { + +class ConvertFieldToTypeTest : public ::testing::Test { +protected: + void SetUp() override {} +}; + +// Test FieldVisitorToJsonb with different field types using the same pattern as convert_field_to_typeImpl +TEST_F(ConvertFieldToTypeTest, FieldVisitorToJsonb_Null) { + JsonbWriter writer; + + // Test null field using Field::dispatch pattern + Field null_field; + Field::dispatch([&writer](const auto& value) { FieldVisitorToJsonb()(value, &writer); }, + null_field); + + auto* output = writer.getOutput(); + ASSERT_NE(output, nullptr); + ASSERT_GT(output->getSize(), 0); + + // Verify the output is valid JSONB + JsonbDocument* doc = nullptr; + auto status = + JsonbDocument::checkAndCreateDocument(output->getBuffer(), output->getSize(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + + // Verify it's a null value + ASSERT_TRUE(doc->getValue()->isNull()); +} + +TEST_F(ConvertFieldToTypeTest, FieldVisitorToJsonb_Int64) { + JsonbWriter writer; + + // Test Int64 field using Field::dispatch pattern + Int64 test_value = 12345; + Field int_field = Field::create_field(test_value); + Field::dispatch([&writer](const auto& value) { FieldVisitorToJsonb()(value, &writer); }, + int_field); + + auto* output = writer.getOutput(); + ASSERT_NE(output, nullptr); + ASSERT_GT(output->getSize(), 0); + + // Verify the output is valid JSONB + JsonbDocument* doc = nullptr; + auto status = + JsonbDocument::checkAndCreateDocument(output->getBuffer(), output->getSize(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + + // Verify it's an integer value + ASSERT_TRUE(doc->getValue()->isInt64()); + ASSERT_EQ(doc->getValue()->int_val(), test_value); +} + +TEST_F(ConvertFieldToTypeTest, FieldVisitorToJsonb_UInt64) { + JsonbWriter writer; + + // Test UInt64 field using Field::dispatch pattern + UInt64 test_value = 12345; + Field uint_field = Field::create_field(test_value); + Field::dispatch([&writer](const auto& value) { FieldVisitorToJsonb()(value, &writer); }, + uint_field); + + auto* output = writer.getOutput(); + ASSERT_NE(output, nullptr); + ASSERT_GT(output->getSize(), 0); + + // Verify the output is valid JSONB + JsonbDocument* doc = nullptr; + auto status = + JsonbDocument::checkAndCreateDocument(output->getBuffer(), output->getSize(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + + // Verify it's an integer value + ASSERT_TRUE(doc->getValue()->isInt64()); + ASSERT_EQ(doc->getValue()->int_val(), static_cast(test_value)); +} + +TEST_F(ConvertFieldToTypeTest, FieldVisitorToJsonb_Float64) { + JsonbWriter writer; + + // Test Float64 field using Field::dispatch pattern + Float64 test_value = 123.456; + Field double_field = Field::create_field(test_value); + Field::dispatch([&writer](const auto& value) { FieldVisitorToJsonb()(value, &writer); }, + double_field); + + auto* output = writer.getOutput(); + ASSERT_NE(output, nullptr); + ASSERT_GT(output->getSize(), 0); + + // Verify the output is valid JSONB + JsonbDocument* doc = nullptr; + auto status = + JsonbDocument::checkAndCreateDocument(output->getBuffer(), output->getSize(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + + // Verify it's a double value + ASSERT_TRUE(doc->getValue()->isDouble()); + ASSERT_DOUBLE_EQ(doc->getValue()->unpack()->val(), test_value); +} + +TEST_F(ConvertFieldToTypeTest, FieldVisitorToJsonb_String) { + JsonbWriter writer; + + // Test String field using Field::dispatch pattern + String test_value = "hello world"; + Field string_field = Field::create_field(test_value); + Field::dispatch([&writer](const auto& value) { FieldVisitorToJsonb()(value, &writer); }, + string_field); + + auto* output = writer.getOutput(); + ASSERT_NE(output, nullptr); + ASSERT_GT(output->getSize(), 0); + + // Verify the output is valid JSONB + JsonbDocument* doc = nullptr; + auto status = + JsonbDocument::checkAndCreateDocument(output->getBuffer(), output->getSize(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + + // Verify it's a string value + ASSERT_TRUE(doc->getValue()->isString()); + const auto* string_val = doc->getValue()->unpack(); + std::string real_string(string_val->getBlob(), string_val->getBlobLen()); + ASSERT_EQ(real_string, test_value); +} + +TEST_F(ConvertFieldToTypeTest, FieldVisitorToJsonb_JsonbField) { + JsonbWriter writer; + JsonBinaryValue jsonb_value; + std::string test_data = R"({"a": ["1", "2"]})"; + THROW_IF_ERROR(jsonb_value.from_json_string(test_data)); + Field jsonb_field_obj = + Field::create_field(JsonbField(jsonb_value.value(), jsonb_value.size())); + + // Test JsonbField using Field::dispatch pattern + Field::dispatch([&writer](const auto& value) { FieldVisitorToJsonb()(value, &writer); }, + jsonb_field_obj); + + auto* output = writer.getOutput(); + ASSERT_NE(output, nullptr); + ASSERT_GT(output->getSize(), 0); + + // Verify the output is valid JSONB + JsonbDocument* doc = nullptr; + auto status = + JsonbDocument::checkAndCreateDocument(output->getBuffer(), output->getSize(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + + // Verify it's an object value + ASSERT_TRUE(doc->getValue()->isObject()); +} + +TEST_F(ConvertFieldToTypeTest, FieldVisitorToJsonb_Array) { + JsonbWriter writer; + + // Create an array with mixed types + Array array_field; + array_field.push_back(Field::create_field(123)); + array_field.push_back(Field::create_field("hello")); + array_field.push_back(Field::create_field(456.789)); + + Field array_obj = Field::create_field(array_field); + + // Test Array using Field::dispatch pattern + Field::dispatch([&writer](const auto& value) { FieldVisitorToJsonb()(value, &writer); }, + array_obj); + + auto* output = writer.getOutput(); + ASSERT_NE(output, nullptr); + ASSERT_GT(output->getSize(), 0); + + // Verify the output is valid JSONB + JsonbDocument* doc = nullptr; + auto status = + JsonbDocument::checkAndCreateDocument(output->getBuffer(), output->getSize(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + + // Verify it's an array value + ASSERT_TRUE(doc->getValue()->isArray()); + ASSERT_EQ(doc->getValue()->numElements(), 3); +} + +TEST_F(ConvertFieldToTypeTest, FieldVisitorToJsonb_NestedArray) { + JsonbWriter writer; + + // Create a nested array + Array inner_array; + inner_array.push_back(Field::create_field(1)); + inner_array.push_back(Field::create_field(2)); + + Array outer_array; + outer_array.push_back(Field::create_field(inner_array)); + outer_array.push_back(Field::create_field("nested")); + + Field nested_array_obj = Field::create_field(outer_array); + + // Test nested Array using Field::dispatch pattern + Field::dispatch([&writer](const auto& value) { FieldVisitorToJsonb()(value, &writer); }, + nested_array_obj); + + auto* output = writer.getOutput(); + ASSERT_NE(output, nullptr); + ASSERT_GT(output->getSize(), 0); + + // Verify the output is valid JSONB + JsonbDocument* doc = nullptr; + auto status = + JsonbDocument::checkAndCreateDocument(output->getBuffer(), output->getSize(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + + // Verify it's an array value + ASSERT_TRUE(doc->getValue()->isArray()); + ASSERT_EQ(doc->getValue()->numElements(), 2); +} + +TEST_F(ConvertFieldToTypeTest, FieldVisitorToJsonb_LargeInt) { + JsonbWriter writer; + + // Test Int128 field using Field::dispatch pattern + Int128 test_value = 1234567890123456789; + Field largeint_field = Field::create_field(test_value); + Field::dispatch([&writer](const auto& value) { FieldVisitorToJsonb()(value, &writer); }, + largeint_field); + + auto* output = writer.getOutput(); + ASSERT_NE(output, nullptr); + ASSERT_GT(output->getSize(), 0); + + // Verify the output is valid JSONB + JsonbDocument* doc = nullptr; + auto status = + JsonbDocument::checkAndCreateDocument(output->getBuffer(), output->getSize(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + + // Verify it's an int128 value + ASSERT_TRUE(doc->getValue()->isInt128()); + ASSERT_EQ(doc->getValue()->int_val(), test_value); +} + +TEST_F(ConvertFieldToTypeTest, FieldVisitorToJsonb_UInt128) { + JsonbWriter writer; + + // Test UInt128 field using Field::dispatch pattern + UInt128 test_value = 1234567890123456789; + Field uint128_field = Field::create_field(test_value); + Field::dispatch([&writer](const auto& value) { FieldVisitorToJsonb()(value, &writer); }, + uint128_field); + + auto* output = writer.getOutput(); + ASSERT_NE(output, nullptr); + ASSERT_GT(output->getSize(), 0); + + // Verify the output is valid JSONB + JsonbDocument* doc = nullptr; + auto status = + JsonbDocument::checkAndCreateDocument(output->getBuffer(), output->getSize(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + + // Verify it's an int128 value + ASSERT_TRUE(doc->getValue()->isInt128()); + ASSERT_EQ(doc->getValue()->int_val(), static_cast(test_value)); +} + +// Test convert_field_to_type function with JSONB type (similar to convert_field_to_typeImpl) +TEST_F(ConvertFieldToTypeTest, ConvertFieldToType_ToJsonb) { + DataTypeJsonb jsonb_type; + + // Test converting Int64 to JSONB + { + Int64 test_value = 12345; + Field int_field = Field::create_field(test_value); + Field result; + + convert_field_to_type(int_field, jsonb_type, &result); + + ASSERT_EQ(result.get_type(), TYPE_JSONB); + ASSERT_FALSE(result.is_null()); + + const JsonbField& jsonb_result = result.get(); + ASSERT_NE(jsonb_result.get_value(), nullptr); + ASSERT_GT(jsonb_result.get_size(), 0); + + // Verify the JSONB content + JsonbDocument* doc = nullptr; + auto status = JsonbDocument::checkAndCreateDocument(jsonb_result.get_value(), + jsonb_result.get_size(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + ASSERT_TRUE(doc->getValue()->isInt64()); + ASSERT_EQ(doc->getValue()->int_val(), test_value); + } + + // Test converting String to JSONB + { + String test_value = "hello world"; + Field string_field = Field::create_field(test_value); + Field result; + + convert_field_to_type(string_field, jsonb_type, &result); + + ASSERT_EQ(result.get_type(), TYPE_JSONB); + ASSERT_FALSE(result.is_null()); + + const JsonbField& jsonb_result = result.get(); + ASSERT_NE(jsonb_result.get_value(), nullptr); + ASSERT_GT(jsonb_result.get_size(), 0); + + // Verify the JSONB content + JsonbDocument* doc = nullptr; + auto status = JsonbDocument::checkAndCreateDocument(jsonb_result.get_value(), + jsonb_result.get_size(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + ASSERT_TRUE(doc->getValue()->isString()); + const auto* string_val = doc->getValue()->unpack(); + std::string real_string(string_val->getBlob(), string_val->getBlobLen()); + ASSERT_EQ(real_string, test_value); + } + + // Test converting Array to JSONB + { + Array array_field; + array_field.push_back(Field::create_field(1)); + array_field.push_back(Field::create_field("test")); + array_field.push_back(Field::create_field(3.14)); + + Field array_obj = Field::create_field(array_field); + Field result; + + convert_field_to_type(array_obj, jsonb_type, &result); + + ASSERT_EQ(result.get_type(), TYPE_JSONB); + ASSERT_FALSE(result.is_null()); + + const JsonbField& jsonb_result = result.get(); + ASSERT_NE(jsonb_result.get_value(), nullptr); + ASSERT_GT(jsonb_result.get_size(), 0); + + // Verify the JSONB content + JsonbDocument* doc = nullptr; + auto status = JsonbDocument::checkAndCreateDocument(jsonb_result.get_value(), + jsonb_result.get_size(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + ASSERT_TRUE(doc->getValue()->isArray()); + ASSERT_EQ(doc->getValue()->numElements(), 3); + } + + // Test converting JSONB to JSONB (should be no-op) + { + JsonbWriter test_writer; + test_writer.writeStartObject(); + test_writer.writeKey("key"); + test_writer.writeString("value"); + test_writer.writeEndObject(); + + auto* test_output = test_writer.getOutput(); + JsonbField original_jsonb(test_output->getBuffer(), test_output->getSize()); + Field jsonb_field = Field::create_field(original_jsonb); + Field result; + + convert_field_to_type(jsonb_field, jsonb_type, &result); + + ASSERT_EQ(result.get_type(), TYPE_JSONB); + ASSERT_FALSE(result.is_null()); + + const JsonbField& jsonb_result = result.get(); + ASSERT_NE(jsonb_result.get_value(), nullptr); + ASSERT_EQ(jsonb_result.get_size(), original_jsonb.get_size()); + ASSERT_EQ(memcmp(jsonb_result.get_value(), original_jsonb.get_value(), + original_jsonb.get_size()), + 0); + } +} + +// Test convert_field_to_type with nullable JSONB type +TEST_F(ConvertFieldToTypeTest, ConvertFieldToType_ToNullableJsonb) { + auto nullable_jsonb_type = + std::make_shared(std::make_shared()); + + // Test converting null field + { + Field null_field; + Field result; + + convert_field_to_type(null_field, *nullable_jsonb_type, &result); + + ASSERT_TRUE(result.is_null()); + } + + // Test converting non-null field + { + String test_value = "test string"; + Field string_field = Field::create_field(test_value); + Field result; + + convert_field_to_type(string_field, *nullable_jsonb_type, &result); + + ASSERT_EQ(result.get_type(), TYPE_JSONB); + ASSERT_FALSE(result.is_null()); + + const JsonbField& jsonb_result = result.get(); + ASSERT_NE(jsonb_result.get_value(), nullptr); + ASSERT_GT(jsonb_result.get_size(), 0); + + // Verify the JSONB content + JsonbDocument* doc = nullptr; + auto status = JsonbDocument::checkAndCreateDocument(jsonb_result.get_value(), + jsonb_result.get_size(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument: " << status.to_string(); + ASSERT_NE(doc, nullptr); + ASSERT_TRUE(doc->getValue()->isString()); + const auto* string_val = doc->getValue()->unpack(); + std::string real_string(string_val->getBlob(), string_val->getBlobLen()); + ASSERT_EQ(real_string, test_value); + } +} + +// Test convert_field_to_type with array of JSONB +TEST_F(ConvertFieldToTypeTest, ConvertFieldToType_ArrayToJsonb) { + auto array_jsonb_type = std::make_shared(std::make_shared()); + + // Create an array with mixed types that will be converted to JSONB + Array array_field; + array_field.push_back(Field::create_field(123)); + array_field.push_back(Field::create_field("hello")); + array_field.push_back(Field::create_field(456.789)); + + Field array_obj = Field::create_field(array_field); + Field result; + + convert_field_to_type(array_obj, *array_jsonb_type, &result); + + ASSERT_EQ(result.get_type(), TYPE_ARRAY); + ASSERT_FALSE(result.is_null()); + + const Array& result_array = result.get(); + ASSERT_EQ(result_array.size(), 3); + + // Verify each element is converted to JSONB + for (size_t i = 0; i < result_array.size(); ++i) { + ASSERT_EQ(result_array[i].get_type(), TYPE_JSONB); + ASSERT_FALSE(result_array[i].is_null()); + + const auto& jsonb_element = result_array[i].get(); + ASSERT_NE(jsonb_element.get_value(), nullptr); + ASSERT_GT(jsonb_element.get_size(), 0); + + // Verify the JSONB content + JsonbDocument* doc = nullptr; + auto status = JsonbDocument::checkAndCreateDocument(jsonb_element.get_value(), + jsonb_element.get_size(), &doc); + ASSERT_TRUE(status.ok()) << "Failed to create JsonbDocument for element " << i << ": " + << status.to_string(); + ASSERT_NE(doc, nullptr); + } +} + +// Test error cases +TEST_F(ConvertFieldToTypeTest, ConvertFieldToType_ErrorCases) { + DataTypeJsonb jsonb_type; + + // Test with unsupported types (should throw exception) + { + Field tuple_field = Field::create_field(Tuple()); + + EXPECT_THROW( + { + Field result; + convert_field_to_type(tuple_field, jsonb_type, &result); + }, + doris::Exception); + } +} + +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/test/vec/jsonb/json_parser_test.cpp b/be/test/vec/jsonb/json_parser_test.cpp new file mode 100644 index 00000000000000..6d248a73cd0d91 --- /dev/null +++ b/be/test/vec/jsonb/json_parser_test.cpp @@ -0,0 +1,169 @@ +// 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/json/json_parser.h" + +#include + +#include + +using doris::vectorized::JSONDataParser; +using doris::vectorized::SimdJSONParser; +using doris::vectorized::ParseConfig; + +TEST(JsonParserTest, ParseSimpleTypes) { + JSONDataParser parser; + ParseConfig config; + + // int + auto result = parser.parse("123", 3, config); + ASSERT_TRUE(result.has_value()); + EXPECT_EQ(result->values.size(), 1); + + // double + result = parser.parse("1.23", 4, config); + ASSERT_TRUE(result.has_value()); + + // bool + result = parser.parse("true", 4, config); + ASSERT_TRUE(result.has_value()); + + // null + result = parser.parse("null", 4, config); + ASSERT_TRUE(result.has_value()); + + // string + result = parser.parse("\"abc\"", 5, config); + ASSERT_TRUE(result.has_value()); +} + +TEST(JsonParserTest, ParseObjectAndArray) { + JSONDataParser parser; + ParseConfig config; + + // Object + auto result = parser.parse(R"({"a":1,"b":2})", 13, config); + ASSERT_TRUE(result.has_value()); + EXPECT_EQ(result->values.size(), 2); + + // Array + result = parser.parse("[1,2,3]", 7, config); + ASSERT_TRUE(result.has_value()); + EXPECT_EQ(result->values.size(), 1); +} + +TEST(JsonParserTest, ParseMultiLevelNestedArray) { + JSONDataParser parser; + ParseConfig config; + + auto result = parser.parse("[[1,2],[3,4]]", 13, config); + ASSERT_TRUE(result.has_value()); + EXPECT_EQ(result->values.size(), 1); + EXPECT_EQ(result->paths.size(), 1); + EXPECT_EQ(result->values[0].get_type(), doris::PrimitiveType::TYPE_ARRAY); + + result = parser.parse("[[[1],[2]],[[3],[4]]]", 21, config); + ASSERT_TRUE(result.has_value()); + EXPECT_EQ(result->values.size(), 1); + EXPECT_EQ(result->paths.size(), 1); + EXPECT_EQ(result->values[0].get_type(), doris::PrimitiveType::TYPE_ARRAY); + + result = parser.parse("[[1,2],[3],[4,5,6]]", 19, config); + ASSERT_TRUE(result.has_value()); + EXPECT_EQ(result->values.size(), 1); + EXPECT_EQ(result->paths.size(), 1); + + // Test complex nested structure + config.enable_flatten_nested = false; + std::string json1 = R"({"a":[[1,2],[3],[4,5,6]]})"; + // multi level nested array in object + result = parser.parse(json1.c_str(), json1.size(), config); + ASSERT_TRUE(result.has_value()); + EXPECT_EQ(result->values.size(), 1); + EXPECT_EQ(result->paths.size(), 1); + EXPECT_EQ(result->values[0].get_type(), doris::PrimitiveType::TYPE_ARRAY); + + std::string json = R"({"nested": [{"a": [1,2,3]}]})"; + // result should be jsonbField + result = parser.parse(json.c_str(), json.size(), config); + ASSERT_TRUE(result.has_value()); + EXPECT_EQ(result->values.size(), 1); + EXPECT_EQ(result->paths.size(), 1); + EXPECT_EQ(result->values[0].get_type(), doris::PrimitiveType::TYPE_JSONB); + + // multi level nested array in nested array object + std::string json2 = R"({"a":[{"b":[[1,2,3]]}]})"; + result = parser.parse(json2.c_str(), json2.size(), config); + ASSERT_TRUE(result.has_value()); + EXPECT_EQ(result->values.size(), 1); + EXPECT_EQ(result->paths.size(), 1); + EXPECT_EQ(result->values[0].get_type(), doris::PrimitiveType::TYPE_JSONB); + + // test flatten nested + config.enable_flatten_nested = true; + EXPECT_ANY_THROW(parser.parse(json.c_str(), json.size(), config)); + // test flatten nested with multi level nested array + // no throw because it is not nested object array + result = parser.parse(json1.c_str(), json1.size(), config); + ASSERT_TRUE(result.has_value()); + EXPECT_EQ(result->values.size(), 1); + EXPECT_EQ(result->paths.size(), 1); + EXPECT_EQ(result->values[0].get_type(), doris::PrimitiveType::TYPE_ARRAY); + + EXPECT_ANY_THROW(parser.parse(json2.c_str(), json2.size(), config)); +} + +TEST(JsonParserTest, ParseNestedAndFlatten) { + JSONDataParser parser; + ParseConfig config; + config.enable_flatten_nested = true; + + std::string json = R"({"a":[{"b":1},{"b":2}]})"; + auto result = parser.parse(json.c_str(), json.size(), config); + ASSERT_TRUE(result.has_value()); + EXPECT_GT(result->values.size(), 0); + + config.enable_flatten_nested = false; + std::string json2 = R"({"a":[{"b":1},{"b":2}]})"; + result = parser.parse(json2.c_str(), json2.size(), config); + ASSERT_TRUE(result.has_value()); +} + +TEST(JsonParserTest, ParseInvalidJson) { + JSONDataParser parser; + ParseConfig config; + + auto result = parser.parse("{a:1}", 5, config); + ASSERT_FALSE(result.has_value()); + + result = parser.parse("", 0, config); + ASSERT_FALSE(result.has_value()); +} + +TEST(JsonParserTest, ParseCornerCases) { + JSONDataParser parser; + ParseConfig config; + + auto result = parser.parse("{}", 2, config); + ASSERT_TRUE(result.has_value()); + + result = parser.parse("[]", 2, config); + ASSERT_TRUE(result.has_value()); + + result = parser.parse(R"({"a":"\n\t"})", 12, config); + ASSERT_TRUE(result.has_value()); +} diff --git a/regression-test/data/variant_p0/desc.out b/regression-test/data/variant_p0/desc.out index 1eff52e44842f3..71f804cc25c5bb 100644 --- a/regression-test/data/variant_p0/desc.out +++ b/regression-test/data/variant_p0/desc.out @@ -16,7 +16,7 @@ v.b json Yes false \N NONE v.c.c smallint Yes false \N NONE v.c.e double Yes false \N NONE v.ddd.aaa tinyint Yes false \N NONE -v.ddd.mxmxm json Yes false \N NONE +v.ddd.mxmxm array Yes false \N NONE v.oooo.xxxx.xxx tinyint Yes false \N NONE v.xxxx text Yes false \N NONE @@ -37,7 +37,7 @@ v.b json Yes false \N NONE v.c.c smallint Yes false \N NONE v.c.e double Yes false \N NONE v.ddd.aaa tinyint Yes false \N NONE -v.ddd.mxmxm json Yes false \N NONE +v.ddd.mxmxm array Yes false \N NONE v.oooo.xxxx.xxx tinyint Yes false \N NONE v.xxxx text Yes false \N NONE @@ -66,7 +66,7 @@ v.b json Yes false \N NONE v.c.c smallint Yes false \N NONE v.c.e double Yes false \N NONE v.ddd.aaa tinyint Yes false \N NONE -v.ddd.mxmxm json Yes false \N NONE +v.ddd.mxmxm array Yes false \N NONE v.oooo.xxxx.xxx tinyint Yes false \N NONE v.xxxx text Yes false \N NONE diff --git a/regression-test/data/variant_p0/load.out b/regression-test/data/variant_p0/load.out index dbdee9d7940e30..4d2799444ca338 100644 --- a/regression-test/data/variant_p0/load.out +++ b/regression-test/data/variant_p0/load.out @@ -212,7 +212,7 @@ [123] -- !sql_25 -- -50000 54999.99999999898 6150000 +50000 55000.000000007036 6150000 -- !sql_26 -- 5000 @@ -234,8 +234,8 @@ ["123",123,[123]] 123 123456 -[123,"123",1.11111] -[123,1.11,"123"] +[123, "123", 1.11111] +[123, 1.11, "123"] [123,{"xx":1}] [123,{"a":1}] [{"a":1},123] diff --git a/regression-test/data/variant_p0/nested2.out b/regression-test/data/variant_p0/nested2.out new file mode 100644 index 00000000000000..c7790a107de3da --- /dev/null +++ b/regression-test/data/variant_p0/nested2.out @@ -0,0 +1,337 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql_desc_1 -- +k bigint Yes true \N +v variant Yes false \N NONE +v.nested.a array Yes false \N NONE +v.nested.b array Yes false \N NONE +v.nested.c array Yes false \N NONE + +-- !sql_0 -- +1 {"nested":[{"a":1,"c":1.1},{"b":"1"}]} + +-- !sql_1 -- +[1, null] + +-- !sql_2 -- +[null, "1"] + +-- !sql_3 -- +[1.1, null] + +-- !sql_4 -- +[{"a":1,"c":1.1}, {"b":"1"}] + +-- !sql_8 -- +[1, null] 2 + +-- !sql_9 -- +[null, 1] 2 + +-- !sql_10 -- +[1, null] 2 + +-- !sql_11 -- +["1", null] 2 + +-- !sql_12 -- +[null, "1"] 2 + +-- !sql_13 -- +["1.1", null] 2 + +-- !sql_14 -- +[1, null] 2 + +-- !sql_15 -- +[null, 1] 2 + +-- !sql_16 -- +[1.1, null] 2 + +-- !sql_desc_2 -- +k bigint Yes true \N +v variant Yes false \N NONE +v.nested.a array Yes false \N NONE +v.nested.b array Yes false \N NONE +v.nested.c array Yes false \N NONE + +-- !sql_0 -- +1 {"nested":[{"a":1.0,"c":1.1},{"b":"1"}]} +2 {"nested":[{"a":2.5,"b":123.1}]} + +-- !sql_1 -- +[1, null] +[2.5] + +-- !sql_2 -- +[null, ""1""] +["123.1"] + +-- !sql_3 -- +[1.1, null] +\N + +-- !sql_4 -- +[{"a":1.0,"c":1.1}, {"b":"1"}] +[{"a":2.5,"b":123.1}] + +-- !sql_8 -- +[1, null] 2 +[2] 1 + +-- !sql_9 -- +[null, 1] 2 +[123] 1 + +-- !sql_10 -- +[1, null] 2 +\N \N + +-- !sql_11 -- +["1", null] 2 +["2.5"] 1 + +-- !sql_12 -- +[null, "1"] 2 +["123.1"] 1 + +-- !sql_13 -- +["1.1", null] 2 +\N \N + +-- !sql_14 -- +[1, null] 2 +[2.5] 1 + +-- !sql_15 -- +[null, 1] 2 +[123.1] 1 + +-- !sql_16 -- +[1.1, null] 2 +\N \N + +-- !sql_0 -- +1 {"nested":[{"a":1.0,"c":1.1},{"b":"1"}]} +2 {"nested":[{"a":2.5,"b":123.1}]} + +-- !sql_1 -- +[1, null] +[2.5] + +-- !sql_2 -- +[null, ""1""] +["123.1"] + +-- !sql_3 -- +[1.1, null] +[null] + +-- !sql_4 -- +[{"a":1.0,"c":1.1}, {"b":"1"}] +[{"a":2.5,"b":123.1}] + +-- !sql_8 -- +[1, null] 2 +[2] 1 + +-- !sql_9 -- +[null, 1] 2 +[123] 1 + +-- !sql_10 -- +[1, null] 2 +[null] 1 + +-- !sql_11 -- +["1", null] 2 +["2.5"] 1 + +-- !sql_12 -- +[null, "1"] 2 +["123.1"] 1 + +-- !sql_13 -- +["1.1", null] 2 +[null] 1 + +-- !sql_14 -- +[1, null] 2 +[2.5] 1 + +-- !sql_15 -- +[null, 1] 2 +[123.1] 1 + +-- !sql_16 -- +[1.1, null] 2 +[null] 1 + +-- !sql_desc_4 -- +k bigint Yes true \N +v variant Yes false \N NONE +v.nested.a array Yes false \N NONE +v.nested.b array Yes false \N NONE +v.nested.c array Yes false \N NONE + +-- !sql_0 -- +1 {"nested":[{"a":1,"b":1.1},{"a":"1","b":"1","c":"1"}]} + +-- !sql_1 -- +["1", ""1""] + +-- !sql_2 -- +["1.1", ""1""] + +-- !sql_3 -- +[null, "1"] + +-- !sql_4 -- +[{"a":1,"b":1.1}, {"a":"1","b":"1","c":"1"}] + +-- !sql_8 -- +[1, 1] 2 + +-- !sql_9 -- +[1, 1] 2 + +-- !sql_10 -- +[null, 1] 2 + +-- !sql_11 -- +["1", "1"] 2 + +-- !sql_12 -- +["1.1", "1"] 2 + +-- !sql_13 -- +[null, "1"] 2 + +-- !sql_14 -- +[1, 1] 2 + +-- !sql_15 -- +[1.1, 1] 2 + +-- !sql_16 -- +[null, 1] 2 + +-- !sql_desc_5 -- +k bigint Yes true \N +v variant Yes false \N NONE +v.nested.a array Yes false \N NONE +v.nested.b array Yes false \N NONE +v.nested.c array Yes false \N NONE + +-- !sql_0 -- +1 {"nested":[{"a":1,"b":1.1},{"a":"1","b":"1","c":"1"}]} +2 {"nested":[{"a":1,"c":1.1}]} + +-- !sql_1 -- +["1", ""1""] +["1"] + +-- !sql_2 -- +["1.1", ""1""] +\N + +-- !sql_3 -- +[null, ""1""] +["1.1"] + +-- !sql_4 -- +[{"a":1,"b":1.1}, {"a":"1","b":"1","c":"1"}] +[{"a":1,"c":1.1}] + +-- !sql_8 -- +[1, 1] 2 +[1] 1 + +-- !sql_9 -- +[1, 1] 2 +\N \N + +-- !sql_10 -- +[null, 1] 2 +[1] 1 + +-- !sql_11 -- +["1", "1"] 2 +["1"] 1 + +-- !sql_12 -- +["1.1", "1"] 2 +\N \N + +-- !sql_13 -- +[null, "1"] 2 +["1.1"] 1 + +-- !sql_14 -- +[1, 1] 2 +[1] 1 + +-- !sql_15 -- +[1.1, 1] 2 +\N \N + +-- !sql_16 -- +[null, 1] 2 +[1.1] 1 + +-- !sql_0 -- +1 {"nested":[{"a":1,"b":1.1},{"a":"1","b":"1","c":"1"}]} +2 {"nested":[{"a":1,"c":1.1}]} + +-- !sql_1 -- +["1", ""1""] +["1"] + +-- !sql_2 -- +["1.1", ""1""] +[null] + +-- !sql_3 -- +[null, ""1""] +["1.1"] + +-- !sql_4 -- +[{"a":1,"b":1.1}, {"a":"1","b":"1","c":"1"}] +[{"a":1,"c":1.1}] + +-- !sql_8 -- +[1, 1] 2 +[1] 1 + +-- !sql_9 -- +[1, 1] 2 +[null] 1 + +-- !sql_10 -- +[null, 1] 2 +[1] 1 + +-- !sql_11 -- +["1", "1"] 2 +["1"] 1 + +-- !sql_12 -- +["1.1", "1"] 2 +[null] 1 + +-- !sql_13 -- +[null, "1"] 2 +["1.1"] 1 + +-- !sql_14 -- +[1, 1] 2 +[1] 1 + +-- !sql_15 -- +[1.1, 1] 2 +[null] 1 + +-- !sql_16 -- +[null, 1] 2 +[1.1] 1 + diff --git a/regression-test/suites/variant_p0/nested2.groovy b/regression-test/suites/variant_p0/nested2.groovy new file mode 100644 index 00000000000000..8d48fcfce9b1c0 --- /dev/null +++ b/regression-test/suites/variant_p0/nested2.groovy @@ -0,0 +1,151 @@ +// 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. + +// this test is used to test the type conflict of nested array +suite("variant_nested_type_conflict", "p0"){ + + try { + + def table_name = "var_nested_type_conflict" + sql "DROP TABLE IF EXISTS ${table_name}" + sql """set describe_extend_variant_column = true""" + + sql """ + CREATE TABLE IF NOT EXISTS ${table_name} ( + k bigint, + v variant + ) + DUPLICATE KEY(`k`) + DISTRIBUTED BY HASH(k) BUCKETS 1 -- 1 bucket make really compaction in conflict case + properties("replication_num" = "1", "disable_auto_compaction" = "false", "variant_enable_flatten_nested" = "true"); + """ + def sql_select_batch = { + qt_sql_0 """select * from ${table_name} order by k""" + + qt_sql_1 """select v['nested']['a'] from ${table_name} order by k""" + qt_sql_2 """select v['nested']['b'] from ${table_name} order by k""" + qt_sql_3 """select v['nested']['c'] from ${table_name} order by k""" + + qt_sql_4 """select v['nested'] from ${table_name} order by k""" + } + + def sql_test_cast_to_array = { + // test cast to array + qt_sql_8 """select cast(v['nested']['a'] as array), size(cast(v['nested']['a'] as array)) from ${table_name} order by k""" + qt_sql_9 """select cast(v['nested']['b'] as array), size(cast(v['nested']['b'] as array)) from ${table_name} order by k""" + qt_sql_10 """select cast(v['nested']['c'] as array), size(cast(v['nested']['c'] as array)) from ${table_name} order by k""" + + // test cast to array + qt_sql_11 """select cast(v['nested']['a'] as array), size(cast(v['nested']['a'] as array)) from ${table_name} order by k""" + qt_sql_12 """select cast(v['nested']['b'] as array), size(cast(v['nested']['b'] as array)) from ${table_name} order by k""" + qt_sql_13 """select cast(v['nested']['c'] as array), size(cast(v['nested']['c'] as array)) from ${table_name} order by k""" + + // test cast to array + qt_sql_14 """select cast(v['nested']['a'] as array), size(cast(v['nested']['a'] as array)) from ${table_name} order by k""" + qt_sql_15 """select cast(v['nested']['b'] as array), size(cast(v['nested']['b'] as array)) from ${table_name} order by k""" + qt_sql_16 """select cast(v['nested']['c'] as array), size(cast(v['nested']['c'] as array)) from ${table_name} order by k""" + + } + // insert Nested array in Nested array which is not supported + test { + sql """ + insert into ${table_name} values (1, '{"nested": [{"a": [1,2,3]}]}'); + """ + exception "Nesting of array in Nested array within variant subcolumns is currently not supported." + } + /// insert a array of object for a, b, c + // insert type conflict in multiple rows + sql """ + insert into ${table_name} values (1, '{"nested": [{"a": 1, "c": 1.1}, {"b": "1"}]}'); + """ + + // for cloud we should select first and then desc for syncing rowset to get latest schema + sql """ + select * from ${table_name} order by k limit 1; + """ + qt_sql_desc_1 """ + desc ${table_name}; + """ + // now select for a, b, c + sql_select_batch() + sql_test_cast_to_array() + /// insert a, b type changed to double + sql """ + insert into ${table_name} values (2, '{"nested": [{"a": 2.5, "b": 123.1}]}'); + """ + // for cloud we should select first and then desc for syncing rowset to get latest schema + sql """ + select * from ${table_name} order by k limit 1; + """ + qt_sql_desc_2 """ + desc ${table_name}; + """ + // now select for a, b, c + sql_select_batch() + sql_test_cast_to_array() + + // trigger and wait compaction + trigger_and_wait_compaction("${table_name}", "full") + + // now select for a, b, c + sql_select_batch() + sql_test_cast_to_array() + + sql """ truncate table ${table_name} """ + + + // insert type conflict in one row + sql """ + insert into ${table_name} values (1, '{"nested": [{"a": 1, "b": 1.1}, {"a": "1", "b": "1", "c": "1"}]}'); + """ + // for cloud we should select first and then desc for syncing rowset to get latest schema + sql """ + select * from ${table_name} order by k limit 1; + """ + qt_sql_desc_4 """ + desc ${table_name}; + """ + // now select for a, b, c + sql_select_batch() + sql_test_cast_to_array() + + // insert c type changed to double + sql """ + insert into ${table_name} values (2, '{"nested": [{"a": 1, "c": 1.1}]}'); + """ + // for cloud we should select first and then desc for syncing rowset to get latest schema + sql """ + select * from ${table_name} order by k limit 1; + """ + qt_sql_desc_5 """ + desc ${table_name}; + """ + // now select for a, b, c + sql_select_batch() + sql_test_cast_to_array() + + // trigger and wait compaction + trigger_and_wait_compaction("${table_name}", "full") + + // now select for a, b, c + sql_select_batch() + sql_test_cast_to_array() + + } finally { + } + +}