From 1ed9aa6e7f7db9833fa16f1e7d9ed390eb299a71 Mon Sep 17 00:00:00 2001 From: daidai Date: Fri, 4 Jul 2025 14:58:33 +0800 Subject: [PATCH 1/4] [feature](external) Support reading Hudi/Paimon/Iceberg tables after schema changes. (#51341) Related PR: #49051 Problem Summary: Support reading Hudi and Paimon Iceberg tables after the internal schema of struct is changed. 1. Introduce `hive_reader` to avoid confusion between `hive` and `parquet/orc` reader 2. Before this, support for reading tables after schema changes of ordinary columns relied on changing the column name in block, so that parquet/orc reader can read specific file columns when `get_next_block`, and `hudi/iceberg/paimon reader` will mix `file column names` with `table column names` when using parquet/orc reader. This pr clarifies that all calls to `parquet/orc reader` are based on the concept of `table column names`, and then introduces `TableSchemaChangeHelper::Node` to help `parquet/orc reader` find the specific file columns to be read. --- be/src/olap/push_handler.cpp | 8 +- be/src/runtime/descriptors.h | 8 +- be/src/vec/exec/format/orc/vorc_reader.cpp | 378 +++--- be/src/vec/exec/format/orc/vorc_reader.h | 93 +- .../vec/exec/format/parquet/schema_desc.cpp | 30 +- be/src/vec/exec/format/parquet/schema_desc.h | 11 +- .../format/parquet/vparquet_column_reader.cpp | 64 +- .../format/parquet/vparquet_column_reader.h | 22 +- .../format/parquet/vparquet_group_reader.cpp | 31 +- .../format/parquet/vparquet_group_reader.h | 9 +- .../exec/format/parquet/vparquet_reader.cpp | 149 +-- .../vec/exec/format/parquet/vparquet_reader.h | 32 +- be/src/vec/exec/format/table/hive_reader.cpp | 132 ++ be/src/vec/exec/format/table/hive_reader.h | 83 ++ be/src/vec/exec/format/table/hudi_reader.cpp | 59 +- be/src/vec/exec/format/table/hudi_reader.h | 24 - .../vec/exec/format/table/iceberg_reader.cpp | 128 +- be/src/vec/exec/format/table/iceberg_reader.h | 11 +- .../vec/exec/format/table/paimon_reader.cpp | 18 - be/src/vec/exec/format/table/paimon_reader.h | 42 +- .../exec/format/table/table_format_reader.cpp | 687 +++++++++-- .../exec/format/table/table_format_reader.h | 292 ++++- .../table/transactional_hive_reader.cpp | 65 +- .../format/table/transactional_hive_reader.h | 2 +- be/src/vec/exec/scan/vfile_scanner.cpp | 199 ++- be/src/vec/exec/scan/vfile_scanner.h | 10 +- .../paimon/paimon_schema_change_test.cpp | 149 --- .../format/parquet/parquet_reader_test.cpp | 97 +- .../table/table_schema_change_helper_test.cpp | 1087 +++++++++++------ .../exec/orc/orc_reader_fill_data_test.cpp | 30 +- .../exec/orc/orc_reader_init_column_test.cpp | 140 +-- be/test/vec/exec/orc_reader_test.cpp | 4 +- .../docker-compose/iceberg/entrypoint.sh.tpl | 2 +- .../docker-compose/iceberg/iceberg.yaml.tpl | 2 - .../iceberg/run01.sql | 2 +- .../iceberg/run02.sql | 1 + .../iceberg/run03.sql | 1 + .../iceberg/run04.sql | 2 +- .../iceberg/run06.sql | 1 + .../iceberg/run07.sql | 2 +- .../iceberg/run08.sql | 2 +- .../iceberg/run09.sql | 1 + .../iceberg/run10.sql | 2 +- .../iceberg/run16.sql | 421 +++++++ .../iceberg/run17.sql | 421 +++++++ .../paimon/run01.sql | 13 +- .../paimon/run05.sql | 406 ++++++ fe/check/checkstyle/suppressions.xml | 3 + .../java/org/apache/doris/catalog/Column.java | 1 + .../apache/doris/datasource/ExternalUtil.java | 83 ++ .../doris/datasource/FileQueryScanNode.java | 33 +- .../doris/datasource/TableFormatType.java | 3 +- .../datasource/hive/HMSExternalTable.java | 7 +- .../datasource/hive/source/HiveScanNode.java | 38 + .../doris/datasource/hudi/HudiUtils.java | 119 +- .../datasource/hudi/source/HudiScanNode.java | 26 +- .../datasource/iceberg/IcebergUtils.java | 32 +- .../iceberg/source/IcebergScanNode.java | 2 + .../paimon/PaimonExternalTable.java | 3 +- .../doris/datasource/paimon/PaimonUtil.java | 112 ++ .../paimon/source/PaimonScanNode.java | 29 +- .../datasource/tvf/source/TVFScanNode.java | 12 + .../ExternalFileTableValuedFunction.java | 27 +- gensrc/thrift/ExternalTableSchema.thrift | 65 + gensrc/thrift/PlanNodes.thrift | 8 +- .../test_iceberg_full_schema_change.out | 649 ++++++++++ .../paimon/test_paimon_full_schema_change.out | 273 +++++ .../hudi/test_hudi_full_schema_change.out | 325 +++++ .../test_iceberg_full_schema_change.groovy | 116 ++ .../test_paimon_full_schema_change.groovy | 80 ++ .../tvf/test_hdfs_parquet_group4.groovy | 2 +- .../tvf/test_hdfs_parquet_group6.groovy | 2 +- .../hudi/test_hudi_full_schema_change.groovy | 151 +++ 73 files changed, 5896 insertions(+), 1678 deletions(-) create mode 100644 be/src/vec/exec/format/table/hive_reader.cpp create mode 100644 be/src/vec/exec/format/table/hive_reader.h delete mode 100644 be/test/vec/exec/format/paimon/paimon_schema_change_test.cpp create mode 100644 docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run16.sql create mode 100644 docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run17.sql create mode 100644 docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run05.sql create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalUtil.java create mode 100644 gensrc/thrift/ExternalTableSchema.thrift create mode 100644 regression-test/data/external_table_p0/iceberg/test_iceberg_full_schema_change.out create mode 100644 regression-test/data/external_table_p0/paimon/test_paimon_full_schema_change.out create mode 100644 regression-test/data/external_table_p2/hudi/test_hudi_full_schema_change.out create mode 100644 regression-test/suites/external_table_p0/iceberg/test_iceberg_full_schema_change.groovy create mode 100644 regression-test/suites/external_table_p0/paimon/test_paimon_full_schema_change.groovy create mode 100644 regression-test/suites/external_table_p2/hudi/test_hudi_full_schema_change.groovy diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp index 4233b79e865480..99637eaf764a7b 100644 --- a/be/src/olap/push_handler.cpp +++ b/be/src/olap/push_handler.cpp @@ -643,11 +643,11 @@ Status PushBrokerReader::_get_next_reader() { const_cast(&_runtime_state->timezone_obj()), _io_ctx.get(), _runtime_state.get()); - std::vector place_holder; init_status = parquet_reader->init_reader( - _all_col_names, place_holder, _colname_to_value_range, _push_down_exprs, - _real_tuple_desc, _default_val_row_desc.get(), _col_name_to_slot_id, - &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts, false); + _all_col_names, _colname_to_value_range, _push_down_exprs, _real_tuple_desc, + _default_val_row_desc.get(), _col_name_to_slot_id, + &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts, + vectorized::TableSchemaChangeHelper::ConstNode::get_instance(), false); _cur_reader = std::move(parquet_reader); if (!init_status.ok()) { return Status::InternalError("failed to init reader for file {}, err: {}", range.path, diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h index b807c567543038..adab042c95c973 100644 --- a/be/src/runtime/descriptors.h +++ b/be/src/runtime/descriptors.h @@ -47,6 +47,12 @@ class RepeatedField; namespace doris { +#ifdef BE_TEST +#define MOCK_REMOVE(str) +#else +#define MOCK_REMOVE(str) str +#endif + class ObjectPool; class PTupleDescriptor; class PSlotDescriptor; @@ -105,7 +111,7 @@ class SlotDescriptor { const TupleId _parent; const int _col_pos; bool _is_nullable; - const std::string _col_name; + MOCK_REMOVE(const) std::string _col_name; const std::string _col_name_lower_case; const int32_t _col_unique_id; diff --git a/be/src/vec/exec/format/orc/vorc_reader.cpp b/be/src/vec/exec/format/orc/vorc_reader.cpp index 6570e5b78a6eab..a35155199eb15c 100644 --- a/be/src/vec/exec/format/orc/vorc_reader.cpp +++ b/be/src/vec/exec/format/orc/vorc_reader.cpp @@ -277,6 +277,10 @@ void OrcReader::_init_profile() { } Status OrcReader::_create_file_reader() { + if (_reader != nullptr) { + return Status::OK(); + } + if (_file_input_stream == nullptr) { _file_description.mtime = _scan_range.__isset.modification_time ? _scan_range.modification_time : 0; @@ -318,21 +322,20 @@ Status OrcReader::_create_file_reader() { Status OrcReader::init_reader( const std::vector* column_names, - const std::vector& missing_column_names, const std::unordered_map* colname_to_value_range, const VExprContextSPtrs& conjuncts, bool is_acid, const TupleDescriptor* tuple_descriptor, const RowDescriptor* row_descriptor, const VExprContextSPtrs* not_single_slot_filter_conjuncts, const std::unordered_map* slot_id_to_filter_conjuncts, - const bool hive_use_column_names) { - _column_names = column_names; - _missing_column_names_set.insert(missing_column_names.begin(), missing_column_names.end()); + std::shared_ptr table_info_node_ptr) { + _table_column_names = column_names; _colname_to_value_range = colname_to_value_range; _lazy_read_ctx.conjuncts = conjuncts; _is_acid = is_acid; _tuple_descriptor = tuple_descriptor; _row_descriptor = row_descriptor; - _is_hive1_orc_or_use_idx = !hive_use_column_names; + _table_info_node_ptr = table_info_node_ptr; + if (not_single_slot_filter_conjuncts != nullptr && !not_single_slot_filter_conjuncts->empty()) { _not_single_slot_filter_conjuncts.insert(_not_single_slot_filter_conjuncts.end(), not_single_slot_filter_conjuncts->begin(), @@ -365,124 +368,46 @@ Status OrcReader::init_schema_reader() { Status OrcReader::get_parsed_schema(std::vector* col_names, std::vector* col_types) { - const auto& root_type = _is_acid ? _remove_acid(_reader->getType()) : _reader->getType(); + const auto& root_type = _is_acid ? remove_acid(_reader->getType()) : _reader->getType(); for (int i = 0; i < root_type.getSubtypeCount(); ++i) { - col_names->emplace_back(get_field_name_lower_case(&root_type, i)); + col_names->emplace_back(root_type.getFieldName(i)); col_types->emplace_back(convert_to_doris_type(root_type.getSubtype(i))); } return Status::OK(); } -Status OrcReader::get_schema_col_name_attribute(std::vector* col_names, - std::vector* col_attributes, - const std::string& attribute, - bool* exist_attribute) { - *exist_attribute = true; - auto& root_type = _is_acid ? _remove_acid(_reader->getType()) : _reader->getType(); - for (int i = 0; i < root_type.getSubtypeCount(); ++i) { - col_names->emplace_back(get_field_name_lower_case(&root_type, i)); - - if (!root_type.getSubtype(i)->hasAttributeKey(attribute)) { - *exist_attribute = false; - return Status::OK(); - } - col_attributes->emplace_back( - std::stoi(root_type.getSubtype(i)->getAttributeValue(attribute))); - } - return Status::OK(); -} - Status OrcReader::_init_read_columns() { const auto& root_type = _reader->getType(); - std::vector orc_cols; - std::vector orc_cols_lower_case; - bool is_hive1_orc = false; - _init_orc_cols(root_type, orc_cols, orc_cols_lower_case, _type_map, &is_hive1_orc, false); - - // In old version slot_name_to_schema_pos may not be set in _scan_params - // TODO, should be removed in 2.2 or later - _is_hive1_orc_or_use_idx = (is_hive1_orc || _is_hive1_orc_or_use_idx) && - _scan_params.__isset.slot_name_to_schema_pos; - for (size_t i = 0; i < _column_names->size(); ++i) { - auto& col_name = (*_column_names)[i]; - if (_missing_column_names_set.contains(col_name)) { - _missing_cols.emplace_back(col_name); - continue; - } - - if (_is_hive1_orc_or_use_idx) { - auto iter = _scan_params.slot_name_to_schema_pos.find(col_name); - if (iter != _scan_params.slot_name_to_schema_pos.end()) { - int pos = iter->second; - if (_is_acid && i < _column_names->size() - TransactionalHive::READ_PARAMS.size()) { - if (TransactionalHive::ROW_OFFSET + 1 + pos < orc_cols_lower_case.size()) { - // shift TransactionalHive::ROW_OFFSET + 1 offset, 1 is row struct col - orc_cols_lower_case[TransactionalHive::ROW_OFFSET + 1 + pos] = iter->first; - } - } else { - if (pos < orc_cols_lower_case.size()) { - orc_cols_lower_case[pos] = iter->first; - } + if (_is_acid) { + for (uint64_t i = 0; i < root_type.getSubtypeCount(); ++i) { + if (root_type.getSubtype(i)->getKind() == orc::TypeKind::STRUCT) { + auto row_orc_type = root_type.getSubtype(i); + for (uint64_t j = 0; j < row_orc_type->getSubtypeCount(); j++) { + _type_map.emplace(TransactionalHive::ROW + "." + row_orc_type->getFieldName(j), + row_orc_type->getSubtype(j)); } - } - } - auto iter = std::find(orc_cols_lower_case.begin(), orc_cols_lower_case.end(), col_name); - if (iter == orc_cols_lower_case.end()) { - _missing_cols.emplace_back(col_name); - } else { - auto pos = std::distance(orc_cols_lower_case.begin(), iter); - std::string read_col; - if (_is_acid && i < _column_names->size() - TransactionalHive::READ_PARAMS.size()) { - read_col = fmt::format( - "{}.{}", - TransactionalHive::ACID_COLUMN_NAMES[TransactionalHive::ROW_OFFSET], - orc_cols[pos]); - _read_cols.emplace_back(read_col); } else { - read_col = orc_cols[pos]; - _read_cols.emplace_back(read_col); + _type_map.emplace(root_type.getFieldName(i), root_type.getSubtype(i)); } - _read_cols_lower_case.emplace_back(col_name); - // For hive engine, store the orc column name to schema column name map. - // This is for Hive 1.x orc file with internal column name _col0, _col1... - if (_is_hive1_orc_or_use_idx) { - _removed_acid_file_col_name_to_schema_col[orc_cols[pos]] = col_name; - } - - _col_name_to_file_col_name[col_name] = read_col; + } + } else { + for (int i = 0; i < root_type.getSubtypeCount(); ++i) { + _type_map.emplace(root_type.getFieldName(i), root_type.getSubtype(i)); } } - return Status::OK(); -} -void OrcReader::_init_orc_cols(const orc::Type& type, std::vector& orc_cols, - std::vector& orc_cols_lower_case, - std::unordered_map& type_map, - bool* is_hive1_orc, bool should_add_acid_prefix) const { - bool hive1_orc = true; - for (int i = 0; i < type.getSubtypeCount(); ++i) { - orc_cols.emplace_back(type.getFieldName(i)); - auto filed_name_lower_case = get_field_name_lower_case(&type, i); - if (hive1_orc) { - hive1_orc = _is_hive1_col_name(filed_name_lower_case); - } - orc_cols_lower_case.emplace_back(std::move(filed_name_lower_case)); - auto file_name = type.getFieldName(i); - if (should_add_acid_prefix) { - file_name = fmt::format( - "{}.{}", TransactionalHive::ACID_COLUMN_NAMES[TransactionalHive::ROW_OFFSET], - file_name); - } - type_map.emplace(std::move(file_name), type.getSubtype(i)); - if (_is_acid) { - const orc::Type* sub_type = type.getSubtype(i); - if (sub_type->getKind() == orc::TypeKind::STRUCT) { - _init_orc_cols(*sub_type, orc_cols, orc_cols_lower_case, type_map, is_hive1_orc, - true); - } + for (size_t i = 0; i < _table_column_names->size(); ++i) { + const auto& table_column_name = (*_table_column_names)[i]; + if (!_table_info_node_ptr->children_column_exists(table_column_name)) { + _missing_cols.emplace_back(table_column_name); + continue; } + const auto file_column_name = + _table_info_node_ptr->children_file_column_name(table_column_name); + _read_file_cols.emplace_back(file_column_name); + _read_table_cols.emplace_back(table_column_name); } - *is_hive1_orc = hive1_orc; + return Status::OK(); } bool OrcReader::_check_acid_schema(const orc::Type& type) { @@ -505,7 +430,7 @@ bool OrcReader::_check_acid_schema(const orc::Type& type) { return true; } -const orc::Type& OrcReader::_remove_acid(const orc::Type& type) { +const orc::Type& OrcReader::remove_acid(const orc::Type& type) { if (_check_acid_schema(type)) { return *(type.getSubtype(TransactionalHive::ROW_OFFSET)); } else { @@ -664,10 +589,11 @@ std::tuple convert_to_orc_literal(const orc::Type* type, std::tuple OrcReader::_make_orc_literal( const VSlotRef* slot_ref, const VLiteral* literal) { - DCHECK(_col_name_to_file_col_name.contains(slot_ref->expr_name())); - auto file_col_name = _col_name_to_file_col_name[slot_ref->expr_name()]; + DCHECK(_table_info_node_ptr->children_column_exists(slot_ref->expr_name())); + auto file_col_name = _table_info_node_ptr->children_file_column_name(slot_ref->expr_name()); if (!_type_map.contains(file_col_name)) { - LOG(WARNING) << "Column " << slot_ref->expr_name() << " not found in _type_map"; + LOG(WARNING) << "Column " << slot_ref->expr_name() << "in file name" << file_col_name + << " not found in _type_map"; return std::make_tuple(false, orc::Literal(false), orc::PredicateDataType::LONG); } DCHECK(_type_map.contains(file_col_name)); @@ -732,10 +658,11 @@ bool OrcReader::_check_slot_can_push_down(const VExprSPtr& expr) { } const auto* slot_ref = static_cast(expr->children()[0].get()); // check if the slot exists in orc file and not partition column - if (!_col_name_to_file_col_name.contains(slot_ref->expr_name()) || - _lazy_read_ctx.predicate_partition_columns.contains(slot_ref->expr_name())) { + if (_lazy_read_ctx.predicate_partition_columns.contains(slot_ref->expr_name()) || + (!_table_info_node_ptr->children_column_exists(slot_ref->expr_name()))) { return false; } + auto [valid, _, predicate_type] = _make_orc_literal(slot_ref, nullptr); if (valid) { _vslot_ref_to_orc_predicate_data_type[slot_ref] = predicate_type; @@ -834,7 +761,8 @@ void OrcReader::_build_less_than(const VExprSPtr& expr, auto predicate_type = _vslot_ref_to_orc_predicate_data_type[slot_ref]; DCHECK(_vliteral_to_orc_literal.contains(literal)); auto orc_literal = _vliteral_to_orc_literal.find(literal)->second; - builder->lessThan(slot_ref->expr_name(), predicate_type, orc_literal); + builder->lessThan(_table_info_node_ptr->children_file_column_name(slot_ref->expr_name()), + predicate_type, orc_literal); } void OrcReader::_build_less_than_equals(const VExprSPtr& expr, @@ -848,7 +776,8 @@ void OrcReader::_build_less_than_equals(const VExprSPtr& expr, auto predicate_type = _vslot_ref_to_orc_predicate_data_type[slot_ref]; DCHECK(_vliteral_to_orc_literal.contains(literal)); auto orc_literal = _vliteral_to_orc_literal.find(literal)->second; - builder->lessThanEquals(slot_ref->expr_name(), predicate_type, orc_literal); + builder->lessThanEquals(_table_info_node_ptr->children_file_column_name(slot_ref->expr_name()), + predicate_type, orc_literal); } void OrcReader::_build_equals(const VExprSPtr& expr, @@ -862,7 +791,8 @@ void OrcReader::_build_equals(const VExprSPtr& expr, auto predicate_type = _vslot_ref_to_orc_predicate_data_type[slot_ref]; DCHECK(_vliteral_to_orc_literal.contains(literal)); auto orc_literal = _vliteral_to_orc_literal.find(literal)->second; - builder->equals(slot_ref->expr_name(), predicate_type, orc_literal); + builder->equals(_table_info_node_ptr->children_file_column_name(slot_ref->expr_name()), + predicate_type, orc_literal); } void OrcReader::_build_filter_in(const VExprSPtr& expr, @@ -883,9 +813,11 @@ void OrcReader::_build_filter_in(const VExprSPtr& expr, } DCHECK(!literals.empty()); if (literals.size() == 1) { - builder->equals(slot_ref->expr_name(), predicate_type, literals[0]); + builder->equals(_table_info_node_ptr->children_file_column_name(slot_ref->expr_name()), + predicate_type, literals[0]); } else { - builder->in(slot_ref->expr_name(), predicate_type, literals); + builder->in(_table_info_node_ptr->children_file_column_name(slot_ref->expr_name()), + predicate_type, literals); } } @@ -896,7 +828,8 @@ void OrcReader::_build_is_null(const VExprSPtr& expr, const auto* slot_ref = static_cast(expr->children()[0].get()); DCHECK(_vslot_ref_to_orc_predicate_data_type.contains(slot_ref)); auto predicate_type = _vslot_ref_to_orc_predicate_data_type[slot_ref]; - builder->isNull(slot_ref->expr_name(), predicate_type); + builder->isNull(_table_info_node_ptr->children_file_column_name(slot_ref->expr_name()), + predicate_type); } bool OrcReader::_build_search_argument(const VExprSPtr& expr, @@ -1023,16 +956,12 @@ Status OrcReader::set_fill_columns( SCOPED_RAW_TIMER(&_statistics.set_fill_column_time); // std::unordered_map> - std::unordered_map> predicate_columns; + std::unordered_map> predicate_table_columns; std::function visit_slot = [&](VExpr* expr) { if (auto* slot_ref = typeid_cast(expr)) { auto expr_name = slot_ref->expr_name(); - auto iter = _table_col_to_file_col.find(expr_name); - if (iter != _table_col_to_file_col.end()) { - expr_name = iter->second; - } - predicate_columns.emplace(expr_name, - std::make_pair(slot_ref->column_id(), slot_ref->slot_id())); + predicate_table_columns.emplace( + expr_name, std::make_pair(slot_ref->column_id(), slot_ref->slot_id())); if (slot_ref->column_id() == 0) { _lazy_read_ctx.resize_first_column = false; } @@ -1070,24 +999,24 @@ Status OrcReader::set_fill_columns( TransactionalHive::READ_ROW_COLUMN_NAMES.end()); } - for (auto& read_col : _read_cols_lower_case) { - _lazy_read_ctx.all_read_columns.emplace_back(read_col); - if (!predicate_columns.empty()) { - auto iter = predicate_columns.find(read_col); - if (iter == predicate_columns.end()) { + for (auto& read_table_col : _read_table_cols) { + _lazy_read_ctx.all_read_columns.emplace_back(read_table_col); + if (!predicate_table_columns.empty()) { + auto iter = predicate_table_columns.find(read_table_col); + if (iter == predicate_table_columns.end()) { if (!_is_acid || std::find(TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.begin(), TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.end(), - read_col) == + read_table_col) == TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.end()) { - _lazy_read_ctx.lazy_read_columns.emplace_back(read_col); + _lazy_read_ctx.lazy_read_columns.emplace_back(read_table_col); } } else { _lazy_read_ctx.predicate_columns.first.emplace_back(iter->first); _lazy_read_ctx.predicate_columns.second.emplace_back(iter->second.second); + _lazy_read_ctx.predicate_orc_columns.emplace_back( - _col_name_to_file_col_name[iter->first]); - _lazy_read_ctx.all_predicate_col_ids.emplace_back(iter->second.first); + _table_info_node_ptr->children_file_column_name(iter->first)); } } } @@ -1104,18 +1033,17 @@ Status OrcReader::set_fill_columns( } for (const auto& kv : partition_columns) { - auto iter = predicate_columns.find(kv.first); - if (iter == predicate_columns.end()) { + auto iter = predicate_table_columns.find(kv.first); + if (iter == predicate_table_columns.end()) { _lazy_read_ctx.partition_columns.emplace(kv.first, kv.second); } else { _lazy_read_ctx.predicate_partition_columns.emplace(kv.first, kv.second); - _lazy_read_ctx.all_predicate_col_ids.emplace_back(iter->second.first); } } for (const auto& kv : missing_columns) { - auto iter = predicate_columns.find(kv.first); - if (iter == predicate_columns.end()) { + auto iter = predicate_table_columns.find(kv.first); + if (iter == predicate_table_columns.end()) { _lazy_read_ctx.missing_columns.emplace(kv.first, kv.second); } else { //For check missing column : missing column == xx, missing column is null,missing column is not null. @@ -1123,13 +1051,12 @@ Status OrcReader::set_fill_columns( _slot_id_to_filter_conjuncts->end()) { for (const auto& ctx : _slot_id_to_filter_conjuncts->find(iter->second.second)->second) { - _filter_conjuncts.emplace_back(ctx); + _filter_conjuncts.emplace_back(ctx); // todo ?????? } } // predicate_missing_columns is VLiteral.To fill in default values for missing columns. _lazy_read_ctx.predicate_missing_columns.emplace(kv.first, kv.second); - _lazy_read_ctx.all_predicate_col_ids.emplace_back(iter->second.first); } } @@ -1158,7 +1085,7 @@ Status OrcReader::set_fill_columns( try { _row_reader_options.range(_range_start_offset, _range_size); _row_reader_options.setTimezoneName(_ctz == "CST" ? "Asia/Shanghai" : _ctz); - _row_reader_options.include(_read_cols); + _row_reader_options.include(_read_file_cols); _row_reader_options.setEnableLazyDecoding(true); uint64_t number_of_stripes = _reader->getNumberOfStripes(); @@ -1224,7 +1151,23 @@ Status OrcReader::set_fill_columns( _batch = _row_reader->createRowBatch(_batch_size); const auto& selected_type = _row_reader->getSelectedType(); int idx = 0; - RETURN_IF_ERROR(_init_select_types(selected_type, idx)); + if (_is_acid) { + for (int i = 0; i < selected_type.getSubtypeCount(); ++i) { + auto sub_type = selected_type.getSubtype(i); + if (sub_type->getKind() == orc::TypeKind::STRUCT) { + for (int j = 0; j < sub_type->getSubtypeCount(); ++j) { + _colname_to_idx[TransactionalHive::ROW + "." + sub_type->getFieldName(j)] = + idx++; + } + } else { + _colname_to_idx[selected_type.getFieldName(i)] = idx++; + } + } + } else { + for (int i = 0; i < selected_type.getSubtypeCount(); ++i) { + _colname_to_idx[selected_type.getFieldName(i)] = idx++; + } + } _remaining_rows = _row_reader->getNumberOfRows(); @@ -1258,26 +1201,6 @@ Status OrcReader::set_fill_columns( return Status::OK(); } -Status OrcReader::_init_select_types(const orc::Type& type, int idx) { - for (int i = 0; i < type.getSubtypeCount(); ++i) { - std::string name; - // For hive engine, translate the column name in orc file to schema column name. - // This is for Hive 1.x which use internal column name _col0, _col1... - if (_is_hive1_orc_or_use_idx) { - name = _removed_acid_file_col_name_to_schema_col[type.getFieldName(i)]; - } else { - name = get_field_name_lower_case(&type, i); - } - _colname_to_idx[name] = idx++; - const orc::Type* sub_type = type.getSubtype(i); - _col_orc_type.push_back(sub_type); - if (_is_acid && sub_type->getKind() == orc::TypeKind::STRUCT) { - RETURN_IF_ERROR(_init_select_types(*sub_type, idx)); - } - } - return Status::OK(); -} - Status OrcReader::_fill_partition_columns( Block* block, uint64_t rows, const std::unordered_map>& @@ -1440,7 +1363,7 @@ Status OrcReader::get_columns(std::unordered_map* n std::unordered_set* missing_cols) { const auto& root_type = _reader->getType(); for (int i = 0; i < root_type.getSubtypeCount(); ++i) { - name_to_type->emplace(get_field_name_lower_case(&root_type, i), + name_to_type->emplace(root_type.getFieldName(i), convert_to_doris_type(root_type.getSubtype(i))); } for (auto& col : _missing_cols) { @@ -1690,6 +1613,7 @@ template Status OrcReader::_fill_doris_data_column(const std::string& col_name, MutableColumnPtr& data_column, const DataTypePtr& data_type, + std::shared_ptr root_node, const orc::Type* orc_column_type, const orc::ColumnVectorBatch* cvb, size_t num_values) { TypeIndex logical_type = remove_nullable(data_type)->get_type_id(); @@ -1743,7 +1667,8 @@ Status OrcReader::_fill_doris_data_column(const std::string& col_name, std::string element_name = col_name + ".element"; return _orc_column_to_doris_column( element_name, static_cast(*data_column).get_data_ptr(), nested_type, - nested_orc_type, orc_list->elements.get(), element_size); + root_node->get_element_node(), nested_orc_type, orc_list->elements.get(), + element_size); } case TypeIndex::Map: { if (orc_column_type->getKind() != orc::TypeKind::MAP) { @@ -1767,12 +1692,13 @@ Status OrcReader::_fill_doris_data_column(const std::string& col_name, ColumnPtr& doris_value_column = doris_map.get_values_ptr(); std::string key_col_name = col_name + ".key"; std::string value_col_name = col_name + ".value"; - RETURN_IF_ERROR(_orc_column_to_doris_column(key_col_name, doris_key_column, - doris_key_type, orc_key_type, - orc_map->keys.get(), element_size)); - return _orc_column_to_doris_column(value_col_name, doris_value_column, - doris_value_type, orc_value_type, - orc_map->elements.get(), element_size); + RETURN_IF_ERROR(_orc_column_to_doris_column( + key_col_name, doris_key_column, doris_key_type, root_node->get_key_node(), + + orc_key_type, orc_map->keys.get(), element_size)); + return _orc_column_to_doris_column( + value_col_name, doris_value_column, doris_value_type, root_node->get_value_node(), + orc_value_type, orc_map->elements.get(), element_size); } case TypeIndex::Struct: { if (orc_column_type->getKind() != orc::TypeKind::STRUCT) { @@ -1786,19 +1712,19 @@ Status OrcReader::_fill_doris_data_column(const std::string& col_name, std::set missing_fields; const DataTypeStruct* doris_struct_type = reinterpret_cast(remove_nullable(data_type).get()); + for (int i = 0; i < doris_struct.tuple_size(); ++i) { - bool is_missing_col = true; + const auto& table_column_name = doris_struct_type->get_name_by_position(i); + if (!root_node->children_column_exists(table_column_name)) { + missing_fields.insert(i); + continue; + } + const auto& file_column_name = root_node->children_file_column_name(table_column_name); for (int j = 0; j < orc_column_type->getSubtypeCount(); ++j) { - if (boost::iequals(doris_struct_type->get_name_by_position(i), - orc_column_type->getFieldName(j))) { + if (boost::iequals(orc_column_type->getFieldName(j), file_column_name)) { read_fields[i] = j; - is_missing_col = false; - break; } } - if (is_missing_col) { - missing_fields.insert(i); - } } for (int missing_field : missing_fields) { @@ -1820,7 +1746,10 @@ Status OrcReader::_fill_doris_data_column(const std::string& col_name, ColumnPtr& doris_field = doris_struct.get_column_ptr(read_field.first); const DataTypePtr& doris_type = doris_struct_type->get_element(read_field.first); RETURN_IF_ERROR(_orc_column_to_doris_column( - field_name, doris_field, doris_type, orc_type, orc_field, num_values)); + field_name, doris_field, doris_type, + root_node->get_children_node( + doris_struct_type->get_name_by_position(read_field.first)), + orc_type, orc_field, num_values)); } return Status::OK(); } @@ -1831,12 +1760,11 @@ Status OrcReader::_fill_doris_data_column(const std::string& col_name, } template -Status OrcReader::_orc_column_to_doris_column(const std::string& col_name, ColumnPtr& doris_column, - const DataTypePtr& data_type, - const orc::Type* orc_column_type, - const orc::ColumnVectorBatch* cvb, - size_t num_values) { - TypeDescriptor src_type = convert_to_doris_type(orc_column_type); +Status OrcReader::_orc_column_to_doris_column( + const std::string& col_name, ColumnPtr& doris_column, const DataTypePtr& data_type, + std::shared_ptr root_node, const orc::Type* orc_column_type, + const orc::ColumnVectorBatch* cvb, size_t num_values) { + auto src_type = convert_to_doris_type(orc_column_type); bool is_dict_filter_col = false; for (const std::pair& dict_col : _dict_filter_cols) { if (col_name == dict_col.first) { @@ -1892,7 +1820,7 @@ Status OrcReader::_orc_column_to_doris_column(const std::string& col_name, Colum } RETURN_IF_ERROR(_fill_doris_data_column(col_name, data_column, - remove_nullable(resolved_type), + remove_nullable(resolved_type), root_node, orc_column_type, cvb, num_values)); // resolve schema change auto converted_column = doris_column->assume_mutable(); @@ -1979,16 +1907,19 @@ Status OrcReader::get_next_block_impl(Block* block, size_t* read_rows, bool* eof std::vector batch_vec; _fill_batch_vec(batch_vec, _batch.get(), 0); + for (auto& col_name : _lazy_read_ctx.lazy_read_columns) { auto& column_with_type_and_name = block->get_by_name(col_name); auto& column_ptr = column_with_type_and_name.column; auto& column_type = column_with_type_and_name.type; - auto orc_col_idx = _colname_to_idx.find(col_name); + auto file_column_name = _table_info_node_ptr->children_file_column_name(col_name); + auto orc_col_idx = _colname_to_idx.find(file_column_name); if (orc_col_idx == _colname_to_idx.end()) { return Status::InternalError("Wrong read column '{}' in orc file", col_name); } RETURN_IF_ERROR(_orc_column_to_doris_column( - col_name, column_ptr, column_type, _col_orc_type[orc_col_idx->second], + col_name, column_ptr, column_type, + _table_info_node_ptr->get_children_node(col_name), _type_map[file_column_name], batch_vec[orc_col_idx->second], _batch->numElements)); } @@ -2070,12 +2001,14 @@ Status OrcReader::get_next_block_impl(Block* block, size_t* read_rows, bool* eof auto& column_with_type_and_name = block->get_by_name(col_name); auto& column_ptr = column_with_type_and_name.column; auto& column_type = column_with_type_and_name.type; - auto orc_col_idx = _colname_to_idx.find(col_name); + auto file_column_name = _table_info_node_ptr->children_file_column_name(col_name); + auto orc_col_idx = _colname_to_idx.find(file_column_name); if (orc_col_idx == _colname_to_idx.end()) { return Status::InternalError("Wrong read column '{}' in orc file", col_name); } RETURN_IF_ERROR(_orc_column_to_doris_column( - col_name, column_ptr, column_type, _col_orc_type[orc_col_idx->second], + col_name, column_ptr, column_type, + _table_info_node_ptr->get_children_node(col_name), _type_map[file_column_name], batch_vec[orc_col_idx->second], _batch->numElements)); } @@ -2152,10 +2085,19 @@ Status OrcReader::get_next_block_impl(Block* block, size_t* read_rows, bool* eof void OrcReader::_fill_batch_vec(std::vector& result, orc::ColumnVectorBatch* batch, int idx) { - for (auto* field : dynamic_cast(batch)->fields) { - result.push_back(field); - if (_is_acid && _col_orc_type[idx++]->getKind() == orc::TypeKind::STRUCT) { - _fill_batch_vec(result, field, idx); + if (_is_acid) { + for (auto* field : dynamic_cast(batch)->fields) { + if (dynamic_cast(field) != nullptr) { + for (auto* row_field : dynamic_cast(field)->fields) { + result.push_back(row_field); + } + } else { + result.push_back(field); + } + } + } else { + for (auto* field : dynamic_cast(batch)->fields) { + result.push_back(field); } } } @@ -2211,25 +2153,27 @@ Status OrcReader::filter(orc::ColumnVectorBatch& data, uint16_t* sel, uint16_t s } std::vector batch_vec; _fill_batch_vec(batch_vec, &data, 0); - std::vector col_names; - col_names.insert(col_names.end(), _lazy_read_ctx.predicate_columns.first.begin(), - _lazy_read_ctx.predicate_columns.first.end()); + std::vector table_col_names; + table_col_names.insert(table_col_names.end(), _lazy_read_ctx.predicate_columns.first.begin(), + _lazy_read_ctx.predicate_columns.first.end()); if (_is_acid) { - col_names.insert(col_names.end(), - TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.begin(), - TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.end()); + table_col_names.insert(table_col_names.end(), + TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.begin(), + TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.end()); } - for (auto& col_name : col_names) { - auto& column_with_type_and_name = block->get_by_name(col_name); + for (auto& table_col_name : table_col_names) { + auto& column_with_type_and_name = block->get_by_name(table_col_name); auto& column_ptr = column_with_type_and_name.column; auto& column_type = column_with_type_and_name.type; - auto orc_col_idx = _colname_to_idx.find(col_name); + auto file_column_name = _table_info_node_ptr->children_file_column_name(table_col_name); + auto orc_col_idx = _colname_to_idx.find(file_column_name); if (orc_col_idx == _colname_to_idx.end()) { - return Status::InternalError("Wrong read column '{}' in orc file", col_name); + return Status::InternalError("Wrong read column '{}' in orc file", table_col_name); } RETURN_IF_ERROR(_orc_column_to_doris_column( - col_name, column_ptr, column_type, _col_orc_type[orc_col_idx->second], - batch_vec[orc_col_idx->second], data.numElements)); + table_col_name, column_ptr, column_type, + _table_info_node_ptr->get_children_node(table_col_name), + _type_map[file_column_name], batch_vec[orc_col_idx->second], data.numElements)); } RETURN_IF_ERROR( _fill_partition_columns(block, size, _lazy_read_ctx.predicate_partition_columns)); @@ -2268,7 +2212,7 @@ Status OrcReader::filter(orc::ColumnVectorBatch& data, uint16_t* sel, uint16_t s } if (can_filter_all) { - for (auto& col : col_names) { + for (auto& col : table_col_names) { // clean block to read predicate columns and acid columns block->get_by_name(col).column->assume_mutable()->clear(); } @@ -2311,7 +2255,8 @@ Status OrcReader::fill_dict_filter_column_names( int slot_id = predicate_col_slot_ids[i]; if (!_disable_dict_filter && _can_filter_by_dict(slot_id)) { _dict_filter_cols.emplace_back(predicate_col_name, slot_id); - column_names.emplace_back(_col_name_to_file_col_name[predicate_col_name]); + column_names.emplace_back( + _table_info_node_ptr->children_file_column_name(predicate_col_name)); } else { if (_slot_id_to_filter_conjuncts->find(slot_id) != _slot_id_to_filter_conjuncts->end()) { @@ -2379,8 +2324,8 @@ Status OrcReader::on_string_dicts_loaded( msg << "_slot_id_to_filter_conjuncts: slot_id [" << slot_id << "] not found"; return Status::NotFound(msg.str()); } - auto file_column_name_to_dict_map_iter = - file_column_name_to_dict_map.find(_col_name_to_file_col_name[dict_filter_col_name]); + auto file_column_name_to_dict_map_iter = file_column_name_to_dict_map.find( + _table_info_node_ptr->children_file_column_name(dict_filter_col_name)); if (file_column_name_to_dict_map_iter == file_column_name_to_dict_map.end()) { it = _dict_filter_cols.erase(it); for (auto& ctx : ctxs) { @@ -2592,7 +2537,10 @@ Status OrcReader::_convert_dict_cols_to_string_cols( size_t pos = block->get_position_by_name(dict_filter_cols.first); ColumnWithTypeAndName& column_with_type_and_name = block->get_by_position(pos); const ColumnPtr& column = column_with_type_and_name.column; - auto orc_col_idx = _colname_to_idx.find(dict_filter_cols.first); + + auto file_column_name = + _table_info_node_ptr->children_file_column_name(dict_filter_cols.first); + auto orc_col_idx = _colname_to_idx.find(file_column_name); if (orc_col_idx == _colname_to_idx.end()) { return Status::InternalError("Wrong read column '{}' in orc file", dict_filter_cols.first); @@ -2607,7 +2555,7 @@ Status OrcReader::_convert_dict_cols_to_string_cols( if (batch_vec != nullptr) { string_column = _convert_dict_column_to_string_column( dict_column, &null_map, (*batch_vec)[orc_col_idx->second], - _col_orc_type[orc_col_idx->second]); + _type_map[file_column_name]); } else { string_column = ColumnString::create(); } @@ -2623,7 +2571,7 @@ Status OrcReader::_convert_dict_cols_to_string_cols( if (batch_vec != nullptr) { string_column = _convert_dict_column_to_string_column( dict_column, nullptr, (*batch_vec)[orc_col_idx->second], - _col_orc_type[orc_col_idx->second]); + _type_map[file_column_name]); } else { string_column = ColumnString::create(); } diff --git a/be/src/vec/exec/format/orc/vorc_reader.h b/be/src/vec/exec/format/orc/vorc_reader.h index dbc4f9aee92a48..cc33034b2cf1fe 100644 --- a/be/src/vec/exec/format/orc/vorc_reader.h +++ b/be/src/vec/exec/format/orc/vorc_reader.h @@ -52,6 +52,7 @@ #include "vec/exec/format/column_type_convert.h" #include "vec/exec/format/format_common.h" #include "vec/exec/format/generic_reader.h" +#include "vec/exec/format/table/table_format_reader.h" #include "vec/exec/format/table/transactional_hive_reader.h" #include "vec/exprs/vliteral.h" #include "vec/exprs/vslot_ref.h" @@ -113,6 +114,12 @@ class OrcReader : public GenericReader { ENABLE_FACTORY_CREATOR(OrcReader); public: + Status get_file_type(const orc::Type** root) { + RETURN_IF_ERROR(_create_file_reader()); + *root = &(_reader->getType()); + return Status::OK(); + } + struct Statistics { int64_t fs_read_time = 0; int64_t fs_read_calls = 0; @@ -140,21 +147,19 @@ class OrcReader : public GenericReader { //If you want to read the file by index instead of column name, set hive_use_column_names to false. Status init_reader( const std::vector* column_names, - const std::vector& missing_column_names, const std::unordered_map* colname_to_value_range, const VExprContextSPtrs& conjuncts, bool is_acid, const TupleDescriptor* tuple_descriptor, const RowDescriptor* row_descriptor, const VExprContextSPtrs* not_single_slot_filter_conjuncts, const std::unordered_map* slot_id_to_filter_conjuncts, - const bool hive_use_column_names = true); + std::shared_ptr table_info_node_ptr = + TableSchemaChangeHelper::ConstNode::get_instance()); Status set_fill_columns( const std::unordered_map>& partition_columns, const std::unordered_map& missing_columns) override; - Status _init_select_types(const orc::Type& type, int idx); - Status _fill_partition_columns( Block* block, uint64_t rows, const std::unordered_map>& @@ -182,17 +187,10 @@ class OrcReader : public GenericReader { Status get_parsed_schema(std::vector* col_names, std::vector* col_types) override; - Status get_schema_col_name_attribute(std::vector* col_names, - std::vector* col_attributes, - const std::string& attribute, bool* exist_attribute); - void set_table_col_to_file_col( - std::unordered_map table_col_to_file_col) { - _table_col_to_file_col = table_col_to_file_col; - } - - void set_position_delete_rowids(vector* delete_rows) { + void set_position_delete_rowids(std::vector* delete_rows) { _position_delete_ordered_rowids = delete_rows; } + void _execute_filter_position_delete_rowids(IColumn::Filter& filter); void set_delete_rows(const TransactionalHiveReader::AcidRowIDSet* delete_rows) { @@ -212,6 +210,16 @@ class OrcReader : public GenericReader { static TypeDescriptor convert_to_doris_type(const orc::Type* orc_type); static std::string get_field_name_lower_case(const orc::Type* orc_type, int pos); + static bool inline is_hive1_col_name(const orc::Type* orc_type_ptr) { + for (uint64_t idx = 0; idx < orc_type_ptr->getSubtypeCount(); idx++) { + if (!_is_hive1_col_name(orc_type_ptr->getFieldName(idx))) { + return false; + } + } + return true; + } + static const orc::Type& remove_acid(const orc::Type& type); + protected: void _collect_profile_before_close() override; @@ -289,12 +297,8 @@ class OrcReader : public GenericReader { void _init_profile(); Status _init_read_columns(); - void _init_orc_cols(const orc::Type& type, std::vector& orc_cols, - std::vector& orc_cols_lower_case, - std::unordered_map& type_map, - bool* is_hive1_orc, bool should_add_acid_prefix) const; + static bool _check_acid_schema(const orc::Type& type); - static const orc::Type& _remove_acid(const orc::Type& type); // functions for building search argument until _init_search_argument std::tuple _make_orc_literal( @@ -323,12 +327,15 @@ class OrcReader : public GenericReader { template Status _fill_doris_data_column(const std::string& col_name, MutableColumnPtr& data_column, - const DataTypePtr& data_type, const orc::Type* orc_column_type, + const DataTypePtr& data_type, + std::shared_ptr root_node, + const orc::Type* orc_column_type, const orc::ColumnVectorBatch* cvb, size_t num_values); template Status _orc_column_to_doris_column(const std::string& col_name, ColumnPtr& doris_column, const DataTypePtr& data_type, + std::shared_ptr root_node, const orc::Type* orc_column_type, const orc::ColumnVectorBatch* cvb, size_t num_values); @@ -593,38 +600,35 @@ class OrcReader : public GenericReader { int64_t _range_start_offset; int64_t _range_size; const std::string& _ctz; - const std::vector* _column_names; - // _missing_column_names_set: used in iceberg/hudi/paimon, the columns are dropped - // but added back(drop column a then add column a). Shouldn't read this column data in this case. - std::set _missing_column_names_set; + int32_t _offset_days = 0; cctz::time_zone _time_zone; - std::list _read_cols; - std::list _read_cols_lower_case; + // The columns of the table to be read (contain columns that do not exist) + const std::vector* _table_column_names; + + // The columns of the file to be read (file column name) + std::list _read_file_cols; + + // The columns of the table to be read (table column name) + std::list _read_table_cols; + + // _read_table_cols + _missing_cols = _table_column_names std::list _missing_cols; + + // file column name to std::vector idx. std::unordered_map _colname_to_idx; - // Column name in Orc file after removed acid(remove row.) to column name to schema. - // This is used for Hive 1.x which use internal column name in Orc file. - // _col0, _col1... - std::unordered_map _removed_acid_file_col_name_to_schema_col; - // Flag for hive engine. - // 1. True if the external table engine is Hive1.x with orc col name as _col1, col2, ... - // 2. If true, use indexes instead of column names when reading orc tables. - bool _is_hive1_orc_or_use_idx = false; - - // map col name in metastore to col name in orc file - std::unordered_map _col_name_to_file_col_name; - // map col name in orc file to orc type + + // file column name to orc type std::unordered_map _type_map; - std::vector _col_orc_type; + std::unique_ptr _file_input_stream; Statistics _statistics; OrcProfile _orc_profile; orc::ReaderMetrics _reader_metrics; std::unique_ptr _batch; - std::unique_ptr _reader; + std::unique_ptr _reader = nullptr; std::unique_ptr _row_reader; std::unique_ptr _orc_filter; orc::RowReaderOptions _row_reader_options; @@ -660,12 +664,9 @@ class OrcReader : public GenericReader { bool _dict_cols_has_converted = false; bool _has_complex_type = false; - // resolve schema change + // resolve schema type change std::unordered_map> _converters; - //for iceberg table , when table column name != file column name - //TODO(CXY) : remove _table_col_to_file_col,because we hava _col_name_to_file_col_name, - // the two have the same effect. - std::unordered_map _table_col_to_file_col; + //support iceberg position delete . std::vector* _position_delete_ordered_rowids = nullptr; std::unordered_map @@ -676,6 +677,10 @@ class OrcReader : public GenericReader { int64_t _orc_tiny_stripe_threshold_bytes = 8L * 1024L * 1024L; int64_t _orc_once_max_read_bytes = 8L * 1024L * 1024L; int64_t _orc_max_merge_distance_bytes = 1L * 1024L * 1024L; + + // Through this node, you can find the file column based on the table column. + std::shared_ptr _table_info_node_ptr = + TableSchemaChangeHelper::ConstNode::get_instance(); }; class StripeStreamInputStream : public orc::InputStream, public ProfileCollector { diff --git a/be/src/vec/exec/format/parquet/schema_desc.cpp b/be/src/vec/exec/format/parquet/schema_desc.cpp index 1eae65b1a4db19..1f72a1fda5b479 100644 --- a/be/src/vec/exec/format/parquet/schema_desc.cpp +++ b/be/src/vec/exec/format/parquet/schema_desc.cpp @@ -27,6 +27,11 @@ #include "runtime/define_primitive_type.h" #include "util/slice.h" #include "util/string_util.h" +#include "vec/data_types/data_type_array.h" +#include "vec/data_types/data_type_factory.hpp" +#include "vec/data_types/data_type_map.h" +#include "vec/data_types/data_type_struct.h" +#include "vec/exec/format/table/table_format_reader.h" namespace doris::vectorized { @@ -131,15 +136,13 @@ Status FieldDescriptor::parse_from_thrift(const std::vectorsecond.data()}; -} - Status FieldDescriptor::parse_node_field(const std::vector& t_schemas, size_t curr_pos, FieldSchema* node_field) { if (curr_pos >= t_schemas.size()) { @@ -178,7 +173,7 @@ Status FieldDescriptor::parse_node_field(const std::vectorchildren[0]; parse_physical_field(t_schema, false, child); - node_field->name = to_lower(t_schema.name); + node_field->name = t_schema.name; node_field->type.type = TYPE_ARRAY; node_field->type.add_sub_type(child->type); node_field->is_nullable = false; @@ -197,7 +192,7 @@ Status FieldDescriptor::parse_node_field(const std::vectorname = to_lower(physical_schema.name); + physical_field->name = physical_schema.name; physical_field->parquet_schema = physical_schema; physical_field->is_nullable = is_nullable; physical_field->physical_type = physical_schema.type; @@ -474,7 +469,7 @@ Status FieldDescriptor::parse_group_field(const std::vector RETURN_IF_ERROR(parse_struct_field(t_schemas, curr_pos, struct_field)); - group_field->name = to_lower(group_schema.name); + group_field->name = group_schema.name; group_field->type.type = TYPE_ARRAY; group_field->type.add_sub_type(struct_field->type); group_field->is_nullable = false; @@ -543,7 +538,7 @@ Status FieldDescriptor::parse_list_field(const std::vectorname = to_lower(first_level.name); + list_field->name = first_level.name; list_field->type.type = TYPE_ARRAY; list_field->type.add_sub_type(list_field->children[0].type); list_field->is_nullable = is_optional; @@ -607,7 +602,7 @@ Status FieldDescriptor::parse_map_field(const std::vector> RETURN_IF_ERROR(parse_struct_field(t_schemas, curr_pos + 1, map_kv_field)); - map_field->name = to_lower(map_schema.name); + map_field->name = map_schema.name; map_field->type.type = TYPE_MAP; map_field->type.add_sub_type(map_kv_field->type.children[0]); map_field->type.add_sub_type(map_kv_field->type.children[1]); @@ -632,9 +627,10 @@ Status FieldDescriptor::parse_struct_field(const std::vectorchildren[i])); } - struct_field->name = to_lower(struct_schema.name); + struct_field->name = struct_schema.name; struct_field->is_nullable = is_optional; struct_field->type.type = TYPE_STRUCT; + struct_field->field_id = struct_schema.__isset.field_id ? struct_schema.field_id : -1; for (int i = 0; i < num_children; ++i) { struct_field->type.add_sub_type(struct_field->children[i].type, diff --git a/be/src/vec/exec/format/parquet/schema_desc.h b/be/src/vec/exec/format/parquet/schema_desc.h index 83c0de8ee28fd6..408a45eae4bc85 100644 --- a/be/src/vec/exec/format/parquet/schema_desc.h +++ b/be/src/vec/exec/format/parquet/schema_desc.h @@ -27,6 +27,7 @@ #include #include "common/status.h" +#include "gen_cpp/Planner_types.h" #include "runtime/types.h" #include "util/slice.h" @@ -58,7 +59,7 @@ struct FieldSchema { FieldSchema(const FieldSchema& fieldSchema) = default; std::string debug_string() const; - int32_t field_id; + int32_t field_id = -1; }; class FieldDescriptor { @@ -71,8 +72,8 @@ class FieldDescriptor { std::unordered_map _name_to_field; // Used in from_thrift, marking the next schema position that should be parsed size_t _next_schema_pos; - std::map _field_id_name_mapping; +private: void parse_physical_field(const tparquet::SchemaElement& physical_schema, bool is_nullable, FieldSchema* physical_field); @@ -133,11 +134,7 @@ class FieldDescriptor { int32_t size() const { return _fields.size(); } - bool has_parquet_field_id() const { return !_field_id_name_mapping.empty(); } - - std::map get_field_id_name_map() { return _field_id_name_mapping; } - - const doris::Slice get_column_name_from_field_id(int32_t id) const; + const std::vector& get_fields_schema() const { return _fields; } }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp index 2702ab1a795547..e46b9b3b2560fb 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp +++ b/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp @@ -574,9 +574,10 @@ Status ScalarColumnReader::_try_load_dict_page(bool* loaded, bool* has_dict) { return Status::OK(); } -Status ScalarColumnReader::read_column_data(ColumnPtr& doris_column, DataTypePtr& type, - FilterMap& filter_map, size_t batch_size, - size_t* read_rows, bool* eof, bool is_dict_filter) { +Status ScalarColumnReader::read_column_data( + ColumnPtr& doris_column, DataTypePtr& type, + const std::shared_ptr& root_node, FilterMap& filter_map, + size_t batch_size, size_t* read_rows, bool* eof, bool is_dict_filter) { if (_converter == nullptr) { _converter = parquet::PhysicalToLogicalConverter::get_converter( _field_schema, _field_schema->type, type, _ctz, is_dict_filter); @@ -685,9 +686,10 @@ Status ArrayColumnReader::init(std::unique_ptr element_read return Status::OK(); } -Status ArrayColumnReader::read_column_data(ColumnPtr& doris_column, DataTypePtr& type, - FilterMap& filter_map, size_t batch_size, - size_t* read_rows, bool* eof, bool is_dict_filter) { +Status ArrayColumnReader::read_column_data( + ColumnPtr& doris_column, DataTypePtr& type, + const std::shared_ptr& root_node, FilterMap& filter_map, + size_t batch_size, size_t* read_rows, bool* eof, bool is_dict_filter) { MutableColumnPtr data_column; NullMap* null_map_ptr = nullptr; if (doris_column->is_nullable()) { @@ -712,7 +714,8 @@ Status ArrayColumnReader::read_column_data(ColumnPtr& doris_column, DataTypePtr& (reinterpret_cast(remove_nullable(type).get())) ->get_nested_type()); // read nested column - RETURN_IF_ERROR(_element_reader->read_column_data(element_column, element_type, filter_map, + RETURN_IF_ERROR(_element_reader->read_column_data(element_column, element_type, + root_node->get_element_node(), filter_map, batch_size, read_rows, eof, is_dict_filter)); if (*read_rows == 0) { return Status::OK(); @@ -736,9 +739,10 @@ Status MapColumnReader::init(std::unique_ptr key_reader, return Status::OK(); } -Status MapColumnReader::read_column_data(ColumnPtr& doris_column, DataTypePtr& type, - FilterMap& filter_map, size_t batch_size, - size_t* read_rows, bool* eof, bool is_dict_filter) { +Status MapColumnReader::read_column_data( + ColumnPtr& doris_column, DataTypePtr& type, + const std::shared_ptr& root_node, FilterMap& filter_map, + size_t batch_size, size_t* read_rows, bool* eof, bool is_dict_filter) { MutableColumnPtr data_column; NullMap* null_map_ptr = nullptr; if (doris_column->is_nullable()) { @@ -770,14 +774,16 @@ Status MapColumnReader::read_column_data(ColumnPtr& doris_column, DataTypePtr& t size_t value_rows = 0; bool key_eof = false; bool value_eof = false; - RETURN_IF_ERROR(_key_reader->read_column_data(key_column, key_type, filter_map, batch_size, - &key_rows, &key_eof, is_dict_filter)); + + RETURN_IF_ERROR(_key_reader->read_column_data(key_column, key_type, root_node->get_key_node(), + filter_map, batch_size, &key_rows, &key_eof, + is_dict_filter)); while (value_rows < key_rows && !value_eof) { size_t loop_rows = 0; - RETURN_IF_ERROR(_value_reader->read_column_data(value_column, value_type, filter_map, - key_rows - value_rows, &loop_rows, - &value_eof, is_dict_filter)); + RETURN_IF_ERROR(_value_reader->read_column_data( + value_column, value_type, root_node->get_value_node(), filter_map, + key_rows - value_rows, &loop_rows, &value_eof, is_dict_filter)); value_rows += loop_rows; } DCHECK_EQ(key_rows, value_rows); @@ -805,9 +811,10 @@ Status StructColumnReader::init( _child_readers = std::move(child_readers); return Status::OK(); } -Status StructColumnReader::read_column_data(ColumnPtr& doris_column, DataTypePtr& type, - FilterMap& filter_map, size_t batch_size, - size_t* read_rows, bool* eof, bool is_dict_filter) { +Status StructColumnReader::read_column_data( + ColumnPtr& doris_column, DataTypePtr& type, + const std::shared_ptr& root_node, FilterMap& filter_map, + size_t batch_size, size_t* read_rows, bool* eof, bool is_dict_filter) { MutableColumnPtr data_column; NullMap* null_map_ptr = nullptr; if (doris_column->is_nullable()) { @@ -840,23 +847,21 @@ Status StructColumnReader::read_column_data(ColumnPtr& doris_column, DataTypePtr ColumnPtr& doris_field = doris_struct.get_column_ptr(i); auto& doris_type = const_cast(doris_struct_type->get_element(i)); auto& doris_name = const_cast(doris_struct_type->get_element_name(i)); - - // remember the missing column index - if (_child_readers.find(doris_name) == _child_readers.end()) { + if (!root_node->children_column_exists(doris_name)) { missing_column_idxs.push_back(i); continue; } + auto file_name = root_node->children_file_column_name(doris_name); - _read_column_names.emplace_back(doris_name); + _read_column_names.emplace_back(file_name); - // select_vector.reset(); size_t field_rows = 0; bool field_eof = false; if (not_missing_column_id == -1) { not_missing_column_id = i; - RETURN_IF_ERROR(_child_readers[doris_name]->read_column_data( - doris_field, doris_type, filter_map, batch_size, &field_rows, &field_eof, - is_dict_filter)); + RETURN_IF_ERROR(_child_readers[file_name]->read_column_data( + doris_field, doris_type, root_node->get_children_node(doris_name), filter_map, + batch_size, &field_rows, &field_eof, is_dict_filter)); *read_rows = field_rows; *eof = field_eof; /* @@ -871,9 +876,10 @@ Status StructColumnReader::read_column_data(ColumnPtr& doris_column, DataTypePtr } else { while (field_rows < *read_rows && !field_eof) { size_t loop_rows = 0; - RETURN_IF_ERROR(_child_readers[doris_name]->read_column_data( - doris_field, doris_type, filter_map, *read_rows - field_rows, &loop_rows, - &field_eof, is_dict_filter)); + RETURN_IF_ERROR(_child_readers[file_name]->read_column_data( + doris_field, doris_type, root_node->get_children_node(doris_name), + filter_map, *read_rows - field_rows, &loop_rows, &field_eof, + is_dict_filter)); field_rows += loop_rows; } DCHECK_EQ(*read_rows, field_rows); diff --git a/be/src/vec/exec/format/parquet/vparquet_column_reader.h b/be/src/vec/exec/format/parquet/vparquet_column_reader.h index 1c4a431ba38e21..1d2c2afe5637ad 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_column_reader.h @@ -34,6 +34,7 @@ #include "vec/columns/columns_number.h" #include "vec/data_types/data_type.h" #include "vec/exec/format/parquet/parquet_common.h" +#include "vec/exec/format/table/table_format_reader.h" #include "vparquet_column_chunk_reader.h" namespace cctz { @@ -121,6 +122,7 @@ class ParquetColumnReader { : _row_ranges(row_ranges), _ctz(ctz), _io_ctx(io_ctx) {} virtual ~ParquetColumnReader() = default; virtual Status read_column_data(ColumnPtr& doris_column, DataTypePtr& type, + const std::shared_ptr& root_node, FilterMap& filter_map, size_t batch_size, size_t* read_rows, bool* eof, bool is_dict_filter) = 0; @@ -175,8 +177,9 @@ class ScalarColumnReader : public ParquetColumnReader { _offset_index(offset_index) {} ~ScalarColumnReader() override { close(); } Status init(io::FileReaderSPtr file, FieldSchema* field, size_t max_buf_size); - Status read_column_data(ColumnPtr& doris_column, DataTypePtr& type, FilterMap& filter_map, - size_t batch_size, size_t* read_rows, bool* eof, + Status read_column_data(ColumnPtr& doris_column, DataTypePtr& type, + const std::shared_ptr& root_node, + FilterMap& filter_map, size_t batch_size, size_t* read_rows, bool* eof, bool is_dict_filter) override; Status read_dict_values_to_column(MutableColumnPtr& doris_column, bool* has_dict) override; MutableColumnPtr convert_dict_column_to_string_column(const ColumnInt32* dict_column) override; @@ -221,8 +224,9 @@ class ArrayColumnReader : public ParquetColumnReader { : ParquetColumnReader(row_ranges, ctz, io_ctx) {} ~ArrayColumnReader() override { close(); } Status init(std::unique_ptr element_reader, FieldSchema* field); - Status read_column_data(ColumnPtr& doris_column, DataTypePtr& type, FilterMap& filter_map, - size_t batch_size, size_t* read_rows, bool* eof, + Status read_column_data(ColumnPtr& doris_column, DataTypePtr& type, + const std::shared_ptr& root_node, + FilterMap& filter_map, size_t batch_size, size_t* read_rows, bool* eof, bool is_dict_filter) override; const std::vector& get_rep_level() const override { return _element_reader->get_rep_level(); @@ -249,8 +253,9 @@ class MapColumnReader : public ParquetColumnReader { Status init(std::unique_ptr key_reader, std::unique_ptr value_reader, FieldSchema* field); - Status read_column_data(ColumnPtr& doris_column, DataTypePtr& type, FilterMap& filter_map, - size_t batch_size, size_t* read_rows, bool* eof, + Status read_column_data(ColumnPtr& doris_column, DataTypePtr& type, + const std::shared_ptr& root_node, + FilterMap& filter_map, size_t batch_size, size_t* read_rows, bool* eof, bool is_dict_filter) override; const std::vector& get_rep_level() const override { @@ -290,8 +295,9 @@ class StructColumnReader : public ParquetColumnReader { Status init( std::unordered_map>&& child_readers, FieldSchema* field); - Status read_column_data(ColumnPtr& doris_column, DataTypePtr& type, FilterMap& filter_map, - size_t batch_size, size_t* read_rows, bool* eof, + Status read_column_data(ColumnPtr& doris_column, DataTypePtr& type, + const std::shared_ptr& root_node, + FilterMap& filter_map, size_t batch_size, size_t* read_rows, bool* eof, bool is_dict_filter) override; const std::vector& get_rep_level() const override { diff --git a/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp index b24425d67c30c4..f55d170f4672be 100644 --- a/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp +++ b/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp @@ -83,7 +83,7 @@ RowGroupReader::RowGroupReader(io::FileReaderSPtr file_reader, const PositionDeleteContext& position_delete_ctx, const LazyReadContext& lazy_read_ctx, RuntimeState* state) : _file_reader(file_reader), - _read_columns(read_columns), + _read_table_columns(read_columns), _row_group_id(row_group_id), _row_group_meta(row_group), _remaining_rows(row_group.num_rows), @@ -111,15 +111,18 @@ Status RowGroupReader::init( _col_name_to_slot_id = colname_to_slot_id; _slot_id_to_filter_conjuncts = slot_id_to_filter_conjuncts; _merge_read_ranges(row_ranges); - if (_read_columns.empty()) { + if (_read_table_columns.empty()) { // Query task that only select columns in path. return Status::OK(); } const size_t MAX_GROUP_BUF_SIZE = config::parquet_rowgroup_max_buffer_mb << 20; const size_t MAX_COLUMN_BUF_SIZE = config::parquet_column_max_buffer_mb << 20; - size_t max_buf_size = std::min(MAX_COLUMN_BUF_SIZE, MAX_GROUP_BUF_SIZE / _read_columns.size()); - for (const auto& read_col : _read_columns) { - auto* field = const_cast(schema.get_column(read_col)); + size_t max_buf_size = + std::min(MAX_COLUMN_BUF_SIZE, MAX_GROUP_BUF_SIZE / _read_table_columns.size()); + for (const auto& read_table_col : _read_table_columns) { + auto read_file_col = _table_info_node_ptr->children_file_column_name(read_table_col); + + auto* field = const_cast(schema.get_column(read_file_col)); auto physical_index = field->physical_column_index; std::unique_ptr reader; // TODO : support rested column types @@ -133,7 +136,7 @@ Status RowGroupReader::init( VLOG_DEBUG << "Init row group(" << _row_group_id << ") reader failed"; return Status::Corruption("Init row group reader failed"); } - _column_readers[read_col] = std::move(reader); + _column_readers[read_table_col] = std::move(reader); } bool disable_dict_filter = false; @@ -150,7 +153,9 @@ Status RowGroupReader::init( for (size_t i = 0; i < predicate_col_names.size(); ++i) { const string& predicate_col_name = predicate_col_names[i]; int slot_id = predicate_col_slot_ids[i]; - auto field = const_cast(schema.get_column(predicate_col_name)); + auto predicate_file_col_name = + _table_info_node_ptr->children_file_column_name(predicate_col_name); + auto field = const_cast(schema.get_column(predicate_file_col_name)); if (!disable_dict_filter && !_lazy_read_ctx.has_complex_type && _can_filter_by_dict( slot_id, _row_group_meta.columns[field->physical_column_index].meta_data)) { @@ -293,8 +298,9 @@ Status RowGroupReader::next_batch(Block* block, size_t batch_size, size_t* read_ } // Process external table query task that select columns are all from path. - if (_read_columns.empty()) { + if (_read_table_columns.empty()) { RETURN_IF_ERROR(_read_empty_batch(batch_size, read_rows, batch_eof)); + RETURN_IF_ERROR( _fill_partition_columns(block, *read_rows, _lazy_read_ctx.partition_columns)); RETURN_IF_ERROR(_fill_missing_columns(block, *read_rows, _lazy_read_ctx.missing_columns)); @@ -369,12 +375,13 @@ void RowGroupReader::_merge_read_ranges(std::vector& row_ranges) { _read_ranges = row_ranges; } -Status RowGroupReader::_read_column_data(Block* block, const std::vector& columns, +Status RowGroupReader::_read_column_data(Block* block, + const std::vector& table_columns, size_t batch_size, size_t* read_rows, bool* batch_eof, FilterMap& filter_map) { size_t batch_read_rows = 0; bool has_eof = false; - for (auto& read_col_name : columns) { + for (auto& read_col_name : table_columns) { auto& column_with_type_and_name = block->get_by_name(read_col_name); auto& column_ptr = column_with_type_and_name.column; auto& column_type = column_with_type_and_name.type; @@ -407,8 +414,8 @@ Status RowGroupReader::_read_column_data(Block* block, const std::vectorread_column_data( - column_ptr, column_type, filter_map, batch_size - col_read_rows, &loop_rows, - &col_eof, is_dict_filter)); + column_ptr, column_type, _table_info_node_ptr->get_children_node(read_col_name), + filter_map, batch_size - col_read_rows, &loop_rows, &col_eof, is_dict_filter)); col_read_rows += loop_rows; } if (batch_read_rows > 0 && batch_read_rows != col_read_rows) { diff --git a/be/src/vec/exec/format/parquet/vparquet_group_reader.h b/be/src/vec/exec/format/parquet/vparquet_group_reader.h index 758d376396a14f..69c66a4d1e493f 100644 --- a/be/src/vec/exec/format/parquet/vparquet_group_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_group_reader.h @@ -30,6 +30,7 @@ #include "io/fs/file_reader_writer_fwd.h" #include "vec/columns/column.h" #include "vec/exec/format/parquet/parquet_common.h" +#include "vec/exec/format/table/table_format_reader.h" #include "vec/exprs/vexpr_fwd.h" #include "vparquet_column_reader.h" @@ -64,6 +65,7 @@ static constexpr uint32_t MAX_DICT_CODE_PREDICATE_TO_REWRITE = std::numeric_limi class RowGroupReader : public ProfileCollector { public: + std::shared_ptr _table_info_node_ptr; static const std::vector NO_DELETE; struct RowGroupIndex { @@ -74,6 +76,7 @@ class RowGroupReader : public ProfileCollector { : row_group_id(id), first_row(first), last_row(last) {} }; + // table name struct LazyReadContext { VExprContextSPtrs conjuncts; bool can_lazy_read = false; @@ -200,8 +203,10 @@ class RowGroupReader : public ProfileCollector { void _convert_dict_cols_to_string_cols(Block* block); io::FileReaderSPtr _file_reader; - std::unordered_map> _column_readers; - const std::vector& _read_columns; + std::unordered_map> + _column_readers; // table_column_name + const std::vector& _read_table_columns; + const int32_t _row_group_id; const tparquet::RowGroup& _row_group_meta; int64_t _remaining_rows; diff --git a/be/src/vec/exec/format/parquet/vparquet_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_reader.cpp index 41db94aecf240e..316b19701c747f 100644 --- a/be/src/vec/exec/format/parquet/vparquet_reader.cpp +++ b/be/src/vec/exec/format/parquet/vparquet_reader.cpp @@ -38,6 +38,7 @@ #include "runtime/descriptors.h" #include "runtime/types.h" #include "util/slice.h" +#include "util/string_util.h" #include "util/timezone_utils.h" #include "vec/columns/column.h" #include "vec/common/typeid_cast.h" @@ -256,8 +257,11 @@ Status ParquetReader::_open_file() { return Status::OK(); } -const FieldDescriptor ParquetReader::get_file_metadata_schema() { - return _file_metadata->schema(); +Status ParquetReader::get_file_metadata_schema(const FieldDescriptor** ptr) { + RETURN_IF_ERROR(_open_file()); + DCHECK(_file_metadata != nullptr); + *ptr = &_file_metadata->schema(); + return Status::OK(); } void ParquetReader::_init_system_properties() { @@ -291,79 +295,51 @@ void ParquetReader::iceberg_sanitize(const std::vector& read_column Status ParquetReader::init_reader( const std::vector& all_column_names, - const std::vector& missing_column_names, const std::unordered_map* colname_to_value_range, const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, const RowDescriptor* row_descriptor, const std::unordered_map* colname_to_slot_id, const VExprContextSPtrs* not_single_slot_filter_conjuncts, const std::unordered_map* slot_id_to_filter_conjuncts, - bool filter_groups, const bool hive_use_column_names) { + std::shared_ptr table_info_node_ptr, bool filter_groups) { _tuple_descriptor = tuple_descriptor; _row_descriptor = row_descriptor; _colname_to_slot_id = colname_to_slot_id; _not_single_slot_filter_conjuncts = not_single_slot_filter_conjuncts; _slot_id_to_filter_conjuncts = slot_id_to_filter_conjuncts; _colname_to_value_range = colname_to_value_range; - _hive_use_column_names = hive_use_column_names; + _table_info_node_ptr = table_info_node_ptr; + RETURN_IF_ERROR(_open_file()); _t_metadata = &(_file_metadata->to_thrift()); + if (_file_metadata == nullptr) { + return Status::InternalError("failed to init parquet reader, please open reader first"); + } + SCOPED_RAW_TIMER(&_statistics.parse_meta_time); _total_groups = _t_metadata->row_groups.size(); if (_total_groups == 0) { return Status::EndOfFile("init reader failed, empty parquet file: " + _scan_range.path); } - // all_column_names are all the columns required by user sql. - // missing_column_names are the columns required by user sql but not in the parquet file, - // e.g. table added a column after this parquet file was written. - _column_names = &all_column_names; + _table_column_names = &all_column_names; auto schema_desc = _file_metadata->schema(); - if (_hive_use_column_names) { - std::set required_columns(all_column_names.begin(), all_column_names.end()); - // Currently only used in iceberg, the columns are dropped but added back - std::set dropped_columns(missing_column_names.begin(), - missing_column_names.end()); - // Make the order of read columns the same as physical order in parquet file - for (int i = 0; i < schema_desc.size(); ++i) { - auto name = schema_desc.get_column(i)->name; - // If the column in parquet file is included in all_column_names and not in missing_column_names, - // add it to _map_column, which means the reader should read the data of this column. - // Here to check against missing_column_names is for the 'Add a column back to the table - // with the same column name' case. (drop column a then add column a). - // Shouldn't read this column data in this case. - if (required_columns.find(name) != required_columns.end() && - dropped_columns.find(name) == dropped_columns.end()) { - required_columns.erase(name); - _read_columns.emplace_back(name); - } - } - for (const std::string& name : required_columns) { - _missing_cols.emplace_back(name); - } - } else { - const auto& table_column_idxs = _scan_params.column_idxs; - std::map table_col_id_to_idx; - for (int i = 0; i < table_column_idxs.size(); i++) { - table_col_id_to_idx.insert({table_column_idxs[i], i}); + std::map required_file_columns; //file column -> table column + for (auto table_column_name : all_column_names) { + if (_table_info_node_ptr->children_column_exists(table_column_name)) { + required_file_columns.emplace( + _table_info_node_ptr->children_file_column_name(table_column_name), + table_column_name); + } else { + _missing_cols.emplace_back(table_column_name); } - - for (auto [id, idx] : table_col_id_to_idx) { - if (id >= schema_desc.size()) { - _missing_cols.emplace_back(all_column_names[idx]); - } else { - auto& table_col = all_column_names[idx]; - auto file_col = schema_desc.get_column(id)->name; - _read_columns.emplace_back(file_col); - _table_col_to_file_col[table_col] = file_col; - - auto iter = _colname_to_value_range->find(table_col); - if (iter != _colname_to_value_range->end()) { - _colname_to_value_range_index_read.emplace(file_col, iter->second); - } - } + } + for (int i = 0; i < schema_desc.size(); ++i) { + auto name = schema_desc.get_column(i)->name; + if (required_file_columns.find(name) != required_file_columns.end()) { + _read_file_columns.emplace_back(name); + _read_table_columns.emplace_back(required_file_columns[name]); } - _colname_to_value_range = &_colname_to_value_range_index_read; } // build column predicates for column lazy read _lazy_read_ctx.conjuncts = conjuncts; @@ -381,10 +357,6 @@ Status ParquetReader::set_fill_columns( std::function visit_slot = [&](VExpr* expr) { if (VSlotRef* slot_ref = typeid_cast(expr)) { auto expr_name = slot_ref->expr_name(); - auto iter = _table_col_to_file_col.find(expr_name); - if (iter != _table_col_to_file_col.end()) { - expr_name = iter->second; - } predicate_columns.emplace(expr_name, std::make_pair(slot_ref->column_id(), slot_ref->slot_id())); if (slot_ref->column_id() == 0) { @@ -413,6 +385,7 @@ Status ParquetReader::set_fill_columns( } } }; + if (!_lazy_read_ctx.conjuncts.empty()) { for (auto& conjunct : _lazy_read_ctx.conjuncts) { visit_slot(conjunct->root().get()); @@ -420,16 +393,18 @@ Status ParquetReader::set_fill_columns( } const FieldDescriptor& schema = _file_metadata->schema(); - for (auto& read_col : _read_columns) { - _lazy_read_ctx.all_read_columns.emplace_back(read_col); - PrimitiveType column_type = schema.get_column(read_col)->type.type; + for (auto& read_table_col : _read_table_columns) { + _lazy_read_ctx.all_read_columns.emplace_back(read_table_col); + + auto file_column_name = _table_info_node_ptr->children_file_column_name(read_table_col); + PrimitiveType column_type = schema.get_column(file_column_name)->type.type; if (column_type == TYPE_ARRAY || column_type == TYPE_MAP || column_type == TYPE_STRUCT) { _lazy_read_ctx.has_complex_type = true; } if (predicate_columns.size() > 0) { - auto iter = predicate_columns.find(read_col); + auto iter = predicate_columns.find(read_table_col); if (iter == predicate_columns.end()) { - _lazy_read_ctx.lazy_read_columns.emplace_back(read_col); + _lazy_read_ctx.lazy_read_columns.emplace_back(read_table_col); } else { _lazy_read_ctx.predicate_columns.first.emplace_back(iter->first); _lazy_read_ctx.predicate_columns.second.emplace_back(iter->second.second); @@ -551,17 +526,6 @@ Status ParquetReader::get_next_block(Block* block, size_t* read_rows, bool* eof) return Status::OK(); } - std::vector original_block_column_name = block->get_names(); - if (!_hive_use_column_names) { - for (auto i = 0; i < block->get_names().size(); i++) { - auto& col = block->get_by_position(i); - if (_table_col_to_file_col.contains(col.name)) { - col.name = _table_col_to_file_col[col.name]; - } - } - block->initialize_index_by_name(); - } - SCOPED_RAW_TIMER(&_statistics.column_read_time); Status batch_st = _current_group_reader->next_batch(block, _batch_size, read_rows, &_row_group_eof); @@ -573,12 +537,6 @@ Status ParquetReader::get_next_block(Block* block, size_t* read_rows, bool* eof) return Status::OK(); } - if (!_hive_use_column_names) { - for (auto i = 0; i < block->columns(); i++) { - block->get_by_position(i).name = original_block_column_name[i]; - } - block->initialize_index_by_name(); - } if (!batch_st.ok()) { return Status::InternalError("Read parquet file {} failed, reason = {}", _scan_range.path, batch_st.to_string()); @@ -651,9 +609,11 @@ Status ParquetReader::_next_row_group_reader() { : _file_reader; } _current_group_reader.reset(new RowGroupReader( - group_file_reader, _read_columns, row_group_index.row_group_id, row_group, _ctz, + group_file_reader, _read_table_columns, row_group_index.row_group_id, row_group, _ctz, _io_ctx, position_delete_ctx, _lazy_read_ctx, _state)); _row_group_eof = false; + + _current_group_reader->_table_info_node_ptr = _table_info_node_ptr; return _current_group_reader->init(_file_metadata->schema(), candidate_row_ranges, _col_offsets, _tuple_descriptor, _row_descriptor, _colname_to_slot_id, _not_single_slot_filter_conjuncts, @@ -692,7 +652,7 @@ Status ParquetReader::_init_row_groups(const bool& is_filter_groups) { } return size; }; - for (auto& read_col : _read_columns) { + for (auto& read_col : _read_file_columns) { const FieldSchema* field = _file_metadata->schema().get_column(read_col); group_size += column_compressed_size(field); } @@ -749,7 +709,7 @@ std::vector ParquetReader::_generate_random_access_ranges( } }; const tparquet::RowGroup& row_group = _t_metadata->row_groups[group.row_group_id]; - for (const auto& read_col : _read_columns) { + for (const auto& read_col : _read_file_columns) { const FieldSchema* field = _file_metadata->schema().get_column(read_col); scalar_range(field, row_group); } @@ -823,12 +783,16 @@ Status ParquetReader::_process_page_index(const tparquet::RowGroup& row_group, // read twice: parse column index & parse offset index _column_statistics.meta_read_calls += 2; SCOPED_RAW_TIMER(&_statistics.parse_page_index_time); - for (auto& read_col : _read_columns) { - auto conjunct_iter = _colname_to_value_range->find(read_col); + + for (size_t idx = 0; idx < _read_table_columns.size(); idx++) { + const auto& read_table_col = _read_table_columns[idx]; + const auto& read_file_col = _read_file_columns[idx]; + auto conjunct_iter = _colname_to_value_range->find(read_table_col); if (_colname_to_value_range->end() == conjunct_iter) { continue; } - int parquet_col_id = _file_metadata->schema().get_column(read_col)->physical_column_index; + int parquet_col_id = + _file_metadata->schema().get_column(read_file_col)->physical_column_index; if (parquet_col_id < 0) { // complex type, not support page index yet. continue; @@ -845,7 +809,7 @@ Status ParquetReader::_process_page_index(const tparquet::RowGroup& row_group, } auto& conjuncts = conjunct_iter->second; std::vector skipped_page_range; - const FieldSchema* col_schema = schema_desc.get_column(read_col); + const FieldSchema* col_schema = schema_desc.get_column(read_file_col); RETURN_IF_ERROR(page_index.collect_skipped_page_range(&column_index, conjuncts, col_schema, skipped_page_range, *_ctz)); if (skipped_page_range.empty()) { @@ -913,12 +877,19 @@ Status ParquetReader::_process_column_stat_filter(const std::vectorschema(); - for (auto& col_name : _read_columns) { - auto slot_iter = _colname_to_value_range->find(col_name); + for (auto& table_col_name : _read_table_columns) { + if (_table_info_node_ptr->children_column_exists(table_col_name)) { + continue; + } + + auto slot_iter = _colname_to_value_range->find(table_col_name); if (slot_iter == _colname_to_value_range->end()) { continue; } - int parquet_col_id = _file_metadata->schema().get_column(col_name)->physical_column_index; + + auto file_col_name = _table_info_node_ptr->children_file_column_name(table_col_name); + int parquet_col_id = + _file_metadata->schema().get_column(file_col_name)->physical_column_index; if (parquet_col_id < 0) { // complex type, not support filter yet. continue; @@ -932,7 +903,7 @@ Status ParquetReader::_process_column_stat_filter(const std::vector& all_column_names, - const std::vector& missing_column_names, const std::unordered_map* colname_to_value_range, const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, const RowDescriptor* row_descriptor, const std::unordered_map* colname_to_slot_id, const VExprContextSPtrs* not_single_slot_filter_conjuncts, const std::unordered_map* slot_id_to_filter_conjuncts, - bool filter_groups = true, const bool hive_use_column_names = true); + std::shared_ptr table_info_node_ptr = + TableSchemaChangeHelper::ConstNode::get_instance(), + bool filter_groups = true); Status get_next_block(Block* block, size_t* read_rows, bool* eof) override; @@ -149,10 +151,7 @@ class ParquetReader : public GenericReader { partition_columns, const std::unordered_map& missing_columns) override; - const FieldDescriptor get_file_metadata_schema(); - void set_table_to_file_col_map(std::unordered_map& map) { - _table_col_to_file_col = map; - } + Status get_file_metadata_schema(const FieldDescriptor** ptr); protected: void _collect_profile_before_close() override; @@ -250,15 +249,17 @@ class ParquetReader : public GenericReader { // read to the end of current reader bool _row_group_eof = true; int32_t _total_groups; // num of groups(stripes) of a parquet(orc) file - // table column name to file column name map. For iceberg schema evolution. - std::unordered_map _table_col_to_file_col; + + // Through this node, you can find the file column based on the table column. + std::shared_ptr _table_info_node_ptr = + TableSchemaChangeHelper::ConstNode::get_instance(); + const std::unordered_map* _colname_to_value_range = nullptr; - // During initialization, multiple vfile_scanner's _colname_to_value_range will point to the same object, - // so the content in the object cannot be modified (there is a multi-threading problem). - // _colname_to_value_range_index_read used when _hive_use_column_names = false. - std::unordered_map _colname_to_value_range_index_read; - std::vector _read_columns; + //sequence in file, need to read + std::vector _read_table_columns; + std::vector _read_file_columns; + RowRange _whole_range = RowRange(0, 0); const std::vector* _delete_rows = nullptr; int64_t _delete_rows_index = 0; @@ -274,9 +275,11 @@ class ParquetReader : public GenericReader { cctz::time_zone* _ctz = nullptr; std::unordered_map _col_offsets; - const std::vector* _column_names = nullptr; std::vector _missing_cols; + // _table_column_names = _missing_cols + _read_table_columns + const std::vector* _table_column_names = nullptr; + Statistics _statistics; ParquetColumnReader::Statistics _column_statistics; ParquetProfile _parquet_profile; @@ -294,6 +297,5 @@ class ParquetReader : public GenericReader { const VExprContextSPtrs* _not_single_slot_filter_conjuncts = nullptr; const std::unordered_map* _slot_id_to_filter_conjuncts = nullptr; std::unordered_map _ignored_stats; - bool _hive_use_column_names = false; }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/table/hive_reader.cpp b/be/src/vec/exec/format/table/hive_reader.cpp new file mode 100644 index 00000000000000..ee7f805b076a3c --- /dev/null +++ b/be/src/vec/exec/format/table/hive_reader.cpp @@ -0,0 +1,132 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "hive_reader.h" + +#include + +#include "common/status.h" +#include "runtime/runtime_state.h" + +namespace doris::vectorized { +#include "common/compile_check_begin.h" + +Status HiveReader::get_next_block_inner(Block* block, size_t* read_rows, bool* eof) { + RETURN_IF_ERROR(_file_format_reader->get_next_block(block, read_rows, eof)); + return Status::OK(); +}; + +Status HiveOrcReader::init_reader( + const std::vector& read_table_col_names, + const std::unordered_map* table_col_name_to_value_range, + const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, + const RowDescriptor* row_descriptor, + const VExprContextSPtrs* not_single_slot_filter_conjuncts, + const std::unordered_map* slot_id_to_filter_conjuncts) { + auto* orc_reader = static_cast(_file_format_reader.get()); + + const orc::Type* orc_type_ptr = nullptr; + RETURN_IF_ERROR(orc_reader->get_file_type(&orc_type_ptr)); + bool is_hive_col_name = OrcReader::is_hive1_col_name(orc_type_ptr); + + if (_state->query_options().hive_orc_use_column_names && !is_hive_col_name) { + // Directly use the table column name to match the file column name, but pay attention to the case issue. + RETURN_IF_ERROR(BuildTableInfoUtil::by_orc_name(tuple_descriptor, orc_type_ptr, + table_info_node_ptr)); + } else { + // hive1 / use index + std::map slot_map; // table_name to slot + for (const auto& slot : tuple_descriptor->slots()) { + slot_map.emplace(slot->col_name(), slot); + } + + // For top-level columns, use indexes to match, and for sub-columns, still use name to match columns. + for (size_t idx = 0; idx < _params.column_idxs.size(); idx++) { + auto table_column_name = read_table_col_names[idx]; + auto file_index = _params.column_idxs[idx]; + + if (file_index >= orc_type_ptr->getSubtypeCount()) { + table_info_node_ptr->add_not_exist_children(table_column_name); + } else { + auto field_node = std::make_shared(); + // For sub-columns, still use name to match columns. + RETURN_IF_ERROR(BuildTableInfoUtil::by_orc_name( + slot_map[table_column_name]->get_data_type_ptr(), + orc_type_ptr->getSubtype(file_index), field_node)); + table_info_node_ptr->add_children( + table_column_name, orc_type_ptr->getFieldName(file_index), field_node); + } + } + } + + return orc_reader->init_reader(&read_table_col_names, table_col_name_to_value_range, conjuncts, + false, tuple_descriptor, row_descriptor, + not_single_slot_filter_conjuncts, slot_id_to_filter_conjuncts, + table_info_node_ptr); +} + +Status HiveParquetReader::init_reader( + const std::vector& read_table_col_names, + const std::unordered_map* table_col_name_to_value_range, + const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, + const RowDescriptor* row_descriptor, + const std::unordered_map* colname_to_slot_id, + const VExprContextSPtrs* not_single_slot_filter_conjuncts, + const std::unordered_map* slot_id_to_filter_conjuncts) { + auto* parquet_reader = static_cast(_file_format_reader.get()); + const FieldDescriptor* field_desc = nullptr; + RETURN_IF_ERROR(parquet_reader->get_file_metadata_schema(&field_desc)); + DCHECK(field_desc != nullptr); + + if (_state->query_options().hive_parquet_use_column_names) { + // Directly use the table column name to match the file column name, but pay attention to the case issue. + RETURN_IF_ERROR(BuildTableInfoUtil::by_parquet_name(tuple_descriptor, *field_desc, + table_info_node_ptr)); + } else { // use idx + std::map slot_map; //table_name to slot + for (const auto& slot : tuple_descriptor->slots()) { + slot_map.emplace(slot->col_name(), slot); + } + + // For top-level columns, use indexes to match, and for sub-columns, still use name to match columns. + auto parquet_fields_schema = field_desc->get_fields_schema(); + for (size_t idx = 0; idx < _params.column_idxs.size(); idx++) { + auto table_column_name = read_table_col_names[idx]; + auto file_index = _params.column_idxs[idx]; + + if (file_index >= parquet_fields_schema.size()) { + table_info_node_ptr->add_not_exist_children(table_column_name); + } else { + auto field_node = std::make_shared(); + // for sub-columns, still use name to match columns. + RETURN_IF_ERROR(BuildTableInfoUtil::by_parquet_name( + slot_map[table_column_name]->get_data_type_ptr(), + parquet_fields_schema[file_index], field_node)); + table_info_node_ptr->add_children( + table_column_name, parquet_fields_schema[file_index].name, field_node); + } + } + } + + return parquet_reader->init_reader(read_table_col_names, table_col_name_to_value_range, + conjuncts, tuple_descriptor, row_descriptor, + colname_to_slot_id, not_single_slot_filter_conjuncts, + slot_id_to_filter_conjuncts, table_info_node_ptr); +} + +#include "common/compile_check_end.h" +} // namespace doris::vectorized diff --git a/be/src/vec/exec/format/table/hive_reader.h b/be/src/vec/exec/format/table/hive_reader.h new file mode 100644 index 00000000000000..2f2c1151799466 --- /dev/null +++ b/be/src/vec/exec/format/table/hive_reader.h @@ -0,0 +1,83 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once +#include +#include + +#include "vec/exec/format/orc/vorc_reader.h" +#include "vec/exec/format/parquet/vparquet_reader.h" +#include "vec/exec/format/table/table_format_reader.h" +namespace doris::vectorized { +#include "common/compile_check_begin.h" + +// By holding a parquet/orc reader, used to read the parquet/orc table of hive. +class HiveReader : public TableFormatReader, public TableSchemaChangeHelper { +public: + HiveReader(std::unique_ptr file_format_reader, RuntimeProfile* profile, + RuntimeState* state, const TFileScanRangeParams& params, const TFileRangeDesc& range, + io::IOContext* io_ctx) + : TableFormatReader(std::move(file_format_reader), state, profile, params, range, + io_ctx) {}; + + ~HiveReader() override = default; + + Status get_next_block_inner(Block* block, size_t* read_rows, bool* eof) final; + + Status init_row_filters() final { return Status::OK(); }; +}; + +class HiveOrcReader final : public HiveReader { +public: + ENABLE_FACTORY_CREATOR(HiveOrcReader); + HiveOrcReader(std::unique_ptr file_format_reader, RuntimeProfile* profile, + RuntimeState* state, const TFileScanRangeParams& params, + const TFileRangeDesc& range, io::IOContext* io_ctx) + : HiveReader(std::move(file_format_reader), profile, state, params, range, io_ctx) {}; + ~HiveOrcReader() final = default; + + Status init_reader( + const std::vector& read_table_col_names, + const std::unordered_map* + table_col_name_to_value_range, + const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, + const RowDescriptor* row_descriptor, + const VExprContextSPtrs* not_single_slot_filter_conjuncts, + const std::unordered_map* slot_id_to_filter_conjuncts); +}; + +class HiveParquetReader final : public HiveReader { +public: + ENABLE_FACTORY_CREATOR(HiveParquetReader); + HiveParquetReader(std::unique_ptr file_format_reader, RuntimeProfile* profile, + RuntimeState* state, const TFileScanRangeParams& params, + const TFileRangeDesc& range, io::IOContext* io_ctx) + : HiveReader(std::move(file_format_reader), profile, state, params, range, io_ctx) {}; + ~HiveParquetReader() final = default; + + Status init_reader( + const std::vector& read_table_col_names, + const std::unordered_map* + table_col_name_to_value_range, + const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, + const RowDescriptor* row_descriptor, + const std::unordered_map* colname_to_slot_id, + const VExprContextSPtrs* not_single_slot_filter_conjuncts, + const std::unordered_map* slot_id_to_filter_conjuncts); +}; +#include "common/compile_check_end.h" +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/exec/format/table/hudi_reader.cpp b/be/src/vec/exec/format/table/hudi_reader.cpp index 6caeb87badbe9e..ce0d69cc92d14a 100644 --- a/be/src/vec/exec/format/table/hudi_reader.cpp +++ b/be/src/vec/exec/format/table/hudi_reader.cpp @@ -25,67 +25,32 @@ namespace doris::vectorized { #include "common/compile_check_begin.h" -Status HudiReader::get_file_col_id_to_name(bool& exist_schema, - std::map& file_col_id_to_name) { - if (!_params.__isset.history_schema_info) [[unlikely]] { - exist_schema = false; - return Status::OK(); - } - - if (!_params.history_schema_info.contains(_range.table_format_params.hudi_params.schema_id)) - [[unlikely]] { - return Status::InternalError("hudi file schema info is missing in history schema info."); - } - - file_col_id_to_name = - _params.history_schema_info.at(_range.table_format_params.hudi_params.schema_id); - return Status::OK(); -} - Status HudiReader::get_next_block_inner(Block* block, size_t* read_rows, bool* eof) { - RETURN_IF_ERROR(TableSchemaChangeHelper::get_next_block_before(block)); RETURN_IF_ERROR(_file_format_reader->get_next_block(block, read_rows, eof)); - RETURN_IF_ERROR(TableSchemaChangeHelper::get_next_block_after(block)); return Status::OK(); }; -Status HudiOrcReader::init_reader( - const std::vector& read_table_col_names, - const std::unordered_map& table_col_id_table_name_map, - const std::unordered_map* table_col_name_to_value_range, - const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, - const RowDescriptor* row_descriptor, - const VExprContextSPtrs* not_single_slot_filter_conjuncts, - const std::unordered_map* slot_id_to_filter_conjuncts) { - RETURN_IF_ERROR(TableSchemaChangeHelper::init_schema_info( - read_table_col_names, table_col_id_table_name_map, table_col_name_to_value_range)); - - auto* orc_reader = static_cast(_file_format_reader.get()); - orc_reader->set_table_col_to_file_col(_table_col_to_file_col); - return orc_reader->init_reader(&_all_required_col_names, _not_in_file_col_names, - &_new_colname_to_value_range, conjuncts, false, tuple_descriptor, - row_descriptor, not_single_slot_filter_conjuncts, - slot_id_to_filter_conjuncts); -} - Status HudiParquetReader::init_reader( const std::vector& read_table_col_names, - const std::unordered_map& table_col_id_table_name_map, const std::unordered_map* table_col_name_to_value_range, const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, const RowDescriptor* row_descriptor, const std::unordered_map* colname_to_slot_id, const VExprContextSPtrs* not_single_slot_filter_conjuncts, const std::unordered_map* slot_id_to_filter_conjuncts) { - RETURN_IF_ERROR(TableSchemaChangeHelper::init_schema_info( - read_table_col_names, table_col_id_table_name_map, table_col_name_to_value_range)); auto* parquet_reader = static_cast(_file_format_reader.get()); - parquet_reader->set_table_to_file_col_map(_table_col_to_file_col); - - return parquet_reader->init_reader( - _all_required_col_names, _not_in_file_col_names, &_new_colname_to_value_range, - conjuncts, tuple_descriptor, row_descriptor, colname_to_slot_id, - not_single_slot_filter_conjuncts, slot_id_to_filter_conjuncts); + const FieldDescriptor* field_desc = nullptr; + RETURN_IF_ERROR(parquet_reader->get_file_metadata_schema(&field_desc)); + DCHECK(field_desc != nullptr); + + auto parquet_fields_schema = field_desc->get_fields_schema(); + RETURN_IF_ERROR(gen_table_info_node_by_field_id( + _params, _range.table_format_params.hudi_params.schema_id, tuple_descriptor, + *field_desc)); + return parquet_reader->init_reader(read_table_col_names, table_col_name_to_value_range, + conjuncts, tuple_descriptor, row_descriptor, + colname_to_slot_id, not_single_slot_filter_conjuncts, + slot_id_to_filter_conjuncts, table_info_node_ptr); } #include "common/compile_check_end.h" diff --git a/be/src/vec/exec/format/table/hudi_reader.h b/be/src/vec/exec/format/table/hudi_reader.h index 2779f296f66266..add7d094ae246b 100644 --- a/be/src/vec/exec/format/table/hudi_reader.h +++ b/be/src/vec/exec/format/table/hudi_reader.h @@ -33,34 +33,11 @@ class HudiReader : public TableFormatReader, public TableSchemaChangeHelper { ~HudiReader() override = default; - Status get_file_col_id_to_name(bool& exist_schema, - std::map& file_col_id_to_name) final; - Status get_next_block_inner(Block* block, size_t* read_rows, bool* eof) final; Status init_row_filters() final { return Status::OK(); }; }; -class HudiOrcReader final : public HudiReader { -public: - ENABLE_FACTORY_CREATOR(HudiOrcReader); - HudiOrcReader(std::unique_ptr file_format_reader, RuntimeProfile* profile, - RuntimeState* state, const TFileScanRangeParams& params, - const TFileRangeDesc& range, io::IOContext* io_ctx) - : HudiReader(std::move(file_format_reader), profile, state, params, range, io_ctx) {}; - ~HudiOrcReader() final = default; - - Status init_reader( - const std::vector& read_table_col_names, - const std::unordered_map& table_col_id_table_name_map, - const std::unordered_map* - table_col_name_to_value_range, - const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, - const RowDescriptor* row_descriptor, - const VExprContextSPtrs* not_single_slot_filter_conjuncts, - const std::unordered_map* slot_id_to_filter_conjuncts); -}; - class HudiParquetReader final : public HudiReader { public: ENABLE_FACTORY_CREATOR(HudiParquetReader); @@ -72,7 +49,6 @@ class HudiParquetReader final : public HudiReader { Status init_reader( const std::vector& read_table_col_names, - const std::unordered_map& table_col_id_table_name_map, const std::unordered_map* table_col_name_to_value_range, const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, diff --git a/be/src/vec/exec/format/table/iceberg_reader.cpp b/be/src/vec/exec/format/table/iceberg_reader.cpp index e45a747a7ac303..d6471a49efb596 100644 --- a/be/src/vec/exec/format/table/iceberg_reader.cpp +++ b/be/src/vec/exec/format/table/iceberg_reader.cpp @@ -72,6 +72,8 @@ class VExprContext; } // namespace doris namespace doris::vectorized { +const std::string IcebergOrcReader::ICEBERG_ORC_ATTRIBUTE = "iceberg.id"; + IcebergTableReader::IcebergTableReader(std::unique_ptr file_format_reader, RuntimeProfile* profile, RuntimeState* state, const TFileScanRangeParams& params, @@ -94,9 +96,7 @@ IcebergTableReader::IcebergTableReader(std::unique_ptr file_forma Status IcebergTableReader::get_next_block_inner(Block* block, size_t* read_rows, bool* eof) { RETURN_IF_ERROR(_expand_block_if_need(block)); - RETURN_IF_ERROR(TableSchemaChangeHelper::get_next_block_before(block)); RETURN_IF_ERROR(_file_format_reader->get_next_block(block, read_rows, eof)); - RETURN_IF_ERROR(TableSchemaChangeHelper::get_next_block_after(block)); if (_equality_delete_impl != nullptr) { RETURN_IF_ERROR(_equality_delete_impl->filter_data_block(block)); @@ -149,7 +149,6 @@ Status IcebergTableReader::_equality_delete_base( std::unordered_map> partition_columns; std::unordered_map missing_columns; - std::vector not_in_file_col_names; for (const auto& delete_file : delete_files) { TFileRangeDesc delete_desc; @@ -169,13 +168,12 @@ Status IcebergTableReader::_equality_delete_base( init_schema = true; } if (auto* parquet_reader = typeid_cast(delete_reader.get())) { - RETURN_IF_ERROR(parquet_reader->init_reader(equality_delete_col_names, - not_in_file_col_names, nullptr, {}, nullptr, - nullptr, nullptr, nullptr, nullptr, false)); + RETURN_IF_ERROR(parquet_reader->init_reader( + equality_delete_col_names, nullptr, {}, nullptr, nullptr, nullptr, nullptr, + nullptr, TableSchemaChangeHelper::ConstNode::get_instance(), false)); } else if (auto* orc_reader = typeid_cast(delete_reader.get())) { - RETURN_IF_ERROR(orc_reader->init_reader(&equality_delete_col_names, - not_in_file_col_names, nullptr, {}, false, {}, - {}, nullptr, nullptr)); + RETURN_IF_ERROR(orc_reader->init_reader(&equality_delete_col_names, nullptr, {}, false, + {}, {}, nullptr, nullptr)); } else { return Status::InternalError("Unsupported format of delete file"); } @@ -419,7 +417,6 @@ void IcebergTableReader::_gen_position_delete_file_range(Block& block, DeleteFil Status IcebergParquetReader::init_reader( const std::vector& file_col_names, - const std::unordered_map& col_id_name_map, const std::unordered_map* colname_to_value_range, const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, const RowDescriptor* row_descriptor, @@ -428,16 +425,32 @@ Status IcebergParquetReader::init_reader( const std::unordered_map* slot_id_to_filter_conjuncts) { _file_format = Fileformat::PARQUET; auto* parquet_reader = static_cast(_file_format_reader.get()); - RETURN_IF_ERROR(TableSchemaChangeHelper::init_schema_info(file_col_names, col_id_name_map, - colname_to_value_range)); - parquet_reader->set_table_to_file_col_map(_table_col_to_file_col); - parquet_reader->iceberg_sanitize(_all_required_col_names); - RETURN_IF_ERROR(init_row_filters()); + const FieldDescriptor* field_desc = nullptr; + RETURN_IF_ERROR(parquet_reader->get_file_metadata_schema(&field_desc)); + DCHECK(field_desc != nullptr); - return parquet_reader->init_reader( - _all_required_col_names, _not_in_file_col_names, &_new_colname_to_value_range, - conjuncts, tuple_descriptor, row_descriptor, colname_to_slot_id, - not_single_slot_filter_conjuncts, slot_id_to_filter_conjuncts); + if (!_params.__isset.history_schema_info || _params.history_schema_info.empty()) [[unlikely]] { + RETURN_IF_ERROR(BuildTableInfoUtil::by_parquet_name(tuple_descriptor, *field_desc, + table_info_node_ptr)); + } else { + bool exist_field_id = true; + // Iceberg will record the field id in the parquet file and find the column to read by matching it with the field id of the table (from fe). + RETURN_IF_ERROR(BuildTableInfoUtil::by_parquet_field_id( + _params.history_schema_info.front().root_field, *field_desc, table_info_node_ptr, + exist_field_id)); + if (!exist_field_id) { + // For early iceberg version, field id may not be available, so name matching is used here. + RETURN_IF_ERROR(BuildTableInfoUtil::by_parquet_name(tuple_descriptor, *field_desc, + table_info_node_ptr)); + } + } + + _all_required_col_names = file_col_names; + RETURN_IF_ERROR(init_row_filters()); + return parquet_reader->init_reader(_all_required_col_names, colname_to_value_range, conjuncts, + tuple_descriptor, row_descriptor, colname_to_slot_id, + not_single_slot_filter_conjuncts, + slot_id_to_filter_conjuncts, table_info_node_ptr); } Status IcebergParquetReader ::_read_position_delete_file(const TFileRangeDesc* delete_range, @@ -445,9 +458,9 @@ Status IcebergParquetReader ::_read_position_delete_file(const TFileRangeDesc* d ParquetReader parquet_delete_reader( _profile, _params, *delete_range, READ_DELETE_FILE_BATCH_SIZE, const_cast(&_state->timezone_obj()), _io_ctx, _state); - RETURN_IF_ERROR(parquet_delete_reader.init_reader(delete_file_col_names, {}, nullptr, {}, - nullptr, nullptr, nullptr, nullptr, nullptr, - false)); + RETURN_IF_ERROR(parquet_delete_reader.init_reader( + delete_file_col_names, nullptr, {}, nullptr, nullptr, nullptr, nullptr, nullptr, + TableSchemaChangeHelper::ConstNode::get_instance(), false)); std::unordered_map> partition_columns; @@ -486,7 +499,6 @@ Status IcebergParquetReader ::_read_position_delete_file(const TFileRangeDesc* d Status IcebergOrcReader::init_reader( const std::vector& file_col_names, - const std::unordered_map& col_id_name_map, const std::unordered_map* colname_to_value_range, const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, const RowDescriptor* row_descriptor, @@ -495,16 +507,31 @@ Status IcebergOrcReader::init_reader( const std::unordered_map* slot_id_to_filter_conjuncts) { _file_format = Fileformat::ORC; auto* orc_reader = static_cast(_file_format_reader.get()); + const orc::Type* orc_type_ptr = nullptr; + RETURN_IF_ERROR(orc_reader->get_file_type(&orc_type_ptr)); + _all_required_col_names = file_col_names; - RETURN_IF_ERROR(TableSchemaChangeHelper::init_schema_info(file_col_names, col_id_name_map, - colname_to_value_range)); + if (!_params.__isset.history_schema_info || _params.history_schema_info.empty()) [[unlikely]] { + RETURN_IF_ERROR(BuildTableInfoUtil::by_orc_name(tuple_descriptor, orc_type_ptr, + table_info_node_ptr)); + } else { + bool exist_field_id = true; + // Iceberg will record the field id in the parquet file and find the column to read by matching it with the field id of the table (from fe). + RETURN_IF_ERROR(BuildTableInfoUtil::by_orc_field_id( + _params.history_schema_info.front().root_field, orc_type_ptr, ICEBERG_ORC_ATTRIBUTE, + table_info_node_ptr, exist_field_id)); + if (!exist_field_id) { + // For early iceberg version, field id may not be available, so name matching is used here. + RETURN_IF_ERROR(BuildTableInfoUtil::by_orc_name(tuple_descriptor, orc_type_ptr, + table_info_node_ptr)); + } + } - orc_reader->set_table_col_to_file_col(_table_col_to_file_col); RETURN_IF_ERROR(init_row_filters()); - return orc_reader->init_reader(&_all_required_col_names, _not_in_file_col_names, - &_new_colname_to_value_range, conjuncts, false, tuple_descriptor, - row_descriptor, not_single_slot_filter_conjuncts, - slot_id_to_filter_conjuncts); + return orc_reader->init_reader(&_all_required_col_names, colname_to_value_range, conjuncts, + false, tuple_descriptor, row_descriptor, + not_single_slot_filter_conjuncts, slot_id_to_filter_conjuncts, + table_info_node_ptr); } Status IcebergOrcReader::_read_position_delete_file(const TFileRangeDesc* delete_range, @@ -512,9 +539,8 @@ Status IcebergOrcReader::_read_position_delete_file(const TFileRangeDesc* delete OrcReader orc_delete_reader(_profile, _state, _params, *delete_range, READ_DELETE_FILE_BATCH_SIZE, _state->timezone(), _io_ctx); std::unordered_map colname_to_value_range; - RETURN_IF_ERROR(orc_delete_reader.init_reader(&delete_file_col_names, {}, - &colname_to_value_range, {}, false, {}, {}, - nullptr, nullptr)); + RETURN_IF_ERROR(orc_delete_reader.init_reader(&delete_file_col_names, &colname_to_value_range, + {}, false, {}, {}, nullptr, nullptr)); std::unordered_map> partition_columns; @@ -535,41 +561,5 @@ Status IcebergOrcReader::_read_position_delete_file(const TFileRangeDesc* delete return Status::OK(); } -// To support schema evolution, Iceberg write the column id to column name map to parquet file key_value_metadata. -Status IcebergParquetReader::get_file_col_id_to_name( - bool& exist_schema, std::map& file_col_id_to_name) { - auto* parquet_reader = static_cast(_file_format_reader.get()); - RETURN_IF_ERROR(parquet_reader->init_schema_reader()); - FieldDescriptor field_desc = parquet_reader->get_file_metadata_schema(); - - if (field_desc.has_parquet_field_id()) { - file_col_id_to_name = field_desc.get_field_id_name_map(); - } else { - //For early iceberg version, it doesn't write any schema information to Parquet file. - exist_schema = false; - } - - return Status::OK(); -} - -//To support schema evolution, Iceberg write the column id to orc file attribute. -Status IcebergOrcReader::get_file_col_id_to_name( - bool& exist_schema, std::map& file_col_id_to_name) { - auto* orc_reader = static_cast(_file_format_reader.get()); - - std::vector col_names; - std::vector col_ids; - RETURN_IF_ERROR(orc_reader->init_schema_reader()); - RETURN_IF_ERROR(orc_reader->get_schema_col_name_attribute( - &col_names, &col_ids, ICEBERG_ORC_ATTRIBUTE, &exist_schema)); - if (!exist_schema) { - return Status::OK(); - } - for (auto i = 0; i < col_names.size(); i++) { - file_col_id_to_name.emplace(col_ids[i], std::move(col_names[i])); - } - return Status::OK(); -} - #include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/table/iceberg_reader.h b/be/src/vec/exec/format/table/iceberg_reader.h index 4292af6a1d58b8..67a1c1cc66b5b1 100644 --- a/be/src/vec/exec/format/table/iceberg_reader.h +++ b/be/src/vec/exec/format/table/iceberg_reader.h @@ -133,6 +133,7 @@ class IcebergTableReader : public TableFormatReader, public TableSchemaChangeHel std::vector _iceberg_delete_rows; std::vector _expand_col_names; std::vector _expand_columns; + std::vector _all_required_col_names; Fileformat _file_format = Fileformat::NONE; @@ -168,7 +169,6 @@ class IcebergParquetReader final : public IcebergTableReader { kv_cache, io_ctx) {} Status init_reader( const std::vector& file_col_names, - const std::unordered_map& col_id_name_map, const std::unordered_map* colname_to_value_range, const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, const RowDescriptor* row_descriptor, @@ -184,9 +184,6 @@ class IcebergParquetReader final : public IcebergTableReader { parquet_reader->set_delete_rows(&_iceberg_delete_rows); } - Status get_file_col_id_to_name(bool& exist_schema, - std::map& file_col_id_to_name) final; - protected: std::unique_ptr _create_equality_reader( const TFileRangeDesc& delete_desc) final { @@ -215,7 +212,6 @@ class IcebergOrcReader final : public IcebergTableReader { Status init_reader( const std::vector& file_col_names, - const std::unordered_map& col_id_name_map, const std::unordered_map* colname_to_value_range, const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, const RowDescriptor* row_descriptor, @@ -223,9 +219,6 @@ class IcebergOrcReader final : public IcebergTableReader { const VExprContextSPtrs* not_single_slot_filter_conjuncts, const std::unordered_map* slot_id_to_filter_conjuncts); - Status get_file_col_id_to_name(bool& exist_schema, - std::map& file_col_id_to_name) final; - protected: std::unique_ptr _create_equality_reader( const TFileRangeDesc& delete_desc) override { @@ -234,7 +227,7 @@ class IcebergOrcReader final : public IcebergTableReader { } private: - const std::string ICEBERG_ORC_ATTRIBUTE = "iceberg.id"; + static const std::string ICEBERG_ORC_ATTRIBUTE; }; } // namespace vectorized diff --git a/be/src/vec/exec/format/table/paimon_reader.cpp b/be/src/vec/exec/format/table/paimon_reader.cpp index c7ef9b552b7db9..5a84a22863c32d 100644 --- a/be/src/vec/exec/format/table/paimon_reader.cpp +++ b/be/src/vec/exec/format/table/paimon_reader.cpp @@ -38,22 +38,6 @@ PaimonReader::PaimonReader(std::unique_ptr file_format_reader, ADD_CHILD_TIMER(_profile, "DeleteFileReadTime", paimon_profile); } -Status PaimonReader::get_file_col_id_to_name(bool& exist_schema, - std::map& file_col_id_to_name) { - if (!_params.__isset.history_schema_info) [[unlikely]] { - return Status::RuntimeError("miss paimon schema info."); - } - - if (!_params.history_schema_info.contains(_range.table_format_params.paimon_params.schema_id)) - [[unlikely]] { - return Status::InternalError("miss paimon schema info."); - } - - file_col_id_to_name = - _params.history_schema_info.at(_range.table_format_params.paimon_params.schema_id); - return Status::OK(); -} - Status PaimonReader::init_row_filters() { const auto& table_desc = _range.table_format_params.paimon_params; if (!table_desc.__isset.deletion_file) { @@ -123,9 +107,7 @@ Status PaimonReader::init_row_filters() { } Status PaimonReader::get_next_block_inner(Block* block, size_t* read_rows, bool* eof) { - RETURN_IF_ERROR(TableSchemaChangeHelper::get_next_block_before(block)); RETURN_IF_ERROR(_file_format_reader->get_next_block(block, read_rows, eof)); - RETURN_IF_ERROR(TableSchemaChangeHelper::get_next_block_after(block)); return Status::OK(); } #include "common/compile_check_end.h" diff --git a/be/src/vec/exec/format/table/paimon_reader.h b/be/src/vec/exec/format/table/paimon_reader.h index 4133deed734ff5..eb6d909bac5c21 100644 --- a/be/src/vec/exec/format/table/paimon_reader.h +++ b/be/src/vec/exec/format/table/paimon_reader.h @@ -38,9 +38,6 @@ class PaimonReader : public TableFormatReader, public TableSchemaChangeHelper { Status get_next_block_inner(Block* block, size_t* read_rows, bool* eof) final; - Status get_file_col_id_to_name(bool& exist_schema, - std::map& file_col_id_to_name) final; - protected: struct PaimonProfile { RuntimeProfile::Counter* num_delete_rows; @@ -68,22 +65,23 @@ class PaimonOrcReader final : public PaimonReader { Status init_reader( const std::vector& read_table_col_names, - const std::unordered_map& table_col_id_table_name_map, const std::unordered_map* table_col_name_to_value_range, const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, const RowDescriptor* row_descriptor, const VExprContextSPtrs* not_single_slot_filter_conjuncts, const std::unordered_map* slot_id_to_filter_conjuncts) { - RETURN_IF_ERROR(TableSchemaChangeHelper::init_schema_info( - read_table_col_names, table_col_id_table_name_map, table_col_name_to_value_range)); - auto* orc_reader = static_cast(_file_format_reader.get()); - orc_reader->set_table_col_to_file_col(_table_col_to_file_col); - return orc_reader->init_reader( - &_all_required_col_names, _not_in_file_col_names, &_new_colname_to_value_range, - conjuncts, false, tuple_descriptor, row_descriptor, - not_single_slot_filter_conjuncts, slot_id_to_filter_conjuncts); + const orc::Type* orc_type_ptr = nullptr; + RETURN_IF_ERROR(orc_reader->get_file_type(&orc_type_ptr)); + RETURN_IF_ERROR(gen_table_info_node_by_field_id( + _params, _range.table_format_params.paimon_params.schema_id, tuple_descriptor, + orc_type_ptr)); + + return orc_reader->init_reader(&read_table_col_names, table_col_name_to_value_range, + conjuncts, false, tuple_descriptor, row_descriptor, + not_single_slot_filter_conjuncts, + slot_id_to_filter_conjuncts, table_info_node_ptr); } }; @@ -103,7 +101,6 @@ class PaimonParquetReader final : public PaimonReader { Status init_reader( const std::vector& read_table_col_names, - const std::unordered_map& table_col_id_table_name_map, const std::unordered_map* table_col_name_to_value_range, const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, @@ -111,15 +108,20 @@ class PaimonParquetReader final : public PaimonReader { const std::unordered_map* colname_to_slot_id, const VExprContextSPtrs* not_single_slot_filter_conjuncts, const std::unordered_map* slot_id_to_filter_conjuncts) { - RETURN_IF_ERROR(TableSchemaChangeHelper::init_schema_info( - read_table_col_names, table_col_id_table_name_map, table_col_name_to_value_range)); auto* parquet_reader = static_cast(_file_format_reader.get()); - parquet_reader->set_table_to_file_col_map(_table_col_to_file_col); - return parquet_reader->init_reader( - _all_required_col_names, _not_in_file_col_names, &_new_colname_to_value_range, - conjuncts, tuple_descriptor, row_descriptor, colname_to_slot_id, - not_single_slot_filter_conjuncts, slot_id_to_filter_conjuncts); + const FieldDescriptor* field_desc = nullptr; + RETURN_IF_ERROR(parquet_reader->get_file_metadata_schema(&field_desc)); + DCHECK(field_desc != nullptr); + + RETURN_IF_ERROR(gen_table_info_node_by_field_id( + _params, _range.table_format_params.paimon_params.schema_id, tuple_descriptor, + *field_desc)); + + return parquet_reader->init_reader(read_table_col_names, table_col_name_to_value_range, + conjuncts, tuple_descriptor, row_descriptor, + colname_to_slot_id, not_single_slot_filter_conjuncts, + slot_id_to_filter_conjuncts, table_info_node_ptr); } }; #include "common/compile_check_end.h" diff --git a/be/src/vec/exec/format/table/table_format_reader.cpp b/be/src/vec/exec/format/table/table_format_reader.cpp index 86f0dea38e4ff7..3bb2957981bcaf 100644 --- a/be/src/vec/exec/format/table/table_format_reader.cpp +++ b/be/src/vec/exec/format/table/table_format_reader.cpp @@ -21,113 +21,638 @@ #include #include "common/status.h" +#include "gen_cpp/ExternalTableSchema_types.h" +#include "util/string_util.h" #include "vec/core/block.h" +#include "vec/data_types/data_type_array.h" +#include "vec/data_types/data_type_map.h" +#include "vec/data_types/data_type_struct.h" #include "vec/exec/format/generic_reader.h" namespace doris::vectorized { #include "common/compile_check_begin.h" +const Status TableSchemaChangeHelper::BuildTableInfoUtil::SCHEMA_ERROR = Status::NotSupported( + "In the parquet/orc reader, it is not possible to read scenarios where the complex column " + "types" + "of the table and the file are inconsistent."); -Status TableSchemaChangeHelper::init_schema_info( - const std::vector& read_table_col_names, - const std::unordered_map& table_id_to_name, - const std::unordered_map* - table_col_name_to_value_range) { - bool exist_schema = true; - std::map file_id_to_name; - RETURN_IF_ERROR(get_file_col_id_to_name(exist_schema, file_id_to_name)); - if (!exist_schema) { - file_id_to_name.clear(); - for (const auto& [table_col_id, table_col_name] : table_id_to_name) { - file_id_to_name.emplace(table_col_id, table_col_name); - } - } - - /** This is to compare the table schema from FE (table_id_to_name) with - * the current file schema (file_id_to_name) , generate two maps for future use: - * 1. table column name to file column name. - * 2. file column name to table column name. - * For example, file has a column 'col1', - * after this file was written, iceberg changed the column name to 'col1_new'. - * The two maps would contain: - * 1. col1_new -> col1 - * 2. col1 -> col1_new - */ - for (const auto& [file_col_id, file_col_name] : file_id_to_name) { - if (table_id_to_name.find(file_col_id) == table_id_to_name.end()) { - continue; - } - - auto& table_col_name = table_id_to_name.at(file_col_id); - _table_col_to_file_col.emplace(table_col_name, file_col_name); - _file_col_to_table_col.emplace(file_col_name, table_col_name); - if (table_col_name != file_col_name) { - _has_schema_change = true; - } - } - - /** Generate _all_required_col_names and _not_in_file_col_names. - * - * _all_required_col_names is all the columns required by user sql. - * If the column name has been modified after the data file was written, - * put the old name in data file to _all_required_col_names. - * - * _not_in_file_col_names is all the columns required by user sql but not in the data file. - * e.g. New columns added after this data file was written. - * The columns added with names used by old dropped columns should consider as a missing column, - * which should be in _not_in_file_col_names. - */ - _all_required_col_names.clear(); - _not_in_file_col_names.clear(); - for (auto table_col_name : read_table_col_names) { - auto iter = _table_col_to_file_col.find(table_col_name); - if (iter == _table_col_to_file_col.end()) { - _all_required_col_names.emplace_back(table_col_name); - _not_in_file_col_names.emplace_back(table_col_name); +Status TableSchemaChangeHelper::BuildTableInfoUtil::by_parquet_name( + const TupleDescriptor* table_tuple_descriptor, const FieldDescriptor& parquet_field_desc, + std::shared_ptr& node) { + auto struct_node = std::make_shared(); + auto parquet_fields_schema = parquet_field_desc.get_fields_schema(); + std::map file_column_name_idx_map; + for (size_t idx = 0; idx < parquet_fields_schema.size(); idx++) { + file_column_name_idx_map.emplace(to_lower(parquet_fields_schema[idx].name), idx); + } + + for (const auto& slot : table_tuple_descriptor->slots()) { + const auto& table_column_name = slot->col_name(); + + if (file_column_name_idx_map.contains(table_column_name)) { + auto file_column_idx = file_column_name_idx_map[table_column_name]; + std::shared_ptr field_node = nullptr; + RETURN_IF_ERROR(by_parquet_name(slot->get_data_type_ptr(), + parquet_fields_schema[file_column_idx], field_node)); + + struct_node->add_children(table_column_name, + parquet_fields_schema[file_column_idx].name, field_node); } else { - _all_required_col_names.emplace_back(iter->second); + struct_node->add_not_exist_children(table_column_name); } } - /** Generate _new_colname_to_value_range, by replacing the column name in - * _colname_to_value_range with column name in data file. - */ - for (auto& it : *table_col_name_to_value_range) { - auto iter = _table_col_to_file_col.find(it.first); - if (iter == _table_col_to_file_col.end()) { - _new_colname_to_value_range.emplace(it.first, it.second); + node = struct_node; + return Status::OK(); +}; + +Status TableSchemaChangeHelper::BuildTableInfoUtil::by_parquet_name( + const DataTypePtr& table_data_type, const FieldSchema& file_field, + std::shared_ptr& node) { + switch (remove_nullable(table_data_type)->get_type_id()) { + case TypeIndex::Map: { + if (file_field.type.type != TYPE_MAP) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(file_field.children.size() == 1)); + MOCK_REMOVE(DCHECK(file_field.children[0].children.size() == 2)); + std::shared_ptr key_node = nullptr; + + { + const auto& key_type = assert_cast( + assert_cast(remove_nullable(table_data_type).get()) + ->get_key_type()); + + RETURN_IF_ERROR( + by_parquet_name(key_type, file_field.children[0].children[0], key_node)); + } + + std::shared_ptr value_node = nullptr; + { + const auto& value_type = assert_cast( + assert_cast(remove_nullable(table_data_type).get()) + ->get_value_type()); + + RETURN_IF_ERROR( + by_parquet_name(value_type, file_field.children[0].children[1], value_node)); + } + node = std::make_shared(key_node, value_node); + break; + } + case TypeIndex::Array: { + if (file_field.type.type != TYPE_ARRAY) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(file_field.children.size() == 1)); + + std::shared_ptr element_node = nullptr; + const auto& element_type = assert_cast( + assert_cast(remove_nullable(table_data_type).get()) + ->get_nested_type()); + + RETURN_IF_ERROR(by_parquet_name(element_type, file_field.children[0], element_node)); + + node = std::make_shared(element_node); + break; + } + case TypeIndex::Struct: { + if (file_field.type.type != TYPE_STRUCT) [[unlikely]] { + return SCHEMA_ERROR; + } + + auto struct_node = std::make_shared(); + + const auto struct_data_type = + assert_cast(remove_nullable(table_data_type).get()); + + std::map parquet_field_names; + for (size_t idx = 0; idx < file_field.children.size(); idx++) { + parquet_field_names.emplace(to_lower(file_field.children[idx].name), idx); + } + for (size_t idx = 0; idx < struct_data_type->get_elements().size(); idx++) { + const auto& doris_field_name = struct_data_type->get_element_name(idx); + + if (parquet_field_names.contains(doris_field_name)) { + auto parquet_field_idx = parquet_field_names[doris_field_name]; + std::shared_ptr field_node = nullptr; + + RETURN_IF_ERROR(by_parquet_name(struct_data_type->get_element(idx), + file_field.children[parquet_field_idx], + field_node)); + struct_node->add_children(doris_field_name, + file_field.children[parquet_field_idx].name, field_node); + } else { + struct_node->add_not_exist_children(doris_field_name); + } + } + node = struct_node; + break; + } + default: { + node = std::make_shared(); + break; + } + } + + return Status::OK(); +} + +Status TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_name( + const TupleDescriptor* table_tuple_descriptor, const orc::Type* orc_type_ptr, + std::shared_ptr& node) { + auto struct_node = std::make_shared(); + + std::map file_column_name_idx_map; + for (uint64_t idx = 0; idx < orc_type_ptr->getSubtypeCount(); idx++) { + // to_lower for match table column name. + file_column_name_idx_map.emplace(to_lower(orc_type_ptr->getFieldName(idx)), idx); + } + + for (const auto& slot : table_tuple_descriptor->slots()) { + const auto& table_column_name = slot->col_name(); + if (file_column_name_idx_map.contains(table_column_name)) { + auto file_column_idx = file_column_name_idx_map[table_column_name]; + std::shared_ptr field_node = nullptr; + RETURN_IF_ERROR(by_orc_name(slot->get_data_type_ptr(), + orc_type_ptr->getSubtype(file_column_idx), field_node)); + struct_node->add_children(table_column_name, + orc_type_ptr->getFieldName(file_column_idx), field_node); } else { - _new_colname_to_value_range.emplace(iter->second, it.second); + struct_node->add_not_exist_children(table_column_name); } } + node = struct_node; return Status::OK(); } -Status TableSchemaChangeHelper::get_next_block_before(Block* block) const { - if (_has_schema_change) { - for (int i = 0; i < block->columns(); i++) { - ColumnWithTypeAndName& col = block->get_by_position(i); - auto iter = _table_col_to_file_col.find(col.name); - if (iter != _table_col_to_file_col.end()) { - col.name = iter->second; +Status TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_name( + const DataTypePtr& table_data_type, const orc::Type* orc_root, + std::shared_ptr& node) { + switch (remove_nullable(table_data_type)->get_type_id()) { + case TypeIndex::Map: { + if (orc_root->getKind() != orc::TypeKind::MAP) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(orc_root->getSubtypeCount() == 2)); + + std::shared_ptr key_node = nullptr; + const auto& key_type = assert_cast( + assert_cast(remove_nullable(table_data_type).get()) + ->get_key_type()); + RETURN_IF_ERROR(by_orc_name(key_type, orc_root->getSubtype(0), key_node)); + + std::shared_ptr value_node = nullptr; + const auto& value_type = assert_cast( + assert_cast(remove_nullable(table_data_type).get()) + ->get_value_type()); + RETURN_IF_ERROR(by_orc_name(value_type, orc_root->getSubtype(1), value_node)); + node = std::make_shared(key_node, value_node); + + break; + } + case TypeIndex::Array: { + if (orc_root->getKind() != orc::TypeKind::LIST) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(orc_root->getSubtypeCount() == 1)); + + std::shared_ptr element_node = nullptr; + const auto& element_type = assert_cast( + assert_cast(remove_nullable(table_data_type).get()) + ->get_nested_type()); + + RETURN_IF_ERROR(by_orc_name(element_type, orc_root->getSubtype(0), element_node)); + node = std::make_shared(element_node); + break; + } + case TypeIndex::Struct: { + if (orc_root->getKind() != orc::TypeKind::STRUCT) [[unlikely]] { + return SCHEMA_ERROR; + } + auto struct_node = std::make_shared(); + + const auto struct_data_type = + assert_cast(remove_nullable(table_data_type).get()); + std::map orc_field_names; + for (uint64_t idx = 0; idx < orc_root->getSubtypeCount(); idx++) { + orc_field_names.emplace(to_lower(orc_root->getFieldName(idx)), idx); + } + + for (size_t idx = 0; idx < struct_data_type->get_elements().size(); idx++) { + const auto& doris_field_name = struct_data_type->get_element_name(idx); + + if (orc_field_names.contains(doris_field_name)) { + std::shared_ptr field_node = nullptr; + + auto orc_field_idx = orc_field_names[doris_field_name]; + RETURN_IF_ERROR(by_orc_name(struct_data_type->get_element(idx), + orc_root->getSubtype(orc_field_idx), field_node)); + struct_node->add_children(doris_field_name, orc_root->getFieldName(orc_field_idx), + field_node); + } else { + struct_node->add_not_exist_children(doris_field_name); } } - block->initialize_index_by_name(); + node = struct_node; + break; + } + default: { + node = std::make_shared(); + break; + } + } + return Status::OK(); +} + +Status TableSchemaChangeHelper::BuildTableInfoUtil::by_table_field_id( + const schema::external::TField table_schema, const schema::external::TField file_schema, + std::shared_ptr& node) { + switch (table_schema.type.type) { + case TPrimitiveType::MAP: { + if (file_schema.type.type != TPrimitiveType::MAP) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(table_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.__isset.map_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.__isset.key_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.__isset.value_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.key_field.field_ptr != nullptr)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.value_field.field_ptr != nullptr)); + + MOCK_REMOVE(DCHECK(file_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(file_schema.nestedField.__isset.map_field)); + MOCK_REMOVE(DCHECK(file_schema.nestedField.map_field.__isset.key_field)); + MOCK_REMOVE(DCHECK(file_schema.nestedField.map_field.__isset.value_field)); + MOCK_REMOVE(DCHECK(file_schema.nestedField.map_field.key_field.field_ptr != nullptr)); + MOCK_REMOVE(DCHECK(file_schema.nestedField.map_field.value_field.field_ptr != nullptr)); + + std::shared_ptr key_node = nullptr; + RETURN_IF_ERROR(by_table_field_id(*table_schema.nestedField.map_field.key_field.field_ptr, + *file_schema.nestedField.map_field.key_field.field_ptr, + key_node)); + + std::shared_ptr value_node = nullptr; + RETURN_IF_ERROR(by_table_field_id(*table_schema.nestedField.map_field.value_field.field_ptr, + *file_schema.nestedField.map_field.value_field.field_ptr, + value_node)); + + node = std::make_shared(key_node, value_node); + break; + } + case TPrimitiveType::ARRAY: { + if (file_schema.type.type != TPrimitiveType::ARRAY) [[unlikely]] { + return SCHEMA_ERROR; + } + + MOCK_REMOVE(DCHECK(table_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.__isset.array_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.array_field.__isset.item_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.array_field.item_field.field_ptr != nullptr)); + + MOCK_REMOVE(DCHECK(file_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(file_schema.nestedField.__isset.array_field)); + MOCK_REMOVE(DCHECK(file_schema.nestedField.array_field.__isset.item_field)); + MOCK_REMOVE(DCHECK(file_schema.nestedField.array_field.item_field.field_ptr != nullptr)); + + std::shared_ptr item_node = nullptr; + RETURN_IF_ERROR(by_table_field_id( + *table_schema.nestedField.array_field.item_field.field_ptr, + *file_schema.nestedField.array_field.item_field.field_ptr, item_node)); + + node = std::make_shared(item_node); + break; + } + case TPrimitiveType::STRUCT: { + if (file_schema.type.type != TPrimitiveType::STRUCT) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(table_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.__isset.struct_field)); + + MOCK_REMOVE(DCHECK(file_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(file_schema.nestedField.__isset.struct_field)); + + RETURN_IF_ERROR(by_table_field_id(table_schema.nestedField.struct_field, + file_schema.nestedField.struct_field, node)); + break; + } + default: { + node = std::make_shared(); + break; + } + } + + return Status::OK(); +} + +Status TableSchemaChangeHelper::BuildTableInfoUtil::by_table_field_id( + const schema::external::TStructField& table_schema, + const schema::external::TStructField& file_schema, + std::shared_ptr& node) { + std::map file_field_id_to_idx; + for (size_t idx = 0; idx < file_schema.fields.size(); ++idx) { + file_field_id_to_idx.emplace(file_schema.fields[idx].field_ptr->id, idx); + } + auto struct_node = std::make_shared(); + + for (const auto& table_field : table_schema.fields) { + const auto& table_column_name = table_field.field_ptr->name; + + if (file_field_id_to_idx.contains(table_field.field_ptr->id)) { + const auto& file_field = + file_schema.fields.at(file_field_id_to_idx[table_field.field_ptr->id]); + + std::shared_ptr field_node = nullptr; + RETURN_IF_ERROR( + by_table_field_id(*table_field.field_ptr, *file_field.field_ptr, field_node)); + + struct_node->add_children(table_column_name, file_field.field_ptr->name, field_node); + } else { + struct_node->add_not_exist_children(table_column_name); + } } + node = std::move(struct_node); return Status::OK(); } -Status TableSchemaChangeHelper::get_next_block_after(Block* block) const { - if (_has_schema_change) { - for (int i = 0; i < block->columns(); i++) { - ColumnWithTypeAndName& col = block->get_by_position(i); - auto iter = _file_col_to_table_col.find(col.name); - if (iter != _file_col_to_table_col.end()) { - col.name = iter->second; +Status TableSchemaChangeHelper::BuildTableInfoUtil::by_parquet_field_id( + const schema::external::TStructField& table_schema, + const FieldDescriptor& parquet_field_desc, + std::shared_ptr& node, bool& exist_field_id) { + auto struct_node = std::make_shared(); + auto parquet_fields_schema = parquet_field_desc.get_fields_schema(); + std::map file_column_id_idx_map; + for (size_t idx = 0; idx < parquet_fields_schema.size(); idx++) { + if (parquet_fields_schema[idx].field_id == -1) { + exist_field_id = false; + return Status::OK(); + } else { + file_column_id_idx_map.emplace(parquet_fields_schema[idx].field_id, idx); + } + } + + for (const auto& table_field : table_schema.fields) { + const auto& table_column_name = table_field.field_ptr->name; + + if (file_column_id_idx_map.contains(table_field.field_ptr->id)) { + auto file_column_idx = file_column_id_idx_map[table_field.field_ptr->id]; + std::shared_ptr field_node = nullptr; + RETURN_IF_ERROR(by_parquet_field_id(*table_field.field_ptr, + parquet_fields_schema[file_column_idx], field_node, + exist_field_id)); + struct_node->add_children(table_column_name, + parquet_fields_schema[file_column_idx].name, field_node); + } else { + struct_node->add_not_exist_children(table_column_name); + } + } + + node = struct_node; + return Status::OK(); +} + +Status TableSchemaChangeHelper::BuildTableInfoUtil::by_parquet_field_id( + const schema::external::TField& table_schema, const FieldSchema& parquet_field, + std::shared_ptr& node, bool& exist_field_id) { + switch (table_schema.type.type) { + case TPrimitiveType::MAP: { + if (parquet_field.type.type != TYPE_MAP) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(table_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.__isset.map_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.__isset.key_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.__isset.value_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.key_field.field_ptr != nullptr)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.value_field.field_ptr != nullptr)); + + MOCK_REMOVE(DCHECK(parquet_field.children.size() == 1)); + MOCK_REMOVE(DCHECK(parquet_field.children[0].children.size() == 2)); + + std::shared_ptr key_node = nullptr; + std::shared_ptr value_node = nullptr; + + RETURN_IF_ERROR(by_parquet_field_id(*table_schema.nestedField.map_field.key_field.field_ptr, + parquet_field.children[0].children[0], key_node, + exist_field_id)); + + RETURN_IF_ERROR(by_parquet_field_id( + *table_schema.nestedField.map_field.value_field.field_ptr, + parquet_field.children[0].children[1], value_node, exist_field_id)); + + node = std::make_shared(key_node, value_node); + break; + } + case TPrimitiveType::ARRAY: { + if (parquet_field.type.type != TYPE_ARRAY) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(table_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.__isset.array_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.array_field.__isset.item_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.array_field.item_field.field_ptr != nullptr)); + + MOCK_REMOVE(DCHECK(parquet_field.children.size() == 1)); + + std::shared_ptr element_node = nullptr; + RETURN_IF_ERROR( + by_parquet_field_id(*table_schema.nestedField.array_field.item_field.field_ptr, + parquet_field.children[0], element_node, exist_field_id)); + + node = std::make_shared(element_node); + break; + } + case TPrimitiveType::STRUCT: { + if (parquet_field.type.type != TYPE_STRUCT) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(table_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.__isset.struct_field)); + + auto struct_node = std::make_shared(); + + std::map file_column_id_idx_map; + for (size_t idx = 0; idx < parquet_field.children.size(); idx++) { + if (parquet_field.children[idx].field_id == -1) { + exist_field_id = false; + return Status::OK(); + } else { + file_column_id_idx_map.emplace(parquet_field.children[idx].field_id, idx); + } + } + + for (const auto& table_field : table_schema.nestedField.struct_field.fields) { + const auto& table_column_name = table_field.field_ptr->name; + if (file_column_id_idx_map.contains(table_field.field_ptr->id)) { + const auto& file_field = parquet_field.children.at( + file_column_id_idx_map[table_field.field_ptr->id]); + std::shared_ptr field_node = nullptr; + RETURN_IF_ERROR(by_parquet_field_id(*table_field.field_ptr, file_field, field_node, + exist_field_id)); + struct_node->add_children(table_column_name, file_field.name, field_node); + } else { + struct_node->add_not_exist_children(table_column_name); } } - block->initialize_index_by_name(); + node = struct_node; + break; + } + default: { + node = std::make_shared(); + break; + } } return Status::OK(); } + +Status TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_field_id( + const schema::external::TStructField& table_schema, const orc::Type* orc_root, + const std::string& field_id_attribute_key, + std::shared_ptr& node, bool& exist_field_id) { + auto struct_node = std::make_shared(); + + std::map file_column_id_idx_map; + for (size_t idx = 0; idx < orc_root->getSubtypeCount(); idx++) { + if (orc_root->getSubtype(idx)->hasAttributeKey(field_id_attribute_key)) { + auto field_id = + std::stoi(orc_root->getSubtype(idx)->getAttributeValue(field_id_attribute_key)); + file_column_id_idx_map.emplace(field_id, idx); + } else { + exist_field_id = false; + return Status::OK(); + } + } + + for (const auto& table_field : table_schema.fields) { + const auto& table_column_name = table_field.field_ptr->name; + if (file_column_id_idx_map.contains(table_field.field_ptr->id)) { + auto file_field_idx = file_column_id_idx_map[table_field.field_ptr->id]; + const auto& file_field = orc_root->getSubtype(file_field_idx); + std::shared_ptr field_node = nullptr; + RETURN_IF_ERROR(by_orc_field_id(*table_field.field_ptr, file_field, + field_id_attribute_key, field_node, exist_field_id)); + struct_node->add_children(table_column_name, orc_root->getFieldName(file_field_idx), + field_node); + } else { + struct_node->add_not_exist_children(table_column_name); + } + } + node = struct_node; + return Status::OK(); +} + +Status TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_field_id( + const schema::external::TField& table_schema, const orc::Type* orc_root, + const std::string& field_id_attribute_key, + std::shared_ptr& node, bool& exist_field_id) { + switch (table_schema.type.type) { + case TPrimitiveType::MAP: { + if (orc_root->getKind() != orc::TypeKind::MAP) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(table_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.__isset.map_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.__isset.key_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.__isset.value_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.key_field.field_ptr != nullptr)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.map_field.value_field.field_ptr != nullptr)); + + MOCK_REMOVE(DCHECK(orc_root->getSubtypeCount() == 2)); + + std::shared_ptr key_node = nullptr; + std::shared_ptr value_node = nullptr; + + RETURN_IF_ERROR(by_orc_field_id(*table_schema.nestedField.map_field.key_field.field_ptr, + orc_root->getSubtype(0), field_id_attribute_key, key_node, + exist_field_id)); + + RETURN_IF_ERROR(by_orc_field_id(*table_schema.nestedField.map_field.value_field.field_ptr, + orc_root->getSubtype(1), field_id_attribute_key, value_node, + exist_field_id)); + + node = std::make_shared(key_node, value_node); + break; + } + case TPrimitiveType::ARRAY: { + if (orc_root->getKind() != orc::TypeKind::LIST) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(table_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.__isset.array_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.array_field.__isset.item_field)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.array_field.item_field.field_ptr != nullptr)); + + MOCK_REMOVE(DCHECK(orc_root->getSubtypeCount() == 1)); + + std::shared_ptr element_node = nullptr; + RETURN_IF_ERROR(by_orc_field_id(*table_schema.nestedField.array_field.item_field.field_ptr, + orc_root->getSubtype(0), field_id_attribute_key, + element_node, exist_field_id)); + + node = std::make_shared(element_node); + break; + } + case TPrimitiveType::STRUCT: { + if (orc_root->getKind() != orc::TypeKind::STRUCT) [[unlikely]] { + return SCHEMA_ERROR; + } + MOCK_REMOVE(DCHECK(table_schema.__isset.nestedField)); + MOCK_REMOVE(DCHECK(table_schema.nestedField.__isset.struct_field)); + RETURN_IF_ERROR(by_orc_field_id(table_schema.nestedField.struct_field, orc_root, + field_id_attribute_key, node, exist_field_id)); + + break; + } + default: { + node = std::make_shared(); + break; + } + } + + return Status::OK(); +} + +std::string TableSchemaChangeHelper::debug(const std::shared_ptr& root, size_t level) { + std::string ans; + + auto indent = [](size_t level) { return std::string(level * 2, ' '); }; + + std::string prefix = indent(level); + + if (std::dynamic_pointer_cast(root)) { + ans += prefix + "ScalarNode\n"; + } else if (auto struct_node = std::dynamic_pointer_cast(root)) { + ans += prefix + "StructNode\n"; + for (const auto& [table_col_name, value] : struct_node->get_childrens()) { + const auto& [child_node, file_col_name, exist] = value; + ans += indent(level + 1) + table_col_name; + if (exist) { + ans += " (file: " + file_col_name + ")"; + } else { + ans += " (not exists)"; + } + ans += "\n"; + if (child_node) { + ans += debug(child_node, level + 2); + } + } + } else if (auto array_node = std::dynamic_pointer_cast(root)) { + ans += prefix + "ArrayNode\n"; + ans += indent(level + 1) + "Element:\n"; + ans += debug(array_node->get_element_node(), level + 2); + } else if (auto map_node = std::dynamic_pointer_cast(root)) { + ans += prefix + "MapNode\n"; + ans += indent(level + 1) + "Key:\n"; + ans += debug(map_node->get_key_node(), level + 2); + ans += indent(level + 1) + "Value:\n"; + ans += debug(map_node->get_value_node(), level + 2); + } else if (std::dynamic_pointer_cast(root)) { + ans += prefix + "ConstNode\n"; + } else { + ans += prefix + "UnknownNodeType\n"; + } + + return ans; +} #include "common/compile_check_end.h" } // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/exec/format/table/table_format_reader.h b/be/src/vec/exec/format/table/table_format_reader.h index 201bbe27ec083b..fbba6a51033530 100644 --- a/be/src/vec/exec/format/table/table_format_reader.h +++ b/be/src/vec/exec/format/table/table_format_reader.h @@ -25,8 +25,13 @@ #include "exec/olap_common.h" #include "runtime/runtime_state.h" #include "util/runtime_profile.h" +#include "util/string_util.h" #include "vec/core/block.h" +#include "vec/data_types/data_type_array.h" +#include "vec/data_types/data_type_map.h" +#include "vec/data_types/data_type_struct.h" #include "vec/exec/format/generic_reader.h" +#include "vec/exec/format/parquet/schema_desc.h" namespace doris { class TFileRangeDesc; @@ -117,42 +122,263 @@ class TableFormatReader : public GenericReader { class TableSchemaChangeHelper { public: - /** Get the mapping from the unique ID of the column in the current file to the file column name. - * Iceberg/Hudi/Paimon usually maintains field IDs to support schema changes. If you cannot obtain this - * information (maybe the old version does not have this information), you need to set `exist_schema` = `false`. - */ - virtual Status get_file_col_id_to_name(bool& exist_schema, - std::map& file_col_id_to_name) = 0; + ~TableSchemaChangeHelper() = default; - virtual ~TableSchemaChangeHelper() = default; +public: + class Node { + public: + virtual ~Node() = default; + virtual std::shared_ptr get_children_node(std::string table_column_name) const { + throw std::logic_error("get_children_node should not be called on base TableInfoNode"); + }; + + virtual std::string children_file_column_name(std::string table_column_name) const { + throw std::logic_error( + "children_file_column_name should not be called on base TableInfoNode"); + } + + virtual bool children_column_exists(std::string table_column_name) const { + throw std::logic_error( + "children_column_exists should not be called on base TableInfoNode"); + } + + virtual std::shared_ptr get_element_node() const { + throw std::logic_error("get_element_node should not be called on base TableInfoNode"); + } + + virtual std::shared_ptr get_key_node() const { + throw std::logic_error("get_key_node should not be called on base TableInfoNode"); + } + virtual std::shared_ptr get_value_node() const { + throw std::logic_error("get_value_node should not be called on base TableInfoNode"); + } + + virtual void add_not_exist_children(std::string table_column_name) { + throw std::logic_error( + "add_not_exist_children should not be called on base TableInfoNode"); + }; + + virtual void add_children(std::string table_column_name, std::string file_column_name, + std::shared_ptr children_node) { + throw std::logic_error("add_children should not be called on base TableInfoNode"); + } + }; + + class ScalarNode : public Node {}; + + class StructNode : public Node { + using ChildrenType = std::tuple, std::string, bool>; + + // table column name -> { node, file_column_name, exists_in_file} + std::map children; + + public: + std::shared_ptr get_children_node(std::string table_column_name) const override { + DCHECK(children.contains(table_column_name)); + DCHECK(children_column_exists(table_column_name)); + return std::get<0>(children.at(table_column_name)); + } + + std::string children_file_column_name(std::string table_column_name) const override { + DCHECK(children.contains(table_column_name)); + DCHECK(children_column_exists(table_column_name)); + return std::get<1>(children.at(table_column_name)); + } + + bool children_column_exists(std::string table_column_name) const override { + DCHECK(children.contains(table_column_name)); + return std::get<2>(children.at(table_column_name)); + } + + void add_not_exist_children(std::string table_column_name) override { + children.emplace(table_column_name, std::make_tuple(nullptr, "", false)); + } + + void add_children(std::string table_column_name, std::string file_column_name, + std::shared_ptr children_node) override { + children.emplace(table_column_name, + std::make_tuple(children_node, file_column_name, true)); + } + + const std::map& get_childrens() const { return children; } + }; + + class ArrayNode : public Node { + std::shared_ptr _element_node; + + public: + ArrayNode(const std::shared_ptr& element_node) : _element_node(element_node) {} + + std::shared_ptr get_element_node() const override { return _element_node; } + }; + + class MapNode : public Node { + std::shared_ptr _key_node; + std::shared_ptr _value_node; + + public: + MapNode(const std::shared_ptr& key_node, const std::shared_ptr& value_node) + : _key_node(key_node), _value_node(value_node) {} + + std::shared_ptr get_key_node() const override { return _key_node; } + + std::shared_ptr get_value_node() const override { return _value_node; } + }; + + class ConstNode : public Node { + // If you can be sure that there has been no schema change between the table and the file, + // you can use constNode (of course, you need to pay attention to case sensitivity). + public: + std::shared_ptr get_children_node(std::string table_column_name) const override { + return get_instance(); + }; + + std::string children_file_column_name(std::string table_column_name) const override { + return table_column_name; + } + + bool children_column_exists(std::string table_column_name) const override { return true; } + + std::shared_ptr get_element_node() const override { return get_instance(); } + + std::shared_ptr get_key_node() const override { return get_instance(); } + + std::shared_ptr get_value_node() const override { return get_instance(); } + + static const std::shared_ptr& get_instance() { + static const std::shared_ptr instance = std::make_shared(); + return instance; + } + }; + + static std::string debug(const std::shared_ptr& root, size_t level = 0); protected: - /** table_id_to_name : table column unique id to table name map */ - Status init_schema_info(const std::vector& read_table_col_names, - const std::unordered_map& table_id_to_name, - const std::unordered_map* - table_col_name_to_value_range); - - /** To support schema evolution. We change the column name in block to - * make it match with the column name in file before reading data. and - * set the name back to table column name before return this block. - */ - Status get_next_block_before(Block* block) const; - - /** Set the name back to table column name before return this block.*/ - Status get_next_block_after(Block* block) const; - - // copy from _colname_to_value_range with new column name that is in parquet/orc file - std::unordered_map _new_colname_to_value_range; - // all the columns required by user sql. - std::vector _all_required_col_names; - // col names in table but not in parquet,orc file - std::vector _not_in_file_col_names; - bool _has_schema_change = false; - // file column name to table column name map - std::unordered_map _file_col_to_table_col; - // table column name to file column name map. - std::unordered_map _table_col_to_file_col; + // Whenever external components invoke the Parquet/ORC reader (e.g., init_reader, get_next_block, set_fill_columns), + // the parameters passed in are based on `table column names`. + // The table_info_node_ptr assists the Parquet/ORC reader in mapping these to the actual + // `file columns name` to be read and enables min/max filtering. + std::shared_ptr table_info_node_ptr = std::make_shared(); + +protected: + Status gen_table_info_node_by_field_id(const TFileScanRangeParams& params, + int64_t split_schema_id, + const TupleDescriptor* tuple_descriptor, + const FieldDescriptor& parquet_field_desc) { + if (!params.__isset.history_schema_info) [[unlikely]] { + RETURN_IF_ERROR(BuildTableInfoUtil::by_parquet_name( + tuple_descriptor, parquet_field_desc, table_info_node_ptr)); + return Status::OK(); + } + return gen_table_info_node_by_field_id(params, split_schema_id); + } + + Status gen_table_info_node_by_field_id(const TFileScanRangeParams& params, + int64_t split_schema_id, + const TupleDescriptor* tuple_descriptor, + const orc::Type* orc_type_ptr) { + if (!params.__isset.history_schema_info) [[unlikely]] { + RETURN_IF_ERROR(BuildTableInfoUtil::by_orc_name(tuple_descriptor, orc_type_ptr, + table_info_node_ptr)); + return Status::OK(); + } + return gen_table_info_node_by_field_id(params, split_schema_id); + } + +private: + // The filed id of both the table and the file come from the pass from fe. (params.history_schema_info) + Status gen_table_info_node_by_field_id(const TFileScanRangeParams& params, + int64_t split_schema_id) { + if (params.current_schema_id == split_schema_id) { + table_info_node_ptr = ConstNode::get_instance(); + return Status::OK(); + } + + int32_t table_schema_idx = -1; + int32_t file_schema_idx = -1; + //todo : Perhaps this process can be optimized by pre-generating a map + for (int32_t idx = 0; idx < params.history_schema_info.size(); idx++) { + if (params.history_schema_info[idx].schema_id == params.current_schema_id) { + table_schema_idx = idx; + } else if (params.history_schema_info[idx].schema_id == split_schema_id) { + file_schema_idx = idx; + } + } + + if (table_schema_idx == -1 || file_schema_idx == -1) [[unlikely]] { + return Status::InternalError( + "miss table/file schema info, table_schema_idx:{} file_schema_idx:{}", + table_schema_idx, file_schema_idx); + } + RETURN_IF_ERROR(BuildTableInfoUtil::by_table_field_id( + params.history_schema_info.at(table_schema_idx).root_field, + params.history_schema_info.at(file_schema_idx).root_field, table_info_node_ptr)); + return Status::OK(); + } + +public: + /* Schema change Util. Used to generate `std::shared_ptr node`. + Passed node to parquet/orc reader to find file columns based on table columns, + */ + struct BuildTableInfoUtil { + static const Status SCHEMA_ERROR; + + // todo : Maybe I can use templates to implement this functionality. + + // for hive parquet : The table column names passed from fe are lowercase, so use lowercase file column names to match table column names. + static Status by_parquet_name(const TupleDescriptor* table_tuple_descriptor, + const FieldDescriptor& parquet_field_desc, + std::shared_ptr& node); + + // for hive parquet + static Status by_parquet_name(const DataTypePtr& table_data_type, + const FieldSchema& file_field, + std::shared_ptr& node); + + // for hive orc: The table column names passed from fe are lowercase, so use lowercase file column names to match table column names. + static Status by_orc_name(const TupleDescriptor* table_tuple_descriptor, + const orc::Type* orc_type_ptr, + std::shared_ptr& node); + // for hive orc + static Status by_orc_name(const DataTypePtr& table_data_type, const orc::Type* orc_root, + std::shared_ptr& node); + + // for paimon hudi: Use the field id in the `table schema` and `history table schema` to match columns. + static Status by_table_field_id(const schema::external::TField table_schema, + const schema::external::TField file_schema, + std::shared_ptr& node); + + // for paimon hudi + static Status by_table_field_id(const schema::external::TStructField& table_schema, + const schema::external::TStructField& file_schema, + std::shared_ptr& node); + + //for iceberg parquet: Use the field id in the `table schema` and the parquet file to match columns. + static Status by_parquet_field_id(const schema::external::TStructField& table_schema, + const FieldDescriptor& parquet_field_desc, + std::shared_ptr& node, + bool& exist_field_id); + + // for iceberg parquet + static Status by_parquet_field_id(const schema::external::TField& table_schema, + const FieldSchema& parquet_field, + std::shared_ptr& node, + bool& exist_field_id); + + // for iceberg orc : Use the field id in the `table schema` and the orc file to match columns. + static Status by_orc_field_id(const schema::external::TStructField& table_schema, + const orc::Type* orc_root, + const std::string& field_id_attribute_key, + std::shared_ptr& node, + bool& exist_field_id); + + // for iceberg orc + static Status by_orc_field_id(const schema::external::TField& table_schema, + const orc::Type* orc_root, + const std::string& field_id_attribute_key, + std::shared_ptr& node, + bool& exist_field_id); + }; }; #include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/table/transactional_hive_reader.cpp b/be/src/vec/exec/format/table/transactional_hive_reader.cpp index 78fb62bef2a36b..79e611fe066c8a 100644 --- a/be/src/vec/exec/format/table/transactional_hive_reader.cpp +++ b/be/src/vec/exec/format/table/transactional_hive_reader.cpp @@ -20,6 +20,7 @@ #include "transactional_hive_common.h" #include "vec/data_types/data_type_factory.hpp" #include "vec/exec/format/orc/vorc_reader.h" +#include "vec/exec/format/table/table_format_reader.h" namespace doris { #include "common/compile_check_begin.h" @@ -60,9 +61,55 @@ Status TransactionalHiveReader::init_reader( _col_names.insert(_col_names.end(), column_names.begin(), column_names.end()); _col_names.insert(_col_names.end(), TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.begin(), TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.end()); + + // https://issues.apache.org/jira/browse/HIVE-15190 + const orc::Type* orc_type_ptr = nullptr; + RETURN_IF_ERROR(orc_reader->get_file_type(&orc_type_ptr)); + const auto& orc_type = *orc_type_ptr; + + for (auto idx = 0; idx < TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.size(); idx++) { + table_info_node_ptr->add_children(TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE[idx], + TransactionalHive::READ_ROW_COLUMN_NAMES[idx], + std::make_shared()); + } + + auto row_orc_type = orc_type.getSubtype(TransactionalHive::ROW_OFFSET); + // struct> + std::vector row_names; + std::map row_names_map; + for (uint64_t idx = 0; idx < row_orc_type->getSubtypeCount(); idx++) { + const auto& file_column_name = row_orc_type->getFieldName(idx); + row_names.emplace_back(file_column_name); + row_names_map.emplace(file_column_name, idx); + } + + // use name for match. + for (const auto& slot : tuple_descriptor->slots()) { + const auto& slot_name = slot->col_name(); + + if (std::count(TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.begin(), + TransactionalHive::READ_ROW_COLUMN_NAMES_LOWER_CASE.end(), slot_name) > 0) { + return Status::InternalError("xxxx"); + } + + if (row_names_map.contains(slot_name)) { + std::shared_ptr child_node = nullptr; + RETURN_IF_ERROR(BuildTableInfoUtil::by_orc_name( + slot->get_data_type_ptr(), row_orc_type->getSubtype(row_names_map[slot_name]), + child_node)); + auto file_column_name = fmt::format( + "{}.{}", TransactionalHive::ACID_COLUMN_NAMES[TransactionalHive::ROW_OFFSET], + slot_name); + table_info_node_ptr->add_children(slot_name, file_column_name, child_node); + + } else { + table_info_node_ptr->add_not_exist_children(slot_name); + } + } + Status status = orc_reader->init_reader( - &_col_names, {}, colname_to_value_range, conjuncts, true, tuple_descriptor, - row_descriptor, not_single_slot_filter_conjuncts, slot_id_to_filter_conjuncts); + &_col_names, colname_to_value_range, conjuncts, true, tuple_descriptor, row_descriptor, + not_single_slot_filter_conjuncts, slot_id_to_filter_conjuncts, table_info_node_ptr); return status; } @@ -118,9 +165,19 @@ Status TransactionalHiveReader::init_row_filters() { OrcReader delete_reader(_profile, _state, _params, delete_range, _MIN_BATCH_SIZE, _state->timezone(), _io_ctx, false); + auto acid_info_node = std::make_shared(); + for (auto idx = 0; idx < TransactionalHive::DELETE_ROW_COLUMN_NAMES_LOWER_CASE.size(); + idx++) { + auto const& table_column_name = + TransactionalHive::DELETE_ROW_COLUMN_NAMES_LOWER_CASE[idx]; + auto const& file_column_name = TransactionalHive::DELETE_ROW_COLUMN_NAMES[idx]; + acid_info_node->add_children(table_column_name, file_column_name, + std::make_shared()); + } + RETURN_IF_ERROR(delete_reader.init_reader( - &TransactionalHive::DELETE_ROW_COLUMN_NAMES_LOWER_CASE, {}, nullptr, {}, false, - nullptr, nullptr, nullptr, nullptr)); + &TransactionalHive::DELETE_ROW_COLUMN_NAMES_LOWER_CASE, nullptr, {}, false, nullptr, + nullptr, nullptr, nullptr, acid_info_node)); std::unordered_map> partition_columns; diff --git a/be/src/vec/exec/format/table/transactional_hive_reader.h b/be/src/vec/exec/format/table/transactional_hive_reader.h index 29cdde4dd6e0b0..60114bbb29c4f0 100644 --- a/be/src/vec/exec/format/table/transactional_hive_reader.h +++ b/be/src/vec/exec/format/table/transactional_hive_reader.h @@ -48,7 +48,7 @@ class GenericReader; class ShardedKVCache; class VExprContext; -class TransactionalHiveReader : public TableFormatReader { +class TransactionalHiveReader : public TableFormatReader, public TableSchemaChangeHelper { ENABLE_FACTORY_CREATOR(TransactionalHiveReader); public: diff --git a/be/src/vec/exec/scan/vfile_scanner.cpp b/be/src/vec/exec/scan/vfile_scanner.cpp index 18c904e97e9c4a..b4a90e4ce038b5 100644 --- a/be/src/vec/exec/scan/vfile_scanner.cpp +++ b/be/src/vec/exec/scan/vfile_scanner.cpp @@ -61,6 +61,7 @@ #include "vec/exec/format/json/new_json_reader.h" #include "vec/exec/format/orc/vorc_reader.h" #include "vec/exec/format/parquet/vparquet_reader.h" +#include "vec/exec/format/table/hive_reader.h" #include "vec/exec/format/table/hudi_jni_reader.h" #include "vec/exec/format/table/hudi_reader.h" #include "vec/exec/format/table/iceberg_reader.h" @@ -511,7 +512,7 @@ Status VFileScanner::_init_src_block(Block* block) { // _input_tuple_desc also contains columns from path for (auto& slot : _input_tuple_desc->slots()) { DataTypePtr data_type; - auto it = _name_to_col_type.find(slot->col_name()); + auto it = _slot_lower_name_to_col_type.find(slot->col_name()); if (slot->is_skip_bitmap_col()) { _skip_bitmap_col_idx = idx; } @@ -520,7 +521,7 @@ Status VFileScanner::_init_src_block(Block* block) { _sequence_map_col_uid = slot->col_unique_id(); } } - if (it == _name_to_col_type.end()) { + if (it == _slot_lower_name_to_col_type.end()) { // not exist in file, using type from _input_tuple_desc RETURN_IF_CATCH_EXCEPTION(data_type = DataTypeFactory::instance().create_data_type( slot->type(), slot->is_nullable())); @@ -555,7 +556,8 @@ Status VFileScanner::_cast_to_input_block(Block* block) { // cast primitive type(PT0) to primitive type(PT1) size_t idx = 0; for (auto& slot_desc : _input_tuple_desc->slots()) { - if (_name_to_col_type.find(slot_desc->col_name()) == _name_to_col_type.end()) { + if (_slot_lower_name_to_col_type.find(slot_desc->col_name()) == + _slot_lower_name_to_col_type.end()) { // skip columns which does not exist in file continue; } @@ -811,10 +813,10 @@ Status VFileScanner::_truncate_char_or_varchar_columns(Block* block) { } auto iter = _source_file_col_name_types.find(slot_desc->col_name()); if (iter != _source_file_col_name_types.end()) { - const TypeDescriptor* file_type_desc = + const TypeDescriptor& file_type_desc = _source_file_col_name_types[slot_desc->col_name()]; if ((type_desc.len > 0) && - (type_desc.len < file_type_desc->len || file_type_desc->len < 0)) { + (type_desc.len < file_type_desc.len || file_type_desc.len < 0)) { _truncate_char_or_varchar_column(block, idx, type_desc.len); } } else { @@ -984,10 +986,9 @@ Status VFileScanner::_get_next_reader() { _state, *_params, range, _kv_cache, _io_ctx.get()); init_status = iceberg_reader->init_reader( - _file_col_names, _col_id_name_map, _colname_to_value_range, - _push_down_conjuncts, _real_tuple_desc, _default_val_row_desc.get(), - _col_name_to_slot_id, &_not_single_slot_filter_conjuncts, - &_slot_id_to_filter_conjuncts); + _file_col_names, _colname_to_value_range, _push_down_conjuncts, + _real_tuple_desc, _default_val_row_desc.get(), _col_name_to_slot_id, + &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts); _cur_reader = std::move(iceberg_reader); } else if (range.__isset.table_format_params && range.table_format_params.table_format_type == "paimon") { @@ -995,10 +996,9 @@ Status VFileScanner::_get_next_reader() { PaimonParquetReader::create_unique(std::move(parquet_reader), _profile, _state, *_params, range, _io_ctx.get()); init_status = paimon_reader->init_reader( - _file_col_names, _col_id_name_map, _colname_to_value_range, - _push_down_conjuncts, _real_tuple_desc, _default_val_row_desc.get(), - _col_name_to_slot_id, &_not_single_slot_filter_conjuncts, - &_slot_id_to_filter_conjuncts); + _file_col_names, _colname_to_value_range, _push_down_conjuncts, + _real_tuple_desc, _default_val_row_desc.get(), _col_name_to_slot_id, + &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts); RETURN_IF_ERROR(paimon_reader->init_row_filters()); _cur_reader = std::move(paimon_reader); } else if (range.__isset.table_format_params && @@ -1007,27 +1007,66 @@ Status VFileScanner::_get_next_reader() { HudiParquetReader::create_unique(std::move(parquet_reader), _profile, _state, *_params, range, _io_ctx.get()); init_status = hudi_reader->init_reader( - _file_col_names, _col_id_name_map, _colname_to_value_range, - _push_down_conjuncts, _real_tuple_desc, _default_val_row_desc.get(), - _col_name_to_slot_id, &_not_single_slot_filter_conjuncts, - &_slot_id_to_filter_conjuncts); + _file_col_names, _colname_to_value_range, _push_down_conjuncts, + _real_tuple_desc, _default_val_row_desc.get(), _col_name_to_slot_id, + &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts); _cur_reader = std::move(hudi_reader); - } else { - bool hive_parquet_use_column_names = true; - - if (range.__isset.table_format_params && - range.table_format_params.table_format_type == "hive" && _state != nullptr) - [[likely]] { - hive_parquet_use_column_names = - _state->query_options().hive_parquet_use_column_names; + } else if (range.table_format_params.table_format_type == "hive") { + auto hive_reader = + HiveParquetReader::create_unique(std::move(parquet_reader), _profile, + _state, *_params, range, _io_ctx.get()); + init_status = hive_reader->init_reader( + _file_col_names, _colname_to_value_range, _push_down_conjuncts, + _real_tuple_desc, _default_val_row_desc.get(), _col_name_to_slot_id, + &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts); + _cur_reader = std::move(hive_reader); + } else if (range.table_format_params.table_format_type == "tvf") { + const FieldDescriptor* parquet_meta = nullptr; + RETURN_IF_ERROR(parquet_reader->get_file_metadata_schema(&parquet_meta)); + DCHECK(parquet_meta != nullptr); + + // TVF will first `get_parsed_schema` to obtain file information from BE, and FE will convert + // the column names to lowercase (because the query process is case-insensitive), + // so the lowercase file column names are used here to match the read columns. + std::shared_ptr tvf_info_node = nullptr; + RETURN_IF_ERROR(TableSchemaChangeHelper::BuildTableInfoUtil::by_parquet_name( + _real_tuple_desc, *parquet_meta, tvf_info_node)); + init_status = parquet_reader->init_reader( + _file_col_names, _colname_to_value_range, _push_down_conjuncts, + _real_tuple_desc, _default_val_row_desc.get(), _col_name_to_slot_id, + &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts, + tvf_info_node); + _cur_reader = std::move(parquet_reader); + } else if (_is_load) { + const FieldDescriptor* parquet_meta = nullptr; + RETURN_IF_ERROR(parquet_reader->get_file_metadata_schema(&parquet_meta)); + DCHECK(parquet_meta != nullptr); + + // Load is case-insensitive, so you to match the columns in the file. + std::map file_lower_name_to_native; + for (const auto& parquet_field : parquet_meta->get_fields_schema()) { + file_lower_name_to_native.emplace(doris::to_lower(parquet_field.name), + parquet_field.name); + } + auto load_info_node = std::make_shared(); + for (const auto slot : _real_tuple_desc->slots()) { + if (file_lower_name_to_native.contains(slot->col_name())) { + load_info_node->add_children( + slot->col_name(), file_lower_name_to_native[slot->col_name()], + TableSchemaChangeHelper::ConstNode::get_instance()); + // For Load, `file_scanner` will create block columns using the file type, + // there is no schema change when reading inside the struct, + // so use `TableSchemaChangeHelper::ConstNode`. + } else { + load_info_node->add_not_exist_children(slot->col_name()); + } } - std::vector place_holder; init_status = parquet_reader->init_reader( - _file_col_names, place_holder, _colname_to_value_range, - _push_down_conjuncts, _real_tuple_desc, _default_val_row_desc.get(), - _col_name_to_slot_id, &_not_single_slot_filter_conjuncts, - &_slot_id_to_filter_conjuncts, true, hive_parquet_use_column_names); + _file_col_names, _colname_to_value_range, _push_down_conjuncts, + _real_tuple_desc, _default_val_row_desc.get(), _col_name_to_slot_id, + &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts, + load_info_node); _cur_reader = std::move(parquet_reader); } need_to_get_parsed_schema = true; @@ -1060,10 +1099,9 @@ Status VFileScanner::_get_next_reader() { *_params, range, _kv_cache, _io_ctx.get()); init_status = iceberg_reader->init_reader( - _file_col_names, _col_id_name_map, _colname_to_value_range, - _push_down_conjuncts, _real_tuple_desc, _default_val_row_desc.get(), - _col_name_to_slot_id, &_not_single_slot_filter_conjuncts, - &_slot_id_to_filter_conjuncts); + _file_col_names, _colname_to_value_range, _push_down_conjuncts, + _real_tuple_desc, _default_val_row_desc.get(), _col_name_to_slot_id, + &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts); _cur_reader = std::move(iceberg_reader); } else if (range.__isset.table_format_params && range.table_format_params.table_format_type == "paimon") { @@ -1071,36 +1109,64 @@ Status VFileScanner::_get_next_reader() { std::move(orc_reader), _profile, _state, *_params, range, _io_ctx.get()); init_status = paimon_reader->init_reader( - _file_col_names, _col_id_name_map, _colname_to_value_range, - _push_down_conjuncts, _real_tuple_desc, _default_val_row_desc.get(), + _file_col_names, _colname_to_value_range, _push_down_conjuncts, + _real_tuple_desc, _default_val_row_desc.get(), &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts); RETURN_IF_ERROR(paimon_reader->init_row_filters()); _cur_reader = std::move(paimon_reader); } else if (range.__isset.table_format_params && - range.table_format_params.table_format_type == "hudi") { - std::unique_ptr hudi_reader = HudiOrcReader::create_unique( + range.table_format_params.table_format_type == "hive") { + std::unique_ptr hive_reader = HiveOrcReader::create_unique( std::move(orc_reader), _profile, _state, *_params, range, _io_ctx.get()); - init_status = hudi_reader->init_reader( - _file_col_names, _col_id_name_map, _colname_to_value_range, - _push_down_conjuncts, _real_tuple_desc, _default_val_row_desc.get(), + init_status = hive_reader->init_reader( + _file_col_names, _colname_to_value_range, _push_down_conjuncts, + _real_tuple_desc, _default_val_row_desc.get(), &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts); - _cur_reader = std::move(hudi_reader); - } else { - bool hive_orc_use_column_names = true; + _cur_reader = std::move(hive_reader); + } else if (range.__isset.table_format_params && + range.table_format_params.table_format_type == "tvf") { + const orc::Type* orc_type_ptr = nullptr; + RETURN_IF_ERROR(orc_reader->get_file_type(&orc_type_ptr)); + + std::shared_ptr tvf_info_node = nullptr; + RETURN_IF_ERROR(TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_name( + _real_tuple_desc, orc_type_ptr, tvf_info_node)); + init_status = orc_reader->init_reader(&_file_col_names, _colname_to_value_range, + _push_down_conjuncts, false, _real_tuple_desc, + _default_val_row_desc.get(), + &_not_single_slot_filter_conjuncts, + &_slot_id_to_filter_conjuncts, tvf_info_node); + _cur_reader = std::move(orc_reader); + } else if (_is_load) { + const orc::Type* orc_type_ptr = nullptr; + RETURN_IF_ERROR(orc_reader->get_file_type(&orc_type_ptr)); + + std::map file_lower_name_to_native; + for (uint64_t idx = 0; idx < orc_type_ptr->getSubtypeCount(); idx++) { + file_lower_name_to_native.emplace( + doris::to_lower(orc_type_ptr->getFieldName(idx)), + orc_type_ptr->getFieldName(idx)); + } - if (range.__isset.table_format_params && - range.table_format_params.table_format_type == "hive" && _state != nullptr) - [[likely]] { - hive_orc_use_column_names = _state->query_options().hive_orc_use_column_names; + auto load_info_node = std::make_shared(); + for (const auto slot : _real_tuple_desc->slots()) { + if (file_lower_name_to_native.contains(slot->col_name())) { + load_info_node->add_children( + slot->col_name(), file_lower_name_to_native[slot->col_name()], + TableSchemaChangeHelper::ConstNode::get_instance()); + } else { + load_info_node->add_not_exist_children(slot->col_name()); + } } init_status = orc_reader->init_reader( - &_file_col_names, {}, _colname_to_value_range, _push_down_conjuncts, false, + &_file_col_names, _colname_to_value_range, _push_down_conjuncts, false, _real_tuple_desc, _default_val_row_desc.get(), &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts, - hive_orc_use_column_names); + load_info_node); _cur_reader = std::move(orc_reader); } + need_to_get_parsed_schema = true; break; } @@ -1178,11 +1244,16 @@ Status VFileScanner::_get_next_reader() { } else if (!init_status.ok()) { return Status::InternalError("failed to init reader, err: {}", init_status.to_string()); } + _cur_reader->set_push_down_agg_type(_get_push_down_agg_type()); - _name_to_col_type.clear(); _missing_cols.clear(); - RETURN_IF_ERROR(_cur_reader->get_columns(&_name_to_col_type, &_missing_cols)); - _cur_reader->set_push_down_agg_type(_get_push_down_agg_type()); + _slot_lower_name_to_col_type.clear(); + std::unordered_map name_to_col_type; + RETURN_IF_ERROR(_cur_reader->get_columns(&name_to_col_type, &_missing_cols)); + for (const auto& [col_name, col_type] : name_to_col_type) { + _slot_lower_name_to_col_type.emplace(to_lower(col_name), col_type); + } + RETURN_IF_ERROR(_generate_missing_columns()); RETURN_IF_ERROR(_cur_reader->set_fill_columns(_partition_col_descs, _missing_col_descs)); if (VLOG_NOTICE_IS_ON && !_missing_cols.empty() && _is_load) { @@ -1194,18 +1265,20 @@ Status VFileScanner::_get_next_reader() { range.path); } - _source_file_col_names.clear(); - _source_file_col_types.clear(); _source_file_col_name_types.clear(); + // The col names and types of source file, such as parquet, orc files. if (_state->query_options().truncate_char_or_varchar_columns && need_to_get_parsed_schema) { - Status status = _cur_reader->get_parsed_schema(&_source_file_col_names, - &_source_file_col_types); + std::vector source_file_col_names; + std::vector source_file_col_types; + Status status = + _cur_reader->get_parsed_schema(&source_file_col_names, &source_file_col_types); if (!status.ok() && status.code() != TStatusCode::NOT_IMPLEMENTED_ERROR) { return status; } - DCHECK(_source_file_col_names.size() == _source_file_col_types.size()); - for (int i = 0; i < _source_file_col_names.size(); ++i) { - _source_file_col_name_types[_source_file_col_names[i]] = &_source_file_col_types[i]; + DCHECK(source_file_col_names.size() == source_file_col_types.size()); + for (int i = 0; i < source_file_col_names.size(); ++i) { + _source_file_col_name_types[to_lower(source_file_col_names[i])] = + source_file_col_types[i]; } } _cur_reader_eof = false; @@ -1287,6 +1360,7 @@ Status VFileScanner::_init_expr_ctxes() { } _num_of_columns_from_file = _params->num_of_columns_from_file; + for (const auto& slot_info : _params->required_slots) { auto slot_id = slot_info.slot_id; auto it = full_src_slot_map.find(slot_id); @@ -1297,11 +1371,6 @@ Status VFileScanner::_init_expr_ctxes() { if (slot_info.is_file_slot) { _file_slot_descs.emplace_back(it->second); _file_col_names.push_back(it->second->col_name()); - if (it->second->col_unique_id() >= 0) { - // Iceberg field unique ID starts from 1, Paimon/Hudi field unique ID starts from 0. - // For other data sources, all columns are set to -1. - _col_id_name_map.emplace(it->second->col_unique_id(), it->second->col_name()); - } } else { _partition_slot_descs.emplace_back(it->second); if (_is_load) { diff --git a/be/src/vec/exec/scan/vfile_scanner.h b/be/src/vec/exec/scan/vfile_scanner.h index 77d73b33746399..d2e6a220a15826 100644 --- a/be/src/vec/exec/scan/vfile_scanner.h +++ b/be/src/vec/exec/scan/vfile_scanner.h @@ -110,8 +110,6 @@ class VFileScanner : public VScanner { std::vector _file_slot_descs; // col names from _file_slot_descs std::vector _file_col_names; - // column id to name map. Collect from FE slot descriptor. - std::unordered_map _col_id_name_map; // Partition source slot descriptors std::vector _partition_slot_descs; @@ -134,15 +132,13 @@ class VFileScanner : public VScanner { std::unordered_map _src_block_name_to_idx; // Get from GenericReader, save the existing columns in file to their type. - std::unordered_map _name_to_col_type; + std::unordered_map _slot_lower_name_to_col_type; // Get from GenericReader, save columns that required by scan but not exist in file. // These columns will be filled by default value or null. std::unordered_set _missing_cols; - // The col names and types of source file, such as parquet, orc files. - std::vector _source_file_col_names; - std::vector _source_file_col_types; - std::map _source_file_col_name_types; + // The col lowercase name of source file to type of source file. + std::map _source_file_col_name_types; // For load task vectorized::VExprContextSPtrs _pre_conjunct_ctxs; diff --git a/be/test/vec/exec/format/paimon/paimon_schema_change_test.cpp b/be/test/vec/exec/format/paimon/paimon_schema_change_test.cpp deleted file mode 100644 index 8063549afc5ced..00000000000000 --- a/be/test/vec/exec/format/paimon/paimon_schema_change_test.cpp +++ /dev/null @@ -1,149 +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 "io/file_factory.h" -#include "io/fs/file_reader.h" -#include "io/io_common.h" -#include "runtime/runtime_state.h" -#include "util/runtime_profile.h" -#include "vec/exec/format/table/paimon_reader.h" - -namespace doris::vectorized { - -class PaimonMockReader final : public PaimonReader { -public: - PaimonMockReader(std::unique_ptr file_format_reader, RuntimeProfile* profile, - RuntimeState* state, const TFileScanRangeParams& params, - const TFileRangeDesc& range, io::IOContext* io_ctx) - : PaimonReader(std::move(file_format_reader), profile, state, params, range, io_ctx) {}; - ~PaimonMockReader() final = default; - - void set_delete_rows() final { - (reinterpret_cast(_file_format_reader.get())) - ->set_position_delete_rowids(&_delete_rows); - } - - void check() { - ASSERT_TRUE(_has_schema_change == true); - ASSERT_TRUE(_new_colname_to_value_range.empty()); - std::unordered_map table_col_to_file_col_ans; - table_col_to_file_col_ans["b"] = "map_col"; - table_col_to_file_col_ans["e"] = "array_col"; - table_col_to_file_col_ans["d"] = "struct_col"; - table_col_to_file_col_ans["a"] = "vvv"; - table_col_to_file_col_ans["c"] = "k"; - for (auto [table_col, file_col] : table_col_to_file_col_ans) { - ASSERT_TRUE(_table_col_to_file_col[table_col] == file_col); - ASSERT_TRUE(_file_col_to_table_col[file_col] == table_col); - } - ASSERT_TRUE(_all_required_col_names.size() == 6); - - std::set all_required_col_names_set; - all_required_col_names_set.emplace("map_col"); - all_required_col_names_set.emplace("array_col"); - all_required_col_names_set.emplace("struct_col"); - all_required_col_names_set.emplace("vvv"); - all_required_col_names_set.emplace("k"); - all_required_col_names_set.emplace("nonono"); - - for (auto i : _all_required_col_names) { - ASSERT_TRUE(all_required_col_names_set.contains(i)); - } - - ASSERT_TRUE(_not_in_file_col_names.size() == 1); - ASSERT_TRUE(_not_in_file_col_names.back() == "nonono"); - } -}; - -class PaimonReaderTest : public ::testing::Test { -protected: - void SetUp() override { - _profile = new RuntimeProfile("test_profile"); - _state = new RuntimeState(TQueryGlobals()); - _io_ctx = new io::IOContext(); - } - - void TearDown() override { - delete _profile; - delete _state; - delete _io_ctx; - } - - RuntimeProfile* _profile; - RuntimeState* _state; - io::IOContext* _io_ctx; -}; - -TEST_F(PaimonReaderTest, ReadSchemaFile) { - std::map file_id_to_name; - file_id_to_name[0] = "k"; - file_id_to_name[1] = "vvv"; - file_id_to_name[2] = "array_col"; - file_id_to_name[3] = "struct_col"; - file_id_to_name[6] = "map_col"; - - TFileScanRangeParams params; - params.file_type = TFileType::FILE_LOCAL; - params.properties = {}; - params.hdfs_params = {}; - params.__isset.history_schema_info = true; - params.history_schema_info[0] = file_id_to_name; - TFileRangeDesc range; - range.table_format_params.paimon_params.schema_id = 0; - - PaimonMockReader reader(nullptr, _profile, _state, params, range, _io_ctx); - - // create table tmp5 ( - // k int, - // vVV string, - // array_col array, - // struct_COL struct, - // map_COL map - // ) tblproperties ( - // 'primary-key' = 'k', - // "file.format" = "parquet" - // ); - - std::vector read_table_col_names; - read_table_col_names.emplace_back("a"); - read_table_col_names.emplace_back("b"); - read_table_col_names.emplace_back("c"); - read_table_col_names.emplace_back("d"); - read_table_col_names.emplace_back("e"); - read_table_col_names.emplace_back("nonono"); - - std::unordered_map table_col_id_table_name_map; - table_col_id_table_name_map[1] = "a"; - table_col_id_table_name_map[6] = "b"; - table_col_id_table_name_map[0] = "c"; - table_col_id_table_name_map[3] = "d"; - table_col_id_table_name_map[2] = "e"; - table_col_id_table_name_map[10] = "nonono"; - - std::unordered_map table_col_name_to_value_range; - Status status = reader.init_schema_info(read_table_col_names, table_col_id_table_name_map, - &table_col_name_to_value_range); - ASSERT_TRUE(status.ok()); - reader.check(); -} - -} // namespace doris::vectorized \ No newline at end of file diff --git a/be/test/vec/exec/format/parquet/parquet_reader_test.cpp b/be/test/vec/exec/format/parquet/parquet_reader_test.cpp index afa4d7f9d5fabe..9a0c9c22c35c4f 100644 --- a/be/test/vec/exec/format/parquet/parquet_reader_test.cpp +++ b/be/test/vec/exec/format/parquet/parquet_reader_test.cpp @@ -133,7 +133,6 @@ TEST_F(ParquetReaderTest, normal) { TimezoneUtils::find_cctz_time_zone(TimezoneUtils::default_time_zone, ctz); auto tuple_desc = desc_tbl->get_tuple_descriptor(0); std::vector column_names; - std::vector missing_column_names; for (int i = 0; i < slot_descs.size(); i++) { column_names.push_back(slot_descs[i]->col_name()); } @@ -150,8 +149,8 @@ TEST_F(ParquetReaderTest, normal) { runtime_state.set_desc_tbl(desc_tbl); std::unordered_map colname_to_value_range; - static_cast(p_reader->init_reader(column_names, missing_column_names, nullptr, {}, - nullptr, nullptr, nullptr, nullptr, nullptr)); + static_cast(p_reader->init_reader(column_names, nullptr, {}, nullptr, nullptr, nullptr, + nullptr, nullptr)); std::unordered_map> partition_columns; std::unordered_map missing_columns; @@ -196,7 +195,6 @@ static ParquetReader* create_parquet_reader(TFileScanRangeParams& scan_params, cctz::time_zone ctz; TimezoneUtils::find_cctz_time_zone(TimezoneUtils::default_time_zone, ctz); std::vector column_names; - std::vector missing_column_names; for (int i = 0; i < slot_descs.size(); i++) { column_names.push_back(slot_descs[i]->col_name()); } @@ -212,8 +210,6 @@ static ParquetReader* create_parquet_reader(TFileScanRangeParams& scan_params, } TEST_F(ParquetReaderTest, use_column_name) { - bool use_column_name = true; - std::vector table_column_names = {"boolean_col", "tinyint_col", "smallint_col", "int_col", "bigint_col", "float_col", "double_col"}; @@ -230,14 +226,14 @@ TEST_F(ParquetReaderTest, use_column_name) { colname_to_value_range.emplace("smallint_col", ColumnValueRange("smallint_col")); colname_to_value_range.emplace("int_col", ColumnValueRange("int_col")); - static_cast(p_reader->init_reader(table_column_names, {}, &colname_to_value_range, {}, - nullptr, nullptr, nullptr, nullptr, nullptr, false, - use_column_name)); + static_cast(p_reader->init_reader( + table_column_names, &colname_to_value_range, {}, nullptr, nullptr, nullptr, nullptr, + nullptr, TableSchemaChangeHelper::ConstNode::get_instance(), false)); std::vector read_columns_ans = {"tinyint_col", "smallint_col", "int_col", "bigint_col", "boolean_col", "float_col", "double_col"}; - EXPECT_EQ(p_reader->_read_columns, read_columns_ans); + EXPECT_EQ(p_reader->_read_file_columns, read_columns_ans); std::vector miss_columns_ans = {}; EXPECT_EQ(p_reader->_missing_cols, miss_columns_ans); @@ -250,86 +246,5 @@ TEST_F(ParquetReaderTest, use_column_name) { delete p_reader; } -TEST_F(ParquetReaderTest, use_column_name2) { - bool use_column_name = true; - - std::vector table_column_names = {"boolean_col", "tinyint_col", "smallint_col", - "int_col", "bigint_col", "float_col", - "test1", "double_col", "test2"}; - std::vector table_column_types = { - TPrimitiveType::BOOLEAN, TPrimitiveType::TINYINT, TPrimitiveType::SMALLINT, - TPrimitiveType::INT, TPrimitiveType::BIGINT, TPrimitiveType::FLOAT, - TPrimitiveType::FLOAT, TPrimitiveType::DOUBLE, TPrimitiveType::DOUBLE}; - TFileScanRangeParams scan_params; - - auto p_reader = create_parquet_reader(scan_params, table_column_names, table_column_types); - std::unordered_map colname_to_value_range; - colname_to_value_range.emplace("boolean_col", ColumnValueRange("boolean_col")); - colname_to_value_range.emplace("tinyint_col", ColumnValueRange("tinyint_col")); - colname_to_value_range.emplace("smallint_col", ColumnValueRange("smallint_col")); - colname_to_value_range.emplace("int_col", ColumnValueRange("int_col")); - - static_cast(p_reader->init_reader(table_column_names, {"boolean_col"}, - &colname_to_value_range, {}, nullptr, nullptr, nullptr, - nullptr, nullptr, false, use_column_name)); - - std::vector read_columns_ans = {"tinyint_col", "smallint_col", "int_col", - "bigint_col", "float_col", "double_col"}; - EXPECT_EQ(p_reader->_read_columns, read_columns_ans); - - std::vector miss_columns_ans = {"boolean_col", "test1", "test2"}; - EXPECT_EQ(p_reader->_missing_cols, miss_columns_ans); - std::vector colname_to_value_range_names_ans = {"tinyint_col", "smallint_col", - "int_col", "boolean_col"}; - for (auto col : colname_to_value_range_names_ans) { - EXPECT_TRUE(p_reader->_colname_to_value_range->contains(col)); - } - EXPECT_EQ(p_reader->_colname_to_value_range->size(), colname_to_value_range_names_ans.size()); - delete p_reader; -} - -TEST_F(ParquetReaderTest, use_column_idx) { - bool use_column_name = false; - - std::vector table_column_names = {"col0", "col1", "col3", - "col7", "col100", "col102"}; - std::vector table_column_types = { - TPrimitiveType::BOOLEAN, TPrimitiveType::TINYINT, TPrimitiveType::SMALLINT, - TPrimitiveType::INT, TPrimitiveType::BIGINT, TPrimitiveType::BIGINT}; - TFileScanRangeParams scan_params; - scan_params.column_idxs.emplace_back(0); - scan_params.column_idxs.emplace_back(1); - scan_params.column_idxs.emplace_back(3); - scan_params.column_idxs.emplace_back(7); - scan_params.column_idxs.emplace_back(100); - scan_params.column_idxs.emplace_back(102); - - auto p_reader = create_parquet_reader(scan_params, table_column_names, table_column_types); - std::unordered_map colname_to_value_range; - colname_to_value_range.emplace("col0", ColumnValueRange("col0")); - colname_to_value_range.emplace("col1", ColumnValueRange("col1")); - colname_to_value_range.emplace("col3", ColumnValueRange("col3")); - colname_to_value_range.emplace("col102", ColumnValueRange("col102")); - - static_cast(p_reader->init_reader(table_column_names, {}, &colname_to_value_range, {}, - nullptr, nullptr, nullptr, nullptr, nullptr, false, - use_column_name)); - - std::vector read_columns_ans = {"tinyint_col", "smallint_col", "bigint_col", - "string_col"}; - EXPECT_EQ(p_reader->_read_columns, read_columns_ans); - - std::vector miss_columns_ans = {"col100", "col102"}; - EXPECT_EQ(p_reader->_missing_cols, miss_columns_ans); - - std::vector colname_to_value_range_names_ans = {"tinyint_col", "smallint_col", - "bigint_col"}; - for (auto col : colname_to_value_range_names_ans) { - EXPECT_TRUE(p_reader->_colname_to_value_range->contains(col)); - } - EXPECT_EQ(p_reader->_colname_to_value_range->size(), colname_to_value_range_names_ans.size()); - delete p_reader; -} - } // namespace vectorized } // namespace doris diff --git a/be/test/vec/exec/format/table/table_schema_change_helper_test.cpp b/be/test/vec/exec/format/table/table_schema_change_helper_test.cpp index a7940eb4a544d1..84e9fa55cc385b 100644 --- a/be/test/vec/exec/format/table/table_schema_change_helper_test.cpp +++ b/be/test/vec/exec/format/table/table_schema_change_helper_test.cpp @@ -21,451 +21,770 @@ #include #include -#include "common/status.h" +#include "testutil/desc_tbl_builder.h" #include "vec/columns/column_string.h" -#include "vec/core/block.h" -#include "vec/data_types/data_type_string.h" +#include "vec/data_types/data_type_factory.hpp" +#include "vec/exec/format/table/iceberg_reader.h" #include "vec/exec/format/table/table_format_reader.h" namespace doris::vectorized { -class MockTableSchemaChangeHelper : public TableSchemaChangeHelper { -public: - MockTableSchemaChangeHelper(std::map file_schema, bool exist_schema = true) - : _file_schema(std::move(file_schema)), _exist_schema(exist_schema) {} - - Status get_file_col_id_to_name(bool& exist_schema, - std::map& file_col_id_to_name) override { - exist_schema = _exist_schema; - if (_exist_schema) { - file_col_id_to_name = _file_schema; - } - return Status::OK(); - } +class MockTableSchemaChangeHelper : public TableSchemaChangeHelper {}; - bool has_schema_change() const { return _has_schema_change; } - const std::vector& all_required_col_names() const { - return _all_required_col_names; - } - const std::vector& not_in_file_col_names() const { return _not_in_file_col_names; } - const std::unordered_map& file_col_to_table_col() const { - return _file_col_to_table_col; - } - const std::unordered_map& table_col_to_file_col() const { - return _table_col_to_file_col; +TEST(MockTableSchemaChangeHelper, OrcNameNoSchemaChange) { + std::vector data_types; + std::vector column_names; + + TSlotDescriptor tslot1; + { + TypeDescriptor type_desc(TYPE_BIGINT); + tslot1.__set_slotType(type_desc.to_thrift()); } - const std::unordered_map& new_colname_to_value_range() - const { - return _new_colname_to_value_range; + SlotDescriptor slot1(tslot1); + slot1._col_name = "col1"; + + TSlotDescriptor tslot2; + { + TypeDescriptor type_desc(TYPE_STRUCT); + type_desc.add_sub_type(TYPE_BIGINT, "a", true); + type_desc.add_sub_type(TYPE_BIGINT, "b", true); + tslot2.__set_slotType(type_desc.to_thrift()); } + SlotDescriptor slot2(tslot2); + slot2._col_name = "col2"; -private: - std::map _file_schema; - bool _exist_schema; -}; - -TEST(TableSchemaChangeHelperTest, NoSchemaChange) { - std::map file_schema = {{1, "col1"}, {2, "col2"}, {3, "col3"}}; - std::unordered_map table_id_to_name = { - {1, "col1"}, {2, "col2"}, {3, "col3"}}; - - std::vector read_cols = {"col1", "col3"}; - std::unordered_map col_ranges; - - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); - - ASSERT_FALSE(helper.has_schema_change()); - ASSERT_EQ(helper.all_required_col_names().size(), 2); - ASSERT_EQ(helper.all_required_col_names()[0], "col1"); - ASSERT_EQ(helper.all_required_col_names()[1], "col3"); - ASSERT_TRUE(helper.not_in_file_col_names().empty()); -} - -TEST(TableSchemaChangeHelperTest, WithSchemaChange) { - std::map file_schema = {{1, "col1"}, {2, "col2_old"}, {3, "col3_old"}}; - - std::unordered_map table_id_to_name = { - {1, "col1"}, {2, "col2_new"}, {3, "col3_new"}}; - - std::vector read_cols = {"col1", "col2_new", "col3_new"}; - - std::unordered_map col_ranges = { - {"col2_new", ColumnValueRangeType()}}; - - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); - - ASSERT_TRUE(helper.has_schema_change()); - ASSERT_EQ(helper.all_required_col_names().size(), 3); - ASSERT_EQ(helper.all_required_col_names()[0], "col1"); - ASSERT_EQ(helper.all_required_col_names()[1], "col2_old"); - ASSERT_EQ(helper.all_required_col_names()[2], "col3_old"); - ASSERT_TRUE(helper.not_in_file_col_names().empty()); - - ASSERT_EQ(helper.table_col_to_file_col().size(), 3); - ASSERT_EQ(helper.table_col_to_file_col().at("col2_new"), "col2_old"); - ASSERT_EQ(helper.table_col_to_file_col().at("col3_new"), "col3_old"); - - ASSERT_EQ(helper.file_col_to_table_col().size(), 3); - ASSERT_EQ(helper.file_col_to_table_col().at("col2_old"), "col2_new"); - ASSERT_EQ(helper.file_col_to_table_col().at("col3_old"), "col3_new"); - - ASSERT_EQ(helper.new_colname_to_value_range().size(), 1); - ASSERT_TRUE(helper.new_colname_to_value_range().find("col2_old") != - helper.new_colname_to_value_range().end()); -} - -TEST(TableSchemaChangeHelperTest, MissingColumns) { - std::map file_schema = {{1, "col1"}, {2, "col2"} - - }; - - std::unordered_map table_id_to_name = { - {1, "col1"}, {2, "col2"}, {3, "col3"}, {4, "col4"}}; - std::vector read_cols = {"col1", "col3", "col4"}; - std::unordered_map col_ranges = { - {"col3", ColumnValueRangeType()}}; - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); - - ASSERT_FALSE(helper.has_schema_change()); - ASSERT_EQ(helper.all_required_col_names().size(), 3); - ASSERT_EQ(helper.all_required_col_names()[0], "col1"); - ASSERT_EQ(helper.all_required_col_names()[1], "col3"); - ASSERT_EQ(helper.all_required_col_names()[2], "col4"); - ASSERT_EQ(helper.not_in_file_col_names().size(), 2); - ASSERT_EQ(helper.not_in_file_col_names()[0], "col3"); - ASSERT_EQ(helper.not_in_file_col_names()[1], "col4"); -} + TSlotDescriptor tslot3; -TEST(TableSchemaChangeHelperTest, NoFileSchema) { - std::map file_schema; - - std::unordered_map table_id_to_name = { - {1, "col1"}, {2, "col2"}, {3, "col3"}}; - - std::vector read_cols = {"col1", "col2"}; - std::unordered_map col_ranges; - MockTableSchemaChangeHelper helper(file_schema, false); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); - - ASSERT_FALSE(helper.has_schema_change()); - ASSERT_EQ(helper.all_required_col_names().size(), 2); - ASSERT_EQ(helper.all_required_col_names()[0], "col1"); - ASSERT_EQ(helper.all_required_col_names()[1], "col2"); - ASSERT_TRUE(helper.not_in_file_col_names().empty()); -} - -TEST(TableSchemaChangeHelperTest, MixedScenario) { - std::map file_schema = {{1, "col1"}, {2, "col2_old"}, {4, "col4_old"}}; - std::unordered_map table_id_to_name = { - {1, "col1"}, {2, "col2_new"}, {3, "col3"}, {4, "col4_new"}, {5, "col5"}}; - std::vector read_cols = {"col1", "col2_new", "col3", "col4_new", "col5"}; - std::unordered_map col_ranges = { - {"col2_new", ColumnValueRangeType()}, - {"col3", ColumnValueRangeType()}, - {"col5", ColumnValueRangeType()}}; - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); - ASSERT_TRUE(helper.has_schema_change()); - ASSERT_EQ(helper.all_required_col_names().size(), 5); - ASSERT_EQ(helper.all_required_col_names()[0], "col1"); - ASSERT_EQ(helper.all_required_col_names()[1], "col2_old"); - ASSERT_EQ(helper.all_required_col_names()[2], "col3"); - ASSERT_EQ(helper.all_required_col_names()[3], "col4_old"); - ASSERT_EQ(helper.all_required_col_names()[4], "col5"); - ASSERT_EQ(helper.not_in_file_col_names().size(), 2); - ASSERT_EQ(helper.not_in_file_col_names()[0], "col3"); - ASSERT_EQ(helper.not_in_file_col_names()[1], "col5"); - ASSERT_EQ(helper.table_col_to_file_col().at("col2_new"), "col2_old"); - ASSERT_EQ(helper.table_col_to_file_col().at("col4_new"), "col4_old"); - ASSERT_EQ(helper.new_colname_to_value_range().size(), 3); - ASSERT_TRUE(helper.new_colname_to_value_range().find("col2_old") != - helper.new_colname_to_value_range().end()); - ASSERT_TRUE(helper.new_colname_to_value_range().find("col3") != - helper.new_colname_to_value_range().end()); - ASSERT_TRUE(helper.new_colname_to_value_range().find("col5") != - helper.new_colname_to_value_range().end()); + { + TypeDescriptor type_desc(TYPE_INT); + tslot3.__set_slotType(type_desc.to_thrift()); + } + SlotDescriptor slot3(tslot3); + slot3._col_name = "col3"; + + TTupleDescriptor ttuple_desc; + TupleDescriptor tuple_desc(ttuple_desc); + tuple_desc.add_slot(&slot1); + tuple_desc.add_slot(&slot2); + tuple_desc.add_slot(&slot3); + + std::cout << tuple_desc.debug_string() << "\n"; + + std::unique_ptr orc_type( + orc::Type::buildTypeFromString("struct,col3:int>")); + + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_name(&tuple_desc, + orc_type.get(), ans_node) + .ok()); + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (file: col1)\n" + " ScalarNode\n" + " col2 (file: col2)\n" + " StructNode\n" + " a (file: a)\n" + " ScalarNode\n" + " b (file: b)\n" + " ScalarNode\n" + " col3 (file: col3)\n" + " ScalarNode\n"); } -TEST(TableSchemaChangeHelperTest, EmptySchemas) { - std::map file_schema; - std::unordered_map table_id_to_name; - std::vector read_cols; - std::unordered_map col_ranges; - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); - - ASSERT_FALSE(helper.has_schema_change()); - ASSERT_TRUE(helper.all_required_col_names().empty()); - ASSERT_TRUE(helper.not_in_file_col_names().empty()); - ASSERT_TRUE(helper.table_col_to_file_col().empty()); - ASSERT_TRUE(helper.file_col_to_table_col().empty()); - ASSERT_TRUE(helper.new_colname_to_value_range().empty()); -} +TEST(MockTableSchemaChangeHelper, OrcNameSchemaChange1) { + std::vector data_types; + std::vector column_names; + TSlotDescriptor tslot1; + { + TypeDescriptor type_desc(TYPE_BIGINT); + tslot1.__set_slotType(type_desc.to_thrift()); + } + SlotDescriptor slot1(tslot1); + slot1._col_name = "col1"; -TEST(TableSchemaChangeHelperTest, IdMismatch) { - std::map file_schema = {{1, "col1"}, {2, "col2"}, {3, "col3"}}; + std::cout << " slot1 = " << slot1.get_data_type_ptr()->get_name() << "\n"; - std::unordered_map table_id_to_name = { - {10, "col1"}, {20, "col2"}, {30, "col3"}}; + TSlotDescriptor tslot2; + TypeDescriptor type_desc_2(TYPE_STRUCT); + type_desc_2.add_sub_type(TYPE_BIGINT, "a", true); + type_desc_2.add_sub_type(TYPE_BIGINT, "b", true); + tslot2.__set_slotType(type_desc_2.to_thrift()); + SlotDescriptor slot2(tslot2); + slot2._col_name = "col2"; + std::cout << " slot2 = " << slot2.get_data_type_ptr()->get_name() << "\n"; - std::vector read_cols = {"col1", "col2", "col3"}; + TSlotDescriptor tslot3; - std::unordered_map col_ranges; + { + TypeDescriptor type_desc(TYPE_INT); + tslot3.__set_slotType(type_desc.to_thrift()); + } + SlotDescriptor slot3(tslot3); + slot3._col_name = "col3"; + + TTupleDescriptor ttuple_desc; + TupleDescriptor tuple_desc(ttuple_desc); + tuple_desc.add_slot(&slot1); + tuple_desc.add_slot(&slot2); + tuple_desc.add_slot(&slot3); + + std::cout << tuple_desc.debug_string() << "\n"; + { + std::unique_ptr orc_type( + orc::Type::buildTypeFromString("struct,col3:int>")); + + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_name( + &tuple_desc, orc_type.get(), ans_node) + .ok()); + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (file: col1)\n" + " ScalarNode\n" + " col2 (file: col2)\n" + " StructNode\n" + " a (file: a)\n" + " ScalarNode\n" + " b (not exists)\n" + " col3 (file: col3)\n" + " ScalarNode\n"); + } + { + std::unique_ptr orc_type( + orc::Type::buildTypeFromString("struct,Col3:int>")); + + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_name( + &tuple_desc, orc_type.get(), ans_node) + .ok()); + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (not exists)\n" + " col2 (file: COL2)\n" + " StructNode\n" + " a (file: A)\n" + " ScalarNode\n" + " b (not exists)\n" + " col3 (file: Col3)\n" + " ScalarNode\n"); + } - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); + { + std::unique_ptr orc_type(orc::Type::buildTypeFromString( + "struct>")); + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_name( + &tuple_desc, orc_type.get(), ans_node) + .ok()); + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (file: col1)\n" + " ScalarNode\n" + " col2 (file: COL2)\n" + " StructNode\n" + " a (file: A)\n" + " ScalarNode\n" + " b (not exists)\n" + " col3 (file: CoL3)\n" + " ScalarNode\n"); + } - ASSERT_FALSE(helper.has_schema_change()); - ASSERT_EQ(helper.all_required_col_names().size(), 3); - ASSERT_EQ(helper.not_in_file_col_names().size(), 3); - ASSERT_TRUE(helper.table_col_to_file_col().empty()); - ASSERT_TRUE(helper.file_col_to_table_col().empty()); + { + std::unique_ptr orc_type(orc::Type::buildTypeFromString("struct")); + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_name( + &tuple_desc, orc_type.get(), ans_node) + .ok()); + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (not exists)\n" + " col2 (not exists)\n" + " col3 (not exists)\n"); + } } -TEST(TableSchemaChangeHelperTest, DuplicateColumnNames) { - std::map file_schema = {{1, "col1"}, {2, "col2"}}; - - std::unordered_map table_id_to_name = { - {1, "col1"}, {2, "col2"}, {3, "col2"}, {4, "col1"}}; - - std::vector read_cols = {"col1", "col2"}; - std::unordered_map col_ranges; - - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); +TEST(MockTableSchemaChangeHelper, ParquetNameSchemaChange) { + std::vector data_types; + std::vector column_names; - ASSERT_FALSE(helper.has_schema_change()); - ASSERT_EQ(helper.all_required_col_names().size(), 2); - ASSERT_EQ(helper.all_required_col_names()[0], "col1"); - ASSERT_EQ(helper.all_required_col_names()[1], "col2"); - ASSERT_TRUE(helper.not_in_file_col_names().empty()); - ASSERT_EQ(helper.table_col_to_file_col().size(), 2); -} + TSlotDescriptor tslot1; + { + TypeDescriptor type_desc(TYPE_BIGINT); + tslot1.__set_slotType(type_desc.to_thrift()); + } + SlotDescriptor slot1(tslot1); + slot1._col_name = "col1"; + + TSlotDescriptor tslot2; + { + TypeDescriptor type_desc(TYPE_STRUCT); + type_desc.add_sub_type(TYPE_BIGINT, "a", true); + type_desc.add_sub_type(TYPE_BIGINT, "b", true); + tslot2.__set_slotType(type_desc.to_thrift()); + } + SlotDescriptor slot2(tslot2); + slot2._col_name = "col2"; -TEST(TableSchemaChangeHelperTest, ValueRangeForNonReadColumns) { - std::map file_schema = {{1, "col1"}, {2, "col2"}, {3, "col3"}, {4, "col4"}}; + TSlotDescriptor tslot3; - std::unordered_map table_id_to_name = { - {1, "col1"}, {2, "col2_new"}, {3, "col3"}, {4, "col4"}}; + { + TypeDescriptor type_desc(TYPE_INT); + tslot3.__set_slotType(type_desc.to_thrift()); + } + SlotDescriptor slot3(tslot3); + slot3._col_name = "col3"; - std::vector read_cols = {"col1", "col3"}; + TTupleDescriptor ttuple_desc; + TupleDescriptor tuple_desc(ttuple_desc); + tuple_desc.add_slot(&slot1); + tuple_desc.add_slot(&slot2); + tuple_desc.add_slot(&slot3); - std::unordered_map col_ranges = { - {"col1", ColumnValueRangeType()}, - {"col2_new", ColumnValueRangeType()}, - {"col4", ColumnValueRangeType()}}; + FieldDescriptor parquet_field; - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); + FieldSchema parquet_field_col1; + { + parquet_field_col1.name = "col1"; - ASSERT_TRUE(helper.has_schema_change()); - ASSERT_EQ(helper.all_required_col_names().size(), 2); - ASSERT_EQ(helper.all_required_col_names()[0], "col1"); - ASSERT_EQ(helper.all_required_col_names()[1], "col3"); - ASSERT_TRUE(helper.not_in_file_col_names().empty()); + { + TypeDescriptor type_desc(TYPE_BIGINT); + parquet_field_col1.type = type_desc; + } - ASSERT_EQ(helper.new_colname_to_value_range().size(), 3); - ASSERT_TRUE(helper.new_colname_to_value_range().find("col1") != - helper.new_colname_to_value_range().end()); - ASSERT_TRUE(helper.new_colname_to_value_range().find("col2") != - helper.new_colname_to_value_range().end()); - ASSERT_TRUE(helper.new_colname_to_value_range().find("col4") != - helper.new_colname_to_value_range().end()); + parquet_field_col1.field_id = -1; + parquet_field._fields.emplace_back(parquet_field_col1); + } + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_parquet_name( + &tuple_desc, parquet_field, ans_node) + .ok()); + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (file: col1)\n" + " ScalarNode\n" + " col2 (not exists)\n" + " col3 (not exists)\n"); } -TEST(TableSchemaChangeHelperTest, PartialIdMatch) { - std::map file_schema = {{1, "col1"}, {2, "col2"}, {3, "col3"}, {4, "col4"}}; - - std::unordered_map table_id_to_name = { - {1, "col1"}, {20, "col2"}, {3, "col3_new"}, {40, "col4_new"}}; - std::vector read_cols = {"col1", "col2", "col3_new", "col4_new"}; - std::unordered_map col_ranges; +TEST(MockTableSchemaChangeHelper, IcebergParquetSchemaChange) { + schema::external::TStructField root_field; + { + TColumnType int_type; + int_type.type = TPrimitiveType::INT; - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); + TColumnType struct_type; + struct_type.type = TPrimitiveType::STRUCT; - ASSERT_TRUE(helper.has_schema_change()); + { + auto col1_field = std::make_shared(); + col1_field->name = "col1"; + col1_field->id = 1; + col1_field->type = int_type; - ASSERT_EQ(helper.all_required_col_names().size(), 4); - ASSERT_EQ(helper.all_required_col_names()[0], "col1"); - ASSERT_EQ(helper.all_required_col_names()[1], "col2"); - ASSERT_EQ(helper.all_required_col_names()[2], "col3"); - ASSERT_EQ(helper.all_required_col_names()[3], "col4_new"); + schema::external::TFieldPtr col1_ptr; + col1_ptr.field_ptr = col1_field; + root_field.fields.emplace_back(col1_ptr); + } - ASSERT_EQ(helper.not_in_file_col_names().size(), 2); - ASSERT_EQ(helper.not_in_file_col_names()[0], "col2"); - ASSERT_EQ(helper.not_in_file_col_names()[1], "col4_new"); + { + auto col2_field = std::make_shared(); + col2_field->name = "col2"; + col2_field->id = 2; + col2_field->type = struct_type; + + schema::external::TStructField struct_field; + { + auto a_field = std::make_shared(); + a_field->name = "a"; + a_field->id = 3; + a_field->type = int_type; + schema::external::TFieldPtr a_ptr; + a_ptr.field_ptr = a_field; + struct_field.fields.emplace_back(a_ptr); + } + + col2_field->nestedField.struct_field = struct_field; + schema::external::TFieldPtr col2_ptr; + col2_ptr.field_ptr = col2_field; + root_field.fields.emplace_back(col2_ptr); + } + } - ASSERT_EQ(helper.table_col_to_file_col().size(), 2); - ASSERT_EQ(helper.table_col_to_file_col().at("col1"), "col1"); - ASSERT_EQ(helper.table_col_to_file_col().at("col3_new"), "col3"); -} + FieldDescriptor parquet_field; + { + { + FieldSchema parquet_field_col1; + parquet_field_col1.name = "col1"; + TypeDescriptor type_desc_col1(TYPE_BIGINT); + parquet_field_col1.type = type_desc_col1; + parquet_field_col1.field_id = 1; + parquet_field._fields.emplace_back(parquet_field_col1); + } -Block create_test_block(const std::vector& column_names) { - Block block; - for (const auto& name : column_names) { - auto column = ColumnString::create(); - block.insert( - ColumnWithTypeAndName(std::move(column), std::make_shared(), name)); + { + FieldSchema parquet_field_col2; + parquet_field_col2.name = "coL1"; + TypeDescriptor type_desc_col2(TYPE_STRUCT); + + { + FieldSchema b_field; + b_field.name = "b5555555"; + b_field.field_id = 4; + TypeDescriptor b_type_desc(TYPE_BIGINT); + b_field.type = b_type_desc; + type_desc_col2.children.emplace_back(b_type_desc); + parquet_field_col2.children.emplace_back(b_field); + } + { + FieldSchema a_field; + a_field.name = "a33333333"; + a_field.field_id = 3; + TypeDescriptor a_type_desc(TYPE_BIGINT); + a_field.type = a_type_desc; + type_desc_col2.children.emplace_back(a_type_desc); + parquet_field_col2.children.emplace_back(a_field); + } + parquet_field_col2.type = type_desc_col2; + parquet_field_col2.field_id = 2; + parquet_field._fields.emplace_back(parquet_field_col2); + } } - return block; + bool exist_field_id = true; + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_parquet_field_id( + root_field, parquet_field, ans_node, exist_field_id) + .ok()); + ASSERT_TRUE(exist_field_id); + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (file: col1)\n" + " ScalarNode\n" + " col2 (file: coL1)\n" + " StructNode\n" + " a (file: a33333333)\n" + " ScalarNode\n" + + ); } -TEST(TableSchemaChangeHelperTest, BasicColumnNameConversion) { - std::map file_schema = {{1, "col1"}, {2, "col2_old"}, {3, "col3_old"}}; - - std::unordered_map table_id_to_name = { - {1, "col1"}, {2, "col2_new"}, {3, "col3_new"}}; - - std::vector read_cols = {"col1", "col2_new", "col3_new"}; - std::unordered_map col_ranges; - - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); - - ASSERT_TRUE(helper.has_schema_change()); - - Block before_block = create_test_block({"col1", "col2_new", "col3_new"}); - ASSERT_TRUE(helper.get_next_block_before(&before_block).ok()); - - ASSERT_EQ(before_block.get_by_position(0).name, "col1"); - ASSERT_EQ(before_block.get_by_position(1).name, "col2_old"); - ASSERT_EQ(before_block.get_by_position(2).name, "col3_old"); +TEST(MockTableSchemaChangeHelper, IcebergOrcSchemaChange) { + schema::external::TStructField root_field; + { + TColumnType int_type; + int_type.type = TPrimitiveType::INT; + + TColumnType struct_type; + struct_type.type = TPrimitiveType::STRUCT; + + { + auto col1_field = std::make_shared(); + col1_field->name = "col1"; + col1_field->id = 1; + col1_field->type = int_type; + schema::external::TFieldPtr col1_ptr; + col1_ptr.field_ptr = col1_field; + root_field.fields.emplace_back(col1_ptr); + } - Block after_block = create_test_block({"col1", "col2_old", "col3_old"}); - ASSERT_TRUE(helper.get_next_block_after(&after_block).ok()); + { + auto col2_field = std::make_shared(); + col2_field->name = "col2"; + col2_field->id = 2; + col2_field->type = struct_type; + + schema::external::TStructField struct_field; + { + auto a_field = std::make_shared(); + a_field->name = "a"; + a_field->id = 3; + a_field->type = int_type; + schema::external::TFieldPtr a_ptr; + a_ptr.field_ptr = a_field; + struct_field.fields.emplace_back(a_ptr); + } + + { + auto b_field = std::make_shared(); + b_field->name = "b"; + b_field->id = 4; + b_field->type = int_type; + schema::external::TFieldPtr b_ptr; + b_ptr.field_ptr = b_field; + struct_field.fields.emplace_back(b_ptr); + } + + col2_field->nestedField.struct_field = struct_field; + schema::external::TFieldPtr col2_ptr; + col2_ptr.field_ptr = col2_field; + root_field.fields.emplace_back(col2_ptr); + } + } - ASSERT_EQ(after_block.get_by_position(0).name, "col1"); - ASSERT_EQ(after_block.get_by_position(1).name, "col2_new"); - ASSERT_EQ(after_block.get_by_position(2).name, "col3_new"); + std::unique_ptr orc_type(orc::Type::buildTypeFromString( + "struct,COL369:int>")); + const auto& attribute = IcebergOrcReader::ICEBERG_ORC_ATTRIBUTE; + orc_type->getSubtype(0)->setAttribute(attribute, "1"); + orc_type->getSubtype(1)->setAttribute(attribute, "2"); + orc_type->getSubtype(1)->getSubtype(0)->setAttribute(attribute, "3"); + orc_type->getSubtype(1)->getSubtype(1)->setAttribute(attribute, "4"); + orc_type->getSubtype(2)->setAttribute(attribute, "5"); + + bool exist_field_id = true; + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_field_id( + root_field, orc_type.get(), attribute, ans_node, exist_field_id) + .ok()); + ASSERT_TRUE(exist_field_id); + + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (file: col1)\n" + " ScalarNode\n" + " col2 (file: col1122)\n" + " StructNode\n" + " a (file: a)\n" + " ScalarNode\n" + " b (file: aa)\n" + " ScalarNode\n"); } -TEST(TableSchemaChangeHelperTest, NoSchemaChangeBlocks) { - std::map file_schema = {{1, "col1"}, {2, "col2"}, {3, "col3"}}; - - std::unordered_map table_id_to_name = { - {1, "col1"}, {2, "col2"}, {3, "col3"}}; - - std::vector read_cols = {"col1", "col2", "col3"}; - - std::unordered_map col_ranges; - - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); - - ASSERT_FALSE(helper.has_schema_change()); +TEST(MockTableSchemaChangeHelper, NestedMapArrayStruct) { + // struct, struct>> - Block before_block = create_test_block({"col1", "col2", "col3"}); - ASSERT_TRUE(helper.get_next_block_before(&before_block).ok()); + TSlotDescriptor tslot1; + { + TypeDescriptor map_type_desc(TYPE_MAP); - ASSERT_EQ(before_block.get_by_position(0).name, "col1"); - ASSERT_EQ(before_block.get_by_position(1).name, "col2"); - ASSERT_EQ(before_block.get_by_position(2).name, "col3"); + TypeDescriptor array_type_desc(TYPE_ARRAY); + array_type_desc.add_sub_type(TYPE_BIGINT, "a", true); - Block after_block = create_test_block({"col1", "col2", "col3"}); - ASSERT_TRUE(helper.get_next_block_after(&after_block).ok()); + TypeDescriptor struct_type_desc(TYPE_STRUCT); + struct_type_desc.add_sub_type(TYPE_BIGINT, "a", true); + struct_type_desc.add_sub_type(TYPE_BIGINT, "b", true); - ASSERT_EQ(after_block.get_by_position(0).name, "col1"); - ASSERT_EQ(after_block.get_by_position(1).name, "col2"); - ASSERT_EQ(after_block.get_by_position(2).name, "col3"); + map_type_desc.add_sub_type(array_type_desc); + map_type_desc.add_sub_type(struct_type_desc); + tslot1.__set_slotType(map_type_desc.to_thrift()); + } + SlotDescriptor slot1(tslot1); + slot1._col_name = "col1"; + TTupleDescriptor ttuple_desc; + TupleDescriptor tuple_desc(ttuple_desc); + tuple_desc.add_slot(&slot1); + + std::unique_ptr orc_type( + orc::Type::buildTypeFromString("struct,struct>>")); + + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_name(&tuple_desc, + orc_type.get(), ans_node) + .ok()); + + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (file: COl1)\n" + " MapNode\n" + " Key:\n" + " ArrayNode\n" + " Element:\n" + " ScalarNode\n" + " Value:\n" + " StructNode\n" + " a (file: A)\n" + " ScalarNode\n" + " b (file: B)\n" + " ScalarNode\n"); } -TEST(TableSchemaChangeHelperTest, MixedColumnNameConversion) { - std::map file_schema = { - {1, "col1"}, {2, "col2_old"}, {3, "col3"}, {4, "col4_old"}}; - - std::unordered_map table_id_to_name = { - {1, "col1"}, {2, "col2_new"}, {3, "col3"}, {4, "col4_new"}, {5, "col5"}}; - - std::vector read_cols = {"col1", "col2_new", "col3", "col4_new", "col5"}; - - std::unordered_map col_ranges; - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); - - ASSERT_TRUE(helper.has_schema_change()); - Block before_block = - create_test_block({"col1", "col2_new", "col3", "col4_new", "col5", "extra_col"}); - - ASSERT_TRUE(helper.get_next_block_before(&before_block).ok()); +TEST(MockTableSchemaChangeHelper, NestedArrayStruct) { + // struct>>> - ASSERT_EQ(before_block.get_by_position(0).name, "col1"); - ASSERT_EQ(before_block.get_by_position(1).name, "col2_old"); - ASSERT_EQ(before_block.get_by_position(2).name, "col3"); - ASSERT_EQ(before_block.get_by_position(3).name, "col4_old"); - ASSERT_EQ(before_block.get_by_position(4).name, "col5"); - ASSERT_EQ(before_block.get_by_position(5).name, "extra_col"); + TSlotDescriptor tslot1; + { + TypeDescriptor array_type_desc(TYPE_ARRAY); - Block after_block = - create_test_block({"col1", "col2_old", "col3", "col4_old", "col5", "extra_col"}); - - ASSERT_TRUE(helper.get_next_block_after(&after_block).ok()); - ASSERT_EQ(after_block.get_by_position(0).name, "col1"); - ASSERT_EQ(after_block.get_by_position(1).name, "col2_new"); - ASSERT_EQ(after_block.get_by_position(2).name, "col3"); - ASSERT_EQ(after_block.get_by_position(3).name, "col4_new"); - ASSERT_EQ(after_block.get_by_position(4).name, "col5"); - ASSERT_EQ(after_block.get_by_position(5).name, "extra_col"); + TypeDescriptor struct_type_desc2(TYPE_STRUCT); + struct_type_desc2.add_sub_type(TYPE_BIGINT, "a", true); + TypeDescriptor array_type_desc2(TYPE_ARRAY); + array_type_desc2.add_sub_type(TYPE_BIGINT, true); + struct_type_desc2.add_sub_type(array_type_desc2, "b", true); + array_type_desc.add_sub_type(struct_type_desc2); + tslot1.__set_slotType(array_type_desc.to_thrift()); + } + SlotDescriptor slot1(tslot1); + slot1._col_name = "col1"; + + TTupleDescriptor ttuple_desc; + TupleDescriptor tuple_desc(ttuple_desc); + tuple_desc.add_slot(&slot1); + + std::unique_ptr orc_type( + orc::Type::buildTypeFromString("struct>>>")); + + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_name(&tuple_desc, + orc_type.get(), ans_node) + .ok()); + + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (file: coL1)\n" + " ArrayNode\n" + " Element:\n" + " StructNode\n" + " a (file: a)\n" + " ScalarNode\n" + " b (file: B)\n" + " ArrayNode\n" + " Element:\n" + " ScalarNode\n"); } -TEST(TableSchemaChangeHelperTest, EmptyAndSingleColumnBlocks) { - std::map file_schema = {{1, "col1"}, {2, "col2_old"}}; - - std::unordered_map table_id_to_name = {{1, "col1"}, {2, "col2_new"}}; - - std::vector read_cols = {"col1", "col2_new"}; - std::unordered_map col_ranges; - - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); - ASSERT_TRUE(helper.has_schema_change()); +TEST(MockTableSchemaChangeHelper, NestedMapStruct) { + // struct>>> - Block empty_block; - ASSERT_TRUE(helper.get_next_block_before(&empty_block).ok()); - ASSERT_TRUE(helper.get_next_block_after(&empty_block).ok()); - ASSERT_EQ(empty_block.columns(), 0); + TSlotDescriptor tslot1; + { + TypeDescriptor map_type_desc(TYPE_MAP); - Block single_block1 = create_test_block({"col1"}); - ASSERT_TRUE(helper.get_next_block_before(&single_block1).ok()); - ASSERT_EQ(single_block1.get_by_position(0).name, "col1"); + TypeDescriptor struct_type_desc2(TYPE_STRUCT); + struct_type_desc2.add_sub_type(TYPE_BIGINT, "a", true); + TypeDescriptor array_type_desc2(TYPE_MAP); + array_type_desc2.add_sub_type(TYPE_BIGINT); + array_type_desc2.add_sub_type(TYPE_BIGINT); - ASSERT_TRUE(helper.get_next_block_after(&single_block1).ok()); - ASSERT_EQ(single_block1.get_by_position(0).name, "col1"); + struct_type_desc2.add_sub_type(array_type_desc2, "b", true); - Block single_block2 = create_test_block({"col2_new"}); - ASSERT_TRUE(helper.get_next_block_before(&single_block2).ok()); - ASSERT_EQ(single_block2.get_by_position(0).name, "col2_old"); + map_type_desc.add_sub_type(TYPE_BIGINT); + map_type_desc.add_sub_type(struct_type_desc2); - Block single_block3 = create_test_block({"col2_old"}); - ASSERT_TRUE(helper.get_next_block_after(&single_block3).ok()); - ASSERT_EQ(single_block3.get_by_position(0).name, "col2_new"); + tslot1.__set_slotType(map_type_desc.to_thrift()); + } + SlotDescriptor slot1(tslot1); + slot1._col_name = "col1"; + + TTupleDescriptor ttuple_desc; + TupleDescriptor tuple_desc(ttuple_desc); + tuple_desc.add_slot(&slot1); + + std::unique_ptr orc_type( + orc::Type::buildTypeFromString("struct>>>")); + + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_name(&tuple_desc, + orc_type.get(), ans_node) + .ok()); + + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (file: col1)\n" + " MapNode\n" + " Key:\n" + " ScalarNode\n" + " Value:\n" + " StructNode\n" + " a (not exists)\n" + " b (file: b)\n" + " MapNode\n" + " Key:\n" + " ScalarNode\n" + " Value:\n" + " ScalarNode\n"); } -TEST(TableSchemaChangeHelperTest, ColumnOrderChange) { - std::map file_schema = {{1, "col1"}, {2, "col2_old"}, {3, "col3_old"}}; - std::unordered_map table_id_to_name = { - {1, "col1"}, {2, "col2_new"}, {3, "col3_new"}}; - std::vector read_cols = {"col1", "col2_new", "col3_new"}; - std::unordered_map col_ranges; - MockTableSchemaChangeHelper helper(file_schema); - ASSERT_TRUE(helper.init_schema_info(read_cols, table_id_to_name, &col_ranges).ok()); - - ASSERT_TRUE(helper.has_schema_change()); - - Block before_block = create_test_block({"col3_new", "col1", "col2_new"}); - ASSERT_TRUE(helper.get_next_block_before(&before_block).ok()); +TEST(MockTableSchemaChangeHelper, TableFieldIdNestedArrayStruct) { + // struct>>> + schema::external::TStructField table_schema; + { + auto col1_field = std::make_shared(); + col1_field->name = "col1"; + col1_field->id = 1; + col1_field->type.type = TPrimitiveType::ARRAY; + + auto item_field = std::make_shared(); + item_field->type.type = TPrimitiveType::STRUCT; + schema::external::TStructField struct_field; + { + auto a_field = std::make_shared(); + a_field->name = "a"; + a_field->id = 2; + a_field->type.type = TPrimitiveType::INT; + schema::external::TFieldPtr a_ptr; + a_ptr.field_ptr = a_field; + struct_field.fields.emplace_back(a_ptr); + } + { + auto b_field = std::make_shared(); + b_field->name = "b"; + b_field->id = 3; + b_field->type.type = TPrimitiveType::ARRAY; + + { + auto b_element_filed = std::make_shared(); + b_field->nestedField.array_field.item_field.field_ptr = b_element_filed; + } + schema::external::TFieldPtr b_ptr; + b_ptr.field_ptr = b_field; + struct_field.fields.emplace_back(b_ptr); + } + item_field->nestedField.struct_field = struct_field; + col1_field->nestedField.array_field.item_field.field_ptr = item_field; + schema::external::TFieldPtr col1_ptr; + col1_ptr.field_ptr = col1_field; + table_schema.fields.emplace_back(col1_ptr); + } - ASSERT_EQ(before_block.get_by_position(0).name, "col3_old"); - ASSERT_EQ(before_block.get_by_position(1).name, "col1"); - ASSERT_EQ(before_block.get_by_position(2).name, "col2_old"); + schema::external::TStructField file_schema = table_schema; + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_table_field_id( + table_schema, file_schema, ans_node) + .ok()); + + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col1 (file: col1)\n" + " ArrayNode\n" + " Element:\n" + " StructNode\n" + " a (file: a)\n" + " ScalarNode\n" + " b (file: b)\n" + " ArrayNode\n" + " Element:\n" + " ScalarNode\n"); +} - Block after_block = create_test_block({"col3_old", "col1", "col2_old"}); - ASSERT_TRUE(helper.get_next_block_after(&after_block).ok()); +TEST(MockTableSchemaChangeHelper, OrcFieldIdNestedStructMap) { + // struct, b:struct>>> + schema::external::TStructField table_schema; + { + auto col1_field = std::make_shared(); + col1_field->name = "col111111"; + col1_field->id = 1; + col1_field->type.type = TPrimitiveType::STRUCT; + schema::external::TStructField struct_field; + { + auto a_field = std::make_shared(); + a_field->name = "xxxxxx"; + a_field->id = 2; + a_field->type.type = TPrimitiveType::MAP; + { + schema::external::TMapField map_field; + + auto key_field = std::make_shared(); + key_field->type.type = TPrimitiveType::INT; + auto value_field = std::make_shared(); + value_field->type.type = TPrimitiveType::INT; + map_field.key_field.field_ptr = key_field; + map_field.value_field.field_ptr = value_field; + a_field->nestedField.map_field = map_field; + } + schema::external::TFieldPtr a_ptr; + a_ptr.field_ptr = a_field; + struct_field.fields.emplace_back(a_ptr); + } + { + auto b_field = std::make_shared(); + b_field->name = "AAAAAAA"; + b_field->id = 3; + b_field->type.type = TPrimitiveType::STRUCT; + + schema::external::TStructField nested_struct_field; + { + auto c_field = std::make_shared(); + c_field->name = "d"; + c_field->id = 4; + c_field->type.type = TPrimitiveType::INT; + schema::external::TFieldPtr c_ptr; + c_ptr.field_ptr = c_field; + nested_struct_field.fields.emplace_back(c_ptr); + } + { + auto d_field = std::make_shared(); + d_field->name = "CCCCCCCCC"; + d_field->id = 5; + d_field->type.type = TPrimitiveType::MAP; + { + schema::external::TMapField map_field; + + auto key_field = std::make_shared(); + key_field->type.type = TPrimitiveType::INT; + auto value_field = std::make_shared(); + value_field->type.type = TPrimitiveType::INT; + map_field.key_field.field_ptr = key_field; + map_field.value_field.field_ptr = value_field; + d_field->nestedField.map_field = map_field; + } + schema::external::TFieldPtr d_ptr; + d_ptr.field_ptr = d_field; + nested_struct_field.fields.emplace_back(d_ptr); + } + b_field->nestedField.struct_field = nested_struct_field; + schema::external::TFieldPtr b_ptr; + b_ptr.field_ptr = b_field; + struct_field.fields.emplace_back(b_ptr); + } + col1_field->nestedField.struct_field = struct_field; + schema::external::TFieldPtr col1_ptr; + col1_ptr.field_ptr = col1_field; + table_schema.fields.emplace_back(col1_ptr); + } - ASSERT_EQ(after_block.get_by_position(0).name, "col3_new"); - ASSERT_EQ(after_block.get_by_position(1).name, "col1"); - ASSERT_EQ(after_block.get_by_position(2).name, "col2_new"); + std::unique_ptr orc_type(orc::Type::buildTypeFromString( + "struct,b:struct>>>")); + const auto& attribute = IcebergOrcReader::ICEBERG_ORC_ATTRIBUTE; + orc_type->getSubtype(0)->setAttribute(attribute, "1"); + orc_type->getSubtype(0)->getSubtype(0)->setAttribute(attribute, "2"); + orc_type->getSubtype(0)->getSubtype(1)->setAttribute(attribute, "3"); + orc_type->getSubtype(0)->getSubtype(1)->getSubtype(0)->setAttribute(attribute, "4"); + orc_type->getSubtype(0)->getSubtype(1)->getSubtype(1)->setAttribute(attribute, "5"); + + bool exist_field_id = true; + std::shared_ptr ans_node = nullptr; + ASSERT_TRUE(TableSchemaChangeHelper::BuildTableInfoUtil::by_orc_field_id( + table_schema, orc_type.get(), attribute, ans_node, exist_field_id) + .ok()); + + ASSERT_TRUE(exist_field_id); + std::cout << TableSchemaChangeHelper::debug(ans_node) << "\n"; + + ASSERT_EQ(TableSchemaChangeHelper::debug(ans_node), + "StructNode\n" + " col111111 (file: col1)\n" + " StructNode\n" + " AAAAAAA (file: b)\n" + " StructNode\n" + " CCCCCCCCC (file: d)\n" + " MapNode\n" + " Key:\n" + " ScalarNode\n" + " Value:\n" + " ScalarNode\n" + " d (file: c)\n" + " ScalarNode\n" + " xxxxxx (file: a)\n" + " MapNode\n" + " Key:\n" + " ScalarNode\n" + " Value:\n" + " ScalarNode\n"); } } // namespace doris::vectorized diff --git a/be/test/vec/exec/orc/orc_reader_fill_data_test.cpp b/be/test/vec/exec/orc/orc_reader_fill_data_test.cpp index d896419a338199..dcdc07349b3b4d 100644 --- a/be/test/vec/exec/orc/orc_reader_fill_data_test.cpp +++ b/be/test/vec/exec/orc/orc_reader_fill_data_test.cpp @@ -38,6 +38,9 @@ class OrcReaderFillDataTest : public ::testing::Test { void SetUp() override {} void TearDown() override {} + + std::shared_ptr const_node = + std::make_shared(); }; std::unique_ptr create_long_batch(size_t size, @@ -83,7 +86,7 @@ TEST_F(OrcReaderFillDataTest, TestFillLongColumn) { MutableColumnPtr xx = column->assume_mutable(); Status status = reader->_fill_doris_data_column( - "test_long", xx, data_type, orc_type_ptr.get(), batch.get(), values.size()); + "test_long", xx, data_type, const_node, orc_type_ptr.get(), batch.get(), values.size()); ASSERT_TRUE(status.ok()); ASSERT_EQ(column->size(), values.size()); @@ -108,8 +111,9 @@ TEST_F(OrcReaderFillDataTest, TestFillLongColumnWithNull) { MutableColumnPtr xx = column->assume_mutable(); - Status status = reader->_fill_doris_data_column( - "test_long_with_null", xx, data_type, orc_type_ptr.get(), batch.get(), values.size()); + Status status = + reader->_fill_doris_data_column("test_long_with_null", xx, data_type, const_node, + orc_type_ptr.get(), batch.get(), values.size()); ASSERT_TRUE(status.ok()); ASSERT_EQ(column->size(), values.size()); @@ -165,8 +169,9 @@ TEST_F(OrcReaderFillDataTest, ComplexTypeConversionTest) { std::vector {"col1"}); MutableColumnPtr doris_column = doris_struct_type->create_column()->assume_mutable(); - Status status = reader->_fill_doris_data_column( - "test", doris_column, doris_struct_type, type.get(), structBatch, rowCount); + Status status = reader->_fill_doris_data_column("test", doris_column, + doris_struct_type, const_node, + type.get(), structBatch, rowCount); ASSERT_TRUE(status.ok()); std::string line; @@ -250,8 +255,9 @@ TEST_F(OrcReaderFillDataTest, ComplexTypeConversionTest) { std::vector {"col1", "col2"}); MutableColumnPtr doris_column = doris_struct_type->create_column()->assume_mutable(); - Status status = reader->_fill_doris_data_column( - "test", doris_column, doris_struct_type, type.get(), &structBatch, rowCount); + Status status = reader->_fill_doris_data_column("test", doris_column, + doris_struct_type, const_node, + type.get(), &structBatch, rowCount); ASSERT_TRUE(status.ok()); @@ -335,8 +341,9 @@ TEST_F(OrcReaderFillDataTest, ComplexTypeConversionTest) { MutableColumnPtr doris_column = doris_struct_type->create_column()->assume_mutable(); reader->_decimal_scale_params.resize(0); reader->_decimal_scale_params_index = 0; - Status status = reader->_fill_doris_data_column( - "test", doris_column, doris_struct_type, type.get(), structBatch, rowCount); + Status status = reader->_fill_doris_data_column("test", doris_column, + doris_struct_type, const_node, + type.get(), structBatch, rowCount); ASSERT_TRUE(status.ok()); @@ -446,8 +453,9 @@ TEST_F(OrcReaderFillDataTest, ComplexTypeConversionTest) { std::make_shared()); MutableColumnPtr doris_column = doris_struct_type->create_column()->assume_mutable(); - Status status = reader->_fill_doris_data_column( - "test", doris_column, doris_struct_type, type.get(), &mapBatch, rowCount); + Status status = + reader->_fill_doris_data_column("test", doris_column, doris_struct_type, + const_node, type.get(), &mapBatch, rowCount); ASSERT_TRUE(status.ok()); diff --git a/be/test/vec/exec/orc/orc_reader_init_column_test.cpp b/be/test/vec/exec/orc/orc_reader_init_column_test.cpp index 44cc9cdfc59899..841b7fb813e5ef 100644 --- a/be/test/vec/exec/orc/orc_reader_init_column_test.cpp +++ b/be/test/vec/exec/orc/orc_reader_init_column_test.cpp @@ -59,139 +59,13 @@ TEST_F(OrcReaderInitColumnTest, InitReadColumn) { std::vector tmp; tmp.emplace_back("col1"); - reader->_column_names = &tmp; + reader->_table_column_names = &tmp; Status st = reader->_init_read_columns(); std::cout << "st =" << st << "\n"; std::list ans; ans.emplace_back("col1"); - ASSERT_EQ(ans, reader->_read_cols); - } - - { - using namespace orc; - size_t rowCount = 10; - MemoryOutputStream memStream(100 * 1024 * 1024); - MemoryPool* pool = getDefaultPool(); - auto type = std::unique_ptr(Type::buildTypeFromString("struct")); - WriterOptions options; - options.setMemoryPool(pool); - auto writer = createWriter(*type, &memStream, options); - auto batch = writer->createRowBatch(rowCount); - writer->add(*batch); - writer->close(); - - auto inStream = - std::make_unique(memStream.getData(), memStream.getLength()); - ReaderOptions readerOptions; - readerOptions.setMemoryPool(*pool); - auto orc_reader = createReader(std::move(inStream), readerOptions); - - TFileScanRangeParams params; - params.slot_name_to_schema_pos.insert({"xxxxx", 0}); - params.__isset.slot_name_to_schema_pos = true; - TFileRangeDesc range; - auto reader = OrcReader::create_unique(params, range, "", nullptr, true); - reader->_reader = std::move(orc_reader); - reader->_is_hive1_orc_or_use_idx = true; - std::vector column_names; - column_names.emplace_back("xxxxx"); - - reader->_column_names = &column_names; - Status st = reader->_init_read_columns(); - - std::cout << "st =" << st << "\n"; - - std::list ans; - ans.emplace_back("col1"); - ASSERT_EQ(ans, reader->_read_cols); - } - { - using namespace orc; - size_t rowCount = 10; - MemoryOutputStream memStream(100 * 1024 * 1024); - MemoryPool* pool = getDefaultPool(); - auto type = std::unique_ptr( - Type::buildTypeFromString("struct<_col0:int,_col1:int,_col2:bigint>")); - WriterOptions options; - options.setMemoryPool(pool); - auto writer = createWriter(*type, &memStream, options); - auto batch = writer->createRowBatch(rowCount); - writer->add(*batch); - writer->close(); - - auto inStream = - std::make_unique(memStream.getData(), memStream.getLength()); - ReaderOptions readerOptions; - readerOptions.setMemoryPool(*pool); - auto orc_reader = createReader(std::move(inStream), readerOptions); - - TFileScanRangeParams params; - params.slot_name_to_schema_pos.insert({"a", 0}); - params.slot_name_to_schema_pos.insert({"b", 1}); - params.slot_name_to_schema_pos.insert({"c", 2}); - - params.__isset.slot_name_to_schema_pos = true; - TFileRangeDesc range; - auto reader = OrcReader::create_unique(params, range, "", nullptr, true); - reader->_reader = std::move(orc_reader); - std::vector column_names; - column_names.emplace_back("b"); - column_names.emplace_back("c"); - - reader->_column_names = &column_names; - Status st = reader->_init_read_columns(); - - std::list ans; - ans.emplace_back("_col1"); - ans.emplace_back("_col2"); - ASSERT_EQ(ans, reader->_read_cols); - } - - { - using namespace orc; - auto acid_type = createStructType(); - acid_type->addStructField("operation", createPrimitiveType(orc::TypeKind::INT)); - acid_type->addStructField("originalTransaction", createPrimitiveType(orc::TypeKind::LONG)); - acid_type->addStructField("bucket", createPrimitiveType(orc::TypeKind::INT)); - acid_type->addStructField("rowId", createPrimitiveType(orc::TypeKind::LONG)); - acid_type->addStructField("currentTransaction", createPrimitiveType(orc::TypeKind::LONG)); - auto row_type = createStructType(); - row_type->addStructField("CoL1", createPrimitiveType(orc::TypeKind::LONG)); - row_type->addStructField("col2", createPrimitiveType(orc::TypeKind::LONG)); - row_type->addStructField("colUMN3", createPrimitiveType(orc::TypeKind::LONG)); - acid_type->addStructField("row", std::move(row_type)); - - size_t rowCount = 10; - MemoryOutputStream memStream(100 * 1024 * 1024); - MemoryPool* pool = getDefaultPool(); - WriterOptions options; - options.setMemoryPool(pool); - auto writer = createWriter(*acid_type, &memStream, options); - auto batch = writer->createRowBatch(rowCount); - writer->add(*batch); - writer->close(); - - auto inStream = - std::make_unique(memStream.getData(), memStream.getLength()); - ReaderOptions readerOptions; - readerOptions.setMemoryPool(*pool); - auto orc_reader = createReader(std::move(inStream), readerOptions); - - TFileScanRangeParams params; - TFileRangeDesc range; - auto reader = OrcReader::create_unique(params, range, "", nullptr, true); - reader->_reader = std::move(orc_reader); - std::vector column_names; - column_names.emplace_back("col1"); - column_names.emplace_back("column3"); - reader->_column_names = &column_names; - reader->_is_acid = true; - Status st = reader->_init_read_columns(); - - std::list ans; - ans.emplace_back("row.CoL1"); - ans.emplace_back("row.colUMN3"); - ASSERT_EQ(ans, reader->_read_cols); + ASSERT_EQ(ans, reader->_read_file_cols); + ASSERT_EQ(ans, reader->_read_table_cols); } } @@ -284,7 +158,7 @@ TEST_F(OrcReaderInitColumnTest, RemoveAcidTest) { acid_type->addStructField("row", std::move(row_type)); // Verify that after removing ACID we get the type of the row field - const orc::Type& removed_type = _reader->_remove_acid(*acid_type); + const orc::Type& removed_type = _reader->remove_acid(*acid_type); ASSERT_EQ(removed_type.getKind(), orc::TypeKind::STRUCT); ASSERT_EQ(removed_type.getSubtypeCount(), 2); // id and name fields ASSERT_EQ(removed_type.getFieldName(0), "id"); @@ -298,7 +172,7 @@ TEST_F(OrcReaderInitColumnTest, RemoveAcidTest) { normal_type->addStructField("field1", createPrimitiveType(orc::TypeKind::INT)); normal_type->addStructField("field2", createPrimitiveType(orc::TypeKind::STRING)); - const orc::Type& result_type = _reader->_remove_acid(*normal_type); + const orc::Type& result_type = _reader->remove_acid(*normal_type); ASSERT_EQ(&result_type, normal_type.get()); // Should return the same type ASSERT_EQ(result_type.getSubtypeCount(), 2); ASSERT_EQ(result_type.getFieldName(0), "field1"); @@ -308,7 +182,7 @@ TEST_F(OrcReaderInitColumnTest, RemoveAcidTest) { // 3. Test primitive types (non-struct) remain unchanged { auto int_type = createPrimitiveType(orc::TypeKind::INT); - const orc::Type& result_type = _reader->_remove_acid(*int_type); + const orc::Type& result_type = _reader->remove_acid(*int_type); ASSERT_EQ(&result_type, int_type.get()); ASSERT_EQ(result_type.getKind(), orc::TypeKind::INT); } @@ -341,7 +215,7 @@ TEST_F(OrcReaderInitColumnTest, RemoveAcidTest) { acid_type->addStructField("row", std::move(row_type)); // Verify structure after removing ACID - const orc::Type& removed_type = _reader->_remove_acid(*acid_type); + const orc::Type& removed_type = _reader->remove_acid(*acid_type); ASSERT_EQ(removed_type.getKind(), orc::TypeKind::STRUCT); ASSERT_EQ(removed_type.getSubtypeCount(), 3); // id, tags, properties ASSERT_EQ(removed_type.getFieldName(0), "id"); diff --git a/be/test/vec/exec/orc_reader_test.cpp b/be/test/vec/exec/orc_reader_test.cpp index e27bdf08c9d5e7..ff7452ae625428 100644 --- a/be/test/vec/exec/orc_reader_test.cpp +++ b/be/test/vec/exec/orc_reader_test.cpp @@ -66,8 +66,8 @@ class OrcReaderTest : public testing::Test { range.start_offset = 0; range.size = 1293; auto reader = OrcReader::create_unique(params, range, "", nullptr, true); - auto status = reader->init_reader(&column_names, {}, nullptr, {}, false, tuple_desc, - &row_desc, nullptr, nullptr); + auto status = reader->init_reader(&column_names, nullptr, {}, false, tuple_desc, &row_desc, + nullptr, nullptr); EXPECT_TRUE(status.ok()); // deserialize expr diff --git a/docker/thirdparties/docker-compose/iceberg/entrypoint.sh.tpl b/docker/thirdparties/docker-compose/iceberg/entrypoint.sh.tpl index 4ccdf54b030cae..8c8471deb92c85 100644 --- a/docker/thirdparties/docker-compose/iceberg/entrypoint.sh.tpl +++ b/docker/thirdparties/docker-compose/iceberg/entrypoint.sh.tpl @@ -45,7 +45,7 @@ echo "Script iceberg total: {} executed in $EXECUTION_TIME1 seconds" START_TIME2=$(date +%s) find /mnt/scripts/create_preinstalled_scripts/paimon -name '*.sql' | sed 's|^|source |' | sed 's|$|;|'> paimon_total.sql -spark-sql --master spark://doris--spark-iceberg:7077 --conf spark.sql.extensions=org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions -f paimon_total.sql +spark-sql --packages org.apache.paimon:paimon-spark-3.5:1.0.1,org.apache.paimon:paimon-s3:1.0.1 --master spark://doris--spark-iceberg:7077 --conf spark.sql.extensions=org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions -f paimon_total.sql END_TIME2=$(date +%s) EXECUTION_TIME2=$((END_TIME2 - START_TIME2)) echo "Script paimon total: {} executed in $EXECUTION_TIME2 seconds" diff --git a/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl b/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl index 9ba8987d02bf97..f4df47f3cd3ffa 100644 --- a/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl +++ b/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl @@ -35,8 +35,6 @@ services: - ./data:/mnt/data - ./scripts:/mnt/scripts - ./spark-defaults.conf:/opt/spark/conf/spark-defaults.conf - - ./data/input/jars/paimon-spark-3.5-0.8.0.jar:/opt/spark/jars/paimon-spark-3.5-0.8.0.jar - - ./data/input/jars/paimon-s3-0.8.0.jar:/opt/spark/jars/paimon-s3-0.8.0.jar environment: - AWS_ACCESS_KEY_ID=admin - AWS_SECRET_ACCESS_KEY=password diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run01.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run01.sql index e5f70bc6366eb1..aeb4b69512645d 100644 --- a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run01.sql +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run01.sql @@ -1,4 +1,4 @@ - +create database if not exists demo.test_db; use demo.test_db; drop table if exists complex_orc_v1_schema_change; diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run02.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run02.sql index 500e3d200fbe1d..a55bb47a4ad574 100644 --- a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run02.sql +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run02.sql @@ -1,3 +1,4 @@ +create database if not exists demo.test_db; use demo.test_db; drop table if exists complex_orc_v2_schema_change; diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run03.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run03.sql index 0860783249ccb4..1d842b2483d29b 100644 --- a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run03.sql +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run03.sql @@ -1,3 +1,4 @@ +create database if not exists demo.test_db; use demo.test_db; drop table if exists complex_parquet_v2_schema_change; diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run04.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run04.sql index d5ba0048f39641..f29c6c7e2fc3df 100644 --- a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run04.sql +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run04.sql @@ -1,4 +1,4 @@ - +create database if not exists demo.test_db; use demo.test_db; drop table if exists complex_parquet_v1_schema_change; diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run06.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run06.sql index 3ac97c50099e10..80c41072344f40 100644 --- a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run06.sql +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run06.sql @@ -1,3 +1,4 @@ +create database if not exists demo.test_db; use demo.test_db; drop table if exists dangling_delete_after_write; diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run07.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run07.sql index 3c4131ba619f84..8c8403bdc1b048 100644 --- a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run07.sql +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run07.sql @@ -1,4 +1,4 @@ - +create database if not exists demo.test_db; use demo.test_db; CREATE TABLE iceberg_add_partition ( id INT, diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run08.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run08.sql index aa573e0af1e9fb..4524860155e2d5 100644 --- a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run08.sql +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run08.sql @@ -1,4 +1,4 @@ - +create database if not exists demo.test_db; use demo.test_db; CREATE TABLE no_partition ( id INT, diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run09.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run09.sql index c5795883c4defd..fc4080227e7f83 100644 --- a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run09.sql +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run09.sql @@ -1,3 +1,4 @@ +create database if not exists demo.test_db; use demo.test_db; create table schema_change_with_time_travel (c1 int); diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run10.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run10.sql index 1a3d844ef6027e..0d2b2240de4478 100644 --- a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run10.sql +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run10.sql @@ -1,4 +1,4 @@ - +create database if not exists demo.test_db; use demo.test_db; CREATE TABLE sc_drop_add_orc ( diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run16.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run16.sql new file mode 100644 index 00000000000000..e187406ad8732a --- /dev/null +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run16.sql @@ -0,0 +1,421 @@ +create database if not exists demo.test_db; +use demo.test_db; + + +-- Create the initial Iceberg table +CREATE TABLE iceberg_full_schema_change_parquet ( + id int, + map_column map < string, + struct < name: string, + age: int > >, + struct_column struct < city: string, + population: int >, + array_column array < struct < product: string, + price: float > > +) USING iceberg TBLPROPERTIES ('write.format.default' = 'parquet'); + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 0, + map('person0', struct('zero', 2)), + struct('cn', 1000000), + array(struct('Apple', 1.99), struct('Banana', 0.99)) + ); + +-- Schema Change 1: Add 'address' string to map_column's struct +ALTER TABLE + iceberg_full_schema_change_parquet +ADD + COLUMN map_column.value.address string; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 1, + map('person1', struct('Alice', 25, '123 Main St')), + struct('New York', 8000000), + array(struct('Apple', 1.99), struct('Banana', 0.99)) + ); + +-- Schema Change 2: Rename 'city' to 'location' in struct_column +ALTER TABLE + iceberg_full_schema_change_parquet RENAME COLUMN struct_column.city TO location; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 2, + map('person2', struct('Bob', 30, '456 Elm St')), + struct('Los Angeles', 4000000), + array(struct('Orange', 2.49), struct('Grape', 3.99)) + ); + +-- Schema Change 3: Delete 'price' from array_column's struct +ALTER TABLE + iceberg_full_schema_change_parquet DROP COLUMN array_column.element.price; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 3, + map('person3', struct('Charlie', 28, '789 Oak St')), + struct('Chicago', 2700000), + array(struct('Pear'), struct('Mango')) + ); + +ALTER TABLE + iceberg_full_schema_change_parquet +ALTER COLUMN + map_column.value.age first; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 4, + map('person4', struct(35, 'David', '101 Pine St')), + struct('Houston', 2300000), + array(struct('Kiwi'), struct('Pineapple')) + ); + +-- Schema Change 5: Add 'country' string to struct_column +ALTER TABLE + iceberg_full_schema_change_parquet +ADD + COLUMN struct_column.country string; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 5, + map('person5', struct(40, 'Eve', '202 Birch St')), + struct('Phoenix', 1600000, 'USA'), + array(struct('Lemon'), struct('Lime')) + ); + +-- Schema Change 6: Rename 'product' to 'item' in array_column's struct +ALTER TABLE + iceberg_full_schema_change_parquet RENAME COLUMN array_column.element.product TO item; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 6, + map('person6', struct(22, 'Frank', '303 Cedar St')), + struct('Philadelphia', 1500000, 'USA'), + array(struct('Watermelon'), struct('Strawberry')) + ); + +-- Schema Change 7: Delete 'address' from map_column's struct +ALTER TABLE + iceberg_full_schema_change_parquet DROP COLUMN map_column.value.address; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 7, + map('person7', struct(27, 'Grace')), + struct('San Antonio', 1400000, 'USA'), + array(struct('Blueberry'), struct('Raspberry')) + ); + +-- Schema Change 8: Add 'quantity' int to array_column's struct +ALTER TABLE + iceberg_full_schema_change_parquet +ADD + COLUMN array_column.element.quantity int; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 8, + map('person8', struct(32, 'Hank')), + struct('San Diego', 1300000, 'USA'), + array(struct('Cherry', 5), struct('Plum', 3)) + ); + +-- Schema Change 9: Swap 'location' and 'country' in struct_column +ALTER TABLE + iceberg_full_schema_change_parquet +ALTER COLUMN + struct_column.location +after + country; + +ALTER TABLE + iceberg_full_schema_change_parquet +ALTER COLUMN + struct_column.country first; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 9, + map('person9', struct(29, 'Ivy')), + struct('USA', 1200000, 'Dallas'), + array(struct('Peach', 4), struct('Apricot', 2)) + ); + +-- Schema Change 10: Rename 'name' to 'full_name' in map_column's struct +ALTER TABLE + iceberg_full_schema_change_parquet RENAME COLUMN map_column.value.name TO full_name; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 10, + map('person10', struct(26, 'Jack')), + struct('USA', 1100000, 'Austin'), + array(struct('Fig', 6), struct('Date', 7)) + ); + +-- Schema Change 11: Add 'gender' string to map_column's struct +ALTER TABLE + iceberg_full_schema_change_parquet +ADD + COLUMN map_column.value.gender string; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 11, + map('person11', struct(31, 'Karen', 'Female')), + struct('USA', 1000000, 'Seattle'), + array(struct('Coconut', 1), struct('Papaya', 2)) + ); + +-- Schema Change 12: Delete 'population' from struct_column +ALTER TABLE + iceberg_full_schema_change_parquet DROP COLUMN struct_column.population; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 12, + map('person12', struct(24, 'Leo', 'Male')), + struct('USA', 'Portland'), + array(struct('Guava', 3), struct('Lychee', 4)) + ); + +-- Schema Change 13: Add 'category' string to array_column's struct +ALTER TABLE + iceberg_full_schema_change_parquet +ADD + COLUMN array_column.element.category string; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 13, + map('person13', struct(33, 'Mona', 'Female')), + struct('USA', 'Denver'), + array( + struct('Avocado', 2, 'Fruit'), + struct('Tomato', 5, 'Vegetable') + ) + ); + +-- Schema Change 14: Rename 'location' to 'city' in struct_column +ALTER TABLE + iceberg_full_schema_change_parquet RENAME COLUMN struct_column.location TO city; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 14, + map('person14', struct(28, 'Nina', 'Female')), + struct('USA', 'Miami'), + array( + struct('Cucumber', 6, 'Vegetable'), + struct('Carrot', 7, 'Vegetable') + ) + ); + +alter table + iceberg_full_schema_change_parquet RENAME COLUMN map_column to new_map_column; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 15, + map('person15', struct(30, 'Emma Smith', 'Female')), + struct('USA', 'New York'), + array( + struct('Banana', 3, 'Fruit'), + struct('Potato', 8, 'Vegetable') + ) + ); + +alter table + iceberg_full_schema_change_parquet +ADD + COLUMN new_struct_column struct < a: struct < aa :int, + bb :string >, + b: struct < cc :string, + dd :int >, + c int >; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 16, + map('person16', struct(28, 'Liam Brown', 'Male')), + struct('UK', 'London'), + array( + struct('Bread', 2, 'Food'), + struct('Milk', 1, 'Dairy') + ), + struct( + struct(50, 'NestedBB'), + struct('NestedCC', 75), + 9 + ) + ); + +alter table + iceberg_full_schema_change_parquet rename column new_struct_column.a.aa to new_aa; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 17, + map('person17', struct(40, 'Olivia Davis', 'Female')), + struct('Australia', 'Sydney'), + array( + struct('Orange', 4, 'Fruit'), + struct('Broccoli', 6, 'Vegetable') + ), + struct( + struct(60, 'UpdatedBB'), + struct('UpdatedCC', 88), + 12 + ) + ); + +alter table + iceberg_full_schema_change_parquet rename column new_struct_column.a to new_a; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 18, + map('person18', struct(33, 'Noah Wilson', 'Male')), + struct('Germany', 'Berlin'), + array( + struct('Cheese', 2, 'Dairy'), + struct('Lettuce', 5, 'Vegetable') + ), + struct( + struct(70, 'NestedBB18'), + struct('NestedCC18', 95), + 15 + ) + ); + +alter table + iceberg_full_schema_change_parquet +ALTER COLUMN + new_struct_column.b first; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 19, + map('person19', struct(29, 'Ava Martinez', 'Female')), + struct('France', 'Paris'), + array( + struct('Strawberry', 12, 'Fruit'), + struct('Spinach', 7, 'Vegetable') + ), + struct( + struct('ReorderedCC', 101), + struct(85, 'ReorderedBB'), + 18 + ) + ); + +alter table + iceberg_full_schema_change_parquet rename column new_struct_column.b.dd to new_dd; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 20, + map('person20', struct(38, 'James Lee', 'Male')), + struct('Japan', 'Osaka'), + array( + struct('Mango', 6, 'Fruit'), + struct('Onion', 3, 'Vegetable') + ), + struct( + struct('FinalCC', 110), + struct(95, 'FinalBB'), + 21 + ) + ); + +alter table + iceberg_full_schema_change_parquet rename column new_struct_column to struct_column2; + +INSERT INTO + iceberg_full_schema_change_parquet +VALUES + ( + 21, + map('person21', struct(45, 'Sophia White', 'Female')), + struct('Italy', 'Rome'), + array( + struct('Pasta', 4, 'Food'), + struct('Olive', 9, 'Food') + ), + struct( + struct('ExampleCC', 120), + struct(100, 'ExampleBB'), + 25 + ) + ); + +-- spark-sql (iceberg)> select * from iceberg_full_schema_change_parquet order by id ; +-- 0 {"person0":{"age":2,"full_name":"zero","gender":null}} {"country":null,"city":"cn"} [{"item":"Apple","quantity":null,"category":null},{"item":"Banana","quantity":null,"category":null}] NULL +-- 1 {"person1":{"age":25,"full_name":"Alice","gender":null}} {"country":null,"city":"New York"} [{"item":"Apple","quantity":null,"category":null},{"item":"Banana","quantity":null,"category":null}] NULL +-- 2 {"person2":{"age":30,"full_name":"Bob","gender":null}} {"country":null,"city":"Los Angeles"} [{"item":"Orange","quantity":null,"category":null},{"item":"Grape","quantity":null,"category":null}] NULL +-- 3 {"person3":{"age":28,"full_name":"Charlie","gender":null}} {"country":null,"city":"Chicago"} [{"item":"Pear","quantity":null,"category":null},{"item":"Mango","quantity":null,"category":null}] NULL +-- 4 {"person4":{"age":35,"full_name":"David","gender":null}} {"country":null,"city":"Houston"} [{"item":"Kiwi","quantity":null,"category":null},{"item":"Pineapple","quantity":null,"category":null}] NULL +-- 5 {"person5":{"age":40,"full_name":"Eve","gender":null}} {"country":"USA","city":"Phoenix"} [{"item":"Lemon","quantity":null,"category":null},{"item":"Lime","quantity":null,"category":null}] NULL +-- 6 {"person6":{"age":22,"full_name":"Frank","gender":null}} {"country":"USA","city":"Philadelphia"} [{"item":"Watermelon","quantity":null,"category":null},{"item":"Strawberry","quantity":null,"category":null}] NULL +-- 7 {"person7":{"age":27,"full_name":"Grace","gender":null}} {"country":"USA","city":"San Antonio"} [{"item":"Blueberry","quantity":null,"category":null},{"item":"Raspberry","quantity":null,"category":null}] NULL +-- 8 {"person8":{"age":32,"full_name":"Hank","gender":null}} {"country":"USA","city":"San Diego"} [{"item":"Cherry","quantity":5,"category":null},{"item":"Plum","quantity":3,"category":null}] NULL +-- 9 {"person9":{"age":29,"full_name":"Ivy","gender":null}} {"country":"USA","city":"Dallas"} [{"item":"Peach","quantity":4,"category":null},{"item":"Apricot","quantity":2,"category":null}] NULL +-- 10 {"person10":{"age":26,"full_name":"Jack","gender":null}} {"country":"USA","city":"Austin"} [{"item":"Fig","quantity":6,"category":null},{"item":"Date","quantity":7,"category":null}] NULL +-- 11 {"person11":{"age":31,"full_name":"Karen","gender":"Female"}} {"country":"USA","city":"Seattle"} [{"item":"Coconut","quantity":1,"category":null},{"item":"Papaya","quantity":2,"category":null}] NULL +-- 12 {"person12":{"age":24,"full_name":"Leo","gender":"Male"}} {"country":"USA","city":"Portland"} [{"item":"Guava","quantity":3,"category":null},{"item":"Lychee","quantity":4,"category":null}] NULL +-- 13 {"person13":{"age":33,"full_name":"Mona","gender":"Female"}} {"country":"USA","city":"Denver"} [{"item":"Avocado","quantity":2,"category":"Fruit"},{"item":"Tomato","quantity":5,"category":"Vegetable"}] NULL +-- 14 {"person14":{"age":28,"full_name":"Nina","gender":"Female"}} {"country":"USA","city":"Miami"} [{"item":"Cucumber","quantity":6,"category":"Vegetable"},{"item":"Carrot","quantity":7,"category":"Vegetable"}] NULL +-- 15 {"person15":{"age":30,"full_name":"Emma Smith","gender":"Female"}} {"country":"USA","city":"New York"} [{"item":"Banana","quantity":3,"category":"Fruit"},{"item":"Potato","quantity":8,"category":"Vegetable"}] NULL +-- 16 {"person16":{"age":28,"full_name":"Liam Brown","gender":"Male"}} {"country":"UK","city":"London"} [{"item":"Bread","quantity":2,"category":"Food"},{"item":"Milk","quantity":1,"category":"Dairy"}] {"b":{"cc":"NestedCC","new_dd":75},"new_a":{"new_aa":50,"bb":"NestedBB"},"c":9} +-- 17 {"person17":{"age":40,"full_name":"Olivia Davis","gender":"Female"}} {"country":"Australia","city":"Sydney"} [{"item":"Orange","quantity":4,"category":"Fruit"},{"item":"Broccoli","quantity":6,"category":"Vegetable"}] {"b":{"cc":"UpdatedCC","new_dd":88},"new_a":{"new_aa":60,"bb":"UpdatedBB"},"c":12} +-- 18 {"person18":{"age":33,"full_name":"Noah Wilson","gender":"Male"}} {"country":"Germany","city":"Berlin"} [{"item":"Cheese","quantity":2,"category":"Dairy"},{"item":"Lettuce","quantity":5,"category":"Vegetable"}] {"b":{"cc":"NestedCC18","new_dd":95},"new_a":{"new_aa":70,"bb":"NestedBB18"},"c":15} +-- 19 {"person19":{"age":29,"full_name":"Ava Martinez","gender":"Female"}} {"country":"France","city":"Paris"} [{"item":"Strawberry","quantity":12,"category":"Fruit"},{"item":"Spinach","quantity":7,"category":"Vegetable"}] {"b":{"cc":"ReorderedCC","new_dd":101},"new_a":{"new_aa":85,"bb":"ReorderedBB"},"c":18} +-- 20 {"person20":{"age":38,"full_name":"James Lee","gender":"Male"}} {"country":"Japan","city":"Osaka"} [{"item":"Mango","quantity":6,"category":"Fruit"},{"item":"Onion","quantity":3,"category":"Vegetable"}] {"b":{"cc":"FinalCC","new_dd":110},"new_a":{"new_aa":95,"bb":"FinalBB"},"c":21} +-- 21 {"person21":{"age":45,"full_name":"Sophia White","gender":"Female"}} {"country":"Italy","city":"Rome"} [{"item":"Pasta","quantity":4,"category":"Food"},{"item":"Olive","quantity":9,"category":"Food"}] {"b":{"cc":"ExampleCC","new_dd":120},"new_a":{"new_aa":100,"bb":"ExampleBB"},"c":25} diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run17.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run17.sql new file mode 100644 index 00000000000000..9554892f497056 --- /dev/null +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run17.sql @@ -0,0 +1,421 @@ +create database if not exists demo.test_db; +use demo.test_db; + + +-- Create the initial Iceberg table +CREATE TABLE iceberg_full_schema_change_orc ( + id int, + map_column map < string, + struct < name: string, + age: int > >, + struct_column struct < city: string, + population: int >, + array_column array < struct < product: string, + price: float > > +) USING iceberg TBLPROPERTIES ('write.format.default' = 'orc'); + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 0, + map('person0', struct('zero', 2)), + struct('cn', 1000000), + array(struct('Apple', 1.99), struct('Banana', 0.99)) + ); + +-- Schema Change 1: Add 'address' string to map_column's struct +ALTER TABLE + iceberg_full_schema_change_orc +ADD + COLUMN map_column.value.address string; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 1, + map('person1', struct('Alice', 25, '123 Main St')), + struct('New York', 8000000), + array(struct('Apple', 1.99), struct('Banana', 0.99)) + ); + +-- Schema Change 2: Rename 'city' to 'location' in struct_column +ALTER TABLE + iceberg_full_schema_change_orc RENAME COLUMN struct_column.city TO location; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 2, + map('person2', struct('Bob', 30, '456 Elm St')), + struct('Los Angeles', 4000000), + array(struct('Orange', 2.49), struct('Grape', 3.99)) + ); + +-- Schema Change 3: Delete 'price' from array_column's struct +ALTER TABLE + iceberg_full_schema_change_orc DROP COLUMN array_column.element.price; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 3, + map('person3', struct('Charlie', 28, '789 Oak St')), + struct('Chicago', 2700000), + array(struct('Pear'), struct('Mango')) + ); + +ALTER TABLE + iceberg_full_schema_change_orc +ALTER COLUMN + map_column.value.age first; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 4, + map('person4', struct(35, 'David', '101 Pine St')), + struct('Houston', 2300000), + array(struct('Kiwi'), struct('Pineapple')) + ); + +-- Schema Change 5: Add 'country' string to struct_column +ALTER TABLE + iceberg_full_schema_change_orc +ADD + COLUMN struct_column.country string; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 5, + map('person5', struct(40, 'Eve', '202 Birch St')), + struct('Phoenix', 1600000, 'USA'), + array(struct('Lemon'), struct('Lime')) + ); + +-- Schema Change 6: Rename 'product' to 'item' in array_column's struct +ALTER TABLE + iceberg_full_schema_change_orc RENAME COLUMN array_column.element.product TO item; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 6, + map('person6', struct(22, 'Frank', '303 Cedar St')), + struct('Philadelphia', 1500000, 'USA'), + array(struct('Watermelon'), struct('Strawberry')) + ); + +-- Schema Change 7: Delete 'address' from map_column's struct +ALTER TABLE + iceberg_full_schema_change_orc DROP COLUMN map_column.value.address; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 7, + map('person7', struct(27, 'Grace')), + struct('San Antonio', 1400000, 'USA'), + array(struct('Blueberry'), struct('Raspberry')) + ); + +-- Schema Change 8: Add 'quantity' int to array_column's struct +ALTER TABLE + iceberg_full_schema_change_orc +ADD + COLUMN array_column.element.quantity int; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 8, + map('person8', struct(32, 'Hank')), + struct('San Diego', 1300000, 'USA'), + array(struct('Cherry', 5), struct('Plum', 3)) + ); + +-- Schema Change 9: Swap 'location' and 'country' in struct_column +ALTER TABLE + iceberg_full_schema_change_orc +ALTER COLUMN + struct_column.location +after + country; + +ALTER TABLE + iceberg_full_schema_change_orc +ALTER COLUMN + struct_column.country first; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 9, + map('person9', struct(29, 'Ivy')), + struct('USA', 1200000, 'Dallas'), + array(struct('Peach', 4), struct('Apricot', 2)) + ); + +-- Schema Change 10: Rename 'name' to 'full_name' in map_column's struct +ALTER TABLE + iceberg_full_schema_change_orc RENAME COLUMN map_column.value.name TO full_name; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 10, + map('person10', struct(26, 'Jack')), + struct('USA', 1100000, 'Austin'), + array(struct('Fig', 6), struct('Date', 7)) + ); + +-- Schema Change 11: Add 'gender' string to map_column's struct +ALTER TABLE + iceberg_full_schema_change_orc +ADD + COLUMN map_column.value.gender string; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 11, + map('person11', struct(31, 'Karen', 'Female')), + struct('USA', 1000000, 'Seattle'), + array(struct('Coconut', 1), struct('Papaya', 2)) + ); + +-- Schema Change 12: Delete 'population' from struct_column +ALTER TABLE + iceberg_full_schema_change_orc DROP COLUMN struct_column.population; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 12, + map('person12', struct(24, 'Leo', 'Male')), + struct('USA', 'Portland'), + array(struct('Guava', 3), struct('Lychee', 4)) + ); + +-- Schema Change 13: Add 'category' string to array_column's struct +ALTER TABLE + iceberg_full_schema_change_orc +ADD + COLUMN array_column.element.category string; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 13, + map('person13', struct(33, 'Mona', 'Female')), + struct('USA', 'Denver'), + array( + struct('Avocado', 2, 'Fruit'), + struct('Tomato', 5, 'Vegetable') + ) + ); + +-- Schema Change 14: Rename 'location' to 'city' in struct_column +ALTER TABLE + iceberg_full_schema_change_orc RENAME COLUMN struct_column.location TO city; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 14, + map('person14', struct(28, 'Nina', 'Female')), + struct('USA', 'Miami'), + array( + struct('Cucumber', 6, 'Vegetable'), + struct('Carrot', 7, 'Vegetable') + ) + ); + +alter table + iceberg_full_schema_change_orc RENAME COLUMN map_column to new_map_column; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 15, + map('person15', struct(30, 'Emma Smith', 'Female')), + struct('USA', 'New York'), + array( + struct('Banana', 3, 'Fruit'), + struct('Potato', 8, 'Vegetable') + ) + ); + +alter table + iceberg_full_schema_change_orc +ADD + COLUMN new_struct_column struct < a: struct < aa :int, + bb :string >, + b: struct < cc :string, + dd :int >, + c int >; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 16, + map('person16', struct(28, 'Liam Brown', 'Male')), + struct('UK', 'London'), + array( + struct('Bread', 2, 'Food'), + struct('Milk', 1, 'Dairy') + ), + struct( + struct(50, 'NestedBB'), + struct('NestedCC', 75), + 9 + ) + ); + +alter table + iceberg_full_schema_change_orc rename column new_struct_column.a.aa to new_aa; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 17, + map('person17', struct(40, 'Olivia Davis', 'Female')), + struct('Australia', 'Sydney'), + array( + struct('Orange', 4, 'Fruit'), + struct('Broccoli', 6, 'Vegetable') + ), + struct( + struct(60, 'UpdatedBB'), + struct('UpdatedCC', 88), + 12 + ) + ); + +alter table + iceberg_full_schema_change_orc rename column new_struct_column.a to new_a; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 18, + map('person18', struct(33, 'Noah Wilson', 'Male')), + struct('Germany', 'Berlin'), + array( + struct('Cheese', 2, 'Dairy'), + struct('Lettuce', 5, 'Vegetable') + ), + struct( + struct(70, 'NestedBB18'), + struct('NestedCC18', 95), + 15 + ) + ); + +alter table + iceberg_full_schema_change_orc +ALTER COLUMN + new_struct_column.b first; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 19, + map('person19', struct(29, 'Ava Martinez', 'Female')), + struct('France', 'Paris'), + array( + struct('Strawberry', 12, 'Fruit'), + struct('Spinach', 7, 'Vegetable') + ), + struct( + struct('ReorderedCC', 101), + struct(85, 'ReorderedBB'), + 18 + ) + ); + +alter table + iceberg_full_schema_change_orc rename column new_struct_column.b.dd to new_dd; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 20, + map('person20', struct(38, 'James Lee', 'Male')), + struct('Japan', 'Osaka'), + array( + struct('Mango', 6, 'Fruit'), + struct('Onion', 3, 'Vegetable') + ), + struct( + struct('FinalCC', 110), + struct(95, 'FinalBB'), + 21 + ) + ); + +alter table + iceberg_full_schema_change_orc rename column new_struct_column to struct_column2; + +INSERT INTO + iceberg_full_schema_change_orc +VALUES + ( + 21, + map('person21', struct(45, 'Sophia White', 'Female')), + struct('Italy', 'Rome'), + array( + struct('Pasta', 4, 'Food'), + struct('Olive', 9, 'Food') + ), + struct( + struct('ExampleCC', 120), + struct(100, 'ExampleBB'), + 25 + ) + ); + +-- spark-sql (iceberg)> select * from iceberg_full_schema_change_orc order by id ; +-- 0 {"person0":{"age":2,"full_name":"zero","gender":null}} {"country":null,"city":"cn"} [{"item":"Apple","quantity":null,"category":null},{"item":"Banana","quantity":null,"category":null}] NULL +-- 1 {"person1":{"age":25,"full_name":"Alice","gender":null}} {"country":null,"city":"New York"} [{"item":"Apple","quantity":null,"category":null},{"item":"Banana","quantity":null,"category":null}] NULL +-- 2 {"person2":{"age":30,"full_name":"Bob","gender":null}} {"country":null,"city":"Los Angeles"} [{"item":"Orange","quantity":null,"category":null},{"item":"Grape","quantity":null,"category":null}] NULL +-- 3 {"person3":{"age":28,"full_name":"Charlie","gender":null}} {"country":null,"city":"Chicago"} [{"item":"Pear","quantity":null,"category":null},{"item":"Mango","quantity":null,"category":null}] NULL +-- 4 {"person4":{"age":35,"full_name":"David","gender":null}} {"country":null,"city":"Houston"} [{"item":"Kiwi","quantity":null,"category":null},{"item":"Pineapple","quantity":null,"category":null}] NULL +-- 5 {"person5":{"age":40,"full_name":"Eve","gender":null}} {"country":"USA","city":"Phoenix"} [{"item":"Lemon","quantity":null,"category":null},{"item":"Lime","quantity":null,"category":null}] NULL +-- 6 {"person6":{"age":22,"full_name":"Frank","gender":null}} {"country":"USA","city":"Philadelphia"} [{"item":"Watermelon","quantity":null,"category":null},{"item":"Strawberry","quantity":null,"category":null}] NULL +-- 7 {"person7":{"age":27,"full_name":"Grace","gender":null}} {"country":"USA","city":"San Antonio"} [{"item":"Blueberry","quantity":null,"category":null},{"item":"Raspberry","quantity":null,"category":null}] NULL +-- 8 {"person8":{"age":32,"full_name":"Hank","gender":null}} {"country":"USA","city":"San Diego"} [{"item":"Cherry","quantity":5,"category":null},{"item":"Plum","quantity":3,"category":null}] NULL +-- 9 {"person9":{"age":29,"full_name":"Ivy","gender":null}} {"country":"USA","city":"Dallas"} [{"item":"Peach","quantity":4,"category":null},{"item":"Apricot","quantity":2,"category":null}] NULL +-- 10 {"person10":{"age":26,"full_name":"Jack","gender":null}} {"country":"USA","city":"Austin"} [{"item":"Fig","quantity":6,"category":null},{"item":"Date","quantity":7,"category":null}] NULL +-- 11 {"person11":{"age":31,"full_name":"Karen","gender":"Female"}} {"country":"USA","city":"Seattle"} [{"item":"Coconut","quantity":1,"category":null},{"item":"Papaya","quantity":2,"category":null}] NULL +-- 12 {"person12":{"age":24,"full_name":"Leo","gender":"Male"}} {"country":"USA","city":"Portland"} [{"item":"Guava","quantity":3,"category":null},{"item":"Lychee","quantity":4,"category":null}] NULL +-- 13 {"person13":{"age":33,"full_name":"Mona","gender":"Female"}} {"country":"USA","city":"Denver"} [{"item":"Avocado","quantity":2,"category":"Fruit"},{"item":"Tomato","quantity":5,"category":"Vegetable"}] NULL +-- 14 {"person14":{"age":28,"full_name":"Nina","gender":"Female"}} {"country":"USA","city":"Miami"} [{"item":"Cucumber","quantity":6,"category":"Vegetable"},{"item":"Carrot","quantity":7,"category":"Vegetable"}] NULL +-- 15 {"person15":{"age":30,"full_name":"Emma Smith","gender":"Female"}} {"country":"USA","city":"New York"} [{"item":"Banana","quantity":3,"category":"Fruit"},{"item":"Potato","quantity":8,"category":"Vegetable"}] NULL +-- 16 {"person16":{"age":28,"full_name":"Liam Brown","gender":"Male"}} {"country":"UK","city":"London"} [{"item":"Bread","quantity":2,"category":"Food"},{"item":"Milk","quantity":1,"category":"Dairy"}] {"b":{"cc":"NestedCC","new_dd":75},"new_a":{"new_aa":50,"bb":"NestedBB"},"c":9} +-- 17 {"person17":{"age":40,"full_name":"Olivia Davis","gender":"Female"}} {"country":"Australia","city":"Sydney"} [{"item":"Orange","quantity":4,"category":"Fruit"},{"item":"Broccoli","quantity":6,"category":"Vegetable"}] {"b":{"cc":"UpdatedCC","new_dd":88},"new_a":{"new_aa":60,"bb":"UpdatedBB"},"c":12} +-- 18 {"person18":{"age":33,"full_name":"Noah Wilson","gender":"Male"}} {"country":"Germany","city":"Berlin"} [{"item":"Cheese","quantity":2,"category":"Dairy"},{"item":"Lettuce","quantity":5,"category":"Vegetable"}] {"b":{"cc":"NestedCC18","new_dd":95},"new_a":{"new_aa":70,"bb":"NestedBB18"},"c":15} +-- 19 {"person19":{"age":29,"full_name":"Ava Martinez","gender":"Female"}} {"country":"France","city":"Paris"} [{"item":"Strawberry","quantity":12,"category":"Fruit"},{"item":"Spinach","quantity":7,"category":"Vegetable"}] {"b":{"cc":"ReorderedCC","new_dd":101},"new_a":{"new_aa":85,"bb":"ReorderedBB"},"c":18} +-- 20 {"person20":{"age":38,"full_name":"James Lee","gender":"Male"}} {"country":"Japan","city":"Osaka"} [{"item":"Mango","quantity":6,"category":"Fruit"},{"item":"Onion","quantity":3,"category":"Vegetable"}] {"b":{"cc":"FinalCC","new_dd":110},"new_a":{"new_aa":95,"bb":"FinalBB"},"c":21} +-- 21 {"person21":{"age":45,"full_name":"Sophia White","gender":"Female"}} {"country":"Italy","city":"Rome"} [{"item":"Pasta","quantity":4,"category":"Food"},{"item":"Olive","quantity":9,"category":"Food"}] {"b":{"cc":"ExampleCC","new_dd":120},"new_a":{"new_aa":100,"bb":"ExampleBB"},"c":25} diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run01.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run01.sql index 13fd983744a22f..422855d98721da 100644 --- a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run01.sql +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run01.sql @@ -42,12 +42,21 @@ insert into two_partition values(4,'2038-01-01','sh'); insert into two_partition values(5,'2038-01-02','bj'); drop table if exists null_partition; +-- CREATE TABLE null_partition ( +-- id BIGINT, +-- region STRING +-- ) PARTITIONED BY (region) TBLPROPERTIES ( +-- 'primary-key' = 'region,id', +-- 'bucket'=10, +-- 'file.format'='orc' +-- ); +-- in paimon 1.0.1 ,primary-key is `not null`. CREATE TABLE null_partition ( id BIGINT, region STRING ) PARTITIONED BY (region) TBLPROPERTIES ( - 'primary-key' = 'region,id', - 'bucket'=10, + 'primary-key' = 'id', + 'bucket'='-1', 'file.format'='orc' ); -- null NULL "null" all will be in partition [null] diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run05.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run05.sql new file mode 100644 index 00000000000000..629977946cac00 --- /dev/null +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run05.sql @@ -0,0 +1,406 @@ +use paimon; + +create database if not exists test_paimon_schema_change; + +use test_paimon_schema_change; + + + +CREATE TABLE paimon_full_schema_change_orc ( + id int, + map_column map>, + struct_column struct, + array_column array> +) +USING paimon +TBLPROPERTIES ("file.format" = "orc"); +; + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 0, + map('person0', struct('zero', 2)), + struct('cn', 1000000), + array(struct('Apple', 1.99), struct('Banana', 0.99)) +); + + +-- Schema Change 1: Add 'address' string to map_column's struct +ALTER TABLE paimon_full_schema_change_orc ADD COLUMN map_column.value.address string; +INSERT INTO paimon_full_schema_change_orc VALUES ( + 1, + map('person1', struct('Alice', 25, '123 Main St')), + struct('New York', 8000000), + array(struct('Apple', 1.99), struct('Banana', 0.99)) +); + + +ALTER TABLE paimon_full_schema_change_orc RENAME COLUMN map_column.value.name TO full_name; + + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 2, + map('person2', struct('Bob Smith', 30, '456 Oak Ave')), + struct('Los Angeles', 4000000), + array(struct('Orange', 2.49), struct('Grapes', 3.99)) +); + + + +ALTER TABLE paimon_full_schema_change_orc RENAME COLUMN struct_column.city TO location; + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 3, + map('person3', struct('Charlie', 28, '789 Pine Rd')), + struct('Chicago', 2700000), + array(struct('Mango', 2.99), struct('Peach', 1.49)) +); + + + +ALTER TABLE paimon_full_schema_change_orc DROP COLUMN struct_column.population; + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 4, + map('person4', struct('Diana Green', 35, '321 Elm St')), + struct('San Francisco'), + array(struct('Strawberry', 4.29), struct('Blueberry', 5.19)) +); + + + +ALTER TABLE paimon_full_schema_change_orc ADD COLUMN struct_column.population bigint; + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 5, + map('person5', struct('Edward Black', 42, '987 Willow Ln')), + struct('Seattle', 750000), + array(struct('Kiwi', 1.79), struct('Pineapple', 3.49)) +); + + +ALTER TABLE paimon_full_schema_change_orc RENAME COLUMN array_column.element.product to new_product; + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 6, + map('person6', struct('Fiona Lake', 29, '654 Birch Ln')), + struct('Austin', 950000), + array(struct('Pineapple', 3.99), struct('Kiwi', 2.59)) +); + + + + +alter table paimon_full_schema_change_orc ADD COLUMN new_struct_column struct , b int> ; + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 7, + map('person7', struct('George Hall', 41, '987 Elm St')), + struct('Seattle', 730000), + array(struct('Mango', 4.49), struct('Papaya', 3.75)), + struct(struct(1001, 'inner string'), 2025) +); + + + + +alter table paimon_full_schema_change_orc rename column new_struct_column.a.aa to new_aa; + + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 8, + map('person8', struct('Hannah King', 34, '321 Oak Blvd')), + struct('Boston', 690000), + array(struct('Dragonfruit', 5.25), struct('Lychee', 4.10)), + struct(struct(2002, 'deep inner string'), 3025) +); + + + +alter table paimon_full_schema_change_orc ADD COLUMN new_struct_column.c struct; + + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 9, + map('person9', struct('Ian Moore', 38, '888 Maple Way')), + struct('Denver', 620000), + array(struct('Peach', 2.89), struct('Plum', 2.45)), + struct( + struct(3003, 'nested value'), + 4025, + struct('extra info', 123) + ) +); + + + +alter table paimon_full_schema_change_orc rename column new_struct_column.a to new_a; + + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 10, + map('person10', struct('Julia Nash', 27, '456 Cedar Ct')), + struct('Phoenix', 820000), + array(struct('Cherry', 3.15), struct('Apricot', 2.95)), + struct( + struct(4004, 'renamed inner value'), + 5025, + struct('details', 456) + ) +); + + + + +alter table paimon_full_schema_change_orc rename column new_struct_column.c.dd to new_dd; + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 11, + map('person11', struct('Kevin Orr', 45, '789 Spruce Dr')), + struct('San Diego', 770000), + array(struct('Nectarine', 3.60), struct('Coconut', 4.20)), + struct( + struct(5005, 'final structure'), + 6025, + struct('notes', 789) + ) +); + + + +alter table paimon_full_schema_change_orc rename column new_struct_column to struct_column2; + + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 12, + map('person12', struct('Laura Price', 36, '1010 Aspen Way')), + struct('Dallas', 880000), + array(struct('Cranberry', 3.30), struct('Fig', 2.70)), + struct( + struct(6006, 'finalized'), + 7025, + struct('metadata', 321) + ) +); + +alter table paimon_full_schema_change_orc change column struct_column2 after struct_column; + +INSERT INTO paimon_full_schema_change_orc VALUES ( + 13, + map('person13', struct('Michael Reed', 31, '2020 Pine Cir')), + struct('Atlanta', 810000), + struct( + struct(7007, 'relocated field'), + 8025, + struct('info', 654) + ), + array(struct('Guava', 3.95), struct('Passionfruit', 4.60)) +); + + +CREATE TABLE paimon_full_schema_change_parquet ( + id int, + map_column map>, + struct_column struct, + array_column array> +) +USING paimon +TBLPROPERTIES ("file.format" = "parquet"); +; + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 0, + map('person0', struct('zero', 2)), + struct('cn', 1000000), + array(struct('Apple', 1.99), struct('Banana', 0.99)) +); + + +-- Schema Change 1: Add 'address' string to map_column's struct +ALTER TABLE paimon_full_schema_change_parquet ADD COLUMN map_column.value.address string; +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 1, + map('person1', struct('Alice', 25, '123 Main St')), + struct('New York', 8000000), + array(struct('Apple', 1.99), struct('Banana', 0.99)) +); + + +ALTER TABLE paimon_full_schema_change_parquet RENAME COLUMN map_column.value.name TO full_name; + + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 2, + map('person2', struct('Bob Smith', 30, '456 Oak Ave')), + struct('Los Angeles', 4000000), + array(struct('Orange', 2.49), struct('Grapes', 3.99)) +); + + + +ALTER TABLE paimon_full_schema_change_parquet RENAME COLUMN struct_column.city TO location; + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 3, + map('person3', struct('Charlie', 28, '789 Pine Rd')), + struct('Chicago', 2700000), + array(struct('Mango', 2.99), struct('Peach', 1.49)) +); + + + +ALTER TABLE paimon_full_schema_change_parquet DROP COLUMN struct_column.population; + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 4, + map('person4', struct('Diana Green', 35, '321 Elm St')), + struct('San Francisco'), + array(struct('Strawberry', 4.29), struct('Blueberry', 5.19)) +); + + + +ALTER TABLE paimon_full_schema_change_parquet ADD COLUMN struct_column.population bigint; + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 5, + map('person5', struct('Edward Black', 42, '987 Willow Ln')), + struct('Seattle', 750000), + array(struct('Kiwi', 1.79), struct('Pineapple', 3.49)) +); + + +ALTER TABLE paimon_full_schema_change_parquet RENAME COLUMN array_column.element.product to new_product; + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 6, + map('person6', struct('Fiona Lake', 29, '654 Birch Ln')), + struct('Austin', 950000), + array(struct('Pineapple', 3.99), struct('Kiwi', 2.59)) +); + + + + +alter table paimon_full_schema_change_parquet ADD COLUMN new_struct_column struct , b int> ; + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 7, + map('person7', struct('George Hall', 41, '987 Elm St')), + struct('Seattle', 730000), + array(struct('Mango', 4.49), struct('Papaya', 3.75)), + struct(struct(1001, 'inner string'), 2025) +); + + + + +alter table paimon_full_schema_change_parquet rename column new_struct_column.a.aa to new_aa; + + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 8, + map('person8', struct('Hannah King', 34, '321 Oak Blvd')), + struct('Boston', 690000), + array(struct('Dragonfruit', 5.25), struct('Lychee', 4.10)), + struct(struct(2002, 'deep inner string'), 3025) +); + + + +alter table paimon_full_schema_change_parquet ADD COLUMN new_struct_column.c struct; + + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 9, + map('person9', struct('Ian Moore', 38, '888 Maple Way')), + struct('Denver', 620000), + array(struct('Peach', 2.89), struct('Plum', 2.45)), + struct( + struct(3003, 'nested value'), + 4025, + struct('extra info', 123) + ) +); + + + +alter table paimon_full_schema_change_parquet rename column new_struct_column.a to new_a; + + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 10, + map('person10', struct('Julia Nash', 27, '456 Cedar Ct')), + struct('Phoenix', 820000), + array(struct('Cherry', 3.15), struct('Apricot', 2.95)), + struct( + struct(4004, 'renamed inner value'), + 5025, + struct('details', 456) + ) +); + + + + +alter table paimon_full_schema_change_parquet rename column new_struct_column.c.dd to new_dd; + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 11, + map('person11', struct('Kevin Orr', 45, '789 Spruce Dr')), + struct('San Diego', 770000), + array(struct('Nectarine', 3.60), struct('Coconut', 4.20)), + struct( + struct(5005, 'final structure'), + 6025, + struct('notes', 789) + ) +); + + + +alter table paimon_full_schema_change_parquet rename column new_struct_column to struct_column2; + + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 12, + map('person12', struct('Laura Price', 36, '1010 Aspen Way')), + struct('Dallas', 880000), + array(struct('Cranberry', 3.30), struct('Fig', 2.70)), + struct( + struct(6006, 'finalized'), + 7025, + struct('metadata', 321) + ) +); + +alter table paimon_full_schema_change_parquet change column struct_column2 after struct_column; + +INSERT INTO paimon_full_schema_change_parquet VALUES ( + 13, + map('person13', struct('Michael Reed', 31, '2020 Pine Cir')), + struct('Atlanta', 810000), + struct( + struct(7007, 'relocated field'), + 8025, + struct('info', 654) + ), + array(struct('Guava', 3.95), struct('Passionfruit', 4.60)) +); + +-- spark-sql (test_paimon_schema_change)> +-- > select * from paimon_full_schema_change_orc order by id; +-- 0 {"person0":{"full_name":"zero","age":2,"address":null}} {"location":"cn","population":null} NULL [{"new_product":"Apple","price":1.99},{"new_product":"Banana","price":0.99}] +-- 1 {"person1":{"full_name":"Alice","age":25,"address":"123 Main St"}} {"location":"New York","population":null} NULL [{"new_product":"Apple","price":1.99},{"new_product":"Banana","price":0.99}] +-- 2 {"person2":{"full_name":"Bob Smith","age":30,"address":"456 Oak Ave"}} {"location":"Los Angeles","population":null} NULL [{"new_product":"Orange","price":2.49},{"new_product":"Grapes","price":3.99}] +-- 3 {"person3":{"full_name":"Charlie","age":28,"address":"789 Pine Rd"}} {"location":"Chicago","population":null} NULL [{"new_product":"Mango","price":2.99},{"new_product":"Peach","price":1.49}] +-- 4 {"person4":{"full_name":"Diana Green","age":35,"address":"321 Elm St"}} {"location":"San Francisco","population":null} NULL [{"new_product":"Strawberry","price":4.29},{"new_product":"Blueberry","price":5.19}] +-- 5 {"person5":{"full_name":"Edward Black","age":42,"address":"987 Willow Ln"}} {"location":"Seattle","population":750000} NULL [{"new_product":"Kiwi","price":1.79},{"new_product":"Pineapple","price":3.49}] +-- 6 {"person6":{"full_name":"Fiona Lake","age":29,"address":"654 Birch Ln"}} {"location":"Austin","population":950000} NULL [{"new_product":"Pineapple","price":3.99},{"new_product":"Kiwi","price":2.59}] +-- 7 {"person7":{"full_name":"George Hall","age":41,"address":"987 Elm St"}} {"location":"Seattle","population":730000} {"new_a":{"new_aa":1001,"bb":"inner string"},"b":2025,"c":null} [{"new_product":"Mango","price":4.49},{"new_product":"Papaya","price":3.75}] +-- 8 {"person8":{"full_name":"Hannah King","age":34,"address":"321 Oak Blvd"}} {"location":"Boston","population":690000} {"new_a":{"new_aa":2002,"bb":"deep inner string"},"b":3025,"c":null} [{"new_product":"Dragonfruit","price":5.25},{"new_product":"Lychee","price":4.1}] +-- 9 {"person9":{"full_name":"Ian Moore","age":38,"address":"888 Maple Way"}} {"location":"Denver","population":620000} {"new_a":{"new_aa":3003,"bb":"nested value"},"b":4025,"c":{"cc":"extra info","new_dd":123}} [{"new_product":"Peach","price":2.89},{"new_product":"Plum","price":2.45}] +-- 10 {"person10":{"full_name":"Julia Nash","age":27,"address":"456 Cedar Ct"}} {"location":"Phoenix","population":820000} {"new_a":{"new_aa":4004,"bb":"renamed inner value"},"b":5025,"c":{"cc":"details","new_dd":456}} [{"new_product":"Cherry","price":3.15},{"new_product":"Apricot","price":2.95}] +-- 11 {"person11":{"full_name":"Kevin Orr","age":45,"address":"789 Spruce Dr"}} {"location":"San Diego","population":770000} {"new_a":{"new_aa":5005,"bb":"final structure"},"b":6025,"c":{"cc":"notes","new_dd":789}} [{"new_product":"Nectarine","price":3.6},{"new_product":"Coconut","price":4.2}] +-- 12 {"person12":{"full_name":"Laura Price","age":36,"address":"1010 Aspen Way"}} {"location":"Dallas","population":880000} {"new_a":{"new_aa":6006,"bb":"finalized"},"b":7025,"c":{"cc":"metadata","new_dd":321}} [{"new_product":"Cranberry","price":3.3},{"new_product":"Fig","price":2.7}] +-- 13 {"person13":{"full_name":"Michael Reed","age":31,"address":"2020 Pine Cir"}} {"location":"Atlanta","population":810000} {"new_a":{"new_aa":7007,"bb":"relocated field"},"b":8025,"c":{"cc":"info","new_dd":654}} [{"new_product":"Guava","price":3.95},{"new_product":"Passionfruit","price":4.6}] +-- Time taken: 0.991 seconds, Fetched 14 row(s) \ No newline at end of file diff --git a/fe/check/checkstyle/suppressions.xml b/fe/check/checkstyle/suppressions.xml index bcd034531c69ab..8f000bb7616ca9 100644 --- a/fe/check/checkstyle/suppressions.xml +++ b/fe/check/checkstyle/suppressions.xml @@ -68,4 +68,7 @@ under the License. + + + diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index 966891848feb11..51f232648415e8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -1279,4 +1279,5 @@ public void setDefaultValueInfo(Column refColumn) { this.defaultValueExprDef = refColumn.defaultValueExprDef; this.realDefaultValue = refColumn.realDefaultValue; } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalUtil.java new file mode 100644 index 00000000000000..ed3b34db3273b8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalUtil.java @@ -0,0 +1,83 @@ +// 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. + +package org.apache.doris.datasource; + +import org.apache.doris.catalog.Column; +import org.apache.doris.thrift.TFileScanRangeParams; +import org.apache.doris.thrift.schema.external.TArrayField; +import org.apache.doris.thrift.schema.external.TField; +import org.apache.doris.thrift.schema.external.TFieldPtr; +import org.apache.doris.thrift.schema.external.TMapField; +import org.apache.doris.thrift.schema.external.TNestedField; +import org.apache.doris.thrift.schema.external.TSchema; +import org.apache.doris.thrift.schema.external.TStructField; + +import java.util.List; + +public class ExternalUtil { + private static TField getExternalSchema(Column column) { + TField root = new TField(); + root.setName(column.getName()); + root.setId(column.getUniqueId()); + root.setIsOptional(column.isAllowNull()); + root.setType(column.getType().toColumnTypeThrift()); + + TNestedField nestedField = new TNestedField(); + if (column.getType().isStructType()) { + nestedField.setStructField(getExternalSchema(column.getChildren())); + root.setNestedField(nestedField); + } else if (column.getType().isArrayType()) { + TArrayField listField = new TArrayField(); + TFieldPtr fieldPtr = new TFieldPtr(); + fieldPtr.setFieldPtr(getExternalSchema(column.getChildren().get(0))); + listField.setItemField(fieldPtr); + nestedField.setArrayField(listField); + root.setNestedField(nestedField); + } else if (column.getType().isMapType()) { + TMapField mapField = new TMapField(); + TFieldPtr keyPtr = new TFieldPtr(); + keyPtr.setFieldPtr(getExternalSchema(column.getChildren().get(0))); + mapField.setKeyField(keyPtr); + TFieldPtr valuePtr = new TFieldPtr(); + valuePtr.setFieldPtr(getExternalSchema(column.getChildren().get(1))); + mapField.setValueField(valuePtr); + nestedField.setMapField(mapField); + root.setNestedField(nestedField); + } + return root; + } + + private static TStructField getExternalSchema(List columns) { + TStructField structField = new TStructField(); + for (Column child : columns) { + TFieldPtr fieldPtr = new TFieldPtr(); + fieldPtr.setFieldPtr(getExternalSchema(child)); + structField.addToFields(fieldPtr); + } + return structField; + } + + + public static void initSchemaInfo(TFileScanRangeParams params, Long schemaId, List columns) { + params.setCurrentSchemaId(schemaId); + TSchema tSchema = new TSchema(); + tSchema.setSchemaId(schemaId); + tSchema.setRootField(getExternalSchema(columns)); + params.addToHistorySchemaInfo(tSchema); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java index 6f6715eb9ebaed..46b455a2205146 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java @@ -34,8 +34,6 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.BrokerUtil; import org.apache.doris.common.util.Util; -import org.apache.doris.datasource.hive.AcidInfo; -import org.apache.doris.datasource.hive.AcidInfo.DeleteDeltaInfo; import org.apache.doris.datasource.hive.source.HiveSplit; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.qe.ConnectContext; @@ -60,9 +58,6 @@ import org.apache.doris.thrift.TScanRangeLocation; import org.apache.doris.thrift.TScanRangeLocations; import org.apache.doris.thrift.TSplitSource; -import org.apache.doris.thrift.TTableFormatFileDesc; -import org.apache.doris.thrift.TTransactionalHiveDeleteDeltaDesc; -import org.apache.doris.thrift.TTransactionalHiveDesc; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -73,7 +68,6 @@ import org.apache.logging.log4j.Logger; import java.net.URI; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -430,35 +424,10 @@ private TScanRangeLocations splitToScanRange( TFileRangeDesc rangeDesc = createFileRangeDesc(fileSplit, partitionValuesFromPath, pathPartitionKeys); TFileCompressType fileCompressType = getFileCompressType(fileSplit); rangeDesc.setCompressType(fileCompressType); - if (fileSplit instanceof HiveSplit) { - if (isACID) { - HiveSplit hiveSplit = (HiveSplit) fileSplit; - hiveSplit.setTableFormatType(TableFormatType.TRANSACTIONAL_HIVE); - TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); - tableFormatFileDesc.setTableFormatType(hiveSplit.getTableFormatType().value()); - AcidInfo acidInfo = (AcidInfo) hiveSplit.getInfo(); - TTransactionalHiveDesc transactionalHiveDesc = new TTransactionalHiveDesc(); - transactionalHiveDesc.setPartition(acidInfo.getPartitionLocation()); - List deleteDeltaDescs = new ArrayList<>(); - for (DeleteDeltaInfo deleteDeltaInfo : acidInfo.getDeleteDeltas()) { - TTransactionalHiveDeleteDeltaDesc deleteDeltaDesc = new TTransactionalHiveDeleteDeltaDesc(); - deleteDeltaDesc.setDirectoryLocation(deleteDeltaInfo.getDirectoryLocation()); - deleteDeltaDesc.setFileNames(deleteDeltaInfo.getFileNames()); - deleteDeltaDescs.add(deleteDeltaDesc); - } - transactionalHiveDesc.setDeleteDeltas(deleteDeltaDescs); - tableFormatFileDesc.setTransactionalHiveParams(transactionalHiveDesc); - rangeDesc.setTableFormatParams(tableFormatFileDesc); - } else { - TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); - tableFormatFileDesc.setTableFormatType(TableFormatType.HIVE.value()); - rangeDesc.setTableFormatParams(tableFormatFileDesc); - } - } - // set file format type, and the type might fall back to native format in setScanParams rangeDesc.setFormatType(getFileFormatType()); setScanParams(rangeDesc, fileSplit); + curLocations.getScanRange().getExtScanRange().getFileScanRange().addToRanges(rangeDesc); TScanRangeLocation location = new TScanRangeLocation(); setLocationPropertiesIfNecessary(backend, fileSplit.getLocationType(), locationProperties); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/TableFormatType.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/TableFormatType.java index e91187464b6fa2..5f67cb3329e3b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/TableFormatType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/TableFormatType.java @@ -25,7 +25,8 @@ public enum TableFormatType { MAX_COMPUTE("max_compute"), TRANSACTIONAL_HIVE("transactional_hive"), LAKESOUL("lakesoul"), - TRINO_CONNECTOR("trino_connector"); + TRINO_CONNECTOR("trino_connector"), + TVF("tvf"); private final String tableFormatType; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index 56e3327e99d6ff..d2f2f9d821b077 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -652,9 +652,12 @@ private Optional initHudiSchema(SchemaCacheKey key) { Types.Field hudiInternalfield = hudiInternalSchema.getRecord().fields().get(i); org.apache.avro.Schema.Field hudiAvroField = hudiSchema.getFields().get(i); String columnName = hudiAvroField.name().toLowerCase(Locale.ROOT); - tmpSchema.add(new Column(columnName, HudiUtils.fromAvroHudiTypeToDorisType(hudiAvroField.schema()), + Column column = new Column(columnName, HudiUtils.fromAvroHudiTypeToDorisType(hudiAvroField.schema()), true, null, true, null, "", true, null, - hudiInternalfield.fieldId(), null)); + -1, null); + HudiUtils.updateHudiColumnUniqueId(column, hudiInternalfield); + tmpSchema.add(column); + colTypes.add(HudiUtils.convertAvroToHiveType(hudiAvroField.schema())); } List partitionColumns = initPartitionColumns(tmpSchema); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java index d9ae7dc7be9458..444aa1a5d4c843 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java @@ -33,6 +33,9 @@ import org.apache.doris.datasource.FileQueryScanNode; import org.apache.doris.datasource.FileSplit; import org.apache.doris.datasource.FileSplitter; +import org.apache.doris.datasource.TableFormatType; +import org.apache.doris.datasource.hive.AcidInfo; +import org.apache.doris.datasource.hive.AcidInfo.DeleteDeltaInfo; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HiveMetaStoreCache; @@ -52,8 +55,12 @@ import org.apache.doris.thrift.TFileAttributes; import org.apache.doris.thrift.TFileCompressType; import org.apache.doris.thrift.TFileFormatType; +import org.apache.doris.thrift.TFileRangeDesc; import org.apache.doris.thrift.TFileTextScanRangeParams; import org.apache.doris.thrift.TPushAggOp; +import org.apache.doris.thrift.TTableFormatFileDesc; +import org.apache.doris.thrift.TTransactionalHiveDeleteDeltaDesc; +import org.apache.doris.thrift.TTransactionalHiveDesc; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -66,6 +73,7 @@ import org.apache.logging.log4j.Logger; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -439,6 +447,36 @@ public TFileFormatType getFileFormatType() throws UserException { return type; } + + @Override + protected void setScanParams(TFileRangeDesc rangeDesc, Split split) { + if (split instanceof HiveSplit) { + HiveSplit hiveSplit = (HiveSplit) split; + if (hiveSplit.isACID()) { + hiveSplit.setTableFormatType(TableFormatType.TRANSACTIONAL_HIVE); + TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); + tableFormatFileDesc.setTableFormatType(hiveSplit.getTableFormatType().value()); + AcidInfo acidInfo = (AcidInfo) hiveSplit.getInfo(); + TTransactionalHiveDesc transactionalHiveDesc = new TTransactionalHiveDesc(); + transactionalHiveDesc.setPartition(acidInfo.getPartitionLocation()); + List deleteDeltaDescs = new ArrayList<>(); + for (DeleteDeltaInfo deleteDeltaInfo : acidInfo.getDeleteDeltas()) { + TTransactionalHiveDeleteDeltaDesc deleteDeltaDesc = new TTransactionalHiveDeleteDeltaDesc(); + deleteDeltaDesc.setDirectoryLocation(deleteDeltaInfo.getDirectoryLocation()); + deleteDeltaDesc.setFileNames(deleteDeltaInfo.getFileNames()); + deleteDeltaDescs.add(deleteDeltaDesc); + } + transactionalHiveDesc.setDeleteDeltas(deleteDeltaDescs); + tableFormatFileDesc.setTransactionalHiveParams(transactionalHiveDesc); + rangeDesc.setTableFormatParams(tableFormatFileDesc); + } else { + TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); + tableFormatFileDesc.setTableFormatType(TableFormatType.HIVE.value()); + rangeDesc.setTableFormatParams(tableFormatFileDesc); + } + } + } + @Override protected Map getLocationProperties() throws UserException { return hmsTable.getHadoopProperties(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java index 461e5d02b36d83..c20915e06fd9f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.ArrayType; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MapType; import org.apache.doris.catalog.ScalarType; @@ -32,6 +33,15 @@ import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; import org.apache.doris.datasource.hudi.source.HudiCachedPartitionProcessor; +import org.apache.doris.thrift.TColumnType; +import org.apache.doris.thrift.TPrimitiveType; +import org.apache.doris.thrift.schema.external.TArrayField; +import org.apache.doris.thrift.schema.external.TField; +import org.apache.doris.thrift.schema.external.TFieldPtr; +import org.apache.doris.thrift.schema.external.TMapField; +import org.apache.doris.thrift.schema.external.TNestedField; +import org.apache.doris.thrift.schema.external.TSchema; +import org.apache.doris.thrift.schema.external.TStructField; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; @@ -47,15 +57,14 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; import java.text.ParseException; import java.time.LocalDate; import java.time.format.DateTimeFormatter; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; @@ -164,6 +173,30 @@ private static String handleUnsupportedType(Schema schema) { throw new IllegalArgumentException(String.format("Unsupported logical type: %s", schema.getLogicalType())); } + public static void updateHudiColumnUniqueId(Column column, Types.Field hudiInternalfield) { + column.setUniqueId(hudiInternalfield.fieldId()); + + List hudiInternalfields = new ArrayList<>(); + switch (hudiInternalfield.type().typeId()) { + case ARRAY: + hudiInternalfields = ((Types.ArrayType) hudiInternalfield.type()).fields(); + break; + case MAP: + hudiInternalfields = ((Types.MapType) hudiInternalfield.type()).fields(); + break; + case RECORD: + hudiInternalfields = ((Types.RecordType) hudiInternalfield.type()).fields(); + break; + default: + return; + } + + List childColumns = column.getChildren(); + for (int idx = 0; idx < childColumns.size(); idx++) { + updateHudiColumnUniqueId(childColumns.get(idx), hudiInternalfields.get(idx)); + } + } + public static Type fromAvroHudiTypeToDorisType(Schema avroSchema) { Schema.Type columnType = avroSchema.getType(); LogicalType logicalType = avroSchema.getLogicalType(); @@ -323,14 +356,7 @@ public static HoodieTableMetaClient buildHudiTableMetaClient(String hudiBasePath .setConf(hadoopStorageConfiguration).setBasePath(hudiBasePath).build()); } - public static Map getSchemaInfo(InternalSchema internalSchema) { - Types.RecordType record = internalSchema.getRecord(); - Map schemaInfo = new HashMap<>(record.fields().size()); - for (Types.Field field : record.fields()) { - schemaInfo.put(field.fieldId(), field.name().toLowerCase()); - } - return schemaInfo; - } + public static HudiSchemaCacheValue getSchemaCacheValue(HMSExternalTable hmsTable, String queryInstant) { ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(hmsTable.getCatalog()); @@ -338,4 +364,77 @@ public static HudiSchemaCacheValue getSchemaCacheValue(HMSExternalTable hmsTable Optional schemaCacheValue = cache.getSchemaValue(key); return (HudiSchemaCacheValue) schemaCacheValue.get(); } + + public static TStructField getSchemaInfo(List hudiFields) { + TStructField structField = new TStructField(); + for (Types.Field field : hudiFields) { + TFieldPtr fieldPtr = new TFieldPtr(); + fieldPtr.setFieldPtr(getSchemaInfo(field)); + structField.addToFields(fieldPtr); + } + return structField; + } + + + public static TField getSchemaInfo(Types.Field hudiInternalField) { + TField root = new TField(); + root.setName(hudiInternalField.name()); + root.setId(hudiInternalField.fieldId()); + root.setIsOptional(hudiInternalField.isOptional()); + + TNestedField nestedField = new TNestedField(); + switch (hudiInternalField.type().typeId()) { + case ARRAY: { + TColumnType tColumnType = new TColumnType(); + tColumnType.setType(TPrimitiveType.ARRAY); + root.setType(tColumnType); + + TArrayField listField = new TArrayField(); + List hudiFields = ((Types.ArrayType) hudiInternalField.type()).fields(); + TFieldPtr fieldPtr = new TFieldPtr(); + fieldPtr.setFieldPtr(getSchemaInfo(hudiFields.get(0))); + listField.setItemField(fieldPtr); + nestedField.setArrayField(listField); + root.setNestedField(nestedField); + break; + } case MAP: { + TColumnType tColumnType = new TColumnType(); + tColumnType.setType(TPrimitiveType.MAP); + root.setType(tColumnType); + + TMapField mapField = new TMapField(); + List hudiFields = ((Types.MapType) hudiInternalField.type()).fields(); + TFieldPtr keyPtr = new TFieldPtr(); + keyPtr.setFieldPtr(getSchemaInfo(hudiFields.get(0))); + mapField.setKeyField(keyPtr); + TFieldPtr valuePtr = new TFieldPtr(); + valuePtr.setFieldPtr(getSchemaInfo(hudiFields.get(1))); + mapField.setValueField(valuePtr); + nestedField.setMapField(mapField); + root.setNestedField(nestedField); + break; + } case RECORD: { + TColumnType tColumnType = new TColumnType(); + tColumnType.setType(TPrimitiveType.STRUCT); + root.setType(tColumnType); + + List hudiFields = ((Types.RecordType) hudiInternalField.type()).fields(); + nestedField.setStructField(getSchemaInfo(hudiFields)); + root.setNestedField(nestedField); + break; + } default: { + root.setType(fromAvroHudiTypeToDorisType(AvroInternalSchemaConverter.convert( + hudiInternalField.type(), hudiInternalField.name())).toColumnTypeThrift()); + break; + } + } + return root; + } + + public static TSchema getSchemaInfo(InternalSchema hudiInternalSchema) { + TSchema tschema = new TSchema(); + tschema.setSchemaId(hudiInternalSchema.schemaId()); + tschema.setRootField(getSchemaInfo(hudiInternalSchema.getRecord().fields())); + return tschema; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java index 5d0b703e74be74..6fdab3a0dada55 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java @@ -30,6 +30,7 @@ import org.apache.doris.common.util.FileFormatUtils; import org.apache.doris.common.util.LocationPath; import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.ExternalUtil; import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.TableFormatType; import org.apache.doris.datasource.hive.HivePartition; @@ -115,6 +116,9 @@ public class HudiScanNode extends HiveScanNode { private IncrementalRelation incrementalRelation; private HoodieTableFileSystemView fsView; + // The schema information involved in the current query process (including historical schema). + protected ConcurrentHashMap currentQuerySchema = new ConcurrentHashMap<>(); + /** * External file scan node for Query Hudi table * needCheckColumnPriv: Some of ExternalFileScanNode do not need to check column @@ -217,7 +221,12 @@ protected void doInitialize() throws UserException { .getExtMetaCacheMgr() .getFsViewProcessor(hmsTable.getCatalog()) .getFsView(hmsTable.getDbName(), hmsTable.getName(), hudiClient); - params.setHistorySchemaInfo(new ConcurrentHashMap<>()); + // Todo: Get the current schema id of the table, instead of using -1. + // In Be Parquet/Rrc reader, if `current table schema id == current file schema id`, then its + // `table_info_node_ptr` will be `TableSchemaChangeHelper::ConstNode`. When using `ConstNode`, + // you need to pay special attention to the `case difference` between the `table column name` + // and `the file column name`. + ExternalUtil.initSchemaInfo(params, -1L, table.getColumns()); } @Override @@ -252,6 +261,13 @@ protected void setScanParams(TFileRangeDesc rangeDesc, Split split) { } } + + private void putHistorySchemaInfo(InternalSchema internalSchema) { + if (currentQuerySchema.putIfAbsent(internalSchema.schemaId(), Boolean.TRUE) == null) { + params.addToHistorySchemaInfo(HudiUtils.getSchemaInfo(internalSchema)); + } + } + private void setHudiParams(TFileRangeDesc rangeDesc, HudiSplit hudiSplit) { TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); tableFormatFileDesc.setTableFormatType(hudiSplit.getTableFormatType().value()); @@ -275,18 +291,14 @@ private void setHudiParams(TFileRangeDesc rangeDesc, HudiSplit hudiSplit) { new File(hudiSplit.getPath().get()).getName())); InternalSchema internalSchema = hudiSchemaCacheValue .getCommitInstantInternalSchema(hudiClient, commitInstantTime); - params.history_schema_info.computeIfAbsent( - internalSchema.schemaId(), - k -> HudiUtils.getSchemaInfo(internalSchema)); //for schema change. (native reader) + putHistorySchemaInfo(internalSchema); //for schema change. (native reader) fileDesc.setSchemaId(internalSchema.schemaId()); } else { try { TableSchemaResolver schemaUtil = new TableSchemaResolver(hudiClient); InternalSchema internalSchema = AvroInternalSchemaConverter.convert(schemaUtil.getTableAvroSchema(true)); - params.history_schema_info.computeIfAbsent( - internalSchema.schemaId(), - k -> HudiUtils.getSchemaInfo(internalSchema)); // Handle column name case for BE. + putHistorySchemaInfo(internalSchema); //Handle column name case for BE fileDesc.setSchemaId(internalSchema.schemaId()); } catch (Exception e) { throw new RuntimeException("Cannot get hudi table schema.", e); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java index 7c8bd51f954324..09ce839dd9b545 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java @@ -100,6 +100,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.types.Type.TypeID; import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.util.LocationUtil; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.StructProjection; @@ -607,6 +608,29 @@ public static org.apache.iceberg.Table getIcebergTable(ExternalTable dorisTable) .getIcebergMetadataCache().getIcebergTable(dorisTable); } + private static void updateIcebergColumnUniqueId(Column column, Types.NestedField icebergField) { + column.setUniqueId(icebergField.fieldId()); + List icebergFields = Lists.newArrayList(); + switch (icebergField.type().typeId()) { + case LIST: + icebergFields = ((Types.ListType) icebergField.type()).fields(); + break; + case MAP: + icebergFields = ((Types.MapType) icebergField.type()).fields(); + break; + case STRUCT: + icebergFields = ((Types.StructType) icebergField.type()).fields(); + break; + default: + return; + } + + List childColumns = column.getChildren(); + for (int idx = 0; idx < childColumns.size(); idx++) { + updateIcebergColumnUniqueId(childColumns.get(idx), icebergFields.get(idx)); + } + } + /** * Get iceberg schema from catalog and convert them to doris schema */ @@ -647,9 +671,11 @@ public static List parseSchema(Schema schema) { List columns = schema.columns(); List resSchema = Lists.newArrayListWithCapacity(columns.size()); for (Types.NestedField field : columns) { - resSchema.add(new Column(field.name().toLowerCase(Locale.ROOT), - IcebergUtils.icebergTypeToDorisType(field.type()), true, null, true, field.doc(), true, - schema.caseInsensitiveFindField(field.name()).fieldId())); + Column column = new Column(field.name().toLowerCase(Locale.ROOT), + IcebergUtils.icebergTypeToDorisType(field.type()), true, null, + true, field.doc(), true, -1); + updateIcebergColumnUniqueId(column, field); + resSchema.add(column); } return resSchema; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java index c2bfc9ebfcc085..488c232de62898 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java @@ -30,6 +30,7 @@ import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.common.util.LocationPath; import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.ExternalUtil; import org.apache.doris.datasource.FileQueryScanNode; import org.apache.doris.datasource.TableFormatType; import org.apache.doris.datasource.hive.HMSExternalTable; @@ -152,6 +153,7 @@ protected void doInitialize() throws UserException { formatVersion = ((BaseTable) icebergTable).operations().current().formatVersion(); preExecutionAuthenticator = source.getCatalog().getPreExecutionAuthenticator(); super.doInitialize(); + ExternalUtil.initSchemaInfo(params, -1L, source.getTargetTable().getColumns()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java index 7daccdab8bbab7..38f8c9122b7220 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java @@ -237,7 +237,8 @@ public Optional initSchema(SchemaCacheKey key) { for (DataField field : columns) { Column column = new Column(field.name().toLowerCase(), PaimonUtil.paimonTypeToDorisType(field.type()), true, null, true, field.description(), true, - field.id()); + -1); + PaimonUtil.updatePaimonColumnUniqueId(column, field); dorisColumns.add(column); if (partitionColumnNames.contains(field.name())) { partitionColumns.add(column); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java index a4a7eea0156cde..800c0d9413e8b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java @@ -26,6 +26,15 @@ import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.datasource.hive.HiveUtil; +import org.apache.doris.thrift.TColumnType; +import org.apache.doris.thrift.TPrimitiveType; +import org.apache.doris.thrift.schema.external.TArrayField; +import org.apache.doris.thrift.schema.external.TField; +import org.apache.doris.thrift.schema.external.TFieldPtr; +import org.apache.doris.thrift.schema.external.TMapField; +import org.apache.doris.thrift.schema.external.TNestedField; +import org.apache.doris.thrift.schema.external.TSchema; +import org.apache.doris.thrift.schema.external.TStructField; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -39,11 +48,13 @@ import org.apache.paimon.partition.Partition; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.ReadBuilder; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.CharType; import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataType; import org.apache.paimon.types.DecimalType; import org.apache.paimon.types.MapType; import org.apache.paimon.types.RowType; @@ -255,4 +266,105 @@ public static String encodeObjectToString(T t) { } } + public static void updatePaimonColumnUniqueId(Column column, DataType dataType) { + List columns = column.getChildren(); + switch (dataType.getTypeRoot()) { + case ARRAY: + ArrayType arrayType = (ArrayType) dataType; + updatePaimonColumnUniqueId(columns.get(0), arrayType.getElementType()); + break; + case MAP: + MapType mapType = (MapType) dataType; + updatePaimonColumnUniqueId(columns.get(0), mapType.getKeyType()); + updatePaimonColumnUniqueId(columns.get(1), mapType.getValueType()); + break; + case ROW: + RowType rowType = (RowType) dataType; + for (int idx = 0; idx < columns.size(); idx++) { + updatePaimonColumnUniqueId(columns.get(idx), rowType.getFields().get(idx)); + } + break; + default: + return; + } + } + + public static void updatePaimonColumnUniqueId(Column column, DataField field) { + column.setUniqueId(field.id()); + updatePaimonColumnUniqueId(column, field.type()); + } + + public static TField getSchemaInfo(DataType dataType) { + TField field = new TField(); + field.setIsOptional(dataType.isNullable()); + TNestedField nestedField = new TNestedField(); + switch (dataType.getTypeRoot()) { + case ARRAY: { + TArrayField listField = new TArrayField(); + org.apache.paimon.types.ArrayType paimonArrayType = (org.apache.paimon.types.ArrayType) dataType; + TFieldPtr fieldPtr = new TFieldPtr(); + fieldPtr.setFieldPtr(getSchemaInfo(paimonArrayType.getElementType())); + listField.setItemField(fieldPtr); + nestedField.setArrayField(listField); + field.setNestedField(nestedField); + + TColumnType tColumnType = new TColumnType(); + tColumnType.setType(TPrimitiveType.ARRAY); + field.setType(tColumnType); + break; + } + case MAP: { + TMapField mapField = new TMapField(); + org.apache.paimon.types.MapType mapType = (org.apache.paimon.types.MapType) dataType; + TFieldPtr keyField = new TFieldPtr(); + keyField.setFieldPtr(getSchemaInfo(mapType.getKeyType())); + mapField.setKeyField(keyField); + TFieldPtr valueField = new TFieldPtr(); + valueField.setFieldPtr(getSchemaInfo(mapType.getValueType())); + mapField.setValueField(valueField); + nestedField.setMapField(mapField); + field.setNestedField(nestedField); + + TColumnType tColumnType = new TColumnType(); + tColumnType.setType(TPrimitiveType.MAP); + field.setType(tColumnType); + break; + } + case ROW: { + RowType rowType = (RowType) dataType; + TStructField structField = getSchemaInfo(rowType.getFields()); + nestedField.setStructField(structField); + field.setNestedField(nestedField); + + TColumnType tColumnType = new TColumnType(); + tColumnType.setType(TPrimitiveType.STRUCT); + field.setType(tColumnType); + break; + } + default: + field.setType(paimonPrimitiveTypeToDorisType(dataType).toColumnTypeThrift()); + break; + } + return field; + } + + public static TStructField getSchemaInfo(List paimonFields) { + TStructField structField = new TStructField(); + for (DataField paimonField : paimonFields) { + TField childField = getSchemaInfo(paimonField.type()); + childField.setName(paimonField.name()); + childField.setId(paimonField.id()); + TFieldPtr fieldPtr = new TFieldPtr(); + fieldPtr.setFieldPtr(childField); + structField.addToFields(fieldPtr); + } + return structField; + } + + public static TSchema getSchemaInfo(TableSchema paimonTableSchema) { + TSchema tSchema = new TSchema(); + tSchema.setSchemaId(paimonTableSchema.id()); + tSchema.setRootField(getSchemaInfo(paimonTableSchema.fields())); + return tSchema; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java index c8fdcd15d77c92..7481abfb24c9b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java @@ -25,6 +25,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.FileFormatUtils; import org.apache.doris.common.util.LocationPath; +import org.apache.doris.datasource.ExternalUtil; import org.apache.doris.datasource.FileQueryScanNode; import org.apache.doris.datasource.FileSplitter; import org.apache.doris.datasource.paimon.PaimonExternalCatalog; @@ -43,7 +44,6 @@ import org.apache.doris.thrift.TTableFormatFileDesc; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -136,6 +136,9 @@ public String toString() { private List splitStats = new ArrayList<>(); private String serializedTable; + // The schema information involved in the current query process (including historical schema). + protected ConcurrentHashMap currentQuerySchema = new ConcurrentHashMap<>(); + public PaimonScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckColumnPriv, @@ -147,9 +150,9 @@ public PaimonScanNode(PlanNodeId id, protected void doInitialize() throws UserException { super.doInitialize(); source = new PaimonSource(desc); - serializedTable = PaimonUtil.encodeObjectToString(source.getPaimonTable()); - Preconditions.checkNotNull(source); - params.setHistorySchemaInfo(new ConcurrentHashMap<>()); + serializedTable = encodeObjectToString(source.getPaimonTable()); + // Todo: Get the current schema id of the table, instead of using -1. + ExternalUtil.initSchemaInfo(params, -1L, source.getTargetTable().getColumns()); } @VisibleForTesting @@ -176,16 +179,13 @@ protected Optional getSerializedTable() { return Optional.of(serializedTable); } - private Map getSchemaInfo(Long schemaId) { - PaimonExternalTable table = (PaimonExternalTable) source.getTargetTable(); - TableSchema tableSchema = Env.getCurrentEnv().getExtMetaCacheMgr().getPaimonMetadataCache() - .getPaimonSchemaCacheValue(table.getOrBuildNameMapping(), schemaId).getTableSchema(); - Map columnIdToName = new HashMap<>(tableSchema.fields().size()); - for (DataField dataField : tableSchema.fields()) { - columnIdToName.put(dataField.id(), dataField.name().toLowerCase()); + private void putHistorySchemaInfo(Long schemaId) { + if (currentQuerySchema.putIfAbsent(schemaId, Boolean.TRUE) == null) { + PaimonExternalTable table = (PaimonExternalTable) source.getTargetTable(); + TableSchema tableSchema = Env.getCurrentEnv().getExtMetaCacheMgr().getPaimonMetadataCache() + .getPaimonSchemaCacheValue(table.getOrBuildNameMapping(), schemaId).getTableSchema(); + params.addToHistorySchemaInfo(PaimonUtil.getSchemaInfo(tableSchema)); } - - return columnIdToName; } private void setPaimonParams(TFileRangeDesc rangeDesc, PaimonSplit paimonSplit) { @@ -209,8 +209,9 @@ private void setPaimonParams(TFileRangeDesc rangeDesc, PaimonSplit paimonSplit) } else { throw new RuntimeException("Unsupported file format: " + fileFormat); } + + putHistorySchemaInfo(paimonSplit.getSchemaId()); fileDesc.setSchemaId(paimonSplit.getSchemaId()); - params.history_schema_info.computeIfAbsent(paimonSplit.getSchemaId(), this::getSchemaInfo); } fileDesc.setFileFormat(fileFormat); fileDesc.setPaimonPredicate(PaimonUtil.encodeObjectToString(predicates)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/tvf/source/TVFScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/tvf/source/TVFScanNode.java index 82f42ee110546b..552489e6f8c7d7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/tvf/source/TVFScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/tvf/source/TVFScanNode.java @@ -30,6 +30,7 @@ import org.apache.doris.datasource.FileSplit; import org.apache.doris.datasource.FileSplit.FileSplitCreator; import org.apache.doris.datasource.FileSplitter; +import org.apache.doris.datasource.TableFormatType; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.qe.SessionVariable; import org.apache.doris.spi.Split; @@ -41,8 +42,10 @@ import org.apache.doris.thrift.TFileAttributes; import org.apache.doris.thrift.TFileCompressType; import org.apache.doris.thrift.TFileFormatType; +import org.apache.doris.thrift.TFileRangeDesc; import org.apache.doris.thrift.TFileType; import org.apache.doris.thrift.TPushAggOp; +import org.apache.doris.thrift.TTableFormatFileDesc; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -161,6 +164,15 @@ public List getSplits(int numBackends) throws UserException { return splits; } + @Override + protected void setScanParams(TFileRangeDesc rangeDesc, Split split) { + if (split instanceof FileSplit) { + TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); + tableFormatFileDesc.setTableFormatType(TableFormatType.TVF.value()); + rangeDesc.setTableFormatParams(tableFormatFileDesc); + } + } + @Override public int getNumInstances() { return scanRangeLocations.size(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java index 636db0943f67f4..9f9f769c78069c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java @@ -46,6 +46,7 @@ import org.apache.doris.datasource.property.storage.StorageProperties; import org.apache.doris.datasource.tvf.source.TVFScanNode; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.exceptions.NotSupportedException; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.planner.ScanNode; import org.apache.doris.proto.InternalService; @@ -84,9 +85,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.stream.Collectors; @@ -337,11 +340,19 @@ private Pair getColumnType(List typeNodes, int start) } else if (tPrimitiveType == TPrimitiveType.STRUCT) { parsedNodes = 1; ArrayList fields = new ArrayList<>(); + Set fieldLowerNames = new HashSet<>(); + for (int i = 0; i < typeNodes.get(start).getStructFieldsCount(); ++i) { Pair fieldType = getColumnType(typeNodes, start + parsedNodes); PStructField structField = typeNodes.get(start).getStructFields(i); - fields.add(new StructField(structField.getName(), fieldType.key(), structField.getComment(), - structField.getContainsNull())); + String fieldName = structField.getName().toLowerCase(); + if (fieldLowerNames.contains(fieldName)) { + throw new NotSupportedException("Repeated lowercase field names: " + fieldName); + } else { + fieldLowerNames.add(fieldName); + fields.add(new StructField(fieldName, fieldType.key(), structField.getComment(), + structField.getContainsNull())); + } parsedNodes += fieldType.value(); } type = new StructType(fields); @@ -361,10 +372,18 @@ private void fillColumns(InternalService.PFetchTableSchemaResult result) { return; } // add fetched file columns + Set columnLowerNames = new HashSet<>(); for (int idx = 0; idx < result.getColumnNums(); ++idx) { PTypeDesc type = result.getColumnTypes(idx); - String colName = result.getColumnNames(idx); - columns.add(new Column(colName, getColumnType(type.getTypesList(), 0).key(), true)); + String colName = result.getColumnNames(idx).toLowerCase(); + // Since doris does not distinguish between upper and lower case columns when querying, in order to avoid + // query ambiguity, two columns with the same name but different capitalization are not allowed. + if (columnLowerNames.contains(colName)) { + throw new NotSupportedException("Repeated lowercase column names: " + colName); + } else { + columnLowerNames.add(colName); + columns.add(new Column(colName, getColumnType(type.getTypesList(), 0).key(), true)); + } } // add path columns // HACK(tsy): path columns are all treated as STRING type now, after BE supports reading all columns diff --git a/gensrc/thrift/ExternalTableSchema.thrift b/gensrc/thrift/ExternalTableSchema.thrift new file mode 100644 index 00000000000000..aff93b811431d1 --- /dev/null +++ b/gensrc/thrift/ExternalTableSchema.thrift @@ -0,0 +1,65 @@ +// 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. + +namespace cpp doris.schema.external +namespace java org.apache.doris.thrift.schema.external + +include "Types.thrift" + +struct TFieldPtr { + 1: optional TField& field_ptr; // `&` is used to generate std::shared_ptr on cpp. +} + +struct TArrayField { + 1: optional TFieldPtr item_field // Element field of the array +} + +struct TMapField { + 1: optional TFieldPtr key_field, // Key field of the map + 2: optional TFieldPtr value_field // Value field of the map +} + +struct TStructField { + 1: optional list fields // List of sub-fields for the struct +} + +union TNestedField { + 1: TArrayField array_field, + 2: TStructField struct_field, + 3: TMapField map_field +} + +struct TField { + 1: optional bool is_optional, + 2: optional i32 id, // Field unique id + 3: optional string name, // Field name + 4: optional Types.TColumnType type, // Corresponding Doris column type + 5: optional TNestedField nestedField // Nested field definition (for array, struct, or map types) +} + + +struct TSchema { + 1: optional i64 schema_id, // Each time a iceberg/hudi/paimon table schema changes, a new schema id is generated. + + // Used to represent all columns in the current table, treating all columns in the table as a struct. + // The reason for not using `list` is to reduce logical duplication in the code. + // For example: + // desc table: a int , + // b string + // struct + 2: optional TStructField root_field +} diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index 172ec0fa8a5ad1..b5c51c09b4c976 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -23,6 +23,7 @@ include "Types.thrift" include "Opcodes.thrift" include "Partitions.thrift" include "Descriptors.thrift" +include "ExternalTableSchema.thrift" enum TPlanNodeType { OLAP_SCAN_NODE, @@ -471,7 +472,12 @@ struct TFileScanRangeParams { // 1. Reduce the access to HMS and HDFS on the JNI side. // 2. There will be no inconsistency between the fe and be tables. 24: optional string serialized_table - 25: optional map> history_schema_info // paimon/hudi map> : for schema change. (native reader) + + // Every time a iceberg/hudi/paimon table makes a schema change, a new schema id is generated. + // This is used to represent the latest id. + 25: optional i64 current_schema_id; + // All schema information used in the current query process + 26: optional list history_schema_info } struct TFileRangeDesc { diff --git a/regression-test/data/external_table_p0/iceberg/test_iceberg_full_schema_change.out b/regression-test/data/external_table_p0/iceberg/test_iceberg_full_schema_change.out new file mode 100644 index 00000000000000..d0723e4a78263a --- /dev/null +++ b/regression-test/data/external_table_p0/iceberg/test_iceberg_full_schema_change.out @@ -0,0 +1,649 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !all -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !country_usa -- +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !country_usa_cols -- +5 Phoenix Eve 2 +6 Philadelphia Frank 2 +7 San Antonio Grace 2 +8 San Diego Hank 2 +9 Dallas Ivy 2 +10 Austin Jack 2 +11 Seattle Karen 2 +12 Portland Leo 2 +13 Denver Mona 2 +14 Miami Nina 2 +15 New York Emma Smith 2 + +-- !city_new -- +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !city_new_cols -- +1 \N 25 Apple +15 USA 30 Banana + +-- !age_over_30 -- +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !age_over_30_cols -- +4 Houston \N +5 Phoenix \N +8 San Diego \N +11 Seattle \N +13 Denver Vegetable +17 Sydney Vegetable +18 Berlin Vegetable +20 Osaka Vegetable +21 Rome Food + +-- !age_under_25 -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N + +-- !age_under_25_cols -- +0 \N person0 +6 USA person6 +12 USA person12 + +-- !name_alice -- +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N + +-- !name_alice_cols -- +1 New York 2 + +-- !name_j -- +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} + +-- !name_j_cols -- +10 USA \N +20 Japan Male + +-- !map_person5 -- +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N + +-- !map_person5_cols -- +5 Phoenix 40 + +-- !array_size_2 -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !array_size_2_cols -- +0 \N \N +1 \N \N +2 \N \N +3 \N \N +4 \N \N +5 USA \N +6 USA \N +7 USA \N +8 USA \N +9 USA \N +10 USA \N +11 USA \N +12 USA \N +13 USA \N +14 USA \N +15 USA \N +16 UK NestedCC +17 Australia UpdatedCC +18 Germany NestedCC18 +19 France ReorderedCC +20 Japan FinalCC +21 Italy ExampleCC + +-- !quantity_not_null -- +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !quantity_not_null_cols -- +8 San Diego Hank +9 Dallas Ivy +10 Austin Jack +11 Seattle Karen +12 Portland Leo +13 Denver Mona +14 Miami Nina +15 New York Emma Smith +16 London Liam Brown +17 Sydney Olivia Davis +18 Berlin Noah Wilson +19 Paris Ava Martinez +20 Osaka James Lee +21 Rome Sophia White + +-- !quantity_null -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N + +-- !quantity_null_cols -- +0 \N 2 +1 \N 2 +2 \N 2 +3 \N 2 +4 \N 2 +5 USA 2 +6 USA 2 +7 USA 2 + +-- !struct2_not_null -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !struct2_not_null_cols -- +16 9 50 +17 12 60 +18 15 70 +19 18 85 +20 21 95 +21 25 100 + +-- !struct2_null -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !struct2_null_cols -- +0 cn +1 New York +2 Los Angeles +3 Chicago +4 Houston +5 Phoenix +6 Philadelphia +7 San Antonio +8 San Diego +9 Dallas +10 Austin +11 Seattle +12 Portland +13 Denver +14 Miami +15 New York + +-- !cc_nested -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} + +-- !cc_nested_cols -- +16 9 +18 15 + +-- !c_over_20 -- +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !c_over_20_cols -- +20 FinalCC +21 ExampleCC + +-- !new_aa_50 -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} + +-- !new_aa_50_cols -- +16 9 + +-- !gender_female -- +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !gender_female_cols -- +11 Seattle 2 +13 Denver 2 +14 Miami 2 +15 New York 2 +17 Sydney 2 +19 Paris 2 +21 Rome 2 + +-- !category_fruit -- + +-- !category_fruit_cols -- + +-- !category_vegetable -- +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} + +-- !category_vegetable_cols -- +13 Denver 33 +14 Miami 28 +15 New York 30 +17 Sydney 40 +18 Berlin 33 +19 Paris 29 +20 Osaka 38 + +-- !all -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !country_usa -- +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !country_usa_cols -- +5 Phoenix Eve 2 +6 Philadelphia Frank 2 +7 San Antonio Grace 2 +8 San Diego Hank 2 +9 Dallas Ivy 2 +10 Austin Jack 2 +11 Seattle Karen 2 +12 Portland Leo 2 +13 Denver Mona 2 +14 Miami Nina 2 +15 New York Emma Smith 2 + +-- !city_new -- +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !city_new_cols -- +1 \N 25 Apple +15 USA 30 Banana + +-- !age_over_30 -- +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !age_over_30_cols -- +4 Houston \N +5 Phoenix \N +8 San Diego \N +11 Seattle \N +13 Denver Vegetable +17 Sydney Vegetable +18 Berlin Vegetable +20 Osaka Vegetable +21 Rome Food + +-- !age_under_25 -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N + +-- !age_under_25_cols -- +0 \N person0 +6 USA person6 +12 USA person12 + +-- !name_alice -- +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N + +-- !name_alice_cols -- +1 New York 2 + +-- !name_j -- +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} + +-- !name_j_cols -- +10 USA \N +20 Japan Male + +-- !map_person5 -- +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N + +-- !map_person5_cols -- +5 Phoenix 40 + +-- !array_size_2 -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !array_size_2_cols -- +0 \N \N +1 \N \N +2 \N \N +3 \N \N +4 \N \N +5 USA \N +6 USA \N +7 USA \N +8 USA \N +9 USA \N +10 USA \N +11 USA \N +12 USA \N +13 USA \N +14 USA \N +15 USA \N +16 UK NestedCC +17 Australia UpdatedCC +18 Germany NestedCC18 +19 France ReorderedCC +20 Japan FinalCC +21 Italy ExampleCC + +-- !quantity_not_null -- +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !quantity_not_null_cols -- +8 San Diego Hank +9 Dallas Ivy +10 Austin Jack +11 Seattle Karen +12 Portland Leo +13 Denver Mona +14 Miami Nina +15 New York Emma Smith +16 London Liam Brown +17 Sydney Olivia Davis +18 Berlin Noah Wilson +19 Paris Ava Martinez +20 Osaka James Lee +21 Rome Sophia White + +-- !quantity_null -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N + +-- !quantity_null_cols -- +0 \N 2 +1 \N 2 +2 \N 2 +3 \N 2 +4 \N 2 +5 USA 2 +6 USA 2 +7 USA 2 + +-- !struct2_not_null -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !struct2_not_null_cols -- +16 9 50 +17 12 60 +18 15 70 +19 18 85 +20 21 95 +21 25 100 + +-- !struct2_null -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !struct2_null_cols -- +0 cn +1 New York +2 Los Angeles +3 Chicago +4 Houston +5 Phoenix +6 Philadelphia +7 San Antonio +8 San Diego +9 Dallas +10 Austin +11 Seattle +12 Portland +13 Denver +14 Miami +15 New York + +-- !cc_nested -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} + +-- !cc_nested_cols -- +16 9 +18 15 + +-- !c_over_20 -- +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !c_over_20_cols -- +20 FinalCC +21 ExampleCC + +-- !new_aa_50 -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} + +-- !new_aa_50_cols -- +16 9 + +-- !gender_female -- +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !gender_female_cols -- +11 Seattle 2 +13 Denver 2 +14 Miami 2 +15 New York 2 +17 Sydney 2 +19 Paris 2 +21 Rome 2 + +-- !category_fruit -- + +-- !category_fruit_cols -- + +-- !category_vegetable -- +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} + +-- !category_vegetable_cols -- +13 Denver 33 +14 Miami 28 +15 New York 30 +17 Sydney 40 +18 Berlin 33 +19 Paris 29 +20 Osaka 38 + diff --git a/regression-test/data/external_table_p0/paimon/test_paimon_full_schema_change.out b/regression-test/data/external_table_p0/paimon/test_paimon_full_schema_change.out new file mode 100644 index 00000000000000..9c05c5d3b2c32d --- /dev/null +++ b/regression-test/data/external_table_p0/paimon/test_paimon_full_schema_change.out @@ -0,0 +1,273 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !all -- +0 {"person0":{"full_name":"zero", "age":2, "address":null}} {"location":"cn", "population":null} \N [{"new_product":"Apple", "price":1.99}, {"new_product":"Banana", "price":0.99}] +1 {"person1":{"full_name":"Alice", "age":25, "address":"123 Main St"}} {"location":"New York", "population":null} \N [{"new_product":"Apple", "price":1.99}, {"new_product":"Banana", "price":0.99}] +2 {"person2":{"full_name":"Bob Smith", "age":30, "address":"456 Oak Ave"}} {"location":"Los Angeles", "population":null} \N [{"new_product":"Orange", "price":2.49}, {"new_product":"Grapes", "price":3.99}] +3 {"person3":{"full_name":"Charlie", "age":28, "address":"789 Pine Rd"}} {"location":"Chicago", "population":null} \N [{"new_product":"Mango", "price":2.99}, {"new_product":"Peach", "price":1.49}] +4 {"person4":{"full_name":"Diana Green", "age":35, "address":"321 Elm St"}} {"location":"San Francisco", "population":null} \N [{"new_product":"Strawberry", "price":4.29}, {"new_product":"Blueberry", "price":5.19}] +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] +6 {"person6":{"full_name":"Fiona Lake", "age":29, "address":"654 Birch Ln"}} {"location":"Austin", "population":950000} \N [{"new_product":"Pineapple", "price":3.99}, {"new_product":"Kiwi", "price":2.59}] +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] +8 {"person8":{"full_name":"Hannah King", "age":34, "address":"321 Oak Blvd"}} {"location":"Boston", "population":690000} {"new_a":{"new_aa":2002, "bb":"deep inner string"}, "b":3025, "c":null} [{"new_product":"Dragonfruit", "price":5.25}, {"new_product":"Lychee", "price":4.1}] +9 {"person9":{"full_name":"Ian Moore", "age":38, "address":"888 Maple Way"}} {"location":"Denver", "population":620000} {"new_a":{"new_aa":3003, "bb":"nested value"}, "b":4025, "c":{"cc":"extra info", "new_dd":123}} [{"new_product":"Peach", "price":2.89}, {"new_product":"Plum", "price":2.45}] +10 {"person10":{"full_name":"Julia Nash", "age":27, "address":"456 Cedar Ct"}} {"location":"Phoenix", "population":820000} {"new_a":{"new_aa":4004, "bb":"renamed inner value"}, "b":5025, "c":{"cc":"details", "new_dd":456}} [{"new_product":"Cherry", "price":3.15}, {"new_product":"Apricot", "price":2.95}] +11 {"person11":{"full_name":"Kevin Orr", "age":45, "address":"789 Spruce Dr"}} {"location":"San Diego", "population":770000} {"new_a":{"new_aa":5005, "bb":"final structure"}, "b":6025, "c":{"cc":"notes", "new_dd":789}} [{"new_product":"Nectarine", "price":3.6}, {"new_product":"Coconut", "price":4.2}] +12 {"person12":{"full_name":"Laura Price", "age":36, "address":"1010 Aspen Way"}} {"location":"Dallas", "population":880000} {"new_a":{"new_aa":6006, "bb":"finalized"}, "b":7025, "c":{"cc":"metadata", "new_dd":321}} [{"new_product":"Cranberry", "price":3.3}, {"new_product":"Fig", "price":2.7}] +13 {"person13":{"full_name":"Michael Reed", "age":31, "address":"2020 Pine Cir"}} {"location":"Atlanta", "population":810000} {"new_a":{"new_aa":7007, "bb":"relocated field"}, "b":8025, "c":{"cc":"info", "new_dd":654}} [{"new_product":"Guava", "price":3.95}, {"new_product":"Passionfruit", "price":4.6}] + +-- !location_seattle -- +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] + +-- !location_seattle_cols -- +5 750000 Edward Black 2 +7 730000 George Hall 2 + +-- !location_s -- +4 {"person4":{"full_name":"Diana Green", "age":35, "address":"321 Elm St"}} {"location":"San Francisco", "population":null} \N [{"new_product":"Strawberry", "price":4.29}, {"new_product":"Blueberry", "price":5.19}] +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] +11 {"person11":{"full_name":"Kevin Orr", "age":45, "address":"789 Spruce Dr"}} {"location":"San Diego", "population":770000} {"new_a":{"new_aa":5005, "bb":"final structure"}, "b":6025, "c":{"cc":"notes", "new_dd":789}} [{"new_product":"Nectarine", "price":3.6}, {"new_product":"Coconut", "price":4.2}] + +-- !location_s_cols -- +4 \N 35 Strawberry +5 750000 42 Kiwi +7 730000 41 Mango +11 770000 45 Nectarine + +-- !age_over_30 -- +4 {"person4":{"full_name":"Diana Green", "age":35, "address":"321 Elm St"}} {"location":"San Francisco", "population":null} \N [{"new_product":"Strawberry", "price":4.29}, {"new_product":"Blueberry", "price":5.19}] +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] +8 {"person8":{"full_name":"Hannah King", "age":34, "address":"321 Oak Blvd"}} {"location":"Boston", "population":690000} {"new_a":{"new_aa":2002, "bb":"deep inner string"}, "b":3025, "c":null} [{"new_product":"Dragonfruit", "price":5.25}, {"new_product":"Lychee", "price":4.1}] +9 {"person9":{"full_name":"Ian Moore", "age":38, "address":"888 Maple Way"}} {"location":"Denver", "population":620000} {"new_a":{"new_aa":3003, "bb":"nested value"}, "b":4025, "c":{"cc":"extra info", "new_dd":123}} [{"new_product":"Peach", "price":2.89}, {"new_product":"Plum", "price":2.45}] +11 {"person11":{"full_name":"Kevin Orr", "age":45, "address":"789 Spruce Dr"}} {"location":"San Diego", "population":770000} {"new_a":{"new_aa":5005, "bb":"final structure"}, "b":6025, "c":{"cc":"notes", "new_dd":789}} [{"new_product":"Nectarine", "price":3.6}, {"new_product":"Coconut", "price":4.2}] +12 {"person12":{"full_name":"Laura Price", "age":36, "address":"1010 Aspen Way"}} {"location":"Dallas", "population":880000} {"new_a":{"new_aa":6006, "bb":"finalized"}, "b":7025, "c":{"cc":"metadata", "new_dd":321}} [{"new_product":"Cranberry", "price":3.3}, {"new_product":"Fig", "price":2.7}] +13 {"person13":{"full_name":"Michael Reed", "age":31, "address":"2020 Pine Cir"}} {"location":"Atlanta", "population":810000} {"new_a":{"new_aa":7007, "bb":"relocated field"}, "b":8025, "c":{"cc":"info", "new_dd":654}} [{"new_product":"Guava", "price":3.95}, {"new_product":"Passionfruit", "price":4.6}] + +-- !age_over_30_cols -- +4 San Francisco 5.19 +5 Seattle 3.49 +7 Seattle 3.75 +8 Boston 4.1 +9 Denver 2.45 +11 San Diego 4.2 +12 Dallas 2.7 +13 Atlanta 4.6 + +-- !name_alice -- +1 {"person1":{"full_name":"Alice", "age":25, "address":"123 Main St"}} {"location":"New York", "population":null} \N [{"new_product":"Apple", "price":1.99}, {"new_product":"Banana", "price":0.99}] + +-- !name_alice_cols -- +1 New York 2 + +-- !map_person5 -- +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] + +-- !map_person5_cols -- +5 750000 987 Willow Ln + +-- !array_size_2 -- +0 {"person0":{"full_name":"zero", "age":2, "address":null}} {"location":"cn", "population":null} \N [{"new_product":"Apple", "price":1.99}, {"new_product":"Banana", "price":0.99}] +1 {"person1":{"full_name":"Alice", "age":25, "address":"123 Main St"}} {"location":"New York", "population":null} \N [{"new_product":"Apple", "price":1.99}, {"new_product":"Banana", "price":0.99}] +2 {"person2":{"full_name":"Bob Smith", "age":30, "address":"456 Oak Ave"}} {"location":"Los Angeles", "population":null} \N [{"new_product":"Orange", "price":2.49}, {"new_product":"Grapes", "price":3.99}] +3 {"person3":{"full_name":"Charlie", "age":28, "address":"789 Pine Rd"}} {"location":"Chicago", "population":null} \N [{"new_product":"Mango", "price":2.99}, {"new_product":"Peach", "price":1.49}] +4 {"person4":{"full_name":"Diana Green", "age":35, "address":"321 Elm St"}} {"location":"San Francisco", "population":null} \N [{"new_product":"Strawberry", "price":4.29}, {"new_product":"Blueberry", "price":5.19}] +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] +6 {"person6":{"full_name":"Fiona Lake", "age":29, "address":"654 Birch Ln"}} {"location":"Austin", "population":950000} \N [{"new_product":"Pineapple", "price":3.99}, {"new_product":"Kiwi", "price":2.59}] +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] +8 {"person8":{"full_name":"Hannah King", "age":34, "address":"321 Oak Blvd"}} {"location":"Boston", "population":690000} {"new_a":{"new_aa":2002, "bb":"deep inner string"}, "b":3025, "c":null} [{"new_product":"Dragonfruit", "price":5.25}, {"new_product":"Lychee", "price":4.1}] +9 {"person9":{"full_name":"Ian Moore", "age":38, "address":"888 Maple Way"}} {"location":"Denver", "population":620000} {"new_a":{"new_aa":3003, "bb":"nested value"}, "b":4025, "c":{"cc":"extra info", "new_dd":123}} [{"new_product":"Peach", "price":2.89}, {"new_product":"Plum", "price":2.45}] +10 {"person10":{"full_name":"Julia Nash", "age":27, "address":"456 Cedar Ct"}} {"location":"Phoenix", "population":820000} {"new_a":{"new_aa":4004, "bb":"renamed inner value"}, "b":5025, "c":{"cc":"details", "new_dd":456}} [{"new_product":"Cherry", "price":3.15}, {"new_product":"Apricot", "price":2.95}] +11 {"person11":{"full_name":"Kevin Orr", "age":45, "address":"789 Spruce Dr"}} {"location":"San Diego", "population":770000} {"new_a":{"new_aa":5005, "bb":"final structure"}, "b":6025, "c":{"cc":"notes", "new_dd":789}} [{"new_product":"Nectarine", "price":3.6}, {"new_product":"Coconut", "price":4.2}] +12 {"person12":{"full_name":"Laura Price", "age":36, "address":"1010 Aspen Way"}} {"location":"Dallas", "population":880000} {"new_a":{"new_aa":6006, "bb":"finalized"}, "b":7025, "c":{"cc":"metadata", "new_dd":321}} [{"new_product":"Cranberry", "price":3.3}, {"new_product":"Fig", "price":2.7}] +13 {"person13":{"full_name":"Michael Reed", "age":31, "address":"2020 Pine Cir"}} {"location":"Atlanta", "population":810000} {"new_a":{"new_aa":7007, "bb":"relocated field"}, "b":8025, "c":{"cc":"info", "new_dd":654}} [{"new_product":"Guava", "price":3.95}, {"new_product":"Passionfruit", "price":4.6}] + +-- !array_size_2_cols -- +0 cn Apple +1 New York Apple +2 Los Angeles Orange +3 Chicago Mango +4 San Francisco Strawberry +5 Seattle Kiwi +6 Austin Pineapple +7 Seattle Mango +8 Boston Dragonfruit +9 Denver Peach +10 Phoenix Cherry +11 San Diego Nectarine +12 Dallas Cranberry +13 Atlanta Guava + +-- !struct2_not_null -- +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] +8 {"person8":{"full_name":"Hannah King", "age":34, "address":"321 Oak Blvd"}} {"location":"Boston", "population":690000} {"new_a":{"new_aa":2002, "bb":"deep inner string"}, "b":3025, "c":null} [{"new_product":"Dragonfruit", "price":5.25}, {"new_product":"Lychee", "price":4.1}] +9 {"person9":{"full_name":"Ian Moore", "age":38, "address":"888 Maple Way"}} {"location":"Denver", "population":620000} {"new_a":{"new_aa":3003, "bb":"nested value"}, "b":4025, "c":{"cc":"extra info", "new_dd":123}} [{"new_product":"Peach", "price":2.89}, {"new_product":"Plum", "price":2.45}] +10 {"person10":{"full_name":"Julia Nash", "age":27, "address":"456 Cedar Ct"}} {"location":"Phoenix", "population":820000} {"new_a":{"new_aa":4004, "bb":"renamed inner value"}, "b":5025, "c":{"cc":"details", "new_dd":456}} [{"new_product":"Cherry", "price":3.15}, {"new_product":"Apricot", "price":2.95}] +11 {"person11":{"full_name":"Kevin Orr", "age":45, "address":"789 Spruce Dr"}} {"location":"San Diego", "population":770000} {"new_a":{"new_aa":5005, "bb":"final structure"}, "b":6025, "c":{"cc":"notes", "new_dd":789}} [{"new_product":"Nectarine", "price":3.6}, {"new_product":"Coconut", "price":4.2}] +12 {"person12":{"full_name":"Laura Price", "age":36, "address":"1010 Aspen Way"}} {"location":"Dallas", "population":880000} {"new_a":{"new_aa":6006, "bb":"finalized"}, "b":7025, "c":{"cc":"metadata", "new_dd":321}} [{"new_product":"Cranberry", "price":3.3}, {"new_product":"Fig", "price":2.7}] +13 {"person13":{"full_name":"Michael Reed", "age":31, "address":"2020 Pine Cir"}} {"location":"Atlanta", "population":810000} {"new_a":{"new_aa":7007, "bb":"relocated field"}, "b":8025, "c":{"cc":"info", "new_dd":654}} [{"new_product":"Guava", "price":3.95}, {"new_product":"Passionfruit", "price":4.6}] + +-- !struct2_not_null_cols -- +7 2025 \N +8 3025 \N +9 4025 extra info +10 5025 details +11 6025 notes +12 7025 metadata +13 8025 info + +-- !new_aa_1001 -- +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] + +-- !new_aa_1001_cols -- +7 2025 person7 + +-- !population_over_800k -- +6 {"person6":{"full_name":"Fiona Lake", "age":29, "address":"654 Birch Ln"}} {"location":"Austin", "population":950000} \N [{"new_product":"Pineapple", "price":3.99}, {"new_product":"Kiwi", "price":2.59}] +10 {"person10":{"full_name":"Julia Nash", "age":27, "address":"456 Cedar Ct"}} {"location":"Phoenix", "population":820000} {"new_a":{"new_aa":4004, "bb":"renamed inner value"}, "b":5025, "c":{"cc":"details", "new_dd":456}} [{"new_product":"Cherry", "price":3.15}, {"new_product":"Apricot", "price":2.95}] +12 {"person12":{"full_name":"Laura Price", "age":36, "address":"1010 Aspen Way"}} {"location":"Dallas", "population":880000} {"new_a":{"new_aa":6006, "bb":"finalized"}, "b":7025, "c":{"cc":"metadata", "new_dd":321}} [{"new_product":"Cranberry", "price":3.3}, {"new_product":"Fig", "price":2.7}] +13 {"person13":{"full_name":"Michael Reed", "age":31, "address":"2020 Pine Cir"}} {"location":"Atlanta", "population":810000} {"new_a":{"new_aa":7007, "bb":"relocated field"}, "b":8025, "c":{"cc":"info", "new_dd":654}} [{"new_product":"Guava", "price":3.95}, {"new_product":"Passionfruit", "price":4.6}] + +-- !population_over_800k_cols -- +6 Austin 29 +10 Phoenix 27 +12 Dallas 36 +13 Atlanta 31 + +-- !all -- +0 {"person0":{"full_name":"zero", "age":2, "address":null}} {"location":"cn", "population":null} \N [{"new_product":"Apple", "price":1.99}, {"new_product":"Banana", "price":0.99}] +1 {"person1":{"full_name":"Alice", "age":25, "address":"123 Main St"}} {"location":"New York", "population":null} \N [{"new_product":"Apple", "price":1.99}, {"new_product":"Banana", "price":0.99}] +2 {"person2":{"full_name":"Bob Smith", "age":30, "address":"456 Oak Ave"}} {"location":"Los Angeles", "population":null} \N [{"new_product":"Orange", "price":2.49}, {"new_product":"Grapes", "price":3.99}] +3 {"person3":{"full_name":"Charlie", "age":28, "address":"789 Pine Rd"}} {"location":"Chicago", "population":null} \N [{"new_product":"Mango", "price":2.99}, {"new_product":"Peach", "price":1.49}] +4 {"person4":{"full_name":"Diana Green", "age":35, "address":"321 Elm St"}} {"location":"San Francisco", "population":null} \N [{"new_product":"Strawberry", "price":4.29}, {"new_product":"Blueberry", "price":5.19}] +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] +6 {"person6":{"full_name":"Fiona Lake", "age":29, "address":"654 Birch Ln"}} {"location":"Austin", "population":950000} \N [{"new_product":"Pineapple", "price":3.99}, {"new_product":"Kiwi", "price":2.59}] +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] +8 {"person8":{"full_name":"Hannah King", "age":34, "address":"321 Oak Blvd"}} {"location":"Boston", "population":690000} {"new_a":{"new_aa":2002, "bb":"deep inner string"}, "b":3025, "c":null} [{"new_product":"Dragonfruit", "price":5.25}, {"new_product":"Lychee", "price":4.1}] +9 {"person9":{"full_name":"Ian Moore", "age":38, "address":"888 Maple Way"}} {"location":"Denver", "population":620000} {"new_a":{"new_aa":3003, "bb":"nested value"}, "b":4025, "c":{"cc":"extra info", "new_dd":123}} [{"new_product":"Peach", "price":2.89}, {"new_product":"Plum", "price":2.45}] +10 {"person10":{"full_name":"Julia Nash", "age":27, "address":"456 Cedar Ct"}} {"location":"Phoenix", "population":820000} {"new_a":{"new_aa":4004, "bb":"renamed inner value"}, "b":5025, "c":{"cc":"details", "new_dd":456}} [{"new_product":"Cherry", "price":3.15}, {"new_product":"Apricot", "price":2.95}] +11 {"person11":{"full_name":"Kevin Orr", "age":45, "address":"789 Spruce Dr"}} {"location":"San Diego", "population":770000} {"new_a":{"new_aa":5005, "bb":"final structure"}, "b":6025, "c":{"cc":"notes", "new_dd":789}} [{"new_product":"Nectarine", "price":3.6}, {"new_product":"Coconut", "price":4.2}] +12 {"person12":{"full_name":"Laura Price", "age":36, "address":"1010 Aspen Way"}} {"location":"Dallas", "population":880000} {"new_a":{"new_aa":6006, "bb":"finalized"}, "b":7025, "c":{"cc":"metadata", "new_dd":321}} [{"new_product":"Cranberry", "price":3.3}, {"new_product":"Fig", "price":2.7}] +13 {"person13":{"full_name":"Michael Reed", "age":31, "address":"2020 Pine Cir"}} {"location":"Atlanta", "population":810000} {"new_a":{"new_aa":7007, "bb":"relocated field"}, "b":8025, "c":{"cc":"info", "new_dd":654}} [{"new_product":"Guava", "price":3.95}, {"new_product":"Passionfruit", "price":4.6}] + +-- !location_seattle -- +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] + +-- !location_seattle_cols -- +5 750000 Edward Black 2 +7 730000 George Hall 2 + +-- !location_s -- +4 {"person4":{"full_name":"Diana Green", "age":35, "address":"321 Elm St"}} {"location":"San Francisco", "population":null} \N [{"new_product":"Strawberry", "price":4.29}, {"new_product":"Blueberry", "price":5.19}] +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] +11 {"person11":{"full_name":"Kevin Orr", "age":45, "address":"789 Spruce Dr"}} {"location":"San Diego", "population":770000} {"new_a":{"new_aa":5005, "bb":"final structure"}, "b":6025, "c":{"cc":"notes", "new_dd":789}} [{"new_product":"Nectarine", "price":3.6}, {"new_product":"Coconut", "price":4.2}] + +-- !location_s_cols -- +4 \N 35 Strawberry +5 750000 42 Kiwi +7 730000 41 Mango +11 770000 45 Nectarine + +-- !age_over_30 -- +4 {"person4":{"full_name":"Diana Green", "age":35, "address":"321 Elm St"}} {"location":"San Francisco", "population":null} \N [{"new_product":"Strawberry", "price":4.29}, {"new_product":"Blueberry", "price":5.19}] +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] +8 {"person8":{"full_name":"Hannah King", "age":34, "address":"321 Oak Blvd"}} {"location":"Boston", "population":690000} {"new_a":{"new_aa":2002, "bb":"deep inner string"}, "b":3025, "c":null} [{"new_product":"Dragonfruit", "price":5.25}, {"new_product":"Lychee", "price":4.1}] +9 {"person9":{"full_name":"Ian Moore", "age":38, "address":"888 Maple Way"}} {"location":"Denver", "population":620000} {"new_a":{"new_aa":3003, "bb":"nested value"}, "b":4025, "c":{"cc":"extra info", "new_dd":123}} [{"new_product":"Peach", "price":2.89}, {"new_product":"Plum", "price":2.45}] +11 {"person11":{"full_name":"Kevin Orr", "age":45, "address":"789 Spruce Dr"}} {"location":"San Diego", "population":770000} {"new_a":{"new_aa":5005, "bb":"final structure"}, "b":6025, "c":{"cc":"notes", "new_dd":789}} [{"new_product":"Nectarine", "price":3.6}, {"new_product":"Coconut", "price":4.2}] +12 {"person12":{"full_name":"Laura Price", "age":36, "address":"1010 Aspen Way"}} {"location":"Dallas", "population":880000} {"new_a":{"new_aa":6006, "bb":"finalized"}, "b":7025, "c":{"cc":"metadata", "new_dd":321}} [{"new_product":"Cranberry", "price":3.3}, {"new_product":"Fig", "price":2.7}] +13 {"person13":{"full_name":"Michael Reed", "age":31, "address":"2020 Pine Cir"}} {"location":"Atlanta", "population":810000} {"new_a":{"new_aa":7007, "bb":"relocated field"}, "b":8025, "c":{"cc":"info", "new_dd":654}} [{"new_product":"Guava", "price":3.95}, {"new_product":"Passionfruit", "price":4.6}] + +-- !age_over_30_cols -- +4 San Francisco 5.19 +5 Seattle 3.49 +7 Seattle 3.75 +8 Boston 4.1 +9 Denver 2.45 +11 San Diego 4.2 +12 Dallas 2.7 +13 Atlanta 4.6 + +-- !name_alice -- +1 {"person1":{"full_name":"Alice", "age":25, "address":"123 Main St"}} {"location":"New York", "population":null} \N [{"new_product":"Apple", "price":1.99}, {"new_product":"Banana", "price":0.99}] + +-- !name_alice_cols -- +1 New York 2 + +-- !map_person5 -- +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] + +-- !map_person5_cols -- +5 750000 987 Willow Ln + +-- !array_size_2 -- +0 {"person0":{"full_name":"zero", "age":2, "address":null}} {"location":"cn", "population":null} \N [{"new_product":"Apple", "price":1.99}, {"new_product":"Banana", "price":0.99}] +1 {"person1":{"full_name":"Alice", "age":25, "address":"123 Main St"}} {"location":"New York", "population":null} \N [{"new_product":"Apple", "price":1.99}, {"new_product":"Banana", "price":0.99}] +2 {"person2":{"full_name":"Bob Smith", "age":30, "address":"456 Oak Ave"}} {"location":"Los Angeles", "population":null} \N [{"new_product":"Orange", "price":2.49}, {"new_product":"Grapes", "price":3.99}] +3 {"person3":{"full_name":"Charlie", "age":28, "address":"789 Pine Rd"}} {"location":"Chicago", "population":null} \N [{"new_product":"Mango", "price":2.99}, {"new_product":"Peach", "price":1.49}] +4 {"person4":{"full_name":"Diana Green", "age":35, "address":"321 Elm St"}} {"location":"San Francisco", "population":null} \N [{"new_product":"Strawberry", "price":4.29}, {"new_product":"Blueberry", "price":5.19}] +5 {"person5":{"full_name":"Edward Black", "age":42, "address":"987 Willow Ln"}} {"location":"Seattle", "population":750000} \N [{"new_product":"Kiwi", "price":1.79}, {"new_product":"Pineapple", "price":3.49}] +6 {"person6":{"full_name":"Fiona Lake", "age":29, "address":"654 Birch Ln"}} {"location":"Austin", "population":950000} \N [{"new_product":"Pineapple", "price":3.99}, {"new_product":"Kiwi", "price":2.59}] +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] +8 {"person8":{"full_name":"Hannah King", "age":34, "address":"321 Oak Blvd"}} {"location":"Boston", "population":690000} {"new_a":{"new_aa":2002, "bb":"deep inner string"}, "b":3025, "c":null} [{"new_product":"Dragonfruit", "price":5.25}, {"new_product":"Lychee", "price":4.1}] +9 {"person9":{"full_name":"Ian Moore", "age":38, "address":"888 Maple Way"}} {"location":"Denver", "population":620000} {"new_a":{"new_aa":3003, "bb":"nested value"}, "b":4025, "c":{"cc":"extra info", "new_dd":123}} [{"new_product":"Peach", "price":2.89}, {"new_product":"Plum", "price":2.45}] +10 {"person10":{"full_name":"Julia Nash", "age":27, "address":"456 Cedar Ct"}} {"location":"Phoenix", "population":820000} {"new_a":{"new_aa":4004, "bb":"renamed inner value"}, "b":5025, "c":{"cc":"details", "new_dd":456}} [{"new_product":"Cherry", "price":3.15}, {"new_product":"Apricot", "price":2.95}] +11 {"person11":{"full_name":"Kevin Orr", "age":45, "address":"789 Spruce Dr"}} {"location":"San Diego", "population":770000} {"new_a":{"new_aa":5005, "bb":"final structure"}, "b":6025, "c":{"cc":"notes", "new_dd":789}} [{"new_product":"Nectarine", "price":3.6}, {"new_product":"Coconut", "price":4.2}] +12 {"person12":{"full_name":"Laura Price", "age":36, "address":"1010 Aspen Way"}} {"location":"Dallas", "population":880000} {"new_a":{"new_aa":6006, "bb":"finalized"}, "b":7025, "c":{"cc":"metadata", "new_dd":321}} [{"new_product":"Cranberry", "price":3.3}, {"new_product":"Fig", "price":2.7}] +13 {"person13":{"full_name":"Michael Reed", "age":31, "address":"2020 Pine Cir"}} {"location":"Atlanta", "population":810000} {"new_a":{"new_aa":7007, "bb":"relocated field"}, "b":8025, "c":{"cc":"info", "new_dd":654}} [{"new_product":"Guava", "price":3.95}, {"new_product":"Passionfruit", "price":4.6}] + +-- !array_size_2_cols -- +0 cn Apple +1 New York Apple +2 Los Angeles Orange +3 Chicago Mango +4 San Francisco Strawberry +5 Seattle Kiwi +6 Austin Pineapple +7 Seattle Mango +8 Boston Dragonfruit +9 Denver Peach +10 Phoenix Cherry +11 San Diego Nectarine +12 Dallas Cranberry +13 Atlanta Guava + +-- !struct2_not_null -- +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] +8 {"person8":{"full_name":"Hannah King", "age":34, "address":"321 Oak Blvd"}} {"location":"Boston", "population":690000} {"new_a":{"new_aa":2002, "bb":"deep inner string"}, "b":3025, "c":null} [{"new_product":"Dragonfruit", "price":5.25}, {"new_product":"Lychee", "price":4.1}] +9 {"person9":{"full_name":"Ian Moore", "age":38, "address":"888 Maple Way"}} {"location":"Denver", "population":620000} {"new_a":{"new_aa":3003, "bb":"nested value"}, "b":4025, "c":{"cc":"extra info", "new_dd":123}} [{"new_product":"Peach", "price":2.89}, {"new_product":"Plum", "price":2.45}] +10 {"person10":{"full_name":"Julia Nash", "age":27, "address":"456 Cedar Ct"}} {"location":"Phoenix", "population":820000} {"new_a":{"new_aa":4004, "bb":"renamed inner value"}, "b":5025, "c":{"cc":"details", "new_dd":456}} [{"new_product":"Cherry", "price":3.15}, {"new_product":"Apricot", "price":2.95}] +11 {"person11":{"full_name":"Kevin Orr", "age":45, "address":"789 Spruce Dr"}} {"location":"San Diego", "population":770000} {"new_a":{"new_aa":5005, "bb":"final structure"}, "b":6025, "c":{"cc":"notes", "new_dd":789}} [{"new_product":"Nectarine", "price":3.6}, {"new_product":"Coconut", "price":4.2}] +12 {"person12":{"full_name":"Laura Price", "age":36, "address":"1010 Aspen Way"}} {"location":"Dallas", "population":880000} {"new_a":{"new_aa":6006, "bb":"finalized"}, "b":7025, "c":{"cc":"metadata", "new_dd":321}} [{"new_product":"Cranberry", "price":3.3}, {"new_product":"Fig", "price":2.7}] +13 {"person13":{"full_name":"Michael Reed", "age":31, "address":"2020 Pine Cir"}} {"location":"Atlanta", "population":810000} {"new_a":{"new_aa":7007, "bb":"relocated field"}, "b":8025, "c":{"cc":"info", "new_dd":654}} [{"new_product":"Guava", "price":3.95}, {"new_product":"Passionfruit", "price":4.6}] + +-- !struct2_not_null_cols -- +7 2025 \N +8 3025 \N +9 4025 extra info +10 5025 details +11 6025 notes +12 7025 metadata +13 8025 info + +-- !new_aa_1001 -- +7 {"person7":{"full_name":"George Hall", "age":41, "address":"987 Elm St"}} {"location":"Seattle", "population":730000} {"new_a":{"new_aa":1001, "bb":"inner string"}, "b":2025, "c":null} [{"new_product":"Mango", "price":4.49}, {"new_product":"Papaya", "price":3.75}] + +-- !new_aa_1001_cols -- +7 2025 person7 + +-- !population_over_800k -- +6 {"person6":{"full_name":"Fiona Lake", "age":29, "address":"654 Birch Ln"}} {"location":"Austin", "population":950000} \N [{"new_product":"Pineapple", "price":3.99}, {"new_product":"Kiwi", "price":2.59}] +10 {"person10":{"full_name":"Julia Nash", "age":27, "address":"456 Cedar Ct"}} {"location":"Phoenix", "population":820000} {"new_a":{"new_aa":4004, "bb":"renamed inner value"}, "b":5025, "c":{"cc":"details", "new_dd":456}} [{"new_product":"Cherry", "price":3.15}, {"new_product":"Apricot", "price":2.95}] +12 {"person12":{"full_name":"Laura Price", "age":36, "address":"1010 Aspen Way"}} {"location":"Dallas", "population":880000} {"new_a":{"new_aa":6006, "bb":"finalized"}, "b":7025, "c":{"cc":"metadata", "new_dd":321}} [{"new_product":"Cranberry", "price":3.3}, {"new_product":"Fig", "price":2.7}] +13 {"person13":{"full_name":"Michael Reed", "age":31, "address":"2020 Pine Cir"}} {"location":"Atlanta", "population":810000} {"new_a":{"new_aa":7007, "bb":"relocated field"}, "b":8025, "c":{"cc":"info", "new_dd":654}} [{"new_product":"Guava", "price":3.95}, {"new_product":"Passionfruit", "price":4.6}] + +-- !population_over_800k_cols -- +6 Austin 29 +10 Phoenix 27 +12 Dallas 36 +13 Atlanta 31 + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_full_schema_change.out b/regression-test/data/external_table_p2/hudi/test_hudi_full_schema_change.out new file mode 100644 index 00000000000000..8a1e62f8249b9a --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/test_hudi_full_schema_change.out @@ -0,0 +1,325 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !all -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !country_usa -- +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !country_usa_cols -- +5 Phoenix Eve 2 +6 Philadelphia Frank 2 +7 San Antonio Grace 2 +8 San Diego Hank 2 +9 Dallas Ivy 2 +10 Austin Jack 2 +11 Seattle Karen 2 +12 Portland Leo 2 +13 Denver Mona 2 +14 Miami Nina 2 +15 New York Emma Smith 2 + +-- !city_new -- +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !city_new_cols -- +1 \N 25 Apple +15 USA 30 Banana + +-- !age_over_30 -- +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !age_over_30_cols -- +4 Houston \N +5 Phoenix \N +8 San Diego \N +11 Seattle \N +13 Denver Vegetable +17 Sydney Vegetable +18 Berlin Vegetable +20 Osaka Vegetable +21 Rome Food + +-- !age_under_25 -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N + +-- !age_under_25_cols -- +0 \N person0 +6 USA person6 +12 USA person12 + +-- !name_alice -- +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N + +-- !name_alice_cols -- +1 New York 2 + +-- !name_j -- +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} + +-- !name_j_cols -- +10 USA \N +20 Japan Male + +-- !map_person5 -- +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N + +-- !map_person5_cols -- +5 Phoenix 40 + +-- !array_size_2 -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !array_size_2_cols -- +0 \N \N +1 \N \N +2 \N \N +3 \N \N +4 \N \N +5 USA \N +6 USA \N +7 USA \N +8 USA \N +9 USA \N +10 USA \N +11 USA \N +12 USA \N +13 USA \N +14 USA \N +15 USA \N +16 UK NestedCC +17 Australia UpdatedCC +18 Germany NestedCC18 +19 France ReorderedCC +20 Japan FinalCC +21 Italy ExampleCC + +-- !quantity_not_null -- +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !quantity_not_null_cols -- +8 San Diego Hank +9 Dallas Ivy +10 Austin Jack +11 Seattle Karen +12 Portland Leo +13 Denver Mona +14 Miami Nina +15 New York Emma Smith +16 London Liam Brown +17 Sydney Olivia Davis +18 Berlin Noah Wilson +19 Paris Ava Martinez +20 Osaka James Lee +21 Rome Sophia White + +-- !quantity_null -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N + +-- !quantity_null_cols -- +0 \N 2 +1 \N 2 +2 \N 2 +3 \N 2 +4 \N 2 +5 USA 2 +6 USA 2 +7 USA 2 + +-- !struct2_not_null -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !struct2_not_null_cols -- +16 9 50 +17 12 60 +18 15 70 +19 18 85 +20 21 95 +21 25 100 + +-- !struct2_null -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !struct2_null_cols -- +0 cn +1 New York +2 Los Angeles +3 Chicago +4 Houston +5 Phoenix +6 Philadelphia +7 San Antonio +8 San Diego +9 Dallas +10 Austin +11 Seattle +12 Portland +13 Denver +14 Miami +15 New York + +-- !cc_nested -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} + +-- !cc_nested_cols -- +16 9 +18 15 + +-- !c_over_20 -- +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !c_over_20_cols -- +20 FinalCC +21 ExampleCC + +-- !new_aa_50 -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} + +-- !new_aa_50_cols -- +16 9 + +-- !gender_female -- +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !gender_female_cols -- +11 Seattle 2 +13 Denver 2 +14 Miami 2 +15 New York 2 +17 Sydney 2 +19 Paris 2 +21 Rome 2 + +-- !category_fruit -- + +-- !category_fruit_cols -- + +-- !category_vegetable -- +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} + +-- !category_vegetable_cols -- +13 Denver 33 +14 Miami 28 +15 New York 30 +17 Sydney 40 +18 Berlin 33 +19 Paris 29 +20 Osaka 38 + diff --git a/regression-test/suites/external_table_p0/iceberg/test_iceberg_full_schema_change.groovy b/regression-test/suites/external_table_p0/iceberg/test_iceberg_full_schema_change.groovy new file mode 100644 index 00000000000000..b47ee42afd935a --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/test_iceberg_full_schema_change.groovy @@ -0,0 +1,116 @@ +// 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. + +suite("test_iceberg_full_schema_change", "p0,external,doris,external_docker,external_docker_doris") { + + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable iceberg test.") + return + } + + String catalog_name = "test_iceberg_full_schema_change" + String rest_port = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + sql """drop catalog if exists ${catalog_name}""" + sql """ + CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://${externalEnvIp}:${rest_port}', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + logger.info("catalog " + catalog_name + " created") + sql """switch ${catalog_name};""" + logger.info("switched to catalog " + catalog_name) + sql """ use test_db;""" + + def tables = ["iceberg_full_schema_change_parquet", "iceberg_full_schema_change_orc"] + + + for (String table: tables) { + qt_all """ select * FROM ${table} ORDER BY id""" + + qt_country_usa """select * FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'country') = 'USA' ORDER BY id""" + qt_country_usa_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') AS full_name, ARRAY_SIZE(array_column) AS array_size FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'country') = 'USA' ORDER BY id""" + + qt_city_new """select * FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'city') LIKE 'New%' ORDER BY id""" + qt_city_new_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') AS age, STRUCT_ELEMENT(array_column[1], 'item') AS first_item FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'city') LIKE 'New%' ORDER BY id""" + + qt_age_over_30 """select * FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') > 30 ORDER BY id""" + qt_age_over_30_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, STRUCT_ELEMENT(array_column[2], 'category') AS second_category FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') > 30 ORDER BY id""" + + qt_age_under_25 """select * FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') < 25 ORDER BY id""" + qt_age_under_25_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, MAP_KEYS(new_map_column)[1] AS map_key FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') < 25 ORDER BY id""" + + qt_name_alice """select * FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') = 'Alice' ORDER BY id""" + qt_name_alice_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, ARRAY_SIZE(array_column) AS array_size FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') = 'Alice' ORDER BY id""" + + qt_name_j """select * FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') like 'J%' ORDER BY id""" + qt_name_j_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'gender') AS gender FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') LIKE 'J%' ORDER BY id""" + + qt_map_person5 """select * FROM ${table} WHERE ARRAY_CONTAINS(MAP_KEYS(new_map_column), 'person5') ORDER BY id""" + qt_map_person5_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') AS age FROM ${table} WHERE ARRAY_CONTAINS(MAP_KEYS(new_map_column), 'person5') ORDER BY id""" + + qt_array_size_2 """select * FROM ${table} WHERE ARRAY_SIZE(array_column) = 2 ORDER BY id""" + qt_array_size_2_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'b'), 'cc') AS b_cc FROM ${table} WHERE ARRAY_SIZE(array_column) = 2 ORDER BY id""" + + qt_quantity_not_null """select * FROM ${table} WHERE STRUCT_ELEMENT(array_column[1], 'quantity') IS NOT NULL ORDER BY id""" + qt_quantity_not_null_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') AS full_name FROM ${table} WHERE STRUCT_ELEMENT(array_column[1], 'quantity') IS NOT NULL ORDER BY id""" + + qt_quantity_null """select * FROM ${table} WHERE STRUCT_ELEMENT(array_column[1], 'quantity') IS NULL ORDER BY id""" + qt_quantity_null_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, ARRAY_SIZE(array_column) AS array_size FROM ${table} WHERE STRUCT_ELEMENT(array_column[1], 'quantity') IS NULL ORDER BY id""" + + qt_struct2_not_null """select * FROM ${table} WHERE struct_column2 IS NOT NULL ORDER BY id""" + qt_struct2_not_null_cols """select id, STRUCT_ELEMENT(struct_column2, 'c') AS c_value, STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'new_a'), 'new_aa') AS new_aa FROM ${table} WHERE struct_column2 IS NOT NULL ORDER BY id""" + + qt_struct2_null """select * FROM ${table} WHERE struct_column2 IS NULL ORDER BY id""" + qt_struct2_null_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city FROM ${table} WHERE struct_column2 IS NULL ORDER BY id""" + + qt_cc_nested """select * FROM ${table} WHERE STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'b'), 'cc') like 'NestedC%' ORDER BY id""" + qt_cc_nested_cols """select id, STRUCT_ELEMENT(struct_column2, 'c') AS c_value FROM ${table} WHERE STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'b'), 'cc') LIKE 'NestedC%' ORDER BY id""" + + qt_c_over_20 """select * FROM ${table} WHERE STRUCT_ELEMENT(struct_column2, 'c') > 20 ORDER BY id""" + qt_c_over_20_cols """select id, STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'b'), 'cc') AS b_cc FROM ${table} WHERE STRUCT_ELEMENT(struct_column2, 'c') > 20 ORDER BY id""" + + qt_new_aa_50 """select * FROM ${table} WHERE STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'new_a'), 'new_aa') = 50 ORDER BY id""" + qt_new_aa_50_cols """select id, STRUCT_ELEMENT(struct_column2, 'c') AS c_value FROM ${table} WHERE STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'new_a'), 'new_aa') = 50 ORDER BY id""" + + qt_gender_female """select * FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'gender') = 'Female' ORDER BY id""" + qt_gender_female_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, ARRAY_SIZE(array_column) AS array_size FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'gender') = 'Female' ORDER BY id""" + + qt_category_fruit """select * FROM ${table} WHERE STRUCT_ELEMENT(array_column[2], 'category') = 'Fruit' ORDER BY id""" + qt_category_fruit_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') AS full_name FROM ${table} WHERE STRUCT_ELEMENT(array_column[2], 'category') = 'Fruit' ORDER BY id""" + + qt_category_vegetable """select * FROM ${table} WHERE STRUCT_ELEMENT(array_column[2], 'category') = 'Vegetable' ORDER BY id""" + qt_category_vegetable_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') AS age FROM ${table} WHERE STRUCT_ELEMENT(array_column[2], 'category') = 'Vegetable' ORDER BY id""" + } +} + + + + + + + + diff --git a/regression-test/suites/external_table_p0/paimon/test_paimon_full_schema_change.groovy b/regression-test/suites/external_table_p0/paimon/test_paimon_full_schema_change.groovy new file mode 100644 index 00000000000000..0c71625e05283e --- /dev/null +++ b/regression-test/suites/external_table_p0/paimon/test_paimon_full_schema_change.groovy @@ -0,0 +1,80 @@ +// 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. + +suite("test_paimon_full_schema_change", "p0,external,doris,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enablePaimonTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String catalog_name = "test_paimon_full_schema_change" + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + String table_name = "ts_scale_orc" + + sql """drop catalog if exists ${catalog_name}""" + + sql """ + CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type' = 'paimon', + 'warehouse' = 's3://warehouse/wh', + 's3.endpoint' = 'http://${externalEnvIp}:${minio_port}', + 's3.access_key' = 'admin', + 's3.secret_key' = 'password', + 's3.path.style.access' = 'true' + ); + """ + sql """switch `${catalog_name}`""" + sql """show databases; """ + sql """use `test_paimon_schema_change`;""" + + + def tables = ["paimon_full_schema_change_parquet","paimon_full_schema_change_orc"] + + for (String table: tables) { + qt_all """ select * FROM ${table} ORDER BY id""" + + qt_location_seattle """select * FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'location') = 'Seattle' ORDER BY id""" + qt_location_seattle_cols """select id, STRUCT_ELEMENT(struct_column, 'population') AS population, STRUCT_ELEMENT(MAP_VALUES(map_column)[1], 'full_name') AS full_name, ARRAY_SIZE(array_column) AS array_size FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'location') = 'Seattle' ORDER BY id""" + + qt_location_s """select * FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'location') LIKE 'S%' ORDER BY id""" + qt_location_s_cols """select id, STRUCT_ELEMENT(struct_column, 'population') AS population, STRUCT_ELEMENT(MAP_VALUES(map_column)[1], 'age') AS age, STRUCT_ELEMENT(array_column[1], 'new_product') AS first_product FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'location') LIKE 'S%' ORDER BY id""" + + qt_age_over_30 """select * FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(map_column)[1], 'age') > 30 ORDER BY id""" + qt_age_over_30_cols """select id, STRUCT_ELEMENT(struct_column, 'location') AS location, STRUCT_ELEMENT(array_column[2], 'price') AS second_price FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(map_column)[1], 'age') > 30 ORDER BY id""" + + qt_name_alice """select * FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(map_column)[1], 'full_name') = 'Alice' ORDER BY id""" + qt_name_alice_cols """select id, STRUCT_ELEMENT(struct_column, 'location') AS location, ARRAY_SIZE(array_column) AS array_size FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(map_column)[1], 'full_name') = 'Alice' ORDER BY id""" + + qt_map_person5 """select * FROM ${table} WHERE ARRAY_CONTAINS(MAP_KEYS(map_column), 'person5') ORDER BY id""" + qt_map_person5_cols """select id, STRUCT_ELEMENT(struct_column, 'population') AS population, STRUCT_ELEMENT(MAP_VALUES(map_column)[1], 'address') AS address FROM ${table} WHERE ARRAY_CONTAINS(MAP_KEYS(map_column), 'person5') ORDER BY id""" + + qt_array_size_2 """select * FROM ${table} WHERE ARRAY_SIZE(array_column) = 2 ORDER BY id""" + qt_array_size_2_cols """select id, STRUCT_ELEMENT(struct_column, 'location') AS location, STRUCT_ELEMENT(array_column[1], 'new_product') AS first_product FROM ${table} WHERE ARRAY_SIZE(array_column) = 2 ORDER BY id""" + + qt_struct2_not_null """select * FROM ${table} WHERE struct_column2 IS NOT NULL ORDER BY id""" + qt_struct2_not_null_cols """select id, STRUCT_ELEMENT(struct_column2, 'b') AS b_value, STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'c'), 'cc') AS c_cc FROM ${table} WHERE struct_column2 IS NOT NULL ORDER BY id""" + + qt_new_aa_1001 """select * FROM ${table} WHERE STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'new_a'), 'new_aa') = 1001 ORDER BY id""" + qt_new_aa_1001_cols """select id, STRUCT_ELEMENT(struct_column2, 'b') AS b_value, MAP_KEYS(map_column)[1] AS map_key FROM ${table} WHERE STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'new_a'), 'new_aa') = 1001 ORDER BY id""" + + qt_population_over_800k """select * FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'population') > 800000 ORDER BY id""" + qt_population_over_800k_cols """select id, STRUCT_ELEMENT(struct_column, 'location') AS location, STRUCT_ELEMENT(MAP_VALUES(map_column)[1], 'age') AS age FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'population') > 800000 ORDER BY id""" + + } + } +} + + diff --git a/regression-test/suites/external_table_p0/tvf/test_hdfs_parquet_group4.groovy b/regression-test/suites/external_table_p0/tvf/test_hdfs_parquet_group4.groovy index 9b6fd1e1c60b87..7e212db35c43da 100644 --- a/regression-test/suites/external_table_p0/tvf/test_hdfs_parquet_group4.groovy +++ b/regression-test/suites/external_table_p0/tvf/test_hdfs_parquet_group4.groovy @@ -2045,7 +2045,7 @@ suite("test_hdfs_parquet_group4","external,hive,tvf,external_docker") { "uri" = "${uri}", "hadoop.username" = "${hdfsUserName}", "format" = "parquet") limit 10; """ - exception "The column type of 'column1' is not supported: INT32 => TimeV2" + exception "The column type of 'COLUMN1' is not supported: INT32 => TimeV2" } diff --git a/regression-test/suites/external_table_p0/tvf/test_hdfs_parquet_group6.groovy b/regression-test/suites/external_table_p0/tvf/test_hdfs_parquet_group6.groovy index 7fd6231f71806f..6b0a1a3fe46351 100644 --- a/regression-test/suites/external_table_p0/tvf/test_hdfs_parquet_group6.groovy +++ b/regression-test/suites/external_table_p0/tvf/test_hdfs_parquet_group6.groovy @@ -214,7 +214,7 @@ suite("test_hdfs_parquet_group6","external,hive,tvf,external_docker") { "uri" = "${uri}", "hadoop.username" = "${hdfsUserName}", "format" = "parquet") limit 10; """ - exception "Duplicated field name: col1" + exception "Repeated lowercase column names: col1" } diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_full_schema_change.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_full_schema_change.groovy new file mode 100644 index 00000000000000..1b427faf53b5af --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_full_schema_change.groovy @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hudi_full_schema_change", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + return + } + + String catalog_name = "test_hudi_full_schema_change" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + sql """set force_jni_scanner = false;""" + + + + def tables = ["hudi_full_schema_change_parquet"] + + + for (String table: tables) { + qt_all """ select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} ORDER BY id""" + + qt_country_usa """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'country') = 'USA' ORDER BY id""" + qt_country_usa_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') AS full_name, ARRAY_SIZE(array_column) AS array_size FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'country') = 'USA' ORDER BY id""" + + qt_city_new """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'city') LIKE 'New%' ORDER BY id""" + qt_city_new_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') AS age, STRUCT_ELEMENT(array_column[1], 'item') AS first_item FROM ${table} WHERE STRUCT_ELEMENT(struct_column, 'city') LIKE 'New%' ORDER BY id""" + + qt_age_over_30 """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') > 30 ORDER BY id""" + qt_age_over_30_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, STRUCT_ELEMENT(array_column[2], 'category') AS second_category FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') > 30 ORDER BY id""" + + qt_age_under_25 """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') < 25 ORDER BY id""" + qt_age_under_25_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, MAP_KEYS(new_map_column)[1] AS map_key FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') < 25 ORDER BY id""" + + qt_name_alice """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') = 'Alice' ORDER BY id""" + qt_name_alice_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, ARRAY_SIZE(array_column) AS array_size FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') = 'Alice' ORDER BY id""" + + qt_name_j """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') like 'J%' ORDER BY id""" + qt_name_j_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'gender') AS gender FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') LIKE 'J%' ORDER BY id""" + + qt_map_person5 """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE ARRAY_CONTAINS(MAP_KEYS(new_map_column), 'person5') ORDER BY id""" + qt_map_person5_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') AS age FROM ${table} WHERE ARRAY_CONTAINS(MAP_KEYS(new_map_column), 'person5') ORDER BY id""" + + qt_array_size_2 """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE ARRAY_SIZE(array_column) = 2 ORDER BY id""" + qt_array_size_2_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'b'), 'cc') AS b_cc FROM ${table} WHERE ARRAY_SIZE(array_column) = 2 ORDER BY id""" + + qt_quantity_not_null """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(array_column[1], 'quantity') IS NOT NULL ORDER BY id""" + qt_quantity_not_null_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') AS full_name FROM ${table} WHERE STRUCT_ELEMENT(array_column[1], 'quantity') IS NOT NULL ORDER BY id""" + + qt_quantity_null """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(array_column[1], 'quantity') IS NULL ORDER BY id""" + qt_quantity_null_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, ARRAY_SIZE(array_column) AS array_size FROM ${table} WHERE STRUCT_ELEMENT(array_column[1], 'quantity') IS NULL ORDER BY id""" + + qt_struct2_not_null """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE struct_column2 IS NOT NULL ORDER BY id""" + qt_struct2_not_null_cols """select id, STRUCT_ELEMENT(struct_column2, 'c') AS c_value, STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'new_a'), 'new_aa') AS new_aa FROM ${table} WHERE struct_column2 IS NOT NULL ORDER BY id""" + + qt_struct2_null """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE struct_column2 IS NULL ORDER BY id""" + qt_struct2_null_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city FROM ${table} WHERE struct_column2 IS NULL ORDER BY id""" + + qt_cc_nested """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'b'), 'cc') like 'NestedC%' ORDER BY id""" + qt_cc_nested_cols """select id, STRUCT_ELEMENT(struct_column2, 'c') AS c_value FROM ${table} WHERE STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'b'), 'cc') LIKE 'NestedC%' ORDER BY id""" + + qt_c_over_20 """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(struct_column2, 'c') > 20 ORDER BY id""" + qt_c_over_20_cols """select id, STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'b'), 'cc') AS b_cc FROM ${table} WHERE STRUCT_ELEMENT(struct_column2, 'c') > 20 ORDER BY id""" + + qt_new_aa_50 """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'new_a'), 'new_aa') = 50 ORDER BY id""" + qt_new_aa_50_cols """select id, STRUCT_ELEMENT(struct_column2, 'c') AS c_value FROM ${table} WHERE STRUCT_ELEMENT(STRUCT_ELEMENT(struct_column2, 'new_a'), 'new_aa') = 50 ORDER BY id""" + + qt_gender_female """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'gender') = 'Female' ORDER BY id""" + qt_gender_female_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, ARRAY_SIZE(array_column) AS array_size FROM ${table} WHERE STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'gender') = 'Female' ORDER BY id""" + + qt_category_fruit """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(array_column[2], 'category') = 'Fruit' ORDER BY id""" + qt_category_fruit_cols """select id, STRUCT_ELEMENT(struct_column, 'country') AS country, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'full_name') AS full_name FROM ${table} WHERE STRUCT_ELEMENT(array_column[2], 'category') = 'Fruit' ORDER BY id""" + + qt_category_vegetable """select id,new_map_column,struct_column, array_column,struct_column2 FROM ${table} WHERE STRUCT_ELEMENT(array_column[2], 'category') = 'Vegetable' ORDER BY id""" + qt_category_vegetable_cols """select id, STRUCT_ELEMENT(struct_column, 'city') AS city, STRUCT_ELEMENT(MAP_VALUES(new_map_column)[1], 'age') AS age FROM ${table} WHERE STRUCT_ELEMENT(array_column[2], 'category') = 'Vegetable' ORDER BY id""" + } +} +/* + +spark-sql \ +--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ +--conf spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension \ +--conf spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog \ +--conf spark.hadoop.hive.metastore.disallow.incompatible.col.type.changes=false + +set hoodie.schema.on.read.enable=true; +set hoodie.metadata.enable=false; +set hoodie.parquet.small.file.limit = 100; + + +CREATE TABLE hudi_full_schema_change_parquet ( + id int, + map_column map < string, struct < name: string, age: int > >, + struct_column struct < city: string, population: int >, + array_column array < struct < product: string, price: float > > +) USING hudi +OPTIONS ( + type = 'cow', + primaryKey = 'id' +); + + +spark-sql (regression_hudi)> select * from hudi_full_schema_change_parquet order by id; +20250516231815213 20250516231815213_0_0 0 300a99ec-a83a-46e3-8c42-e221a3e4ecb0-0_0-850-897_20250516231815213.parquet 0 {"person0":{"age":2,"full_name":"zero","gender":null}} {"country":null,"city":"cn"} [{"item":"Apple","quantity":null,"category":null},{"item":"Banana","quantity":null,"category":null}] NULL +20250516231858211 20250516231858211_0_0 1 198531d4-6ed9-405a-8934-3412e3779f05-0_0-861-907_20250516231858211.parquet 1 {"person1":{"age":25,"full_name":"Alice","gender":null}} {"country":null,"city":"New York"} [{"item":"Apple","quantity":null,"category":null},{"item":"Banana","quantity":null,"category":null}] NULL +20250516231902116 20250516231902116_0_0 2 b61f118e-caeb-44fd-876d-1f113a7176bc-0_0-869-914_20250516231902116.parquet 2 {"person2":{"age":30,"full_name":"Bob","gender":null}} {"country":null,"city":"Los Angeles"} [{"item":"Orange","quantity":null,"category":null},{"item":"Grape","quantity":null,"category":null}] NULL +20250516231907136 20250516231907136_0_0 3 6ce16dd7-39bd-41d7-a9c1-53626b110466-0_0-877-921_20250516231907136.parquet 3 {"person3":{"age":28,"full_name":"Charlie","gender":null}} {"country":null,"city":"Chicago"} [{"item":"Pear","quantity":null,"category":null},{"item":"Mango","quantity":null,"category":null}] NULL +20250516231911544 20250516231911544_0_0 4 d71ea47c-63fe-4c4b-9178-fd58f8edb6ad-0_0-885-928_20250516231911544.parquet 4 {"person4":{"age":35,"full_name":"David","gender":null}} {"country":null,"city":"Houston"} [{"item":"Kiwi","quantity":null,"category":null},{"item":"Pineapple","quantity":null,"category":null}] NULL +20250516231916994 20250516231916994_0_0 5 65e4615a-f513-4713-a4f6-c94954a028ef-0_0-893-935_20250516231916994.parquet 5 {"person5":{"age":40,"full_name":"Eve","gender":null}} {"country":"USA","city":"Phoenix"} [{"item":"Lemon","quantity":null,"category":null},{"item":"Lime","quantity":null,"category":null}] NULL +20250516231922791 20250516231922791_0_0 6 f358d129-97e1-4b03-b521-64c3c61dce06-0_0-905-946_20250516231922791.parquet 6 {"person6":{"age":22,"full_name":"Frank","gender":null}} {"country":"USA","city":"Philadelphia"} [{"item":"Watermelon","quantity":null,"category":null},{"item":"Strawberry","quantity":null,"category":null}] NULL +20250516231927993 20250516231927993_0_0 7 a30b7559-20c2-4f48-8149-f7e53f7fac6b-0_0-917-957_20250516231927993.parquet 7 {"person7":{"age":27,"full_name":"Grace","gender":null}} {"country":"USA","city":"San Antonio"} [{"item":"Blueberry","quantity":null,"category":null},{"item":"Raspberry","quantity":null,"category":null}] NULL +20250516231933098 20250516231933098_0_0 8 a6efa92f-b96c-4479-9350-7f51bbe12f82-0_0-929-968_20250516231933098.parquet 8 {"person8":{"age":32,"full_name":"Hank","gender":null}} {"country":"USA","city":"San Diego"} [{"item":"Cherry","quantity":5,"category":null},{"item":"Plum","quantity":3,"category":null}] NULL +20250516231940027 20250516231940027_0_0 9 cdd96b17-42d0-46cc-aa57-91470f824670-0_0-943-981_20250516231940027.parquet 9 {"person9":{"age":29,"full_name":"Ivy","gender":null}} {"country":"USA","city":"Dallas"} [{"item":"Peach","quantity":4,"category":null},{"item":"Apricot","quantity":2,"category":null}] NULL +20250516231944713 20250516231944713_0_0 10 b6bc2558-9978-48e8-b924-b1a8331b8ea0-0_0-955-992_20250516231944713.parquet 10 {"person10":{"age":26,"full_name":"Jack","gender":null}} {"country":"USA","city":"Austin"} [{"item":"Fig","quantity":6,"category":null},{"item":"Date","quantity":7,"category":null}] NULL +20250516231949176 20250516231949176_0_0 11 746d057a-bf94-4ccd-8fc7-80e1bc15b945-0_0-967-1003_20250516231949176.parquet 11 {"person11":{"age":31,"full_name":"Karen","gender":"Female"}} {"country":"USA","city":"Seattle"} [{"item":"Coconut","quantity":1,"category":null},{"item":"Papaya","quantity":2,"category":null}] NULL +20250516231954989 20250516231954989_0_0 12 631edcd5-1e33-44ef-8ae1-5d0ba147cefe-0_0-979-1014_20250516231954989.parquet 12 {"person12":{"age":24,"full_name":"Leo","gender":"Male"}} {"country":"USA","city":"Portland"} [{"item":"Guava","quantity":3,"category":null},{"item":"Lychee","quantity":4,"category":null}] NULL +20250516232000323 20250516232000323_0_0 13 0d676984-e541-48db-afdf-6a265f80f4ca-0_0-991-1025_20250516232000323.parquet 13 {"person13":{"age":33,"full_name":"Mona","gender":"Female"}} {"country":"USA","city":"Denver"} [{"item":"Avocado","quantity":2,"category":"Fruit"},{"item":"Tomato","quantity":5,"category":"Vegetable"}] NULL +20250516232005908 20250516232005908_0_0 14 64daf9fd-b106-413f-bc35-2d4791b2cb40-0_0-1003-1036_20250516232005908.parquet 14 {"person14":{"age":28,"full_name":"Nina","gender":"Female"}} {"country":"USA","city":"Miami"} [{"item":"Cucumber","quantity":6,"category":"Vegetable"},{"item":"Carrot","quantity":7,"category":"Vegetable"}] NULL +20250516232011850 20250516232011850_0_0 15 66e015e2-36d7-485b-bbba-e649a0b61276-0_0-1017-1080_20250516232011850.parquet 15 {"person15":{"age":30,"full_name":"Emma Smith","gender":"Female"}} {"country":"USA","city":"New York"} [{"item":"Banana","quantity":3,"category":"Fruit"},{"item":"Potato","quantity":8,"category":"Vegetable"}] NULL +20250516232016796 20250516232016796_0_0 16 a86f2b41-cf45-40d8-be0a-0c2bf5d657a7-0_0-1029-1091_20250516232016796.parquet 16 {"person16":{"age":28,"full_name":"Liam Brown","gender":"Male"}} {"country":"UK","city":"London"} [{"item":"Bread","quantity":2,"category":"Food"},{"item":"Milk","quantity":1,"category":"Dairy"}] {"b":{"cc":"NestedCC","new_dd":75},"new_a":{"new_aa":50,"bb":"NestedBB"},"c":9} +20250516232022163 20250516232022163_0_0 17 ad21e571-c277-42d6-ad5c-b5801cb81301-0_0-1041-1102_20250516232022163.parquet 17 {"person17":{"age":40,"full_name":"Olivia Davis","gender":"Female"}} {"country":"Australia","city":"Sydney"} [{"item":"Orange","quantity":4,"category":"Fruit"},{"item":"Broccoli","quantity":6,"category":"Vegetable"}] {"b":{"cc":"UpdatedCC","new_dd":88},"new_a":{"new_aa":60,"bb":"UpdatedBB"},"c":12} +20250516232027268 20250516232027268_0_0 18 74b620c3-2fef-40de-bad5-db7b36fc2ee4-0_0-1053-1113_20250516232027268.parquet 18 {"person18":{"age":33,"full_name":"Noah Wilson","gender":"Male"}} {"country":"Germany","city":"Berlin"} [{"item":"Cheese","quantity":2,"category":"Dairy"},{"item":"Lettuce","quantity":5,"category":"Vegetable"}] {"b":{"cc":"NestedCC18","new_dd":95},"new_a":{"new_aa":70,"bb":"NestedBB18"},"c":15} +20250516232032532 20250516232032532_0_0 19 f14b6c61-71e2-412e-9709-5cd5d524657e-0_0-1065-1124_20250516232032532.parquet 19 {"person19":{"age":29,"full_name":"Ava Martinez","gender":"Female"}} {"country":"France","city":"Paris"} [{"item":"Strawberry","quantity":12,"category":"Fruit"},{"item":"Spinach","quantity":7,"category":"Vegetable"}] {"b":{"cc":"ReorderedCC","new_dd":101},"new_a":{"new_aa":85,"bb":"ReorderedBB"},"c":18} +20250516232037285 20250516232037285_0_0 20 27b13aa1-b2ad-4e7e-941c-b7d6c404c1bf-0_0-1077-1135_20250516232037285.parquet 20 {"person20":{"age":38,"full_name":"James Lee","gender":"Male"}} {"country":"Japan","city":"Osaka"} [{"item":"Mango","quantity":6,"category":"Fruit"},{"item":"Onion","quantity":3,"category":"Vegetable"}] {"b":{"cc":"FinalCC","new_dd":110},"new_a":{"new_aa":95,"bb":"FinalBB"},"c":21} +20250516232043525 20250516232043525_0_0 21 faeb9136-4d2e-4ce7-a6fd-4d648ea3c12d-0_0-1091-1179_20250516232043525.parquet 21 {"person21":{"age":45,"full_name":"Sophia White","gender":"Female"}} {"country":"Italy","city":"Rome"} [{"item":"Pasta","quantity":4,"category":"Food"},{"item":"Olive","quantity":9,"category":"Food"}] {"b":{"cc":"ExampleCC","new_dd":120},"new_a":{"new_aa":100,"bb":"ExampleBB"},"c":25} +Time taken: 1.033 seconds, Fetched 22 row(s) +*/ From 692111064f43bdfeeb5f09f3f1dddfdfc8f211c9 Mon Sep 17 00:00:00 2001 From: daidai Date: Wed, 9 Jul 2025 22:31:44 +0800 Subject: [PATCH 2/4] [fix](hudi)add hudiOrcReader for read hudi table & orc format. (#52964) Related PR: #51341 Problem Summary: In pr #51341, hudiOrcReader was deleted, and this pr reintroduced it to read hudi orc table. Although I encountered this error when testing spark-hudi to read orc, the orc file was indeed generated by spark-hudi. ``` java.lang.UnsupportedOperationException: Base file format is not currently supported (ORC) at org.apache.hudi.HoodieBaseRelation.createBaseFileReader(HoodieBaseRelation.scala:574) ~[hudi-spark3.4-bundle_2.12-0.14.0-1.jar:0.14.0-1] at org.apache.hudi.BaseFileOnlyRelation.composeRDD(BaseFileOnlyRelation.scala:96) ~[hudi-spark3.4-bundle_2.12-0.14.0-1.jar:0.14.0-1] at org.apache.hudi.HoodieBaseRelation.buildScan(HoodieBaseRelation.scala:381) ~[hudi-spark3.4-bundle_2.12-0.14.0-1.jar:0.14.0-1] at org.apache.spark.sql.execution.datasources.DataSourceStrategy$.$anonfun$apply$4(DataSourceStrategy.scala:329) ~[spark-sql_2.12-3.4.2.jar:0.14.0-1] ``` --- be/src/vec/exec/format/table/hudi_reader.h | 32 ++ be/src/vec/exec/scan/vfile_scanner.cpp | 20 +- .../vec/exec/vfile_scanner_exception_test.cpp | 2 +- .../hudi/test_hudi_full_schema_change.out | 323 ++++++++++++++++++ .../hudi/test_hudi_full_schema_change.groovy | 2 +- 5 files changed, 374 insertions(+), 5 deletions(-) diff --git a/be/src/vec/exec/format/table/hudi_reader.h b/be/src/vec/exec/format/table/hudi_reader.h index add7d094ae246b..751094018c942a 100644 --- a/be/src/vec/exec/format/table/hudi_reader.h +++ b/be/src/vec/exec/format/table/hudi_reader.h @@ -57,5 +57,37 @@ class HudiParquetReader final : public HudiReader { const VExprContextSPtrs* not_single_slot_filter_conjuncts, const std::unordered_map* slot_id_to_filter_conjuncts); }; + +class HudiOrcReader final : public HudiReader { +public: + ENABLE_FACTORY_CREATOR(HudiOrcReader); + HudiOrcReader(std::unique_ptr file_format_reader, RuntimeProfile* profile, + RuntimeState* state, const TFileScanRangeParams& params, + const TFileRangeDesc& range, io::IOContext* io_ctx) + : HudiReader(std::move(file_format_reader), profile, state, params, range, io_ctx) {}; + ~HudiOrcReader() final = default; + + Status init_reader( + const std::vector& read_table_col_names, + const std::unordered_map* + table_col_name_to_value_range, + const VExprContextSPtrs& conjuncts, const TupleDescriptor* tuple_descriptor, + const RowDescriptor* row_descriptor, + const VExprContextSPtrs* not_single_slot_filter_conjuncts, + const std::unordered_map* slot_id_to_filter_conjuncts) { + auto* orc_reader = static_cast(_file_format_reader.get()); + const orc::Type* orc_type_ptr = nullptr; + RETURN_IF_ERROR(orc_reader->get_file_type(&orc_type_ptr)); + RETURN_IF_ERROR(gen_table_info_node_by_field_id( + _params, _range.table_format_params.hudi_params.schema_id, tuple_descriptor, + orc_type_ptr)); + + return orc_reader->init_reader(&read_table_col_names, table_col_name_to_value_range, + conjuncts, false, tuple_descriptor, row_descriptor, + not_single_slot_filter_conjuncts, + slot_id_to_filter_conjuncts, table_info_node_ptr); + } +}; + #include "common/compile_check_end.h" } // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/exec/scan/vfile_scanner.cpp b/be/src/vec/exec/scan/vfile_scanner.cpp index b4a90e4ce038b5..0cba6b8a0c28be 100644 --- a/be/src/vec/exec/scan/vfile_scanner.cpp +++ b/be/src/vec/exec/scan/vfile_scanner.cpp @@ -1114,6 +1114,16 @@ Status VFileScanner::_get_next_reader() { &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts); RETURN_IF_ERROR(paimon_reader->init_row_filters()); _cur_reader = std::move(paimon_reader); + } else if (range.__isset.table_format_params && + range.table_format_params.table_format_type == "hudi") { + std::unique_ptr hudi_reader = HudiOrcReader::create_unique( + std::move(orc_reader), _profile, _state, *_params, range, _io_ctx.get()); + + init_status = hudi_reader->init_reader( + _file_col_names, _colname_to_value_range, _push_down_conjuncts, + _real_tuple_desc, _default_val_row_desc.get(), + &_not_single_slot_filter_conjuncts, &_slot_id_to_filter_conjuncts); + _cur_reader = std::move(hudi_reader); } else if (range.__isset.table_format_params && range.table_format_params.table_format_type == "hive") { std::unique_ptr hive_reader = HiveOrcReader::create_unique( @@ -1220,12 +1230,16 @@ Status VFileScanner::_get_next_reader() { break; } default: - return Status::InternalError("Not supported file format: {}", _params->format_type); + return Status::NotSupported("Not supported create reader for file format: {}.", + to_string(_params->format_type)); } if (_cur_reader == nullptr) { - return Status::InternalError("Failed to create reader for file format: {}", - _params->format_type); + return Status::NotSupported( + "Not supported create reader for table format: {} / file format: {}.", + range.__isset.table_format_params ? range.table_format_params.table_format_type + : "NotSet", + to_string(_params->format_type)); } COUNTER_UPDATE(_file_counter, 1); // The VFileScanner for external table may try to open not exist files, diff --git a/be/test/vec/exec/vfile_scanner_exception_test.cpp b/be/test/vec/exec/vfile_scanner_exception_test.cpp index 4b6ce46bd88cf3..c593a53025d467 100644 --- a/be/test/vec/exec/vfile_scanner_exception_test.cpp +++ b/be/test/vec/exec/vfile_scanner_exception_test.cpp @@ -298,7 +298,7 @@ TEST_F(VfileScannerExceptionTest, failure_case) { auto st = scanner->get_block(&_runtime_state, block.get(), &eof); ASSERT_FALSE(st.ok()); auto msg = st.to_string(); - auto pos = msg.find("Failed to create reader for"); + auto pos = msg.find("Not supported create reader"); std::cout << "msg = " << msg << std::endl; ASSERT_TRUE(pos != msg.npos); WARN_IF_ERROR(scanner->close(&_runtime_state), "fail to close scanner"); diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_full_schema_change.out b/regression-test/data/external_table_p2/hudi/test_hudi_full_schema_change.out index 8a1e62f8249b9a..e6f6b83663508e 100644 --- a/regression-test/data/external_table_p2/hudi/test_hudi_full_schema_change.out +++ b/regression-test/data/external_table_p2/hudi/test_hudi_full_schema_change.out @@ -323,3 +323,326 @@ 19 Paris 29 20 Osaka 38 +-- !all -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !country_usa -- +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !country_usa_cols -- +5 Phoenix Eve 2 +6 Philadelphia Frank 2 +7 San Antonio Grace 2 +8 San Diego Hank 2 +9 Dallas Ivy 2 +10 Austin Jack 2 +11 Seattle Karen 2 +12 Portland Leo 2 +13 Denver Mona 2 +14 Miami Nina 2 +15 New York Emma Smith 2 + +-- !city_new -- +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !city_new_cols -- +1 \N 25 Apple +15 USA 30 Banana + +-- !age_over_30 -- +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !age_over_30_cols -- +4 Houston \N +5 Phoenix \N +8 San Diego \N +11 Seattle \N +13 Denver Vegetable +17 Sydney Vegetable +18 Berlin Vegetable +20 Osaka Vegetable +21 Rome Food + +-- !age_under_25 -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N + +-- !age_under_25_cols -- +0 \N person0 +6 USA person6 +12 USA person12 + +-- !name_alice -- +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N + +-- !name_alice_cols -- +1 New York 2 + +-- !name_j -- +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} + +-- !name_j_cols -- +10 USA \N +20 Japan Male + +-- !map_person5 -- +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N + +-- !map_person5_cols -- +5 Phoenix 40 + +-- !array_size_2 -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !array_size_2_cols -- +0 \N \N +1 \N \N +2 \N \N +3 \N \N +4 \N \N +5 USA \N +6 USA \N +7 USA \N +8 USA \N +9 USA \N +10 USA \N +11 USA \N +12 USA \N +13 USA \N +14 USA \N +15 USA \N +16 UK NestedCC +17 Australia UpdatedCC +18 Germany NestedCC18 +19 France ReorderedCC +20 Japan FinalCC +21 Italy ExampleCC + +-- !quantity_not_null -- +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !quantity_not_null_cols -- +8 San Diego Hank +9 Dallas Ivy +10 Austin Jack +11 Seattle Karen +12 Portland Leo +13 Denver Mona +14 Miami Nina +15 New York Emma Smith +16 London Liam Brown +17 Sydney Olivia Davis +18 Berlin Noah Wilson +19 Paris Ava Martinez +20 Osaka James Lee +21 Rome Sophia White + +-- !quantity_null -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N + +-- !quantity_null_cols -- +0 \N 2 +1 \N 2 +2 \N 2 +3 \N 2 +4 \N 2 +5 USA 2 +6 USA 2 +7 USA 2 + +-- !struct2_not_null -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !struct2_not_null_cols -- +16 9 50 +17 12 60 +18 15 70 +19 18 85 +20 21 95 +21 25 100 + +-- !struct2_null -- +0 {"person0":{"age":2, "full_name":"zero", "gender":null}} {"country":null, "city":"cn"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +1 {"person1":{"age":25, "full_name":"Alice", "gender":null}} {"country":null, "city":"New York"} [{"item":"Apple", "quantity":null, "category":null}, {"item":"Banana", "quantity":null, "category":null}] \N +2 {"person2":{"age":30, "full_name":"Bob", "gender":null}} {"country":null, "city":"Los Angeles"} [{"item":"Orange", "quantity":null, "category":null}, {"item":"Grape", "quantity":null, "category":null}] \N +3 {"person3":{"age":28, "full_name":"Charlie", "gender":null}} {"country":null, "city":"Chicago"} [{"item":"Pear", "quantity":null, "category":null}, {"item":"Mango", "quantity":null, "category":null}] \N +4 {"person4":{"age":35, "full_name":"David", "gender":null}} {"country":null, "city":"Houston"} [{"item":"Kiwi", "quantity":null, "category":null}, {"item":"Pineapple", "quantity":null, "category":null}] \N +5 {"person5":{"age":40, "full_name":"Eve", "gender":null}} {"country":"USA", "city":"Phoenix"} [{"item":"Lemon", "quantity":null, "category":null}, {"item":"Lime", "quantity":null, "category":null}] \N +6 {"person6":{"age":22, "full_name":"Frank", "gender":null}} {"country":"USA", "city":"Philadelphia"} [{"item":"Watermelon", "quantity":null, "category":null}, {"item":"Strawberry", "quantity":null, "category":null}] \N +7 {"person7":{"age":27, "full_name":"Grace", "gender":null}} {"country":"USA", "city":"San Antonio"} [{"item":"Blueberry", "quantity":null, "category":null}, {"item":"Raspberry", "quantity":null, "category":null}] \N +8 {"person8":{"age":32, "full_name":"Hank", "gender":null}} {"country":"USA", "city":"San Diego"} [{"item":"Cherry", "quantity":5, "category":null}, {"item":"Plum", "quantity":3, "category":null}] \N +9 {"person9":{"age":29, "full_name":"Ivy", "gender":null}} {"country":"USA", "city":"Dallas"} [{"item":"Peach", "quantity":4, "category":null}, {"item":"Apricot", "quantity":2, "category":null}] \N +10 {"person10":{"age":26, "full_name":"Jack", "gender":null}} {"country":"USA", "city":"Austin"} [{"item":"Fig", "quantity":6, "category":null}, {"item":"Date", "quantity":7, "category":null}] \N +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +12 {"person12":{"age":24, "full_name":"Leo", "gender":"Male"}} {"country":"USA", "city":"Portland"} [{"item":"Guava", "quantity":3, "category":null}, {"item":"Lychee", "quantity":4, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N + +-- !struct2_null_cols -- +0 cn +1 New York +2 Los Angeles +3 Chicago +4 Houston +5 Phoenix +6 Philadelphia +7 San Antonio +8 San Diego +9 Dallas +10 Austin +11 Seattle +12 Portland +13 Denver +14 Miami +15 New York + +-- !cc_nested -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} + +-- !cc_nested_cols -- +16 9 +18 15 + +-- !c_over_20 -- +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !c_over_20_cols -- +20 FinalCC +21 ExampleCC + +-- !new_aa_50 -- +16 {"person16":{"age":28, "full_name":"Liam Brown", "gender":"Male"}} {"country":"UK", "city":"London"} [{"item":"Bread", "quantity":2, "category":"Food"}, {"item":"Milk", "quantity":1, "category":"Dairy"}] {"b":{"cc":"NestedCC", "new_dd":75}, "new_a":{"new_aa":50, "bb":"NestedBB"}, "c":9} + +-- !new_aa_50_cols -- +16 9 + +-- !gender_female -- +11 {"person11":{"age":31, "full_name":"Karen", "gender":"Female"}} {"country":"USA", "city":"Seattle"} [{"item":"Coconut", "quantity":1, "category":null}, {"item":"Papaya", "quantity":2, "category":null}] \N +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +21 {"person21":{"age":45, "full_name":"Sophia White", "gender":"Female"}} {"country":"Italy", "city":"Rome"} [{"item":"Pasta", "quantity":4, "category":"Food"}, {"item":"Olive", "quantity":9, "category":"Food"}] {"b":{"cc":"ExampleCC", "new_dd":120}, "new_a":{"new_aa":100, "bb":"ExampleBB"}, "c":25} + +-- !gender_female_cols -- +11 Seattle 2 +13 Denver 2 +14 Miami 2 +15 New York 2 +17 Sydney 2 +19 Paris 2 +21 Rome 2 + +-- !category_fruit -- + +-- !category_fruit_cols -- + +-- !category_vegetable -- +13 {"person13":{"age":33, "full_name":"Mona", "gender":"Female"}} {"country":"USA", "city":"Denver"} [{"item":"Avocado", "quantity":2, "category":"Fruit"}, {"item":"Tomato", "quantity":5, "category":"Vegetable"}] \N +14 {"person14":{"age":28, "full_name":"Nina", "gender":"Female"}} {"country":"USA", "city":"Miami"} [{"item":"Cucumber", "quantity":6, "category":"Vegetable"}, {"item":"Carrot", "quantity":7, "category":"Vegetable"}] \N +15 {"person15":{"age":30, "full_name":"Emma Smith", "gender":"Female"}} {"country":"USA", "city":"New York"} [{"item":"Banana", "quantity":3, "category":"Fruit"}, {"item":"Potato", "quantity":8, "category":"Vegetable"}] \N +17 {"person17":{"age":40, "full_name":"Olivia Davis", "gender":"Female"}} {"country":"Australia", "city":"Sydney"} [{"item":"Orange", "quantity":4, "category":"Fruit"}, {"item":"Broccoli", "quantity":6, "category":"Vegetable"}] {"b":{"cc":"UpdatedCC", "new_dd":88}, "new_a":{"new_aa":60, "bb":"UpdatedBB"}, "c":12} +18 {"person18":{"age":33, "full_name":"Noah Wilson", "gender":"Male"}} {"country":"Germany", "city":"Berlin"} [{"item":"Cheese", "quantity":2, "category":"Dairy"}, {"item":"Lettuce", "quantity":5, "category":"Vegetable"}] {"b":{"cc":"NestedCC18", "new_dd":95}, "new_a":{"new_aa":70, "bb":"NestedBB18"}, "c":15} +19 {"person19":{"age":29, "full_name":"Ava Martinez", "gender":"Female"}} {"country":"France", "city":"Paris"} [{"item":"Strawberry", "quantity":12, "category":"Fruit"}, {"item":"Spinach", "quantity":7, "category":"Vegetable"}] {"b":{"cc":"ReorderedCC", "new_dd":101}, "new_a":{"new_aa":85, "bb":"ReorderedBB"}, "c":18} +20 {"person20":{"age":38, "full_name":"James Lee", "gender":"Male"}} {"country":"Japan", "city":"Osaka"} [{"item":"Mango", "quantity":6, "category":"Fruit"}, {"item":"Onion", "quantity":3, "category":"Vegetable"}] {"b":{"cc":"FinalCC", "new_dd":110}, "new_a":{"new_aa":95, "bb":"FinalBB"}, "c":21} + +-- !category_vegetable_cols -- +13 Denver 33 +14 Miami 28 +15 New York 30 +17 Sydney 40 +18 Berlin 33 +19 Paris 29 +20 Osaka 38 diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_full_schema_change.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_full_schema_change.groovy index 1b427faf53b5af..173e6f646570e3 100644 --- a/regression-test/suites/external_table_p2/hudi/test_hudi_full_schema_change.groovy +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_full_schema_change.groovy @@ -38,7 +38,7 @@ suite("test_hudi_full_schema_change", "p2,external,hudi,external_remote,external - def tables = ["hudi_full_schema_change_parquet"] + def tables = ["hudi_full_schema_change_parquet","hudi_full_schema_change_orc"] for (String table: tables) { From 8dd41da49cc3fbf6518f9a992d2671cf382c85ae Mon Sep 17 00:00:00 2001 From: daidai Date: Thu, 10 Jul 2025 23:54:51 +0800 Subject: [PATCH 3/4] [echcement](hive)support read hive table that change timestamp column to bigint. (#52954) Related PR: #47471 Problem Summary: This pr is a supplement to #47471. This pr is used to support reading hive tables that convert timestamp columns to bigint columns and display them in `ms` precision. (parquet/orc hive table.) --- .../vec/exec/format/column_type_convert.cpp | 22 ++ be/src/vec/exec/format/column_type_convert.h | 57 ++++ .../format/parquet/parquet_column_convert.cpp | 5 +- be/test/vec/exec/column_type_convert_test.cpp | 304 ++++++++++++------ .../hive/test_hive_schema_change_orc.out | 112 +++++++ .../hive/test_hive_schema_change_parquet.out | 70 ++++ 6 files changed, 476 insertions(+), 94 deletions(-) diff --git a/be/src/vec/exec/format/column_type_convert.cpp b/be/src/vec/exec/format/column_type_convert.cpp index eaed1d6ff0eddc..b4b9f7077528f1 100644 --- a/be/src/vec/exec/format/column_type_convert.cpp +++ b/be/src/vec/exec/format/column_type_convert.cpp @@ -279,6 +279,23 @@ static std::unique_ptr _numeric_to_decimal_converter( } } +static std::unique_ptr _datetime_to_numeric_converter( + const DataTypePtr& src_type, const DataTypePtr& dst_type) { + PrimitiveType dst_primitive_type = dst_type->get_primitive_type(); + + switch (dst_primitive_type) { +#define DISPATCH(DST_TYPE) \ + case DST_TYPE: { \ + return std::make_unique>(); \ + } + FOR_LOGICAL_INTEGER_TYPES(DISPATCH) +#undef DISPATCH + default: { + return std::make_unique(src_type, dst_type); + } + }; +} + static std::unique_ptr _decimal_to_numeric_converter( const TypeDescriptor& src_type, const DataTypePtr& dst_type) { PrimitiveType src_primitive_type = src_type.type; @@ -403,6 +420,11 @@ std::unique_ptr ColumnTypeConverter::get_converter( return std::make_unique>(); } + // datetime to bigint (ms) + if (src_primitive_type == TYPE_DATETIMEV2 && _is_numeric_type(dst_primitive_type)) { + return _datetime_to_numeric_converter(src_type, dst_type); + } + // numeric to decimal if (_is_numeric_type(src_primitive_type) && _is_decimal_type(dst_primitive_type)) { return _numeric_to_decimal_converter(src_type, dst_type); diff --git a/be/src/vec/exec/format/column_type_convert.h b/be/src/vec/exec/format/column_type_convert.h index 75f4693a573960..838e2707c3c6e4 100644 --- a/be/src/vec/exec/format/column_type_convert.h +++ b/be/src/vec/exec/format/column_type_convert.h @@ -17,6 +17,7 @@ #pragma once +#include #include #include @@ -594,6 +595,62 @@ class CastStringConverter : public ColumnTypeConverter { } }; +template +class DateTimeToNumericConverter : public ColumnTypeConverter { +public: + Status convert(ColumnPtr& src_col, MutableColumnPtr& dst_col) override { + using SrcColumnType = typename PrimitiveTypeTraits::ColumnType; + using DstColumnType = typename PrimitiveTypeTraits::ColumnType; + using SrcCppType = typename PrimitiveTypeTraits::CppType; + using DstCppType = typename PrimitiveTypeTraits::CppType; + + ColumnPtr from_col = remove_nullable(src_col); + MutableColumnPtr to_col = remove_nullable(dst_col->get_ptr())->assume_mutable(); + + NullMap* null_map = nullptr; + if (dst_col->is_nullable()) { + null_map = &reinterpret_cast(dst_col.get()) + ->get_null_map_data(); + } + + size_t rows = from_col->size(); + auto& src_data = static_cast(from_col.get())->get_data(); + size_t start_idx = to_col->size(); + to_col->resize(start_idx + rows); + auto& data = static_cast(*to_col.get()).get_data(); + + for (int i = 0; i < rows; ++i) { + const SrcCppType& src_value = src_data[i]; + auto& dst_value = reinterpret_cast(data[start_idx + i]); + + int64_t ts_s = 0; + if (!src_value.unix_timestamp(&ts_s, cctz::utc_time_zone())) { + if (null_map == nullptr) { + return Status::InternalError("Failed to cast value '{}' to {} column", + src_data[i], dst_col->get_name()); + } else { + (*null_map)[start_idx + i] = 1; + } + } + auto micro = src_value.microsecond(); + int64_t ts_ms = ts_s * 1000 + micro / 1000; + if constexpr (DstPrimitiveType != TYPE_LARGEINT && DstPrimitiveType != TYPE_BIGINT) { + if ((Int64)std::numeric_limits::min() > ts_ms || + ts_ms > (Int64)std::numeric_limits::max()) { + if (null_map == nullptr) { + return Status::InternalError("Failed to cast value '{}' to {} column", + src_data[i], dst_col->get_name()); + } else { + (*null_map)[start_idx + i] = 1; + } + } + } + dst_value = static_cast(ts_ms); + } + return Status::OK(); + } +}; + // only support date & datetime v2 template class TimeV2Converter : public ColumnTypeConverter { diff --git a/be/src/vec/exec/format/parquet/parquet_column_convert.cpp b/be/src/vec/exec/format/parquet/parquet_column_convert.cpp index 0d24a14612a3d7..226cf1be311836 100644 --- a/be/src/vec/exec/format/parquet/parquet_column_convert.cpp +++ b/be/src/vec/exec/format/parquet/parquet_column_convert.cpp @@ -285,9 +285,8 @@ std::unique_ptr PhysicalToLogicalConverter::get_conv convert_params->reset_time_scale_if_missing(9); physical_converter.reset(new Int96toTimestamp()); } else if (src_physical_type == tparquet::Type::INT64) { - convert_params->reset_time_scale_if_missing( - remove_nullable(dst_logical_type)->get_scale()); - physical_converter.reset(new Int64ToTimestamp()); + convert_params->reset_time_scale_if_missing(src_logical_type.scale); + physical_converter = std::make_unique(); } else { physical_converter.reset(new UnsupportedConverter(src_physical_type, src_logical_type)); } diff --git a/be/test/vec/exec/column_type_convert_test.cpp b/be/test/vec/exec/column_type_convert_test.cpp index 3a45befd028530..decc572e3e3c8d 100644 --- a/be/test/vec/exec/column_type_convert_test.cpp +++ b/be/test/vec/exec/column_type_convert_test.cpp @@ -912,102 +912,56 @@ TEST_F(ColumnTypeConverterTest, TestUnsupportedConversions) { { std::vector> unsupported_conversions = { - {TYPE_BOOLEAN, TYPE_TINYINT}, - {TYPE_BOOLEAN, TYPE_SMALLINT}, - {TYPE_BOOLEAN, TYPE_INT}, - {TYPE_BOOLEAN, TYPE_BIGINT}, - {TYPE_BOOLEAN, TYPE_FLOAT}, - {TYPE_BOOLEAN, TYPE_DOUBLE}, - {TYPE_BOOLEAN, TYPE_DATE}, - {TYPE_BOOLEAN, TYPE_DATEV2}, - {TYPE_BOOLEAN, TYPE_TIMEV2}, - {TYPE_BOOLEAN, TYPE_DATETIME}, + {TYPE_BOOLEAN, TYPE_TINYINT}, {TYPE_BOOLEAN, TYPE_SMALLINT}, + {TYPE_BOOLEAN, TYPE_INT}, {TYPE_BOOLEAN, TYPE_BIGINT}, + {TYPE_BOOLEAN, TYPE_FLOAT}, {TYPE_BOOLEAN, TYPE_DOUBLE}, + {TYPE_BOOLEAN, TYPE_DATE}, {TYPE_BOOLEAN, TYPE_DATEV2}, + {TYPE_BOOLEAN, TYPE_TIMEV2}, {TYPE_BOOLEAN, TYPE_DATETIME}, {TYPE_BOOLEAN, TYPE_DATETIMEV2}, - {TYPE_TINYINT, TYPE_BOOLEAN}, - {TYPE_SMALLINT, TYPE_BOOLEAN}, - {TYPE_INT, TYPE_BOOLEAN}, - {TYPE_BIGINT, TYPE_BOOLEAN}, - - {TYPE_TINYINT, TYPE_DATE}, - {TYPE_SMALLINT, TYPE_DATE}, - {TYPE_INT, TYPE_DATE}, - {TYPE_BIGINT, TYPE_DATE}, - {TYPE_TINYINT, TYPE_DATEV2}, - {TYPE_SMALLINT, TYPE_DATEV2}, - {TYPE_INT, TYPE_DATEV2}, - {TYPE_BIGINT, TYPE_DATEV2}, - {TYPE_TINYINT, TYPE_DATETIME}, - {TYPE_SMALLINT, TYPE_DATETIME}, - {TYPE_INT, TYPE_DATETIME}, - {TYPE_BIGINT, TYPE_DATETIME}, - {TYPE_TINYINT, TYPE_DATETIMEV2}, - {TYPE_SMALLINT, TYPE_DATETIMEV2}, - {TYPE_INT, TYPE_DATETIMEV2}, - {TYPE_BIGINT, TYPE_DATETIMEV2}, - {TYPE_TINYINT, TYPE_TIMEV2}, - {TYPE_SMALLINT, TYPE_TIMEV2}, - {TYPE_INT, TYPE_TIMEV2}, - {TYPE_BIGINT, TYPE_TIMEV2}, - - {TYPE_FLOAT, TYPE_BOOLEAN}, - {TYPE_FLOAT, TYPE_INT}, - {TYPE_FLOAT, TYPE_SMALLINT}, - {TYPE_FLOAT, TYPE_TINYINT}, - {TYPE_FLOAT, TYPE_BIGINT}, - {TYPE_FLOAT, TYPE_DATE}, - {TYPE_FLOAT, TYPE_DATEV2}, - {TYPE_FLOAT, TYPE_TIMEV2}, - {TYPE_FLOAT, TYPE_DATETIME}, - {TYPE_FLOAT, TYPE_DATETIMEV2}, - - {TYPE_DOUBLE, TYPE_BOOLEAN}, - {TYPE_DOUBLE, TYPE_INT}, - {TYPE_DOUBLE, TYPE_SMALLINT}, - {TYPE_DOUBLE, TYPE_TINYINT}, - {TYPE_DOUBLE, TYPE_BIGINT}, - {TYPE_DOUBLE, TYPE_DATE}, - {TYPE_DOUBLE, TYPE_DATEV2}, - {TYPE_DOUBLE, TYPE_TIMEV2}, - {TYPE_DOUBLE, TYPE_DATETIME}, - {TYPE_DOUBLE, TYPE_DATETIMEV2}, + {TYPE_TINYINT, TYPE_BOOLEAN}, {TYPE_SMALLINT, TYPE_BOOLEAN}, + {TYPE_INT, TYPE_BOOLEAN}, {TYPE_BIGINT, TYPE_BOOLEAN}, + + {TYPE_TINYINT, TYPE_DATE}, {TYPE_SMALLINT, TYPE_DATE}, + {TYPE_INT, TYPE_DATE}, {TYPE_BIGINT, TYPE_DATE}, + {TYPE_TINYINT, TYPE_DATEV2}, {TYPE_SMALLINT, TYPE_DATEV2}, + {TYPE_INT, TYPE_DATEV2}, {TYPE_BIGINT, TYPE_DATEV2}, + {TYPE_TINYINT, TYPE_DATETIME}, {TYPE_SMALLINT, TYPE_DATETIME}, + {TYPE_INT, TYPE_DATETIME}, {TYPE_BIGINT, TYPE_DATETIME}, + {TYPE_TINYINT, TYPE_DATETIMEV2}, {TYPE_SMALLINT, TYPE_DATETIMEV2}, + {TYPE_INT, TYPE_DATETIMEV2}, {TYPE_BIGINT, TYPE_DATETIMEV2}, + {TYPE_TINYINT, TYPE_TIMEV2}, {TYPE_SMALLINT, TYPE_TIMEV2}, + {TYPE_INT, TYPE_TIMEV2}, {TYPE_BIGINT, TYPE_TIMEV2}, + + {TYPE_FLOAT, TYPE_BOOLEAN}, {TYPE_FLOAT, TYPE_INT}, + {TYPE_FLOAT, TYPE_SMALLINT}, {TYPE_FLOAT, TYPE_TINYINT}, + {TYPE_FLOAT, TYPE_BIGINT}, {TYPE_FLOAT, TYPE_DATE}, + {TYPE_FLOAT, TYPE_DATEV2}, {TYPE_FLOAT, TYPE_TIMEV2}, + {TYPE_FLOAT, TYPE_DATETIME}, {TYPE_FLOAT, TYPE_DATETIMEV2}, + + {TYPE_DOUBLE, TYPE_BOOLEAN}, {TYPE_DOUBLE, TYPE_INT}, + {TYPE_DOUBLE, TYPE_SMALLINT}, {TYPE_DOUBLE, TYPE_TINYINT}, + {TYPE_DOUBLE, TYPE_BIGINT}, {TYPE_DOUBLE, TYPE_DATE}, + {TYPE_DOUBLE, TYPE_DATEV2}, {TYPE_DOUBLE, TYPE_TIMEV2}, + {TYPE_DOUBLE, TYPE_DATETIME}, {TYPE_DOUBLE, TYPE_DATETIMEV2}, {TYPE_DOUBLE, TYPE_FLOAT}, - {TYPE_DATE, TYPE_BOOLEAN}, - {TYPE_DATE, TYPE_TINYINT}, - {TYPE_DATE, TYPE_SMALLINT}, - {TYPE_DATE, TYPE_INT}, - {TYPE_DATE, TYPE_BIGINT}, - {TYPE_DATE, TYPE_FLOAT}, - {TYPE_DATE, TYPE_DOUBLE}, - {TYPE_DATEV2, TYPE_BOOLEAN}, - {TYPE_DATEV2, TYPE_TINYINT}, - {TYPE_DATEV2, TYPE_SMALLINT}, - {TYPE_DATEV2, TYPE_INT}, - {TYPE_DATEV2, TYPE_BIGINT}, - {TYPE_DATEV2, TYPE_FLOAT}, - {TYPE_DATEV2, TYPE_DOUBLE}, - {TYPE_TIMEV2, TYPE_BOOLEAN}, - {TYPE_TIMEV2, TYPE_TINYINT}, - {TYPE_TIMEV2, TYPE_SMALLINT}, - {TYPE_TIMEV2, TYPE_INT}, - {TYPE_TIMEV2, TYPE_BIGINT}, - {TYPE_TIMEV2, TYPE_FLOAT}, - {TYPE_TIMEV2, TYPE_DOUBLE}, - {TYPE_DATETIME, TYPE_BOOLEAN}, - {TYPE_DATETIME, TYPE_TINYINT}, - {TYPE_DATETIME, TYPE_SMALLINT}, - {TYPE_DATETIME, TYPE_INT}, - {TYPE_DATETIME, TYPE_BIGINT}, - {TYPE_DATETIME, TYPE_FLOAT}, - {TYPE_DATETIME, TYPE_DOUBLE}, - {TYPE_DATETIMEV2, TYPE_BOOLEAN}, - {TYPE_DATETIMEV2, TYPE_TINYINT}, - {TYPE_DATETIMEV2, TYPE_SMALLINT}, - {TYPE_DATETIMEV2, TYPE_INT}, - {TYPE_DATETIMEV2, TYPE_BIGINT}, - {TYPE_DATETIMEV2, TYPE_FLOAT}, + {TYPE_DATE, TYPE_BOOLEAN}, {TYPE_DATE, TYPE_TINYINT}, + {TYPE_DATE, TYPE_SMALLINT}, {TYPE_DATE, TYPE_INT}, + {TYPE_DATE, TYPE_BIGINT}, {TYPE_DATE, TYPE_FLOAT}, + {TYPE_DATE, TYPE_DOUBLE}, {TYPE_DATEV2, TYPE_BOOLEAN}, + {TYPE_DATEV2, TYPE_TINYINT}, {TYPE_DATEV2, TYPE_SMALLINT}, + {TYPE_DATEV2, TYPE_INT}, {TYPE_DATEV2, TYPE_BIGINT}, + {TYPE_DATEV2, TYPE_FLOAT}, {TYPE_DATEV2, TYPE_DOUBLE}, + {TYPE_TIMEV2, TYPE_BOOLEAN}, {TYPE_TIMEV2, TYPE_TINYINT}, + {TYPE_TIMEV2, TYPE_SMALLINT}, {TYPE_TIMEV2, TYPE_INT}, + {TYPE_TIMEV2, TYPE_BIGINT}, {TYPE_TIMEV2, TYPE_FLOAT}, + {TYPE_TIMEV2, TYPE_DOUBLE}, {TYPE_DATETIME, TYPE_BOOLEAN}, + {TYPE_DATETIME, TYPE_TINYINT}, {TYPE_DATETIME, TYPE_SMALLINT}, + {TYPE_DATETIME, TYPE_INT}, {TYPE_DATETIME, TYPE_BIGINT}, + {TYPE_DATETIME, TYPE_FLOAT}, {TYPE_DATETIME, TYPE_DOUBLE}, + {TYPE_DATETIMEV2, TYPE_BOOLEAN}, {TYPE_DATETIMEV2, TYPE_FLOAT}, {TYPE_DATETIMEV2, TYPE_DOUBLE}, }; @@ -1089,6 +1043,174 @@ TEST_F(ColumnTypeConverterTest, TestUnsupportedConversions) { } } +TEST_F(ColumnTypeConverterTest, TestDateTimeV2ToNumericConversions) { + using namespace doris::vectorized; + + auto make_datetimev2_col = + [](const std::vector>& datetimes) { + auto col = ColumnDateTimeV2::create(); + for (const auto& [y, m, d, h, min, s, micro] : datetimes) { + DateV2Value v; + v.unchecked_set_time(y, m, d, h, min, s, micro); + col->get_data().push_back(*reinterpret_cast(&v)); + } + return col; + }; + + auto parse_datetimev2_str = [](const std::string& datetime_str) { + UInt64 x = 0; + ReadBuffer buf((char*)datetime_str.data(), datetime_str.size()); + bool ok = read_datetime_v2_text_impl(x, buf, 6); + CHECK(ok) << "parse_datetimev2_str failed for: " << datetime_str; + return x; + }; + + // 1. DATETIMEV2 -> BIGINT + { + auto src_type = + vectorized::DataTypeFactory::instance().create_data_type(TYPE_DATETIMEV2, false); + auto dst_type = std::make_shared(); + auto converter = converter::ColumnTypeConverter::get_converter(src_type, dst_type, + converter::COMMON); + + ASSERT_TRUE(converter->support()); + + // 2024-01-01 00:00:00.123456 + auto src_col = make_datetimev2_col({{2024, 1, 1, 0, 0, 0, 123456}}); + auto dst_col = dst_type->create_column(); + auto mutable_dst = dst_col->assume_mutable(); + + Status st = converter->convert(reinterpret_cast(src_col), mutable_dst); + ASSERT_TRUE(st.ok()); + + auto& dst_data = static_cast(*mutable_dst).get_data(); + ASSERT_EQ(1, dst_data.size()); + EXPECT_EQ(1704067200123, dst_data[0]); + } + + // 2. DATETIMEV2 -> INT + { + auto src_type = + vectorized::DataTypeFactory::instance().create_data_type(TYPE_DATETIMEV2, false); + auto dst_type = std::make_shared(); + auto nullable_dst_type = std::make_shared(dst_type); + auto converter = converter::ColumnTypeConverter::get_converter(src_type, nullable_dst_type, + converter::COMMON); + + ASSERT_TRUE(converter->support()); + + // 1970-01-01 00:00:00.000000 + // 3000-01-01 00:00:00.000000 + auto src_col = make_datetimev2_col({{1970, 1, 1, 0, 0, 0, 0}, {3000, 1, 1, 0, 0, 0, 0}}); + auto dst_col = nullable_dst_type->create_column(); + auto mutable_dst = dst_col->assume_mutable(); + auto& nullable_col = static_cast(*mutable_dst); + auto& null_map = nullable_col.get_null_map_data(); + null_map.resize_fill(src_col->size(), 0); + + Status st = converter->convert(reinterpret_cast(src_col), mutable_dst); + ASSERT_TRUE(st.ok()); + auto& nested_col = static_cast(nullable_col.get_nested_column()); + auto& dst_data = nested_col.get_data(); + + ASSERT_EQ(2, nested_col.size()); + EXPECT_EQ(0, null_map[0]); + ASSERT_EQ(0, dst_data[0]); + EXPECT_EQ(1, null_map[1]); + } + + // 3. DATETIMEV2 -> INT, non-nullable + { + auto src_type = + vectorized::DataTypeFactory::instance().create_data_type(TYPE_DATETIMEV2, false); + auto dst_type = std::make_shared(); + auto converter = converter::ColumnTypeConverter::get_converter(src_type, dst_type, + converter::COMMON); + + ASSERT_TRUE(converter->support()); + + // 3000-01-01 00:00:00.000000(会溢出int32) + auto src_col = make_datetimev2_col({{3000, 1, 1, 0, 0, 0, 0}}); + auto dst_col = dst_type->create_column(); + auto mutable_dst = dst_col->assume_mutable(); + + Status st = converter->convert(reinterpret_cast(src_col), mutable_dst); + ASSERT_FALSE(st.ok()); + } + + { + auto src_type = + vectorized::DataTypeFactory::instance().create_data_type(TYPE_DATETIMEV2, false); + auto dst_type = std::make_shared(); + auto nullable_dst_type = std::make_shared(dst_type); + auto converter = converter::ColumnTypeConverter::get_converter(src_type, nullable_dst_type, + converter::COMMON); + + ASSERT_TRUE(converter->support()); + + auto src_col = ColumnDateTimeV2::create(); + src_col->get_data().push_back(parse_datetimev2_str("2024-01-01 12:34:56.123456")); + src_col->get_data().push_back(parse_datetimev2_str("1970-01-01 00:00:00.000000")); + src_col->get_data().push_back(parse_datetimev2_str("3000-01-01 00:00:00.000000")); + src_col->get_data().push_back(parse_datetimev2_str("1900-01-01 00:00:00.000000")); + src_col->get_data().push_back(parse_datetimev2_str("1999-12-31 23:59:59.999999")); + src_col->get_data().push_back(parse_datetimev2_str("2000-01-01 00:00:00.000000")); + src_col->get_data().push_back(parse_datetimev2_str("2025-07-08 16:00:00.123456")); + src_col->get_data().push_back(parse_datetimev2_str("2100-01-01 00:00:00.000000")); + src_col->get_data().push_back(parse_datetimev2_str("9999-12-31 23:59:59.999999")); + src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 12:00:00.000001")); + src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 13:00:00.000002")); + src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 14:00:00.000004")); + src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 12:00:00")); + src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 13:00:00")); + src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 14:00:00")); + + auto dst_col = nullable_dst_type->create_column(); + auto mutable_dst = dst_col->assume_mutable(); + auto& nullable_col = static_cast(*mutable_dst); + auto& null_map = nullable_col.get_null_map_data(); + null_map.resize_fill(src_col->size(), 0); + + Status st = converter->convert(reinterpret_cast(src_col), mutable_dst); + ASSERT_TRUE(st.ok()); + + ASSERT_EQ(15, null_map.size()); + EXPECT_EQ(0, null_map[0]); + EXPECT_EQ(0, null_map[1]); + EXPECT_EQ(0, null_map[2]); + EXPECT_EQ(0, null_map[3]); + EXPECT_EQ(0, null_map[4]); + EXPECT_EQ(0, null_map[5]); + EXPECT_EQ(0, null_map[6]); + EXPECT_EQ(0, null_map[7]); + EXPECT_EQ(0, null_map[8]); + EXPECT_EQ(0, null_map[9]); + EXPECT_EQ(0, null_map[10]); + EXPECT_EQ(0, null_map[11]); + EXPECT_EQ(0, null_map[12]); + EXPECT_EQ(0, null_map[13]); + EXPECT_EQ(0, null_map[14]); + + auto& dst_data = static_cast(nullable_col.get_nested_column()).get_data(); + ASSERT_EQ(15, dst_data.size()); + EXPECT_EQ(1704112496123L, dst_data[0]); + EXPECT_EQ(0L, dst_data[1]); + EXPECT_EQ(32503680000000L, dst_data[2]); + EXPECT_EQ(-2208988800000L, dst_data[3]); + EXPECT_EQ(946684799999L, dst_data[4]); + EXPECT_EQ(946684800000L, dst_data[5]); + EXPECT_EQ(1751990400123, dst_data[6]); + EXPECT_EQ(4102444800000L, dst_data[7]); + EXPECT_EQ(253402300799999, dst_data[8]); + EXPECT_EQ(1651406400000, dst_data[9]); + EXPECT_EQ(1651410000000, dst_data[10]); + EXPECT_EQ(1651413600000, dst_data[11]); + EXPECT_EQ(1651406400000, dst_data[12]); + EXPECT_EQ(1651410000000, dst_data[13]); + EXPECT_EQ(1651413600000, dst_data[14]); + } +} + TEST_F(ColumnTypeConverterTest, TestEmptyColumnConversions) { // Test empty column { diff --git a/regression-test/data/external_table_p0/hive/test_hive_schema_change_orc.out b/regression-test/data/external_table_p0/hive/test_hive_schema_change_orc.out index 1028bdad18420a..9e819aded4f4fd 100644 --- a/regression-test/data/external_table_p0/hive/test_hive_schema_change_orc.out +++ b/regression-test/data/external_table_p0/hive/test_hive_schema_change_orc.out @@ -153,6 +153,20 @@ true \N \N +-- !orc_timestamp_to_smallint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + -- !orc_decimal1_to_smallint -- \N \N @@ -251,6 +265,20 @@ true \N \N +-- !orc_timestamp_to_tinyint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + -- !orc_decimal1_to_tinyint -- \N \N @@ -363,6 +391,20 @@ true \N \N +-- !orc_timestamp_to_smallint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + -- !orc_decimal1_to_smallint -- \N \N @@ -489,6 +531,20 @@ true \N \N +-- !orc_timestamp_to_int -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + -- !orc_decimal1_to_int -- 99999999 -99999999 @@ -629,6 +685,20 @@ true \N \N +-- !orc_timestamp_to_bigint -- +1697978096000 +1577836800000 +\N +\N +\N +1577836799000 +1653040800000 +\N +1672531200000 +1672531200000 +1672531200000 +1672531200000 + -- !orc_decimal1_to_bigint -- 99999999 -99999999 @@ -1091,6 +1161,48 @@ Sample2 true 1.2345 +-- !orc_char1_to_string -- +CHAR1CHAR1 +CHAR1CHAR1 + +4578 +12345667 +1 +2 +\N +-123 +1.23e-10 +false +1.24 + +-- !orc_char2_to_string -- +CHAR1CHAR1CHAR1CHAR1 +CHAR1CHAR1CHAR1CHAR1 + +58910.67 +345678.76543 +3 +3 +\N ++123 +1.23E+10 +TRUE +-1.24 + +-- !orc_varchar_to_string -- +VARCHAR123 +VARCHAR123 + +354111.345 +123.456 +1 +4 +\N +00123 +.123 +FALSE +-1.674 + -- !orc_date_to_string -- 2023-10-22 2020-01-01 diff --git a/regression-test/data/external_table_p0/hive/test_hive_schema_change_parquet.out b/regression-test/data/external_table_p0/hive/test_hive_schema_change_parquet.out index 8645792bfa837e..719e9bbddbfff0 100644 --- a/regression-test/data/external_table_p0/hive/test_hive_schema_change_parquet.out +++ b/regression-test/data/external_table_p0/hive/test_hive_schema_change_parquet.out @@ -153,6 +153,20 @@ true \N \N +-- !parquet_timestamp_to_smallint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + -- !parquet_decimal1_to_smallint -- \N \N @@ -251,6 +265,20 @@ true \N \N +-- !parquet_timestamp_to_tinyint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + -- !parquet_decimal1_to_tinyint -- \N \N @@ -363,6 +391,20 @@ true \N \N +-- !parquet_timestamp_to_smallint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + -- !parquet_decimal1_to_smallint -- \N \N @@ -489,6 +531,20 @@ true \N \N +-- !parquet_timestamp_to_int -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + -- !parquet_decimal1_to_int -- 99999999 -99999999 @@ -629,6 +685,20 @@ true \N \N +-- !parquet_timestamp_to_bigint -- +1697978096000 +1577836800000 +\N +\N +\N +1577836799000 +1653040800000 +\N +1672531200000 +1672531200000 +1672531200000 +1672531200000 + -- !parquet_decimal1_to_bigint -- 99999999 -99999999 From b6c5fdbd0de72a0b5eb5aedc259fe4c13e77e0a6 Mon Sep 17 00:00:00 2001 From: daidai Date: Thu, 10 Jul 2025 19:46:48 +0800 Subject: [PATCH 4/4] [fix](docker)Fix pipeline instability caused by upgrading docker paimon version. (#53055) ### What problem does this PR solve? Related PR: #51341 Problem Summary: In PR #51341, the Docker Paimon was upgraded from version 0.8 to 1.0.1. Since the required JAR files are pulled from a Maven repository, some machines may not be able to access the repository. To fix this, the JAR file has been uploaded to object storage, ensuring that it can be reliably accessed across different environments. --- be/src/vec/exec/format/column_type_convert.cpp | 5 +++-- be/src/vec/exec/format/column_type_convert.h | 1 + be/test/vec/exec/column_type_convert_test.cpp | 12 ++++-------- .../docker-compose/iceberg/entrypoint.sh.tpl | 2 +- .../docker-compose/iceberg/iceberg.yaml.tpl | 2 ++ docker/thirdparties/run-thirdparties-docker.sh | 8 ++++---- .../datasource/paimon/source/PaimonScanNode.java | 3 +-- 7 files changed, 16 insertions(+), 17 deletions(-) diff --git a/be/src/vec/exec/format/column_type_convert.cpp b/be/src/vec/exec/format/column_type_convert.cpp index b4b9f7077528f1..cdaa28d2ad6830 100644 --- a/be/src/vec/exec/format/column_type_convert.cpp +++ b/be/src/vec/exec/format/column_type_convert.cpp @@ -280,8 +280,9 @@ static std::unique_ptr _numeric_to_decimal_converter( } static std::unique_ptr _datetime_to_numeric_converter( - const DataTypePtr& src_type, const DataTypePtr& dst_type) { - PrimitiveType dst_primitive_type = dst_type->get_primitive_type(); + const TypeDescriptor& src_type, const DataTypePtr& dst_type) { + PrimitiveType dst_primitive_type = + remove_nullable(dst_type)->get_type_as_type_descriptor().type; switch (dst_primitive_type) { #define DISPATCH(DST_TYPE) \ diff --git a/be/src/vec/exec/format/column_type_convert.h b/be/src/vec/exec/format/column_type_convert.h index 838e2707c3c6e4..ca82e0ecb5c60f 100644 --- a/be/src/vec/exec/format/column_type_convert.h +++ b/be/src/vec/exec/format/column_type_convert.h @@ -18,6 +18,7 @@ #pragma once #include + #include #include diff --git a/be/test/vec/exec/column_type_convert_test.cpp b/be/test/vec/exec/column_type_convert_test.cpp index decc572e3e3c8d..bbc5a5b947c17c 100644 --- a/be/test/vec/exec/column_type_convert_test.cpp +++ b/be/test/vec/exec/column_type_convert_test.cpp @@ -1067,8 +1067,7 @@ TEST_F(ColumnTypeConverterTest, TestDateTimeV2ToNumericConversions) { // 1. DATETIMEV2 -> BIGINT { - auto src_type = - vectorized::DataTypeFactory::instance().create_data_type(TYPE_DATETIMEV2, false); + TypeDescriptor src_type(TYPE_DATETIMEV2); auto dst_type = std::make_shared(); auto converter = converter::ColumnTypeConverter::get_converter(src_type, dst_type, converter::COMMON); @@ -1090,8 +1089,7 @@ TEST_F(ColumnTypeConverterTest, TestDateTimeV2ToNumericConversions) { // 2. DATETIMEV2 -> INT { - auto src_type = - vectorized::DataTypeFactory::instance().create_data_type(TYPE_DATETIMEV2, false); + TypeDescriptor src_type(TYPE_DATETIMEV2); auto dst_type = std::make_shared(); auto nullable_dst_type = std::make_shared(dst_type); auto converter = converter::ColumnTypeConverter::get_converter(src_type, nullable_dst_type, @@ -1121,8 +1119,7 @@ TEST_F(ColumnTypeConverterTest, TestDateTimeV2ToNumericConversions) { // 3. DATETIMEV2 -> INT, non-nullable { - auto src_type = - vectorized::DataTypeFactory::instance().create_data_type(TYPE_DATETIMEV2, false); + TypeDescriptor src_type(TYPE_DATETIMEV2); auto dst_type = std::make_shared(); auto converter = converter::ColumnTypeConverter::get_converter(src_type, dst_type, converter::COMMON); @@ -1139,8 +1136,7 @@ TEST_F(ColumnTypeConverterTest, TestDateTimeV2ToNumericConversions) { } { - auto src_type = - vectorized::DataTypeFactory::instance().create_data_type(TYPE_DATETIMEV2, false); + TypeDescriptor src_type(TYPE_DATETIMEV2); auto dst_type = std::make_shared(); auto nullable_dst_type = std::make_shared(dst_type); auto converter = converter::ColumnTypeConverter::get_converter(src_type, nullable_dst_type, diff --git a/docker/thirdparties/docker-compose/iceberg/entrypoint.sh.tpl b/docker/thirdparties/docker-compose/iceberg/entrypoint.sh.tpl index 8c8471deb92c85..dd0dcae18c460a 100644 --- a/docker/thirdparties/docker-compose/iceberg/entrypoint.sh.tpl +++ b/docker/thirdparties/docker-compose/iceberg/entrypoint.sh.tpl @@ -45,7 +45,7 @@ echo "Script iceberg total: {} executed in $EXECUTION_TIME1 seconds" START_TIME2=$(date +%s) find /mnt/scripts/create_preinstalled_scripts/paimon -name '*.sql' | sed 's|^|source |' | sed 's|$|;|'> paimon_total.sql -spark-sql --packages org.apache.paimon:paimon-spark-3.5:1.0.1,org.apache.paimon:paimon-s3:1.0.1 --master spark://doris--spark-iceberg:7077 --conf spark.sql.extensions=org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions -f paimon_total.sql +spark-sql --master spark://doris--spark-iceberg:7077 --conf spark.sql.extensions=org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions -f paimon_total.sql END_TIME2=$(date +%s) EXECUTION_TIME2=$((END_TIME2 - START_TIME2)) echo "Script paimon total: {} executed in $EXECUTION_TIME2 seconds" diff --git a/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl b/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl index f4df47f3cd3ffa..73de5e9b05f094 100644 --- a/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl +++ b/docker/thirdparties/docker-compose/iceberg/iceberg.yaml.tpl @@ -35,6 +35,8 @@ services: - ./data:/mnt/data - ./scripts:/mnt/scripts - ./spark-defaults.conf:/opt/spark/conf/spark-defaults.conf + - ./data/input/jars/paimon-spark-3.5-1.0.1.jar:/opt/spark/jars/paimon-spark-3.5-1.0.1.jar + - ./data/input/jars/paimon-s3-1.0.1.jar:/opt/spark/jars/paimon-s3-1.0.1.jar environment: - AWS_ACCESS_KEY_ID=admin - AWS_SECRET_ACCESS_KEY=password diff --git a/docker/thirdparties/run-thirdparties-docker.sh b/docker/thirdparties/run-thirdparties-docker.sh index bebeb70306992b..0c7241ea1413e7 100755 --- a/docker/thirdparties/run-thirdparties-docker.sh +++ b/docker/thirdparties/run-thirdparties-docker.sh @@ -460,11 +460,11 @@ start_iceberg() { if [[ ! -d "${ICEBERG_DIR}/data" ]]; then echo "${ICEBERG_DIR}/data does not exist" cd "${ICEBERG_DIR}" \ - && rm -f iceberg_data.zip \ - && wget -P "${ROOT}"/docker-compose/iceberg https://"${s3BucketName}.${s3Endpoint}"/regression/datalake/pipeline_data/iceberg_data.zip \ - && sudo unzip iceberg_data.zip \ + && rm -f iceberg_data*.zip \ + && wget -P "${ROOT}"/docker-compose/iceberg https://"${s3BucketName}.${s3Endpoint}"/regression/datalake/pipeline_data/iceberg_data_paimon_101.zip \ + && sudo unzip iceberg_data_paimon_101.zip \ && sudo mv iceberg_data data \ - && sudo rm -rf iceberg_data.zip + && sudo rm -rf iceberg_data_paimon_101.zip cd - else echo "${ICEBERG_DIR}/data exist, continue !" diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java index 7481abfb24c9b9..87731534078f79 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java @@ -56,7 +56,6 @@ import org.apache.paimon.table.source.DeletionFile; import org.apache.paimon.table.source.RawFile; import org.apache.paimon.table.source.ReadBuilder; -import org.apache.paimon.types.DataField; import java.io.IOException; import java.util.ArrayList; @@ -150,7 +149,7 @@ public PaimonScanNode(PlanNodeId id, protected void doInitialize() throws UserException { super.doInitialize(); source = new PaimonSource(desc); - serializedTable = encodeObjectToString(source.getPaimonTable()); + serializedTable = PaimonUtil.encodeObjectToString(source.getPaimonTable()); // Todo: Get the current schema id of the table, instead of using -1. ExternalUtil.initSchemaInfo(params, -1L, source.getTargetTable().getColumns()); }