From 60b7eae092149067816a981b7999fa7d826e56d8 Mon Sep 17 00:00:00 2001 From: amorynan Date: Mon, 23 Dec 2024 22:35:48 +0800 Subject: [PATCH 01/23] add ut for column_array --- .../segment_v2/hierarchical_data_reader.h | 2 +- be/src/vec/columns/column.h | 2 - be/src/vec/columns/column_array.cpp | 28 +- be/src/vec/columns/column_array.h | 3 - be/src/vec/columns/column_nullable.h | 1 - be/src/vec/columns/column_object.cpp | 2 +- be/src/vec/functions/match.cpp | 2 +- be/test/vec/columns/column_array_test.cpp | 425 ++++++++++++++++++ be/test/vec/columns/common_column_test.h | 29 +- .../data_types/common_data_type_serder_test.h | 13 +- be/test/vec/data_types/data_type_ip_test.cpp | 6 - be/test/vec/function/function_test_util.cpp | 3 + 12 files changed, 463 insertions(+), 53 deletions(-) create mode 100644 be/test/vec/columns/column_array_test.cpp diff --git a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h index f85038713cadb7..910bd4f41b5c21 100644 --- a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h +++ b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h @@ -173,7 +173,7 @@ class HierarchicalDataReader : public ColumnIterator { for (const auto& subcolumn : entry.second) { const auto& column = subcolumn.column; const auto& type = subcolumn.type; - if (!remove_nullable(column)->is_column_array()) { + if (!check_column(remove_nullable(column))) { return Status::InvalidArgument( "Meet none array column when flatten nested array, path {}, type {}", subcolumn.path.get_path(), subcolumn.type->get_name()); diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index 729e5470c97ad5..ed1f4ea76b71c1 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -609,8 +609,6 @@ class IColumn : public COW { virtual bool is_column_dictionary() const { return false; } - virtual bool is_column_array() const { return false; } - virtual bool is_column_map() const { return false; } virtual bool is_column_struct() const { return false; } diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp index 0df0ddcb0f30d5..1a3f0ff3e58749 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -44,6 +44,8 @@ namespace doris::vectorized { ColumnArray::ColumnArray(MutableColumnPtr&& nested_column, MutableColumnPtr&& offsets_column) : data(std::move(nested_column)), offsets(std::move(offsets_column)) { + data = data->convert_to_full_column_if_const(); + offsets = offsets->convert_to_full_column_if_const(); const auto* offsets_concrete = typeid_cast(offsets.get()); if (!offsets_concrete) { @@ -70,12 +72,12 @@ ColumnArray::ColumnArray(MutableColumnPtr&& nested_column, MutableColumnPtr&& of } ColumnArray::ColumnArray(MutableColumnPtr&& nested_column) : data(std::move(nested_column)) { + data = data->convert_to_full_column_if_const(); if (!data->empty()) { throw doris::Exception(ErrorCode::INTERNAL_ERROR, "Not empty data passed to ColumnArray, but no offsets passed"); __builtin_unreachable(); } - offsets = ColumnOffsets::create(); } @@ -122,10 +124,10 @@ Field ColumnArray::operator[](size_t n) const { size_t size = size_at(n); if (size > max_array_size_as_field) - throw doris::Exception( - ErrorCode::INVALID_ARGUMENT, - "Array of size {}, is too large to be manipulated as single field, maximum size {}", - size, max_array_size_as_field); + throw doris::Exception(ErrorCode::INVALID_ARGUMENT, + "Array of size {} in row {}, is too large to be manipulated as " + "single field, maximum size {}", + size, n, max_array_size_as_field); Array res(size); @@ -139,10 +141,10 @@ void ColumnArray::get(size_t n, Field& res) const { size_t size = size_at(n); if (size > max_array_size_as_field) - throw doris::Exception( - ErrorCode::INVALID_ARGUMENT, - "Array of size {}, is too large to be manipulated as single field, maximum size {}", - size, max_array_size_as_field); + throw doris::Exception(ErrorCode::INVALID_ARGUMENT, + "Array of size {} in row {}, is too large to be manipulated as " + "single field, maximum size {}", + size, n, max_array_size_as_field); res = Array(size); Array& res_arr = doris::vectorized::get(res); @@ -362,7 +364,7 @@ void ColumnArray::insert_default() { void ColumnArray::pop_back(size_t n) { auto& offsets_data = get_offsets(); - DCHECK(n <= offsets_data.size()); + DCHECK(n <= offsets_data.size()) << " n:" << n << " with offsets size: " << offsets_data.size(); size_t nested_n = offsets_data.back() - offset_at(offsets_data.size() - n); if (nested_n) get_data().pop_back(nested_n); offsets_data.resize_assume_reserved(offsets_data.size() - n); @@ -390,12 +392,6 @@ size_t ColumnArray::allocated_bytes() const { return get_data().allocated_bytes() + get_offsets().allocated_bytes(); } -ColumnPtr ColumnArray::convert_to_full_column_if_const() const { - /// It is possible to have an array with constant data and non-constant offsets. - /// Example is the result of expression: replicate('hello', [1]) - return ColumnArray::create(data->convert_to_full_column_if_const(), offsets); -} - bool ColumnArray::has_equal_offsets(const ColumnArray& other) const { const Offsets64& offsets1 = get_offsets(); const Offsets64& offsets2 = other.get_offsets(); diff --git a/be/src/vec/columns/column_array.h b/be/src/vec/columns/column_array.h index 4dbc8e91e52b88..510cf2ae3ca130 100644 --- a/be/src/vec/columns/column_array.h +++ b/be/src/vec/columns/column_array.h @@ -118,7 +118,6 @@ class ColumnArray final : public COWHelper { using ColumnOffsets = ColumnVector; std::string get_name() const override; - bool is_column_array() const override { return true; } bool is_variable_length() const override { return true; } bool is_exclusive() const override { @@ -165,8 +164,6 @@ class ColumnArray final : public COWHelper { ColumnPtr replicate(const IColumn::Offsets& replicate_offsets) const override; void insert_many_from(const IColumn& src, size_t position, size_t length) override; - ColumnPtr convert_to_full_column_if_const() const override; - /** More efficient methods of manipulation */ IColumn& get_data() { return *data; } const IColumn& get_data() const { return *data; } diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 84b3ce0f82aadb..cc8281912104e7 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -329,7 +329,6 @@ class ColumnNullable final : public COWHelper, public N bool is_nullable() const override { return true; } bool is_concrete_nullable() const override { return true; } bool is_column_string() const override { return get_nested_column().is_column_string(); } - bool is_column_array() const override { return get_nested_column().is_column_array(); } bool is_column_map() const override { return get_nested_column().is_column_map(); } bool is_column_struct() const override { return get_nested_column().is_column_struct(); } diff --git a/be/src/vec/columns/column_object.cpp b/be/src/vec/columns/column_object.cpp index d67a70d2f630f0..1dd86d7e095113 100644 --- a/be/src/vec/columns/column_object.cpp +++ b/be/src/vec/columns/column_object.cpp @@ -1271,7 +1271,7 @@ bool ColumnObject::is_finalized() const { void ColumnObject::Subcolumn::wrapp_array_nullable() { // Wrap array with nullable, treat empty array as null to elimate conflict at present auto& result_column = get_finalized_column_ptr(); - if (result_column->is_column_array() && !result_column->is_nullable()) { + if (check_column(result_column) && !result_column->is_nullable()) { auto new_null_map = ColumnUInt8::create(); new_null_map->reserve(result_column->size()); auto& null_map_data = new_null_map->get_data(); diff --git a/be/src/vec/functions/match.cpp b/be/src/vec/functions/match.cpp index d6ed3ca944a27a..97ce518a211dda 100644 --- a/be/src/vec/functions/match.cpp +++ b/be/src/vec/functions/match.cpp @@ -106,7 +106,7 @@ Status FunctionMatchBase::execute_impl(FunctionContext* context, Block& block, block.get_by_position(arguments[0]).column->convert_to_full_column_if_const(); const auto* values = check_and_get_column(source_col.get()); const ColumnArray* array_col = nullptr; - if (source_col->is_column_array()) { + if (check_column(source_col)) { array_col = check_and_get_column(source_col.get()); if (array_col && !array_col->get_data().is_column_string()) { return Status::NotSupported(fmt::format( diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp new file mode 100644 index 00000000000000..89e2e152033507 --- /dev/null +++ b/be/test/vec/columns/column_array_test.cpp @@ -0,0 +1,425 @@ +// 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 +#include + +#include "vec/columns/column.h" +#include "vec/columns/common_column_test.h" +#include "vec/core/types.h" +#include "vec/function/function_test_util.h" + +// this test is gonna to make a template ColumnTest +// for example column_ip should test these functions + +namespace doris::vectorized { +class ColumnArrayTest : public CommonColumnTest { +protected: + void SetUp() override { + // insert from data csv and assert insert result + std::string data_file_dir = "regression-test/data/nereids_function_p0/array/"; + MutableColumns array_cols; + // we need to load data from csv file into column_array list + // step1. create data type for array nested type (const and nullable) + // array + BaseInputTypeSet array_tinyint = {TypeIndex::Array, TypeIndex::Int8}; + // array + BaseInputTypeSet array_smallint = {TypeIndex::Array, TypeIndex::Int16}; + // array + BaseInputTypeSet array_int = {TypeIndex::Array, TypeIndex::Int32}; + // array + BaseInputTypeSet array_bigint = {TypeIndex::Array, TypeIndex::Int64}; + // array + BaseInputTypeSet array_largeint = {TypeIndex::Array, TypeIndex::Int128}; + // array + BaseInputTypeSet array_float = {TypeIndex::Array, TypeIndex::Float32}; + // array + BaseInputTypeSet array_double = {TypeIndex::Array, TypeIndex::Float64}; + // array + BaseInputTypeSet array_ipv4 = {TypeIndex::Array, TypeIndex::IPv4}; + // array + BaseInputTypeSet array_ipv6 = {TypeIndex::Array, TypeIndex::IPv6}; + // array + BaseInputTypeSet array_date = {TypeIndex::Array, TypeIndex::Date}; + // array + BaseInputTypeSet array_datetime = {TypeIndex::Array, TypeIndex::DateTime}; + // array + BaseInputTypeSet array_datev2 = {TypeIndex::Array, TypeIndex::DateV2}; + // array + BaseInputTypeSet array_datetimev2 = {TypeIndex::Array, TypeIndex::DateTimeV2}; + // array + BaseInputTypeSet array_varchar = {TypeIndex::Array, TypeIndex::String}; + // array UT + BaseInputTypeSet array_decimal = {TypeIndex::Array, TypeIndex::Decimal32}; + // array UT + BaseInputTypeSet array_decimal64 = {TypeIndex::Array, TypeIndex::Decimal64}; + // array UT + BaseInputTypeSet array_decimal128 = {TypeIndex::Array, TypeIndex::Decimal128V3}; + // array UT + BaseInputTypeSet array_decimal256 = {TypeIndex::Array, TypeIndex::Decimal256}; + std::vector array_typeIndex = { + array_tinyint, array_smallint, array_int, array_bigint, array_largeint, + array_float, array_double, array_ipv4, array_ipv6, array_date, + array_datetime, array_datev2, array_datetimev2, array_varchar, array_decimal, + array_decimal64, array_decimal128, array_decimal256}; + + vector descs; + descs.reserve(array_typeIndex.size()); + for (int i = 0; i < array_typeIndex.size(); i++) { + descs.push_back(ut_type::UTDataTypeDescs()); + InputTypeSet input_types {}; + input_types.push_back(array_typeIndex[i][0]); + input_types.push_back(Nullable {static_cast(array_typeIndex[i][1])}); + EXPECT_EQ(input_types[1].type(), &typeid(Nullable)) << "nested type is not nullable"; + EXPECT_TRUE(parse_ut_data_type(input_types, descs[i])); + } + + // create column_array for each data type + vector data_files = {data_file_dir + "test_array_tinyint.csv", + data_file_dir + "test_array_smallint.csv", + data_file_dir + "test_array_int.csv", + data_file_dir + "test_array_bigint.csv", + data_file_dir + "test_array_largeint.csv", + data_file_dir + "test_array_float.csv", + data_file_dir + "test_array_double.csv", + data_file_dir + "test_array_ipv4.csv", + data_file_dir + "test_array_ipv6.csv", + data_file_dir + "test_array_date.csv", + data_file_dir + "test_array_datetime.csv", + data_file_dir + "test_array_date.csv", + data_file_dir + "test_array_datetimev2(6).csv", + data_file_dir + "test_array_varchar(65535).csv", + data_file_dir + "test_array_decimalv3(7,4).csv", + data_file_dir + "test_array_decimalv3(16,10).csv", + data_file_dir + "test_array_decimalv3(38,30).csv", + data_file_dir + "test_array_decimalv3(76,56).csv"}; + + // step2. according to the datatype to make column_array + // && load data from csv file into column_array + EXPECT_EQ(descs.size(), data_files.size()); + for (int i = 0; i < array_typeIndex.size(); i++) { + auto& desc = descs[i]; + auto& data_file = data_files[i]; + // first is array type + auto& type = desc[0].data_type; + std::cout << "type: " << type->get_name() << " with file: " << data_file << std::endl; + MutableColumns columns; + columns.push_back(type->create_column()); + auto serde = type->get_serde(1); + load_data_from_csv({serde}, columns, data_file, ';'); + array_columns.push_back(std::move(columns[0])); + array_types.push_back(type); + serdes.push_back(serde); + } + // step3. show array column data + for (int i = 0; i < array_columns.size(); i++) { + // auto& column = array_columns[i]; + // printColumn(*column, *descs[i][0].data_type); + } + } + + MutableColumns array_columns; // column_array list + DataTypes array_types; + DataTypeSerDeSPtrs serdes; +}; + +//////////////////////// basic function from column.h //////////////////////// +TEST_F(ColumnArrayTest, InsertRangeFromTest) { + assert_insert_range_from_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, InsertManyFromTest) { + assert_insert_many_from_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, InsertIndicesFromTest) { + assert_insert_indices_from_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, InsertDefaultTest) { + assert_insert_default_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, InsertManyDefaultsTest) { + assert_insert_many_defaults_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, GetDataAtTest) { + assert_get_data_at_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, FieldTest) { + assert_field_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, GetRawDataTest) { + EXPECT_ANY_THROW({ array_columns[0]->get_raw_data(); }); +} + +TEST_F(ColumnArrayTest, GetBoolTest) { + EXPECT_ANY_THROW({ array_columns[0]->get_bool(0); }); +} + +TEST_F(ColumnArrayTest, GetIntTest) { + EXPECT_ANY_THROW({ array_columns[0]->get_int(0); }); +} + +TEST_F(ColumnArrayTest, SerDeVecTest) { + ser_deser_vec(array_columns, array_types); +} + +TEST_F(ColumnArrayTest, serDeserializeWithArenaImpl) { + ser_deserialize_with_arena_impl(array_columns, array_types); +} + +TEST_F(ColumnArrayTest, SizeTest) { + assert_size_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, ByteSizeTest) { + assert_byte_size_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, AllocateBytesTest) { + assert_allocated_bytes_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, PopbackTest) { + assert_pop_back_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, CloneTest) { + assert_clone_resized_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, CutTest) { + assert_cut_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, ResizeTest) { + assert_resize_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, ReserveTest) { + assert_reserve_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, ReplicateTest) { + assert_replicate_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, ReplaceColumnTest) { + assert_replace_column_data_callback(array_columns, serdes); + assert_replace_column_null_data_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, AppendDataBySelectorTest) { + assert_append_data_by_selector_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, PermutationAndSortTest) { + for (int i = 0; i < array_columns.size(); i++) { + auto& column = array_columns[i]; + auto& type = array_types[i]; + auto column_type = type->get_name(); + std::cout << "column_type: " << column_type << std::endl; + // permutation + assert_column_permutations(column->assume_mutable_ref(), type); + } +} + +TEST_F(ColumnArrayTest, FilterTest) { + assert_filter_callback(array_columns, serdes); +} + +// HASH Interfaces +TEST_F(ColumnArrayTest, HashTest) { + // XXHash + assert_update_hashes_with_value_callback(array_columns, serdes); + // XXhash with null_data + + // CrcHash + std::vector pts(array_columns.size(), PrimitiveType::TYPE_ARRAY); + assert_update_crc_hashes_callback(array_columns, serdes, pts); + // CrcHash with null_data +}; + +//////////////////////// special function from column_array.h //////////////////////// +TEST_F(ColumnArrayTest, CreateArrayTest) { + // test create_array : nested_column && offsets_column should not be const, and convert_to_full_column_if_const should not impl in array + // in some situation, + // like join_probe_operator.cpp::_build_output_block, + // we call column.convert_to_full_column_if_const, + // then we may call clear_column_data() to clear the column (eg. in HashJoinProbeOperatorX::pull() which call local_state._probe_block.clear_column_data after filter_data_and_build_output()) + // in clear_column_data() if use_count() == 1, we will call column->clear() to clear the column data + // + // however in array impl for convert_to_full_column_if_const: ``` ColumnArray::create(data->convert_to_full_column_if_const(), offsets);``` + // may make the nested_column use_count() more than 1 which means it is shared with other block, but return ColumnArray is new which use_count() is 1, + // then in clear_column_data() if we will call array_column->use_count() == 1 will be true to clear the column with nested_column, and shared nested_column block will meet undefined behavior cause maybe core + // + // so actually according to the semantics of the function, it should not impl in array, + // but we should make sure in creation of array, the nested_column && offsets_column should not be const + for (int i = 0; i < array_columns.size(); i++) { + auto column = check_and_get_column( + remove_nullable(array_columns[i]->assume_mutable())); + auto& type = array_types[i]; + auto column_size = column->size(); + auto column_type = type->get_name(); + std::cout << "column_type: " << column_type << std::endl; + // test create_array + auto last_offset = column->get_offsets().back(); + EXPECT_ANY_THROW( + { auto const_col = ColumnConst::create(column->get_data_ptr(), last_offset); }); + auto tmp_data_col = column->get_data_ptr()->clone_resized(1); + auto const_col = ColumnConst::create(tmp_data_col->assume_mutable(), last_offset); + EXPECT_ANY_THROW({ + // const_col is not empty + auto new_array_column = ColumnArray::create(const_col->assume_mutable()); + }); + auto new_array_column = + ColumnArray::create(const_col->assume_mutable(), column->get_offsets_ptr()); + EXPECT_EQ(new_array_column->size(), column_size) + << "array_column size is not equal to column size"; + EXPECT_EQ(new_array_column->get_data_ptr()->size(), column->get_data_ptr()->size()); + EXPECT_EQ(new_array_column->get_offsets_ptr()->size(), column->get_offsets_ptr()->size()); + } +} + +TEST_F(ColumnArrayTest, MetaInfoTest) { + // test is_variable_length which should all be true + for (int i = 0; i < array_columns.size(); i++) { + auto& column = array_columns[i]; + auto& type = array_types[i]; + auto column_type = type->get_name(); + std::cout << "column_type: " << column_type << std::endl; + EXPECT_TRUE(column->is_variable_length()) << "column is not variable length"; + } +} + +TEST_F(ColumnArrayTest, ConvertIfOverflowAndInsertTest) { + // test nested string in array which like ColumnArray only use in join + // test convert_column_if_overflow && insert_range_from_ignore_overflow + for (int i = 0; i < array_columns.size(); i++) { + auto& column = array_columns[i]; + DataTypeArray type = array_types[i]; + if (!is_string(type.get_nested_type())) { + // check ptr is itself + auto ptr = column->convert_column_if_overflow(); + EXPECT_EQ(ptr.get(), column.get()); + auto* array_col = check_and_get_column(column); + auto nested_col = array_col->get_data_ptr(); + auto* array_col1 = check_and_get_column(ptr); + auto nested_col1 = array_col1->get_data_ptr(); + EXPECT_EQ(nested_col.get(), nested_col1.get()); + } else { + auto ptr = column->convert_column_if_overflow(); + } + } +} + +TEST_F(ColumnArrayTest, GetNumberOfDimensionsTest) { + // test dimension of array + for (int i = 0; i < array_columns.size(); i++) { + auto column = check_and_get_column( + remove_nullable(array_columns[i]->assume_mutable())); + auto check_type = remove_nullable(array_types[i]); + auto dimension = 0; + while (is_array(check_type)) { + auto nested_type = reinterpret_cast(*check_type) + .get_nested_type(); + dimension++; + check_type = nested_type; + } + EXPECT_EQ(column->get_number_of_dimensions(), dimension) + << "column dimension is not equal to check_type dimension"; + } +} + +TEST_F(ColumnArrayTest, MaxArraySizeAsFieldTest) { + // test array max_array_size_as_field which is set to 100w + // in operator[] and get() + for (int i = 0; i < array_columns.size(); i++) { + auto column = check_and_get_column( + remove_nullable(array_columns[i]->assume_mutable())); + auto check_type = remove_nullable(array_types[i]); + Field a; + column->get(column->size() - 1, a); + Array af = a.get(); + if (af.size() > 0) { + Field ef = af[0]; + for (int j = 0; j < max_array_size_as_field; ++j) { + af.push_back(ef); + } + std::cout << "array size: " << af.size() << std::endl; + auto cloned = column->clone_resized(0); + cloned->insert(af); + std::cout << "cloned size: " << cloned->size() << std::endl; + // get cloned offset size + auto cloned_offset_size = + check_and_get_column(cloned)->get_offsets().back(); + std::cout << "cloned offset size: " << cloned_offset_size << std::endl; + + Field f; + // test get + EXPECT_ANY_THROW({ cloned->get(0, f); }); + // test operator[] + EXPECT_ANY_THROW({ cloned->operator[](0); }); + } + } +} + +TEST_F(ColumnArrayTest, IsDefaultAtTest) { + // default means meet empty array row in column_array, now just only used in ColumnObject. + // test is_default_at + for (int i = 0; i < array_columns.size(); i++) { + auto column = check_and_get_column( + remove_nullable(array_columns[i]->assume_mutable())); + auto column_size = column->size(); + for (int j = 0; j < column_size; j++) { + auto is_default = column->is_default_at(j); + if (is_default) { + // check field Array is empty + Field f; + column->get(j, f); + auto array = f.get(); + EXPECT_EQ(array.size(), 0) << "array is not empty"; + } + } + } +} + +TEST_F(ColumnArrayTest, HasEqualOffsetsTest) { + // test has_equal_offsets which more likely used in function, eg: function_array_zip + for (int i = 0; i < array_columns.size(); i++) { + auto column = check_and_get_column( + remove_nullable(array_columns[i]->assume_mutable())); + auto cloned = array_columns[i]->clone_resized(array_columns[i]->size()); + auto cloned_arr = + check_and_get_column(remove_nullable(cloned->assume_mutable())); + // test expect true + EXPECT_EQ(column->get_offsets().size(), cloned_arr->get_offsets().size()); + EXPECT_TRUE(column->has_equal_offsets(*cloned_arr)); + // cloned column size is not equal to column size + cloned->pop_back(1); + EXPECT_FALSE(column->has_equal_offsets(*cloned_arr)); + cloned->insert_default(); + EXPECT_FALSE(column->has_equal_offsets(*cloned_arr)); + } +} + +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/test/vec/columns/common_column_test.h b/be/test/vec/columns/common_column_test.h index 8e1b86c0168f99..6eb129cc20dc40 100644 --- a/be/test/vec/columns/common_column_test.h +++ b/be/test/vec/columns/common_column_test.h @@ -72,9 +72,9 @@ class CommonColumnTest : public ::testing::Test { int l_idx = 0; int c_idx = 0; while (std::getline(lineStream, value, spliter)) { - if (idxes.contains(l_idx)) { + if (!value.starts_with("//") && idxes.contains(l_idx)) { Slice string_slice(value.data(), value.size()); - std::cout << string_slice << std::endl; + // std::cout << string_slice << std::endl; if (auto st = serders[c_idx]->deserialize_one_cell_from_json( *columns[c_idx], string_slice, options); !st.ok()) { @@ -412,11 +412,17 @@ class CommonColumnTest : public ::testing::Test { // insert_range_from now we have no any exception error data to handle, so here will meet crash continue; } else if (*pos + *cl > source_column->size()) { + if (check_column( + remove_nullable(source_column->assume_mutable()))) { + // insert_range_from in array has DCHECK_LG + continue; + } target_column->clear(); // insert_range_from now we have no any exception error data to handle and also no crash std::cout << "we expect exception insert_many_from from " << *pos - << " with length " << *cl - << "with source size: " << source_column->size() << std::endl; + << " with length " << *cl << " for column " + << source_column->get_name() + << " with source size: " << source_column->size() << std::endl; target_column->insert_many_from(*source_column, *pos, *cl); } @@ -461,9 +467,9 @@ class CommonColumnTest : public ::testing::Test { // uint32_t(-1) now we have compiler to make sure it will not appear // and this function - std::vector check_indices = {0, uint32_t(source_column->size()), - uint32_t(source_column->size() + 1), - uint32_t((source_column->size() + 1) >> 1)}; + std::vector check_indices = {0, uint32_t((source_column->size() + 1) >> 1), + uint32_t(source_column->size()), + uint32_t(source_column->size() + 1)}; for (auto from_idx = check_indices.begin(); from_idx < check_indices.end(); ++from_idx) { for (auto end_idx = check_indices.begin(); end_idx < check_indices.end(); @@ -477,11 +483,12 @@ class CommonColumnTest : public ::testing::Test { // here we will meet `heap-buffer-overflow on address` continue; } else { + std::cout << source_column->get_name() << " now insert_indices_from from " + << *from_idx << " to " << *end_idx + << " with source size: " << source_column->size() << std::endl; target_column->insert_indices_from(*source_column, &(*from_idx), &(*end_idx)); } - std::cout << source_column->get_name() << " now insert_indices_from from " - << *from_idx << " to " << *end_idx << std::endl; // Verify the inserted data matches the expected results in `assert_res` auto ser_col = ColumnString::create(); ser_col->reserve(target_column->size()); @@ -1691,7 +1698,7 @@ class CommonColumnTest : public ::testing::Test { if (column->size() > rows) { std::cerr << "Column size mismatch: " << column->size() << " vs " << rows << std::endl; - column->pop_back(rows - column->size()); + column->pop_back(column->size() - rows); } else if (column->size() < rows) { std::cerr << "Column size mismatch: " << column->size() << " vs " << rows << std::endl; @@ -1753,7 +1760,7 @@ class CommonColumnTest : public ::testing::Test { if (column->size() > rows) { std::cerr << "Column size mismatch: " << column->size() << " vs " << rows << std::endl; - column->pop_back(rows - column->size()); + column->pop_back(column->size() - rows); } else if (column->size() < rows) { std::cerr << "Column size mismatch: " << column->size() << " vs " << rows << std::endl; diff --git a/be/test/vec/data_types/common_data_type_serder_test.h b/be/test/vec/data_types/common_data_type_serder_test.h index 46206d5ed7eb15..a900439e5c0340 100644 --- a/be/test/vec/data_types/common_data_type_serder_test.h +++ b/be/test/vec/data_types/common_data_type_serder_test.h @@ -24,22 +24,13 @@ #include #include -#include "olap/schema.h" -#include "runtime/descriptors.cpp" #include "runtime/descriptors.h" #include "util/arrow/block_convertor.h" #include "util/arrow/row_batch.h" #include "vec/columns/column.h" -#include "vec/columns/column_array.h" -#include "vec/columns/column_map.h" -#include "vec/columns/columns_number.h" #include "vec/core/field.h" -#include "vec/core/sort_block.h" -#include "vec/core/sort_description.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_map.h" #include "vec/utils/arrow_column_to_doris_column.h" // this test is gonna to be a data type serialize and deserialize functions @@ -155,7 +146,7 @@ class CommonDataTypeSerdeTest : public ::testing::Test { std::string value; int l_idx = 0; int c_idx = 0; - std::vector row; + std::vector row; while (std::getline(lineStream, value, spliter)) { if (idxes.contains(l_idx)) { // load csv data @@ -210,7 +201,7 @@ class CommonDataTypeSerdeTest : public ::testing::Test { if (generate_res_file) { // generate res auto pos = file_path.find_last_of("."); - string hive_format = is_hive_format ? "_hive" : ""; + std::string hive_format = is_hive_format ? "_hive" : ""; std::string res_file = file_path.substr(0, pos) + hive_format + "_serde_res.csv"; std::ofstream res_f(res_file); if (!res_f.is_open()) { diff --git a/be/test/vec/data_types/data_type_ip_test.cpp b/be/test/vec/data_types/data_type_ip_test.cpp index 72a340b9e145b8..80861a70bd1412 100644 --- a/be/test/vec/data_types/data_type_ip_test.cpp +++ b/be/test/vec/data_types/data_type_ip_test.cpp @@ -20,17 +20,11 @@ #include #include -#include #include -#include "olap/schema.h" #include "vec/columns/column.h" -#include "vec/columns/column_array.h" -#include "vec/columns/column_map.h" #include "vec/columns/columns_number.h" #include "vec/core/field.h" -#include "vec/core/sort_block.h" -#include "vec/core/sort_description.h" #include "vec/core/types.h" #include "vec/data_types/common_data_type_serder_test.h" #include "vec/data_types/common_data_type_test.h" diff --git a/be/test/vec/function/function_test_util.cpp b/be/test/vec/function/function_test_util.cpp index ddc97c440ed07a..7cc381c260a21c 100644 --- a/be/test/vec/function/function_test_util.cpp +++ b/be/test/vec/function/function_test_util.cpp @@ -189,6 +189,9 @@ size_t type_index_to_data_type(const std::vector& input_types, size_t i return ret; } desc.children.push_back(sub_desc.type_desc); + if (sub_desc.is_nullable) { + sub_type = make_nullable(sub_type); + } type = std::make_shared(sub_type); return ret + 1; } From cfb1bb00ea30cfe33873fa5987ca18104c925310 Mon Sep 17 00:00:00 2001 From: amorynan Date: Wed, 25 Dec 2024 14:30:50 +0800 Subject: [PATCH 02/23] change check_column to is_column and del is_column_map/struct --- be/src/olap/rowset/segment_v2/hierarchical_data_reader.h | 2 +- be/src/pipeline/exec/join_probe_operator.cpp | 2 +- be/src/vec/columns/column.cpp | 4 ++-- be/src/vec/columns/column.h | 8 ++------ be/src/vec/columns/column_map.h | 1 - be/src/vec/columns/column_nullable.h | 2 -- be/src/vec/columns/column_object.cpp | 2 +- be/src/vec/columns/column_struct.h | 1 - be/src/vec/exprs/table_function/vexplode_json_object.cpp | 2 +- be/src/vec/exprs/table_function/vexplode_map.cpp | 2 +- be/src/vec/exprs/table_function/vposexplode.cpp | 2 +- be/src/vec/functions/array/function_array_element.h | 2 +- be/src/vec/functions/array/function_array_index.h | 2 +- be/src/vec/functions/array/function_array_map.h | 2 +- be/src/vec/functions/array/function_array_remove.h | 2 +- be/src/vec/functions/array/function_array_set.h | 2 +- be/src/vec/functions/function_helpers.cpp | 2 +- be/src/vec/functions/function_helpers.h | 2 +- be/src/vec/functions/function_reverse.h | 2 +- be/src/vec/functions/function_string.h | 4 ++-- be/src/vec/functions/match.cpp | 2 +- 21 files changed, 21 insertions(+), 29 deletions(-) diff --git a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h index 910bd4f41b5c21..a699d46f19ca1e 100644 --- a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h +++ b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h @@ -173,7 +173,7 @@ class HierarchicalDataReader : public ColumnIterator { for (const auto& subcolumn : entry.second) { const auto& column = subcolumn.column; const auto& type = subcolumn.type; - if (!check_column(remove_nullable(column))) { + if (!is_column(remove_nullable(column).get())) { return Status::InvalidArgument( "Meet none array column when flatten nested array, path {}, type {}", subcolumn.path.get_path(), subcolumn.type->get_name()); diff --git a/be/src/pipeline/exec/join_probe_operator.cpp b/be/src/pipeline/exec/join_probe_operator.cpp index 11b5b29c8b556b..7b205c3052d4e7 100644 --- a/be/src/pipeline/exec/join_probe_operator.cpp +++ b/be/src/pipeline/exec/join_probe_operator.cpp @@ -150,7 +150,7 @@ Status JoinProbeLocalState::_build_output_block( /// TODO: maybe need a method to check if a column need to be converted to full /// column. if (is_column_const(*origin_column) || - check_column(origin_column)) { + is_column(origin_column)) { auto column_ptr = origin_column->convert_to_full_column_if_const(); insert_column_datas(mutable_columns[i], column_ptr, rows); } else { diff --git a/be/src/vec/columns/column.cpp b/be/src/vec/columns/column.cpp index 57fd127f52de42..5b0ecf8de52b25 100644 --- a/be/src/vec/columns/column.cpp +++ b/be/src/vec/columns/column.cpp @@ -73,11 +73,11 @@ void IColumn::compare_internal(size_t rhs_row_id, const IColumn& rhs, int nan_di } bool is_column_nullable(const IColumn& column) { - return check_column(column); + return is_column(column); } bool is_column_const(const IColumn& column) { - return check_column(column); + return is_column(column); } } // namespace doris::vectorized diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index ed1f4ea76b71c1..eb76b18386411e 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -609,10 +609,6 @@ class IColumn : public COW { virtual bool is_column_dictionary() const { return false; } - virtual bool is_column_map() const { return false; } - - virtual bool is_column_struct() const { return false; } - /// If the only value column can contain is NULL. virtual bool only_null() const { return false; } @@ -713,12 +709,12 @@ const Type* check_and_get_column(const IColumn* column) { } template -bool check_column(const IColumn& column) { +bool is_column(const IColumn& column) { return check_and_get_column(&column); } template -bool check_column(const IColumn* column) { +bool is_column(const IColumn* column) { return check_and_get_column(column); } diff --git a/be/src/vec/columns/column_map.h b/be/src/vec/columns/column_map.h index ae482a2d4e058c..f3575da1b744d8 100644 --- a/be/src/vec/columns/column_map.h +++ b/be/src/vec/columns/column_map.h @@ -113,7 +113,6 @@ class ColumnMap final : public COWHelper { void insert_default() override; void pop_back(size_t n) override; - bool is_column_map() const override { return true; } 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; diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index cc8281912104e7..fa500fac42fb96 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -329,8 +329,6 @@ class ColumnNullable final : public COWHelper, public N bool is_nullable() const override { return true; } bool is_concrete_nullable() const override { return true; } bool is_column_string() const override { return get_nested_column().is_column_string(); } - bool is_column_map() const override { return get_nested_column().is_column_map(); } - bool is_column_struct() const override { return get_nested_column().is_column_struct(); } bool is_exclusive() const override { return IColumn::is_exclusive() && nested_column->is_exclusive() && diff --git a/be/src/vec/columns/column_object.cpp b/be/src/vec/columns/column_object.cpp index 1dd86d7e095113..9500cf06544d90 100644 --- a/be/src/vec/columns/column_object.cpp +++ b/be/src/vec/columns/column_object.cpp @@ -1271,7 +1271,7 @@ bool ColumnObject::is_finalized() const { void ColumnObject::Subcolumn::wrapp_array_nullable() { // Wrap array with nullable, treat empty array as null to elimate conflict at present auto& result_column = get_finalized_column_ptr(); - if (check_column(result_column) && !result_column->is_nullable()) { + if (is_column(result_column) && !result_column->is_nullable()) { auto new_null_map = ColumnUInt8::create(); new_null_map->reserve(result_column->size()); auto& null_map_data = new_null_map->get_data(); diff --git a/be/src/vec/columns/column_struct.h b/be/src/vec/columns/column_struct.h index e9f8014d9db06e..98d264b64e73f1 100644 --- a/be/src/vec/columns/column_struct.h +++ b/be/src/vec/columns/column_struct.h @@ -82,7 +82,6 @@ class ColumnStruct final : public COWHelper { } std::string get_name() const override; - bool is_column_struct() const override { return true; } MutableColumnPtr clone_empty() const override; MutableColumnPtr clone_resized(size_t size) const override; size_t size() const override { return columns.at(0)->size(); } diff --git a/be/src/vec/exprs/table_function/vexplode_json_object.cpp b/be/src/vec/exprs/table_function/vexplode_json_object.cpp index 7db4da395aeae2..eaaba1bcbff95f 100644 --- a/be/src/vec/exprs/table_function/vexplode_json_object.cpp +++ b/be/src/vec/exprs/table_function/vexplode_json_object.cpp @@ -110,7 +110,7 @@ void VExplodeJsonObjectTableFunction::get_same_many_values(MutableColumnPtr& col assert_cast( assert_cast(column.get())->get_null_map_column_ptr().get()) ->insert_many_defaults(length); - } else if (column->is_column_struct()) { + } else if (is_column(column.get())) { ret = assert_cast(column.get()); } else { throw Exception(ErrorCode::INTERNAL_ERROR, diff --git a/be/src/vec/exprs/table_function/vexplode_map.cpp b/be/src/vec/exprs/table_function/vexplode_map.cpp index a9b59f3c69ccad..ed3353351434cd 100644 --- a/be/src/vec/exprs/table_function/vexplode_map.cpp +++ b/be/src/vec/exprs/table_function/vexplode_map.cpp @@ -106,7 +106,7 @@ void VExplodeMapTableFunction::get_same_many_values(MutableColumnPtr& column, in assert_cast( assert_cast(column.get())->get_null_map_column_ptr().get()) ->insert_many_defaults(length); - } else if (column->is_column_struct()) { + } else if (is_column(column.get())) { ret = assert_cast(column.get()); } else { throw Exception(ErrorCode::INTERNAL_ERROR, diff --git a/be/src/vec/exprs/table_function/vposexplode.cpp b/be/src/vec/exprs/table_function/vposexplode.cpp index 20d04a219f831a..9369ef26be33eb 100644 --- a/be/src/vec/exprs/table_function/vposexplode.cpp +++ b/be/src/vec/exprs/table_function/vposexplode.cpp @@ -99,7 +99,7 @@ void VPosExplodeTableFunction::get_same_many_values(MutableColumnPtr& column, in assert_cast( assert_cast(column.get())->get_null_map_column_ptr().get()) ->insert_many_defaults(length); - } else if (column->is_column_struct()) { + } else if (is_column(column.get())) { ret = assert_cast(column.get()); } else { throw Exception(ErrorCode::INTERNAL_ERROR, diff --git a/be/src/vec/functions/array/function_array_element.h b/be/src/vec/functions/array/function_array_element.h index e4b57348c883ae..c1a0b53c7d7867 100644 --- a/be/src/vec/functions/array/function_array_element.h +++ b/be/src/vec/functions/array/function_array_element.h @@ -114,7 +114,7 @@ class FunctionArrayElement : public IFunction { args = {col_left, block.get_by_position(arguments[1])}; } ColumnPtr res_column = nullptr; - if (args[0].column->is_column_map() || + if (is_column(args[0].column.get()) || check_column_const(args[0].column.get())) { res_column = _execute_map(args, input_rows_count, src_null_map, dst_null_map); } else { diff --git a/be/src/vec/functions/array/function_array_index.h b/be/src/vec/functions/array/function_array_index.h index e602d67a73b01c..7f08bfdd4b4dda 100644 --- a/be/src/vec/functions/array/function_array_index.h +++ b/be/src/vec/functions/array/function_array_index.h @@ -340,7 +340,7 @@ class FunctionArrayIndex : public IFunction { const IColumn& right_column, const UInt8* right_nested_null_map, const UInt8* outer_null_map) const { - if (check_column(right_column)) { + if (is_column(right_column)) { return _execute_number( offsets, nested_null_map, nested_column, right_column, right_nested_null_map, outer_null_map); diff --git a/be/src/vec/functions/array/function_array_map.h b/be/src/vec/functions/array/function_array_map.h index fd4a2fc59f3548..f7ba63e3588ed4 100644 --- a/be/src/vec/functions/array/function_array_map.h +++ b/be/src/vec/functions/array/function_array_map.h @@ -178,7 +178,7 @@ struct ArrayMapImpl { static bool _execute_internal(ColumnArrayMutableData& dst, ColumnArrayExecutionDatas datas, std::vector& col_const, size_t start_row, size_t end_row) { for (auto data : datas) { - if (!check_column(*data.nested_col)) { + if (!is_column(*data.nested_col)) { return false; } } diff --git a/be/src/vec/functions/array/function_array_remove.h b/be/src/vec/functions/array/function_array_remove.h index 197b032b0f8a4b..ef9c9939e96420 100644 --- a/be/src/vec/functions/array/function_array_remove.h +++ b/be/src/vec/functions/array/function_array_remove.h @@ -254,7 +254,7 @@ class FunctionArrayRemove : public IFunction { ColumnPtr _execute_number_expanded(const ColumnArray::Offsets64& offsets, const IColumn& nested_column, const IColumn& right_column, const UInt8* nested_null_map) const { - if (check_column(right_column)) { + if (is_column(right_column)) { return _execute_number( offsets, nested_column, right_column, nested_null_map); } diff --git a/be/src/vec/functions/array/function_array_set.h b/be/src/vec/functions/array/function_array_set.h index 1ecf6d72531c73..f7404507e9eebf 100644 --- a/be/src/vec/functions/array/function_array_set.h +++ b/be/src/vec/functions/array/function_array_set.h @@ -174,7 +174,7 @@ struct ArraySetImpl { const ColumnArrayExecutionData& left_data, const ColumnArrayExecutionData& right_data) { using Impl = OpenSetImpl; - if (!check_column(*left_data.nested_col)) { + if (!is_column(*left_data.nested_col)) { return false; } constexpr auto execute_left_column_first = Impl::Action::execute_left_column_first; diff --git a/be/src/vec/functions/function_helpers.cpp b/be/src/vec/functions/function_helpers.cpp index aa93ace35d8c35..6f6e4b2aa5d332 100644 --- a/be/src/vec/functions/function_helpers.cpp +++ b/be/src/vec/functions/function_helpers.cpp @@ -139,7 +139,7 @@ const ColumnConst* check_and_get_column_const_string_or_fixedstring(const IColum const ColumnConst* res = assert_cast(column); - if (check_column(&res->get_data_column())) return res; + if (is_column(&res->get_data_column())) return res; return {}; } diff --git a/be/src/vec/functions/function_helpers.h b/be/src/vec/functions/function_helpers.h index 8c7eec28fe2f6f..14f08652c366e6 100644 --- a/be/src/vec/functions/function_helpers.h +++ b/be/src/vec/functions/function_helpers.h @@ -57,7 +57,7 @@ const ColumnConst* check_and_get_column_const(const IColumn* column) { const ColumnConst* res = assert_cast(column); - if (!check_column(&res->get_data_column())) return {}; + if (!is_column(&res->get_data_column())) return {}; return res; } diff --git a/be/src/vec/functions/function_reverse.h b/be/src/vec/functions/function_reverse.h index ee0005a305d8ce..ed26a456cae90a 100644 --- a/be/src/vec/functions/function_reverse.h +++ b/be/src/vec/functions/function_reverse.h @@ -47,7 +47,7 @@ class FunctionReverseCommon : public IFunction { RETURN_IF_ERROR(ReverseImpl::vector(col_string->get_chars(), col_string->get_offsets(), col_res->get_chars(), col_res->get_offsets())); block.replace_by_position(result, std::move(col_res)); - } else if (check_column(src_column.get())) { + } else if (is_column(src_column.get())) { return ArrayReverseImpl::_execute(block, arguments, result, input_rows_count); } else { return Status::RuntimeError("Illegal column {} used for argument of function {}", diff --git a/be/src/vec/functions/function_string.h b/be/src/vec/functions/function_string.h index 6e4a18fdd3100e..2f9cd512d953e5 100644 --- a/be/src/vec/functions/function_string.h +++ b/be/src/vec/functions/function_string.h @@ -1363,7 +1363,7 @@ class FunctionStringConcatWs : public IFunction { null_list[i] = &const_null_map->get_data(); } - if (check_column(argument_columns[i].get())) { + if (is_column(argument_columns[i].get())) { continue; } @@ -1380,7 +1380,7 @@ class FunctionStringConcatWs : public IFunction { fmt::memory_buffer buffer; std::vector views; - if (check_column(argument_columns[1].get())) { + if (is_column(argument_columns[1].get())) { // Determine if the nested type of the array is String const auto& array_column = reinterpret_cast(*argument_columns[1]); if (!array_column.get_data().is_column_string()) { diff --git a/be/src/vec/functions/match.cpp b/be/src/vec/functions/match.cpp index 97ce518a211dda..9a3eb52d79b824 100644 --- a/be/src/vec/functions/match.cpp +++ b/be/src/vec/functions/match.cpp @@ -106,7 +106,7 @@ Status FunctionMatchBase::execute_impl(FunctionContext* context, Block& block, block.get_by_position(arguments[0]).column->convert_to_full_column_if_const(); const auto* values = check_and_get_column(source_col.get()); const ColumnArray* array_col = nullptr; - if (check_column(source_col)) { + if (is_column(source_col)) { array_col = check_and_get_column(source_col.get()); if (array_col && !array_col->get_data().is_column_string()) { return Status::NotSupported(fmt::format( From 21716af679a5faae701205ec0b75b818a7b603e0 Mon Sep 17 00:00:00 2001 From: amorynan Date: Wed, 25 Dec 2024 14:53:35 +0800 Subject: [PATCH 03/23] fix clangformat --- be/src/vec/functions/function_helpers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/src/vec/functions/function_helpers.h b/be/src/vec/functions/function_helpers.h index 1f08dbe5507150..0e1dc40d89f08b 100644 --- a/be/src/vec/functions/function_helpers.h +++ b/be/src/vec/functions/function_helpers.h @@ -57,7 +57,7 @@ const ColumnConst* check_and_get_column_const(const IColumn* column) { const auto* res = assert_cast(column); - if (!is_column(&res->get_data_column())) return {}; + if (!is_column(&res->get_data_column())) { return nullptr; } From c96e646315cb1105a1cbc12f0c528536e2e54860 Mon Sep 17 00:00:00 2001 From: amorynan Date: Wed, 25 Dec 2024 16:03:10 +0800 Subject: [PATCH 04/23] fix compile --- be/src/pipeline/exec/join_probe_operator.cpp | 2 +- be/src/vec/columns/column_object.cpp | 2 +- be/src/vec/functions/match.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/be/src/pipeline/exec/join_probe_operator.cpp b/be/src/pipeline/exec/join_probe_operator.cpp index 7b205c3052d4e7..e48ae13c10c5e1 100644 --- a/be/src/pipeline/exec/join_probe_operator.cpp +++ b/be/src/pipeline/exec/join_probe_operator.cpp @@ -150,7 +150,7 @@ Status JoinProbeLocalState::_build_output_block( /// TODO: maybe need a method to check if a column need to be converted to full /// column. if (is_column_const(*origin_column) || - is_column(origin_column)) { + is_column(origin_column.get())) { auto column_ptr = origin_column->convert_to_full_column_if_const(); insert_column_datas(mutable_columns[i], column_ptr, rows); } else { diff --git a/be/src/vec/columns/column_object.cpp b/be/src/vec/columns/column_object.cpp index 92e1b1a523416f..141e30710b6d00 100644 --- a/be/src/vec/columns/column_object.cpp +++ b/be/src/vec/columns/column_object.cpp @@ -1271,7 +1271,7 @@ bool ColumnObject::is_finalized() const { void ColumnObject::Subcolumn::wrapp_array_nullable() { // Wrap array with nullable, treat empty array as null to elimate conflict at present auto& result_column = get_finalized_column_ptr(); - if (is_column(result_column) && !result_column->is_nullable()) { + if (is_column(result_column.get()) && !result_column->is_nullable()) { auto new_null_map = ColumnUInt8::create(); new_null_map->reserve(result_column->size()); auto& null_map_data = new_null_map->get_data(); diff --git a/be/src/vec/functions/match.cpp b/be/src/vec/functions/match.cpp index 3c4fc93c2f0767..8ba4f2f8ebb9cd 100644 --- a/be/src/vec/functions/match.cpp +++ b/be/src/vec/functions/match.cpp @@ -106,7 +106,7 @@ Status FunctionMatchBase::execute_impl(FunctionContext* context, Block& block, block.get_by_position(arguments[0]).column->convert_to_full_column_if_const(); const auto* values = check_and_get_column(source_col.get()); const ColumnArray* array_col = nullptr; - if (is_column(source_col)) { + if (is_column(source_col.get())) { array_col = check_and_get_column(source_col.get()); if (array_col && !array_col->get_data().is_column_string()) { return Status::NotSupported(fmt::format( From f544b3daab46361f8610d468c6e3b907da2bdc15 Mon Sep 17 00:00:00 2001 From: amorynan Date: Thu, 26 Dec 2024 23:15:57 +0800 Subject: [PATCH 05/23] fix column test --- be/test/vec/columns/column_array_test.cpp | 18 +++++++++--------- be/test/vec/columns/common_column_test.h | 4 ++-- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp index 89e2e152033507..847eba877b3624 100644 --- a/be/test/vec/columns/column_array_test.cpp +++ b/be/test/vec/columns/column_array_test.cpp @@ -276,7 +276,7 @@ TEST_F(ColumnArrayTest, CreateArrayTest) { // but we should make sure in creation of array, the nested_column && offsets_column should not be const for (int i = 0; i < array_columns.size(); i++) { auto column = check_and_get_column( - remove_nullable(array_columns[i]->assume_mutable())); + remove_nullable(array_columns[i]->assume_mutable()).get()); auto& type = array_types[i]; auto column_size = column->size(); auto column_type = type->get_name(); @@ -321,9 +321,9 @@ TEST_F(ColumnArrayTest, ConvertIfOverflowAndInsertTest) { // check ptr is itself auto ptr = column->convert_column_if_overflow(); EXPECT_EQ(ptr.get(), column.get()); - auto* array_col = check_and_get_column(column); + auto* array_col = check_and_get_column(column.get()); auto nested_col = array_col->get_data_ptr(); - auto* array_col1 = check_and_get_column(ptr); + auto* array_col1 = check_and_get_column(ptr.get()); auto nested_col1 = array_col1->get_data_ptr(); EXPECT_EQ(nested_col.get(), nested_col1.get()); } else { @@ -336,7 +336,7 @@ TEST_F(ColumnArrayTest, GetNumberOfDimensionsTest) { // test dimension of array for (int i = 0; i < array_columns.size(); i++) { auto column = check_and_get_column( - remove_nullable(array_columns[i]->assume_mutable())); + remove_nullable(array_columns[i]->assume_mutable()).get()); auto check_type = remove_nullable(array_types[i]); auto dimension = 0; while (is_array(check_type)) { @@ -355,7 +355,7 @@ TEST_F(ColumnArrayTest, MaxArraySizeAsFieldTest) { // in operator[] and get() for (int i = 0; i < array_columns.size(); i++) { auto column = check_and_get_column( - remove_nullable(array_columns[i]->assume_mutable())); + remove_nullable(array_columns[i]->assume_mutable()).get()); auto check_type = remove_nullable(array_types[i]); Field a; column->get(column->size() - 1, a); @@ -371,7 +371,7 @@ TEST_F(ColumnArrayTest, MaxArraySizeAsFieldTest) { std::cout << "cloned size: " << cloned->size() << std::endl; // get cloned offset size auto cloned_offset_size = - check_and_get_column(cloned)->get_offsets().back(); + check_and_get_column(cloned.get())->get_offsets().back(); std::cout << "cloned offset size: " << cloned_offset_size << std::endl; Field f; @@ -388,7 +388,7 @@ TEST_F(ColumnArrayTest, IsDefaultAtTest) { // test is_default_at for (int i = 0; i < array_columns.size(); i++) { auto column = check_and_get_column( - remove_nullable(array_columns[i]->assume_mutable())); + remove_nullable(array_columns[i]->assume_mutable()).get()); auto column_size = column->size(); for (int j = 0; j < column_size; j++) { auto is_default = column->is_default_at(j); @@ -407,10 +407,10 @@ TEST_F(ColumnArrayTest, HasEqualOffsetsTest) { // test has_equal_offsets which more likely used in function, eg: function_array_zip for (int i = 0; i < array_columns.size(); i++) { auto column = check_and_get_column( - remove_nullable(array_columns[i]->assume_mutable())); + remove_nullable(array_columns[i]->assume_mutable()).get()); auto cloned = array_columns[i]->clone_resized(array_columns[i]->size()); auto cloned_arr = - check_and_get_column(remove_nullable(cloned->assume_mutable())); + check_and_get_column(remove_nullable(cloned->assume_mutable()).get()); // test expect true EXPECT_EQ(column->get_offsets().size(), cloned_arr->get_offsets().size()); EXPECT_TRUE(column->has_equal_offsets(*cloned_arr)); diff --git a/be/test/vec/columns/common_column_test.h b/be/test/vec/columns/common_column_test.h index 9c6931856468de..9e941611d5cb40 100644 --- a/be/test/vec/columns/common_column_test.h +++ b/be/test/vec/columns/common_column_test.h @@ -412,8 +412,8 @@ class CommonColumnTest : public ::testing::Test { // insert_range_from now we have no any exception error data to handle, so here will meet crash continue; } else if (*pos + *cl > source_column->size()) { - if (check_column( - remove_nullable(source_column->assume_mutable()))) { + if (is_column( + remove_nullable(source_column->assume_mutable()).get())) { // insert_range_from in array has DCHECK_LG continue; } From 7ccb484cf1d22e10cac68826f84e3afe7d49142b Mon Sep 17 00:00:00 2001 From: amorynan Date: Mon, 30 Dec 2024 11:55:07 +0800 Subject: [PATCH 06/23] fix comment for column array and improve replicate function --- be/src/vec/columns/column_array.cpp | 79 +++- be/test/vec/columns/column_array_test.cpp | 348 ++++++++++++++++- be/test/vec/columns/common_column_test.h | 18 +- be/test/vec/core/column_array_test.cpp | 73 ---- .../data_types/common_data_type_serder_test.h | 357 ------------------ .../vec/data_types/common_data_type_test.h | 225 ----------- be/test/vec/function/function_test_util.cpp | 51 +++ 7 files changed, 457 insertions(+), 694 deletions(-) delete mode 100644 be/test/vec/data_types/common_data_type_serder_test.h delete mode 100644 be/test/vec/data_types/common_data_type_test.h diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp index 1a3f0ff3e58749..e143cc252c6c27 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -795,27 +795,74 @@ void ColumnArray::insert_many_from(const IColumn& src, size_t position, size_t l } ColumnPtr ColumnArray::replicate(const IColumn::Offsets& replicate_offsets) const { - if (replicate_offsets.empty()) return clone_empty(); + if (replicate_offsets.empty()) { + return clone_empty(); + } // keep ColumnUInt8 for ColumnNullable::null_map - if (typeid_cast(data.get())) + if (typeid_cast(data.get())) { return replicate_number(replicate_offsets); - if (typeid_cast(data.get())) + } + if (typeid_cast(data.get())) { return replicate_number(replicate_offsets); - if (typeid_cast(data.get())) + } + if (typeid_cast(data.get())) { + return replicate_number(replicate_offsets); + } + if (typeid_cast(data.get())) { return replicate_number(replicate_offsets); - if (typeid_cast(data.get())) + } + if (typeid_cast(data.get())) { + return replicate_number(replicate_offsets); + } + if (typeid_cast(data.get())) { return replicate_number(replicate_offsets); - if (typeid_cast(data.get())) + } + if (typeid_cast(data.get())) { + return replicate_number(replicate_offsets); + } + if (typeid_cast(data.get())) { return replicate_number(replicate_offsets); - if (typeid_cast(data.get())) + } + if (typeid_cast(data.get())) { + return replicate_number(replicate_offsets); + } + if (typeid_cast(data.get())) { + return replicate_number(replicate_offsets); + } + if (typeid_cast(data.get())) { + return replicate_number(replicate_offsets); + } + if (typeid_cast(data.get())) { return replicate_number(replicate_offsets); - if (typeid_cast(data.get())) + } + if (typeid_cast(data.get())) { return replicate_number(replicate_offsets); - if (typeid_cast(data.get())) return replicate_string(replicate_offsets); - if (typeid_cast(data.get())) return replicate_const(replicate_offsets); - if (typeid_cast(data.get())) + } + if (typeid_cast(data.get())) { + return replicate_number(replicate_offsets); + } + if (typeid_cast(data.get())) { + return replicate_number(replicate_offsets); + } + if (typeid_cast(data.get())) { + return replicate_number(replicate_offsets); + } + if (typeid_cast(data.get())) { + return replicate_number(replicate_offsets); + } + if (typeid_cast(data.get())) { + return replicate_number(replicate_offsets); + } + if (typeid_cast(data.get())) { + return replicate_string(replicate_offsets); + } + if (typeid_cast(data.get())) { + return replicate_const(replicate_offsets); + } + if (typeid_cast(data.get())) { return replicate_nullable(replicate_offsets); + } return replicate_generic(replicate_offsets); } @@ -832,12 +879,12 @@ ColumnPtr ColumnArray::replicate_number(const IColumn::Offsets& replicate_offset auto& res_arr = assert_cast(*res); - const typename ColumnVector::Container& src_data = - assert_cast&>(*data).get_data(); + const typename ColumnVectorOrDecimal::Container& src_data = + assert_cast&>(*data).get_data(); const auto& src_offsets = get_offsets(); - typename ColumnVector::Container& res_data = - assert_cast&>(res_arr.get_data()).get_data(); + typename ColumnVectorOrDecimal::Container& res_data = + assert_cast&>(res_arr.get_data()).get_data(); auto& res_offsets = res_arr.get_offsets(); res_data.reserve(data->size() / col_size * replicate_offsets.back()); @@ -991,6 +1038,8 @@ ColumnPtr ColumnArray::replicate_generic(const IColumn::Offsets& replicate_offse size_t size_to_replicate = replicate_offsets[i] - prev_offset; prev_offset = replicate_offsets[i]; + std::cout << "replicate_generic: " << i << " " << i << " with prev:" << prev_offset + << " with size_to_replicate: " << size_to_replicate << std::endl; for (size_t j = 0; j < size_to_replicate; ++j) { res_concrete.insert_from(*this, i); } diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp index 847eba877b3624..c47e51148e1242 100644 --- a/be/test/vec/columns/column_array_test.cpp +++ b/be/test/vec/columns/column_array_test.cpp @@ -72,11 +72,90 @@ class ColumnArrayTest : public CommonColumnTest { BaseInputTypeSet array_decimal128 = {TypeIndex::Array, TypeIndex::Decimal128V3}; // array UT BaseInputTypeSet array_decimal256 = {TypeIndex::Array, TypeIndex::Decimal256}; + // array> + BaseInputTypeSet array_array_tinyint = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::Int8}; + // array> + BaseInputTypeSet array_array_smallint = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::Int16}; + // array> + BaseInputTypeSet array_array_int = {TypeIndex::Array, TypeIndex::Array, TypeIndex::Int32}; + // array> + BaseInputTypeSet array_array_bigint = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::Int64}; + // array> + BaseInputTypeSet array_array_largeint = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::Int128}; + // array> + BaseInputTypeSet array_array_float = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::Float32}; + // array> + BaseInputTypeSet array_array_double = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::Float64}; + // array> + BaseInputTypeSet array_array_ipv4 = {TypeIndex::Array, TypeIndex::Array, TypeIndex::IPv4}; + // array> + BaseInputTypeSet array_array_ipv6 = {TypeIndex::Array, TypeIndex::Array, TypeIndex::IPv6}; + // array> + BaseInputTypeSet array_array_date = {TypeIndex::Array, TypeIndex::Array, TypeIndex::Date}; + // array> + BaseInputTypeSet array_array_datetime = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::DateTime}; + // array> + BaseInputTypeSet array_array_datev2 = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::DateV2}; + // array> + BaseInputTypeSet array_array_datetimev2 = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::DateTimeV2}; + // array> + BaseInputTypeSet array_array_varchar = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::String}; + // array> UT + BaseInputTypeSet array_array_decimal = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::Decimal32}; + // array> UT + BaseInputTypeSet array_array_decimal64 = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::Decimal64}; + // array> UT + BaseInputTypeSet array_array_decimal128 = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::Decimal128V3}; + // array> UT + BaseInputTypeSet array_array_decimal256 = {TypeIndex::Array, TypeIndex::Array, + TypeIndex::Decimal256}; + // array> + BaseInputTypeSet array_map_char_double = {TypeIndex::Array, TypeIndex::Map, + TypeIndex::String, TypeIndex::Float64}; + // test_array_map>.csv + BaseInputTypeSet array_map_datetime_decimal = {TypeIndex::Array, TypeIndex::Map, + TypeIndex::DateTime, TypeIndex::Decimal256}; + // test_array_map.csv + BaseInputTypeSet array_map_ipv4_ipv6 = {TypeIndex::Array, TypeIndex::Map, TypeIndex::IPv4, + TypeIndex::IPv6}; + // test_array_map.csv + BaseInputTypeSet array_map_largeint_string = {TypeIndex::Array, TypeIndex::Map, + TypeIndex::Int128, TypeIndex::String}; + // array> + BaseInputTypeSet array_struct = { + TypeIndex::Array, TypeIndex::Struct, TypeIndex::Int32, + TypeIndex::Date, TypeIndex::Decimal32, TypeIndex::String, + TypeIndex::Float64, TypeIndex::IPv4, TypeIndex::IPv6}; + std::vector array_typeIndex = { array_tinyint, array_smallint, array_int, array_bigint, array_largeint, array_float, array_double, array_ipv4, array_ipv6, array_date, array_datetime, array_datev2, array_datetimev2, array_varchar, array_decimal, array_decimal64, array_decimal128, array_decimal256}; + std::vector array_array_typeIndex = { + array_array_tinyint, array_array_smallint, array_array_int, + array_array_bigint, array_array_largeint, array_array_float, + array_array_double, array_array_ipv4, array_array_ipv6, + array_array_date, array_array_datetime, array_array_datev2, + array_array_datetimev2, array_array_varchar, array_array_decimal, + array_array_decimal64, array_array_decimal128, array_array_decimal256}; + std::vector array_map_typeIndex = { + array_map_char_double, array_map_datetime_decimal, array_map_ipv4_ipv6, + array_map_largeint_string}; + std::vector array_struct_typeIndex = {array_struct}; vector descs; descs.reserve(array_typeIndex.size()); @@ -89,6 +168,60 @@ class ColumnArrayTest : public CommonColumnTest { EXPECT_TRUE(parse_ut_data_type(input_types, descs[i])); } + for (int i = 0; i < array_array_typeIndex.size(); i++) { + descs.push_back(ut_type::UTDataTypeDescs()); + InputTypeSet input_types {}; + input_types.push_back(array_array_typeIndex[i][0]); + input_types.push_back(Nullable {static_cast(array_array_typeIndex[i][1])}); + input_types.push_back(Nullable {static_cast(array_array_typeIndex[i][2])}); + EXPECT_EQ(input_types[1].type(), &typeid(Nullable)) << "nested type is not nullable"; + EXPECT_EQ(input_types[2].type(), &typeid(Nullable)) << "nested type is not nullable"; + EXPECT_TRUE(parse_ut_data_type(input_types, descs[i + array_typeIndex.size()])); + } + + for (int i = 0; i < array_map_typeIndex.size(); i++) { + descs.push_back(ut_type::UTDataTypeDescs()); + InputTypeSet input_types {}; + input_types.push_back(array_map_typeIndex[i][0]); // array + input_types.push_back( + Nullable {static_cast(array_map_typeIndex[i][1])}); // map + input_types.push_back( + Nullable {static_cast(array_map_typeIndex[i][2])}); // key + input_types.push_back( + Nullable {static_cast(array_map_typeIndex[i][3])}); // val + EXPECT_EQ(input_types[1].type(), &typeid(Nullable)) << "nested type is not nullable"; + EXPECT_EQ(input_types[2].type(), &typeid(Nullable)) << "nested type is not nullable"; + EXPECT_TRUE(parse_ut_data_type( + input_types, descs[i + array_typeIndex.size() + array_array_typeIndex.size()])); + } + + for (int i = 0; i < array_struct_typeIndex.size(); i++) { + descs.push_back(ut_type::UTDataTypeDescs()); + InputTypeSet input_types {}; + input_types.push_back(array_struct_typeIndex[i][0]); // arr + input_types.push_back( + Nullable {static_cast(array_struct_typeIndex[i][1])}); // struct + input_types.push_back( + Nullable {static_cast(array_struct_typeIndex[i][2])}); // f1 + input_types.push_back( + Nullable {static_cast(array_struct_typeIndex[i][3])}); // f2 + input_types.push_back( + Nullable {static_cast(array_struct_typeIndex[i][4])}); // f3 + input_types.push_back( + Nullable {static_cast(array_struct_typeIndex[i][5])}); // f4 + input_types.push_back( + Nullable {static_cast(array_struct_typeIndex[i][6])}); // f5 + input_types.push_back( + Nullable {static_cast(array_struct_typeIndex[i][7])}); // f6 + input_types.push_back( + Nullable {static_cast(array_struct_typeIndex[i][8])}); // f7 + + EXPECT_EQ(input_types[1].type(), &typeid(Nullable)) << "nested type is not nullable"; + EXPECT_TRUE(parse_ut_data_type( + input_types, descs[i + array_typeIndex.size() + array_array_typeIndex.size() + + array_map_typeIndex.size()])); + } + // create column_array for each data type vector data_files = {data_file_dir + "test_array_tinyint.csv", data_file_dir + "test_array_smallint.csv", @@ -109,6 +242,34 @@ class ColumnArrayTest : public CommonColumnTest { data_file_dir + "test_array_decimalv3(38,30).csv", data_file_dir + "test_array_decimalv3(76,56).csv"}; + data_files.insert(data_files.end(), + {data_file_dir + "test_array_array_tinyint.csv", + data_file_dir + "test_array_array_smallint.csv", + data_file_dir + "test_array_array_int.csv", + data_file_dir + "test_array_array_bigint.csv", + data_file_dir + "test_array_array_largeint.csv", + data_file_dir + "test_array_array_float.csv", + data_file_dir + "test_array_array_double.csv", + data_file_dir + "test_array_array_ipv4.csv", + data_file_dir + "test_array_array_ipv6.csv", + data_file_dir + "test_array_array_date.csv", + data_file_dir + "test_array_array_datetime.csv", + data_file_dir + "test_array_array_date.csv", + data_file_dir + "test_array_array_datetimev2(5).csv", + data_file_dir + "test_array_array_varchar(65535).csv", + data_file_dir + "test_array_array_decimalv3(1,0).csv", + data_file_dir + "test_array_array_decimalv3(27,9).csv", + data_file_dir + "test_array_array_decimalv3(38,30).csv", + data_file_dir + "test_array_array_decimalv3(76,56).csv"}); + + data_files.insert(data_files.end(), + {data_file_dir + "test_array_map.csv", + data_file_dir + "test_array_map>.csv", + data_file_dir + "test_array_map.csv", + data_file_dir + "test_array_map.csv"}); + + data_files.insert(data_files.end(), {data_file_dir + "test_array_struct.csv"}); + // step2. according to the datatype to make column_array // && load data from csv file into column_array EXPECT_EQ(descs.size(), data_files.size()); @@ -126,6 +287,54 @@ class ColumnArrayTest : public CommonColumnTest { array_types.push_back(type); serdes.push_back(serde); } + + for (int i = 0; i < array_array_typeIndex.size(); i++) { + auto& desc = descs[i + array_typeIndex.size()]; + auto& data_file = data_files[i + array_typeIndex.size()]; + // first is array type + auto& type = desc[0].data_type; + std::cout << "type: " << type->get_name() << " with file: " << data_file << std::endl; + MutableColumns columns; + columns.push_back(type->create_column()); + auto serde = type->get_serde(1); + load_data_from_csv({serde}, columns, data_file, ';'); + array_columns.push_back(std::move(columns[0])); + array_types.push_back(type); + serdes.push_back(serde); + } + + for (int i = 0; i < array_map_typeIndex.size(); i++) { + auto& desc = descs[i + array_typeIndex.size() + array_array_typeIndex.size()]; + auto& data_file = data_files[i + array_typeIndex.size() + array_array_typeIndex.size()]; + // first is array type + auto& type = desc[0].data_type; + std::cout << "type: " << type->get_name() << " with file: " << data_file << std::endl; + MutableColumns columns; + columns.push_back(type->create_column()); + auto serde = type->get_serde(1); + load_data_from_csv({serde}, columns, data_file, ';'); + array_columns.push_back(std::move(columns[0])); + array_types.push_back(type); + serdes.push_back(serde); + } + + for (int i = 0; i < array_struct_typeIndex.size(); i++) { + auto& desc = descs[i + array_typeIndex.size() + array_array_typeIndex.size() + + array_map_typeIndex.size()]; + auto& data_file = data_files[i + array_typeIndex.size() + array_array_typeIndex.size() + + array_map_typeIndex.size()]; + // first is array type + auto& type = desc[0].data_type; + std::cout << "type: " << type->get_name() << " with file: " << data_file << std::endl; + MutableColumns columns; + columns.push_back(type->create_column()); + auto serde = type->get_serde(1); + load_data_from_csv({serde}, columns, data_file, ';'); + array_columns.push_back(std::move(columns[0])); + array_types.push_back(type); + serdes.push_back(serde); + } + // step3. show array column data for (int i = 0; i < array_columns.size(); i++) { // auto& column = array_columns[i]; @@ -160,7 +369,8 @@ TEST_F(ColumnArrayTest, InsertManyDefaultsTest) { } TEST_F(ColumnArrayTest, GetDataAtTest) { - assert_get_data_at_callback(array_columns, serdes); + // get_data_at is not support in column_array + EXPECT_ANY_THROW(assert_get_data_at_callback(array_columns, serdes)); } TEST_F(ColumnArrayTest, FieldTest) { @@ -180,7 +390,8 @@ TEST_F(ColumnArrayTest, GetIntTest) { } TEST_F(ColumnArrayTest, SerDeVecTest) { - ser_deser_vec(array_columns, array_types); + // get_max_row_byte_size is not support in column_array + EXPECT_ANY_THROW(ser_deser_vec(array_columns, array_types)); } TEST_F(ColumnArrayTest, serDeserializeWithArenaImpl) { @@ -220,11 +431,25 @@ TEST_F(ColumnArrayTest, ReserveTest) { } TEST_F(ColumnArrayTest, ReplicateTest) { - assert_replicate_callback(array_columns, serdes); + // array_array_char will cause exception in replicate with: string column length is too large: total_length=4295103210, element_number=327295 + // so we need to skip it + MutableColumns array_columns_copy; + DataTypeSerDeSPtrs serdes_copy; + // just skip array_array_char use vector copy + for (int i = 0; i < array_columns.size(); i++) { + if (i == 31) { + std::cout << array_columns[i]->get_name() << " is skipped" << std::endl; + continue; + } + array_columns_copy.push_back(array_columns[i]->assume_mutable()); + serdes_copy.push_back(serdes[i]); + } + assert_replicate_callback(array_columns_copy, serdes_copy); } TEST_F(ColumnArrayTest, ReplaceColumnTest) { - assert_replace_column_data_callback(array_columns, serdes); + // replace_column_data is not support in column_array, only support non-variable length column + EXPECT_ANY_THROW(assert_replace_column_data_callback(array_columns, serdes)); assert_replace_column_null_data_callback(array_columns, serdes); } @@ -238,8 +463,8 @@ TEST_F(ColumnArrayTest, PermutationAndSortTest) { auto& type = array_types[i]; auto column_type = type->get_name(); std::cout << "column_type: " << column_type << std::endl; - // permutation - assert_column_permutations(column->assume_mutable_ref(), type); + // permutation get_permutation is not support in column_array, compare_at maybe not incorrect + EXPECT_ANY_THROW(assert_column_permutations(column->assume_mutable_ref(), type)); } } @@ -316,14 +541,17 @@ TEST_F(ColumnArrayTest, ConvertIfOverflowAndInsertTest) { // test convert_column_if_overflow && insert_range_from_ignore_overflow for (int i = 0; i < array_columns.size(); i++) { auto& column = array_columns[i]; - DataTypeArray type = array_types[i]; - if (!is_string(type.get_nested_type())) { + auto type = array_types[i]; + auto nested_type = + assert_cast(remove_nullable(type).get())->get_nested_type(); + if (!is_string(nested_type)) { // check ptr is itself auto ptr = column->convert_column_if_overflow(); EXPECT_EQ(ptr.get(), column.get()); - auto* array_col = check_and_get_column(column.get()); - auto nested_col = array_col->get_data_ptr(); - auto* array_col1 = check_and_get_column(ptr.get()); + auto arr_col = check_and_get_column( + remove_nullable(column->assume_mutable()).get()); + auto nested_col = arr_col->get_data_ptr(); + auto array_col1 = check_and_get_column(remove_nullable(ptr).get()); auto nested_col1 = array_col1->get_data_ptr(); EXPECT_EQ(nested_col.get(), nested_col1.get()); } else { @@ -422,4 +650,102 @@ TEST_F(ColumnArrayTest, HasEqualOffsetsTest) { } } +TEST_F(ColumnArrayTest, String64ArrayTest) { + auto off_column = ColumnVector::create(); + auto str64_column = ColumnString64::create(); + // init column array with [["abc","d"],["ef"],[], [""]]; + std::vector offs = {0, 2, 3, 3, 4}; + std::vector vals = {"abc", "d", "ef", ""}; + for (size_t i = 1; i < offs.size(); ++i) { + off_column->insert_data((const char*)(&offs[i]), 0); + } + for (auto& v : vals) { + str64_column->insert_data(v.data(), v.size()); + } + auto str64_array_column = ColumnArray::create(std::move(str64_column), std::move(off_column)); + EXPECT_EQ(str64_array_column->size(), offs.size() - 1); + for (size_t i = 0; i < str64_array_column->size(); ++i) { + auto v = get(str64_array_column->operator[](i)); + EXPECT_EQ(v.size(), offs[i + 1] - offs[i]); + for (size_t j = 0; j < v.size(); ++j) { + EXPECT_EQ(vals[offs[i] + j], get(v[j])); + } + } + // test insert ColumnArray> into ColumnArray> + auto str32_column = ColumnString::create(); + auto str32_array_column = ColumnArray::create(std::move(str32_column)); + std::vector indices; + indices.push_back(0); + indices.push_back(1); + indices.push_back(3); + str32_array_column->insert_indices_from(*str64_array_column, indices.data(), + indices.data() + indices.size()); + EXPECT_EQ(str32_array_column->size(), 3); + + auto v = get(str32_array_column->operator[](0)); + EXPECT_EQ(v.size(), 2); + EXPECT_EQ(get(v[0]), vals[0]); + EXPECT_EQ(get(v[1]), vals[1]); + + v = get(str32_array_column->operator[](1)); + EXPECT_EQ(v.size(), 1); + EXPECT_EQ(get(v[0]), vals[2]); + + v = get(str32_array_column->operator[](2)); + EXPECT_EQ(v.size(), 1); + EXPECT_EQ(get(v[0]), vals[3]); +} + +TEST_F(ColumnArrayTest, IntArrayPermuteTest) { + auto off_column = ColumnVector::create(); + auto data_column = ColumnVector::create(); + // init column array with [[1,2,3],[],[4],[5,6]] + std::vector offs = {0, 3, 3, 4, 6}; + std::vector vals = {1, 2, 3, 4, 5, 6}; + for (size_t i = 1; i < offs.size(); ++i) { + off_column->insert_data((const char*)(&offs[i]), 0); + } + for (auto& v : vals) { + data_column->insert_data((const char*)(&v), 0); + } + auto array_column = ColumnArray::create(std::move(data_column), std::move(off_column)); + + IColumn::Permutation perm = {3, 2, 1, 0}; + // return array column: [[5,6],[4]]; + auto res1 = array_column->permute(perm, 2); + // check offsets + IColumn::Offsets offs1 = {2, 3}; + auto arr_col = check_and_get_column(*res1); + ASSERT_EQ(arr_col->size(), offs1.size()); + for (size_t i = 0; i < arr_col->size(); ++i) { + ASSERT_EQ(arr_col->get_offsets()[i], offs1[i]); + } + // check data + std::vector data = {5, 6, 4}; + auto data_col = arr_col->get_data_ptr(); + ASSERT_EQ(data_col->size(), data.size()); + for (size_t i = 0; i < data_col->size(); ++i) { + auto element = data_col->get_data_at(i); + ASSERT_EQ(*((int32_t*)element.data), data[i]); + } + + // return array column: [[5,6],[4],[],[1,2,3]] + auto res2 = array_column->permute(perm, 0); + // check offsets + IColumn::Offsets offs2 = {2, 3, 3, 6}; + arr_col = check_and_get_column(*res2); + ASSERT_EQ(arr_col->size(), offs2.size()); + for (size_t i = 0; i < arr_col->size(); ++i) { + ASSERT_EQ(arr_col->get_offsets()[i], offs2[i]); + } + // check data + std::vector data2 = {5, 6, 4, 1, 2, 3}; + data_col = arr_col->get_data_ptr(); + ASSERT_EQ(data_col->size(), data2.size()); + for (size_t i = 0; i < data_col->size(); ++i) { + auto element = data_col->get_data_at(i); + ASSERT_EQ(*((int32_t*)element.data), data2[i]); + } +} + } // namespace doris::vectorized \ No newline at end of file diff --git a/be/test/vec/columns/common_column_test.h b/be/test/vec/columns/common_column_test.h index 9e941611d5cb40..db67301e86a1ee 100644 --- a/be/test/vec/columns/common_column_test.h +++ b/be/test/vec/columns/common_column_test.h @@ -504,7 +504,6 @@ class CommonColumnTest : public ::testing::Test { } buffer_writer.commit(); std::string actual_str_value = ser_col->get_data_at(j).to_string(); - std::cout << "actual_str_value: " << actual_str_value << std::endl; data.push_back(actual_str_value); } res.push_back(data); @@ -729,7 +728,6 @@ class CommonColumnTest : public ::testing::Test { } buffer_writer.commit(); std::string actual_str_value = ser_col->get_data_at(j).to_string(); - std::cout << "actual_str_value: " << actual_str_value << std::endl; data.push_back(actual_str_value); } res.push_back(data); @@ -760,7 +758,6 @@ class CommonColumnTest : public ::testing::Test { } buffer_writer.commit(); std::string actual_str_value = ser_col->get_data_at(j).to_string(); - std::cout << "actual_str_value: " << actual_str_value << std::endl; data.push_back(actual_str_value); } res2.push_back(data); @@ -809,7 +806,6 @@ class CommonColumnTest : public ::testing::Test { } buffer_writer.commit(); std::string actual_str_value = ser_col->get_data_at(j).to_string(); - std::cout << "actual_str_value: " << actual_str_value << std::endl; data.push_back(actual_str_value); } res.push_back(data); @@ -1248,7 +1244,12 @@ class CommonColumnTest : public ::testing::Test { << " for column size : " << source_column->size() << std::endl; // auto ptr = const_col->convert_to_full_column(); // here will return different ptr + // record replicate cost time + auto start = std::chrono::high_resolution_clock::now(); auto ptr = source_column->replicate(offsets); + auto end = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(end - start); + std::cout << "replicate cost time: " << duration.count() << "ms" << std::endl; // check ptr EXPECT_NE(ptr.get(), source_column.get()); // check after replicate with assert_res @@ -1940,18 +1941,9 @@ class CommonColumnTest : public ::testing::Test { // step1. get expect permutation as stabled sort stable_get_column_permutation(column, ascending, limit, nan_direction_hint, expected_permutation); - std::cout << "expected_permutation size: " << expected_permutation.size() << std::endl; - for (size_t i = 0; i < expected_permutation.size(); i++) { - std::cout << "expected_permutation: " << expected_permutation[i] << std::endl; - } // step2. get permutation by column column.get_permutation(!ascending, limit, nan_direction_hint, actual_permutation); - std::cout << "actual_permutation size: " << actual_permutation.size() << std::endl; - for (size_t i = 0; i < actual_permutation.size(); i++) { - std::cout << "actual_permutation: " << actual_permutation[i] << std::endl; - } - if (limit == 0) { limit = actual_permutation.size(); } diff --git a/be/test/vec/core/column_array_test.cpp b/be/test/vec/core/column_array_test.cpp index 26d1d3267f1060..147260639eaf17 100644 --- a/be/test/vec/core/column_array_test.cpp +++ b/be/test/vec/core/column_array_test.cpp @@ -108,79 +108,6 @@ TEST(ColumnArrayOldTest, StringArrayTest) { } } -TEST(ColumnArrayTest, String64ArrayTest) { - auto off_column = ColumnVector::create(); - auto str64_column = ColumnString64::create(); - // init column array with [["abc","d"],["ef"],[], [""]]; - std::vector offs = {0, 2, 3, 3, 4}; - std::vector vals = {"abc", "d", "ef", ""}; - for (size_t i = 1; i < offs.size(); ++i) { - off_column->insert_data((const char*)(&offs[i]), 0); - } - for (auto& v : vals) { - str64_column->insert_data(v.data(), v.size()); - } - - ColumnArray str64_array_column(std::move(str64_column), std::move(off_column)); - EXPECT_EQ(str64_array_column.size(), offs.size() - 1); - for (size_t i = 0; i < str64_array_column.size(); ++i) { - auto v = get(str64_array_column[i]); - EXPECT_EQ(v.size(), offs[i + 1] - offs[i]); - for (size_t j = 0; j < v.size(); ++j) { - EXPECT_EQ(vals[offs[i] + j], get(v[j])); - } - } - // test insert ColumnArray> into ColumnArray> - auto str32_column = ColumnString::create(); - ColumnArray str32_array_column(std::move(str32_column)); - std::vector indices; - indices.push_back(0); - indices.push_back(1); - indices.push_back(3); - str32_array_column.insert_indices_from(str64_array_column, indices.data(), - indices.data() + indices.size()); - EXPECT_EQ(str32_array_column.size(), 3); - - auto v = get(str32_array_column[0]); - EXPECT_EQ(v.size(), 2); - EXPECT_EQ(get(v[0]), vals[0]); - EXPECT_EQ(get(v[1]), vals[1]); - - v = get(str32_array_column[1]); - EXPECT_EQ(v.size(), 1); - EXPECT_EQ(get(v[0]), vals[2]); - - v = get(str32_array_column[2]); - EXPECT_EQ(v.size(), 1); - EXPECT_EQ(get(v[0]), vals[3]); -} - -TEST(ColumnArrayTest, IntArrayPermuteTest) { - auto off_column = ColumnVector::create(); - auto data_column = ColumnVector::create(); - // init column array with [[1,2,3],[],[4],[5,6]] - std::vector offs = {0, 3, 3, 4, 6}; - std::vector vals = {1, 2, 3, 4, 5, 6}; - for (size_t i = 1; i < offs.size(); ++i) { - off_column->insert_data((const char*)(&offs[i]), 0); - } - for (auto& v : vals) { - data_column->insert_data((const char*)(&v), 0); - } - ColumnArray array_column(std::move(data_column), std::move(off_column)); - - IColumn::Permutation perm = {3, 2, 1, 0}; - // return array column: [[5,6],[4]]; - auto res1 = array_column.permute(perm, 2); - check_array_offsets(*res1, {2, 3}); - check_array_data(*res1, {5, 6, 4}); - - // return array column: [[5,6],[4],[],[1,2,3]] - auto res2 = array_column.permute(perm, 0); - check_array_offsets(*res2, {2, 3, 3, 6}); - check_array_data(*res2, {5, 6, 4, 1, 2, 3}); -} - TEST(ColumnArrayOldTest, StringArrayPermuteTest) { auto off_column = ColumnVector::create(); auto data_column = ColumnString::create(); diff --git a/be/test/vec/data_types/common_data_type_serder_test.h b/be/test/vec/data_types/common_data_type_serder_test.h deleted file mode 100644 index a900439e5c0340..00000000000000 --- a/be/test/vec/data_types/common_data_type_serder_test.h +++ /dev/null @@ -1,357 +0,0 @@ -// 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 -#include -#include -#include - -#include -#include -#include - -#include "runtime/descriptors.h" -#include "util/arrow/block_convertor.h" -#include "util/arrow/row_batch.h" -#include "vec/columns/column.h" -#include "vec/core/field.h" -#include "vec/core/types.h" -#include "vec/data_types/data_type.h" -#include "vec/utils/arrow_column_to_doris_column.h" - -// this test is gonna to be a data type serialize and deserialize functions -// such as -// 1. standard hive text ser-deserialize -// deserialize_one_cell_from_hive_text (IColumn &column, Slice &slice, const FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const -// deserialize_column_from_hive_text_vector (IColumn &column, std::vector< Slice > &slices, int *num_deserialized, const FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const -// serialize_one_cell_to_hive_text (const IColumn &column, int row_num, BufferWritable &bw, FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const -// 2. json format ser-deserialize which used in table not in doris database -// serialize_one_cell_to_json (const IColumn &column, int row_num, BufferWritable &bw, FormatOptions &options) const =0 -// serialize_column_to_json (const IColumn &column, int start_idx, int end_idx, BufferWritable &bw, FormatOptions &options) const =0 -// deserialize_one_cell_from_json (IColumn &column, Slice &slice, const FormatOptions &options) const =0 -// deserialize_column_from_json_vector (IColumn &column, std::vector< Slice > &slices, int *num_deserialized, const FormatOptions &options) const =0 -// deserialize_column_from_fixed_json (IColumn &column, Slice &slice, int rows, int *num_deserialized, const FormatOptions &options) const -// insert_column_last_value_multiple_times (IColumn &column, int times) const -// 3. fe|be protobuffer ser-deserialize -// write_column_to_pb (const IColumn &column, PValues &result, int start, int end) const =0 -// read_column_from_pb (IColumn &column, const PValues &arg) const =0 -// 4. jsonb ser-deserialize which used in row-store situation -// write_one_cell_to_jsonb (const IColumn &column, JsonbWriter &result, Arena *mem_pool, int32_t col_id, int row_num) const =0 -// read_one_cell_from_jsonb (IColumn &column, const JsonbValue *arg) const =0 -// 5. mysql text ser-deserialize -// write_column_to_mysql (const IColumn &column, MysqlRowBuffer< false > &row_buffer, int row_idx, bool col_const, const FormatOptions &options) const =0 -// write_column_to_mysql (const IColumn &column, MysqlRowBuffer< true > &row_buffer, int row_idx, bool col_const, const FormatOptions &options) const =0 -// 6. arrow ser-deserialize which used in spark-flink connector -// write_column_to_arrow (const IColumn &column, const NullMap *null_map, arrow::ArrayBuilder *array_builder, int start, int end, const cctz::time_zone &ctz) const =0 -// read_column_from_arrow (IColumn &column, const arrow::Array *arrow_array, int start, int end, const cctz::time_zone &ctz) const =0 -// 7. rapidjson ser-deserialize -// write_one_cell_to_json (const IColumn &column, rapidjson::Value &result, rapidjson::Document::AllocatorType &allocator, Arena &mem_pool, int row_num) const -// read_one_cell_from_json (IColumn &column, const rapidjson::Value &result) const -// convert_field_to_rapidjson (const vectorized::Field &field, rapidjson::Value &target, rapidjson::Document::AllocatorType &allocator) -// convert_array_to_rapidjson (const vectorized::Array &array, rapidjson::Value &target, rapidjson::Document::AllocatorType &allocator) - -namespace doris::vectorized { - -class CommonDataTypeSerdeTest : public ::testing::Test { -public: - ////================================================================================================================== - // this is common function to check data in column against expected results according different function in assert function - // which can be used in all column test - // such as run regress tests - // step1. we can set gen_check_data_in_assert to true, then we will generate a file for check data, otherwise we will read the file to check data - // step2. we should write assert callback function to check data - static void check_data( - MutableColumns& columns, DataTypeSerDeSPtrs serders, char col_spliter, - std::set idxes, const std::string& column_data_file, - std::function - assert_callback, - bool is_hive_format = false, DataTypes dataTypes = {}) { - ASSERT_EQ(serders.size(), columns.size()); - // Step 1: Insert data from `column_data_file` into the column and check result with `check_data_file` - // Load column data and expected data from CSV files - std::vector> res; - struct stat buff; - if (stat(column_data_file.c_str(), &buff) == 0) { - if (S_ISREG(buff.st_mode)) { - // file - if (is_hive_format) { - load_data_and_assert_from_csv(serders, columns, column_data_file, - col_spliter, idxes); - } else { - load_data_and_assert_from_csv(serders, columns, column_data_file, - col_spliter, idxes); - } - } else if (S_ISDIR(buff.st_mode)) { - // dir - std::filesystem::path fs_path(column_data_file); - for (const auto& entry : std::filesystem::directory_iterator(fs_path)) { - std::string file_path = entry.path().string(); - std::cout << "load data from file: " << file_path << std::endl; - if (is_hive_format) { - load_data_and_assert_from_csv(serders, columns, file_path, - col_spliter, idxes); - } else { - load_data_and_assert_from_csv(serders, columns, file_path, - col_spliter, idxes); - } - } - } - } - - // Step 2: Validate the data in `column` matches `expected_data` - assert_callback(columns, serders); - } - - // Helper function to load data from CSV, with index which splited by spliter and load to columns - template - static void load_data_and_assert_from_csv(const DataTypeSerDeSPtrs serders, - MutableColumns& columns, const std::string& file_path, - const char spliter = ';', - const std::set idxes = {0}) { - ASSERT_EQ(serders.size(), columns.size()) - << "serder size: " << serders.size() << " column size: " << columns.size(); - ASSERT_EQ(serders.size(), idxes.size()) - << "serder size: " << serders.size() << " idxes size: " << idxes.size(); - std::ifstream file(file_path); - if (!file) { - throw doris::Exception(ErrorCode::INVALID_ARGUMENT, "can not open the file: {} ", - file_path); - } - - std::string line; - DataTypeSerDe::FormatOptions options; - std::vector> res; - MutableColumns assert_str_cols(columns.size()); - for (size_t i = 0; i < columns.size(); ++i) { - assert_str_cols[i] = ColumnString::create(); - } - - while (std::getline(file, line)) { - std::stringstream lineStream(line); - // std::cout << "whole : " << lineStream.str() << std::endl; - std::string value; - int l_idx = 0; - int c_idx = 0; - std::vector row; - while (std::getline(lineStream, value, spliter)) { - if (idxes.contains(l_idx)) { - // load csv data - Slice string_slice(value.data(), value.size()); - std::cout << "origin : " << string_slice << std::endl; - Status st; - // deserialize data - if constexpr (is_hive_format) { - st = serders[c_idx]->deserialize_one_cell_from_hive_text( - *columns[c_idx], string_slice, options); - } else { - st = serders[c_idx]->deserialize_one_cell_from_json(*columns[c_idx], - string_slice, options); - } - if (!st.ok()) { - // deserialize if happen error now we do not insert any value for input column - // so we push a default value to column for row alignment - columns[c_idx]->insert_default(); - std::cout << "error in deserialize but continue: " << st.to_string() - << std::endl; - } - // serialize data - size_t row_num = columns[c_idx]->size() - 1; - assert_str_cols[c_idx]->reserve(columns[c_idx]->size()); - VectorBufferWriter bw(assert_cast(*assert_str_cols[c_idx])); - if constexpr (is_hive_format) { - st = serders[c_idx]->serialize_one_cell_to_hive_text(*columns[c_idx], - row_num, bw, options); - EXPECT_TRUE(st.ok()) << st.to_string(); - } else { - st = serders[c_idx]->serialize_one_cell_to_json(*columns[c_idx], row_num, - bw, options); - EXPECT_TRUE(st.ok()) << st.to_string(); - } - bw.commit(); - // assert data : origin data and serialized data should be equal or generated - // file to check data - size_t assert_size = assert_str_cols[c_idx]->size(); - if constexpr (!generate_res_file) { - EXPECT_EQ(assert_str_cols[c_idx]->get_data_at(assert_size - 1).to_string(), - string_slice.to_string()) - << "column: " << columns[c_idx]->get_name() << " row: " << row_num - << " is_hive_format: " << is_hive_format; - } - ++c_idx; - } - res.push_back(row); - ++l_idx; - } - } - - if (generate_res_file) { - // generate res - auto pos = file_path.find_last_of("."); - std::string hive_format = is_hive_format ? "_hive" : ""; - std::string res_file = file_path.substr(0, pos) + hive_format + "_serde_res.csv"; - std::ofstream res_f(res_file); - if (!res_f.is_open()) { - throw std::ios_base::failure("Failed to open file." + res_file); - } - for (size_t r = 0; r < assert_str_cols[0]->size(); ++r) { - for (size_t c = 0; c < assert_str_cols.size(); ++c) { - std::cout << assert_str_cols[c]->get_data_at(r).to_string() << spliter - << std::endl; - res_f << assert_str_cols[c]->get_data_at(r).to_string() << spliter; - } - res_f << std::endl; - } - res_f.close(); - std::cout << "generate res file: " << res_file << std::endl; - } - } - - // standard hive text ser-deserialize assert function - static void assert_pb_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& col = load_cols[i]; - std::cout << " now we are testing column : " << col->get_name() << std::endl; - // serialize to pb - PValues pv = PValues(); - Status st = serders[i]->write_column_to_pb(*col, pv, 0, col->size()); - if (!st.ok()) { - std::cerr << "write_column_to_pb error: " << st.msg() << std::endl; - continue; - } - // deserialize from pb - auto except_column = col->clone_empty(); - st = serders[i]->read_column_from_pb(*except_column, pv); - EXPECT_TRUE(st.ok()) << st.to_string(); - // check pb value from expected column - PValues as_pv = PValues(); - st = serders[i]->write_column_to_pb(*except_column, as_pv, 0, except_column->size()); - EXPECT_TRUE(st.ok()) << st.to_string(); - EXPECT_EQ(pv.bytes_value_size(), as_pv.bytes_value_size()); - // check column value - for (size_t j = 0; j < col->size(); ++j) { - auto cell = col->operator[](j); - auto except_cell = except_column->operator[](j); - EXPECT_EQ(cell, except_cell) << "column: " << col->get_name() << " row: " << j; - } - } - } - - // actually this is block_to_jsonb and jsonb_to_block test - static void assert_jsonb_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { - Arena pool; - auto jsonb_column = ColumnString::create(); // jsonb column - jsonb_column->reserve(load_cols[0]->size()); - MutableColumns assert_cols; - for (size_t i = 0; i < load_cols.size(); ++i) { - assert_cols.push_back(load_cols[i]->assume_mutable()); - } - for (size_t r = 0; r < load_cols[0]->size(); ++r) { - JsonbWriterT jw; - jw.writeStartObject(); - // serialize to jsonb - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& col = load_cols[i]; - serders[i]->write_one_cell_to_jsonb(*col, jw, &pool, i, r); - } - jw.writeEndObject(); - jsonb_column->insert_data(jw.getOutput()->getBuffer(), jw.getOutput()->getSize()); - } - // deserialize jsonb column to assert column - EXPECT_EQ(jsonb_column->size(), load_cols[0]->size()); - for (size_t r = 0; r < jsonb_column->size(); ++r) { - StringRef jsonb_data = jsonb_column->get_data_at(r); - auto pdoc = JsonbDocument::createDocument(jsonb_data.data, jsonb_data.size); - JsonbDocument& doc = *pdoc; - size_t cIdx = 0; - for (auto it = doc->begin(); it != doc->end(); ++it) { - serders[cIdx]->read_one_cell_from_jsonb(*assert_cols[cIdx], it->value()); - ++cIdx; - } - } - // check column value - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& col = load_cols[i]; - auto& assert_col = assert_cols[i]; - for (size_t j = 0; j < col->size(); ++j) { - auto cell = col->operator[](j); - auto assert_cell = assert_col->operator[](j); - EXPECT_EQ(cell, assert_cell) << "column: " << col->get_name() << " row: " << j; - } - } - } - - // assert mysql text format, now we just simple assert not to fatal or exception here - static void assert_mysql_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { - MysqlRowBuffer row_buffer; - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& col = load_cols[i]; - for (size_t j = 0; j < col->size(); ++j) { - Status st; - EXPECT_NO_FATAL_FAILURE( - st = serders[i]->write_column_to_mysql(*col, row_buffer, j, false, {})); - EXPECT_TRUE(st.ok()) << st.to_string(); - } - } - } - - // assert arrow serialize - static void assert_arrow_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders, - DataTypes types) { - // make a block to write to arrow - auto block = std::make_shared(); - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& col = load_cols[i]; - block->insert(ColumnWithTypeAndName(std::move(col), types[i], types[i]->get_name())); - } - // print block - std::cout << "block: " << block->dump_structure() << std::endl; - std::shared_ptr block_arrow_schema; - EXPECT_EQ(get_arrow_schema_from_block(*block, &block_arrow_schema, "UTC"), Status::OK()); - // convert block to arrow - std::shared_ptr result; - cctz::time_zone _timezone_obj; //default UTC - Status stt = convert_to_arrow_batch(*block, block_arrow_schema, - arrow::default_memory_pool(), &result, _timezone_obj); - EXPECT_EQ(Status::OK(), stt) << "convert block to arrow failed" << stt.to_string(); - - // deserialize arrow to block - auto assert_block = block->clone_empty(); - auto rows = block->rows(); - for (size_t i = 0; i < load_cols.size(); ++i) { - auto array = result->column(i); - auto& column_with_type_and_name = assert_block.get_by_position(i); - auto ret = arrow_column_to_doris_column( - array.get(), 0, column_with_type_and_name.column, - column_with_type_and_name.type, rows, _timezone_obj); - // do check data - EXPECT_EQ(Status::OK(), ret) << "convert arrow to block failed" << ret.to_string(); - auto& col = block->get_by_position(i).column; - auto& assert_col = column_with_type_and_name.column; - for (size_t j = 0; j < col->size(); ++j) { - auto cell = col->operator[](j); - auto assert_cell = assert_col->operator[](j); - EXPECT_EQ(cell, assert_cell) << "column: " << col->get_name() << " row: " << j; - } - } - } - - // assert rapidjson format - // now rapidjson write_one_cell_to_json and read_one_cell_from_json only used in column_object - // can just be replaced by jsonb format -}; - -} // namespace doris::vectorized \ No newline at end of file diff --git a/be/test/vec/data_types/common_data_type_test.h b/be/test/vec/data_types/common_data_type_test.h deleted file mode 100644 index 36abc3402e2156..00000000000000 --- a/be/test/vec/data_types/common_data_type_test.h +++ /dev/null @@ -1,225 +0,0 @@ -// 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 -#include -#include - -#include -#include -#include - -#include "olap/schema.h" -#include "vec/columns/column.h" -#include "vec/columns/column_array.h" -#include "vec/columns/column_map.h" -#include "vec/columns/columns_number.h" -#include "vec/core/field.h" -#include "vec/core/sort_block.h" -#include "vec/core/sort_description.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_map.h" - -// this test is gonna to be a data type test template for all DataType which should make ut test to coverage the function defined -// for example DataTypeIPv4 should test this function: -// 1. datatype meta info: -// get_type_id, get_type_as_type_descriptor, get_storage_field_type, have_subtypes, get_pdata_type (const IDataType *data_type), to_pb_column_meta (PColumnMeta *col_meta) -// get_family_name, get_is_parametric, should_align_right_in_pretty_formats -// text_can_contain_only_valid_utf8 -// have_maximum_size_of_value, get_maximum_size_of_value_in_memory, get_size_of_value_in_memory -// get_precision, get_scale -// get_field -// is_null_literal, is_value_represented_by_number, is_value_unambiguously_represented_in_contiguous_memory_region, is_value_unambiguously_represented_in_fixed_size_contiguous_memory_region -// 2. datatype creation with column: create_column, create_column_const (size_t size, const Field &field), create_column_const_with_default_value (size_t size), get_uncompressed_serialized_bytes (const IColumn &column, int be_exec_version) -// 3. serde related: get_serde (int nesting_level=1) -// to_string (const IColumn &column, size_t row_num, BufferWritable &ostr), to_string (const IColumn &column, size_t row_num), to_string_batch (const IColumn &column, ColumnString &column_to), from_string (ReadBuffer &rb, IColumn *column) -// this two function should move to DataTypeSerDe and only used in Block -// serialize (const IColumn &column, char *buf, int be_exec_version), deserialize (const char *buf, MutableColumnPtr *column, int be_exec_version) -// 4. compare: equals (const IDataType &rhs), is_comparable - -namespace doris::vectorized { - -static bool gen_check_data_in_assert = true; - -class CommonDataTypeTest : public ::testing::Test { -protected: - // Helper function to load data from CSV, with index which splited by spliter and load to columns - void load_data_from_csv(const DataTypeSerDeSPtrs serders, MutableColumns& columns, - const std::string& file_path, const char spliter = ';', - const std::set idxes = {0}) { - ASSERT_EQ(serders.size(), columns.size()) - << "serder size: " << serders.size() << " column size: " << columns.size(); - ASSERT_EQ(serders.size(), idxes.size()) - << "serder size: " << serders.size() << " idxes size: " << idxes.size(); - std::ifstream file(file_path); - if (!file) { - throw doris::Exception(ErrorCode::INVALID_ARGUMENT, "can not open the file: {} ", - file_path); - } - - std::string line; - DataTypeSerDe::FormatOptions options; - while (std::getline(file, line)) { - std::stringstream lineStream(line); - // std::cout << "whole : " << lineStream.str() << std::endl; - std::string value; - int l_idx = 0; - int c_idx = 0; - while (std::getline(lineStream, value, spliter)) { - if (idxes.contains(l_idx)) { - Slice string_slice(value.data(), value.size()); - std::cout << string_slice << std::endl; - if (auto st = serders[c_idx]->deserialize_one_cell_from_json( - *columns[c_idx], string_slice, options); - !st.ok()) { - // std::cout << "error in deserialize but continue: " << st.to_string() - // << std::endl; - } - ++c_idx; - } - ++l_idx; - } - } - } - -public: - // we make meta info a default value, so assert should change the struct value to the right value - struct DataTypeMetaInfo { - TypeIndex type_id = TypeIndex::Nothing; - TypeDescriptor* type_as_type_descriptor = nullptr; - std::string family_name = ""; - bool has_subtypes = false; - doris::FieldType storage_field_type = doris::FieldType::OLAP_FIELD_TYPE_UNKNOWN; - bool should_align_right_in_pretty_formats = false; - bool text_can_contain_only_valid_utf8 = false; - bool have_maximum_size_of_value = false; - size_t size_of_value_in_memory = -1; - size_t precision = -1; - size_t scale = -1; - bool is_null_literal = true; - bool is_value_represented_by_number = false; - PColumnMeta* pColumnMeta = nullptr; - DataTypeSerDeSPtr serde = nullptr; - // bool is_value_unambiguously_represented_in_contiguous_memory_region = false; - }; - void SetUp() override {} - - // meta info assert is simple and can be used for all DataType - void meta_info_assert(DataTypePtr& data_type, DataTypeMetaInfo& meta_info) { - ASSERT_NE(data_type->get_serde(1), nullptr); - ASSERT_EQ(IDataType::get_pdata_type(data_type.get()), meta_info.pColumnMeta->type()); - ASSERT_EQ(data_type->get_type_id(), meta_info.type_id); - ASSERT_EQ(data_type->get_type_as_type_descriptor(), *meta_info.type_as_type_descriptor); - ASSERT_EQ(data_type->get_family_name(), meta_info.family_name); - ASSERT_EQ(data_type->have_subtypes(), meta_info.has_subtypes); - ASSERT_EQ(data_type->get_storage_field_type(), meta_info.storage_field_type); - ASSERT_EQ(data_type->should_align_right_in_pretty_formats(), - meta_info.should_align_right_in_pretty_formats); - ASSERT_EQ(data_type->text_can_contain_only_valid_utf8(), - meta_info.text_can_contain_only_valid_utf8); - ASSERT_EQ(data_type->have_maximum_size_of_value(), meta_info.have_maximum_size_of_value); - ASSERT_EQ(data_type->get_size_of_value_in_memory(), meta_info.size_of_value_in_memory); - if (is_decimal(data_type)) { - ASSERT_EQ(data_type->get_precision(), meta_info.precision); - ASSERT_EQ(data_type->get_scale(), meta_info.scale); - } else { - EXPECT_ANY_THROW(EXPECT_FALSE(data_type->get_precision())); - EXPECT_THROW(EXPECT_FALSE(data_type->get_scale()), doris::Exception); - } - ASSERT_EQ(data_type->is_null_literal(), meta_info.is_null_literal); - ASSERT_EQ(data_type->is_value_represented_by_number(), - meta_info.is_value_represented_by_number); - // ASSERT_EQ(data_type->is_value_unambiguously_represented_in_contiguous_memory_region(), meta_info.is_value_unambiguously_represented_in_contiguous_memory_region); - } - - // create column assert with default field is simple and can be used for all DataType - void create_column_assert(DataTypePtr& data_type, Field& default_field) { - auto column = data_type->create_column(); - ASSERT_EQ(column->size(), 0); - ColumnPtr const_col = data_type->create_column_const(10, default_field); - ASSERT_EQ(const_col->size(), 10); - ColumnPtr default_const_col = data_type->create_column_const_with_default_value(10); - ASSERT_EQ(default_const_col->size(), 10); - for (int i = 0; i < 10; ++i) { - ASSERT_EQ(const_col->operator[](i), default_const_col->operator[](i)); - } - // get_uncompressed_serialized_bytes - ASSERT_EQ(data_type->get_uncompressed_serialized_bytes(*column, 0), 4); - } - - // get_field assert is simple and can be used for all DataType - void get_field_assert(DataTypePtr& data_type, TExprNode& node, Field& assert_field, - bool assert_false = false) { - if (assert_false) { - EXPECT_ANY_THROW(data_type->get_field(node)) - << "get_field_assert: " - << " datatype:" + data_type->get_name() << " node_type:" << node.node_type - << " field: " << assert_field.get_type() << std::endl; - } else { - Field field = data_type->get_field(node); - ASSERT_EQ(field, assert_field) - << "get_field_assert: " - << " datatype:" + data_type->get_name() << " node_type:" << node.node_type - << " field: " << assert_field.get_type() << std::endl; - } - } - - // to_string | to_string_batch | from_string assert is simple and can be used for all DataType - void assert_to_string_from_string_assert(MutableColumnPtr mutableColumn, - DataTypePtr& data_type) { - // to_string_batch | from_string - auto col_to = ColumnString::create(); - data_type->to_string_batch(*mutableColumn, *col_to); - ASSERT_EQ(col_to->size(), mutableColumn->size()); - // from_string assert col_to to assert_column and check same with mutableColumn - auto assert_column = data_type->create_column(); - for (int i = 0; i < col_to->size(); ++i) { - std::string s = col_to->get_data_at(i).to_string(); - ReadBuffer rb(s.data(), s.size()); - ASSERT_EQ(Status::OK(), data_type->from_string(rb, assert_column.get())); - ASSERT_EQ(assert_column->operator[](i), mutableColumn->operator[](i)); - } - // to_string | from_string - auto ser_col = ColumnString::create(); - ser_col->reserve(mutableColumn->size()); - VectorBufferWriter buffer_writer(*ser_col.get()); - for (int i = 0; i < mutableColumn->size(); ++i) { - data_type->to_string(*mutableColumn, i, buffer_writer); - buffer_writer.commit(); - } - // check ser_col to assert_column and check same with mutableColumn - auto assert_column_1 = data_type->create_column(); - for (int i = 0; i < ser_col->size(); ++i) { - std::string s = ser_col->get_data_at(i).to_string(); - ReadBuffer rb(s.data(), s.size()); - ASSERT_EQ(Status::OK(), data_type->from_string(rb, assert_column_1.get())); - ASSERT_EQ(assert_column_1->operator[](i), mutableColumn->operator[](i)); - } - } - - // should all datatype is compare? - void assert_compare_behavior(DataTypePtr l_dt, DataTypePtr& r_dt) { - ASSERT_TRUE(l_dt->is_comparable()); - ASSERT_TRUE(r_dt->is_comparable()); - // compare - ASSERT_FALSE(l_dt->equals(*r_dt)); - } -}; - -} // namespace doris::vectorized \ No newline at end of file diff --git a/be/test/vec/function/function_test_util.cpp b/be/test/vec/function/function_test_util.cpp index 7cc381c260a21c..9703d0a3075051 100644 --- a/be/test/vec/function/function_test_util.cpp +++ b/be/test/vec/function/function_test_util.cpp @@ -34,7 +34,9 @@ #include "vec/data_types/data_type_ipv4.h" #include "vec/data_types/data_type_ipv6.h" #include "vec/data_types/data_type_jsonb.h" +#include "vec/data_types/data_type_map.h" #include "vec/data_types/data_type_string.h" +#include "vec/data_types/data_type_struct.h" #include "vec/data_types/data_type_time_v2.h" #include "vec/exprs/table_function/table_function.h" #include "vec/runtime/vdatetime_value.h" @@ -195,6 +197,55 @@ size_t type_index_to_data_type(const std::vector& input_types, size_t i type = std::make_shared(sub_type); return ret + 1; } + case TypeIndex::Map: { + desc.type = doris::PrimitiveType::TYPE_MAP; + ut_type::UTDataTypeDesc key_desc; + DataTypePtr key_type = nullptr; + ut_type::UTDataTypeDesc value_desc; + DataTypePtr value_type = nullptr; + ++index; + size_t ret = type_index_to_data_type(input_types, index, key_desc, key_type); + if (ret <= 0) { + return ret; + } + ++index; + ret = type_index_to_data_type(input_types, index, value_desc, value_type); + if (ret <= 0) { + return ret; + } + desc.children.push_back(key_desc.type_desc); + desc.children.push_back(value_desc.type_desc); + if (key_desc.is_nullable) { + key_type = make_nullable(key_type); + } + if (value_desc.is_nullable) { + value_type = make_nullable(value_type); + } + type = std::make_shared(key_type, value_type); + return ret + 1; + } + case TypeIndex::Struct: { + desc.type = doris::PrimitiveType::TYPE_STRUCT; + ++index; + size_t ret = 0; + DataTypes sub_types; + while (index < input_types.size()) { + ut_type::UTDataTypeDesc sub_desc; + DataTypePtr sub_type = nullptr; + ret = type_index_to_data_type(input_types, index, sub_desc, sub_type); + if (ret <= 0) { + return ret; + } + desc.children.push_back(sub_desc.type_desc); + if (sub_desc.is_nullable) { + sub_type = make_nullable(sub_type); + sub_types.push_back(sub_type); + } + ++index; + } + type = std::make_shared(sub_types); + return ret + 1; + } case TypeIndex::Nullable: { ++index; size_t ret = type_index_to_data_type(input_types, index, ut_desc, type); From d3a18aed996e1d427d80782ef7e0b480af04b756 Mon Sep 17 00:00:00 2001 From: amorynan Date: Mon, 30 Dec 2024 14:15:50 +0800 Subject: [PATCH 07/23] fix beut --- .../data_types/common_data_type_serder_test.h | 394 ++++++++++++++++++ 1 file changed, 394 insertions(+) create mode 100644 be/src/vec/data_types/common_data_type_serder_test.h diff --git a/be/src/vec/data_types/common_data_type_serder_test.h b/be/src/vec/data_types/common_data_type_serder_test.h new file mode 100644 index 00000000000000..d1f3e674e51a73 --- /dev/null +++ b/be/src/vec/data_types/common_data_type_serder_test.h @@ -0,0 +1,394 @@ +// 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 +#include +#include +#include + +#include +#include +#include + +#include "arrow/type.h" +#include "runtime/descriptors.h" +#include "util/arrow/block_convertor.h" +#include "util/arrow/row_batch.h" +#include "vec/columns/column.h" +#include "vec/core/field.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type.h" +#include "vec/utils/arrow_column_to_doris_column.h" + +// this test is gonna to be a data type serialize and deserialize functions +// such as +// 1. standard hive text ser-deserialize +// deserialize_one_cell_from_hive_text (IColumn &column, Slice &slice, const FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const +// deserialize_column_from_hive_text_vector (IColumn &column, std::vector< Slice > &slices, int *num_deserialized, const FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const +// serialize_one_cell_to_hive_text (const IColumn &column, int row_num, BufferWritable &bw, FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const +// 2. json format ser-deserialize which used in table not in doris database +// serialize_one_cell_to_json (const IColumn &column, int row_num, BufferWritable &bw, FormatOptions &options) const =0 +// serialize_column_to_json (const IColumn &column, int start_idx, int end_idx, BufferWritable &bw, FormatOptions &options) const =0 +// deserialize_one_cell_from_json (IColumn &column, Slice &slice, const FormatOptions &options) const =0 +// deserialize_column_from_json_vector (IColumn &column, std::vector< Slice > &slices, int *num_deserialized, const FormatOptions &options) const =0 +// deserialize_column_from_fixed_json (IColumn &column, Slice &slice, int rows, int *num_deserialized, const FormatOptions &options) const +// insert_column_last_value_multiple_times (IColumn &column, int times) const +// 3. fe|be protobuffer ser-deserialize +// write_column_to_pb (const IColumn &column, PValues &result, int start, int end) const =0 +// read_column_from_pb (IColumn &column, const PValues &arg) const =0 +// 4. jsonb ser-deserialize which used in row-store situation +// write_one_cell_to_jsonb (const IColumn &column, JsonbWriter &result, Arena *mem_pool, int32_t col_id, int row_num) const =0 +// read_one_cell_from_jsonb (IColumn &column, const JsonbValue *arg) const =0 +// 5. mysql text ser-deserialize +// write_column_to_mysql (const IColumn &column, MysqlRowBuffer< false > &row_buffer, int row_idx, bool col_const, const FormatOptions &options) const =0 +// write_column_to_mysql (const IColumn &column, MysqlRowBuffer< true > &row_buffer, int row_idx, bool col_const, const FormatOptions &options) const =0 +// 6. arrow ser-deserialize which used in spark-flink connector +// write_column_to_arrow (const IColumn &column, const NullMap *null_map, arrow::ArrayBuilder *array_builder, int start, int end, const cctz::time_zone &ctz) const =0 +// read_column_from_arrow (IColumn &column, const arrow::Array *arrow_array, int start, int end, const cctz::time_zone &ctz) const =0 +// 7. rapidjson ser-deserialize +// write_one_cell_to_json (const IColumn &column, rapidjson::Value &result, rapidjson::Document::AllocatorType &allocator, Arena &mem_pool, int row_num) const +// read_one_cell_from_json (IColumn &column, const rapidjson::Value &result) const +// convert_field_to_rapidjson (const vectorized::Field &field, rapidjson::Value &target, rapidjson::Document::AllocatorType &allocator) +// convert_array_to_rapidjson (const vectorized::Array &array, rapidjson::Value &target, rapidjson::Document::AllocatorType &allocator) + +namespace doris::vectorized { + +class CommonDataTypeSerdeTest : public ::testing::Test { +public: + ////================================================================================================================== + // this is common function to check data in column against expected results according different function in assert function + // which can be used in all column test + // such as run regress tests + // step1. we can set gen_check_data_in_assert to true, then we will generate a file for check data, otherwise we will read the file to check data + // step2. we should write assert callback function to check data + static void check_data( + MutableColumns& columns, DataTypeSerDeSPtrs serders, char col_spliter, + std::set idxes, const std::string& column_data_file, + std::function + assert_callback, + bool is_hive_format = false, DataTypes dataTypes = {}) { + ASSERT_EQ(serders.size(), columns.size()); + // Step 1: Insert data from `column_data_file` into the column and check result with `check_data_file` + // Load column data and expected data from CSV files + std::vector> res; + struct stat buff; + if (stat(column_data_file.c_str(), &buff) == 0) { + if (S_ISREG(buff.st_mode)) { + // file + if (is_hive_format) { + load_data_and_assert_from_csv(serders, columns, column_data_file, + col_spliter, idxes); + } else { + load_data_and_assert_from_csv(serders, columns, column_data_file, + col_spliter, idxes); + } + } else if (S_ISDIR(buff.st_mode)) { + // dir + std::filesystem::path fs_path(column_data_file); + for (const auto& entry : std::filesystem::directory_iterator(fs_path)) { + std::string file_path = entry.path().string(); + std::cout << "load data from file: " << file_path << std::endl; + if (is_hive_format) { + load_data_and_assert_from_csv(serders, columns, file_path, + col_spliter, idxes); + } else { + load_data_and_assert_from_csv(serders, columns, file_path, + col_spliter, idxes); + } + } + } + } + + // Step 2: Validate the data in `column` matches `expected_data` + assert_callback(columns, serders); + } + + // Helper function to load data from CSV, with index which splited by spliter and load to columns + template + static void load_data_and_assert_from_csv(const DataTypeSerDeSPtrs serders, + MutableColumns& columns, const std::string& file_path, + const char spliter = ';', + const std::set idxes = {0}) { + ASSERT_EQ(serders.size(), columns.size()) + << "serder size: " << serders.size() << " column size: " << columns.size(); + ASSERT_EQ(serders.size(), idxes.size()) + << "serder size: " << serders.size() << " idxes size: " << idxes.size(); + std::ifstream file(file_path); + if (!file) { + throw doris::Exception(ErrorCode::INVALID_ARGUMENT, "can not open the file: {} ", + file_path); + } + + std::string line; + DataTypeSerDe::FormatOptions options; + std::vector> res; + MutableColumns assert_str_cols(columns.size()); + for (size_t i = 0; i < columns.size(); ++i) { + assert_str_cols[i] = ColumnString::create(); + } + + while (std::getline(file, line)) { + std::stringstream lineStream(line); + // std::cout << "whole : " << lineStream.str() << std::endl; + std::string value; + int l_idx = 0; + int c_idx = 0; + std::vector row; + while (std::getline(lineStream, value, spliter)) { + if (idxes.contains(l_idx)) { + // load csv data + Slice string_slice(value.data(), value.size()); + std::cout << "origin : " << string_slice << std::endl; + Status st; + // deserialize data + if constexpr (is_hive_format) { + st = serders[c_idx]->deserialize_one_cell_from_hive_text( + *columns[c_idx], string_slice, options); + } else { + st = serders[c_idx]->deserialize_one_cell_from_json(*columns[c_idx], + string_slice, options); + } + if (!st.ok()) { + // deserialize if happen error now we do not insert any value for input column + // so we push a default value to column for row alignment + columns[c_idx]->insert_default(); + std::cout << "error in deserialize but continue: " << st.to_string() + << std::endl; + } + // serialize data + size_t row_num = columns[c_idx]->size() - 1; + assert_str_cols[c_idx]->reserve(columns[c_idx]->size()); + VectorBufferWriter bw(assert_cast(*assert_str_cols[c_idx])); + if constexpr (is_hive_format) { + st = serders[c_idx]->serialize_one_cell_to_hive_text(*columns[c_idx], + row_num, bw, options); + EXPECT_TRUE(st.ok()) << st.to_string(); + } else { + st = serders[c_idx]->serialize_one_cell_to_json(*columns[c_idx], row_num, + bw, options); + EXPECT_TRUE(st.ok()) << st.to_string(); + } + bw.commit(); + // assert data : origin data and serialized data should be equal or generated + // file to check data + size_t assert_size = assert_str_cols[c_idx]->size(); + if constexpr (!generate_res_file) { + EXPECT_EQ(assert_str_cols[c_idx]->get_data_at(assert_size - 1).to_string(), + string_slice.to_string()) + << "column: " << columns[c_idx]->get_name() << " row: " << row_num + << " is_hive_format: " << is_hive_format; + } + ++c_idx; + } + res.push_back(row); + ++l_idx; + } + } + + if (generate_res_file) { + // generate res + auto pos = file_path.find_last_of("."); + std::string hive_format = is_hive_format ? "_hive" : ""; + std::string res_file = file_path.substr(0, pos) + hive_format + "_serde_res.csv"; + std::ofstream res_f(res_file); + if (!res_f.is_open()) { + throw std::ios_base::failure("Failed to open file." + res_file); + } + for (size_t r = 0; r < assert_str_cols[0]->size(); ++r) { + for (size_t c = 0; c < assert_str_cols.size(); ++c) { + std::cout << assert_str_cols[c]->get_data_at(r).to_string() << spliter + << std::endl; + res_f << assert_str_cols[c]->get_data_at(r).to_string() << spliter; + } + res_f << std::endl; + } + res_f.close(); + std::cout << "generate res file: " << res_file << std::endl; + } + } + + // standard hive text ser-deserialize assert function + // pb serde now is only used RPCFncall and fold_constant_executor which just write column data to pb value means + // just call write_column_to_pb + static void assert_pb_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& col = load_cols[i]; + std::cout << " now we are testing column : " << col->get_name() << std::endl; + // serialize to pb + PValues pv = PValues(); + Status st = serders[i]->write_column_to_pb(*col, pv, 0, col->size()); + if (!st.ok()) { + std::cerr << "write_column_to_pb error: " << st.msg() << std::endl; + continue; + } + // deserialize from pb + auto except_column = col->clone_empty(); + st = serders[i]->read_column_from_pb(*except_column, pv); + EXPECT_TRUE(st.ok()) << st.to_string(); + // check pb value from expected column + PValues as_pv = PValues(); + st = serders[i]->write_column_to_pb(*except_column, as_pv, 0, except_column->size()); + EXPECT_TRUE(st.ok()) << st.to_string(); + EXPECT_EQ(pv.bytes_value_size(), as_pv.bytes_value_size()); + // check column value + for (size_t j = 0; j < col->size(); ++j) { + auto cell = col->operator[](j); + auto except_cell = except_column->operator[](j); + EXPECT_EQ(cell, except_cell) << "column: " << col->get_name() << " row: " << j; + } + } + } + + // actually this is block_to_jsonb and jsonb_to_block test + static void assert_jsonb_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { + Arena pool; + auto jsonb_column = ColumnString::create(); // jsonb column + // maybe these load_cols has different size, so we keep it same + size_t max_row_size = load_cols[0]->size(); + for (size_t i = 1; i < load_cols.size(); ++i) { + if (load_cols[i]->size() > max_row_size) { + max_row_size = load_cols[i]->size(); + } + } + // keep same rows + for (size_t i = 0; i < load_cols.size(); ++i) { + if (load_cols[i]->size() < max_row_size) { + load_cols[i]->insert_many_defaults(max_row_size - load_cols[i]->size()); + } else if (load_cols[i]->size() > max_row_size) { + load_cols[i]->resize(max_row_size); + } + } + jsonb_column->reserve(load_cols[0]->size()); + MutableColumns assert_cols; + for (size_t i = 0; i < load_cols.size(); ++i) { + assert_cols.push_back(load_cols[i]->assume_mutable()); + } + for (size_t r = 0; r < load_cols[0]->size(); ++r) { + JsonbWriterT jw; + jw.writeStartObject(); + // serialize to jsonb + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& col = load_cols[i]; + serders[i]->write_one_cell_to_jsonb(*col, jw, &pool, i, r); + } + jw.writeEndObject(); + jsonb_column->insert_data(jw.getOutput()->getBuffer(), jw.getOutput()->getSize()); + } + // deserialize jsonb column to assert column + EXPECT_EQ(jsonb_column->size(), load_cols[0]->size()); + for (size_t r = 0; r < jsonb_column->size(); ++r) { + StringRef jsonb_data = jsonb_column->get_data_at(r); + auto pdoc = JsonbDocument::createDocument(jsonb_data.data, jsonb_data.size); + JsonbDocument& doc = *pdoc; + size_t cIdx = 0; + for (auto it = doc->begin(); it != doc->end(); ++it) { + serders[cIdx]->read_one_cell_from_jsonb(*assert_cols[cIdx], it->value()); + ++cIdx; + } + } + // check column value + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& col = load_cols[i]; + auto& assert_col = assert_cols[i]; + for (size_t j = 0; j < col->size(); ++j) { + auto cell = col->operator[](j); + auto assert_cell = assert_col->operator[](j); + EXPECT_EQ(cell, assert_cell) << "column: " << col->get_name() << " row: " << j; + } + } + } + + // assert mysql text format, now we just simple assert not to fatal or exception here + static void assert_mysql_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { + MysqlRowBuffer row_buffer; + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& col = load_cols[i]; + for (size_t j = 0; j < col->size(); ++j) { + Status st; + EXPECT_NO_FATAL_FAILURE( + st = serders[i]->write_column_to_mysql(*col, row_buffer, j, false, {})); + EXPECT_TRUE(st.ok()) << st.to_string(); + } + } + } + + // assert arrow serialize + static void assert_arrow_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders, + DataTypes types) { + // make a block to write to arrow + auto block = std::make_shared(); + // maybe these load_cols has different size, so we keep it same + size_t max_row_size = load_cols[0]->size(); + for (size_t i = 1; i < load_cols.size(); ++i) { + if (load_cols[i]->size() > max_row_size) { + max_row_size = load_cols[i]->size(); + } + } + // keep same rows + for (size_t i = 0; i < load_cols.size(); ++i) { + if (load_cols[i]->size() < max_row_size) { + load_cols[i]->insert_many_defaults(max_row_size - load_cols[i]->size()); + } else if (load_cols[i]->size() > max_row_size) { + load_cols[i]->resize(max_row_size); + } + } + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& col = load_cols[i]; + block->insert(ColumnWithTypeAndName(std::move(col), types[i], types[i]->get_name())); + } + // print block + std::cout << "block: " << block->dump_structure() << std::endl; + std::shared_ptr block_arrow_schema; + EXPECT_EQ(get_arrow_schema_from_block(*block, &block_arrow_schema, "UTC"), Status::OK()); + std::cout << "schema: " << block_arrow_schema->ToString(true) << std::endl; + // convert block to arrow + std::shared_ptr result; + cctz::time_zone _timezone_obj; //default UTC + Status stt = convert_to_arrow_batch(*block, block_arrow_schema, + arrow::default_memory_pool(), &result, _timezone_obj); + EXPECT_EQ(Status::OK(), stt) << "convert block to arrow failed" << stt.to_string(); + std::cout << "arrow record batch: " << result->ToString() << std::endl; + std::cout << "block : " << block->dump_structure() << std::endl; + // deserialize arrow to block + auto assert_block = block->clone_empty(); + auto rows = block->rows(); + std::cout << "assert block rows: " << rows << std::endl; + for (size_t i = 0; i < load_cols.size(); ++i) { + auto array = result->column(i); + auto& column_with_type_and_name = assert_block.get_by_position(i); + auto ret = arrow_column_to_doris_column( + array.get(), 0, column_with_type_and_name.column, + column_with_type_and_name.type, rows, _timezone_obj); + // do check data + EXPECT_EQ(Status::OK(), ret) << "convert arrow to block failed" << ret.to_string(); + auto& col = block->get_by_position(i).column; + auto& assert_col = column_with_type_and_name.column; + for (size_t j = 0; j < col->size(); ++j) { + auto cell = col->operator[](j); + auto assert_cell = assert_col->operator[](j); + EXPECT_EQ(cell, assert_cell) << "column: " << col->get_name() << " row: " << j; + } + } + std::cout << "assert block: " << assert_block.dump_structure() << std::endl; + } + + // assert rapidjson format + // now rapidjson write_one_cell_to_json and read_one_cell_from_json only used in column_object + // can just be replaced by jsonb format +}; + +} // namespace doris::vectorized \ No newline at end of file From 50cbaa3c0167eef9d6627eee6efb26c870367b78 Mon Sep 17 00:00:00 2001 From: amorynan Date: Mon, 30 Dec 2024 14:25:22 +0800 Subject: [PATCH 08/23] update file --- .../data_types/common_data_type_serder_test.h | 394 ++++++++++++++++++ .../vec/data_types/common_data_type_test.h | 238 +++++++++++ 2 files changed, 632 insertions(+) create mode 100644 be/test/vec/data_types/common_data_type_serder_test.h create mode 100644 be/test/vec/data_types/common_data_type_test.h diff --git a/be/test/vec/data_types/common_data_type_serder_test.h b/be/test/vec/data_types/common_data_type_serder_test.h new file mode 100644 index 00000000000000..f8aa9d890e6f3c --- /dev/null +++ b/be/test/vec/data_types/common_data_type_serder_test.h @@ -0,0 +1,394 @@ +// 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 +#include +#include +#include + +#include +#include +#include +#include "arrow/type.h" +#include "runtime/descriptors.h" +#include "util/arrow/block_convertor.h" +#include "util/arrow/row_batch.h" +#include "vec/columns/column.h" +#include "vec/core/field.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type.h" +#include "vec/utils/arrow_column_to_doris_column.h" + +// this test is gonna to be a data type serialize and deserialize functions +// such as +// 1. standard hive text ser-deserialize +// deserialize_one_cell_from_hive_text (IColumn &column, Slice &slice, const FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const +// deserialize_column_from_hive_text_vector (IColumn &column, std::vector< Slice > &slices, int *num_deserialized, const FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const +// serialize_one_cell_to_hive_text (const IColumn &column, int row_num, BufferWritable &bw, FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const +// 2. json format ser-deserialize which used in table not in doris database +// serialize_one_cell_to_json (const IColumn &column, int row_num, BufferWritable &bw, FormatOptions &options) const =0 +// serialize_column_to_json (const IColumn &column, int start_idx, int end_idx, BufferWritable &bw, FormatOptions &options) const =0 +// deserialize_one_cell_from_json (IColumn &column, Slice &slice, const FormatOptions &options) const =0 +// deserialize_column_from_json_vector (IColumn &column, std::vector< Slice > &slices, int *num_deserialized, const FormatOptions &options) const =0 +// deserialize_column_from_fixed_json (IColumn &column, Slice &slice, int rows, int *num_deserialized, const FormatOptions &options) const +// insert_column_last_value_multiple_times (IColumn &column, int times) const +// 3. fe|be protobuffer ser-deserialize +// write_column_to_pb (const IColumn &column, PValues &result, int start, int end) const =0 +// read_column_from_pb (IColumn &column, const PValues &arg) const =0 +// 4. jsonb ser-deserialize which used in row-store situation +// write_one_cell_to_jsonb (const IColumn &column, JsonbWriter &result, Arena *mem_pool, int32_t col_id, int row_num) const =0 +// read_one_cell_from_jsonb (IColumn &column, const JsonbValue *arg) const =0 +// 5. mysql text ser-deserialize +// write_column_to_mysql (const IColumn &column, MysqlRowBuffer< false > &row_buffer, int row_idx, bool col_const, const FormatOptions &options) const =0 +// write_column_to_mysql (const IColumn &column, MysqlRowBuffer< true > &row_buffer, int row_idx, bool col_const, const FormatOptions &options) const =0 +// 6. arrow ser-deserialize which used in spark-flink connector +// write_column_to_arrow (const IColumn &column, const NullMap *null_map, arrow::ArrayBuilder *array_builder, int start, int end, const cctz::time_zone &ctz) const =0 +// read_column_from_arrow (IColumn &column, const arrow::Array *arrow_array, int start, int end, const cctz::time_zone &ctz) const =0 +// 7. rapidjson ser-deserialize +// write_one_cell_to_json (const IColumn &column, rapidjson::Value &result, rapidjson::Document::AllocatorType &allocator, Arena &mem_pool, int row_num) const +// read_one_cell_from_json (IColumn &column, const rapidjson::Value &result) const +// convert_field_to_rapidjson (const vectorized::Field &field, rapidjson::Value &target, rapidjson::Document::AllocatorType &allocator) +// convert_array_to_rapidjson (const vectorized::Array &array, rapidjson::Value &target, rapidjson::Document::AllocatorType &allocator) + +namespace doris::vectorized { + +class CommonDataTypeSerdeTest : public ::testing::Test { +public: + ////================================================================================================================== + // this is common function to check data in column against expected results according different function in assert function + // which can be used in all column test + // such as run regress tests + // step1. we can set gen_check_data_in_assert to true, then we will generate a file for check data, otherwise we will read the file to check data + // step2. we should write assert callback function to check data + static void check_data( + MutableColumns& columns, DataTypeSerDeSPtrs serders, char col_spliter, + std::set idxes, const std::string& column_data_file, + std::function + assert_callback, + bool is_hive_format = false, DataTypes dataTypes = {}) { + ASSERT_EQ(serders.size(), columns.size()); + // Step 1: Insert data from `column_data_file` into the column and check result with `check_data_file` + // Load column data and expected data from CSV files + std::vector> res; + struct stat buff; + if (stat(column_data_file.c_str(), &buff) == 0) { + if (S_ISREG(buff.st_mode)) { + // file + if (is_hive_format) { + load_data_and_assert_from_csv(serders, columns, column_data_file, + col_spliter, idxes); + } else { + load_data_and_assert_from_csv(serders, columns, column_data_file, + col_spliter, idxes); + } + } else if (S_ISDIR(buff.st_mode)) { + // dir + std::filesystem::path fs_path(column_data_file); + for (const auto& entry : std::filesystem::directory_iterator(fs_path)) { + std::string file_path = entry.path().string(); + std::cout << "load data from file: " << file_path << std::endl; + if (is_hive_format) { + load_data_and_assert_from_csv(serders, columns, file_path, + col_spliter, idxes); + } else { + load_data_and_assert_from_csv(serders, columns, file_path, + col_spliter, idxes); + } + } + } + } + + // Step 2: Validate the data in `column` matches `expected_data` + assert_callback(columns, serders); + } + + // Helper function to load data from CSV, with index which splited by spliter and load to columns + template + static void load_data_and_assert_from_csv(const DataTypeSerDeSPtrs serders, + MutableColumns& columns, const std::string& file_path, + const char spliter = ';', + const std::set idxes = {0}) { + ASSERT_EQ(serders.size(), columns.size()) + << "serder size: " << serders.size() << " column size: " << columns.size(); + ASSERT_EQ(serders.size(), idxes.size()) + << "serder size: " << serders.size() << " idxes size: " << idxes.size(); + std::ifstream file(file_path); + if (!file) { + throw doris::Exception(ErrorCode::INVALID_ARGUMENT, "can not open the file: {} ", + file_path); + } + + std::string line; + DataTypeSerDe::FormatOptions options; + std::vector> res; + MutableColumns assert_str_cols(columns.size()); + for (size_t i = 0; i < columns.size(); ++i) { + assert_str_cols[i] = ColumnString::create(); + } + + while (std::getline(file, line)) { + std::stringstream lineStream(line); + // std::cout << "whole : " << lineStream.str() << std::endl; + std::string value; + int l_idx = 0; + int c_idx = 0; + std::vector row; + while (std::getline(lineStream, value, spliter)) { + if (idxes.contains(l_idx)) { + // load csv data + Slice string_slice(value.data(), value.size()); + std::cout << "origin : " << string_slice << std::endl; + Status st; + // deserialize data + if constexpr (is_hive_format) { + st = serders[c_idx]->deserialize_one_cell_from_hive_text( + *columns[c_idx], string_slice, options); + } else { + st = serders[c_idx]->deserialize_one_cell_from_json(*columns[c_idx], + string_slice, options); + } + if (!st.ok()) { + // deserialize if happen error now we do not insert any value for input column + // so we push a default value to column for row alignment + columns[c_idx]->insert_default(); + std::cout << "error in deserialize but continue: " << st.to_string() + << std::endl; + } + // serialize data + size_t row_num = columns[c_idx]->size() - 1; + assert_str_cols[c_idx]->reserve(columns[c_idx]->size()); + VectorBufferWriter bw(assert_cast(*assert_str_cols[c_idx])); + if constexpr (is_hive_format) { + st = serders[c_idx]->serialize_one_cell_to_hive_text(*columns[c_idx], + row_num, bw, options); + EXPECT_TRUE(st.ok()) << st.to_string(); + } else { + st = serders[c_idx]->serialize_one_cell_to_json(*columns[c_idx], row_num, + bw, options); + EXPECT_TRUE(st.ok()) << st.to_string(); + } + bw.commit(); + // assert data : origin data and serialized data should be equal or generated + // file to check data + size_t assert_size = assert_str_cols[c_idx]->size(); + if constexpr (!generate_res_file) { + EXPECT_EQ(assert_str_cols[c_idx]->get_data_at(assert_size - 1).to_string(), + string_slice.to_string()) + << "column: " << columns[c_idx]->get_name() << " row: " << row_num + << " is_hive_format: " << is_hive_format; + } + ++c_idx; + } + res.push_back(row); + ++l_idx; + } + } + + if (generate_res_file) { + // generate res + auto pos = file_path.find_last_of("."); + std::string hive_format = is_hive_format ? "_hive" : ""; + std::string res_file = file_path.substr(0, pos) + hive_format + "_serde_res.csv"; + std::ofstream res_f(res_file); + if (!res_f.is_open()) { + throw std::ios_base::failure("Failed to open file." + res_file); + } + for (size_t r = 0; r < assert_str_cols[0]->size(); ++r) { + for (size_t c = 0; c < assert_str_cols.size(); ++c) { + std::cout << assert_str_cols[c]->get_data_at(r).to_string() << spliter + << std::endl; + res_f << assert_str_cols[c]->get_data_at(r).to_string() << spliter; + } + res_f << std::endl; + } + res_f.close(); + std::cout << "generate res file: " << res_file << std::endl; + } + } + + // standard hive text ser-deserialize assert function + // pb serde now is only used RPCFncall and fold_constant_executor which just write column data to pb value means + // just call write_column_to_pb + static void assert_pb_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& col = load_cols[i]; + std::cout << " now we are testing column : " << col->get_name() << std::endl; + // serialize to pb + PValues pv = PValues(); + Status st = serders[i]->write_column_to_pb(*col, pv, 0, col->size()); + if (!st.ok()) { + std::cerr << "write_column_to_pb error: " << st.msg() << std::endl; + continue; + } + // deserialize from pb + auto except_column = col->clone_empty(); + st = serders[i]->read_column_from_pb(*except_column, pv); + EXPECT_TRUE(st.ok()) << st.to_string(); + // check pb value from expected column + PValues as_pv = PValues(); + st = serders[i]->write_column_to_pb(*except_column, as_pv, 0, except_column->size()); + EXPECT_TRUE(st.ok()) << st.to_string(); + EXPECT_EQ(pv.bytes_value_size(), as_pv.bytes_value_size()); + // check column value + for (size_t j = 0; j < col->size(); ++j) { + auto cell = col->operator[](j); + auto except_cell = except_column->operator[](j); + EXPECT_EQ(cell, except_cell) << "column: " << col->get_name() << " row: " << j; + } + } + } + + // actually this is block_to_jsonb and jsonb_to_block test + static void assert_jsonb_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { + Arena pool; + auto jsonb_column = ColumnString::create(); // jsonb column + // maybe these load_cols has different size, so we keep it same + size_t max_row_size = load_cols[0]->size(); + for (size_t i = 1; i < load_cols.size(); ++i) { + if (load_cols[i]->size() > max_row_size) { + max_row_size = load_cols[i]->size(); + } + } + // keep same rows + for (size_t i = 0; i < load_cols.size(); ++i) { + if (load_cols[i]->size() < max_row_size) { + load_cols[i]->insert_many_defaults(max_row_size - load_cols[i]->size()); + } else if (load_cols[i]->size() > max_row_size) { + load_cols[i]->resize(max_row_size); + } + } + jsonb_column->reserve(load_cols[0]->size()); + MutableColumns assert_cols; + for (size_t i = 0; i < load_cols.size(); ++i) { + assert_cols.push_back(load_cols[i]->assume_mutable()); + } + for (size_t r = 0; r < load_cols[0]->size(); ++r) { + JsonbWriterT jw; + jw.writeStartObject(); + // serialize to jsonb + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& col = load_cols[i]; + serders[i]->write_one_cell_to_jsonb(*col, jw, &pool, i, r); + } + jw.writeEndObject(); + jsonb_column->insert_data(jw.getOutput()->getBuffer(), jw.getOutput()->getSize()); + } + // deserialize jsonb column to assert column + EXPECT_EQ(jsonb_column->size(), load_cols[0]->size()); + for (size_t r = 0; r < jsonb_column->size(); ++r) { + StringRef jsonb_data = jsonb_column->get_data_at(r); + auto pdoc = JsonbDocument::createDocument(jsonb_data.data, jsonb_data.size); + JsonbDocument& doc = *pdoc; + size_t cIdx = 0; + for (auto it = doc->begin(); it != doc->end(); ++it) { + serders[cIdx]->read_one_cell_from_jsonb(*assert_cols[cIdx], it->value()); + ++cIdx; + } + } + // check column value + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& col = load_cols[i]; + auto& assert_col = assert_cols[i]; + for (size_t j = 0; j < col->size(); ++j) { + auto cell = col->operator[](j); + auto assert_cell = assert_col->operator[](j); + EXPECT_EQ(cell, assert_cell) << "column: " << col->get_name() << " row: " << j; + } + } + } + + // assert mysql text format, now we just simple assert not to fatal or exception here + static void assert_mysql_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { + MysqlRowBuffer row_buffer; + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& col = load_cols[i]; + for (size_t j = 0; j < col->size(); ++j) { + Status st; + EXPECT_NO_FATAL_FAILURE( + st = serders[i]->write_column_to_mysql(*col, row_buffer, j, false, {})); + EXPECT_TRUE(st.ok()) << st.to_string(); + } + } + } + + // assert arrow serialize + static void assert_arrow_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders, + DataTypes types) { + // make a block to write to arrow + auto block = std::make_shared(); + // maybe these load_cols has different size, so we keep it same + size_t max_row_size = load_cols[0]->size(); + for (size_t i = 1; i < load_cols.size(); ++i) { + if (load_cols[i]->size() > max_row_size) { + max_row_size = load_cols[i]->size(); + } + } + // keep same rows + for (size_t i = 0; i < load_cols.size(); ++i) { + if (load_cols[i]->size() < max_row_size) { + load_cols[i]->insert_many_defaults(max_row_size - load_cols[i]->size()); + } else if (load_cols[i]->size() > max_row_size) { + load_cols[i]->resize(max_row_size); + } + } + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& col = load_cols[i]; + block->insert(ColumnWithTypeAndName(std::move(col), types[i], types[i]->get_name())); + } + // print block + std::cout << "block: " << block->dump_structure() << std::endl; + std::shared_ptr block_arrow_schema; + EXPECT_EQ(get_arrow_schema_from_block(*block, &block_arrow_schema, "UTC"), Status::OK()); + std::cout << "schema: " << block_arrow_schema->ToString(true) << std::endl; + // convert block to arrow + std::shared_ptr result; + cctz::time_zone _timezone_obj; //default UTC + Status stt = convert_to_arrow_batch(*block, block_arrow_schema, + arrow::default_memory_pool(), &result, _timezone_obj); + EXPECT_EQ(Status::OK(), stt) << "convert block to arrow failed" << stt.to_string(); + // deserialize arrow to block + auto assert_block = block->clone_empty(); + auto rows = block->rows(); + for (size_t i = 0; i < load_cols.size(); ++i) { + auto array = result->column(i); + auto& column_with_type_and_name = assert_block.get_by_position(i); + auto ret = arrow_column_to_doris_column( + array.get(), 0, column_with_type_and_name.column, + column_with_type_and_name.type, rows, _timezone_obj); + // do check data + std::cout << "arrow_column_to_doris_column done: " << column_with_type_and_name.column->get_name() + << " with column size: " << column_with_type_and_name.column->size() <get_by_position(i).column; + auto& assert_col = column_with_type_and_name.column; + EXPECT_EQ(assert_col->size(), col->size()); + for (size_t j = 0; j < col->size(); ++j) { + auto cell = col->operator[](j); + auto assert_cell = assert_col->operator[](j); + EXPECT_EQ(cell, assert_cell) << "column: " << col->get_name() << " row: " << j; + } + } + std::cout << "assert block: " << assert_block.dump_structure() << std::endl; + } + + // assert rapidjson format + // now rapidjson write_one_cell_to_json and read_one_cell_from_json only used in column_object + // can just be replaced by jsonb format +}; + +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/test/vec/data_types/common_data_type_test.h b/be/test/vec/data_types/common_data_type_test.h new file mode 100644 index 00000000000000..e604474bb87a69 --- /dev/null +++ b/be/test/vec/data_types/common_data_type_test.h @@ -0,0 +1,238 @@ +// 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 +#include +#include + +#include +#include +#include + +#include "olap/schema.h" +#include "vec/columns/column.h" +#include "vec/columns/column_array.h" +#include "vec/columns/column_map.h" +#include "vec/columns/columns_number.h" +#include "vec/core/field.h" +#include "vec/core/sort_block.h" +#include "vec/core/sort_description.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_map.h" + +// this test is gonna to be a data type test template for all DataType which should make ut test to coverage the function defined +// for example DataTypeIPv4 should test this function: +// 1. datatype meta info: +// get_type_id, get_type_as_type_descriptor, get_storage_field_type, have_subtypes, get_pdata_type (const IDataType *data_type), to_pb_column_meta (PColumnMeta *col_meta) +// get_family_name, get_is_parametric, should_align_right_in_pretty_formats +// text_can_contain_only_valid_utf8 +// have_maximum_size_of_value, get_maximum_size_of_value_in_memory, get_size_of_value_in_memory +// get_precision, get_scale +// get_field +// is_null_literal, is_value_represented_by_number, is_value_unambiguously_represented_in_contiguous_memory_region, is_value_unambiguously_represented_in_fixed_size_contiguous_memory_region +// 2. datatype creation with column: create_column, create_column_const (size_t size, const Field &field), create_column_const_with_default_value (size_t size), get_uncompressed_serialized_bytes (const IColumn &column, int be_exec_version) +// 3. serde related: get_serde (int nesting_level=1) +// to_string (const IColumn &column, size_t row_num, BufferWritable &ostr), to_string (const IColumn &column, size_t row_num), to_string_batch (const IColumn &column, ColumnString &column_to), from_string (ReadBuffer &rb, IColumn *column) +// this two function should move to DataTypeSerDe and only used in Block +// serialize (const IColumn &column, char *buf, int be_exec_version), deserialize (const char *buf, MutableColumnPtr *column, int be_exec_version) +// 4. compare: equals (const IDataType &rhs), is_comparable + +namespace doris::vectorized { + +static bool gen_check_data_in_assert = true; + +class CommonDataTypeTest : public ::testing::Test { +protected: + // Helper function to load data from CSV, with index which splited by spliter and load to columns + void load_data_from_csv(const DataTypeSerDeSPtrs serders, MutableColumns& columns, + const std::string& file_path, const char spliter = ';', + const std::set idxes = {0}) { + ASSERT_EQ(serders.size(), columns.size()) + << "serder size: " << serders.size() << " column size: " << columns.size(); + ASSERT_EQ(serders.size(), idxes.size()) + << "serder size: " << serders.size() << " idxes size: " << idxes.size(); + std::ifstream file(file_path); + if (!file) { + throw doris::Exception(ErrorCode::INVALID_ARGUMENT, "can not open the file: {} ", + file_path); + } + + std::string line; + DataTypeSerDe::FormatOptions options; + while (std::getline(file, line)) { + std::stringstream lineStream(line); + // std::cout << "whole : " << lineStream.str() << std::endl; + std::string value; + int l_idx = 0; + int c_idx = 0; + while (std::getline(lineStream, value, spliter)) { + if ((!value.starts_with("//") && idxes.contains(l_idx))) { + Slice string_slice(value.data(), value.size()); + if (auto st = serders[c_idx]->deserialize_one_cell_from_json( + *columns[c_idx], string_slice, options); + !st.ok()) { + std::cout << "error in deserialize but continue: " << st.to_string() << std::endl; + } + ++c_idx; + } + ++l_idx; + } + } + } + +public: + // we make meta info a default value, so assert should change the struct value to the right value + struct DataTypeMetaInfo { + TypeIndex type_id = TypeIndex::Nothing; + TypeDescriptor* type_as_type_descriptor = nullptr; + std::string family_name = ""; + bool has_subtypes = false; + doris::FieldType storage_field_type = doris::FieldType::OLAP_FIELD_TYPE_UNKNOWN; + bool should_align_right_in_pretty_formats = false; + bool text_can_contain_only_valid_utf8 = false; + bool have_maximum_size_of_value = false; + size_t size_of_value_in_memory = -1; + size_t precision = -1; + size_t scale = -1; + bool is_null_literal = true; + bool is_value_represented_by_number = false; + PColumnMeta* pColumnMeta = nullptr; + DataTypeSerDeSPtr serde = nullptr; + Field default_field; + }; + void SetUp() override {} + + // meta info assert is simple and can be used for all DataType + void meta_info_assert(DataTypePtr& data_type, DataTypeMetaInfo& meta_info) { + ASSERT_NE(data_type->get_serde(1), nullptr); + ASSERT_EQ(IDataType::get_pdata_type(data_type.get()), meta_info.pColumnMeta->type()); + ASSERT_EQ(data_type->get_type_id(), meta_info.type_id); + ASSERT_EQ(data_type->get_type_as_type_descriptor(), *meta_info.type_as_type_descriptor); + ASSERT_EQ(data_type->get_family_name(), meta_info.family_name); + ASSERT_EQ(data_type->have_subtypes(), meta_info.has_subtypes); + ASSERT_EQ(data_type->get_storage_field_type(), meta_info.storage_field_type); + ASSERT_EQ(data_type->should_align_right_in_pretty_formats(), + meta_info.should_align_right_in_pretty_formats); + ASSERT_EQ(data_type->text_can_contain_only_valid_utf8(), + meta_info.text_can_contain_only_valid_utf8); + ASSERT_EQ(data_type->have_maximum_size_of_value(), meta_info.have_maximum_size_of_value); + if (is_decimal(data_type) || data_type->is_nullable() || is_struct(data_type) + || is_nothing(data_type) || is_number(data_type) || is_columned_as_number(data_type) || is_ip(data_type)) { + ASSERT_EQ(data_type->get_size_of_value_in_memory(), meta_info.size_of_value_in_memory); + } else { + EXPECT_ANY_THROW(EXPECT_FALSE(data_type->get_size_of_value_in_memory())); + } + if (is_decimal(data_type)) { + ASSERT_EQ(data_type->get_precision(), meta_info.precision); + ASSERT_EQ(data_type->get_scale(), meta_info.scale); + } else { + EXPECT_ANY_THROW(EXPECT_FALSE(data_type->get_precision())); + EXPECT_THROW(EXPECT_FALSE(data_type->get_scale()), doris::Exception); + } + ASSERT_EQ(data_type->is_null_literal(), meta_info.is_null_literal); + ASSERT_EQ(data_type->is_value_represented_by_number(), + meta_info.is_value_represented_by_number); + ASSERT_EQ(data_type->get_default(), meta_info.default_field); + } + + // create column assert with default field is simple and can be used for all DataType + void create_column_assert(DataTypePtr& data_type, Field& default_field, size_t uncompressed_serialized_bytes = -1) { + std::cout << "create_column_assert: " << data_type->get_name() << std::endl; + auto column = data_type->create_column(); + ASSERT_EQ(column->size(), 0); + ColumnPtr const_col = data_type->create_column_const(10, default_field); + ASSERT_EQ(const_col->size(), 10); + ColumnPtr default_const_col = data_type->create_column_const_with_default_value(10); + ASSERT_EQ(default_const_col->size(), 10); + for (int i = 0; i < 10; ++i) { + ASSERT_EQ(const_col->operator[](i), default_const_col->operator[](i)); + } + // get_uncompressed_serialized_bytes + ASSERT_EQ(data_type->get_uncompressed_serialized_bytes(*column, 0), uncompressed_serialized_bytes); + } + + // get_field assert is simple and can be used for all DataType + void get_field_assert(DataTypePtr& data_type, TExprNode& node, Field& assert_field, + bool assert_false = false) { + if (assert_false) { + EXPECT_ANY_THROW(data_type->get_field(node)) + << "get_field_assert: " + << " datatype:" + data_type->get_name() << " node_type:" << node.node_type + << " field: " << assert_field.get_type() << std::endl; + } else { + Field field = data_type->get_field(node); + ASSERT_EQ(field, assert_field) + << "get_field_assert: " + << " datatype:" + data_type->get_name() << " node_type:" << node.node_type + << " field: " << assert_field.get_type() << std::endl; + } + } + + // to_string | to_string_batch | from_string assert is simple and can be used for all DataType + void assert_to_string_from_string_assert(MutableColumnPtr mutableColumn, + DataTypePtr& data_type) { + + { + // to_string_batch | from_string + auto col_to = ColumnString::create(); + data_type->to_string_batch(*mutableColumn, *col_to); + ASSERT_EQ(col_to->size(), mutableColumn->size()); + // from_string assert col_to to assert_column and check same with mutableColumn + auto assert_column = data_type->create_column(); + for (int i = 0; i < col_to->size(); ++i) { + std::string s = col_to->get_data_at(i).to_string(); + ReadBuffer rb(s.data(), s.size()); + ASSERT_EQ(Status::OK(), data_type->from_string(rb, assert_column.get())); + ASSERT_EQ(assert_column->operator[](i), mutableColumn->operator[](i)) + << "i: " << i << " s: " << s << " datatype: " << data_type->get_name() + << " assert_column: " << assert_column->get_name() + << " mutableColumn:" << mutableColumn->get_name() << std::endl; + } + } + { + std::cout << "assert to_string from_string is reciprocal: " << data_type->get_name() << std::endl; + // to_string | from_string + auto ser_col = ColumnString::create(); + ser_col->reserve(mutableColumn->size()); + VectorBufferWriter buffer_writer(*ser_col.get()); + for (int i = 0; i < mutableColumn->size(); ++i) { + data_type->to_string(*mutableColumn, i, buffer_writer); + buffer_writer.commit(); + } + // check ser_col to assert_column and check same with mutableColumn + auto assert_column_1 = data_type->create_column(); + for (int i = 0; i < ser_col->size(); ++i) { + std::string s = ser_col->get_data_at(i).to_string(); + ReadBuffer rb(s.data(), s.size()); + ASSERT_EQ(Status::OK(), data_type->from_string(rb, assert_column_1.get())); + ASSERT_EQ(assert_column_1->operator[](i), mutableColumn->operator[](i)); + } + } + } + + // should all datatype is compare? + void assert_compare_behavior(DataTypePtr l_dt, DataTypePtr& r_dt) { + ASSERT_TRUE(l_dt->is_comparable()); + ASSERT_TRUE(r_dt->is_comparable()); + // compare + ASSERT_FALSE(l_dt->equals(*r_dt)); + } +}; + +} // namespace doris::vectorized \ No newline at end of file From d3c00d72cea60ea27ca4367e857a04cdd95515ea Mon Sep 17 00:00:00 2001 From: amorynan Date: Mon, 30 Dec 2024 14:27:46 +0800 Subject: [PATCH 09/23] fix format: --- .../data_types/common_data_type_serder_test.h | 7 +++++-- be/test/vec/data_types/common_data_type_test.h | 18 +++++++++++------- 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/be/test/vec/data_types/common_data_type_serder_test.h b/be/test/vec/data_types/common_data_type_serder_test.h index f8aa9d890e6f3c..c9e83e0cfa4cb0 100644 --- a/be/test/vec/data_types/common_data_type_serder_test.h +++ b/be/test/vec/data_types/common_data_type_serder_test.h @@ -23,6 +23,7 @@ #include #include #include + #include "arrow/type.h" #include "runtime/descriptors.h" #include "util/arrow/block_convertor.h" @@ -370,8 +371,10 @@ class CommonDataTypeSerdeTest : public ::testing::Test { array.get(), 0, column_with_type_and_name.column, column_with_type_and_name.type, rows, _timezone_obj); // do check data - std::cout << "arrow_column_to_doris_column done: " << column_with_type_and_name.column->get_name() - << " with column size: " << column_with_type_and_name.column->size() <get_name() + << " with column size: " << column_with_type_and_name.column->size() + << std::endl; std::cout << assert_block.dump_structure() << std::endl; EXPECT_EQ(Status::OK(), ret) << "convert arrow to block failed" << ret.to_string(); auto& col = block->get_by_position(i).column; diff --git a/be/test/vec/data_types/common_data_type_test.h b/be/test/vec/data_types/common_data_type_test.h index e604474bb87a69..83a8a8f1bee796 100644 --- a/be/test/vec/data_types/common_data_type_test.h +++ b/be/test/vec/data_types/common_data_type_test.h @@ -87,7 +87,8 @@ class CommonDataTypeTest : public ::testing::Test { if (auto st = serders[c_idx]->deserialize_one_cell_from_json( *columns[c_idx], string_slice, options); !st.ok()) { - std::cout << "error in deserialize but continue: " << st.to_string() << std::endl; + std::cout << "error in deserialize but continue: " << st.to_string() + << std::endl; } ++c_idx; } @@ -132,8 +133,9 @@ class CommonDataTypeTest : public ::testing::Test { ASSERT_EQ(data_type->text_can_contain_only_valid_utf8(), meta_info.text_can_contain_only_valid_utf8); ASSERT_EQ(data_type->have_maximum_size_of_value(), meta_info.have_maximum_size_of_value); - if (is_decimal(data_type) || data_type->is_nullable() || is_struct(data_type) - || is_nothing(data_type) || is_number(data_type) || is_columned_as_number(data_type) || is_ip(data_type)) { + if (is_decimal(data_type) || data_type->is_nullable() || is_struct(data_type) || + is_nothing(data_type) || is_number(data_type) || is_columned_as_number(data_type) || + is_ip(data_type)) { ASSERT_EQ(data_type->get_size_of_value_in_memory(), meta_info.size_of_value_in_memory); } else { EXPECT_ANY_THROW(EXPECT_FALSE(data_type->get_size_of_value_in_memory())); @@ -152,7 +154,8 @@ class CommonDataTypeTest : public ::testing::Test { } // create column assert with default field is simple and can be used for all DataType - void create_column_assert(DataTypePtr& data_type, Field& default_field, size_t uncompressed_serialized_bytes = -1) { + void create_column_assert(DataTypePtr& data_type, Field& default_field, + size_t uncompressed_serialized_bytes = -1) { std::cout << "create_column_assert: " << data_type->get_name() << std::endl; auto column = data_type->create_column(); ASSERT_EQ(column->size(), 0); @@ -164,7 +167,8 @@ class CommonDataTypeTest : public ::testing::Test { ASSERT_EQ(const_col->operator[](i), default_const_col->operator[](i)); } // get_uncompressed_serialized_bytes - ASSERT_EQ(data_type->get_uncompressed_serialized_bytes(*column, 0), uncompressed_serialized_bytes); + ASSERT_EQ(data_type->get_uncompressed_serialized_bytes(*column, 0), + uncompressed_serialized_bytes); } // get_field assert is simple and can be used for all DataType @@ -187,7 +191,6 @@ class CommonDataTypeTest : public ::testing::Test { // to_string | to_string_batch | from_string assert is simple and can be used for all DataType void assert_to_string_from_string_assert(MutableColumnPtr mutableColumn, DataTypePtr& data_type) { - { // to_string_batch | from_string auto col_to = ColumnString::create(); @@ -206,7 +209,8 @@ class CommonDataTypeTest : public ::testing::Test { } } { - std::cout << "assert to_string from_string is reciprocal: " << data_type->get_name() << std::endl; + std::cout << "assert to_string from_string is reciprocal: " << data_type->get_name() + << std::endl; // to_string | from_string auto ser_col = ColumnString::create(); ser_col->reserve(mutableColumn->size()); From 221eb1341fbe3f0e309265aa4061f00dcbc803e8 Mon Sep 17 00:00:00 2001 From: amorynan Date: Mon, 30 Dec 2024 15:02:22 +0800 Subject: [PATCH 10/23] fixed --- be/test/vec/columns/column_array_test.cpp | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp index c47e51148e1242..a97d3483b06011 100644 --- a/be/test/vec/columns/column_array_test.cpp +++ b/be/test/vec/columns/column_array_test.cpp @@ -463,8 +463,8 @@ TEST_F(ColumnArrayTest, PermutationAndSortTest) { auto& type = array_types[i]; auto column_type = type->get_name(); std::cout << "column_type: " << column_type << std::endl; - // permutation get_permutation is not support in column_array, compare_at maybe not incorrect - EXPECT_ANY_THROW(assert_column_permutations(column->assume_mutable_ref(), type)); + // permutation + assert_column_permutations(column->assume_mutable_ref(), type); } } @@ -531,7 +531,6 @@ TEST_F(ColumnArrayTest, MetaInfoTest) { auto& column = array_columns[i]; auto& type = array_types[i]; auto column_type = type->get_name(); - std::cout << "column_type: " << column_type << std::endl; EXPECT_TRUE(column->is_variable_length()) << "column is not variable length"; } } @@ -589,18 +588,20 @@ TEST_F(ColumnArrayTest, MaxArraySizeAsFieldTest) { column->get(column->size() - 1, a); Array af = a.get(); if (af.size() > 0) { + auto start_size = af.size(); Field ef = af[0]; for (int j = 0; j < max_array_size_as_field; ++j) { af.push_back(ef); } - std::cout << "array size: " << af.size() << std::endl; + EXPECT_EQ(af.size(), start_size + max_array_size_as_field) + << "array size is not equal to start size + max_array_size_as_field"; auto cloned = column->clone_resized(0); cloned->insert(af); - std::cout << "cloned size: " << cloned->size() << std::endl; // get cloned offset size auto cloned_offset_size = check_and_get_column(cloned.get())->get_offsets().back(); - std::cout << "cloned offset size: " << cloned_offset_size << std::endl; + EXPECT_EQ(cloned_offset_size, start_size + max_array_size_as_field) + << "cloned offset size is not equal to start size + max_array_size_as_field"; Field f; // test get @@ -626,6 +627,11 @@ TEST_F(ColumnArrayTest, IsDefaultAtTest) { column->get(j, f); auto array = f.get(); EXPECT_EQ(array.size(), 0) << "array is not empty"; + } else { + Field f; + column->get(j, f); + auto array = f.get(); + EXPECT_GT(array.size(), 0) << "array is empty"; } } } From e266252337b8e23a3c3b216723ff9e0efcaae510 Mon Sep 17 00:00:00 2001 From: amorynan Date: Mon, 30 Dec 2024 16:33:11 +0800 Subject: [PATCH 11/23] fix datatype ip --- be/test/vec/data_types/data_type_ip_test.cpp | 27 +++++++++++++++++--- 1 file changed, 23 insertions(+), 4 deletions(-) diff --git a/be/test/vec/data_types/data_type_ip_test.cpp b/be/test/vec/data_types/data_type_ip_test.cpp index 80861a70bd1412..c1726b4b6c1126 100644 --- a/be/test/vec/data_types/data_type_ip_test.cpp +++ b/be/test/vec/data_types/data_type_ip_test.cpp @@ -89,7 +89,8 @@ TEST_F(DataTypeIPTest, MetaInfoTest) { .scale = size_t(-1), .is_null_literal = false, .is_value_represented_by_number = true, - .pColumnMeta = col_meta.get() + .pColumnMeta = col_meta.get(), + .default_field = UInt64(0) // .is_value_unambiguously_represented_in_contiguous_memory_region = true }; TypeDescriptor ipv6_type_descriptor = {PrimitiveType::TYPE_IPV6}; @@ -109,7 +110,8 @@ TEST_F(DataTypeIPTest, MetaInfoTest) { .scale = size_t(-1), .is_null_literal = false, .is_value_represented_by_number = true, - .pColumnMeta = col_meta6.get() + .pColumnMeta = col_meta6.get(), + .default_field = Int128(0) // .is_value_unambiguously_represented_in_contiguous_memory_region = true }; meta_info_assert(dt_ipv4, ipv4_meta_info_to_assert); @@ -119,8 +121,8 @@ TEST_F(DataTypeIPTest, MetaInfoTest) { TEST_F(DataTypeIPTest, CreateColumnTest) { Field default_field_ipv4 = IPv4(0); Field default_field_ipv6 = IPv6(0); - create_column_assert(dt_ipv4, default_field_ipv4); - create_column_assert(dt_ipv6, default_field_ipv6); + create_column_assert(dt_ipv4, default_field_ipv4, 4); + create_column_assert(dt_ipv6, default_field_ipv6, 4); } TEST_F(DataTypeIPTest, GetFieldTest) { @@ -241,4 +243,21 @@ TEST_F(DataTypeIPTest, SerdeJsonbTest) { CommonDataTypeSerdeTest::assert_jsonb_format); } +TEST_F(DataTypeIPTest, SerdeMysqlAndArrowTest) { + auto serde_ipv4 = dt_ipv4->get_serde(1); + auto serde_ipv6 = dt_ipv6->get_serde(1); + auto column_ipv4 = dt_ipv4->create_column(); + auto column_ipv6 = dt_ipv6->create_column(); + + // insert from data csv and assert insert result + MutableColumns ip_cols; + ip_cols.push_back(column_ipv4->get_ptr()); + ip_cols.push_back(column_ipv6->get_ptr()); + DataTypeSerDeSPtrs serde = {dt_ipv4->get_serde(), dt_ipv6->get_serde()}; + CommonDataTypeSerdeTest::check_data(ip_cols, serde, ';', {1, 2}, data_files[0], + CommonDataTypeSerdeTest::assert_mysql_format); + + CommonDataTypeSerdeTest::assert_arrow_format(ip_cols, serde, {dt_ipv4, dt_ipv6}); +} + } // namespace doris::vectorized \ No newline at end of file From 076e6795a9bd75bfb4835d27ee3b2791cb3eb88c Mon Sep 17 00:00:00 2001 From: amorynan Date: Mon, 30 Dec 2024 22:03:23 +0800 Subject: [PATCH 12/23] fixed --- be/test/vec/columns/column_array_test.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp index a97d3483b06011..2078d03c74d1a5 100644 --- a/be/test/vec/columns/column_array_test.cpp +++ b/be/test/vec/columns/column_array_test.cpp @@ -464,7 +464,7 @@ TEST_F(ColumnArrayTest, PermutationAndSortTest) { auto column_type = type->get_name(); std::cout << "column_type: " << column_type << std::endl; // permutation - assert_column_permutations(column->assume_mutable_ref(), type); + EXPECT_ANY_THROW(assert_column_permutations(column->assume_mutable_ref(), type)); } } @@ -754,4 +754,4 @@ TEST_F(ColumnArrayTest, IntArrayPermuteTest) { } } -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized From c49cca2fbb217787662f1fb21f0b5216b9870e41 Mon Sep 17 00:00:00 2001 From: amorynan Date: Tue, 31 Dec 2024 18:31:20 +0800 Subject: [PATCH 13/23] change std::cout to log --- be/src/vec/columns/column_array.cpp | 4 +- be/test/vec/columns/column_array_test.cpp | 59 ++++-- be/test/vec/columns/common_column_test.h | 211 ++++++++++------------ 3 files changed, 132 insertions(+), 142 deletions(-) diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp index e143cc252c6c27..c3a23fa3425a25 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -1038,8 +1038,6 @@ ColumnPtr ColumnArray::replicate_generic(const IColumn::Offsets& replicate_offse size_t size_to_replicate = replicate_offsets[i] - prev_offset; prev_offset = replicate_offsets[i]; - std::cout << "replicate_generic: " << i << " " << i << " with prev:" << prev_offset - << " with size_to_replicate: " << size_to_replicate << std::endl; for (size_t j = 0; j < size_to_replicate; ++j) { res_concrete.insert_from(*this, i); } @@ -1107,4 +1105,4 @@ ColumnPtr ColumnArray::permute(const Permutation& perm, size_t limit) const { return res; } -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp index 2078d03c74d1a5..606e574e8d198d 100644 --- a/be/test/vec/columns/column_array_test.cpp +++ b/be/test/vec/columns/column_array_test.cpp @@ -278,7 +278,7 @@ class ColumnArrayTest : public CommonColumnTest { auto& data_file = data_files[i]; // first is array type auto& type = desc[0].data_type; - std::cout << "type: " << type->get_name() << " with file: " << data_file << std::endl; + LOG(INFO) << "type: " << type->get_name() << " with file: " << data_file; MutableColumns columns; columns.push_back(type->create_column()); auto serde = type->get_serde(1); @@ -293,7 +293,7 @@ class ColumnArrayTest : public CommonColumnTest { auto& data_file = data_files[i + array_typeIndex.size()]; // first is array type auto& type = desc[0].data_type; - std::cout << "type: " << type->get_name() << " with file: " << data_file << std::endl; + LOG(INFO) << "type: " << type->get_name() << " with file: " << data_file; MutableColumns columns; columns.push_back(type->create_column()); auto serde = type->get_serde(1); @@ -308,7 +308,7 @@ class ColumnArrayTest : public CommonColumnTest { auto& data_file = data_files[i + array_typeIndex.size() + array_array_typeIndex.size()]; // first is array type auto& type = desc[0].data_type; - std::cout << "type: " << type->get_name() << " with file: " << data_file << std::endl; + LOG(INFO) << "type: " << type->get_name() << " with file: " << data_file; MutableColumns columns; columns.push_back(type->create_column()); auto serde = type->get_serde(1); @@ -325,7 +325,7 @@ class ColumnArrayTest : public CommonColumnTest { array_map_typeIndex.size()]; // first is array type auto& type = desc[0].data_type; - std::cout << "type: " << type->get_name() << " with file: " << data_file << std::endl; + LOG(INFO) << "type: " << type->get_name() << " with file: " << data_file; MutableColumns columns; columns.push_back(type->create_column()); auto serde = type->get_serde(1); @@ -438,13 +438,18 @@ TEST_F(ColumnArrayTest, ReplicateTest) { // just skip array_array_char use vector copy for (int i = 0; i < array_columns.size(); i++) { if (i == 31) { - std::cout << array_columns[i]->get_name() << " is skipped" << std::endl; continue; } array_columns_copy.push_back(array_columns[i]->assume_mutable()); serdes_copy.push_back(serdes[i]); } assert_replicate_callback(array_columns_copy, serdes_copy); + // expect error columns + MutableColumns error_columns; + error_columns.push_back(array_columns[31]->assume_mutable()); + DataTypeSerDeSPtrs error_serdes; + error_serdes.push_back(serdes[31]); + EXPECT_ANY_THROW(assert_replicate_callback(error_columns, error_serdes)); } TEST_F(ColumnArrayTest, ReplaceColumnTest) { @@ -462,7 +467,7 @@ TEST_F(ColumnArrayTest, PermutationAndSortTest) { auto& column = array_columns[i]; auto& type = array_types[i]; auto column_type = type->get_name(); - std::cout << "column_type: " << column_type << std::endl; + LOG(INFO) << "column_type: " << column_type; // permutation EXPECT_ANY_THROW(assert_column_permutations(column->assume_mutable_ref(), type)); } @@ -505,12 +510,14 @@ TEST_F(ColumnArrayTest, CreateArrayTest) { auto& type = array_types[i]; auto column_size = column->size(); auto column_type = type->get_name(); - std::cout << "column_type: " << column_type << std::endl; + LOG(INFO) << "column_type: " << column_type; // test create_array auto last_offset = column->get_offsets().back(); EXPECT_ANY_THROW( { auto const_col = ColumnConst::create(column->get_data_ptr(), last_offset); }); auto tmp_data_col = column->get_data_ptr()->clone_resized(1); + Field assert_field; + column->get(0, assert_field); auto const_col = ColumnConst::create(tmp_data_col->assume_mutable(), last_offset); EXPECT_ANY_THROW({ // const_col is not empty @@ -522,6 +529,12 @@ TEST_F(ColumnArrayTest, CreateArrayTest) { << "array_column size is not equal to column size"; EXPECT_EQ(new_array_column->get_data_ptr()->size(), column->get_data_ptr()->size()); EXPECT_EQ(new_array_column->get_offsets_ptr()->size(), column->get_offsets_ptr()->size()); + // check column data + for (size_t j = 0; j < column_size; j++) { + Field f1; + new_array_column->get(j, f1); + EXPECT_EQ(f1, assert_field) << "array_column data is not equal to column data"; + } } } @@ -543,18 +556,24 @@ TEST_F(ColumnArrayTest, ConvertIfOverflowAndInsertTest) { auto type = array_types[i]; auto nested_type = assert_cast(remove_nullable(type).get())->get_nested_type(); - if (!is_string(nested_type)) { - // check ptr is itself - auto ptr = column->convert_column_if_overflow(); - EXPECT_EQ(ptr.get(), column.get()); - auto arr_col = check_and_get_column( - remove_nullable(column->assume_mutable()).get()); - auto nested_col = arr_col->get_data_ptr(); - auto array_col1 = check_and_get_column(remove_nullable(ptr).get()); - auto nested_col1 = array_col1->get_data_ptr(); - EXPECT_EQ(nested_col.get(), nested_col1.get()); - } else { - auto ptr = column->convert_column_if_overflow(); + // check ptr is itself + auto ptr = column->convert_column_if_overflow(); + EXPECT_EQ(ptr.get(), column.get()); + auto arr_col = + check_and_get_column(remove_nullable(column->assume_mutable()).get()); + auto nested_col = arr_col->get_data_ptr(); + auto array_col1 = check_and_get_column(remove_nullable(ptr).get()); + auto nested_col1 = array_col1->get_data_ptr(); + EXPECT_EQ(nested_col.get(), nested_col1.get()); + // check column data + auto column_size = column->size(); + LOG(INFO) << "column_size: " << column_size; + for (size_t j = 0; j < column_size; j++) { + Field f1; + column->get(j, f1); + Field f2; + ptr->get(j, f2); + EXPECT_EQ(f1, f2) << "array_column data is not equal to column data"; } } } @@ -754,4 +773,4 @@ TEST_F(ColumnArrayTest, IntArrayPermuteTest) { } } -} // namespace doris::vectorized +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/test/vec/columns/common_column_test.h b/be/test/vec/columns/common_column_test.h index db67301e86a1ee..a8888d7a43a817 100644 --- a/be/test/vec/columns/common_column_test.h +++ b/be/test/vec/columns/common_column_test.h @@ -67,19 +67,16 @@ class CommonColumnTest : public ::testing::Test { DataTypeSerDe::FormatOptions options; while (std::getline(file, line)) { std::stringstream lineStream(line); - // std::cout << "whole : " << lineStream.str() << std::endl; std::string value; int l_idx = 0; int c_idx = 0; while (std::getline(lineStream, value, spliter)) { if (!value.starts_with("//") && idxes.contains(l_idx)) { Slice string_slice(value.data(), value.size()); - // std::cout << string_slice << std::endl; if (auto st = serders[c_idx]->deserialize_one_cell_from_json( *columns[c_idx], string_slice, options); !st.ok()) { - // std::cout << "error in deserialize but continue: " << st.to_string() - // << std::endl; + LOG(INFO) << "error in deserialize but continue: " << st.to_string(); } ++c_idx; } @@ -97,8 +94,7 @@ class CommonColumnTest : public ::testing::Test { string filename = "./res_" + function_name + ".csv"; if (gen_check_data_in_assert) { std::ofstream res_file(filename); - std::cout << "gen check data: " << res.size() << " with file: " << filename - << std::endl; + LOG(INFO) << "gen check data: " << res.size() << " with file: " << filename; if (!res_file.is_open()) { throw std::ios_base::failure("Failed to open file."); } @@ -117,7 +113,7 @@ class CommonColumnTest : public ::testing::Test { res_file.close(); } else { // we read generate file to check result - std::cout << "check data: " << res.size() << " with file: " << filename << std::endl; + LOG(INFO) << "check data: " << res.size() << " with file: " << filename; std::ifstream file(filename); if (!file) { throw doris::Exception(ErrorCode::INVALID_ARGUMENT, "can not open the file: {} ", @@ -218,7 +214,7 @@ class CommonColumnTest : public ::testing::Test { std::filesystem::path fs_path(column_data_file); for (const auto& entry : std::filesystem::directory_iterator(fs_path)) { std::string file_path = entry.path().string(); - std::cout << "load data from file: " << file_path << std::endl; + LOG(INFO) << "load data from file: " << file_path; load_data_from_csv(serders, columns, file_path, col_spliter, idxes); } } @@ -257,8 +253,7 @@ class CommonColumnTest : public ::testing::Test { } void printColumn(const IColumn& column, const IDataType& dataType) { - std::cout << "colum: " << column.get_name() << " total size: " << column.size() - << std::endl; + LOG(INFO) << "colum: " << column.get_name() << " total size: " << column.size(); auto serde = dataType.get_serde(0); auto serde_col = ColumnString::create(); auto option = DataTypeSerDe::FormatOptions(); @@ -267,11 +262,11 @@ class CommonColumnTest : public ::testing::Test { for (size_t i = 0; i < column.size(); ++i) { if (auto st = serde->serialize_one_cell_to_json(column, i, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column at row " << i << std::endl; + LOG(ERROR) << "Failed to serialize column at row " << i; break; } buffer_writer.commit(); - std::cout << serde_col->get_data_at(i).to_string() << std::endl; + LOG(INFO) << serde_col->get_data_at(i).to_string(); } } @@ -303,7 +298,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*target_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -332,8 +327,8 @@ class CommonColumnTest : public ::testing::Test { for (auto cl = check_length.begin(); cl < check_length.end(); ++cl) { for (size_t i = 0; i < load_cols.size(); ++i) { std::vector data; - std::cout << "==== now test insert_range_from with col " - << verify_columns[i]->get_name() << std::endl; + LOG(INFO) << "==== now test insert_range_from with col " + << verify_columns[i]->get_name(); auto& source_column = load_cols[i]; auto& target_column = verify_columns[i]; std::vector check_start_pos = {0, source_column->size(), @@ -341,16 +336,15 @@ class CommonColumnTest : public ::testing::Test { // size_t(-1) may cause overflow, but here we have compiler to check it std::vector err_start_pos = {source_column->size() + 1}; for (auto pos = err_start_pos.begin(); pos < err_start_pos.end(); ++pos) { - std::cout << "error insert_range_from from " << *pos << " with length " << *cl - << std::endl; - std::cout << *pos + *cl << " > " << source_column->size() << std::endl; + LOG(INFO) << "error insert_range_from from " << *pos << " with length " << *cl + << *pos + *cl << " > " << source_column->size(); EXPECT_THROW(target_column->insert_range_from(*source_column, *pos, *cl), Exception); } for (auto pos = check_start_pos.begin(); pos < check_start_pos.end(); ++pos) { target_column->clear(); - std::cout << "now insert_range_from from " << *pos << " with length " << *cl - << " with source size: " << source_column->size() << std::endl; + LOG(INFO) << "now insert_range_from from " << *pos << " with length " << *cl + << " with source size: " << source_column->size(); if (*pos + *cl > source_column->size()) { EXPECT_THROW(target_column->insert_range_from(*source_column, *pos, *cl), Exception); @@ -367,8 +361,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*target_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -389,7 +382,7 @@ class CommonColumnTest : public ::testing::Test { static void assert_insert_many_from_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { // Create an empty column to verify `insert_many_from` functionality - std::cout << "now we are in assert_insert_many_from_callback" << std::endl; + LOG(INFO) << "now we are in assert_insert_many_from_callback"; MutableColumns verify_columns; for (auto& col : load_cols) { verify_columns.push_back(col->clone_empty()); @@ -419,10 +412,10 @@ class CommonColumnTest : public ::testing::Test { } target_column->clear(); // insert_range_from now we have no any exception error data to handle and also no crash - std::cout << "we expect exception insert_many_from from " << *pos + LOG(INFO) << "we expect exception insert_many_from from " << *pos << " with length " << *cl << " for column " << source_column->get_name() - << " with source size: " << source_column->size() << std::endl; + << " with source size: " << source_column->size(); target_column->insert_many_from(*source_column, *pos, *cl); } @@ -435,8 +428,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*target_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -483,9 +475,9 @@ class CommonColumnTest : public ::testing::Test { // here we will meet `heap-buffer-overflow on address` continue; } else { - std::cout << source_column->get_name() << " now insert_indices_from from " + LOG(INFO) << source_column->get_name() << " now insert_indices_from from " << *from_idx << " to " << *end_idx - << " with source size: " << source_column->size() << std::endl; + << " with source size: " << source_column->size(); target_column->insert_indices_from(*source_column, &(*from_idx), &(*end_idx)); } @@ -498,8 +490,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*target_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -541,7 +532,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*target_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } std::string actual_str_value = ser_col->get_data_at(j).to_string(); @@ -580,7 +571,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*target_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } std::string actual_str_value = ser_col->get_data_at(j).to_string(); @@ -616,7 +607,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*target_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -655,8 +646,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*target_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -704,12 +694,11 @@ class CommonColumnTest : public ::testing::Test { // just check cols get is the same as assert_res for (size_t i = 0; i < load_cols.size(); ++i) { auto& source_column = load_cols[i]; - std::cout << " new insert field for column : " << assert_cols[i]->get_name() - << " with size : " << assert_cols[i]->size() << std::endl; + LOG(INFO) << " new insert field for column : " << assert_cols[i]->get_name() + << " with size : " << assert_cols[i]->size(); for (size_t j = 0; j < source_column->size(); ++j) { Field f; source_column->get(j, f); - std::cout << "field: " << f.get_type() << std::endl; assert_cols[i]->insert(f); } } @@ -723,7 +712,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*assert_cols[i], j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -753,7 +742,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*assert_cols[i], j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -772,7 +761,7 @@ class CommonColumnTest : public ::testing::Test { static void assert_get_raw_data_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { // just check cols get_raw_data is the same as assert_res - std::cout << "now we are in assert_get_raw_data_callback" << std::endl; + LOG(INFO) << "now we are in assert_get_raw_data_callback"; std::vector> res; MutableColumns assert_cols(load_cols.size()); for (size_t i = 0; i < load_cols.size(); ++i) { @@ -801,7 +790,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*assert_cols[i], j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -902,7 +891,7 @@ class CommonColumnTest : public ::testing::Test { static void assert_allocated_bytes_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { // just check cols allocated_bytes is the same as assert_res - std::cout << "now we are in assert_allocated_bytes_callback" << std::endl; + LOG(INFO) << "now we are in assert_allocated_bytes_callback"; std::vector> res; for (size_t i = 0; i < load_cols.size(); ++i) { std::vector data; @@ -933,13 +922,13 @@ class CommonColumnTest : public ::testing::Test { if (*cl > source_column->size()) { // now we do not check in popback, but we should make sure the arguments are passed correctly, // otherwise we should meet `Check failed: false Amount of memory requested to allocate is more than allowed` - std::cout << "now we are in pop_back column : " << load_cols[i]->get_name() - << " for column size : " << source_column->size() << std::endl; + LOG(INFO) << "now we are in pop_back column : " << load_cols[i]->get_name() + << " for column size : " << source_column->size(); source_column->pop_back(source_column->size()); } else { - std::cout << "now we are in pop_back column : " << load_cols[i]->get_name() + LOG(INFO) << "now we are in pop_back column : " << load_cols[i]->get_name() << " with check length: " << *cl - << " for column size : " << source_column->size() << std::endl; + << " for column size : " << source_column->size(); source_column->pop_back(*cl); } @@ -952,8 +941,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1005,8 +993,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*ptr, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1038,9 +1025,9 @@ class CommonColumnTest : public ::testing::Test { size_t insert_size = *cl > source_column->size() ? source_column->size() : *cl; // now we do not check in cut, but we should make sure the arguments are passed correctly, // otherwise we should meet `Check failed: false Amount of memory requested to allocate is more than allowed` - std::cout << "now we are in cut column : " << load_cols[i]->get_name() + LOG(INFO) << "now we are in cut column : " << load_cols[i]->get_name() << " with check length: " << insert_size - << " for column size : " << source_column->size() << std::endl; + << " for column size : " << source_column->size(); ptr = source_column->cut(0, insert_size); // check size EXPECT_EQ(ptr->size(), insert_size); @@ -1057,8 +1044,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*ptr, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1090,11 +1076,11 @@ class CommonColumnTest : public ::testing::Test { size_t insert_size = *cl > source_column->size() ? source_column->size() : *cl; // now we do not check in cut, but we should make sure the arguments are passed correctly, // otherwise we should meet `Check failed: false Amount of memory requested to allocate is more than allowed` - std::cout << "now we are in shrink column : " << load_cols[i]->get_name() + LOG(INFO) << "now we are in shrink column : " << load_cols[i]->get_name() << " with check length: " << insert_size - << " for column size : " << source_column->size() << std::endl; + << " for column size : " << source_column->size(); ptr = source_column->shrink(insert_size); - std::cout << "use_count : " << source_column->use_count() << std::endl; + LOG(INFO) << "use_count : " << source_column->use_count(); // check size EXPECT_EQ(ptr->size(), insert_size); // check ptr is not the same @@ -1110,8 +1096,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*ptr, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1138,9 +1123,9 @@ class CommonColumnTest : public ::testing::Test { for (size_t i = 0; i < load_cols.size(); ++i) { auto assert_origin_size = load_cols[i]->size(); auto& source_column = load_cols[i]; - std::cout << "now we are in reserve column : " << load_cols[i]->get_name() + LOG(INFO) << "now we are in reserve column : " << load_cols[i]->get_name() << " with check length: " << *cl - << " for column size : " << source_column->size() << std::endl; + << " for column size : " << source_column->size(); source_column->reserve(*cl); // check size no changed after reserve EXPECT_EQ(source_column->size(), assert_origin_size); @@ -1156,8 +1141,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1182,9 +1166,9 @@ class CommonColumnTest : public ::testing::Test { for (auto cl = check_length.begin(); cl < check_length.end(); ++cl) { for (size_t i = 0; i < load_cols.size(); ++i) { auto& source_column = load_cols[i]; - std::cout << "now we are in resize column : " << load_cols[i]->get_name() + LOG(INFO) << "now we are in resize column : " << load_cols[i]->get_name() << " with check length: " << *cl - << " for column size : " << source_column->size() << std::endl; + << " for column size : " << source_column->size(); source_column->resize(*cl); // check size EXPECT_EQ(source_column->size(), *cl); @@ -1200,8 +1184,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1240,8 +1223,8 @@ class CommonColumnTest : public ::testing::Test { EXPECT_ANY_THROW(load_cols[i]->replicate(offsets)); } auto source_column = load_cols[i]->shrink(check_length.size()); - std::cout << "now we are in replicate column : " << load_cols[i]->get_name() - << " for column size : " << source_column->size() << std::endl; + LOG(INFO) << "now we are in replicate column : " << load_cols[i]->get_name() + << " for column size : " << source_column->size(); // auto ptr = const_col->convert_to_full_column(); // here will return different ptr // record replicate cost time @@ -1249,7 +1232,7 @@ class CommonColumnTest : public ::testing::Test { auto ptr = source_column->replicate(offsets); auto end = std::chrono::high_resolution_clock::now(); auto duration = std::chrono::duration_cast(end - start); - std::cout << "replicate cost time: " << duration.count() << "ms" << std::endl; + LOG(INFO) << "replicate cost time: " << duration.count() << "ms"; // check ptr EXPECT_NE(ptr.get(), source_column.get()); // check after replicate with assert_res @@ -1264,7 +1247,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*ptr, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1298,9 +1281,9 @@ class CommonColumnTest : public ::testing::Test { // if replace row is bigger than the source column size here meet pod coredump continue; } - std::cout << "now we are in replace_column_data column : " + LOG(INFO) << "now we are in replace_column_data column : " << load_cols[i]->get_name() << " with check length: " << *cl - << " for column size : " << source_column->size() << std::endl; + << " for column size : " << source_column->size(); source_column->replace_column_data(*source_column, *cl); // check after replace_column_data: the first data is same with the data in source column's *cl row auto ser_col = ColumnString::create(); @@ -1314,8 +1297,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1343,9 +1325,9 @@ class CommonColumnTest : public ::testing::Test { const NullMap null_map = {1, 1, 0, 1}; for (size_t i = 0; i < load_cols.size(); ++i) { auto& source_column = load_cols[i]; - std::cout << "now we are in replace_column_null_data column : " + LOG(INFO) << "now we are in replace_column_null_data column : " << load_cols[i]->get_name() << " for column size : " << source_column->size() - << "with nullmap" << null_map.data() << std::endl; + << "with nullmap" << null_map.data(); source_column->replace_column_null_data(null_map.data()); // check after replace_column_null_data: 1 in nullmap present the load cols data is null and data should be default value @@ -1360,7 +1342,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1388,9 +1370,9 @@ class CommonColumnTest : public ::testing::Test { // selector size should bigger than begin and end , // because selector[i], i in range(begin,end), Make a DCHECK for this const ColumnArray::Selector selector = {1, 2, 3, 0}; - std::cout << "now we are in append_data_by_selector column : " + LOG(INFO) << "now we are in append_data_by_selector column : " << load_cols[i]->get_name() << " for column size : " << source_column->size() - << " with selector size: " << selector.size() << std::endl; + << " with selector size: " << selector.size(); source_column->append_data_by_selector(res_col, selector, 0, 4); // check after append_data_by_selector: 1 in selector present the load cols data is selected and data should be default value auto ser_col = ColumnString::create(); @@ -1404,7 +1386,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*res_col, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1448,8 +1430,8 @@ class CommonColumnTest : public ::testing::Test { ColumnArray::Filter invalid_filter(source_size - 1, 1); invalid_filter.emplace_back(-1); std::vector data; - std::cout << "now we are in filter column : " << load_cols[i]->get_name() - << " for column size : " << source_column->size() << std::endl; + LOG(INFO) << "now we are in filter column : " << load_cols[i]->get_name() + << " for column size : " << source_column->size(); { auto ptr = source_column->filter(all_filtered, source_column->size()); // check filter res @@ -1462,8 +1444,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*ptr, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1486,8 +1467,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*ptr, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1510,8 +1490,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*ptr, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j - << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1538,9 +1517,8 @@ class CommonColumnTest : public ::testing::Test { for (size_t i = 0; i < load_cols.size(); ++i) { auto& source_column = load_cols[i]; std::vector xx_hash_vals(source_column->size()); - std::cout << "now we are in update_hashes_with_value column : " - << load_cols[i]->get_name() << " for column size : " << source_column->size() - << std::endl; + LOG(INFO) << "now we are in update_hashes_with_value column : " + << load_cols[i]->get_name() << " for column size : " << source_column->size(); auto* __restrict xx_hashes = xx_hash_vals.data(); EXPECT_NO_FATAL_FAILURE(source_column->update_hashes_with_value(xx_hashes)); // check after update_hashes_with_value: 1 in selector present the load cols data is selected and data should be default value @@ -1568,8 +1546,8 @@ class CommonColumnTest : public ::testing::Test { for (size_t i = 0; i < load_cols.size(); ++i) { auto& source_column = load_cols[i]; std::vector crc_hash_vals(source_column->size()); - std::cout << "now we are in update_hashes column : " << load_cols[i]->get_name() - << " for column size : " << source_column->size() << std::endl; + LOG(INFO) << "now we are in update_hashes column : " << load_cols[i]->get_name() + << " for column size : " << source_column->size(); EXPECT_NO_FATAL_FAILURE(source_column->update_crcs_with_value( crc_hash_vals.data(), pts[i], source_column->size())); // check after update_hashes: 1 in selector present the load cols data is selected and data should be default value @@ -1600,8 +1578,8 @@ class CommonColumnTest : public ::testing::Test { std::vector> res; for (size_t i = 0; i < load_cols.size(); i++) { auto& source_column = load_cols[i]; - std::cout << "now we are in shrink_padding_chars column : " << load_cols[i]->get_name() - << " for column size : " << source_column->size() << std::endl; + LOG(INFO) << "now we are in shrink_padding_chars column : " << load_cols[i]->get_name() + << " for column size : " << source_column->size(); source_column->shrink_padding_chars(); // check after get_shrinked_column: 1 in selector present the load cols data is selected and data should be default value auto ser_col = ColumnString::create(); @@ -1614,7 +1592,7 @@ class CommonColumnTest : public ::testing::Test { if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, buffer_writer, option); !st) { - std::cerr << "Failed to serialize column " << i << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; break; } buffer_writer.commit(); @@ -1697,12 +1675,10 @@ class CommonColumnTest : public ::testing::Test { size_t rows = columns[0]->size(); for (auto& column : columns) { if (column->size() > rows) { - std::cerr << "Column size mismatch: " << column->size() << " vs " << rows - << std::endl; + LOG(ERROR) << "Column size mismatch: " << column->size() << " vs " << rows; column->pop_back(column->size() - rows); } else if (column->size() < rows) { - std::cerr << "Column size mismatch: " << column->size() << " vs " << rows - << std::endl; + LOG(ERROR) << "Column size mismatch: " << column->size() << " vs " << rows; column->insert_many_defaults(rows - column->size()); } } @@ -1759,12 +1735,10 @@ class CommonColumnTest : public ::testing::Test { int c = 0; for (auto& column : columns) { if (column->size() > rows) { - std::cerr << "Column size mismatch: " << column->size() << " vs " << rows - << std::endl; + LOG(ERROR) << "Column size mismatch: " << column->size() << " vs " << rows; column->pop_back(column->size() - rows); } else if (column->size() < rows) { - std::cerr << "Column size mismatch: " << column->size() << " vs " << rows - << std::endl; + LOG(ERROR) << "Column size mismatch: " << column->size() << " vs " << rows; column->insert_many_defaults(rows - column->size()); } check_columns[c] = column->clone_empty(); @@ -1788,10 +1762,10 @@ class CommonColumnTest : public ::testing::Test { reinterpret_cast(serialized_key_buffer + i * max_one_row_byte_size); input_keys[i].size = 0; } - std::cout << "max_one_row_byte_size : " << max_one_row_byte_size << std::endl; + LOG(INFO) << "max_one_row_byte_size : " << max_one_row_byte_size; for (const auto& column : columns) { - std::cout << "now serialize_vec for column:" << column->get_name() - << " with column size: " << column->size() << std::endl; + LOG(INFO) << "now serialize_vec for column:" << column->get_name() + << " with column size: " << column->size(); column->serialize_vec(input_keys, rows, max_one_row_byte_size); } } @@ -1837,7 +1811,7 @@ class CommonColumnTest : public ::testing::Test { DataTypeSerDe::FormatOptions option; col->clear(); col->insert_many_defaults(should_sel_col->size()); - std::cout << "col size:" << col->size() << std::endl; + LOG(INFO) << "col size:" << col->size(); Status st = col->filter_by_selector(selector.data(), expect_size, should_sel_col.get()); EXPECT_EQ(st, Status::OK()); std::vector> res; @@ -1852,8 +1826,7 @@ class CommonColumnTest : public ::testing::Test { if (auto ret = serder->serialize_one_cell_to_json(*should_sel_col, j, buffer_writer, option); !ret) { - std::cerr << "Failed to serialize column " - << " at row " << j << std::endl; + LOG(ERROR) << "Failed to serialize column " << j; break; } buffer_writer.commit(); @@ -1866,8 +1839,8 @@ class CommonColumnTest : public ::testing::Test { void assert_permutations_with_limit(const IColumn::Permutation& lhs, const IColumn::Permutation& rhs, size_t limit) { - std::cout << "lhs size: " << lhs.size() << " rhs size: " << rhs.size() - << " limit: " << limit << std::endl; + LOG(INFO) << "lhs size: " << lhs.size() << " rhs size: " << rhs.size() + << " limit: " << limit; if (limit == 0) { limit = lhs.size(); } @@ -1912,7 +1885,7 @@ class CommonColumnTest : public ::testing::Test { size_t column_size = column.size(); size_t column_limit_part = (column_size / limit_parts) + 1; - std::cout << "column size: " << column_size << std::endl; + LOG(INFO) << "column size: " << column_size; for (size_t limit = 0; limit < column_size; limit += column_limit_part) { assert_column_permutation(column, true, limit, -1, actual_permutation, expected_permutation); @@ -1933,11 +1906,11 @@ class CommonColumnTest : public ::testing::Test { void assert_column_permutation(const IColumn& column, bool ascending, size_t limit, int nan_direction_hint, IColumn::Permutation& actual_permutation, IColumn::Permutation& expected_permutation) { - std::cout << "assertColumnPermutation start, limit: " << limit + LOG(INFO) << "assertColumnPermutation start, limit: " << limit << " ascending: " << ascending << " nan_direction_hint: " << nan_direction_hint << " column size: " << column.size() << " actual_permutation size: " << actual_permutation.size() - << " expected_permutation size: " << expected_permutation.size() << std::endl; + << " expected_permutation size: " << expected_permutation.size(); // step1. get expect permutation as stabled sort stable_get_column_permutation(column, ascending, limit, nan_direction_hint, expected_permutation); @@ -1950,7 +1923,7 @@ class CommonColumnTest : public ::testing::Test { // step3. check the permutation result assert_permutations_with_limit(actual_permutation, expected_permutation, limit); - std::cout << "assertColumnPermutation done" << std::endl; + LOG(INFO) << "assertColumnPermutation done"; } // permute() From af2c07324fc79a6b66c383215183a0d3f6ebf3eb Mon Sep 17 00:00:00 2001 From: amorynan Date: Tue, 31 Dec 2024 18:33:47 +0800 Subject: [PATCH 14/23] del useless code: : --- be/test/vec/columns/column_array_test.cpp | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp index 606e574e8d198d..02d48c4dff840a 100644 --- a/be/test/vec/columns/column_array_test.cpp +++ b/be/test/vec/columns/column_array_test.cpp @@ -335,11 +335,6 @@ class ColumnArrayTest : public CommonColumnTest { serdes.push_back(serde); } - // step3. show array column data - for (int i = 0; i < array_columns.size(); i++) { - // auto& column = array_columns[i]; - // printColumn(*column, *descs[i][0].data_type); - } } MutableColumns array_columns; // column_array list @@ -773,4 +768,4 @@ TEST_F(ColumnArrayTest, IntArrayPermuteTest) { } } -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized From a0b1efe20c1d30801cecef23faf394c4cffa5770 Mon Sep 17 00:00:00 2001 From: amorynan Date: Tue, 31 Dec 2024 18:34:21 +0800 Subject: [PATCH 15/23] fix format --- be/test/vec/columns/column_array_test.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp index 02d48c4dff840a..ad3f2fb7c9b23f 100644 --- a/be/test/vec/columns/column_array_test.cpp +++ b/be/test/vec/columns/column_array_test.cpp @@ -334,7 +334,6 @@ class ColumnArrayTest : public CommonColumnTest { array_types.push_back(type); serdes.push_back(serde); } - } MutableColumns array_columns; // column_array list From fd79b2fd134be131e08a74d72b3945494921e7b2 Mon Sep 17 00:00:00 2001 From: amorynan Date: Wed, 1 Jan 2025 10:42:39 +0800 Subject: [PATCH 16/23] fix datatypeip --- be/test/vec/data_types/data_type_ip_test.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/be/test/vec/data_types/data_type_ip_test.cpp b/be/test/vec/data_types/data_type_ip_test.cpp index c1726b4b6c1126..e04a5e6de8a7bf 100644 --- a/be/test/vec/data_types/data_type_ip_test.cpp +++ b/be/test/vec/data_types/data_type_ip_test.cpp @@ -111,7 +111,7 @@ TEST_F(DataTypeIPTest, MetaInfoTest) { .is_null_literal = false, .is_value_represented_by_number = true, .pColumnMeta = col_meta6.get(), - .default_field = Int128(0) + .default_field = IPv6(0) // .is_value_unambiguously_represented_in_contiguous_memory_region = true }; meta_info_assert(dt_ipv4, ipv4_meta_info_to_assert); @@ -260,4 +260,4 @@ TEST_F(DataTypeIPTest, SerdeMysqlAndArrowTest) { CommonDataTypeSerdeTest::assert_arrow_format(ip_cols, serde, {dt_ipv4, dt_ipv6}); } -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized From cd28ffbbf3c5ff8a1548887489306b2457afc7f6 Mon Sep 17 00:00:00 2001 From: amorynan Date: Wed, 1 Jan 2025 10:43:20 +0800 Subject: [PATCH 17/23] del useless file --- .../data_types/common_data_type_serder_test.h | 394 ------------------ 1 file changed, 394 deletions(-) delete mode 100644 be/src/vec/data_types/common_data_type_serder_test.h diff --git a/be/src/vec/data_types/common_data_type_serder_test.h b/be/src/vec/data_types/common_data_type_serder_test.h deleted file mode 100644 index d1f3e674e51a73..00000000000000 --- a/be/src/vec/data_types/common_data_type_serder_test.h +++ /dev/null @@ -1,394 +0,0 @@ -// 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 -#include -#include -#include - -#include -#include -#include - -#include "arrow/type.h" -#include "runtime/descriptors.h" -#include "util/arrow/block_convertor.h" -#include "util/arrow/row_batch.h" -#include "vec/columns/column.h" -#include "vec/core/field.h" -#include "vec/core/types.h" -#include "vec/data_types/data_type.h" -#include "vec/utils/arrow_column_to_doris_column.h" - -// this test is gonna to be a data type serialize and deserialize functions -// such as -// 1. standard hive text ser-deserialize -// deserialize_one_cell_from_hive_text (IColumn &column, Slice &slice, const FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const -// deserialize_column_from_hive_text_vector (IColumn &column, std::vector< Slice > &slices, int *num_deserialized, const FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const -// serialize_one_cell_to_hive_text (const IColumn &column, int row_num, BufferWritable &bw, FormatOptions &options, int hive_text_complex_type_delimiter_level=1) const -// 2. json format ser-deserialize which used in table not in doris database -// serialize_one_cell_to_json (const IColumn &column, int row_num, BufferWritable &bw, FormatOptions &options) const =0 -// serialize_column_to_json (const IColumn &column, int start_idx, int end_idx, BufferWritable &bw, FormatOptions &options) const =0 -// deserialize_one_cell_from_json (IColumn &column, Slice &slice, const FormatOptions &options) const =0 -// deserialize_column_from_json_vector (IColumn &column, std::vector< Slice > &slices, int *num_deserialized, const FormatOptions &options) const =0 -// deserialize_column_from_fixed_json (IColumn &column, Slice &slice, int rows, int *num_deserialized, const FormatOptions &options) const -// insert_column_last_value_multiple_times (IColumn &column, int times) const -// 3. fe|be protobuffer ser-deserialize -// write_column_to_pb (const IColumn &column, PValues &result, int start, int end) const =0 -// read_column_from_pb (IColumn &column, const PValues &arg) const =0 -// 4. jsonb ser-deserialize which used in row-store situation -// write_one_cell_to_jsonb (const IColumn &column, JsonbWriter &result, Arena *mem_pool, int32_t col_id, int row_num) const =0 -// read_one_cell_from_jsonb (IColumn &column, const JsonbValue *arg) const =0 -// 5. mysql text ser-deserialize -// write_column_to_mysql (const IColumn &column, MysqlRowBuffer< false > &row_buffer, int row_idx, bool col_const, const FormatOptions &options) const =0 -// write_column_to_mysql (const IColumn &column, MysqlRowBuffer< true > &row_buffer, int row_idx, bool col_const, const FormatOptions &options) const =0 -// 6. arrow ser-deserialize which used in spark-flink connector -// write_column_to_arrow (const IColumn &column, const NullMap *null_map, arrow::ArrayBuilder *array_builder, int start, int end, const cctz::time_zone &ctz) const =0 -// read_column_from_arrow (IColumn &column, const arrow::Array *arrow_array, int start, int end, const cctz::time_zone &ctz) const =0 -// 7. rapidjson ser-deserialize -// write_one_cell_to_json (const IColumn &column, rapidjson::Value &result, rapidjson::Document::AllocatorType &allocator, Arena &mem_pool, int row_num) const -// read_one_cell_from_json (IColumn &column, const rapidjson::Value &result) const -// convert_field_to_rapidjson (const vectorized::Field &field, rapidjson::Value &target, rapidjson::Document::AllocatorType &allocator) -// convert_array_to_rapidjson (const vectorized::Array &array, rapidjson::Value &target, rapidjson::Document::AllocatorType &allocator) - -namespace doris::vectorized { - -class CommonDataTypeSerdeTest : public ::testing::Test { -public: - ////================================================================================================================== - // this is common function to check data in column against expected results according different function in assert function - // which can be used in all column test - // such as run regress tests - // step1. we can set gen_check_data_in_assert to true, then we will generate a file for check data, otherwise we will read the file to check data - // step2. we should write assert callback function to check data - static void check_data( - MutableColumns& columns, DataTypeSerDeSPtrs serders, char col_spliter, - std::set idxes, const std::string& column_data_file, - std::function - assert_callback, - bool is_hive_format = false, DataTypes dataTypes = {}) { - ASSERT_EQ(serders.size(), columns.size()); - // Step 1: Insert data from `column_data_file` into the column and check result with `check_data_file` - // Load column data and expected data from CSV files - std::vector> res; - struct stat buff; - if (stat(column_data_file.c_str(), &buff) == 0) { - if (S_ISREG(buff.st_mode)) { - // file - if (is_hive_format) { - load_data_and_assert_from_csv(serders, columns, column_data_file, - col_spliter, idxes); - } else { - load_data_and_assert_from_csv(serders, columns, column_data_file, - col_spliter, idxes); - } - } else if (S_ISDIR(buff.st_mode)) { - // dir - std::filesystem::path fs_path(column_data_file); - for (const auto& entry : std::filesystem::directory_iterator(fs_path)) { - std::string file_path = entry.path().string(); - std::cout << "load data from file: " << file_path << std::endl; - if (is_hive_format) { - load_data_and_assert_from_csv(serders, columns, file_path, - col_spliter, idxes); - } else { - load_data_and_assert_from_csv(serders, columns, file_path, - col_spliter, idxes); - } - } - } - } - - // Step 2: Validate the data in `column` matches `expected_data` - assert_callback(columns, serders); - } - - // Helper function to load data from CSV, with index which splited by spliter and load to columns - template - static void load_data_and_assert_from_csv(const DataTypeSerDeSPtrs serders, - MutableColumns& columns, const std::string& file_path, - const char spliter = ';', - const std::set idxes = {0}) { - ASSERT_EQ(serders.size(), columns.size()) - << "serder size: " << serders.size() << " column size: " << columns.size(); - ASSERT_EQ(serders.size(), idxes.size()) - << "serder size: " << serders.size() << " idxes size: " << idxes.size(); - std::ifstream file(file_path); - if (!file) { - throw doris::Exception(ErrorCode::INVALID_ARGUMENT, "can not open the file: {} ", - file_path); - } - - std::string line; - DataTypeSerDe::FormatOptions options; - std::vector> res; - MutableColumns assert_str_cols(columns.size()); - for (size_t i = 0; i < columns.size(); ++i) { - assert_str_cols[i] = ColumnString::create(); - } - - while (std::getline(file, line)) { - std::stringstream lineStream(line); - // std::cout << "whole : " << lineStream.str() << std::endl; - std::string value; - int l_idx = 0; - int c_idx = 0; - std::vector row; - while (std::getline(lineStream, value, spliter)) { - if (idxes.contains(l_idx)) { - // load csv data - Slice string_slice(value.data(), value.size()); - std::cout << "origin : " << string_slice << std::endl; - Status st; - // deserialize data - if constexpr (is_hive_format) { - st = serders[c_idx]->deserialize_one_cell_from_hive_text( - *columns[c_idx], string_slice, options); - } else { - st = serders[c_idx]->deserialize_one_cell_from_json(*columns[c_idx], - string_slice, options); - } - if (!st.ok()) { - // deserialize if happen error now we do not insert any value for input column - // so we push a default value to column for row alignment - columns[c_idx]->insert_default(); - std::cout << "error in deserialize but continue: " << st.to_string() - << std::endl; - } - // serialize data - size_t row_num = columns[c_idx]->size() - 1; - assert_str_cols[c_idx]->reserve(columns[c_idx]->size()); - VectorBufferWriter bw(assert_cast(*assert_str_cols[c_idx])); - if constexpr (is_hive_format) { - st = serders[c_idx]->serialize_one_cell_to_hive_text(*columns[c_idx], - row_num, bw, options); - EXPECT_TRUE(st.ok()) << st.to_string(); - } else { - st = serders[c_idx]->serialize_one_cell_to_json(*columns[c_idx], row_num, - bw, options); - EXPECT_TRUE(st.ok()) << st.to_string(); - } - bw.commit(); - // assert data : origin data and serialized data should be equal or generated - // file to check data - size_t assert_size = assert_str_cols[c_idx]->size(); - if constexpr (!generate_res_file) { - EXPECT_EQ(assert_str_cols[c_idx]->get_data_at(assert_size - 1).to_string(), - string_slice.to_string()) - << "column: " << columns[c_idx]->get_name() << " row: " << row_num - << " is_hive_format: " << is_hive_format; - } - ++c_idx; - } - res.push_back(row); - ++l_idx; - } - } - - if (generate_res_file) { - // generate res - auto pos = file_path.find_last_of("."); - std::string hive_format = is_hive_format ? "_hive" : ""; - std::string res_file = file_path.substr(0, pos) + hive_format + "_serde_res.csv"; - std::ofstream res_f(res_file); - if (!res_f.is_open()) { - throw std::ios_base::failure("Failed to open file." + res_file); - } - for (size_t r = 0; r < assert_str_cols[0]->size(); ++r) { - for (size_t c = 0; c < assert_str_cols.size(); ++c) { - std::cout << assert_str_cols[c]->get_data_at(r).to_string() << spliter - << std::endl; - res_f << assert_str_cols[c]->get_data_at(r).to_string() << spliter; - } - res_f << std::endl; - } - res_f.close(); - std::cout << "generate res file: " << res_file << std::endl; - } - } - - // standard hive text ser-deserialize assert function - // pb serde now is only used RPCFncall and fold_constant_executor which just write column data to pb value means - // just call write_column_to_pb - static void assert_pb_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& col = load_cols[i]; - std::cout << " now we are testing column : " << col->get_name() << std::endl; - // serialize to pb - PValues pv = PValues(); - Status st = serders[i]->write_column_to_pb(*col, pv, 0, col->size()); - if (!st.ok()) { - std::cerr << "write_column_to_pb error: " << st.msg() << std::endl; - continue; - } - // deserialize from pb - auto except_column = col->clone_empty(); - st = serders[i]->read_column_from_pb(*except_column, pv); - EXPECT_TRUE(st.ok()) << st.to_string(); - // check pb value from expected column - PValues as_pv = PValues(); - st = serders[i]->write_column_to_pb(*except_column, as_pv, 0, except_column->size()); - EXPECT_TRUE(st.ok()) << st.to_string(); - EXPECT_EQ(pv.bytes_value_size(), as_pv.bytes_value_size()); - // check column value - for (size_t j = 0; j < col->size(); ++j) { - auto cell = col->operator[](j); - auto except_cell = except_column->operator[](j); - EXPECT_EQ(cell, except_cell) << "column: " << col->get_name() << " row: " << j; - } - } - } - - // actually this is block_to_jsonb and jsonb_to_block test - static void assert_jsonb_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { - Arena pool; - auto jsonb_column = ColumnString::create(); // jsonb column - // maybe these load_cols has different size, so we keep it same - size_t max_row_size = load_cols[0]->size(); - for (size_t i = 1; i < load_cols.size(); ++i) { - if (load_cols[i]->size() > max_row_size) { - max_row_size = load_cols[i]->size(); - } - } - // keep same rows - for (size_t i = 0; i < load_cols.size(); ++i) { - if (load_cols[i]->size() < max_row_size) { - load_cols[i]->insert_many_defaults(max_row_size - load_cols[i]->size()); - } else if (load_cols[i]->size() > max_row_size) { - load_cols[i]->resize(max_row_size); - } - } - jsonb_column->reserve(load_cols[0]->size()); - MutableColumns assert_cols; - for (size_t i = 0; i < load_cols.size(); ++i) { - assert_cols.push_back(load_cols[i]->assume_mutable()); - } - for (size_t r = 0; r < load_cols[0]->size(); ++r) { - JsonbWriterT jw; - jw.writeStartObject(); - // serialize to jsonb - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& col = load_cols[i]; - serders[i]->write_one_cell_to_jsonb(*col, jw, &pool, i, r); - } - jw.writeEndObject(); - jsonb_column->insert_data(jw.getOutput()->getBuffer(), jw.getOutput()->getSize()); - } - // deserialize jsonb column to assert column - EXPECT_EQ(jsonb_column->size(), load_cols[0]->size()); - for (size_t r = 0; r < jsonb_column->size(); ++r) { - StringRef jsonb_data = jsonb_column->get_data_at(r); - auto pdoc = JsonbDocument::createDocument(jsonb_data.data, jsonb_data.size); - JsonbDocument& doc = *pdoc; - size_t cIdx = 0; - for (auto it = doc->begin(); it != doc->end(); ++it) { - serders[cIdx]->read_one_cell_from_jsonb(*assert_cols[cIdx], it->value()); - ++cIdx; - } - } - // check column value - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& col = load_cols[i]; - auto& assert_col = assert_cols[i]; - for (size_t j = 0; j < col->size(); ++j) { - auto cell = col->operator[](j); - auto assert_cell = assert_col->operator[](j); - EXPECT_EQ(cell, assert_cell) << "column: " << col->get_name() << " row: " << j; - } - } - } - - // assert mysql text format, now we just simple assert not to fatal or exception here - static void assert_mysql_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { - MysqlRowBuffer row_buffer; - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& col = load_cols[i]; - for (size_t j = 0; j < col->size(); ++j) { - Status st; - EXPECT_NO_FATAL_FAILURE( - st = serders[i]->write_column_to_mysql(*col, row_buffer, j, false, {})); - EXPECT_TRUE(st.ok()) << st.to_string(); - } - } - } - - // assert arrow serialize - static void assert_arrow_format(MutableColumns& load_cols, DataTypeSerDeSPtrs serders, - DataTypes types) { - // make a block to write to arrow - auto block = std::make_shared(); - // maybe these load_cols has different size, so we keep it same - size_t max_row_size = load_cols[0]->size(); - for (size_t i = 1; i < load_cols.size(); ++i) { - if (load_cols[i]->size() > max_row_size) { - max_row_size = load_cols[i]->size(); - } - } - // keep same rows - for (size_t i = 0; i < load_cols.size(); ++i) { - if (load_cols[i]->size() < max_row_size) { - load_cols[i]->insert_many_defaults(max_row_size - load_cols[i]->size()); - } else if (load_cols[i]->size() > max_row_size) { - load_cols[i]->resize(max_row_size); - } - } - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& col = load_cols[i]; - block->insert(ColumnWithTypeAndName(std::move(col), types[i], types[i]->get_name())); - } - // print block - std::cout << "block: " << block->dump_structure() << std::endl; - std::shared_ptr block_arrow_schema; - EXPECT_EQ(get_arrow_schema_from_block(*block, &block_arrow_schema, "UTC"), Status::OK()); - std::cout << "schema: " << block_arrow_schema->ToString(true) << std::endl; - // convert block to arrow - std::shared_ptr result; - cctz::time_zone _timezone_obj; //default UTC - Status stt = convert_to_arrow_batch(*block, block_arrow_schema, - arrow::default_memory_pool(), &result, _timezone_obj); - EXPECT_EQ(Status::OK(), stt) << "convert block to arrow failed" << stt.to_string(); - std::cout << "arrow record batch: " << result->ToString() << std::endl; - std::cout << "block : " << block->dump_structure() << std::endl; - // deserialize arrow to block - auto assert_block = block->clone_empty(); - auto rows = block->rows(); - std::cout << "assert block rows: " << rows << std::endl; - for (size_t i = 0; i < load_cols.size(); ++i) { - auto array = result->column(i); - auto& column_with_type_and_name = assert_block.get_by_position(i); - auto ret = arrow_column_to_doris_column( - array.get(), 0, column_with_type_and_name.column, - column_with_type_and_name.type, rows, _timezone_obj); - // do check data - EXPECT_EQ(Status::OK(), ret) << "convert arrow to block failed" << ret.to_string(); - auto& col = block->get_by_position(i).column; - auto& assert_col = column_with_type_and_name.column; - for (size_t j = 0; j < col->size(); ++j) { - auto cell = col->operator[](j); - auto assert_cell = assert_col->operator[](j); - EXPECT_EQ(cell, assert_cell) << "column: " << col->get_name() << " row: " << j; - } - } - std::cout << "assert block: " << assert_block.dump_structure() << std::endl; - } - - // assert rapidjson format - // now rapidjson write_one_cell_to_json and read_one_cell_from_json only used in column_object - // can just be replaced by jsonb format -}; - -} // namespace doris::vectorized \ No newline at end of file From 3f4f9d3e13d0a10ae4c05b89bfacf1bc7f7f4278 Mon Sep 17 00:00:00 2001 From: amorynan Date: Thu, 2 Jan 2025 23:56:24 +0800 Subject: [PATCH 18/23] update --- be/src/vec/columns/column_array.cpp | 36 --- be/test/vec/columns/column_array_test.cpp | 105 ++++++-- be/test/vec/columns/common_column_test.h | 298 +++++++++++++++++----- 3 files changed, 322 insertions(+), 117 deletions(-) diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp index c3a23fa3425a25..c5e85401b005ef 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -857,9 +857,6 @@ ColumnPtr ColumnArray::replicate(const IColumn::Offsets& replicate_offsets) cons if (typeid_cast(data.get())) { return replicate_string(replicate_offsets); } - if (typeid_cast(data.get())) { - return replicate_const(replicate_offsets); - } if (typeid_cast(data.get())) { return replicate_nullable(replicate_offsets); } @@ -991,39 +988,6 @@ ColumnPtr ColumnArray::replicate_string(const IColumn::Offsets& replicate_offset return res; } -ColumnPtr ColumnArray::replicate_const(const IColumn::Offsets& replicate_offsets) const { - size_t col_size = size(); - column_match_offsets_size(col_size, replicate_offsets.size()); - - if (0 == col_size) return clone_empty(); - - const auto& src_offsets = get_offsets(); - - auto res_column_offsets = ColumnOffsets::create(); - auto& res_offsets = res_column_offsets->get_data(); - res_offsets.reserve(replicate_offsets.back()); - - IColumn::Offset prev_replicate_offset = 0; - Offset64 prev_data_offset = 0; - Offset64 current_new_offset = 0; - - for (size_t i = 0; i < col_size; ++i) { - size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset; - size_t value_size = src_offsets[i] - prev_data_offset; - - for (size_t j = 0; j < size_to_replicate; ++j) { - current_new_offset += value_size; - res_offsets.push_back(current_new_offset); - } - - prev_replicate_offset = replicate_offsets[i]; - prev_data_offset = src_offsets[i]; - } - - return ColumnArray::create(get_data().clone_resized(current_new_offset), - std::move(res_column_offsets)); -} - ColumnPtr ColumnArray::replicate_generic(const IColumn::Offsets& replicate_offsets) const { size_t col_size = size(); column_match_offsets_size(col_size, replicate_offsets.size()); diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp index ad3f2fb7c9b23f..f8c79d935401bf 100644 --- a/be/test/vec/columns/column_array_test.cpp +++ b/be/test/vec/columns/column_array_test.cpp @@ -36,6 +36,8 @@ class ColumnArrayTest : public CommonColumnTest { MutableColumns array_cols; // we need to load data from csv file into column_array list // step1. create data type for array nested type (const and nullable) + // array + BaseInputTypeSet array_uint8 = {TypeIndex::Array, TypeIndex::UInt8}; // array BaseInputTypeSet array_tinyint = {TypeIndex::Array, TypeIndex::Int8}; // array @@ -72,6 +74,8 @@ class ColumnArrayTest : public CommonColumnTest { BaseInputTypeSet array_decimal128 = {TypeIndex::Array, TypeIndex::Decimal128V3}; // array UT BaseInputTypeSet array_decimal256 = {TypeIndex::Array, TypeIndex::Decimal256}; + // array> + BaseInputTypeSet array_array_uint8 = {TypeIndex::Array, TypeIndex::Array, TypeIndex::UInt8}; // array> BaseInputTypeSet array_array_tinyint = {TypeIndex::Array, TypeIndex::Array, TypeIndex::Int8}; @@ -141,17 +145,18 @@ class ColumnArrayTest : public CommonColumnTest { TypeIndex::Float64, TypeIndex::IPv4, TypeIndex::IPv6}; std::vector array_typeIndex = { - array_tinyint, array_smallint, array_int, array_bigint, array_largeint, - array_float, array_double, array_ipv4, array_ipv6, array_date, - array_datetime, array_datev2, array_datetimev2, array_varchar, array_decimal, - array_decimal64, array_decimal128, array_decimal256}; + array_uint8, array_tinyint, array_smallint, array_int, array_bigint, + array_largeint, array_float, array_double, array_ipv4, array_ipv6, + array_date, array_datetime, array_datev2, array_datetimev2, array_varchar, + array_decimal, array_decimal64, array_decimal128, array_decimal256}; std::vector array_array_typeIndex = { - array_array_tinyint, array_array_smallint, array_array_int, - array_array_bigint, array_array_largeint, array_array_float, - array_array_double, array_array_ipv4, array_array_ipv6, - array_array_date, array_array_datetime, array_array_datev2, - array_array_datetimev2, array_array_varchar, array_array_decimal, - array_array_decimal64, array_array_decimal128, array_array_decimal256}; + array_array_uint8, array_array_tinyint, array_array_smallint, + array_array_int, array_array_bigint, array_array_largeint, + array_array_float, array_array_double, array_array_ipv4, + array_array_ipv6, array_array_date, array_array_datetime, + array_array_datev2, array_array_datetimev2, array_array_varchar, + array_array_decimal, array_array_decimal64, array_array_decimal128, + array_array_decimal256}; std::vector array_map_typeIndex = { array_map_char_double, array_map_datetime_decimal, array_map_ipv4_ipv6, array_map_largeint_string}; @@ -223,7 +228,8 @@ class ColumnArrayTest : public CommonColumnTest { } // create column_array for each data type - vector data_files = {data_file_dir + "test_array_tinyint.csv", + vector data_files = {data_file_dir + "test_array_bool.csv", + data_file_dir + "test_array_tinyint.csv", data_file_dir + "test_array_smallint.csv", data_file_dir + "test_array_int.csv", data_file_dir + "test_array_bigint.csv", @@ -243,7 +249,8 @@ class ColumnArrayTest : public CommonColumnTest { data_file_dir + "test_array_decimalv3(76,56).csv"}; data_files.insert(data_files.end(), - {data_file_dir + "test_array_array_tinyint.csv", + {data_file_dir + "test_array_array_bool.csv", + data_file_dir + "test_array_array_tinyint.csv", data_file_dir + "test_array_array_smallint.csv", data_file_dir + "test_array_array_int.csv", data_file_dir + "test_array_array_bigint.csv", @@ -362,13 +369,22 @@ TEST_F(ColumnArrayTest, InsertManyDefaultsTest) { assert_insert_many_defaults_callback(array_columns, serdes); } +TEST_F(ColumnArrayTest, InsertDataTest) { + // we expect insert_data will throw exception + EXPECT_ANY_THROW(assert_insert_data_from_callback(array_columns, serdes)); +} + TEST_F(ColumnArrayTest, GetDataAtTest) { // get_data_at is not support in column_array EXPECT_ANY_THROW(assert_get_data_at_callback(array_columns, serdes)); } TEST_F(ColumnArrayTest, FieldTest) { - assert_field_callback(array_columns, serdes); + MutableColumns array_columns_copy; + DataTypeSerDeSPtrs serdes_copy; + array_columns_copy.push_back(array_columns[42]->assume_mutable()); + serdes_copy.push_back(serdes[42]); + assert_field_callback(array_columns_copy, serdes_copy); } TEST_F(ColumnArrayTest, GetRawDataTest) { @@ -440,15 +456,15 @@ TEST_F(ColumnArrayTest, ReplicateTest) { assert_replicate_callback(array_columns_copy, serdes_copy); // expect error columns MutableColumns error_columns; - error_columns.push_back(array_columns[31]->assume_mutable()); + error_columns.push_back(array_columns[33]->assume_mutable()); DataTypeSerDeSPtrs error_serdes; - error_serdes.push_back(serdes[31]); + error_serdes.push_back(serdes[33]); EXPECT_ANY_THROW(assert_replicate_callback(error_columns, error_serdes)); } TEST_F(ColumnArrayTest, ReplaceColumnTest) { // replace_column_data is not support in column_array, only support non-variable length column - EXPECT_ANY_THROW(assert_replace_column_data_callback(array_columns, serdes)); + assert_replace_column_data_callback(array_columns, serdes); assert_replace_column_null_data_callback(array_columns, serdes); } @@ -468,7 +484,11 @@ TEST_F(ColumnArrayTest, PermutationAndSortTest) { } TEST_F(ColumnArrayTest, FilterTest) { + // The filter method implemented by column_array does not achieve the memory reuse acceleration effect like other basic types, + // and still returns a new ptr, which can be make a todo task assert_filter_callback(array_columns, serdes); + // filter with result_size_hint + assert_filter_with_result_hint_callback(array_columns, serdes); } // HASH Interfaces @@ -481,6 +501,9 @@ TEST_F(ColumnArrayTest, HashTest) { std::vector pts(array_columns.size(), PrimitiveType::TYPE_ARRAY); assert_update_crc_hashes_callback(array_columns, serdes, pts); // CrcHash with null_data + + // SipHash + assert_update_siphashes_with_value_callback(array_columns, serdes); }; //////////////////////// special function from column_array.h //////////////////////// @@ -506,10 +529,29 @@ TEST_F(ColumnArrayTest, CreateArrayTest) { auto column_type = type->get_name(); LOG(INFO) << "column_type: " << column_type; // test create_array + // test create expect exception case + // 1.offsets is not ColumnUInt64 + auto tmp_data_col = column->get_data_ptr()->clone_resized(1); + auto tmp_offsets_col = + assert_cast(column->get_offsets_column().clone_resized(1)); + // make offsets_col into column_int32 + ColumnUInt128 wrong_type_offsets_col; + wrong_type_offsets_col.insert(tmp_offsets_col.back()); + EXPECT_ANY_THROW({ + auto new_array_column = ColumnArray::create(tmp_data_col->assume_mutable(), + wrong_type_offsets_col.assume_mutable()); + }); + // 2.offsets size is not equal to data size + auto tmp_data_col1 = column->get_data_ptr()->clone_resized(2); + EXPECT_ANY_THROW({ + auto new_array_column = ColumnArray::create( + tmp_data_col1->assume_mutable(), + column->get_offsets_column().clone_resized(1)->assume_mutable()); + }); + // 3.data is const auto last_offset = column->get_offsets().back(); EXPECT_ANY_THROW( { auto const_col = ColumnConst::create(column->get_data_ptr(), last_offset); }); - auto tmp_data_col = column->get_data_ptr()->clone_resized(1); Field assert_field; column->get(0, assert_field); auto const_col = ColumnConst::create(tmp_data_col->assume_mutable(), last_offset); @@ -544,7 +586,7 @@ TEST_F(ColumnArrayTest, MetaInfoTest) { TEST_F(ColumnArrayTest, ConvertIfOverflowAndInsertTest) { // test nested string in array which like ColumnArray only use in join - // test convert_column_if_overflow && insert_range_from_ignore_overflow + // test convert_column_if_overflow for (int i = 0; i < array_columns.size(); i++) { auto& column = array_columns[i]; auto type = array_types[i]; @@ -572,6 +614,12 @@ TEST_F(ColumnArrayTest, ConvertIfOverflowAndInsertTest) { } } +// Test insert_range_from_ignore_overflow +TEST_F(ColumnArrayTest, InsertRangeFromIgnoreOverflowTest) { + // test insert_range_from_ignore_overflow + assert_insert_range_from_ignore_overflow(array_columns, array_types); +} + TEST_F(ColumnArrayTest, GetNumberOfDimensionsTest) { // test dimension of array for (int i = 0; i < array_columns.size(); i++) { @@ -584,12 +632,33 @@ TEST_F(ColumnArrayTest, GetNumberOfDimensionsTest) { .get_nested_type(); dimension++; check_type = nested_type; + std::cout << "dimension: " << dimension << std::endl; } EXPECT_EQ(column->get_number_of_dimensions(), dimension) << "column dimension is not equal to check_type dimension"; } } +TEST_F(ColumnArrayTest, IsExclusiveTest) { + for (int i = 0; i < array_columns.size(); i++) { + auto column = check_and_get_column( + remove_nullable(array_columns[i]->assume_mutable()).get()); + auto cloned = array_columns[i]->clone_resized(1); + // test expect true + EXPECT_TRUE(column->is_exclusive()); + // new column with different data column + const ColumnPtr new_data_column = + column->get_data_ptr()->clone_resized(0)->convert_column_if_overflow(); + auto new_array_column = ColumnArray::create(new_data_column); + EXPECT_FALSE(new_array_column->is_exclusive()); + // new column with different offsets column + const ColumnPtr new_offsets_column = + column->get_offsets_ptr()->clone_resized(0)->convert_column_if_overflow(); + new_array_column = ColumnArray::create(column->get_data_ptr(), new_offsets_column); + EXPECT_FALSE(new_array_column->is_exclusive()); + } +} + TEST_F(ColumnArrayTest, MaxArraySizeAsFieldTest) { // test array max_array_size_as_field which is set to 100w // in operator[] and get() diff --git a/be/test/vec/columns/common_column_test.h b/be/test/vec/columns/common_column_test.h index a8888d7a43a817..8828290957fa52 100644 --- a/be/test/vec/columns/common_column_test.h +++ b/be/test/vec/columns/common_column_test.h @@ -376,6 +376,41 @@ class CommonColumnTest : public ::testing::Test { // assert insert_range_from_ignore_overflow which happened in columnStr want to insert from ColumnStr for more column string to be inserted not just limit to the 4G // Define the custom assert callback function to verify insert_range_from_ignore_overflow behavior + static void assert_insert_range_from_ignore_overflow(MutableColumns& load_cols, + DataTypes types) { + // step1. to construct a block for load_cols + Block block; + for (size_t i = 0; i < load_cols.size(); ++i) { + ColumnWithTypeAndName columnTypeAndName; + columnTypeAndName.column = load_cols[i]->assume_mutable(); + columnTypeAndName.type = types[i]; + block.insert(columnTypeAndName); + } + MutableBlock mb = MutableBlock::build_mutable_block(&block); + // step2. to construct a block for assert_cols + Block assert_block; + Block empty_block; + for (size_t i = 0; i < load_cols.size(); ++i) { + ColumnWithTypeAndName columnTypeAndName; + columnTypeAndName.column = load_cols[i]->clone_empty(); + columnTypeAndName.type = types[i]; + assert_block.insert(columnTypeAndName); + empty_block.insert(columnTypeAndName); + } + MutableBlock assert_mb = MutableBlock::build_mutable_block(&empty_block); + // step3. to insert data from load_cols to assert_cols + Status st = mb.merge_impl_ignore_overflow(assert_block); + EXPECT_TRUE(st.ok()) << "Failed to merge block: " << st.to_string(); + Status st2 = assert_mb.merge_impl_ignore_overflow(block); + EXPECT_TRUE(st2.ok()) << "Failed to merge block1: " << st2.to_string(); + // step4. to check data in assert_cols + for (size_t i = 0; i < load_cols.size(); ++i) { + checkColumn(*load_cols[i], *mb.get_column_by_position(i), *types[i], + load_cols[i]->size()); + checkColumn(*load_cols[i], *assert_mb.get_column_by_position(i), *types[i], + load_cols[i]->size()); + } + } // assert insert_many_from (used in join situation, which handle left table col to expand for right table : such as A[1,2,3] inner join B[2,2,4,4] => A[2,2] ) // Define the custom assert callback function to verify insert_many_from behavior @@ -481,6 +516,8 @@ class CommonColumnTest : public ::testing::Test { target_column->insert_indices_from(*source_column, &(*from_idx), &(*end_idx)); } + std::cout << source_column->get_name() << " now insert_indices_from from " + << *from_idx << " to " << *end_idx << std::endl; // Verify the inserted data matches the expected results in `assert_res` auto ser_col = ColumnString::create(); ser_col->reserve(target_column->size()); @@ -685,73 +722,90 @@ class CommonColumnTest : public ::testing::Test { //virtual Field //operator[] (size_t n) const = 0 static void assert_field_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { - MutableColumns assert_cols(load_cols.size()); - for (size_t i = 0; i < load_cols.size(); ++i) { - assert_cols[i] = load_cols[i]->clone_empty(); - } auto option = DataTypeSerDe::FormatOptions(); - std::vector> res; - // just check cols get is the same as assert_res - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& source_column = load_cols[i]; - LOG(INFO) << " new insert field for column : " << assert_cols[i]->get_name() - << " with size : " << assert_cols[i]->size(); - for (size_t j = 0; j < source_column->size(); ++j) { - Field f; - source_column->get(j, f); - assert_cols[i]->insert(f); + { + MutableColumns assert_cols(load_cols.size()); + for (size_t i = 0; i < load_cols.size(); ++i) { + assert_cols[i] = load_cols[i]->clone_empty(); } - } - // Verify the inserted data matches the expected results in `assert_res` - for (size_t i = 0; i < assert_cols.size(); ++i) { - auto ser_col = ColumnString::create(); - ser_col->reserve(load_cols[i]->size()); - VectorBufferWriter buffer_writer(*ser_col.get()); - std::vector data; - for (size_t j = 0; j < assert_cols[i]->size(); ++j) { - if (auto st = serders[i]->serialize_one_cell_to_json(*assert_cols[i], j, - buffer_writer, option); - !st) { - LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; - break; + std::vector> res; + // just check cols get is the same as assert_res + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + LOG(INFO) << " new insert field for column : " << assert_cols[i]->get_name() + << " with size : " << assert_cols[i]->size() << " source_coumn" + << source_column->size(); + for (size_t j = 0; j < source_column->size(); ++j) { + Field f; + source_column->get(j, f); + assert_cols[i]->insert(f); } - buffer_writer.commit(); - std::string actual_str_value = ser_col->get_data_at(j).to_string(); - data.push_back(actual_str_value); + // check with null Field + Field null_field; + assert_cols[i]->insert(null_field); } - res.push_back(data); - } - check_res_file("get_field", res); - // just check cols operator [] to get field same with field - std::vector> res2; - for (size_t i = 0; i < load_cols.size(); ++i) { - auto& source_column = load_cols[i]; - for (size_t j = 0; j < source_column->size(); ++j) { - Field f = source_column->operator[](j); - assert_cols[i]->insert(f); + // Verify the inserted data matches the expected results in `assert_res` + for (size_t i = 0; i < assert_cols.size(); ++i) { + auto ser_col = ColumnString::create(); + ser_col->reserve(load_cols[i]->size()); + VectorBufferWriter buffer_writer(*ser_col.get()); + std::vector data; + for (size_t j = 0; j < assert_cols[i]->size() - 1; ++j) { + if (auto st = serders[i]->serialize_one_cell_to_json(*assert_cols[i], j, + buffer_writer, option); + !st) { + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; + break; + } + buffer_writer.commit(); + std::string actual_str_value = ser_col->get_data_at(j).to_string(); + data.push_back(actual_str_value); + EXPECT_EQ(load_cols[i]->operator[](j), assert_cols[i]->operator[](j)); + } + res.push_back(data); } + check_res_file("get_field", res); } + { + MutableColumns assert_cols(load_cols.size()); + for (size_t i = 0; i < load_cols.size(); ++i) { + assert_cols[i] = load_cols[i]->clone_empty(); + } + // just check cols operator [] to get field same with field + std::vector> res2; + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + for (size_t j = 0; j < source_column->size(); ++j) { + Field f = source_column->operator[](j); + assert_cols[i]->insert(f); + } + // check with null Field + Field null_field; + assert_cols[i]->insert(null_field); + } - // Verify the inserted data matches the expected results in `assert_res` - for (size_t i = 0; i < assert_cols.size(); ++i) { - auto ser_col = ColumnString::create(); - ser_col->reserve(load_cols[i]->size()); - VectorBufferWriter buffer_writer(*ser_col.get()); - std::vector data; - for (size_t j = 0; j < assert_cols[i]->size(); ++j) { - if (auto st = serders[i]->serialize_one_cell_to_json(*assert_cols[i], j, - buffer_writer, option); - !st) { - LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; - break; + // Verify the inserted data matches the expected results in `assert_res` + for (size_t i = 0; i < assert_cols.size(); ++i) { + auto ser_col = ColumnString::create(); + ser_col->reserve(load_cols[i]->size()); + VectorBufferWriter buffer_writer(*ser_col.get()); + std::vector data; + for (size_t j = 0; j < assert_cols[i]->size() - 1; ++j) { + if (auto st = serders[i]->serialize_one_cell_to_json(*assert_cols[i], j, + buffer_writer, option); + !st) { + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; + break; + } + buffer_writer.commit(); + std::string actual_str_value = ser_col->get_data_at(j).to_string(); + data.push_back(actual_str_value); + EXPECT_EQ(load_cols[i]->operator[](j), assert_cols[i]->operator[](j)); } - buffer_writer.commit(); - std::string actual_str_value = ser_col->get_data_at(j).to_string(); - data.push_back(actual_str_value); + res2.push_back(data); } - res2.push_back(data); + check_res_file("get_field_operator", res2); } - check_res_file("get_field_operator", res2); } // //virtual StringRef @@ -1273,8 +1327,7 @@ class CommonColumnTest : public ::testing::Test { for (auto cl = check_length.begin(); cl < check_length.end(); ++cl) { for (size_t i = 0; i < load_cols.size(); ++i) { if (load_cols[i]->is_variable_length()) { - throw std::runtime_error( - "replace_column_data only support non-variable length column"); + EXPECT_ANY_THROW(load_cols[i]->replace_column_data(*load_cols[i], *cl)); } auto& source_column = load_cols[i]; if (*cl > source_column->size()) { @@ -1413,7 +1466,7 @@ class CommonColumnTest : public ::testing::Test { //Clear data of column, just like vector clear. // Column Calculate Interface: filter, compare, permute, sort - // filter (const Filter &filt, ssize_t result_size_hint) const =0 with a result_size_hint to pass, but we should make sure the result_size_hint is not bigger than the source column size + // filter (const Filter &filt) const =0 // Filter is a array contains 0 or 1 to present the row is selected or not, so it should be the same size with the source column static void assert_filter_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { // Create an empty column to verify `filter` functionality @@ -1421,6 +1474,96 @@ class CommonColumnTest : public ::testing::Test { std::vector> res; auto option = DataTypeSerDe::FormatOptions(); + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + auto source_size = source_column->size(); + const ColumnArray::Filter all_filtered(source_size, 0); + const ColumnArray::Filter no_filtered(source_size, 1); + // invalid data -1 will also make data without be filtered ?? + ColumnArray::Filter invalid_filter(source_size - 1, 1); + invalid_filter.emplace_back(-1); + std::vector data; + LOG(INFO) << "now we are in filter column : " << load_cols[i]->get_name() + << " for column size : " << source_column->size(); + { + auto ret_size = source_column->filter(all_filtered); + // check filter res + auto ser_col = ColumnString::create(); + ser_col->reserve(ret_size); + VectorBufferWriter buffer_writer(*ser_col.get()); + data.push_back("column: " + source_column->get_name() + + " with all filtered with ptr: " + std::to_string(ret_size)); + for (size_t j = 0; j < ret_size; ++j) { + if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, + buffer_writer, option); + !st) { + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; + break; + } + buffer_writer.commit(); + std::string actual_str_value = ser_col->get_data_at(j).to_string(); + data.push_back(actual_str_value); + } + res.push_back(data); + } + { + auto ret_size = source_column->filter(no_filtered); + // check filter res + auto ser_col = ColumnString::create(); + ser_col->reserve(ret_size); + VectorBufferWriter buffer_writer(*ser_col.get()); + data.clear(); + data.push_back("column: " + source_column->get_name() + + " with no filtered with ptr: " + std::to_string(ret_size)); + for (size_t j = 0; j < ret_size; ++j) { + if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, + buffer_writer, option); + !st) { + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; + break; + } + buffer_writer.commit(); + std::string actual_str_value = ser_col->get_data_at(j).to_string(); + data.push_back(actual_str_value); + } + res.push_back(data); + } + { + // check filter with invalid filter + auto ret_size = source_column->filter(invalid_filter); + // check filter res + auto ser_col = ColumnString::create(); + ser_col->reserve(ret_size); + VectorBufferWriter buffer_writer(*ser_col.get()); + data.clear(); + data.push_back("column: " + source_column->get_name() + + " with invalid filtered with ptr: " + std::to_string(ret_size)); + for (size_t j = 0; j < ret_size; ++j) { + if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, + buffer_writer, option); + !st) { + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; + break; + } + buffer_writer.commit(); + std::string actual_str_value = ser_col->get_data_at(j).to_string(); + data.push_back(actual_str_value); + } + res.push_back(data); + } + } + check_res_file("filter", res); + } + + // filter with result_hint_size which should return new column ptr + // filter (const Filter &filt, ssize_t result_size_hint) const =0 with a result_size_hint to pass, but we should make sure the result_size_hint is not bigger than the source column size + static void assert_filter_with_result_hint_callback(MutableColumns& load_cols, + DataTypeSerDeSPtrs serders) { + // Create an empty column to verify `filter` functionality + // check filter with different filter + std::vector> res; + auto option = DataTypeSerDe::FormatOptions(); + for (size_t i = 0; i < load_cols.size(); ++i) { auto& source_column = load_cols[i]; auto source_size = source_column->size(); @@ -1453,7 +1596,6 @@ class CommonColumnTest : public ::testing::Test { } res.push_back(data); } - { auto ptr = source_column->filter(no_filtered, source_column->size()); // check filter res @@ -1500,7 +1642,7 @@ class CommonColumnTest : public ::testing::Test { res.push_back(data); } } - check_res_file("filter", res); + check_res_file("filter_hint", res); } // Compare @@ -1563,6 +1705,36 @@ class CommonColumnTest : public ::testing::Test { check_res_file("update_crcs_hashes", res); } + // virtual void + // update_hash_with_value(size_t n, SipHash& hash) + // siphash we still keep siphash for storge layer because we use it in + // EngineChecksumTask::_compute_checksum() and can not to remove it + static void assert_update_siphashes_with_value_callback(MutableColumns& load_cols, + DataTypeSerDeSPtrs serders) { + // Create an empty column to verify `update_hashes` functionality + // check update_hashes with different hashes + std::vector> res; + auto option = DataTypeSerDe::FormatOptions(); + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + SipHash hash; + LOG(INFO) << "now we are in update_hashes column : " << load_cols[i]->get_name() + << " for column size : " << source_column->size(); + for (size_t j = 0; j < source_column->size(); ++j) { + source_column->update_hash_with_value(j, hash); + } + auto ser_col = ColumnString::create(); + ser_col->reserve(source_column->size()); + VectorBufferWriter buffer_writer(*ser_col.get()); + std::vector data; + data.push_back("column: " + source_column->get_name() + + " with hashes: " + std::to_string(hash.get64()) + + " with ptr: " + std::to_string(source_column->size())); + res.push_back(data); + } + check_res_file("update_siphashes_hashes", res); + } + // column size changed calculation: // size, reserve, resize, empty, byte_size, allocated_bytes, clone_resized, get_shrinked_column // cut(LIMIT operation), shrink @@ -1942,4 +2114,4 @@ class CommonColumnTest : public ::testing::Test { } }; -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized From 144db4a6216ddbb557bc69da560e8f634715bd10 Mon Sep 17 00:00:00 2001 From: amorynan Date: Fri, 3 Jan 2025 16:23:01 +0800 Subject: [PATCH 19/23] format code --- be/test/vec/columns/column_array_test.cpp | 25 +++++++------ be/test/vec/columns/common_column_test.h | 45 ++++++++++------------- 2 files changed, 33 insertions(+), 37 deletions(-) diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp index f8c79d935401bf..723e7e758fe59b 100644 --- a/be/test/vec/columns/column_array_test.cpp +++ b/be/test/vec/columns/column_array_test.cpp @@ -447,7 +447,7 @@ TEST_F(ColumnArrayTest, ReplicateTest) { DataTypeSerDeSPtrs serdes_copy; // just skip array_array_char use vector copy for (int i = 0; i < array_columns.size(); i++) { - if (i == 31) { + if (i == 33) { continue; } array_columns_copy.push_back(array_columns[i]->assume_mutable()); @@ -464,7 +464,7 @@ TEST_F(ColumnArrayTest, ReplicateTest) { TEST_F(ColumnArrayTest, ReplaceColumnTest) { // replace_column_data is not support in column_array, only support non-variable length column - assert_replace_column_data_callback(array_columns, serdes); + EXPECT_ANY_THROW(assert_replace_column_data_callback(array_columns, serdes)); assert_replace_column_null_data_callback(array_columns, serdes); } @@ -483,10 +483,13 @@ TEST_F(ColumnArrayTest, PermutationAndSortTest) { } } -TEST_F(ColumnArrayTest, FilterTest) { +TEST_F(ColumnArrayTest, FilterInplaceTest) { // The filter method implemented by column_array does not achieve the memory reuse acceleration effect like other basic types, // and still returns a new ptr, which can be make a todo task assert_filter_callback(array_columns, serdes); +} + +TEST_F(ColumnArrayTest, FilterTest) { // filter with result_size_hint assert_filter_with_result_hint_callback(array_columns, serdes); } @@ -524,22 +527,21 @@ TEST_F(ColumnArrayTest, CreateArrayTest) { for (int i = 0; i < array_columns.size(); i++) { auto column = check_and_get_column( remove_nullable(array_columns[i]->assume_mutable()).get()); - auto& type = array_types[i]; auto column_size = column->size(); - auto column_type = type->get_name(); - LOG(INFO) << "column_type: " << column_type; + LOG(INFO) << "column_type: " << column->get_name(); // test create_array // test create expect exception case // 1.offsets is not ColumnUInt64 auto tmp_data_col = column->get_data_ptr()->clone_resized(1); - auto tmp_offsets_col = - assert_cast(column->get_offsets_column().clone_resized(1)); + MutableColumnPtr tmp_offsets_col = + assert_cast(column->get_offsets_column()) + .clone_resized(1); + UInt64 off = tmp_offsets_col->operator[](0).get(); // make offsets_col into column_int32 - ColumnUInt128 wrong_type_offsets_col; - wrong_type_offsets_col.insert(tmp_offsets_col.back()); + auto wrong_type_offsets_col = vectorized::ColumnVector::create(1, off); EXPECT_ANY_THROW({ auto new_array_column = ColumnArray::create(tmp_data_col->assume_mutable(), - wrong_type_offsets_col.assume_mutable()); + wrong_type_offsets_col->assume_mutable()); }); // 2.offsets size is not equal to data size auto tmp_data_col1 = column->get_data_ptr()->clone_resized(2); @@ -632,7 +634,6 @@ TEST_F(ColumnArrayTest, GetNumberOfDimensionsTest) { .get_nested_type(); dimension++; check_type = nested_type; - std::cout << "dimension: " << dimension << std::endl; } EXPECT_EQ(column->get_number_of_dimensions(), dimension) << "column dimension is not equal to check_type dimension"; diff --git a/be/test/vec/columns/common_column_test.h b/be/test/vec/columns/common_column_test.h index 8828290957fa52..4fc20116446198 100644 --- a/be/test/vec/columns/common_column_test.h +++ b/be/test/vec/columns/common_column_test.h @@ -378,6 +378,15 @@ class CommonColumnTest : public ::testing::Test { // Define the custom assert callback function to verify insert_range_from_ignore_overflow behavior static void assert_insert_range_from_ignore_overflow(MutableColumns& load_cols, DataTypes types) { + size_t max = load_cols[0]->size(); + for (size_t i = 1; i < load_cols.size(); ++i) { + max = std::max(max, load_cols[i]->size()); + } + for (size_t i = 0; i < load_cols.size(); ++i) { + if (load_cols[i]->size() < max) { + load_cols[i]->resize(max); + } + } // step1. to construct a block for load_cols Block block; for (size_t i = 0; i < load_cols.size(); ++i) { @@ -516,8 +525,6 @@ class CommonColumnTest : public ::testing::Test { target_column->insert_indices_from(*source_column, &(*from_idx), &(*end_idx)); } - std::cout << source_column->get_name() << " now insert_indices_from from " - << *from_idx << " to " << *end_idx << std::endl; // Verify the inserted data matches the expected results in `assert_res` auto ser_col = ColumnString::create(); ser_col->reserve(target_column->size()); @@ -1477,6 +1484,7 @@ class CommonColumnTest : public ::testing::Test { for (size_t i = 0; i < load_cols.size(); ++i) { auto& source_column = load_cols[i]; auto source_size = source_column->size(); + auto cloned_col = load_cols[i]->clone_resized(source_size); const ColumnArray::Filter all_filtered(source_size, 0); const ColumnArray::Filter no_filtered(source_size, 1); // invalid data -1 will also make data without be filtered ?? @@ -1485,29 +1493,9 @@ class CommonColumnTest : public ::testing::Test { std::vector data; LOG(INFO) << "now we are in filter column : " << load_cols[i]->get_name() << " for column size : " << source_column->size(); - { - auto ret_size = source_column->filter(all_filtered); - // check filter res - auto ser_col = ColumnString::create(); - ser_col->reserve(ret_size); - VectorBufferWriter buffer_writer(*ser_col.get()); - data.push_back("column: " + source_column->get_name() + - " with all filtered with ptr: " + std::to_string(ret_size)); - for (size_t j = 0; j < ret_size; ++j) { - if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, - buffer_writer, option); - !st) { - LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; - break; - } - buffer_writer.commit(); - std::string actual_str_value = ser_col->get_data_at(j).to_string(); - data.push_back(actual_str_value); - } - res.push_back(data); - } { auto ret_size = source_column->filter(no_filtered); + EXPECT_EQ(ret_size, source_size); // check filter res auto ser_col = ColumnString::create(); ser_col->reserve(ret_size); @@ -1528,9 +1516,15 @@ class CommonColumnTest : public ::testing::Test { } res.push_back(data); } + { + auto ret_size = source_column->filter(all_filtered); + EXPECT_EQ(ret_size, 0); + } { // check filter with invalid filter - auto ret_size = source_column->filter(invalid_filter); + // source_column is filterd, size=0 + EXPECT_ANY_THROW(source_column->filter(invalid_filter)); + auto ret_size = cloned_col->filter(invalid_filter); // check filter res auto ser_col = ColumnString::create(); ser_col->reserve(ret_size); @@ -1539,7 +1533,7 @@ class CommonColumnTest : public ::testing::Test { data.push_back("column: " + source_column->get_name() + " with invalid filtered with ptr: " + std::to_string(ret_size)); for (size_t j = 0; j < ret_size; ++j) { - if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, + if (auto st = serders[i]->serialize_one_cell_to_json(*cloned_col, j, buffer_writer, option); !st) { LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; @@ -1571,6 +1565,7 @@ class CommonColumnTest : public ::testing::Test { const ColumnArray::Filter no_filtered(source_size, 1); // invalid data -1 will also make data without be filtered ?? ColumnArray::Filter invalid_filter(source_size - 1, 1); + // now AddressSanitizer: negative-size-param: (size=-1) can be checked invalid_filter.emplace_back(-1); std::vector data; LOG(INFO) << "now we are in filter column : " << load_cols[i]->get_name() From 533e923663357a7005833ac5d8f63bbed63024c5 Mon Sep 17 00:00:00 2001 From: amorynan Date: Mon, 6 Jan 2025 22:16:19 +0800 Subject: [PATCH 20/23] add left interfaces --- be/test/vec/columns/column_array_test.cpp | 152 +++- be/test/vec/columns/common_column_test.h | 858 ++++++++++++++-------- 2 files changed, 671 insertions(+), 339 deletions(-) diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp index 723e7e758fe59b..16d197cc950fa2 100644 --- a/be/test/vec/columns/column_array_test.cpp +++ b/be/test/vec/columns/column_array_test.cpp @@ -349,6 +349,51 @@ class ColumnArrayTest : public CommonColumnTest { }; //////////////////////// basic function from column.h //////////////////////// +TEST_F(ColumnArrayTest, InsertManyFixLengthDataTest) { + auto callback = [&](MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { + for (auto& col : array_columns) { + EXPECT_ANY_THROW(col->insert_many_fix_len_data(nullptr, 0)); + } + }; + assert_insert_many_fix_len_data(array_columns, serdes, callback); +} + +TEST_F(ColumnArrayTest, InsertManyDictDataTest) { + auto callback = [&](MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { + for (auto& col : array_columns) { + EXPECT_ANY_THROW(col->insert_many_dict_data(nullptr, 0, nullptr, 0)); + } + }; + assert_insert_many_dict_data(array_columns, serdes, callback); +} +// test assert_insert_many_continuous_binary_data +TEST_F(ColumnArrayTest, InsertManyContinuousBinaryDataTest) { + auto callback = [&](MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { + for (auto& col : array_columns) { + EXPECT_ANY_THROW(col->insert_many_continuous_binary_data(nullptr, 0, 0)); + } + }; + assert_insert_many_continuous_binary_data(array_columns, serdes, callback); +} + +TEST_F(ColumnArrayTest, InsertManyStringsOverflowTest) { + assert_insert_many_strings_overflow(array_columns, serdes, nullptr); +} + +TEST_F(ColumnArrayTest, InsertManyStringsTest) { + assert_insert_many_strings(array_columns, serdes, nullptr); +} + +// test insert_from +TEST_F(ColumnArrayTest, InsertFromTest) { + assert_insert_from_callback(array_columns, serdes); +} + +// test assert_insert_from_multi_column_callback +TEST_F(ColumnArrayTest, InsertFromMultiColumnTest) { + assert_insert_from_multi_column_callback(array_columns, serdes); +} + TEST_F(ColumnArrayTest, InsertRangeFromTest) { assert_insert_range_from_callback(array_columns, serdes); } @@ -371,7 +416,12 @@ TEST_F(ColumnArrayTest, InsertManyDefaultsTest) { TEST_F(ColumnArrayTest, InsertDataTest) { // we expect insert_data will throw exception - EXPECT_ANY_THROW(assert_insert_data_from_callback(array_columns, serdes)); + EXPECT_ANY_THROW(assert_insert_data_callback(array_columns, serdes)); +} + +TEST_F(ColumnArrayTest, InsertManyRawDataTest) { + // we expect insert_many_row_data will throw exception + EXPECT_ANY_THROW(assert_insert_many_raw_data_from_callback(array_columns, serdes)); } TEST_F(ColumnArrayTest, GetDataAtTest) { @@ -399,6 +449,18 @@ TEST_F(ColumnArrayTest, GetIntTest) { EXPECT_ANY_THROW({ array_columns[0]->get_int(0); }); } +TEST_F(ColumnArrayTest, GetNameTest) { + for (auto& col : array_columns) { + // name should contains "Array" + EXPECT_TRUE(col->get_name().find("Array") != std::string::npos); + } +} + +// test get_ratio_of_default_rows +TEST_F(ColumnArrayTest, GetRatioOfDefaultRowsTest) { + assert_get_ratio_of_default_rows(array_columns, serdes); +} + TEST_F(ColumnArrayTest, SerDeVecTest) { // get_max_row_byte_size is not support in column_array EXPECT_ANY_THROW(ser_deser_vec(array_columns, array_types)); @@ -428,10 +490,21 @@ TEST_F(ColumnArrayTest, CloneTest) { assert_clone_resized_callback(array_columns, serdes); } +// test assert_clone_empty +TEST_F(ColumnArrayTest, CloneEmptyTest) { + for (auto& col : array_columns) { + assert_clone_empty(*col); + } +} + TEST_F(ColumnArrayTest, CutTest) { assert_cut_callback(array_columns, serdes); } +TEST_F(ColumnArrayTest, ShrinkTest) { + assert_shrink_callback(array_columns, serdes); +} + TEST_F(ColumnArrayTest, ResizeTest) { assert_resize_callback(array_columns, serdes); } @@ -509,6 +582,46 @@ TEST_F(ColumnArrayTest, HashTest) { assert_update_siphashes_with_value_callback(array_columns, serdes); }; +// test assert_convert_to_full_column_if_const_callback +TEST_F(ColumnArrayTest, ConvertToFullColumnIfConstTest) { + assert_convert_to_full_column_if_const_callback(array_columns, array_types, nullptr); +} + +// test assert_convert_column_if_overflow_callback +TEST_F(ColumnArrayTest, ConvertColumnIfOverflowTest) { + assert_convert_column_if_overflow_callback(array_columns, serdes); +} + +// test assert_convert_to_predicate_column_if_dictionary_callback +TEST_F(ColumnArrayTest, ConvertToPredicateColumnIfDictionaryTest) { + assert_convert_to_predicate_column_if_dictionary_callback(array_columns, array_types, nullptr); +} + +// test assert_convert_dict_codes_if_necessary_callback +TEST_F(ColumnArrayTest, ConvertDictCodesIfNecessaryTest) { + assert_convert_dict_codes_if_necessary_callback(array_columns, nullptr); +} + +// test assert_copy_date_types_callback +TEST_F(ColumnArrayTest, CopyDateTypesTest) { + assert_copy_date_types_callback(array_columns); +} + +// test assert_column_nullable_funcs +TEST_F(ColumnArrayTest, ColumnNullableFuncsTest) { + assert_column_nullable_funcs(array_columns, nullptr); +} + +// test assert_column_string_funcs +TEST_F(ColumnArrayTest, ColumnStringFuncsTest) { + assert_column_string_funcs(array_columns); +} + +// test shrink_padding_chars_callback +TEST_F(ColumnArrayTest, ShrinkPaddingCharsTest) { + shrink_padding_chars_callback(array_columns, serdes); +} + //////////////////////// special function from column_array.h //////////////////////// TEST_F(ColumnArrayTest, CreateArrayTest) { // test create_array : nested_column && offsets_column should not be const, and convert_to_full_column_if_const should not impl in array @@ -641,23 +754,26 @@ TEST_F(ColumnArrayTest, GetNumberOfDimensionsTest) { } TEST_F(ColumnArrayTest, IsExclusiveTest) { - for (int i = 0; i < array_columns.size(); i++) { - auto column = check_and_get_column( - remove_nullable(array_columns[i]->assume_mutable()).get()); - auto cloned = array_columns[i]->clone_resized(1); - // test expect true - EXPECT_TRUE(column->is_exclusive()); - // new column with different data column - const ColumnPtr new_data_column = - column->get_data_ptr()->clone_resized(0)->convert_column_if_overflow(); - auto new_array_column = ColumnArray::create(new_data_column); - EXPECT_FALSE(new_array_column->is_exclusive()); - // new column with different offsets column - const ColumnPtr new_offsets_column = - column->get_offsets_ptr()->clone_resized(0)->convert_column_if_overflow(); - new_array_column = ColumnArray::create(column->get_data_ptr(), new_offsets_column); - EXPECT_FALSE(new_array_column->is_exclusive()); - } + auto callback = [&](const MutableColumns& columns, const DataTypeSerDeSPtrs& serdes) { + for (int i = 0; i < columns.size(); i++) { + auto column = check_and_get_column( + remove_nullable(columns[i]->assume_mutable()).get()); + auto cloned = columns[i]->clone_resized(1); + // test expect true + EXPECT_TRUE(column->is_exclusive()); + // new column with different data column + const ColumnPtr new_data_column = + column->get_data_ptr()->clone_resized(0)->convert_column_if_overflow(); + auto new_array_column = ColumnArray::create(new_data_column); + EXPECT_FALSE(new_array_column->is_exclusive()); + // new column with different offsets column + const ColumnPtr new_offsets_column = + column->get_offsets_ptr()->clone_resized(0)->convert_column_if_overflow(); + new_array_column = ColumnArray::create(column->get_data_ptr(), new_offsets_column); + EXPECT_FALSE(new_array_column->is_exclusive()); + } + }; + assert_is_exclusive(array_columns, serdes, callback); } TEST_F(ColumnArrayTest, MaxArraySizeAsFieldTest) { diff --git a/be/test/vec/columns/common_column_test.h b/be/test/vec/columns/common_column_test.h index 4fc20116446198..5e82889a3bc50e 100644 --- a/be/test/vec/columns/common_column_test.h +++ b/be/test/vec/columns/common_column_test.h @@ -26,8 +26,10 @@ #include "olap/schema.h" #include "vec/columns/column.h" #include "vec/columns/column_array.h" +#include "vec/columns/column_dictionary.h" #include "vec/columns/column_map.h" #include "vec/columns/columns_number.h" +#include "vec/common/cow.h" #include "vec/core/field.h" #include "vec/core/sort_block.h" #include "vec/core/sort_description.h" @@ -36,7 +38,7 @@ #include "vec/data_types/data_type_array.h" #include "vec/data_types/data_type_map.h" -// this test is gonna to be a column test template for all column which should make ut test to coverage the function defined in column +// this test is gonna to be a column test template for all column which should make ut test to coverage the function defined in column (all maybe we need 79 interfaces to be tested) // for example column_array should test this function: // size, reserve, resize, empty, byte_size, allocated_bytes, clone_resized, // get_shrinked_column, filter, filter_by_selector, serialize_vec, deserialize_vec, get_max_row_byte_size @@ -270,10 +272,68 @@ class CommonColumnTest : public ::testing::Test { } } + ////////// =================== column data insert interface assert(16) =================== ////////// + // In storage layer such as segment_iterator to call these function + // insert_many_fix_len_data (const char *pos, size_t num); + // insert_many_dict_data (const int32_t *data_array, size_t start_index, const StringRef *dict, size_t data_num, uint32_t dict_num=0) + // insert_many_continuous_binary_data (const char *data, const uint32_t *offsets, const size_t num) + static void assert_insert_many_fix_len_data( + MutableColumns& load_cols, DataTypeSerDeSPtrs serders, + std::function + assert_callback) { + // Create a column to verify `insert_many_fix_len_data` functionality + assert_callback(load_cols, serders); + } + + static void assert_insert_many_dict_data( + MutableColumns& load_cols, DataTypeSerDeSPtrs serders, + std::function + assert_callback) { + // Create a column to verify `insert_many_dict_data` functionality + assert_callback(load_cols, serders); + } + + static void assert_insert_many_continuous_binary_data( + MutableColumns& load_cols, DataTypeSerDeSPtrs serders, + std::function + assert_callback) { + // Create a column to verify `insert_many_continuous_binary_data` functionality + assert_callback(load_cols, serders); + } + + // only support in column_string: insert_many_strings(const StringRef *data, size_t num) && insert_many_strings_overflow + static void assert_insert_many_strings( + MutableColumns& load_cols, DataTypeSerDeSPtrs serders, + std::function + assert_callback) { + for (auto& col : load_cols) { + // Create a column to verify `insert_many_strings` functionality + if (is_column(*col)) { + EXPECT_ANY_THROW(col->insert_many_strings(nullptr, 0)); + } else { + assert_callback(load_cols, serders); + } + } + } + + static void assert_insert_many_strings_overflow( + MutableColumns& load_cols, DataTypeSerDeSPtrs serders, + std::function + assert_callback) { + for (auto& col : load_cols) { + // Create a column to verify `insert_many_strings_overflow` functionality + if (is_column(*col)) { + // just expect throw exception as not support + EXPECT_ANY_THROW(col->insert_many_strings_overflow(nullptr, 0, 0)); + } else { + assert_callback(load_cols, serders); + } + } + } + // assert insert_from // Define the custom assert callback function to verify insert_from behavior - static void assert_insert_from_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders, - std::vector>& assert_res) { + static void assert_insert_from_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { // Create an empty column to verify `insert_from` functionality MutableColumns verify_columns; for (auto& col : load_cols) { @@ -307,7 +367,54 @@ class CommonColumnTest : public ::testing::Test { } res.push_back(data); } - check_res_file("insert_from", assert_res); + check_res_file("insert_from", res); + } + + // insert_from_multi_column (const std::vector< const IColumn * > &srcs, std::vector< size_t > positions) + // speed up for insert_from interface according to avoid virtual call + static void assert_insert_from_multi_column_callback(MutableColumns& load_cols, + DataTypeSerDeSPtrs serders) { + // Create an empty column to verify `insert_from_multi_column` functionality + MutableColumns verify_columns; + size_t max_size = 0; + for (auto& col : load_cols) { + verify_columns.push_back(col->clone_empty()); + max_size = std::max(max_size, col->size()); + } + auto option = DataTypeSerDe::FormatOptions(); + // Insert data from `load_cols` to `verify_columns` using `insert_from_multi_column` + std::vector> res; + std::vector positions = {0, max_size >> 1, max_size - 1}; + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + std::vector s = {source_column.get(), source_column.get(), + source_column.get()}; + auto& target_column = verify_columns[i]; + target_column->insert_from_multi_column(s, positions); + } + + // Verify the inserted data matches the expected results in `assert_res` + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& target_column = verify_columns[i]; + auto ser_col = ColumnString::create(); + ser_col->reserve(target_column->size()); + VectorBufferWriter buffer_writer(*ser_col.get()); + std::vector data; + for (size_t j = 0; j < target_column->size(); ++j) { + data.push_back("now assert insert_from_multi_column for column " + + target_column->get_name() + " at row " + std::to_string(j)); + if (auto st = serders[i]->serialize_one_cell_to_json(*target_column, j, + buffer_writer, option); + !st) { + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; + break; + } + buffer_writer.commit(); + data.push_back(ser_col->get_data_at(j).to_string()); + } + res.push_back(data); + } + check_res_file("insert_from_multi_column", res); } // assert insert_range_from @@ -548,8 +655,7 @@ class CommonColumnTest : public ::testing::Test { check_res_file("insert_indices_from", res); } - static void assert_insert_data_from_callback(MutableColumns& load_cols, - DataTypeSerDeSPtrs serders) { + static void assert_insert_data_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { // Create an empty column to verify `insert_data` functionality MutableColumns verify_columns; for (auto& col : load_cols) { @@ -703,7 +809,7 @@ class CommonColumnTest : public ::testing::Test { check_res_file("insert_many_defaults", res); } - //// data access interfaces: + ////////// =================== column data access interface assert (6)=================== ////////// // virtual StringRef //get_data_at (size_t n) const = 0 // if we implement the get_data_at, we should know the data is stored in the column, and we can get the data by the index @@ -897,29 +1003,30 @@ class CommonColumnTest : public ::testing::Test { check_res_file("get_bool", res); } - //// column meta interfaces: + ////////// =================== column data meta interface assert (7)=================== ////////// // virtual std::string //get_name () const , simple assert to make sure name - static void getNameAssert(IColumn& column, const string expect_name) { + static void assert_get_name(IColumn& column, const string expect_name) { ASSERT_EQ(expect_name, column.get_name()); } + // use in ColumnObject for check_if_sparse_column + static void assert_get_ratio_of_default_rows(MutableColumns& load_cols, + DataTypeSerDeSPtrs serders) { + // just check cols get_ratio_of_default_rows is the same as assert_res + std::vector> res; + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + std::vector data; + data.push_back("in column: " + source_column->get_name() + " ratio of default rows: "); + auto actual_str_value = std::to_string(source_column->get_ratio_of_default_rows()); + data.push_back(actual_str_value); + res.push_back(data); + } + check_res_file("get_ratio_of_default_rows", res); + } + // size related we can check from checked file to make sure the size is right - // - //Returns number of values in column. - //virtual size_t - //size_of_value_if_fixed () const - // - //If values_have_fixed_size, returns size of value, otherwise throw an exception. - //virtual size_t - //byte_size () const =0 - // - //Size of column data in memory (may be approximate) - for profiling. Zero, if could not be determined. - // - // - //virtual size_t - //allocated_bytes () const =0 - // static void assert_size_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { std::vector> res; // just check cols size is the same as assert_res @@ -964,9 +1071,18 @@ class CommonColumnTest : public ::testing::Test { check_res_file("allocated_bytes", res); } - // is_exclusive assert should assert the column which has multiple columnPtr in the column + // empty just use size() == 0 to impl as default behavior + void assert_empty(MutableColumnPtr col) { EXPECT_EQ(col->size(), 0); } + + //The is_exclusive function is implemented differently in different columns, and the correctness should be verified reasonably. + void assert_is_exclusive( + MutableColumns& load_cols, DataTypeSerDeSPtrs serders, + std::function + assert_callback) { + assert_callback(load_cols, serders); + } - //// data manage interfaces + ////////// =================== column data meta manage assert (11)=================== ////////// //virtual void // pop_back (size_t n) = 0 // assert pop_back @@ -1067,8 +1183,6 @@ class CommonColumnTest : public ::testing::Test { check_res_file("clone_resized", res); } - // is_exclusive() means to check the ptr is not shared with other, and we can just clean it. so it's important to check the column is exclusive or not - //virtual Ptr //cut (size_t start, size_t length) const final will call clone_empty and insert_range_from static void assert_cut_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { @@ -1458,20 +1572,7 @@ class CommonColumnTest : public ::testing::Test { check_res_file("append_data_by_selector", res); } - // - //virtual bool - //structure_equals (const IColumn &) const - //void - //copy_date_types (const IColumn &col) - //String - //dump_structure () const - //MutablePtr - //mutate () const && - //virtual void - //clear ()=0 - // - //Clear data of column, just like vector clear. - + ////////// =================== column calculate interface assert (8)=================== ////////// // Column Calculate Interface: filter, compare, permute, sort // filter (const Filter &filt) const =0 // Filter is a array contains 0 or 1 to present the row is selected or not, so it should be the same size with the source column @@ -1640,9 +1741,97 @@ class CommonColumnTest : public ::testing::Test { check_res_file("filter_hint", res); } - // Compare + // sort calculation: (which used in sort_block ) + // get_permutation + // this function helps check permutation result with sort & limit + // by given ColumnValueGetter which how to generate a column value + void assert_column_permutations(vectorized::IColumn& column, DataTypePtr dataType) { + IColumn::Permutation actual_permutation; + IColumn::Permutation expected_permutation; + + static constexpr size_t limit_parts = 4; + printColumn(column, *dataType); + + size_t column_size = column.size(); + size_t column_limit_part = (column_size / limit_parts) + 1; + LOG(INFO) << "column size: " << column_size; + for (size_t limit = 0; limit < column_size; limit += column_limit_part) { + assert_column_permutation(column, true, limit, -1, actual_permutation, + expected_permutation); + assert_column_permutation(column, true, limit, 1, actual_permutation, + expected_permutation); + + assert_column_permutation(column, false, limit, -1, actual_permutation, + expected_permutation); + assert_column_permutation(column, false, limit, 1, actual_permutation, + expected_permutation); + } + } + + // this function helps to check sort permutation behavior for column which use column::compare_at + void stable_get_column_permutation(const IColumn& column, bool ascending, size_t limit, + int nan_direction_hint, + IColumn::Permutation& out_permutation) { + (void)(limit); + + size_t size = column.size(); + out_permutation.resize(size); + std::iota(out_permutation.begin(), out_permutation.end(), + IColumn::Permutation::value_type(0)); + + std::stable_sort(out_permutation.begin(), out_permutation.end(), + [&](size_t lhs, size_t rhs) { + int res = column.compare_at(lhs, rhs, column, nan_direction_hint); + // to check element in column is sorted or not + if (ascending) + return res < 0; + else + return res > 0; + }); + } + // sort calculation: (which used in sort_block ) + // get_permutation means sort data in Column as sort order + // limit should be set to limit the sort result + // nan_direction_hint deal with null|NaN value + void assert_column_permutation(const IColumn& column, bool ascending, size_t limit, + int nan_direction_hint, IColumn::Permutation& actual_permutation, + IColumn::Permutation& expected_permutation) { + LOG(INFO) << "assertColumnPermutation start, limit: " << limit + << " ascending: " << ascending << " nan_direction_hint: " << nan_direction_hint + << " column size: " << column.size() + << " actual_permutation size: " << actual_permutation.size() + << " expected_permutation size: " << expected_permutation.size(); + // step1. get expect permutation as stabled sort + stable_get_column_permutation(column, ascending, limit, nan_direction_hint, + expected_permutation); + // step2. get permutation by column + column.get_permutation(!ascending, limit, nan_direction_hint, actual_permutation); + + if (limit == 0) { + limit = actual_permutation.size(); + } + + // step3. check the permutation result + assert_permutations_with_limit(actual_permutation, expected_permutation, limit); + LOG(INFO) << "assertColumnPermutation done"; + } + + // permute() + // 1/ Key topN set read_orderby_key_reverse = true; SegmentIterator::next_batch will permute the column by the given permutation(which reverse the rows of current segment) + // should check rows with the given permutation + void assert_permute(MutableColumns& cols, IColumn::Permutation& permutation, size_t num_rows) { + std::vector res_permuted; + for (auto& col : cols) { + res_permuted.emplace_back(col->permute(permutation, num_rows)); + } + // check the permutation result for rowsize + size_t res_rows = res_permuted[0]->size(); + for (auto& col : res_permuted) { + EXPECT_EQ(col->size(), res_rows); + } + } - // Column Hash Interfaces: + ////////// =================== column hash interface assert (6)=================== ////////// // update_hashes_with_value (size_t, size_t, Hashes &hashes) const : which inner just use xxhash for column data static void assert_update_hashes_with_value_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { @@ -1730,162 +1919,57 @@ class CommonColumnTest : public ::testing::Test { check_res_file("update_siphashes_hashes", res); } - // column size changed calculation: - // size, reserve, resize, empty, byte_size, allocated_bytes, clone_resized, get_shrinked_column - // cut(LIMIT operation), shrink - - // get_shrinked_column should only happened in char-type column or nested char-type column, - // other column just return the origin column without any data changed, so check file content should be the same as the origin column - // just shrink the end zeros for char-type column which happened in segmentIterator - // eg. column_desc: char(6), insert into char(3), the char(3) will padding the 3 zeros at the end for writing to disk. - // but we select should just print the char(3) without the padding zeros - // limit and topN operation will trigger this function call - void shrink_padding_chars_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { - auto option = DataTypeSerDe::FormatOptions(); - std::vector> res; - for (size_t i = 0; i < load_cols.size(); i++) { - auto& source_column = load_cols[i]; - LOG(INFO) << "now we are in shrink_padding_chars column : " << load_cols[i]->get_name() - << " for column size : " << source_column->size(); - source_column->shrink_padding_chars(); - // check after get_shrinked_column: 1 in selector present the load cols data is selected and data should be default value - auto ser_col = ColumnString::create(); - ser_col->reserve(source_column->size()); - VectorBufferWriter buffer_writer(*ser_col.get()); - std::vector data; - data.push_back("column: " + source_column->get_name() + - " with shrinked column size: " + std::to_string(source_column->size())); - for (size_t j = 0; j < source_column->size(); ++j) { - if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, - buffer_writer, option); - !st) { - LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; - break; + ////////// =================== column serde interface assert (7)=================== ////////// + //serialize and deserialize which usually used in AGG function: + // serialize_value_into_arena, deserialize_and_insert_from_arena (called by AggregateFunctionDistinctMultipleGenericData, group_array_intersect, nested-types serder like: DataTypeArraySerDe::write_one_cell_to_jsonb) + void ser_deserialize_with_arena_impl(MutableColumns& columns, const DataTypes& data_types) { + size_t rows = columns[0]->size(); + for (auto& column : columns) { + if (column->size() > rows) { + LOG(ERROR) << "Column size mismatch: " << column->size() << " vs " << rows; + column->pop_back(column->size() - rows); + } else if (column->size() < rows) { + LOG(ERROR) << "Column size mismatch: " << column->size() << " vs " << rows; + column->insert_many_defaults(rows - column->size()); + } + } + /// check serialization is reversible. + Arena arena; + MutableColumns argument_columns(data_types.size()); + const char* pos = nullptr; + StringRef key(pos, 0); + { + // serialize + for (size_t r = 0; r < columns[0]->size(); ++r) { + for (size_t i = 0; i < columns.size(); ++i) { + auto cur_ref = columns[i]->serialize_value_into_arena(r, arena, pos); + key.data = cur_ref.data - key.size; + key.size += cur_ref.size; + // printColumn(*columns[i], *data_types[i]); } - buffer_writer.commit(); - std::string actual_str_value = ser_col->get_data_at(j).to_string(); - data.push_back(actual_str_value); } - res.push_back(data); } - check_res_file("shrink_padding_chars", res); - } - - void assert_size_eq(MutableColumnPtr col, size_t expect_size) { - EXPECT_EQ(col->size(), expect_size); - } - - // empty just use size() == 0 to impl as default behavior - void assert_empty(MutableColumnPtr col) { EXPECT_EQ(col->size(), 0); } - // reserve, resize, byte_size, allocated_bytes, clone_resized, get_shrinked_column - void assert_reserve_size(MutableColumnPtr col, size_t reserve_size, size_t expect_size) { - col->reserve(reserve_size); - EXPECT_EQ(col->size(), expect_size); - } - - // cut(LIMIT operation) will cut the column with the given from and to, and return the new column - // notice return column is clone from origin column - void assert_cut(MutableColumnPtr col, size_t from, size_t to) { - auto ori = col->size(); - auto ptr = col->cut(from, to); - EXPECT_EQ(ptr->size(), to - from); - EXPECT_EQ(col->size(), ori); - } - - // shrink is cut/append the column with the given size, which called from Block::set_num_rows - // and some Operator may call this set_num_rows to make rows satisfied, like limit operation - // but different from cut behavior which - // return column is mutate from origin column - void assert_shrink(MutableColumnPtr col, size_t shrink_size) { - auto ptr = col->shrink(shrink_size); - EXPECT_EQ(ptr->size(), shrink_size); - EXPECT_EQ(col->size(), shrink_size); - } - - // resize has fixed-column implementation and variable-column implementation - // like string column, the resize will resize the offsets column but not the data column (because it doesn't matter the size of data column, all operation for string column is based on the offsets column) - // like vector column, the resize will resize the data column - // like array column, the resize will resize the offsets column and the data column (which in creator we have check staff for the size of data column is the same as the size of offsets column) - void assert_resize(MutableColumnPtr col, size_t expect_size) { - col->resize(expect_size); - EXPECT_EQ(col->size(), expect_size); - } - - // replicate is clone with new column from the origin column, always from ColumnConst to expand the column - void assert_replicate(MutableColumnPtr col, IColumn::Offsets& offsets) { - auto new_col = col->replicate(offsets); - EXPECT_EQ(new_col->size(), offsets.back()); - } - - // byte size is just appriximate size of the column - // as fixed column type, like column_vector, the byte size is sizeof(columnType) * size() - // as variable column type, like column_string, the byte size is sum of chars size() and offsets size * sizeof(offsetType) - void assert_byte_size(MutableColumnPtr col, size_t expect_size) { - EXPECT_EQ(col->byte_size(), expect_size); - } - - // allocated bytes is the real size of the column - void assert_allocated_bytes(MutableColumnPtr col, size_t expect_size) { - EXPECT_EQ(col->allocated_bytes(), expect_size); - } - - // clone_resized will clone the column and cut/append to the new column with the size of the original column - void assert_clone_resized(MutableColumnPtr col, size_t expect_size) { - auto new_col = col->clone_resized(expect_size); - EXPECT_EQ(new_col->size(), expect_size); - } - - //serialize and deserialize which usually used in AGG function: - // serialize_value_into_arena, deserialize_and_insert_from_arena (called by AggregateFunctionDistinctMultipleGenericData, group_array_intersect, nested-types serder like: DataTypeArraySerDe::write_one_cell_to_jsonb) - void ser_deserialize_with_arena_impl(MutableColumns& columns, const DataTypes& data_types) { - size_t rows = columns[0]->size(); - for (auto& column : columns) { - if (column->size() > rows) { - LOG(ERROR) << "Column size mismatch: " << column->size() << " vs " << rows; - column->pop_back(column->size() - rows); - } else if (column->size() < rows) { - LOG(ERROR) << "Column size mismatch: " << column->size() << " vs " << rows; - column->insert_many_defaults(rows - column->size()); - } - } - /// check serialization is reversible. - Arena arena; - MutableColumns argument_columns(data_types.size()); - const char* pos = nullptr; - StringRef key(pos, 0); - { - // serialize - for (size_t r = 0; r < columns[0]->size(); ++r) { - for (size_t i = 0; i < columns.size(); ++i) { - auto cur_ref = columns[i]->serialize_value_into_arena(r, arena, pos); - key.data = cur_ref.data - key.size; - key.size += cur_ref.size; - // printColumn(*columns[i], *data_types[i]); - } - } - } - - { - // deserialize - for (size_t i = 0; i < data_types.size(); ++i) { - argument_columns[i] = data_types[i]->create_column(); - } - const char* begin = key.data; - for (size_t r = 0; r < columns[0]->size(); ++r) { - for (size_t i = 0; i < argument_columns.size(); ++i) { - begin = argument_columns[i]->deserialize_and_insert_from_arena(begin); - // printColumn(*argument_columns[i], *data_types[i]); - } - } - } - { - // check column data equal - for (size_t i = 0; i < columns.size(); ++i) { - EXPECT_EQ(columns[i]->size(), argument_columns[i]->size()); - checkColumn(*columns[i], *argument_columns[i], *data_types[i], columns[0]->size()); - } - } + { + // deserialize + for (size_t i = 0; i < data_types.size(); ++i) { + argument_columns[i] = data_types[i]->create_column(); + } + const char* begin = key.data; + for (size_t r = 0; r < columns[0]->size(); ++r) { + for (size_t i = 0; i < argument_columns.size(); ++i) { + begin = argument_columns[i]->deserialize_and_insert_from_arena(begin); + // printColumn(*argument_columns[i], *data_types[i]); + } + } + } + { + // check column data equal + for (size_t i = 0; i < columns.size(); ++i) { + EXPECT_EQ(columns[i]->size(), argument_columns[i]->size()); + checkColumn(*columns[i], *argument_columns[i], *data_types[i], columns[0]->size()); + } + } } // serialize_vec, deserialize_vec (called by MethodSerialized.init_serialized_keys), here are some scenarios: @@ -1953,158 +2037,290 @@ class CommonColumnTest : public ::testing::Test { } } - PaddedPODArray create_filter(std::vector data) { - PaddedPODArray filter; - filter.insert(filter.end(), data.begin(), data.end()); - return filter; + ////////// =================== column convert interface assert (5)=================== ////////// + // convert_to_full_column_if_const in ColumnConst will expand the column, if not return itself ptr + static void assert_convert_to_full_column_if_const_callback( + MutableColumns& load_cols, DataTypes typs, std::function assert_func) { + // Create an empty column to verify `convert_to_full_column_if_const` functionality + auto option = DataTypeSerDe::FormatOptions(); + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + auto assert_column = source_column->clone_empty(); + LOG(INFO) << "now we are in convert_to_full_column_if_const column : " + << load_cols[i]->get_name() << " for column size : " << source_column->size(); + auto ptr = source_column->convert_to_full_column_if_const(); + if (is_column(*source_column)) { + // now we should check the ptr is not the same with source_column,we create a new column for the const column + EXPECT_NE(ptr.get(), source_column.get()); + assert_func(ptr); + } else { + EXPECT_EQ(ptr.get(), source_column.get()); + // check the column ptr is the same as the source column + checkColumn(*source_column, *ptr, *typs[i], source_column->size()); + } + } } - - // filter calculation: - // filter (called in Block::filter_block_internal to filter data with filter filled with 0or1 array, like: [0,1,0,1]) - // used in join to filter next block by row_ids, and filter column by row_ids in first read which called in SegmentIterator - void assert_filter(MutableColumnPtr col, std::vector filter, size_t expect_size) { - EXPECT_EQ(col->size(), filter.size()); - auto filted_col = col->filter(create_filter(filter), expect_size); - EXPECT_EQ(filted_col->size(), expect_size); + // convert_column_if_overflow just used in ColumnStr or nested columnStr + static void assert_convert_column_if_overflow_callback(MutableColumns& load_cols, + DataTypeSerDeSPtrs serders) { + // Create an empty column to verify `convert_column_if_overflow` functionality + auto option = DataTypeSerDe::FormatOptions(); + std::vector> res; + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + auto assert_column = source_column->clone_empty(); + LOG(INFO) << "now we are in convert_column_if_overflow column : " + << load_cols[i]->get_name() << " for column size : " << source_column->size(); + auto ptr = source_column->convert_column_if_overflow(); + // check after convert_column_if_overflow: 1 in selector present the load cols data is selected and data should be default value + auto ser_col = ColumnString::create(); + ser_col->reserve(ptr->size()); + VectorBufferWriter buffer_writer(*ser_col.get()); + std::vector data; + data.push_back( + "column: " + ptr->get_name() + + " with convert_column_if_overflow with ptr: " + std::to_string(ptr->size())); + for (size_t j = 0; j < ptr->size(); ++j) { + if (auto st = + serders[i]->serialize_one_cell_to_json(*ptr, j, buffer_writer, option); + !st) { + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; + break; + } + buffer_writer.commit(); + std::string actual_str_value = ser_col->get_data_at(j).to_string(); + data.push_back(actual_str_value); + } + res.push_back(data); + } + check_res_file("convert_column_if_overflow", res); } - // filter_by_selector (called SegmentIterator::copy_column_data_by_selector, - // now just used in filter column, according to the selector to - // select size of row_ids for column by given column, which only used for predict_column and column_dictionary, column_nullable sometimes in Schema::get_predicate_column_ptr() also will return) - void assert_filter_by_selector(vectorized::IColumn::MutablePtr col, - std::vector selector, DataTypeSerDeSPtr serder, - MutableColumnPtr should_sel_col, size_t expect_size) { - // for every data type should assert behavior in own UT case - DataTypeSerDe::FormatOptions option; - col->clear(); - col->insert_many_defaults(should_sel_col->size()); - LOG(INFO) << "col size:" << col->size(); - Status st = col->filter_by_selector(selector.data(), expect_size, should_sel_col.get()); - EXPECT_EQ(st, Status::OK()); + // column_dictionary funcs + // is_column_dictionary + // convert_to_predicate_column_if_dictionary + // If column isn't ColumnDictionary, return itself. Otherwise, transforms is to predicate column. + static void assert_convert_to_predicate_column_if_dictionary_callback( + MutableColumns& load_cols, DataTypes typs, std::function assert_func) { + // Create an empty column to verify `convert_to_predicate_column_if_dictionary` functionality + auto option = DataTypeSerDe::FormatOptions(); std::vector> res; - std::vector data; - auto ser_col = ColumnString::create(); - ser_col->reserve(should_sel_col->size()); - VectorBufferWriter buffer_writer(*ser_col.get()); - data.push_back("column: " + col->get_name() + - " with selector: " + std::to_string(selector.size()) + - " with ptr: " + std::to_string(should_sel_col->size())); - for (size_t j = 0; j < should_sel_col->size(); ++j) { - if (auto ret = serder->serialize_one_cell_to_json(*should_sel_col, j, buffer_writer, - option); - !ret) { - LOG(ERROR) << "Failed to serialize column " << j; - break; + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + LOG(INFO) << "now we are in convert_to_predicate_column_if_dictionary column : " + << load_cols[i]->get_name() << " for column size : " << source_column->size(); + auto ptr = source_column->convert_to_predicate_column_if_dictionary(); + if (source_column->is_column_dictionary()) { + // in dictionary column, we should do some check staff. + EXPECT_NE(ptr.get(), source_column.get()); + assert_func(ptr.get()); + } else { + // just check the column ptr is the same as the source column and res + EXPECT_EQ(ptr.get(), source_column.get()); + checkColumn(*source_column, *ptr, *typs[i], source_column->size()); } - buffer_writer.commit(); - std::string actual_str_value = ser_col->get_data_at(j).to_string(); - data.push_back(actual_str_value); } - res.push_back(data); - check_res_file("filter_by_selector-" + col->get_name(), res); } - void assert_permutations_with_limit(const IColumn::Permutation& lhs, - const IColumn::Permutation& rhs, size_t limit) { - LOG(INFO) << "lhs size: " << lhs.size() << " rhs size: " << rhs.size() - << " limit: " << limit; - if (limit == 0) { - limit = lhs.size(); + // convert_dict_codes_if_necessary just used in ColumnDictionary + // ColumnDictionary and is a range comparison predicate, will convert dict encoding + static void assert_convert_dict_codes_if_necessary_callback( + MutableColumns& load_cols, std::function assert_func) { + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + LOG(INFO) << "now we are in convert_to_predicate_column_if_dictionary column : " + << load_cols[i]->get_name() << " for column size : " << source_column->size(); + EXPECT_NO_FATAL_FAILURE(source_column->convert_dict_codes_if_necessary()); + assert_func(source_column.get()); } + } - for (size_t i = 0; i < limit; ++i) { - ASSERT_EQ(lhs[i], rhs[i]) << "i: " << i << "limit: " << limit; + ////////// =================== column data other interface assert =================== ////////// + // column date or datetime has some weird function which should be deleted in the future + //copy_date_types (const IColumn &col) now we can not delete, just used in ColumnVector to judge the column type is the date or datetime + // which in update_crc_with_value_without_null, called from update_crc_with_value used in situation Crc32HashPartitioner::do_hash + // but it should be deleted in the future, we should not use the column type to judge the column data type and also do not need to set a sign + // for column to present the column belong to datatime or date type + static void assert_copy_date_types_callback(MutableColumns& load_cols) { + //Create an empty column to verify `copy_date_types` functionality + auto option = DataTypeSerDe::FormatOptions(); + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + auto assert_column = source_column->clone_empty(); + LOG(INFO) << "now we are in copy_date_types column : " << load_cols[i]->get_name() + << " for column size : " << source_column->size(); + source_column->copy_date_types(*source_column); + // check after copy_date_types: the column type is the same as the source column + EXPECT_EQ(source_column->is_date_type(), assert_column->is_date_type()); + EXPECT_EQ(source_column->is_datetime_type(), assert_column->is_datetime_type()); } } - // this function helps to check sort permutation behavior for column - void stable_get_column_permutation(const IColumn& column, bool ascending, size_t limit, - int nan_direction_hint, - IColumn::Permutation& out_permutation) { - (void)(limit); + // column_nullable functions + // only_null ; is_null_at ; is_nullable ; has_null ; has_null(size_t) ; + static void assert_column_nullable_funcs(MutableColumns& load_cols, + std::function assert_func) { + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + LOG(INFO) << "now we are in column_nullable_funcs column : " << load_cols[i]->get_name() + << " for column size : " << source_column->size(); + if (source_column->is_nullable() || is_column(*source_column)) { + if (source_column->size() == 1 && source_column->is_null_at(0)) { + EXPECT_EQ(source_column->only_null(), true); + EXPECT_EQ(source_column->has_null(), true); + EXPECT_EQ(source_column->has_null(0), true); + } else { + EXPECT_EQ(source_column->only_null(), false); + } + } else { + assert_func(source_column.get()); + } + } + } - size_t size = column.size(); - out_permutation.resize(size); - std::iota(out_permutation.begin(), out_permutation.end(), - IColumn::Permutation::value_type(0)); + // column_string functions: is_column_string ; is_column_string64 + static void assert_column_string_funcs(MutableColumns& load_cols) { + for (size_t i = 0; i < load_cols.size(); ++i) { + auto& source_column = load_cols[i]; + LOG(INFO) << "now we are in column_string_funcs column : " << load_cols[i]->get_name() + << " for column size : " << source_column->size(); + if (is_column(*source_column)) { + EXPECT_EQ(source_column->is_column_string(), true); + EXPECT_EQ(source_column->is_column_string64(), false); + } else if (is_column(*source_column)) { + EXPECT_EQ(source_column->is_column_string(), false); + EXPECT_EQ(source_column->is_column_string64(), true); + } else { + EXPECT_EQ(source_column->is_column_string(), false); + EXPECT_EQ(source_column->is_column_string64(), false); + } + } + } - std::stable_sort(out_permutation.begin(), out_permutation.end(), - [&](size_t lhs, size_t rhs) { - int res = column.compare_at(lhs, rhs, column, nan_direction_hint); - // to check element in column is sorted or not - if (ascending) - return res < 0; - else - return res > 0; - }); + // get_shrinked_column should only happened in char-type column or nested char-type column, + // other column just return the origin column without any data changed, so check file content should be the same as the origin column + // just shrink the end zeros for char-type column which happened in segmentIterator + // eg. column_desc: char(6), insert into char(3), the char(3) will padding the 3 zeros at the end for writing to disk. + // but we select should just print the char(3) without the padding zeros + // limit and topN operation will trigger this function call + void shrink_padding_chars_callback(MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { + auto option = DataTypeSerDe::FormatOptions(); + std::vector> res; + for (size_t i = 0; i < load_cols.size(); i++) { + auto& source_column = load_cols[i]; + LOG(INFO) << "now we are in shrink_padding_chars column : " << load_cols[i]->get_name() + << " for column size : " << source_column->size(); + source_column->shrink_padding_chars(); + // check after get_shrinked_column: 1 in selector present the load cols data is selected and data should be default value + auto ser_col = ColumnString::create(); + ser_col->reserve(source_column->size()); + VectorBufferWriter buffer_writer(*ser_col.get()); + std::vector data; + data.push_back("column: " + source_column->get_name() + + " with shrinked column size: " + std::to_string(source_column->size())); + for (size_t j = 0; j < source_column->size(); ++j) { + if (auto st = serders[i]->serialize_one_cell_to_json(*source_column, j, + buffer_writer, option); + !st) { + LOG(ERROR) << "Failed to serialize column " << i << " at row " << j; + break; + } + buffer_writer.commit(); + std::string actual_str_value = ser_col->get_data_at(j).to_string(); + data.push_back(actual_str_value); + } + res.push_back(data); + } + check_res_file("shrink_padding_chars", res); } - // sort calculation: (which used in sort_block ) - // get_permutation - // this function helps check permutation result with sort & limit - // by given ColumnValueGetter which how to generate a column value - void assert_column_permutations(vectorized::IColumn& column, DataTypePtr dataType) { - IColumn::Permutation actual_permutation; - IColumn::Permutation expected_permutation; + void assert_size_eq(MutableColumnPtr col, size_t expect_size) { + EXPECT_EQ(col->size(), expect_size); + } - static constexpr size_t limit_parts = 4; - printColumn(column, *dataType); + // reserve, resize, byte_size, allocated_bytes, clone_resized, get_shrinked_column + void assert_reserve_size(MutableColumnPtr col, size_t reserve_size, size_t expect_size) { + col->reserve(reserve_size); + EXPECT_EQ(col->size(), expect_size); + } - size_t column_size = column.size(); - size_t column_limit_part = (column_size / limit_parts) + 1; - LOG(INFO) << "column size: " << column_size; - for (size_t limit = 0; limit < column_size; limit += column_limit_part) { - assert_column_permutation(column, true, limit, -1, actual_permutation, - expected_permutation); - assert_column_permutation(column, true, limit, 1, actual_permutation, - expected_permutation); + // cut(LIMIT operation) will cut the column with the given from and to, and return the new column + // notice return column is clone from origin column + void assert_cut(MutableColumnPtr col, size_t from, size_t to) { + auto ori = col->size(); + auto ptr = col->cut(from, to); + EXPECT_EQ(ptr->size(), to - from); + EXPECT_EQ(col->size(), ori); + } - assert_column_permutation(column, false, limit, -1, actual_permutation, - expected_permutation); - assert_column_permutation(column, false, limit, 1, actual_permutation, - expected_permutation); - } + // shrink is cut/append the column with the given size, which called from Block::set_num_rows + // and some Operator may call this set_num_rows to make rows satisfied, like limit operation + // but different from cut behavior which + // return column is mutate from origin column + void assert_shrink(MutableColumnPtr col, size_t shrink_size) { + auto ptr = col->shrink(shrink_size); + EXPECT_EQ(ptr->size(), shrink_size); + EXPECT_EQ(col->size(), shrink_size); } - // sort calculation: (which used in sort_block ) - // get_permutation means sort data in Column as sort order - // limit should be set to limit the sort result - // nan_direction_hint deal with null|NaN value - void assert_column_permutation(const IColumn& column, bool ascending, size_t limit, - int nan_direction_hint, IColumn::Permutation& actual_permutation, - IColumn::Permutation& expected_permutation) { - LOG(INFO) << "assertColumnPermutation start, limit: " << limit - << " ascending: " << ascending << " nan_direction_hint: " << nan_direction_hint - << " column size: " << column.size() - << " actual_permutation size: " << actual_permutation.size() - << " expected_permutation size: " << expected_permutation.size(); - // step1. get expect permutation as stabled sort - stable_get_column_permutation(column, ascending, limit, nan_direction_hint, - expected_permutation); - // step2. get permutation by column - column.get_permutation(!ascending, limit, nan_direction_hint, actual_permutation); + // resize has fixed-column implementation and variable-column implementation + // like string column, the resize will resize the offsets column but not the data column (because it doesn't matter the size of data column, all operation for string column is based on the offsets column) + // like vector column, the resize will resize the data column + // like array column, the resize will resize the offsets column and the data column (which in creator we have check staff for the size of data column is the same as the size of offsets column) + void assert_resize(MutableColumnPtr col, size_t expect_size) { + col->resize(expect_size); + EXPECT_EQ(col->size(), expect_size); + } - if (limit == 0) { - limit = actual_permutation.size(); - } + // replicate is clone with new column from the origin column, always from ColumnConst to expand the column + void assert_replicate(MutableColumnPtr col, IColumn::Offsets& offsets) { + auto new_col = col->replicate(offsets); + EXPECT_EQ(new_col->size(), offsets.back()); + } - // step3. check the permutation result - assert_permutations_with_limit(actual_permutation, expected_permutation, limit); - LOG(INFO) << "assertColumnPermutation done"; + // byte size is just appriximate size of the column + // as fixed column type, like column_vector, the byte size is sizeof(columnType) * size() + // as variable column type, like column_string, the byte size is sum of chars size() and offsets size * sizeof(offsetType) + void assert_byte_size(MutableColumnPtr col, size_t expect_size) { + EXPECT_EQ(col->byte_size(), expect_size); } - // permute() - // 1/ Key topN set read_orderby_key_reverse = true; SegmentIterator::next_batch will permute the column by the given permutation(which reverse the rows of current segment) - // should check rows with the given permutation - void assert_permute(MutableColumns& cols, IColumn::Permutation& permutation, size_t num_rows) { - std::vector res_permuted; - for (auto& col : cols) { - res_permuted.emplace_back(col->permute(permutation, num_rows)); + // allocated bytes is the real size of the column + void assert_allocated_bytes(MutableColumnPtr col, size_t expect_size) { + EXPECT_EQ(col->allocated_bytes(), expect_size); + } + + // clone_resized will clone the column and cut/append to the new column with the size of the original column + void assert_clone_resized(MutableColumnPtr col, size_t expect_size) { + auto new_col = col->clone_resized(expect_size); + EXPECT_EQ(new_col->size(), expect_size); + } + + PaddedPODArray create_filter(std::vector data) { + PaddedPODArray filter; + filter.insert(filter.end(), data.begin(), data.end()); + return filter; + } + + // filter calculation: + // filter (called in Block::filter_block_internal to filter data with filter filled with 0or1 array, like: [0,1,0,1]) + // used in join to filter next block by row_ids, and filter column by row_ids in first read which called in SegmentIterator + void assert_filter(MutableColumnPtr col, std::vector filter, size_t expect_size) { + EXPECT_EQ(col->size(), filter.size()); + auto filted_col = col->filter(create_filter(filter), expect_size); + EXPECT_EQ(filted_col->size(), expect_size); + } + + void assert_permutations_with_limit(const IColumn::Permutation& lhs, + const IColumn::Permutation& rhs, size_t limit) { + LOG(INFO) << "lhs size: " << lhs.size() << " rhs size: " << rhs.size() + << " limit: " << limit; + if (limit == 0) { + limit = lhs.size(); } - // check the permutation result for rowsize - size_t res_rows = res_permuted[0]->size(); - for (auto& col : res_permuted) { - EXPECT_EQ(col->size(), res_rows); + + for (size_t i = 0; i < limit; ++i) { + ASSERT_EQ(lhs[i], rhs[i]) << "i: " << i << "limit: " << limit; } } }; From 0c85aa9f43ccd20d7d1143d30a29a73bd7d5bbcf Mon Sep 17 00:00:00 2001 From: amorynan Date: Tue, 7 Jan 2025 15:20:56 +0800 Subject: [PATCH 21/23] add case --- be/test/vec/columns/column_array_test.cpp | 7 +++++-- be/test/vec/columns/common_column_test.h | 17 ++++++++++------ .../array/test_array_array_bool.csv | 19 ++++++++++++++++++ .../array/test_array_bool.csv | 20 +++++++++++++++++++ 4 files changed, 55 insertions(+), 8 deletions(-) create mode 100644 regression-test/data/nereids_function_p0/array/test_array_array_bool.csv create mode 100644 regression-test/data/nereids_function_p0/array/test_array_bool.csv diff --git a/be/test/vec/columns/column_array_test.cpp b/be/test/vec/columns/column_array_test.cpp index 16d197cc950fa2..dc2f9283dfd317 100644 --- a/be/test/vec/columns/column_array_test.cpp +++ b/be/test/vec/columns/column_array_test.cpp @@ -370,7 +370,7 @@ TEST_F(ColumnArrayTest, InsertManyDictDataTest) { TEST_F(ColumnArrayTest, InsertManyContinuousBinaryDataTest) { auto callback = [&](MutableColumns& load_cols, DataTypeSerDeSPtrs serders) { for (auto& col : array_columns) { - EXPECT_ANY_THROW(col->insert_many_continuous_binary_data(nullptr, 0, 0)); + EXPECT_ANY_THROW(col->insert_many_continuous_binary_data(nullptr, 0, 1)); } }; assert_insert_many_continuous_binary_data(array_columns, serdes, callback); @@ -599,7 +599,10 @@ TEST_F(ColumnArrayTest, ConvertToPredicateColumnIfDictionaryTest) { // test assert_convert_dict_codes_if_necessary_callback TEST_F(ColumnArrayTest, ConvertDictCodesIfNecessaryTest) { - assert_convert_dict_codes_if_necessary_callback(array_columns, nullptr); + auto callback = [&](IColumn* col, size_t index) { + checkColumn(*col->get_ptr(), *array_columns[index], *array_types[index], col->size()); + }; + assert_convert_dict_codes_if_necessary_callback(array_columns, callback); } // test assert_copy_date_types_callback diff --git a/be/test/vec/columns/common_column_test.h b/be/test/vec/columns/common_column_test.h index 5e82889a3bc50e..a6129a2d3c0e71 100644 --- a/be/test/vec/columns/common_column_test.h +++ b/be/test/vec/columns/common_column_test.h @@ -308,7 +308,7 @@ class CommonColumnTest : public ::testing::Test { assert_callback) { for (auto& col : load_cols) { // Create a column to verify `insert_many_strings` functionality - if (is_column(*col)) { + if (!is_column(*col)) { EXPECT_ANY_THROW(col->insert_many_strings(nullptr, 0)); } else { assert_callback(load_cols, serders); @@ -322,7 +322,7 @@ class CommonColumnTest : public ::testing::Test { assert_callback) { for (auto& col : load_cols) { // Create a column to verify `insert_many_strings_overflow` functionality - if (is_column(*col)) { + if (!is_column(*col)) { // just expect throw exception as not support EXPECT_ANY_THROW(col->insert_many_strings_overflow(nullptr, 0, 0)); } else { @@ -376,16 +376,15 @@ class CommonColumnTest : public ::testing::Test { DataTypeSerDeSPtrs serders) { // Create an empty column to verify `insert_from_multi_column` functionality MutableColumns verify_columns; - size_t max_size = 0; for (auto& col : load_cols) { verify_columns.push_back(col->clone_empty()); - max_size = std::max(max_size, col->size()); } auto option = DataTypeSerDe::FormatOptions(); // Insert data from `load_cols` to `verify_columns` using `insert_from_multi_column` std::vector> res; - std::vector positions = {0, max_size >> 1, max_size - 1}; for (size_t i = 0; i < load_cols.size(); ++i) { + size_t si = load_cols[i]->size(); + std::vector positions = {0, si >> 1, si - 1}; auto& source_column = load_cols[i]; std::vector s = {source_column.get(), source_column.get(), source_column.get()}; @@ -1254,12 +1253,18 @@ class CommonColumnTest : public ::testing::Test { LOG(INFO) << "now we are in shrink column : " << load_cols[i]->get_name() << " with check length: " << insert_size << " for column size : " << source_column->size(); + size_t cnt = source_column->use_count(); ptr = source_column->shrink(insert_size); LOG(INFO) << "use_count : " << source_column->use_count(); // check size EXPECT_EQ(ptr->size(), insert_size); // check ptr is not the same - EXPECT_NE(ptr.get(), source_column.get()); + if (cnt == 1) { + // just return the origin column to avoid the copy + EXPECT_EQ(ptr.get(), source_column.get()); + } else { + EXPECT_NE(ptr.get(), source_column.get()); + } // check after cut with assert_res auto ser_col = ColumnString::create(); ser_col->reserve(ptr->size()); diff --git a/regression-test/data/nereids_function_p0/array/test_array_array_bool.csv b/regression-test/data/nereids_function_p0/array/test_array_array_bool.csv new file mode 100644 index 00000000000000..1ceeef29194079 --- /dev/null +++ b/regression-test/data/nereids_function_p0/array/test_array_array_bool.csv @@ -0,0 +1,19 @@ +// this is data case file for array type, contains normal, corner, exceptional data cases +[[]] +[[null], [null], [null], [null], [null], [null], [null], [null], [null], [null]] +NULL +[[0], [0], [0], [0], [0], [0], [0], [0], [0], [0]] +// min/max value +[[0, 1], [2, 3], [4, 5], [6, 7], [8, 9], [-1, -2]] +// int min/max value + 1 (异常) +[[-2147483647, 2147483648]] +// int min/max value - 1 (异常) +[[-2147483649, 2147483646]] +[[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, null, 11, 12, 13, 14, 15, 16]] +[[true], [false], [true], [false], [true], [false], [true], [false], [true], [false]] +// all data in array + null +[[true], [false], [true], [false], [true], [false], [true], [false], [true], [false], [null]] +// all data in array, with null +[[true], [false], [true], [false], [true], [false], [true], [false], [true], [false], null, [true], [false], [true], [false], [true], [false], [true], [false], [true], [false]] +// all data in array, with null, and min/max value +[[null], [false], [true], [false], [true], [false], [true], [false], [true], [false], null, [true], [false], [true], [false], [true], [false], [true], [false], [true], [false], [-2147483647, 2147483648], [0, 1]] \ No newline at end of file diff --git a/regression-test/data/nereids_function_p0/array/test_array_bool.csv b/regression-test/data/nereids_function_p0/array/test_array_bool.csv new file mode 100644 index 00000000000000..ad12cb1cddf26d --- /dev/null +++ b/regression-test/data/nereids_function_p0/array/test_array_bool.csv @@ -0,0 +1,20 @@ +// this is data case file for array type, contains normal, corner, exceptional data cases +[] +[null] +NULL +// single element +[0] +// min/max value +[0, 1] +// int min/max value + 1 (异常) +[-2147483647, 2147483648] +// int min/max value - 1 (异常) +[-2147483649, 2147483646] +[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, null, 11, 12, 13, 14, 15, 16] +[true, false, true, false, true, false, true, false, true, false] +// all data in array + null +[true, false, true, false, true, false, true, false, true, false, null] +// all data in array, with null +[true, false, true, false, true, false, true, false, true, false, null, true, false, true, false, true, false, true, false, true, false] +// all data in array, with null, and min/max value +[true, false, true, false, true, false, true, false, true, false, null, true, false, true, false, true, false, true, false, true, false, -2147483647, 2147483648] \ No newline at end of file From 13ac07e3a677ee63e7444f334701bb2d51165e9f Mon Sep 17 00:00:00 2001 From: amorynan Date: Tue, 7 Jan 2025 22:21:50 +0800 Subject: [PATCH 22/23] =?UTF-8?q?update=E2=80=9C?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- be/src/vec/columns/column.h | 13 +++++++++++-- be/src/vec/columns/column_array.cpp | 10 ---------- be/src/vec/columns/column_array.h | 2 -- 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index eb76b18386411e..28f2c0a9321742 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -149,7 +149,12 @@ class IColumn : public COW { /// If possible, returns pointer to memory chunk which contains n-th element (if it isn't possible, throws an exception) /// Is used to optimize some computations (in aggregation, for example). - virtual StringRef get_data_at(size_t n) const = 0; + /// this function is used in ColumnString, ColumnFixedString, ColumnVector, not support in ColumnArray|ColumnMap... + /// and should be pair with insert_data + virtual StringRef get_data_at(size_t n) const { + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "Method get_data_at is not supported for " + get_name()); + } virtual Int64 get_int(size_t /*n*/) const { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, @@ -229,10 +234,14 @@ class IColumn : public COW { const uint32_t* indices_end) = 0; /// Appends data located in specified memory chunk if it is possible (throws an exception if it cannot be implemented). + /// used in ColumnString, ColumnFixedString, ColumnVector, not support in ColumnArray|ColumnMap... /// Is used to optimize some computations (in aggregation, for example). /// Parameter length could be ignored if column values have fixed size. /// All data will be inserted as single element - virtual void insert_data(const char* pos, size_t length) = 0; + virtual void insert_data(const char* pos, size_t length) { + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "Method insert_data is not supported for " + get_name()); + } virtual void insert_many_fix_len_data(const char* pos, size_t num) { throw doris::Exception( diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp index c5e85401b005ef..0428dac17945e2 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -152,16 +152,6 @@ void ColumnArray::get(size_t n, Field& res) const { for (size_t i = 0; i < size; ++i) get_data().get(offset + i, res_arr[i]); } -StringRef ColumnArray::get_data_at(size_t n) const { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "Method get_data_at is not supported for " + get_name()); -} - -void ColumnArray::insert_data(const char* pos, size_t length) { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "Method insert_data is not supported for " + get_name()); -} - bool ColumnArray::is_default_at(size_t n) const { const auto& offsets_data = get_offsets(); return offsets_data[n] == offsets_data[static_cast(n) - 1]; diff --git a/be/src/vec/columns/column_array.h b/be/src/vec/columns/column_array.h index 510cf2ae3ca130..8f35af3ffb7f05 100644 --- a/be/src/vec/columns/column_array.h +++ b/be/src/vec/columns/column_array.h @@ -129,9 +129,7 @@ class ColumnArray final : public COWHelper { void resize(size_t n) override; Field operator[](size_t n) const override; void get(size_t n, Field& res) const override; - StringRef get_data_at(size_t n) const override; bool is_default_at(size_t n) const; - void insert_data(const char* pos, size_t length) override; 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; From 8df94c46dbd256698f9944adafc5338f59c0411c Mon Sep 17 00:00:00 2001 From: amorynan Date: Wed, 8 Jan 2025 09:41:55 +0800 Subject: [PATCH 23/23] fix code --- be/test/vec/columns/common_column_test.h | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/be/test/vec/columns/common_column_test.h b/be/test/vec/columns/common_column_test.h index a6129a2d3c0e71..9cb455e1de267c 100644 --- a/be/test/vec/columns/common_column_test.h +++ b/be/test/vec/columns/common_column_test.h @@ -567,6 +567,12 @@ class CommonColumnTest : public ::testing::Test { << source_column->get_name() << " with source size: " << source_column->size(); target_column->insert_many_from(*source_column, *pos, *cl); + } else { + target_column->clear(); + LOG(INFO) << "now insert_many_from from " << *pos << " with length " << *cl + << " for column " << source_column->get_name() + << " with source size: " << source_column->size(); + target_column->insert_many_from(*source_column, *pos, *cl); } // Verify the inserted data matches the expected results in `assert_res` @@ -574,6 +580,9 @@ class CommonColumnTest : public ::testing::Test { ser_col->reserve(target_column->size()); VectorBufferWriter buffer_writer(*ser_col.get()); std::vector data; + data.push_back("now assert insert_many_from for column " + + target_column->get_name() + " from " + std::to_string(*pos) + + " with length " + std::to_string(*cl)); for (size_t j = 0; j < target_column->size(); ++j) { if (auto st = serders[i]->serialize_one_cell_to_json(*target_column, j, buffer_writer, option); @@ -2130,13 +2139,13 @@ class CommonColumnTest : public ::testing::Test { // convert_dict_codes_if_necessary just used in ColumnDictionary // ColumnDictionary and is a range comparison predicate, will convert dict encoding static void assert_convert_dict_codes_if_necessary_callback( - MutableColumns& load_cols, std::function assert_func) { + MutableColumns& load_cols, std::function assert_func) { for (size_t i = 0; i < load_cols.size(); ++i) { auto& source_column = load_cols[i]; LOG(INFO) << "now we are in convert_to_predicate_column_if_dictionary column : " << load_cols[i]->get_name() << " for column size : " << source_column->size(); EXPECT_NO_FATAL_FAILURE(source_column->convert_dict_codes_if_necessary()); - assert_func(source_column.get()); + assert_func(source_column.get(), i); } }