This is an automated email from the ASF dual-hosted git repository. dataroaring pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-3.0 by this push: new 06dfd05c4b3 branch-3.0: [improve](ut) update data type ut and take datatype ip for example #44642 (#44903) 06dfd05c4b3 is described below commit 06dfd05c4b3168a0b009e3a033e63a529ef13973 Author: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> AuthorDate: Wed Dec 25 10:22:30 2024 +0800 branch-3.0: [improve](ut) update data type ut and take datatype ip for example #44642 (#44903) Cherry-picked from #44642 Co-authored-by: amory <wangqian...@selectdb.com> --- .../vec/data_types/common_data_type_serder_test.h | 366 +++++++++++++++++++++ be/test/vec/data_types/common_data_type_test.h | 228 +++++++++++++ be/test/vec/data_types/data_type_ip_test.cpp | 252 ++++++++++++++ 3 files changed, 846 insertions(+) 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 00000000000..46206d5ed7e --- /dev/null +++ b/be/test/vec/data_types/common_data_type_serder_test.h @@ -0,0 +1,366 @@ +// 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 <arrow/record_batch.h> +#include <gen_cpp/data.pb.h> +#include <gtest/gtest-message.h> +#include <gtest/gtest-test-part.h> +#include <gtest/gtest.h> + +#include <filesystem> +#include <fstream> +#include <iostream> + +#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 +// 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<int> idxes, const std::string& column_data_file, + std::function<void(MutableColumns& load_cols, DataTypeSerDeSPtrs serders)> + 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<std::vector<std::string>> 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<true, true>(serders, columns, column_data_file, + col_spliter, idxes); + } else { + load_data_and_assert_from_csv<false, true>(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<true, true>(serders, columns, file_path, + col_spliter, idxes); + } else { + load_data_and_assert_from_csv<false, true>(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 <bool is_hive_format, bool generate_res_file> + static void load_data_and_assert_from_csv(const DataTypeSerDeSPtrs serders, + MutableColumns& columns, const std::string& file_path, + const char spliter = ';', + const std::set<int> 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<std::vector<std::string>> 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<string> 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<ColumnString&>(*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("."); + 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<JsonbOutStream> 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<false> 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<Block>(); + 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<arrow::Schema> block_arrow_schema; + EXPECT_EQ(get_arrow_schema_from_block(*block, &block_arrow_schema, "UTC"), Status::OK()); + // convert block to arrow + std::shared_ptr<arrow::RecordBatch> 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 new file mode 100644 index 00000000000..0480137ac7f --- /dev/null +++ b/be/test/vec/data_types/common_data_type_test.h @@ -0,0 +1,228 @@ +// 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 <gen_cpp/data.pb.h> +#include <gtest/gtest-message.h> +#include <gtest/gtest-test-part.h> +#include <gtest/gtest.h> + +#include <filesystem> +#include <fstream> +#include <iostream> + +#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_represented_by_integer, is_value_represented_by_unsigned_integer, 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<int> 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; + bool is_value_represented_by_unsigned_integer = 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_represented_by_unsigned_integer(), + meta_info.is_value_represented_by_unsigned_integer); + // 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/data_types/data_type_ip_test.cpp b/be/test/vec/data_types/data_type_ip_test.cpp new file mode 100644 index 00000000000..91ae064e447 --- /dev/null +++ b/be/test/vec/data_types/data_type_ip_test.cpp @@ -0,0 +1,252 @@ +// 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 <gtest/gtest-message.h> +#include <gtest/gtest-test-part.h> +#include <gtest/gtest.h> + +#include <filesystem> +#include <fstream> +#include <iostream> + +#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" +#include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_factory.hpp" +#include "vec/data_types/data_type_nullable.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 +// is_null_literal, is_value_represented_by_number, is_value_represented_by_integer, is_value_represented_by_unsigned_integer, is_value_unambiguously_represented_in_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) +// 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 +// 5. others: update_avg_value_size_hint (const IColumn &column, double &avg_value_size_hint) + +namespace doris::vectorized { + +class DataTypeIPTest : public CommonDataTypeTest { +protected: + void SetUp() override { + data_files = {"regression-test/data/nereids_function_p0/fn_test_ip_invalid.csv", + "regression-test/data/nereids_function_p0/fn_test_ip_normal.csv", + "regression-test/data/nereids_function_p0/fn_test_ip_nullable.csv", + "regression-test/data/nereids_function_p0/fn_test_ip_special.csv", + "regression-test/data/nereids_function_p0/fn_test_ip_special_no_null.csv"}; + } + +public: + DataTypePtr dt_ipv4 = + DataTypeFactory::instance().create_data_type(FieldType::OLAP_FIELD_TYPE_IPV4, 0, 0); + DataTypePtr dt_ipv6 = + DataTypeFactory::instance().create_data_type(FieldType::OLAP_FIELD_TYPE_IPV6, 0, 0); + DataTypePtr dt_ipv4_nullable = std::make_shared<vectorized::DataTypeNullable>(dt_ipv4); + DataTypePtr dt_ipv6_nullable = std::make_shared<vectorized::DataTypeNullable>(dt_ipv6); + // common ip data + std::vector<string> data_files; +}; + +TEST_F(DataTypeIPTest, MetaInfoTest) { + TypeDescriptor ipv4_type_descriptor = {PrimitiveType::TYPE_IPV4}; + auto col_meta = std::make_shared<PColumnMeta>(); + col_meta->set_type(PGenericType_TypeId_IPV4); + DataTypeMetaInfo ipv4_meta_info_to_assert = { + .type_id = TypeIndex::IPv4, + .type_as_type_descriptor = &ipv4_type_descriptor, + .family_name = "IPv4", + .has_subtypes = false, + .storage_field_type = doris::FieldType::OLAP_FIELD_TYPE_IPV4, + .should_align_right_in_pretty_formats = true, + .text_can_contain_only_valid_utf8 = true, + .have_maximum_size_of_value = true, + .size_of_value_in_memory = sizeof(IPv4), + .precision = size_t(-1), + .scale = size_t(-1), + .is_null_literal = false, + .is_value_represented_by_number = true, + .is_value_represented_by_unsigned_integer = true, + .pColumnMeta = col_meta.get() + // .is_value_unambiguously_represented_in_contiguous_memory_region = true + }; + TypeDescriptor ipv6_type_descriptor = {PrimitiveType::TYPE_IPV6}; + auto col_meta6 = std::make_shared<PColumnMeta>(); + col_meta6->set_type(PGenericType_TypeId_IPV6); + DataTypeMetaInfo ipv6_meta_info = { + .type_id = TypeIndex::IPv6, + .type_as_type_descriptor = &ipv6_type_descriptor, + .family_name = "IPv6", + .has_subtypes = false, + .storage_field_type = doris::FieldType::OLAP_FIELD_TYPE_IPV6, + .should_align_right_in_pretty_formats = true, + .text_can_contain_only_valid_utf8 = true, + .have_maximum_size_of_value = true, + .size_of_value_in_memory = sizeof(IPv6), + .precision = size_t(-1), + .scale = size_t(-1), + .is_null_literal = false, + .is_value_represented_by_number = true, + .is_value_represented_by_unsigned_integer = true, + .pColumnMeta = col_meta6.get() + // .is_value_unambiguously_represented_in_contiguous_memory_region = true + }; + meta_info_assert(dt_ipv4, ipv4_meta_info_to_assert); + meta_info_assert(dt_ipv6, ipv6_meta_info); +} + +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); +} + +TEST_F(DataTypeIPTest, GetFieldTest) { + 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::load_data_and_assert_from_csv<true, true>(serde, ip_cols, + data_files[1], ';', {1, 2}); + TExprNode node_ipv4; + node_ipv4.node_type = TExprNodeType::IPV4_LITERAL; + for (size_t i = 0; i < ip_cols[0]->size(); ++i) { + node_ipv4.ipv4_literal.value = ip_cols[0]->get_int(i); + Field assert_field; + ip_cols[0]->get(i, assert_field); + get_field_assert(dt_ipv4, node_ipv4, assert_field); + } + + TExprNode node_ipv6; + node_ipv6.node_type = TExprNodeType::IPV6_LITERAL; + for (size_t i = 0; i < ip_cols[1]->size(); ++i) { + node_ipv6.ipv6_literal.value = + IPv6Value::to_string(assert_cast<ColumnIPv6&>(*ip_cols[1]).get_data()[i]); + Field assert_field; + ip_cols[1]->get(i, assert_field); + get_field_assert(dt_ipv6, node_ipv6, assert_field); + } + + TExprNode invalid_node_ipv6; + invalid_node_ipv6.node_type = TExprNodeType::IPV6_LITERAL; + // todo.(check) 2001:db8:::1 this is invalid ipv6 value, but it can pass the test + std::vector<string> invalid_ipv6 = {"2001:db8::12345", + "", + "::fffff:0:0", + "2001:db8::g123", + "2001:db8:85a3::8a2e:0370:", + "2001:0db8:85a3:0000:0000:8a2e:0370:7334:1234", + "::12345:abcd"}; + for (auto& ipv6 : invalid_ipv6) { + invalid_node_ipv6.ipv6_literal.value = ipv6; + Field field; + get_field_assert(dt_ipv6, invalid_node_ipv6, field, true); + } +} + +TEST_F(DataTypeIPTest, FromAndToStringTest) { + 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()}; + load_data_from_csv(serde, ip_cols, data_files[0], ';', {1, 2}); + // test ipv4 + assert_to_string_from_string_assert(ip_cols[0]->get_ptr(), dt_ipv4); + // test ipv6 + assert_to_string_from_string_assert(ip_cols[1]->get_ptr(), dt_ipv6); +} + +TEST_F(DataTypeIPTest, CompareTest) { + assert_compare_behavior(dt_ipv4, dt_ipv6); +} + +TEST_F(DataTypeIPTest, SerdeHiveTextAndJsonFormatTest) { + 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::load_data_and_assert_from_csv<true, true>(serde, ip_cols, + data_files[1], ';', {1, 2}); + CommonDataTypeSerdeTest::load_data_and_assert_from_csv<false, true>(serde, ip_cols, + data_files[1], ';', {1, 2}); +} + +TEST_F(DataTypeIPTest, SerdePbTest) { + 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_pb_format); +} + +TEST_F(DataTypeIPTest, SerdeJsonbTest) { + 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_jsonb_format); +} + +} // namespace doris::vectorized \ No newline at end of file --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org