diff --git a/be/src/exec/scan/file_scanner.cpp b/be/src/exec/scan/file_scanner.cpp index a547ee4e7c80cb..e88d3048dfbd98 100644 --- a/be/src/exec/scan/file_scanner.cpp +++ b/be/src/exec/scan/file_scanner.cpp @@ -1245,6 +1245,17 @@ Status FileScanner::_init_parquet_reader(std::unique_ptr&& parque if (_need_iceberg_rowid_column) { iceberg_reader->set_need_row_id_column(true); } + if (_row_lineage_columns.row_id_column_idx != -1 || + _row_lineage_columns.last_updated_sequence_number_column_idx != -1) { + std::shared_ptr row_lineage_columns; + row_lineage_columns = std::make_shared(); + row_lineage_columns->row_id_column_idx = _row_lineage_columns.row_id_column_idx; + row_lineage_columns->last_updated_sequence_number_column_idx = + _row_lineage_columns.last_updated_sequence_number_column_idx; + iceberg_reader->set_row_lineage_columns(std::move(row_lineage_columns)); + } + iceberg_reader->set_push_down_agg_type(_get_push_down_agg_type()); + init_status = iceberg_reader->init_reader( _file_col_names, &_src_block_name_to_idx, _push_down_conjuncts, slot_id_to_predicates, _real_tuple_desc, _default_val_row_desc.get(), @@ -1359,10 +1370,18 @@ Status FileScanner::_init_orc_reader(std::unique_ptr&& orc_reader, std::unique_ptr iceberg_reader = IcebergOrcReader::create_unique( std::move(orc_reader), _profile, _state, *_params, range, _kv_cache, _io_ctx.get(), file_meta_cache_ptr); - if (_need_iceberg_rowid_column) { iceberg_reader->set_need_row_id_column(true); } + if (_row_lineage_columns.row_id_column_idx != -1 || + _row_lineage_columns.last_updated_sequence_number_column_idx != -1) { + std::shared_ptr row_lineage_columns; + row_lineage_columns = std::make_shared(); + row_lineage_columns->row_id_column_idx = _row_lineage_columns.row_id_column_idx; + row_lineage_columns->last_updated_sequence_number_column_idx = + _row_lineage_columns.last_updated_sequence_number_column_idx; + iceberg_reader->set_row_lineage_columns(std::move(row_lineage_columns)); + } init_status = iceberg_reader->init_reader( _file_col_names, &_src_block_name_to_idx, _push_down_conjuncts, _real_tuple_desc, _default_val_row_desc.get(), _col_name_to_slot_id, @@ -1704,6 +1723,15 @@ Status FileScanner::_init_expr_ctxes() { continue; } + if (it->second->col_name() == IcebergTableReader::ROW_LINEAGE_ROW_ID) { + _row_lineage_columns.row_id_column_idx = _default_val_row_desc->get_column_id(slot_id); + } + + if (it->second->col_name() == IcebergTableReader::ROW_LINEAGE_LAST_UPDATED_SEQ_NUMBER) { + _row_lineage_columns.last_updated_sequence_number_column_idx = + _default_val_row_desc->get_column_id(slot_id); + } + if (slot_info.is_file_slot) { _is_file_slot.emplace(slot_id); _file_slot_descs.emplace_back(it->second); diff --git a/be/src/exec/scan/file_scanner.h b/be/src/exec/scan/file_scanner.h index 5f68599d1ed24b..08b808ef2af1fc 100644 --- a/be/src/exec/scan/file_scanner.h +++ b/be/src/exec/scan/file_scanner.h @@ -35,6 +35,7 @@ #include "format/generic_reader.h" #include "format/orc/vorc_reader.h" #include "format/parquet/vparquet_reader.h" +#include "format/table/iceberg_reader.h" #include "io/io_common.h" #include "runtime/descriptors.h" #include "runtime/runtime_profile.h" @@ -231,6 +232,8 @@ class FileScanner : public Scanner { -1}; bool _need_iceberg_rowid_column = false; int _iceberg_rowid_column_pos = -1; + // for iceberg row lineage + RowLineageColumns _row_lineage_columns; int64_t _last_bytes_read_from_local = 0; int64_t _last_bytes_read_from_remote = 0; diff --git a/be/src/exec/sink/viceberg_delete_sink.cpp b/be/src/exec/sink/viceberg_delete_sink.cpp index fd280af8be6b79..e2b142db8d9f86 100644 --- a/be/src/exec/sink/viceberg_delete_sink.cpp +++ b/be/src/exec/sink/viceberg_delete_sink.cpp @@ -18,6 +18,9 @@ #include "exec/sink/viceberg_delete_sink.h" #include +#include +#include +#include #include "common/logging.h" #include "core/block/column_with_type_and_name.h" @@ -30,14 +33,79 @@ #include "core/data_type/data_type_number.h" #include "core/data_type/data_type_string.h" #include "core/data_type/data_type_struct.h" +#include "exec/common/endian.h" #include "exprs/vexpr.h" +#include "format/table/iceberg_delete_file_reader_helper.h" #include "format/transformer/vfile_format_transformer.h" +#include "io/file_factory.h" #include "runtime/runtime_state.h" +#include "util/slice.h" #include "util/string_util.h" #include "util/uid_util.h" namespace doris { +namespace { + +class RewriteBitmapVisitor final : public IcebergPositionDeleteVisitor { +public: + RewriteBitmapVisitor(const std::string& referenced_data_file_path, + roaring::Roaring64Map* rows_to_delete) + : _referenced_data_file_path(referenced_data_file_path), + _rows_to_delete(rows_to_delete) {} + + Status visit(const std::string& file_path, int64_t pos) override { + if (_rows_to_delete == nullptr) { + return Status::InvalidArgument("rows_to_delete is null"); + } + if (file_path == _referenced_data_file_path) { + _rows_to_delete->add(static_cast(pos)); + } + return Status::OK(); + } + +private: + const std::string& _referenced_data_file_path; + roaring::Roaring64Map* _rows_to_delete; +}; + +Status load_rewritable_delete_rows(RuntimeState* state, RuntimeProfile* profile, + const std::string& referenced_data_file_path, + const std::vector& delete_files, + const std::map& hadoop_conf, + TFileType::type file_type, + const std::vector& broker_addresses, + roaring::Roaring64Map* rows_to_delete) { + if (rows_to_delete == nullptr) { + return Status::InvalidArgument("rows_to_delete is null"); + } + if (state == nullptr || profile == nullptr || delete_files.empty()) { + return Status::OK(); + } + + TFileScanRangeParams params = + build_iceberg_delete_scan_range_params(hadoop_conf, file_type, broker_addresses); + IcebergDeleteFileIOContext delete_file_io_ctx(state); + IcebergDeleteFileReaderOptions options; + options.state = state; + options.profile = profile; + options.scan_params = ¶ms; + options.io_ctx = &delete_file_io_ctx.io_ctx; + options.batch_size = 102400; + + for (const auto& delete_file : delete_files) { + if (is_iceberg_deletion_vector(delete_file)) { + RETURN_IF_ERROR(read_iceberg_deletion_vector(delete_file, options, rows_to_delete)); + continue; + } + RewriteBitmapVisitor visitor(referenced_data_file_path, rows_to_delete); + RETURN_IF_ERROR(read_iceberg_position_delete_file(delete_file, options, &visitor)); + } + return Status::OK(); +} + +} // namespace + VIcebergDeleteSink::VIcebergDeleteSink(const TDataSink& t_sink, const VExprContextSPtrs& output_exprs, std::shared_ptr dep, @@ -95,6 +163,24 @@ Status VIcebergDeleteSink::init_properties(ObjectPool* pool) { _partition_data_json = delete_sink.partition_data_json; } + if (delete_sink.__isset.format_version) { + _format_version = delete_sink.format_version; + } + + // for merge old deletion vector and old position delete to a new deletion vector. + if (_format_version >= 3 && delete_sink.__isset.rewritable_delete_file_sets) { + for (const auto& delete_file_set : delete_sink.rewritable_delete_file_sets) { + if (!delete_file_set.__isset.referenced_data_file_path || + !delete_file_set.__isset.delete_files || + delete_file_set.referenced_data_file_path.empty() || + delete_file_set.delete_files.empty()) { + continue; + } + _rewritable_delete_files.emplace(delete_file_set.referenced_data_file_path, + delete_file_set.delete_files); + } + } + return Status::OK(); } @@ -111,10 +197,13 @@ Status VIcebergDeleteSink::open(RuntimeState* state, RuntimeProfile* profile) { SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(_init_position_delete_output_exprs()); + if (_format_version < 3) { + RETURN_IF_ERROR(_init_position_delete_output_exprs()); + } - LOG(INFO) << fmt::format("VIcebergDeleteSink opened: delete_type={}, output_path={}", - to_string(_delete_type), _output_path); + LOG(INFO) << fmt::format( + "VIcebergDeleteSink opened: delete_type={}, output_path={}, format_version={}", + to_string(_delete_type), _output_path, _format_version); return Status::OK(); } @@ -153,7 +242,11 @@ Status VIcebergDeleteSink::close(Status close_status) { if (_delete_type == TFileContent::POSITION_DELETES && !_file_deletions.empty()) { SCOPED_TIMER(_write_delete_files_timer); - RETURN_IF_ERROR(_write_position_delete_files(_file_deletions)); + if (_format_version >= 3) { + RETURN_IF_ERROR(_write_deletion_vector_files(_file_deletions)); + } else { + RETURN_IF_ERROR(_write_position_delete_files(_file_deletions)); + } } // Update counters @@ -475,6 +568,166 @@ std::string VIcebergDeleteSink::_get_file_extension() const { return fmt::format("{}{}", compress_name, file_format_name); } +Status VIcebergDeleteSink::_write_deletion_vector_files( + const std::map& file_deletions) { + std::vector blobs; + for (const auto& [data_file_path, deletion] : file_deletions) { + if (deletion.rows_to_delete.isEmpty()) { + continue; + } + roaring::Roaring64Map merged_rows = deletion.rows_to_delete; + DeletionVectorBlob blob; + blob.delete_count = static_cast(merged_rows.cardinality()); + auto previous_delete_it = _rewritable_delete_files.find(data_file_path); + if (previous_delete_it != _rewritable_delete_files.end()) { + roaring::Roaring64Map previous_rows; + RETURN_IF_ERROR(load_rewritable_delete_rows( + _state, _state->runtime_profile(), data_file_path, previous_delete_it->second, + _hadoop_conf, _file_type, _broker_addresses, &previous_rows)); + merged_rows |= previous_rows; + } + + size_t bitmap_size = merged_rows.getSizeInBytes(); + blob.referenced_data_file = data_file_path; + blob.partition_spec_id = deletion.partition_spec_id; + blob.partition_data_json = deletion.partition_data_json; + blob.merged_count = static_cast(merged_rows.cardinality()); + blob.content_size_in_bytes = static_cast(4 + 4 + bitmap_size + 4); + blob.blob_data.resize(static_cast(blob.content_size_in_bytes)); + merged_rows.write(blob.blob_data.data() + 8); + + uint32_t total_length = static_cast(4 + bitmap_size); + BigEndian::Store32(blob.blob_data.data(), total_length); + + constexpr char DV_MAGIC[] = {'\xD1', '\xD3', '\x39', '\x64'}; + memcpy(blob.blob_data.data() + 4, DV_MAGIC, 4); + + uint32_t crc = static_cast( + ::crc32(0, reinterpret_cast(blob.blob_data.data() + 4), + 4 + (uInt)bitmap_size)); + BigEndian::Store32(blob.blob_data.data() + 8 + bitmap_size, crc); + blobs.emplace_back(std::move(blob)); + } + + if (blobs.empty()) { + return Status::OK(); + } + + std::string puffin_path = _generate_puffin_file_path(); + int64_t puffin_file_size = 0; + RETURN_IF_ERROR(_write_puffin_file(puffin_path, &blobs, &puffin_file_size)); + + for (const auto& blob : blobs) { + TIcebergCommitData commit_data; + commit_data.__set_file_path(puffin_path); + commit_data.__set_row_count(blob.merged_count); + commit_data.__set_affected_rows(blob.delete_count); + commit_data.__set_file_size(puffin_file_size); + commit_data.__set_file_content(TFileContent::DELETION_VECTOR); + commit_data.__set_content_offset(blob.content_offset); + commit_data.__set_content_size_in_bytes(blob.content_size_in_bytes); + commit_data.__set_referenced_data_file_path(blob.referenced_data_file); + if (blob.partition_spec_id != 0 || !blob.partition_data_json.empty()) { + commit_data.__set_partition_spec_id(blob.partition_spec_id); + commit_data.__set_partition_data_json(blob.partition_data_json); + } + + _commit_data_list.push_back(commit_data); + _delete_file_count++; + } + return Status::OK(); +} + +Status VIcebergDeleteSink::_write_puffin_file(const std::string& puffin_path, + std::vector* blobs, + int64_t* out_file_size) { + DCHECK(blobs != nullptr); + DCHECK(!blobs->empty()); + + io::FSPropertiesRef fs_properties(_file_type); + fs_properties.properties = &_hadoop_conf; + if (!_broker_addresses.empty()) { + fs_properties.broker_addresses = &_broker_addresses; + } + io::FileDescription file_description = {.path = puffin_path, .fs_name {}}; + auto fs = DORIS_TRY(FileFactory::create_fs(fs_properties, file_description)); + io::FileWriterOptions file_writer_options = {.used_by_s3_committer = false}; + io::FileWriterPtr file_writer; + RETURN_IF_ERROR(fs->create_file(file_description.path, &file_writer, &file_writer_options)); + + constexpr char PUFFIN_MAGIC[] = {'\x50', '\x46', '\x41', '\x31'}; + RETURN_IF_ERROR(file_writer->append(Slice(reinterpret_cast(PUFFIN_MAGIC), 4))); + int64_t current_offset = 4; + for (auto& blob : *blobs) { + blob.content_offset = current_offset; + RETURN_IF_ERROR(file_writer->append(Slice( + reinterpret_cast(blob.blob_data.data()), blob.blob_data.size()))); + current_offset += static_cast(blob.blob_data.size()); + } + RETURN_IF_ERROR(file_writer->append(Slice(reinterpret_cast(PUFFIN_MAGIC), 4))); + + std::string footer_json = _build_puffin_footer_json(*blobs); + RETURN_IF_ERROR(file_writer->append( + Slice(reinterpret_cast(footer_json.data()), footer_json.size()))); + + char footer_size_buf[4]; + LittleEndian::Store32(footer_size_buf, static_cast(footer_json.size())); + RETURN_IF_ERROR(file_writer->append( + Slice(reinterpret_cast(footer_size_buf), sizeof(footer_size_buf)))); + + char flags[4] = {0, 0, 0, 0}; + RETURN_IF_ERROR( + file_writer->append(Slice(reinterpret_cast(flags), sizeof(flags)))); + RETURN_IF_ERROR(file_writer->append(Slice(reinterpret_cast(PUFFIN_MAGIC), 4))); + RETURN_IF_ERROR(file_writer->close()); + + *out_file_size = current_offset + 4 + static_cast(footer_json.size()) + 4 + 4 + 4; + return Status::OK(); +} + +std::string VIcebergDeleteSink::_build_puffin_footer_json( + const std::vector& blobs) { + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + writer.StartObject(); + writer.Key("blobs"); + writer.StartArray(); + for (const auto& blob : blobs) { + writer.StartObject(); + writer.Key("type"); + writer.String("deletion-vector-v1"); + writer.Key("fields"); + writer.StartArray(); + writer.EndArray(); + writer.Key("snapshot-id"); + writer.Int64(-1); + writer.Key("sequence-number"); + writer.Int64(-1); + writer.Key("offset"); + writer.Int64(blob.content_offset); + writer.Key("length"); + writer.Int64(blob.content_size_in_bytes); + writer.Key("properties"); + writer.StartObject(); + writer.Key("referenced-data-file"); + writer.String(blob.referenced_data_file.c_str(), + static_cast(blob.referenced_data_file.size())); + std::string cardinality = std::to_string(blob.merged_count); + writer.Key("cardinality"); + writer.String(cardinality.c_str(), static_cast(cardinality.size())); + writer.EndObject(); + writer.EndObject(); + } + writer.EndArray(); + writer.Key("properties"); + writer.StartObject(); + writer.Key("created-by"); + writer.String("doris-puffin-v1"); + writer.EndObject(); + writer.EndObject(); + return {buffer.GetString(), buffer.GetSize()}; +} + std::string VIcebergDeleteSink::_generate_delete_file_path( const std::string& referenced_data_file) { // Generate unique delete file name using UUID @@ -498,4 +751,14 @@ std::string VIcebergDeleteSink::_generate_delete_file_path( return fmt::format("{}{}", base_path, file_name); } +std::string VIcebergDeleteSink::_generate_puffin_file_path() { + std::string uuid = generate_uuid_string(); + std::string file_name = fmt::format("delete_dv_{}.puffin", uuid); + std::string base_path = _output_path.empty() ? _table_location : _output_path; + if (!base_path.empty() && base_path.back() != '/') { + base_path += '/'; + } + return fmt::format("{}{}", base_path, file_name); +} + } // namespace doris diff --git a/be/src/exec/sink/viceberg_delete_sink.h b/be/src/exec/sink/viceberg_delete_sink.h index 9efcb1fe681a68..22ae98cc288100 100644 --- a/be/src/exec/sink/viceberg_delete_sink.h +++ b/be/src/exec/sink/viceberg_delete_sink.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include #include @@ -26,10 +27,10 @@ #include "common/status.h" #include "core/block/block.h" -#include "core/value/bitmap_value.h" #include "exec/sink/writer/async_result_writer.h" #include "exec/sink/writer/iceberg/viceberg_delete_file_writer.h" #include "exprs/vexpr_fwd.h" +#include "roaring/roaring64map.hh" #include "runtime/runtime_profile.h" namespace doris { @@ -47,7 +48,7 @@ struct IcebergFileDeletion { int32_t partition_spec_id = 0; std::string partition_data_json; - doris::detail::Roaring64Map rows_to_delete; + roaring::Roaring64Map rows_to_delete; }; /** @@ -85,6 +86,28 @@ class VIcebergDeleteSink final : public AsyncResultWriter { Status _write_position_delete_files( const std::map& file_deletions); + Status _write_deletion_vector_files( + const std::map& file_deletions); + + struct DeletionVectorBlob { + std::string referenced_data_file; + int32_t partition_spec_id = 0; + std::string partition_data_json; + int64_t delete_count = 0; // The number of rows deleted in this delete operation. + int64_t merged_count = + 0; // The number of rows after merging the old deletion vector and position delete. + int64_t content_offset = 0; + int64_t content_size_in_bytes = 0; + std::vector blob_data; + }; + + Status _write_puffin_file(const std::string& puffin_path, + std::vector* blobs, int64_t* out_file_size); + + std::string _build_puffin_footer_json(const std::vector& blobs); + + std::string _generate_puffin_file_path(); + /** * Generate unique delete file path */ @@ -106,6 +129,7 @@ class VIcebergDeleteSink final : public AsyncResultWriter { TDataSink _t_sink; RuntimeState* _state = nullptr; + int32_t _format_version = 2; TFileContent::type _delete_type = TFileContent::POSITION_DELETES; // Writers for delete files @@ -117,6 +141,7 @@ class VIcebergDeleteSink final : public AsyncResultWriter { // per-file when the upstream guarantees file_path ordering, or flushing // when estimated memory exceeds a threshold, to reduce peak memory usage. std::map _file_deletions; + std::map> _rewritable_delete_files; // Hadoop configuration std::map _hadoop_conf; diff --git a/be/src/exec/sink/viceberg_merge_sink.cpp b/be/src/exec/sink/viceberg_merge_sink.cpp index 167c51cd6fa713..f12abd24d7719e 100644 --- a/be/src/exec/sink/viceberg_merge_sink.cpp +++ b/be/src/exec/sink/viceberg_merge_sink.cpp @@ -293,6 +293,12 @@ Status VIcebergMergeSink::_build_inner_sinks() { if (merge_sink.__isset.broker_addresses) { delete_sink.__set_broker_addresses(merge_sink.broker_addresses); } + if (merge_sink.__isset.format_version) { + delete_sink.__set_format_version(merge_sink.format_version); + } + if (merge_sink.__isset.rewritable_delete_file_sets) { + delete_sink.__set_rewritable_delete_file_sets(merge_sink.rewritable_delete_file_sets); + } _delete_sink.__set_type(TDataSinkType::ICEBERG_DELETE_SINK); _delete_sink.__set_iceberg_delete_sink(delete_sink); diff --git a/be/src/exec/sink/writer/iceberg/viceberg_delete_file_writer.cpp b/be/src/exec/sink/writer/iceberg/viceberg_delete_file_writer.cpp index a531cf342d6a89..8d1724a4396def 100644 --- a/be/src/exec/sink/writer/iceberg/viceberg_delete_file_writer.cpp +++ b/be/src/exec/sink/writer/iceberg/viceberg_delete_file_writer.cpp @@ -128,6 +128,7 @@ Status VIcebergDeleteFileWriter::close(TIcebergCommitData& commit_data) { // Fill commit data (use __set_ to mark optional fields as present) commit_data.__set_file_path(_output_path); commit_data.__set_row_count(_written_rows); + commit_data.__set_affected_rows(_written_rows); commit_data.__set_file_size(_file_size); commit_data.__set_file_content(_delete_type); diff --git a/be/src/exec/sink/writer/iceberg/viceberg_partition_writer.cpp b/be/src/exec/sink/writer/iceberg/viceberg_partition_writer.cpp index 1ae396f309e7b6..8766c945f5c032 100644 --- a/be/src/exec/sink/writer/iceberg/viceberg_partition_writer.cpp +++ b/be/src/exec/sink/writer/iceberg/viceberg_partition_writer.cpp @@ -145,6 +145,7 @@ Status VIcebergPartitionWriter::_build_iceberg_commit_data(TIcebergCommitData* c commit_data->__set_file_path( fmt::format("{}/{}", _write_info.original_write_path, _get_target_file_name())); commit_data->__set_row_count(_row_count); + commit_data->__set_affected_rows(_row_count); commit_data->__set_file_size(_file_format_transformer->written_len()); commit_data->__set_file_content(TFileContent::DATA); commit_data->__set_partition_values(_partition_values); diff --git a/be/src/exec/sink/writer/iceberg/viceberg_table_writer.cpp b/be/src/exec/sink/writer/iceberg/viceberg_table_writer.cpp index fcba431f1596a4..91ce0911200e96 100644 --- a/be/src/exec/sink/writer/iceberg/viceberg_table_writer.cpp +++ b/be/src/exec/sink/writer/iceberg/viceberg_table_writer.cpp @@ -21,6 +21,9 @@ #include "core/block/column_with_type_and_name.h" #include "core/block/materialize_block.h" #include "core/column/column_const.h" +#include "core/column/column_nullable.h" +#include "core/column/column_vector.h" +#include "core/data_type/data_type_nullable.h" #include "core/data_type_serde/data_type_serde.h" #include "exec/sink/writer/iceberg/partition_transformers.h" #include "exec/sink/writer/iceberg/viceberg_partition_writer.h" @@ -54,6 +57,9 @@ Status VIcebergTableWriter::open(RuntimeState* state, RuntimeProfile* profile) { state->query_options().iceberg_write_target_file_size_bytes > 0) { _target_file_size_bytes = state->query_options().iceberg_write_target_file_size_bytes; } + if (_t_sink.iceberg_table_sink.__isset.write_type) { + _write_type = _t_sink.iceberg_table_sink.write_type; + } // add all counter _written_rows_counter = ADD_COUNTER(_operator_profile, "WrittenRows", TUnit::UNIT); @@ -77,6 +83,14 @@ Status VIcebergTableWriter::open(RuntimeState* state, RuntimeProfile* profile) { _partition_spec = iceberg::PartitionSpecParser::from_json(_schema, partition_spec_json); _iceberg_partition_columns = _to_iceberg_partition_columns(); } + for (int i = 0; i < _schema->columns().size(); ++i) { + const auto& col_name = _schema->columns()[i].field_name(); + if (col_name == "_row_id") { + _row_id_column_idx = i; + } else if (col_name == "_last_updated_sequence_number") { + _last_updated_seq_num_column_idx = i; + } + } // Initialize static partition values if present _init_static_partition_values(); @@ -206,7 +220,28 @@ Status VIcebergTableWriter::write_prepared_block(Block& block) { return _write_prepared_block(block); } +Status VIcebergTableWriter::_process_row_lineage_columns(Block& block) { + if (_write_type != TIcebergWriteType::INSERT) { + return Status::OK(); + } + if (_last_updated_seq_num_column_idx < 0 || + _last_updated_seq_num_column_idx >= static_cast(block.columns())) { + return Status::OK(); + } + + auto& col_with_type = block.get_by_position(_last_updated_seq_num_column_idx); + size_t rows = block.rows(); + auto nested_type = remove_nullable(col_with_type.type); + auto nested_col = nested_type->create_column(); + nested_col->insert_many_defaults(rows); + auto null_map = ColumnUInt8::create(rows, 1); + col_with_type.column = ColumnNullable::create(std::move(nested_col), std::move(null_map)); + return Status::OK(); +} + Status VIcebergTableWriter::_write_prepared_block(Block& output_block) { + RETURN_IF_ERROR(_process_row_lineage_columns(output_block)); + std::unordered_map, IColumn::Filter> writer_positions; _row_count += output_block.rows(); diff --git a/be/src/exec/sink/writer/iceberg/viceberg_table_writer.h b/be/src/exec/sink/writer/iceberg/viceberg_table_writer.h index 32d40695ce2433..f94ce4feb6bb52 100644 --- a/be/src/exec/sink/writer/iceberg/viceberg_table_writer.h +++ b/be/src/exec/sink/writer/iceberg/viceberg_table_writer.h @@ -59,6 +59,10 @@ class VIcebergTableWriter final : public AsyncResultWriter { Status close(Status) override; + bool is_rewrite_compaction() const { return _write_type == TIcebergWriteType::REWRITE; } + + TIcebergWriteType::type write_type() const { return _write_type; } + // Getter for the current partition writer. // Used by SpillIcebergTableSinkLocalState to access the current writer for // memory management operations (get_reserve_mem_size, revocable_mem_size, etc.). @@ -126,6 +130,7 @@ class VIcebergTableWriter final : public AsyncResultWriter { doris::Block* output_block); Status _write_prepared_block(Block& output_block); + Status _process_row_lineage_columns(Block& block); // Currently it is a copy, maybe it is better to use move semantics to eliminate it. TDataSink _t_sink; @@ -133,6 +138,9 @@ class VIcebergTableWriter final : public AsyncResultWriter { // Target file size in bytes for controlling when to split files int64_t _target_file_size_bytes = 0; + TIcebergWriteType::type _write_type = TIcebergWriteType::INSERT; + int _row_id_column_idx = -1; + int _last_updated_seq_num_column_idx = -1; std::shared_ptr _schema; std::unique_ptr _partition_spec; diff --git a/be/src/format/orc/vorc_reader.cpp b/be/src/format/orc/vorc_reader.cpp index 61728c303ce46d..a8749336d01a64 100644 --- a/be/src/format/orc/vorc_reader.cpp +++ b/be/src/format/orc/vorc_reader.cpp @@ -81,6 +81,7 @@ #include "exprs/vin_predicate.h" #include "exprs/vruntimefilter_wrapper.h" #include "format/orc/orc_file_reader.h" +#include "format/table/iceberg_reader.h" #include "format/table/transactional_hive_common.h" #include "io/fs/buffered_reader.h" #include "io/fs/file_reader.h" @@ -1173,6 +1174,17 @@ Status OrcReader::set_fill_columns( TransactionalHive::READ_ROW_COLUMN_NAMES.end()); } + auto check_iceberg_row_lineage_column_idx = [&](const auto& col_name) -> int { + if (_row_lineage_columns != nullptr) { + if (col_name == IcebergTableReader::ROW_LINEAGE_ROW_ID) { + return _row_lineage_columns->row_id_column_idx; + } else if (col_name == IcebergTableReader::ROW_LINEAGE_LAST_UPDATED_SEQ_NUMBER) { + return _row_lineage_columns->last_updated_sequence_number_column_idx; + } + } + return -1; + }; + for (auto& read_table_col : _read_table_cols) { _lazy_read_ctx.all_read_columns.emplace_back(read_table_col); if (!predicate_table_columns.empty()) { @@ -1191,6 +1203,10 @@ Status OrcReader::set_fill_columns( _lazy_read_ctx.predicate_orc_columns.emplace_back( _table_info_node_ptr->children_file_column_name(iter->first)); + if (check_iceberg_row_lineage_column_idx(read_table_col) != -1) { + // Todo : enable lazy mat where filter iceberg row lineage column. + _enable_lazy_mat = false; + } } } } @@ -1220,6 +1236,9 @@ Status OrcReader::set_fill_columns( // predicate_missing_columns is VLiteral.To fill in default values for missing columns. _lazy_read_ctx.predicate_missing_columns.emplace(kv.first, kv.second); + if (check_iceberg_row_lineage_column_idx(kv.first) != -1) { + _enable_lazy_mat = false; + } } } @@ -1466,15 +1485,48 @@ Status OrcReader::_fill_missing_columns( } Status OrcReader::_fill_row_id_columns(Block* block, int64_t start_row) { + size_t fill_size = _batch->numElements; if (_row_id_column_iterator_pair.first != nullptr) { RETURN_IF_ERROR(_row_id_column_iterator_pair.first->seek_to_ordinal(start_row)); - size_t fill_size = _batch->numElements; - auto col = block->get_by_position(_row_id_column_iterator_pair.second) .column->assume_mutable(); RETURN_IF_ERROR(_row_id_column_iterator_pair.first->next_batch(&fill_size, col)); } + if (_row_lineage_columns != nullptr && _row_lineage_columns->need_row_ids() && + _row_lineage_columns->first_row_id >= 0) { + auto col = block->get_by_position(_row_lineage_columns->row_id_column_idx) + .column->assume_mutable(); + auto* nullable_column = assert_cast(col.get()); + auto& null_map = nullable_column->get_null_map_data(); + auto& data = + assert_cast(*nullable_column->get_nested_column_ptr()).get_data(); + for (size_t i = 0; i < fill_size; ++i) { + if (null_map[i] != 0) { + null_map[i] = 0; + data[i] = _row_lineage_columns->first_row_id + start_row + static_cast(i); + } + } + } + + if (_row_lineage_columns != nullptr && + _row_lineage_columns->has_last_updated_sequence_number_column() && + _row_lineage_columns->last_updated_sequence_number >= 0) { + auto col = block->get_by_position( + _row_lineage_columns->last_updated_sequence_number_column_idx) + .column->assume_mutable(); + auto* nullable_column = assert_cast(col.get()); + auto& null_map = nullable_column->get_null_map_data(); + auto& data = + assert_cast(*nullable_column->get_nested_column_ptr()).get_data(); + for (size_t i = 0; i < fill_size; ++i) { + if (null_map[i] != 0) { + null_map[i] = 0; + data[i] = _row_lineage_columns->last_updated_sequence_number; + } + } + } + return Status::OK(); } diff --git a/be/src/format/orc/vorc_reader.h b/be/src/format/orc/vorc_reader.h index be296f65b4d244..be4336785099a9 100644 --- a/be/src/format/orc/vorc_reader.h +++ b/be/src/format/orc/vorc_reader.h @@ -68,6 +68,7 @@ class FileSystem; struct IOContext; } // namespace io class Block; +struct RowLineageColumns; template class ColumnVector; template @@ -210,6 +211,9 @@ class OrcReader : public GenericReader { } void set_iceberg_rowid_params(const std::string& file_path, int32_t partition_spec_id, const std::string& partition_data_json, int row_id_column_pos); + void set_row_lineage_columns(std::shared_ptr row_lineage_columns) { + _row_lineage_columns = std::move(row_lineage_columns); + } static bool inline is_hive1_col_name(const orc::Type* orc_type_ptr) { for (uint64_t idx = 0; idx < orc_type_ptr->getSubtypeCount(); idx++) { @@ -746,6 +750,7 @@ class OrcReader : public GenericReader { std::pair, int> _row_id_column_iterator_pair = {nullptr, -1}; IcebergRowIdParams _iceberg_rowid_params; + std::shared_ptr _row_lineage_columns; // Through this node, you can find the file column based on the table column. std::shared_ptr _table_info_node_ptr = diff --git a/be/src/format/parquet/vparquet_group_reader.cpp b/be/src/format/parquet/vparquet_group_reader.cpp index 99961206b57011..40967c179ac029 100644 --- a/be/src/format/parquet/vparquet_group_reader.cpp +++ b/be/src/format/parquet/vparquet_group_reader.cpp @@ -59,6 +59,7 @@ #include "exprs/vslot_ref.h" #include "format/parquet/schema_desc.h" #include "format/parquet/vparquet_column_reader.h" +#include "format/table/iceberg_reader.h" #include "runtime/descriptors.h" #include "runtime/runtime_state.h" #include "runtime/thread_context.h" @@ -216,6 +217,18 @@ Status RowGroupReader::init( for (size_t i = 0; i < predicate_col_names.size(); ++i) { const std::string& predicate_col_name = predicate_col_names[i]; int slot_id = predicate_col_slot_ids[i]; + if (predicate_col_name == IcebergTableReader::ROW_LINEAGE_ROW_ID || + predicate_col_name == IcebergTableReader::ROW_LINEAGE_LAST_UPDATED_SEQ_NUMBER) { + // row lineage column can not dict filter. + if (_slot_id_to_filter_conjuncts->find(slot_id) != + _slot_id_to_filter_conjuncts->end()) { + for (auto& ctx : _slot_id_to_filter_conjuncts->at(slot_id)) { + _filter_conjuncts.push_back(ctx); + } + } + continue; + } + 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)); @@ -904,7 +917,8 @@ Status RowGroupReader::_read_empty_batch(size_t batch_size, size_t* read_rows, b _position_delete_ctx.current_row_id = end_row_id; *batch_eof = _position_delete_ctx.current_row_id == _position_delete_ctx.last_row_id; - if (_row_id_column_iterator_pair.first != nullptr || _iceberg_rowid_params.enabled) { + if (_row_id_column_iterator_pair.first != nullptr || _iceberg_rowid_params.enabled || + (_row_lineage_columns != nullptr && _row_lineage_columns->need_row_ids())) { *modify_row_ids = true; _current_batch_row_ids.clear(); _current_batch_row_ids.resize(*read_rows); @@ -967,16 +981,54 @@ Status RowGroupReader::_get_current_batch_row_id(size_t read_rows) { Status RowGroupReader::_fill_row_id_columns(Block* block, size_t read_rows, bool is_current_row_ids) { + const bool need_row_ids = + _row_id_column_iterator_pair.first != nullptr || + (_row_lineage_columns != nullptr && _row_lineage_columns->need_row_ids()); + if (need_row_ids && !is_current_row_ids) { + RETURN_IF_ERROR(_get_current_batch_row_id(read_rows)); + } if (_row_id_column_iterator_pair.first != nullptr) { - if (!is_current_row_ids) { - RETURN_IF_ERROR(_get_current_batch_row_id(read_rows)); - } auto col = block->get_by_position(_row_id_column_iterator_pair.second) .column->assume_mutable(); RETURN_IF_ERROR(_row_id_column_iterator_pair.first->read_by_rowids( _current_batch_row_ids.data(), _current_batch_row_ids.size(), col)); } + if (_row_lineage_columns != nullptr && _row_lineage_columns->need_row_ids() && + _row_lineage_columns->first_row_id >= 0) { + auto col = block->get_by_position(_row_lineage_columns->row_id_column_idx) + .column->assume_mutable(); + auto* nullable_column = assert_cast(col.get()); + auto& null_map = nullable_column->get_null_map_data(); + auto& data = + assert_cast(*nullable_column->get_nested_column_ptr()).get_data(); + for (size_t i = 0; i < read_rows; ++i) { + if (null_map[i] != 0) { + null_map[i] = 0; + data[i] = _row_lineage_columns->first_row_id + + static_cast(_current_batch_row_ids[i]); + } + } + } + + if (_row_lineage_columns != nullptr && + _row_lineage_columns->has_last_updated_sequence_number_column() && + _row_lineage_columns->last_updated_sequence_number >= 0) { + auto col = block->get_by_position( + _row_lineage_columns->last_updated_sequence_number_column_idx) + .column->assume_mutable(); + auto* nullable_column = assert_cast(col.get()); + auto& null_map = nullable_column->get_null_map_data(); + auto& data = + assert_cast(*nullable_column->get_nested_column_ptr()).get_data(); + for (size_t i = 0; i < read_rows; ++i) { + if (null_map[i] != 0) { + null_map[i] = 0; + data[i] = _row_lineage_columns->last_updated_sequence_number; + } + } + } + return Status::OK(); } diff --git a/be/src/format/parquet/vparquet_group_reader.h b/be/src/format/parquet/vparquet_group_reader.h index cdc5e5b57b7c20..d6cd245e2a67e9 100644 --- a/be/src/format/parquet/vparquet_group_reader.h +++ b/be/src/format/parquet/vparquet_group_reader.h @@ -51,6 +51,7 @@ struct IOContext; } // namespace io class Block; class FieldDescriptor; +struct RowLineageColumns; } // namespace doris namespace tparquet { class ColumnMetaData; @@ -193,6 +194,10 @@ class RowGroupReader : public ProfileCollector { _iceberg_rowid_params = params; } + void set_row_lineage_columns(std::shared_ptr row_lineage_columns) { + _row_lineage_columns = std::move(row_lineage_columns); + } + void set_current_row_group_idx(RowGroupIndex row_group_idx) { _current_row_group_idx = row_group_idx; } @@ -255,6 +260,7 @@ class RowGroupReader : public ProfileCollector { cctz::time_zone* _ctz = nullptr; io::IOContext* _io_ctx = nullptr; PositionDeleteContext _position_delete_ctx; + std::shared_ptr _row_lineage_columns; // merge the row ranges generated from page index and position delete. RowRanges _read_ranges; diff --git a/be/src/format/parquet/vparquet_reader.cpp b/be/src/format/parquet/vparquet_reader.cpp index d4c402202e6c82..e13d899b8ba8b2 100644 --- a/be/src/format/parquet/vparquet_reader.cpp +++ b/be/src/format/parquet/vparquet_reader.cpp @@ -487,6 +487,17 @@ Status ParquetReader::set_fill_columns( const FieldDescriptor& schema = _file_metadata->schema(); + auto check_iceberg_row_lineage_column_idx = [&](const auto& col_name) -> int { + if (_row_lineage_columns != nullptr) { + if (col_name == IcebergTableReader::ROW_LINEAGE_ROW_ID) { + return _row_lineage_columns->row_id_column_idx; + } else if (col_name == IcebergTableReader::ROW_LINEAGE_LAST_UPDATED_SEQ_NUMBER) { + return _row_lineage_columns->last_updated_sequence_number_column_idx; + } + } + return -1; + }; + for (auto& read_table_col : _read_table_columns) { _lazy_read_ctx.all_read_columns.emplace_back(read_table_col); @@ -499,7 +510,21 @@ Status ParquetReader::set_fill_columns( if (predicate_columns.size() > 0) { auto iter = predicate_columns.find(read_table_col); if (iter == predicate_columns.end()) { - _lazy_read_ctx.lazy_read_columns.emplace_back(read_table_col); + if (auto row_lineage_idx = check_iceberg_row_lineage_column_idx(read_table_col); + row_lineage_idx != -1) { + _lazy_read_ctx.predicate_columns.first.emplace_back(read_table_col); + // row lineage column can not dict filter. + int slot_id = 0; + for (auto slot : _tuple_descriptor->slots()) { + if (slot->col_name_lower_case() == read_table_col) { + slot_id = slot->id(); + } + } + _lazy_read_ctx.predicate_columns.second.emplace_back(slot_id); + _lazy_read_ctx.all_predicate_col_ids.emplace_back(row_lineage_idx); + } else { + _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); @@ -523,9 +548,7 @@ Status ParquetReader::set_fill_columns( for (auto& kv : _lazy_read_ctx.fill_missing_columns) { auto iter = predicate_columns.find(kv.first); - if (iter == predicate_columns.end()) { - _lazy_read_ctx.missing_columns.emplace(kv.first, kv.second); - } else { + if (iter != predicate_columns.end()) { //For check missing column : missing column == xx, missing column is null,missing column is not null. if (_slot_id_to_filter_conjuncts->find(iter->second.second) != _slot_id_to_filter_conjuncts->end()) { @@ -536,6 +559,12 @@ Status ParquetReader::set_fill_columns( _lazy_read_ctx.predicate_missing_columns.emplace(kv.first, kv.second); _lazy_read_ctx.all_predicate_col_ids.emplace_back(iter->second.first); + } else if (auto row_lineage_idx = check_iceberg_row_lineage_column_idx(kv.first); + row_lineage_idx != -1) { + _lazy_read_ctx.predicate_missing_columns.emplace(kv.first, kv.second); + _lazy_read_ctx.all_predicate_col_ids.emplace_back(row_lineage_idx); + } else { + _lazy_read_ctx.missing_columns.emplace(kv.first, kv.second); } } @@ -795,6 +824,7 @@ Status ParquetReader::_next_row_group_reader() { _current_group_reader->set_current_row_group_idx(_current_row_group_index); _current_group_reader->set_row_id_column_iterator(_row_id_column_iterator_pair); + _current_group_reader->set_row_lineage_columns(_row_lineage_columns); _current_group_reader->set_col_name_to_block_idx(_col_name_to_block_idx); _current_group_reader->_table_info_node_ptr = _table_info_node_ptr; diff --git a/be/src/format/parquet/vparquet_reader.h b/be/src/format/parquet/vparquet_reader.h index 9b2512873b66aa..dedbbe3e5a3b9f 100644 --- a/be/src/format/parquet/vparquet_reader.h +++ b/be/src/format/parquet/vparquet_reader.h @@ -64,6 +64,7 @@ class FileMetaData; class PageIndex; class ShardedKVCache; class VExprContext; +struct RowLineageColumns; } // namespace doris namespace doris { @@ -165,6 +166,10 @@ class ParquetReader : public GenericReader { void set_iceberg_rowid_params(const std::string& file_path, int32_t partition_spec_id, const std::string& partition_data_json, int row_id_column_pos); + void set_row_lineage_columns(std::shared_ptr row_lineage_columns) { + _row_lineage_columns = std::move(row_lineage_columns); + } + bool count_read_rows() override { return true; } protected: @@ -344,6 +349,9 @@ class ParquetReader : public GenericReader { std::pair, int> _row_id_column_iterator_pair = {nullptr, -1}; + std::shared_ptr _row_lineage_columns; + +protected: bool _filter_groups = true; RowGroupReader::IcebergRowIdParams _iceberg_rowid_params; diff --git a/be/src/format/table/iceberg_delete_file_reader_helper.cpp b/be/src/format/table/iceberg_delete_file_reader_helper.cpp new file mode 100644 index 00000000000000..1029e22c622269 --- /dev/null +++ b/be/src/format/table/iceberg_delete_file_reader_helper.cpp @@ -0,0 +1,324 @@ +// 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 "format/table/iceberg_delete_file_reader_helper.h" + +#include +#include + +#include +#include +#include +#include + +#include "core/assert_cast.h" +#include "core/block/block.h" +#include "core/block/column_with_type_and_name.h" +#include "core/column/column_dictionary.h" +#include "core/column/column_string.h" +#include "core/column/column_vector.h" +#include "core/data_type/data_type_number.h" +#include "core/data_type/data_type_string.h" +#include "exec/common/endian.h" +#include "exprs/vexpr_context.h" +#include "format/orc/vorc_reader.h" +#include "format/parquet/vparquet_column_chunk_reader.h" +#include "format/parquet/vparquet_reader.h" +#include "format/table/deletion_vector_reader.h" +#include "format/table/table_format_reader.h" +#include "io/hdfs_builder.h" +#include "runtime/descriptors.h" +#include "runtime/runtime_state.h" +#include "storage/predicate/column_predicate.h" + +namespace doris { + +namespace { + +constexpr const char* ICEBERG_FILE_PATH = "file_path"; +constexpr const char* ICEBERG_ROW_POS = "pos"; + +const std::vector DELETE_COL_NAMES {ICEBERG_FILE_PATH, ICEBERG_ROW_POS}; +std::unordered_map DELETE_COL_NAME_TO_BLOCK_IDX = {{ICEBERG_FILE_PATH, 0}, + {ICEBERG_ROW_POS, 1}}; + +Status validate_position_delete_file_format(const TIcebergDeleteFileDesc& delete_file, + TFileFormatType::type* file_format) { + if (file_format == nullptr) { + return Status::InvalidArgument("position delete file format output is null"); + } + if (!delete_file.__isset.file_format) { + return Status::InternalError("Iceberg position delete file is missing file format"); + } + if (delete_file.file_format != TFileFormatType::FORMAT_PARQUET && + delete_file.file_format != TFileFormatType::FORMAT_ORC) { + return Status::NotSupported("Unsupported Iceberg delete file format {}", + delete_file.file_format); + } + *file_format = delete_file.file_format; + return Status::OK(); +} + +Status visit_position_delete_block(const Block& block, size_t read_rows, + IcebergPositionDeleteVisitor* visitor) { + if (visitor == nullptr) { + return Status::InvalidArgument("position delete visitor is null"); + } + if (read_rows == 0) { + return Status::OK(); + } + + auto name_to_pos_map = block.get_name_to_pos_map(); + auto path_it = name_to_pos_map.find(ICEBERG_FILE_PATH); + auto pos_it = name_to_pos_map.find(ICEBERG_ROW_POS); + if (path_it == name_to_pos_map.end() || pos_it == name_to_pos_map.end()) { + return Status::InternalError("Position delete block is missing required columns"); + } + + const auto* pos_column = + assert_cast(block.get_by_position(pos_it->second).column.get()); + const auto* path_column = block.get_by_position(path_it->second).column.get(); + + if (const auto* string_column = check_and_get_column(path_column); + string_column != nullptr) { + for (size_t i = 0; i < read_rows; ++i) { + RETURN_IF_ERROR(visitor->visit(string_column->get_data_at(i).to_string(), + pos_column->get_element(i))); + } + return Status::OK(); + } + + if (const auto* dict_column = check_and_get_column(path_column); + dict_column != nullptr) { + const auto& codes = dict_column->get_data(); + for (size_t i = 0; i < read_rows; ++i) { + RETURN_IF_ERROR(visitor->visit(dict_column->get_value(codes[i]).to_string(), + pos_column->get_element(i))); + } + return Status::OK(); + } + + return Status::InternalError("Unsupported file_path column type in position delete block"); +} + +Status init_parquet_delete_reader(ParquetReader* reader, bool* dictionary_coded) { + if (reader == nullptr || dictionary_coded == nullptr) { + return Status::InvalidArgument("invalid parquet delete reader arguments"); + } + + phmap::flat_hash_map>> slot_id_to_predicates; + RETURN_IF_ERROR(reader->init_reader(DELETE_COL_NAMES, &DELETE_COL_NAME_TO_BLOCK_IDX, {}, + slot_id_to_predicates, nullptr, nullptr, nullptr, nullptr, + nullptr, TableSchemaChangeHelper::ConstNode::get_instance(), + false)); + + std::unordered_map> + partition_columns; + std::unordered_map missing_columns; + RETURN_IF_ERROR(reader->set_fill_columns(partition_columns, missing_columns)); + + const tparquet::FileMetaData* meta_data = reader->get_meta_data(); + *dictionary_coded = true; + for (const auto& row_group : meta_data->row_groups) { + const auto& column_chunk = row_group.columns[0]; + if (!(column_chunk.__isset.meta_data && has_dict_page(column_chunk.meta_data))) { + *dictionary_coded = false; + break; + } + } + return Status::OK(); +} + +Status init_orc_delete_reader(OrcReader* reader) { + if (reader == nullptr) { + return Status::InvalidArgument("orc delete reader is null"); + } + + RETURN_IF_ERROR(reader->init_reader(&DELETE_COL_NAMES, &DELETE_COL_NAME_TO_BLOCK_IDX, {}, false, + nullptr, nullptr, nullptr, nullptr, + TableSchemaChangeHelper::ConstNode::get_instance())); + + std::unordered_map> + partition_columns; + std::unordered_map missing_columns; + RETURN_IF_ERROR(reader->set_fill_columns(partition_columns, missing_columns)); + return Status::OK(); +} + +Status decode_deletion_vector_buffer(const char* buf, size_t buffer_size, + roaring::Roaring64Map* rows_to_delete) { + if (buf == nullptr || rows_to_delete == nullptr) { + return Status::InvalidArgument("invalid deletion vector decode arguments"); + } + if (buffer_size < 12) { + return Status::DataQualityError("Deletion vector file size too small: {}", buffer_size); + } + + auto total_length = BigEndian::Load32(buf); + if (total_length + 8 != buffer_size) { + return Status::DataQualityError("Deletion vector length mismatch, expected: {}, actual: {}", + total_length + 8, buffer_size); + } + + constexpr static char MAGIC_NUMBER[] = {'\xD1', '\xD3', '\x39', '\x64'}; + if (memcmp(buf + sizeof(total_length), MAGIC_NUMBER, 4) != 0) { + return Status::DataQualityError("Deletion vector magic number mismatch"); + } + + try { + *rows_to_delete |= roaring::Roaring64Map::readSafe(buf + 8, buffer_size - 12); + } catch (const std::runtime_error& e) { + return Status::DataQualityError("Decode roaring bitmap failed, {}", e.what()); + } + return Status::OK(); +} + +} // namespace + +IcebergDeleteFileIOContext::IcebergDeleteFileIOContext(RuntimeState* state) { + io_ctx.file_cache_stats = &file_cache_stats; + io_ctx.file_reader_stats = &file_reader_stats; + if (state != nullptr) { + io_ctx.query_id = &state->query_id(); + } +} + +TFileScanRangeParams build_iceberg_delete_scan_range_params( + const std::map& hadoop_conf, TFileType::type file_type, + const std::vector& broker_addresses) { + TFileScanRangeParams params; + params.__set_file_type(file_type); + params.__set_properties(hadoop_conf); + if (file_type == TFileType::FILE_HDFS) { + params.__set_hdfs_params(parse_properties(hadoop_conf)); + } + if (!broker_addresses.empty()) { + params.__set_broker_addresses(broker_addresses); + } + return params; +} + +TFileRangeDesc build_iceberg_delete_file_range(const std::string& path) { + TFileRangeDesc range; + range.path = path; + range.start_offset = 0; + range.size = -1; + range.file_size = -1; + return range; +} + +bool is_iceberg_deletion_vector(const TIcebergDeleteFileDesc& delete_file) { + return delete_file.__isset.content && delete_file.content == 3; +} + +Status read_iceberg_position_delete_file(const TIcebergDeleteFileDesc& delete_file, + const IcebergDeleteFileReaderOptions& options, + IcebergPositionDeleteVisitor* visitor) { + if (options.state == nullptr || options.profile == nullptr || options.scan_params == nullptr || + options.io_ctx == nullptr || visitor == nullptr) { + return Status::InvalidArgument("invalid position delete reader options"); + } + + TFileRangeDesc delete_range = build_iceberg_delete_file_range(delete_file.path); + if (options.fs_name != nullptr && !options.fs_name->empty()) { + delete_range.__set_fs_name(*options.fs_name); + } + + TFileFormatType::type file_format; + RETURN_IF_ERROR(validate_position_delete_file_format(delete_file, &file_format)); + + if (file_format == TFileFormatType::FORMAT_PARQUET) { + ParquetReader reader(options.profile, *options.scan_params, delete_range, + options.batch_size, + const_cast(&_state->timezone_obj()), options.io_ctx, + options.state, options.meta_cache); + bool dictionary_coded = false; + RETURN_IF_ERROR(init_parquet_delete_reader(&reader, &dictionary_coded)); + + bool eof = false; + while (!eof) { + Block block; + if (dictionary_coded) { + block.insert(ColumnWithTypeAndName( + ColumnDictI32::create(FieldType::OLAP_FIELD_TYPE_VARCHAR), + std::make_shared(), ICEBERG_FILE_PATH)); + } else { + block.insert(ColumnWithTypeAndName(ColumnString::create(), + std::make_shared(), + ICEBERG_FILE_PATH)); + } + block.insert(ColumnWithTypeAndName(ColumnInt64::create(), + std::make_shared(), ICEBERG_ROW_POS)); + size_t read_rows = 0; + RETURN_IF_ERROR(reader.get_next_block(&block, &read_rows, &eof)); + RETURN_IF_ERROR(visit_position_delete_block(block, read_rows, visitor)); + } + return Status::OK(); + } + + if (file_format == TFileFormatType::FORMAT_ORC) { + OrcReader reader(options.profile, options.state, *options.scan_params, delete_range, + options.batch_size, options.state->timezone(), options.io_ctx, + options.meta_cache); + RETURN_IF_ERROR(init_orc_delete_reader(&reader)); + + bool eof = false; + while (!eof) { + Block block; + block.insert(ColumnWithTypeAndName( + ColumnString::create(), std::make_shared(), ICEBERG_FILE_PATH)); + block.insert(ColumnWithTypeAndName(ColumnInt64::create(), + std::make_shared(), ICEBERG_ROW_POS)); + size_t read_rows = 0; + RETURN_IF_ERROR(reader.get_next_block(&block, &read_rows, &eof)); + RETURN_IF_ERROR(visit_position_delete_block(block, read_rows, visitor)); + } + return Status::OK(); + } + + return Status::NotSupported("Unsupported Iceberg delete file format {}", file_format); +} + +Status read_iceberg_deletion_vector(const TIcebergDeleteFileDesc& delete_file, + const IcebergDeleteFileReaderOptions& options, + roaring::Roaring64Map* rows_to_delete) { + if (options.state == nullptr || options.profile == nullptr || options.scan_params == nullptr || + options.io_ctx == nullptr || rows_to_delete == nullptr) { + return Status::InvalidArgument("invalid deletion vector reader options"); + } + if (!delete_file.__isset.content_offset || !delete_file.__isset.content_size_in_bytes) { + return Status::InternalError("Deletion vector is missing content offset or length"); + } + + TFileRangeDesc delete_range = build_iceberg_delete_file_range(delete_file.path); + if (options.fs_name != nullptr && !options.fs_name->empty()) { + delete_range.__set_fs_name(*options.fs_name); + } + delete_range.start_offset = delete_file.content_offset; + delete_range.size = delete_file.content_size_in_bytes; + + DeletionVectorReader dv_reader(options.state, options.profile, *options.scan_params, + delete_range, options.io_ctx); + RETURN_IF_ERROR(dv_reader.open()); + + std::vector buf(delete_range.size); + RETURN_IF_ERROR(dv_reader.read_at(delete_range.start_offset, + {buf.data(), cast_set(delete_range.size)})); + return decode_deletion_vector_buffer(buf.data(), delete_range.size, rows_to_delete); +} + +} // namespace doris diff --git a/be/src/format/table/iceberg_delete_file_reader_helper.h b/be/src/format/table/iceberg_delete_file_reader_helper.h new file mode 100644 index 00000000000000..a6771212851ae8 --- /dev/null +++ b/be/src/format/table/iceberg_delete_file_reader_helper.h @@ -0,0 +1,78 @@ +// 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 +#include +#include +#include + +#include "common/status.h" +#include "io/io_common.h" +#include "roaring/roaring64map.hh" + +namespace doris { + +class FileMetaCache; +class RuntimeProfile; +class RuntimeState; + +struct IcebergDeleteFileIOContext { + explicit IcebergDeleteFileIOContext(RuntimeState* state); + + io::FileCacheStatistics file_cache_stats; + io::FileReaderStats file_reader_stats; + io::IOContext io_ctx; +}; + +struct IcebergDeleteFileReaderOptions { + RuntimeState* state = nullptr; + RuntimeProfile* profile = nullptr; + const TFileScanRangeParams* scan_params = nullptr; + io::IOContext* io_ctx = nullptr; + FileMetaCache* meta_cache = nullptr; + const std::string* fs_name = nullptr; + size_t batch_size = 102400; +}; + +class IcebergPositionDeleteVisitor { +public: + virtual ~IcebergPositionDeleteVisitor() = default; + virtual Status visit(const std::string& file_path, int64_t pos) = 0; +}; + +TFileScanRangeParams build_iceberg_delete_scan_range_params( + const std::map& hadoop_conf, TFileType::type file_type, + const std::vector& broker_addresses); + +TFileRangeDesc build_iceberg_delete_file_range(const std::string& path); + +bool is_iceberg_deletion_vector(const TIcebergDeleteFileDesc& delete_file); + +Status read_iceberg_position_delete_file(const TIcebergDeleteFileDesc& delete_file, + const IcebergDeleteFileReaderOptions& options, + IcebergPositionDeleteVisitor* visitor); + +Status read_iceberg_deletion_vector(const TIcebergDeleteFileDesc& delete_file, + const IcebergDeleteFileReaderOptions& options, + roaring::Roaring64Map* rows_to_delete); + +} // namespace doris diff --git a/be/src/format/table/iceberg_reader.cpp b/be/src/format/table/iceberg_reader.cpp index 1f10345a075da0..e24efde39d9fd6 100644 --- a/be/src/format/table/iceberg_reader.cpp +++ b/be/src/format/table/iceberg_reader.cpp @@ -37,12 +37,7 @@ #include "core/block/block.h" #include "core/block/column_with_type_and_name.h" #include "core/column/column.h" -#include "core/column/column_string.h" -#include "core/column/column_vector.h" #include "core/data_type/data_type_factory.hpp" -#include "core/data_type/define_primitive_type.h" -#include "core/data_type/primitive_type.h" -#include "core/string_ref.h" #include "exprs/aggregate/aggregate_function.h" #include "format/format_common.h" #include "format/generic_reader.h" @@ -52,6 +47,7 @@ #include "format/table/deletion_vector_reader.h" #include "format/table/iceberg/iceberg_orc_nested_column_utils.h" #include "format/table/iceberg/iceberg_parquet_nested_column_utils.h" +#include "format/table/iceberg_delete_file_reader_helper.h" #include "format/table/nested_column_access_helper.h" #include "format/table/table_format_reader.h" #include "runtime/runtime_state.h" @@ -73,6 +69,42 @@ class VExprContext; } // namespace doris namespace doris { +namespace { + +class GroupedDeleteRowsVisitor final : public IcebergPositionDeleteVisitor { +public: + using DeleteRows = std::vector; + using DeleteFile = phmap::parallel_flat_hash_map< + std::string, std::unique_ptr, std::hash, std::equal_to<>, + std::allocator>>, 8, + std::mutex>; + + explicit GroupedDeleteRowsVisitor(DeleteFile* position_delete) + : _position_delete(position_delete) {} + + Status visit(const std::string& file_path, int64_t pos) override { + if (_position_delete == nullptr) { + return Status::InvalidArgument("position delete map is null"); + } + + auto iter = _position_delete->find(file_path); + DeleteRows* delete_rows = nullptr; + if (iter == _position_delete->end()) { + delete_rows = new DeleteRows; + (*_position_delete)[file_path] = std::unique_ptr(delete_rows); + } else { + delete_rows = iter->second.get(); + } + delete_rows->push_back(pos); + return Status::OK(); + } + +private: + DeleteFile* _position_delete; +}; + +} // namespace + const std::string IcebergOrcReader::ICEBERG_ORC_ATTRIBUTE = "iceberg.id"; IcebergTableReader::IcebergTableReader(std::unique_ptr file_format_reader, @@ -128,6 +160,9 @@ Status IcebergTableReader::init_row_filters() { return Status::OK(); } + auto* parquet_reader = dynamic_cast(_file_format_reader.get()); + auto* orc_reader = dynamic_cast(_file_format_reader.get()); + // Initialize file information for $row_id generation // Extract from table_desc which contains current file's metadata if (_need_row_id_column) { @@ -141,10 +176,10 @@ Status IcebergTableReader::init_row_filters() { partition_data_json = table_desc.partition_data_json; } - if (auto* parquet_reader = dynamic_cast(_file_format_reader.get())) { + if (parquet_reader != nullptr) { parquet_reader->set_iceberg_rowid_params(file_path, partition_spec_id, partition_data_json, _row_id_column_position); - } else if (auto* orc_reader = dynamic_cast(_file_format_reader.get())) { + } else if (orc_reader != nullptr) { orc_reader->set_iceberg_rowid_params(file_path, partition_spec_id, partition_data_json, _row_id_column_position); } @@ -247,15 +282,7 @@ Status IcebergTableReader::_position_delete_base( auto* delete_file_cache = _kv_cache->get( _delet_file_cache_key(delete_file.path), [&]() -> DeleteFile* { auto* position_delete = new DeleteFile; - TFileRangeDesc delete_file_range; - // must use __set() method to make sure __isset is true - delete_file_range.__set_fs_name(_range.fs_name); - delete_file_range.path = delete_file.path; - delete_file_range.start_offset = 0; - delete_file_range.size = -1; - delete_file_range.file_size = -1; - //read position delete file base on delete_file_range , generate DeleteFile , add DeleteFile to kv_cache - create_status = _read_position_delete_file(&delete_file_range, position_delete); + create_status = _read_position_delete_file(delete_file, position_delete); if (!create_status) { return nullptr; @@ -302,45 +329,18 @@ Status IcebergTableReader::_position_delete_base( return Status::OK(); } -IcebergTableReader::PositionDeleteRange IcebergTableReader::_get_range( - const ColumnDictI32& file_path_column) { - IcebergTableReader::PositionDeleteRange range; - size_t read_rows = file_path_column.get_data().size(); - int* code_path = const_cast(file_path_column.get_data().data()); - int* code_path_start = code_path; - int* code_path_end = code_path + read_rows; - while (code_path < code_path_end) { - int code = code_path[0]; - int* code_end = std::upper_bound(code_path, code_path_end, code); - range.data_file_path.emplace_back(file_path_column.get_value(code).to_string()); - range.range.emplace_back(code_path - code_path_start, code_end - code_path_start); - code_path = code_end; - } - return range; -} - -IcebergTableReader::PositionDeleteRange IcebergTableReader::_get_range( - const ColumnString& file_path_column) { - IcebergTableReader::PositionDeleteRange range; - size_t read_rows = file_path_column.size(); - size_t index = 0; - while (index < read_rows) { - StringRef data_path = file_path_column.get_data_at(index); - size_t left = index - 1; - size_t right = read_rows; - while (left + 1 != right) { - size_t mid = left + (right - left) / 2; - if (file_path_column.get_data_at(mid) > data_path) { - right = mid; - } else { - left = mid; - } - } - range.data_file_path.emplace_back(data_path.to_string()); - range.range.emplace_back(index, left + 1); - index = left + 1; - } - return range; +Status IcebergTableReader::_read_position_delete_file(const TIcebergDeleteFileDesc& delete_file, + DeleteFile* position_delete) { + GroupedDeleteRowsVisitor visitor(position_delete); + IcebergDeleteFileReaderOptions options; + options.state = _state; + options.profile = _profile; + options.scan_params = &_params; + options.io_ctx = _io_ctx; + options.meta_cache = _meta_cache; + options.fs_name = &_range.fs_name; + options.batch_size = READ_DELETE_FILE_BATCH_SIZE; + return read_iceberg_position_delete_file(delete_file, options, &visitor); } /** @@ -397,43 +397,6 @@ void IcebergTableReader::_sort_delete_rows( } } -void IcebergTableReader::_gen_position_delete_file_range(Block& block, DeleteFile* position_delete, - size_t read_rows, - bool file_path_column_dictionary_coded) { - SCOPED_TIMER(_iceberg_profile.parse_delete_file_time); - // todo: maybe do not need to build name to index map every time - auto name_to_pos_map = block.get_name_to_pos_map(); - ColumnPtr path_column = block.get_by_position(name_to_pos_map[ICEBERG_FILE_PATH]).column; - DCHECK_EQ(path_column->size(), read_rows); - ColumnPtr pos_column = block.get_by_position(name_to_pos_map[ICEBERG_ROW_POS]).column; - using ColumnType = typename PrimitiveTypeTraits::ColumnType; - const int64_t* src_data = assert_cast(*pos_column).get_data().data(); - IcebergTableReader::PositionDeleteRange range; - if (file_path_column_dictionary_coded) { - range = _get_range(assert_cast(*path_column)); - } else { - range = _get_range(assert_cast(*path_column)); - } - for (int i = 0; i < range.range.size(); ++i) { - std::string key = range.data_file_path[i]; - auto iter = position_delete->find(key); - DeleteRows* delete_rows; - if (iter == position_delete->end()) { - delete_rows = new DeleteRows; - std::unique_ptr delete_rows_ptr(delete_rows); - (*position_delete)[key] = std::move(delete_rows_ptr); - } else { - delete_rows = iter->second.get(); - } - const int64_t* cpy_start = src_data + range.range[i].first; - const int64_t cpy_count = range.range[i].second - range.range[i].first; - int64_t origin_size = delete_rows->size(); - delete_rows->resize(origin_size + cpy_count); - int64_t* dest_position = &(*delete_rows)[origin_size]; - memcpy(dest_position, cpy_start, cpy_count * sizeof(int64_t)); - } -} - Status IcebergParquetReader::init_reader( const std::vector& file_col_names, std::unordered_map* col_name_to_block_idx, @@ -449,6 +412,16 @@ Status IcebergParquetReader::init_reader( auto* parquet_reader = static_cast(_file_format_reader.get()); RETURN_IF_ERROR(parquet_reader->get_file_metadata_schema(&_data_file_field_desc)); DCHECK(_data_file_field_desc != nullptr); + if (_row_lineage_columns != nullptr) { + const auto& table_desc = _range.table_format_params.iceberg_params; + _row_lineage_columns->first_row_id = + table_desc.__isset.first_row_id ? table_desc.first_row_id : -1; + _row_lineage_columns->last_updated_sequence_number = + table_desc.__isset.last_updated_sequence_number + ? table_desc.last_updated_sequence_number + : -1; + parquet_reader->set_row_lineage_columns(_row_lineage_columns); + } auto column_id_result = _create_column_ids(_data_file_field_desc, tuple_descriptor); auto& column_ids = column_id_result.column_ids; @@ -638,54 +611,6 @@ ColumnIdResult IcebergParquetReader::_create_column_ids(const FieldDescriptor* f return ColumnIdResult(std::move(column_ids), std::move(filter_column_ids)); } -Status IcebergParquetReader ::_read_position_delete_file(const TFileRangeDesc* delete_range, - DeleteFile* position_delete) { - ParquetReader parquet_delete_reader( - _profile, _params, *delete_range, READ_DELETE_FILE_BATCH_SIZE, - const_cast(&_state->timezone_obj()), _io_ctx, _state, _meta_cache); - phmap::flat_hash_map>> tmp; - RETURN_IF_ERROR(parquet_delete_reader.init_reader( - delete_file_col_names, - const_cast*>(&DELETE_COL_NAME_TO_BLOCK_IDX), - {}, tmp, nullptr, nullptr, nullptr, nullptr, nullptr, - TableSchemaChangeHelper::ConstNode::get_instance(), false)); - - std::unordered_map> - partition_columns; - std::unordered_map missing_columns; - RETURN_IF_ERROR(parquet_delete_reader.set_fill_columns(partition_columns, missing_columns)); - - const tparquet::FileMetaData* meta_data = parquet_delete_reader.get_meta_data(); - bool dictionary_coded = true; - for (const auto& row_group : meta_data->row_groups) { - const auto& column_chunk = row_group.columns[ICEBERG_FILE_PATH_INDEX]; - if (!(column_chunk.__isset.meta_data && has_dict_page(column_chunk.meta_data))) { - dictionary_coded = false; - break; - } - } - DataTypePtr data_type_file_path {new DataTypeString}; - DataTypePtr data_type_pos {new DataTypeInt64}; - bool eof = false; - while (!eof) { - Block block = {dictionary_coded - ? ColumnWithTypeAndName {ColumnDictI32::create( - FieldType::OLAP_FIELD_TYPE_VARCHAR), - data_type_file_path, ICEBERG_FILE_PATH} - : ColumnWithTypeAndName {data_type_file_path, ICEBERG_FILE_PATH}, - - {data_type_pos, ICEBERG_ROW_POS}}; - size_t read_rows = 0; - RETURN_IF_ERROR(parquet_delete_reader.get_next_block(&block, &read_rows, &eof)); - - if (read_rows <= 0) { - break; - } - _gen_position_delete_file_range(block, position_delete, read_rows, dictionary_coded); - } - return Status::OK(); -}; - Status IcebergOrcReader::init_reader( const std::vector& file_col_names, std::unordered_map* col_name_to_block_idx, @@ -701,6 +626,16 @@ Status IcebergOrcReader::init_reader( std::vector data_file_col_names; std::vector data_file_col_types; RETURN_IF_ERROR(orc_reader->get_parsed_schema(&data_file_col_names, &data_file_col_types)); + if (_row_lineage_columns != nullptr) { + const auto& table_desc = _range.table_format_params.iceberg_params; + _row_lineage_columns->first_row_id = + table_desc.__isset.first_row_id ? table_desc.first_row_id : -1; + _row_lineage_columns->last_updated_sequence_number = + table_desc.__isset.last_updated_sequence_number + ? table_desc.last_updated_sequence_number + : -1; + orc_reader->set_row_lineage_columns(_row_lineage_columns); + } auto column_id_result = _create_column_ids(_data_file_type_desc, tuple_descriptor); auto& column_ids = column_id_result.column_ids; @@ -895,35 +830,6 @@ ColumnIdResult IcebergOrcReader::_create_column_ids(const orc::Type* orc_type, return ColumnIdResult(std::move(column_ids), std::move(filter_column_ids)); } -Status IcebergOrcReader::_read_position_delete_file(const TFileRangeDesc* delete_range, - DeleteFile* position_delete) { - OrcReader orc_delete_reader(_profile, _state, _params, *delete_range, - READ_DELETE_FILE_BATCH_SIZE, _state->timezone(), _io_ctx, - _meta_cache); - RETURN_IF_ERROR(orc_delete_reader.init_reader( - &delete_file_col_names, - const_cast*>(&DELETE_COL_NAME_TO_BLOCK_IDX), - {}, false, {}, {}, nullptr, nullptr)); - - std::unordered_map> - partition_columns; - std::unordered_map missing_columns; - RETURN_IF_ERROR(orc_delete_reader.set_fill_columns(partition_columns, missing_columns)); - - bool eof = false; - DataTypePtr data_type_file_path {new DataTypeString}; - DataTypePtr data_type_pos {new DataTypeInt64}; - while (!eof) { - Block block = {{data_type_file_path, ICEBERG_FILE_PATH}, {data_type_pos, ICEBERG_ROW_POS}}; - - size_t read_rows = 0; - RETURN_IF_ERROR(orc_delete_reader.get_next_block(&block, &read_rows, &eof)); - - _gen_position_delete_file_range(block, position_delete, read_rows, false); - } - return Status::OK(); -} - // Directly read the deletion vector using the `content_offset` and // `content_size_in_bytes` provided by FE in `delete_file_desc`. // These two fields indicate the location of a blob in storage. diff --git a/be/src/format/table/iceberg_reader.h b/be/src/format/table/iceberg_reader.h index 1b5c154f9d9447..b086412142267e 100644 --- a/be/src/format/table/iceberg_reader.h +++ b/be/src/format/table/iceberg_reader.h @@ -26,9 +26,6 @@ #include #include "common/status.h" -#include "core/column/column_dictionary.h" -#include "core/data_type/define_primitive_type.h" -#include "core/data_type/primitive_type.h" #include "exprs/vslot_ref.h" #include "format/orc/vorc_reader.h" #include "format/parquet/vparquet_reader.h" @@ -61,12 +58,23 @@ class GenericReader; class ShardedKVCache; class VExprContext; +struct RowLineageColumns { + int row_id_column_idx = -1; + int last_updated_sequence_number_column_idx = -1; + int64_t first_row_id = -1; + int64_t last_updated_sequence_number = -1; + + bool need_row_ids() const { return row_id_column_idx >= 0; } + bool has_last_updated_sequence_number_column() const { + return last_updated_sequence_number_column_idx >= 0; + } +}; + class IcebergTableReader : public TableFormatReader, public TableSchemaChangeHelper { public: - struct PositionDeleteRange { - std::vector data_file_path; - std::vector> range; - }; + static constexpr const char* ROW_LINEAGE_ROW_ID = "_row_id"; + static constexpr const char* ROW_LINEAGE_LAST_UPDATED_SEQ_NUMBER = + "_last_updated_sequence_number"; IcebergTableReader(std::unique_ptr file_format_reader, RuntimeProfile* profile, RuntimeState* state, const TFileScanRangeParams& params, @@ -90,6 +98,10 @@ class IcebergTableReader : public TableFormatReader, public TableSchemaChangeHel Status read_deletion_vector(const std::string& data_file_path, const TIcebergDeleteFileDesc& delete_file_desc); + void set_row_lineage_columns(std::shared_ptr row_lineage_columns) { + _row_lineage_columns = std::move(row_lineage_columns); + } + protected: struct IcebergProfile { RuntimeProfile::Counter* num_delete_files; @@ -116,10 +128,6 @@ class IcebergTableReader : public TableFormatReader, public TableSchemaChangeHel static void _sort_delete_rows(const std::vector*>& delete_rows_array, int64_t num_delete_rows, std::vector& result); - PositionDeleteRange _get_range(const ColumnDictI32& file_path_column); - - PositionDeleteRange _get_range(const ColumnString& file_path_column); - static std::string _delet_file_cache_key(const std::string& path) { return "delete_" + path; } Status _position_delete_base(const std::string data_file_path, @@ -147,20 +155,10 @@ class IcebergTableReader : public TableFormatReader, public TableSchemaChangeHel Fileformat _file_format = Fileformat::NONE; const int64_t MIN_SUPPORT_DELETE_FILES_VERSION = 2; - const std::string ICEBERG_FILE_PATH = "file_path"; - const std::string ICEBERG_ROW_POS = "pos"; - const std::vector delete_file_col_names {ICEBERG_FILE_PATH, ICEBERG_ROW_POS}; - const std::unordered_map DELETE_COL_NAME_TO_BLOCK_IDX = { - {ICEBERG_FILE_PATH, 0}, {ICEBERG_ROW_POS, 1}}; - const int ICEBERG_FILE_PATH_INDEX = 0; - const int ICEBERG_FILE_POS_INDEX = 1; const int READ_DELETE_FILE_BATCH_SIZE = 102400; - //Read position_delete_file TFileRangeDesc, generate DeleteFile - virtual Status _read_position_delete_file(const TFileRangeDesc*, DeleteFile*) = 0; - - void _gen_position_delete_file_range(Block& block, DeleteFile* const position_delete, - size_t read_rows, bool file_path_column_dictionary_coded); + // Read a position delete file from the full Iceberg delete descriptor. + Status _read_position_delete_file(const TIcebergDeleteFileDesc&, DeleteFile*); // read table colummn + extra equality delete columns std::vector _all_required_col_names; @@ -180,6 +178,8 @@ class IcebergTableReader : public TableFormatReader, public TableSchemaChangeHel // id -> block column name. std::unordered_map _id_to_block_column_name; + + std::shared_ptr _row_lineage_columns; }; class IcebergParquetReader final : public IcebergTableReader { @@ -211,9 +211,6 @@ class IcebergParquetReader final : public IcebergTableReader { private: static ColumnIdResult _create_column_ids(const FieldDescriptor* field_desc, const TupleDescriptor* tuple_descriptor); - - Status _read_position_delete_file(const TFileRangeDesc* delete_range, - DeleteFile* position_delete) final; Status _process_equality_delete(const std::vector& delete_files) final; const FieldDescriptor* _data_file_field_desc = nullptr; @@ -222,9 +219,6 @@ class IcebergOrcReader final : public IcebergTableReader { public: ENABLE_FACTORY_CREATOR(IcebergOrcReader); - Status _read_position_delete_file(const TFileRangeDesc* delete_range, - DeleteFile* position_delete) final; - IcebergOrcReader(std::unique_ptr file_format_reader, RuntimeProfile* profile, RuntimeState* state, const TFileScanRangeParams& params, const TFileRangeDesc& range, ShardedKVCache* kv_cache, io::IOContext* io_ctx, diff --git a/be/test/exec/sink/viceberg_delete_sink_test.cpp b/be/test/exec/sink/viceberg_delete_sink_test.cpp index fd5bec844421b3..d9fc5086503d90 100644 --- a/be/test/exec/sink/viceberg_delete_sink_test.cpp +++ b/be/test/exec/sink/viceberg_delete_sink_test.cpp @@ -18,6 +18,10 @@ #include "exec/sink/viceberg_delete_sink.h" #include +#include + +#include +#include #include "common/consts.h" #include "common/object_pool.h" @@ -28,9 +32,11 @@ #include "core/data_type/data_type_number.h" #include "core/data_type/data_type_string.h" #include "core/data_type/data_type_struct.h" +#include "exec/common/endian.h" #include "gen_cpp/DataSinks_types.h" #include "gen_cpp/Types_types.h" #include "runtime/runtime_state.h" +#include "util/uid_util.h" namespace doris { @@ -56,6 +62,25 @@ class VIcebergDeleteSinkTest : public testing::Test { _t_data_sink.__set_iceberg_delete_sink(delete_sink); } + TDataSink build_local_delete_sink(const std::string& output_path, int32_t format_version) { + TDataSink t_data_sink; + t_data_sink.__set_type(TDataSinkType::ICEBERG_DELETE_SINK); + + TIcebergDeleteSink delete_sink; + delete_sink.__set_db_name("test_db"); + delete_sink.__set_tb_name("test_table"); + delete_sink.__set_delete_type(TFileContent::POSITION_DELETES); + delete_sink.__set_file_format(TFileFormatType::FORMAT_PARQUET); + delete_sink.__set_compress_type(TFileCompressType::SNAPPYBLOCK); + delete_sink.__set_output_path(output_path); + delete_sink.__set_table_location(output_path); + delete_sink.__set_file_type(TFileType::FILE_LOCAL); + delete_sink.__set_format_version(format_version); + + t_data_sink.__set_iceberg_delete_sink(delete_sink); + return t_data_sink; + } + TDataSink _t_data_sink; }; @@ -482,4 +507,90 @@ TEST_F(VIcebergDeleteSinkTest, TestUnsupportedDeleteType) { ASSERT_FALSE(status.ok()); } +TEST_F(VIcebergDeleteSinkTest, TestWriteDeletionVectorsToSingleSharedPuffin) { + std::filesystem::path temp_dir = std::filesystem::temp_directory_path() / + ("iceberg_delete_sink_test_" + generate_uuid_string()); + ASSERT_TRUE(std::filesystem::create_directories(temp_dir)); + + TDataSink t_data_sink = build_local_delete_sink(temp_dir.string(), 3); + VExprContextSPtrs output_exprs; + auto sink = std::make_shared(t_data_sink, output_exprs, nullptr, nullptr); + ObjectPool pool; + ASSERT_TRUE(sink->init_properties(&pool).ok()); + + std::map file_deletions; + auto [file1_it, file1_inserted] = + file_deletions.emplace("file1.parquet", IcebergFileDeletion(1, "[\"p=1\"]")); + ASSERT_TRUE(file1_inserted); + file1_it->second.rows_to_delete.add((uint32_t)10); + file1_it->second.rows_to_delete.add((uint32_t)20); + + auto [file2_it, file2_inserted] = + file_deletions.emplace("file2.parquet", IcebergFileDeletion(2, "[\"p=2\"]")); + ASSERT_TRUE(file2_inserted); + file2_it->second.rows_to_delete.add((uint32_t)30); + + ASSERT_TRUE(sink->_write_deletion_vector_files(file_deletions).ok()); + ASSERT_EQ(2, sink->_commit_data_list.size()); + ASSERT_EQ(2, sink->_delete_file_count); + + const auto& first_commit = sink->_commit_data_list[0]; + const auto& second_commit = sink->_commit_data_list[1]; + ASSERT_EQ(first_commit.file_path, second_commit.file_path); + ASSERT_EQ(first_commit.file_size, second_commit.file_size); + ASSERT_LT(first_commit.content_offset, second_commit.content_offset); + + size_t puffin_file_count = 0; + for (const auto& entry : std::filesystem::directory_iterator(temp_dir)) { + if (entry.path().extension() == ".puffin") { + ++puffin_file_count; + } + } + ASSERT_EQ(1, puffin_file_count); + + std::ifstream input(first_commit.file_path, std::ios::binary); + ASSERT_TRUE(input.good()); + std::string file_bytes((std::istreambuf_iterator(input)), + std::istreambuf_iterator()); + ASSERT_EQ(static_cast(first_commit.file_size), file_bytes.size()); + ASSERT_GE(file_bytes.size(), 16); + ASSERT_EQ("PFA1", std::string(file_bytes.data(), 4)); + ASSERT_EQ("PFA1", std::string(file_bytes.data() + file_bytes.size() - 4, 4)); + + uint32_t footer_size = LittleEndian::Load32(file_bytes.data() + file_bytes.size() - 12); + size_t footer_start = file_bytes.size() - 12 - footer_size; + std::string footer_json = file_bytes.substr(footer_start, footer_size); + + rapidjson::Document footer_doc; + footer_doc.Parse(footer_json.c_str(), footer_json.size()); + ASSERT_FALSE(footer_doc.HasParseError()); + ASSERT_TRUE(footer_doc.HasMember("blobs")); + ASSERT_TRUE(footer_doc["blobs"].IsArray()); + ASSERT_EQ(2u, footer_doc["blobs"].Size()); + + std::map commit_data_by_file; + commit_data_by_file.emplace(first_commit.referenced_data_file_path, &first_commit); + commit_data_by_file.emplace(second_commit.referenced_data_file_path, &second_commit); + + for (const auto& blob : footer_doc["blobs"].GetArray()) { + ASSERT_TRUE(blob.IsObject()); + ASSERT_TRUE(blob.HasMember("properties")); + ASSERT_TRUE(blob["properties"].IsObject()); + const auto& properties = blob["properties"]; + ASSERT_TRUE(properties.HasMember("referenced-data-file")); + ASSERT_TRUE(properties.HasMember("cardinality")); + + std::string referenced_data_file = properties["referenced-data-file"].GetString(); + auto commit_it = commit_data_by_file.find(referenced_data_file); + ASSERT_NE(commit_data_by_file.end(), commit_it); + + const auto* commit_data = commit_it->second; + ASSERT_EQ(commit_data->content_offset, blob["offset"].GetInt64()); + ASSERT_EQ(commit_data->content_size_in_bytes, blob["length"].GetInt64()); + ASSERT_EQ(std::to_string(commit_data->row_count), properties["cardinality"].GetString()); + } + + std::filesystem::remove_all(temp_dir); +} + } // namespace doris diff --git a/be/test/format/table/iceberg/iceberg_delete_file_reader_helper_test.cpp b/be/test/format/table/iceberg/iceberg_delete_file_reader_helper_test.cpp new file mode 100644 index 00000000000000..0f3c3b619eb9f5 --- /dev/null +++ b/be/test/format/table/iceberg/iceberg_delete_file_reader_helper_test.cpp @@ -0,0 +1,44 @@ +// 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 "format/table/iceberg_delete_file_reader_helper.h" + +#include + +namespace doris { + +TEST(IcebergDeleteFileReaderHelperTest, BuildDeleteFileRange) { + auto range = build_iceberg_delete_file_range("s3://bucket/delete.parquet"); + EXPECT_EQ(range.path, "s3://bucket/delete.parquet"); + EXPECT_EQ(range.start_offset, 0); + EXPECT_EQ(range.size, -1); + EXPECT_EQ(range.file_size, -1); +} + +TEST(IcebergDeleteFileReaderHelperTest, IsDeletionVector) { + TIcebergDeleteFileDesc delete_file; + delete_file.__set_content(3); + delete_file.__isset.content = true; + EXPECT_TRUE(is_iceberg_deletion_vector(delete_file)); +} + +TEST(IcebergDeleteFileReaderHelperTest, IsNotDeletionVectorWhenContentMissing) { + TIcebergDeleteFileDesc delete_file; + EXPECT_FALSE(is_iceberg_deletion_vector(delete_file)); +} + +} // namespace doris diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run29.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run29.sql new file mode 100644 index 00000000000000..227fed1242b555 --- /dev/null +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run29.sql @@ -0,0 +1,391 @@ +create database if not exists demo.test_v2_to_v3_doris_spark_compare_db; +use demo.test_v2_to_v3_doris_spark_compare_db; + +drop table if exists v2v3_row_lineage_null_after_upgrade; +drop table if exists v2v3_spark_ops_reference; +drop table if exists v2v3_doris_ops_target; + +create table v2v3_row_lineage_null_after_upgrade ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'parquet', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_row_lineage_null_after_upgrade values +(1, 'base', 100, date '2024-01-01'), +(2, 'base', 200, date '2024-01-02'); + +insert into v2v3_row_lineage_null_after_upgrade values +(3, 'base', 300, date '2024-01-03'); + +update v2v3_row_lineage_null_after_upgrade +set tag = 'base_u', score = score + 10 +where id = 1; + +alter table v2v3_row_lineage_null_after_upgrade +set tblproperties ('format-version' = '3'); + +create table v2v3_spark_ops_reference ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'parquet', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_spark_ops_reference values +(1, 'base', 100, date '2024-02-01'), +(2, 'base', 200, date '2024-02-02'); + +insert into v2v3_spark_ops_reference values +(3, 'base', 300, date '2024-02-03'); + +update v2v3_spark_ops_reference +set tag = 'base_u', score = score + 10 +where id = 1; + +alter table v2v3_spark_ops_reference +set tblproperties ('format-version' = '3'); + +update v2v3_spark_ops_reference +set tag = 'post_v3_u', score = score + 20 +where id = 2; + +insert into v2v3_spark_ops_reference values +(4, 'post_v3_i', 400, date '2024-02-04'); + +call demo.system.rewrite_data_files( + table => 'demo.test_v2_to_v3_doris_spark_compare_db.v2v3_spark_ops_reference', + options => map('target-file-size-bytes', '10485760', 'min-input-files', '1') +); + +create table v2v3_doris_ops_target ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'parquet', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_doris_ops_target values +(1, 'base', 100, date '2024-02-01'), +(2, 'base', 200, date '2024-02-02'); + +insert into v2v3_doris_ops_target values +(3, 'base', 300, date '2024-02-03'); + +update v2v3_doris_ops_target +set tag = 'base_u', score = score + 10 +where id = 1; + +alter table v2v3_doris_ops_target +set tblproperties ('format-version' = '3'); + + +drop table if exists v2v3_doris_upd_case1; +drop table if exists v2v3_doris_upd_case2; +drop table if exists v2v3_doris_upd_case3; +drop table if exists v2v3_doris_upd_case4; + + +create table v2v3_doris_upd_case1 ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'parquet', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_doris_upd_case1 values (1, 'base', 100, date '2024-01-01'),(2, 'base', 200, date '2024-01-02'); +insert into v2v3_doris_upd_case1 values (3, 'base', 300, date '2024-01-03'); +update v2v3_doris_upd_case1 set tag = 'base_u', score = score + 10 where id = 1; +delete from v2v3_doris_upd_case1 where id = 2; +alter table v2v3_doris_upd_case1 set tblproperties ('format-version' = '3'); + + +create table v2v3_doris_upd_case2 ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'parquet', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_doris_upd_case2 values (1, 'base', 100, date '2024-01-01'),(2, 'base', 200, date '2024-01-02'); +insert into v2v3_doris_upd_case2 values (3, 'base', 300, date '2024-01-03'); +update v2v3_doris_upd_case2 set tag = 'base_u', score = score + 10 where id = 1; +delete from v2v3_doris_upd_case2 where id = 2; +alter table v2v3_doris_upd_case2 set tblproperties ('format-version' = '3'); + + +create table v2v3_doris_upd_case3 ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'parquet', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_doris_upd_case3 values (1, 'base', 100, date '2024-01-01'),(2, 'base', 200, date '2024-01-02'); +insert into v2v3_doris_upd_case3 values (3, 'base', 300, date '2024-01-03'); +update v2v3_doris_upd_case3 set tag = 'base_u', score = score + 10 where id = 1; +delete from v2v3_doris_upd_case3 where id = 2; +alter table v2v3_doris_upd_case3 set tblproperties ('format-version' = '3'); + + +create table v2v3_doris_upd_case4 ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'parquet', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_doris_upd_case4 values (1, 'base', 100, date '2024-01-01'),(2, 'base', 200, date '2024-01-02'); +insert into v2v3_doris_upd_case4 values (3, 'base', 300, date '2024-01-03'); +update v2v3_doris_upd_case4 set tag = 'base_u', score = score + 10 where id = 1; +delete from v2v3_doris_upd_case4 where id = 2; +alter table v2v3_doris_upd_case4 set tblproperties ('format-version' = '3'); + + +drop table if exists v2v3_row_lineage_null_after_upgrade_orc; +drop table if exists v2v3_spark_ops_reference_orc; +drop table if exists v2v3_doris_ops_target_orc; +drop table if exists v2v3_doris_upd_case1_orc; +drop table if exists v2v3_doris_upd_case2_orc; +drop table if exists v2v3_doris_upd_case3_orc; +drop table if exists v2v3_doris_upd_case4_orc; + +create table v2v3_row_lineage_null_after_upgrade_orc ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'orc', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_row_lineage_null_after_upgrade_orc values +(1, 'base', 100, date '2024-01-01'), +(2, 'base', 200, date '2024-01-02'); + +insert into v2v3_row_lineage_null_after_upgrade_orc values +(3, 'base', 300, date '2024-01-03'); + +update v2v3_row_lineage_null_after_upgrade_orc +set tag = 'base_u', score = score + 10 +where id = 1; + +alter table v2v3_row_lineage_null_after_upgrade_orc +set tblproperties ('format-version' = '3'); + +create table v2v3_spark_ops_reference_orc ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'orc', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_spark_ops_reference_orc values +(1, 'base', 100, date '2024-02-01'), +(2, 'base', 200, date '2024-02-02'); + +insert into v2v3_spark_ops_reference_orc values +(3, 'base', 300, date '2024-02-03'); + +update v2v3_spark_ops_reference_orc +set tag = 'base_u', score = score + 10 +where id = 1; + +alter table v2v3_spark_ops_reference_orc +set tblproperties ('format-version' = '3'); + +update v2v3_spark_ops_reference_orc +set tag = 'post_v3_u', score = score + 20 +where id = 2; + +insert into v2v3_spark_ops_reference_orc values +(4, 'post_v3_i', 400, date '2024-02-04'); + +call demo.system.rewrite_data_files( + table => 'demo.test_v2_to_v3_doris_spark_compare_db.v2v3_spark_ops_reference_orc', + options => map('target-file-size-bytes', '10485760', 'min-input-files', '1') +); + +create table v2v3_doris_ops_target_orc ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'orc', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_doris_ops_target_orc values +(1, 'base', 100, date '2024-02-01'), +(2, 'base', 200, date '2024-02-02'); + +insert into v2v3_doris_ops_target_orc values +(3, 'base', 300, date '2024-02-03'); + +update v2v3_doris_ops_target_orc +set tag = 'base_u', score = score + 10 +where id = 1; + +alter table v2v3_doris_ops_target_orc +set tblproperties ('format-version' = '3'); + +create table v2v3_doris_upd_case1_orc ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'orc', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_doris_upd_case1_orc values (1, 'base', 100, date '2024-01-01'),(2, 'base', 200, date '2024-01-02'); +insert into v2v3_doris_upd_case1_orc values (3, 'base', 300, date '2024-01-03'); +update v2v3_doris_upd_case1_orc set tag = 'base_u', score = score + 10 where id = 1; +delete from v2v3_doris_upd_case1_orc where id = 2; +alter table v2v3_doris_upd_case1_orc set tblproperties ('format-version' = '3'); + +create table v2v3_doris_upd_case2_orc ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'orc', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_doris_upd_case2_orc values (1, 'base', 100, date '2024-01-01'),(2, 'base', 200, date '2024-01-02'); +insert into v2v3_doris_upd_case2_orc values (3, 'base', 300, date '2024-01-03'); +update v2v3_doris_upd_case2_orc set tag = 'base_u', score = score + 10 where id = 1; +delete from v2v3_doris_upd_case2_orc where id = 2; +alter table v2v3_doris_upd_case2_orc set tblproperties ('format-version' = '3'); + +create table v2v3_doris_upd_case3_orc ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'orc', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_doris_upd_case3_orc values (1, 'base', 100, date '2024-01-01'),(2, 'base', 200, date '2024-01-02'); +insert into v2v3_doris_upd_case3_orc values (3, 'base', 300, date '2024-01-03'); +update v2v3_doris_upd_case3_orc set tag = 'base_u', score = score + 10 where id = 1; +delete from v2v3_doris_upd_case3_orc where id = 2; +alter table v2v3_doris_upd_case3_orc set tblproperties ('format-version' = '3'); + +create table v2v3_doris_upd_case4_orc ( + id int, + tag string, + score int, + dt date +) using iceberg +partitioned by (days(dt)) +tblproperties ( + 'format-version' = '2', + 'write.format.default' = 'orc', + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' +); + +insert into v2v3_doris_upd_case4_orc values (1, 'base', 100, date '2024-01-01'),(2, 'base', 200, date '2024-01-02'); +insert into v2v3_doris_upd_case4_orc values (3, 'base', 300, date '2024-01-03'); +update v2v3_doris_upd_case4_orc set tag = 'base_u', score = score + 10 where id = 1; +delete from v2v3_doris_upd_case4_orc where id = 2; +alter table v2v3_doris_upd_case4_orc set tblproperties ('format-version' = '3'); 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 fd0ff62a80438d..8efefd94c7649b 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 @@ -253,7 +253,7 @@ private void setColumnPositionMapping() } // Pre-index columns into a Map for O(1) lookup - List columns = getColumns(); + List columns = desc.getTable().getFullSchema(); Map columnNameMap = new HashMap<>(columns.size()); for (int i = 0; i < columns.size(); i++) { columnNameMap.putIfAbsent(columns.get(i).getName(), i); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileScanNode.java index ca64c7457e6d33..82504fc6db3439 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileScanNode.java @@ -257,7 +257,7 @@ protected void setDefaultValueExprs(TableIf tbl, nameToSlotDesc.put(slot.getColumn().getName(), slot); } - for (Column column : getColumns()) { + for (Column column : desc.getTable().getFullSchema()) { Expr expr; Expression expression; if (column.getDefaultValue() != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java index eb00d82adc7a5f..ffdec64d3d93bf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java @@ -279,14 +279,13 @@ protected boolean needInternalHiddenColumns() { @Override public List getFullSchema() { List schema = IcebergUtils.getIcebergSchema(this); + schema = new ArrayList<>(schema); - // 添加隐藏列: __DORIS_ICEBERG_ROWID_COL__ - // 只有在 Util.showHiddenColumns() 或内部需要时返回 if (Util.showHiddenColumns() || needInternalHiddenColumns()) { - schema = new ArrayList<>(schema); schema.add(createIcebergRowIdColumn()); } + schema = IcebergUtils.appendRowLineageColumnsForV3(schema, getIcebergTable()); return schema; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java index dbc7e523c40628..1325df321c37ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java @@ -52,6 +52,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -60,6 +61,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -80,6 +82,7 @@ public class IcebergTransaction implements Transaction { private IcebergInsertCommandContext insertCtx; private String branchName; private Long baseSnapshotId; + private Map> rewrittenDeleteFilesByReferencedDataFile = Collections.emptyMap(); // Rewrite operation support long startingSnapshotId = -1L; // Track the starting snapshot ID for rewrite operations @@ -105,6 +108,14 @@ public void clearConflictDetectionFilter() { conflictDetectionFilter = Optional.empty(); } + public void setRewrittenDeleteFilesByReferencedDataFile( + Map> rewrittenDeleteFilesByReferencedDataFile) { + this.rewrittenDeleteFilesByReferencedDataFile = + rewrittenDeleteFilesByReferencedDataFile == null + ? Collections.emptyMap() + : rewrittenDeleteFilesByReferencedDataFile; + } + public List getCommitDataList() { return commitDataList; } @@ -135,6 +146,7 @@ public void beginInsert(ExternalTable dorisTable, Optional } } this.transaction = table.newTransaction(); + this.rewrittenDeleteFilesByReferencedDataFile = Collections.emptyMap(); }); } catch (Exception e) { throw new UserException("Failed to begin insert for iceberg table " + dorisTable.getName() @@ -268,6 +280,7 @@ public void beginDelete(ExternalTable dorisTable) throws UserException { } } this.transaction = table.newTransaction(); + this.rewrittenDeleteFilesByReferencedDataFile = Collections.emptyMap(); LOG.info("Started delete transaction for table: {}", dorisTable.getName()); }); } catch (Exception e) { @@ -294,6 +307,7 @@ public void beginMerge(ExternalTable dorisTable) throws UserException { } } this.transaction = table.newTransaction(); + this.rewrittenDeleteFilesByReferencedDataFile = Collections.emptyMap(); LOG.info("Started merge transaction for table: {}", dorisTable.getName()); return null; }); @@ -348,6 +362,9 @@ private void updateManifestAfterDelete() { return; } List deleteFiles = convertCommitDataToDeleteFiles(fileFormat, commitDataList); + List rewrittenDeleteFiles = shouldRewritePreviousDeleteFiles() + ? collectRewrittenDeleteFiles(commitDataList) + : Collections.emptyList(); if (deleteFiles.isEmpty()) { LOG.info("No delete files generated from commit data"); @@ -365,10 +382,15 @@ private void updateManifestAfterDelete() { rowDelta.addDeletes(deleteFile); } + for (DeleteFile deleteFile : rewrittenDeleteFiles) { + rowDelta.removeDeletes(deleteFile); + } + // Commit the delete operation rowDelta.commit(); - LOG.info("Committed {} delete files", deleteFiles.size()); + LOG.info("Committed {} delete files and removed {} previous delete files", + deleteFiles.size(), rewrittenDeleteFiles.size()); } private List convertCommitDataToDeleteFiles(FileFormat fileFormat, @@ -425,7 +447,8 @@ private void updateManifestAfterMerge() { for (TIcebergCommitData commitData : commitDataList) { if (commitData.isSetFileContent() - && commitData.getFileContent() == TFileContent.POSITION_DELETES) { + && (commitData.getFileContent() == TFileContent.POSITION_DELETES + || commitData.getFileContent() == TFileContent.DELETION_VECTOR)) { deleteCommitData.add(commitData); } else { dataCommitData.add(commitData); @@ -440,6 +463,9 @@ private void updateManifestAfterMerge() { } List deleteFiles = convertCommitDataToDeleteFiles(fileFormat, deleteCommitData); + List rewrittenDeleteFiles = shouldRewritePreviousDeleteFiles() + ? collectRewrittenDeleteFiles(deleteCommitData) + : Collections.emptyList(); if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { LOG.info("No data or delete files generated from commit data"); @@ -457,10 +483,13 @@ private void updateManifestAfterMerge() { for (DeleteFile deleteFile : deleteFiles) { rowDelta.addDeletes(deleteFile); } + for (DeleteFile deleteFile : rewrittenDeleteFiles) { + rowDelta.removeDeletes(deleteFile); + } rowDelta.commit(); - LOG.info("Committed merge with {} data files and {} delete files", - dataFiles.size(), deleteFiles.size()); + LOG.info("Committed merge with {} data files, {} delete files and removed {} previous delete files", + dataFiles.size(), deleteFiles.size(), rewrittenDeleteFiles.size()); } public void finishInsert(NameMapping nameMapping) { @@ -534,11 +563,15 @@ public long getUpdateCnt() { long dataRows = 0; long deleteRows = 0; for (TIcebergCommitData commitData : commitDataList) { + long affectedRows = commitData.isSetAffectedRows() + ? commitData.getAffectedRows() + : commitData.getRowCount(); if (commitData.isSetFileContent() - && commitData.getFileContent() == TFileContent.POSITION_DELETES) { - deleteRows += commitData.getRowCount(); + && (commitData.getFileContent() == TFileContent.POSITION_DELETES + || commitData.getFileContent() == TFileContent.DELETION_VECTOR)) { + deleteRows += affectedRows; } else { - dataRows += commitData.getRowCount(); + dataRows += affectedRows; } } // For UPDATE/MERGE, dataRows includes both inserted and update-inserted rows, @@ -735,6 +768,45 @@ private boolean isSerializableIsolationLevel(Table icebergTable) { return "serializable".equalsIgnoreCase(level); } + private boolean shouldRewritePreviousDeleteFiles() { + return table != null && IcebergUtils.getFormatVersion(table) >= 3; + } + + private List collectRewrittenDeleteFiles(List deleteCommitData) { + if (deleteCommitData == null || deleteCommitData.isEmpty() + || rewrittenDeleteFilesByReferencedDataFile.isEmpty()) { + return Collections.emptyList(); + } + + Map dedup = new LinkedHashMap<>(); + for (TIcebergCommitData commitData : deleteCommitData) { + if (!commitData.isSetReferencedDataFilePath() + || commitData.getReferencedDataFilePath() == null + || commitData.getReferencedDataFilePath().isEmpty()) { + continue; + } + List oldDeleteFiles = + rewrittenDeleteFilesByReferencedDataFile.get(commitData.getReferencedDataFilePath()); + if (oldDeleteFiles == null) { + continue; + } + for (DeleteFile deleteFile : oldDeleteFiles) { + if (deleteFile != null && ContentFileUtil.isFileScoped(deleteFile)) { + dedup.putIfAbsent(buildDeleteFileDedupKey(deleteFile), deleteFile); + } + } + } + return new ArrayList<>(dedup.values()); + } + + private String buildDeleteFileDedupKey(DeleteFile deleteFile) { + if (deleteFile.format() == FileFormat.PUFFIN) { + return deleteFile.path() + "#" + deleteFile.contentOffset() + "#" + + deleteFile.contentSizeInBytes(); + } + return deleteFile.path().toString(); + } + private List collectReferencedDataFiles(List commitDataList) { if (commitDataList == null || commitDataList.isEmpty()) { return Collections.emptyList(); @@ -743,7 +815,8 @@ private List collectReferencedDataFiles(List commitD List referencedDataFiles = new ArrayList<>(); for (TIcebergCommitData commitData : commitDataList) { if (commitData.isSetFileContent() - && commitData.getFileContent() != TFileContent.POSITION_DELETES) { + && commitData.getFileContent() != TFileContent.POSITION_DELETES + && commitData.getFileContent() != TFileContent.DELETION_VECTOR) { continue; } if (commitData.isSetReferencedDataFiles()) { 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 dea5859d3bd81f..f7b4dfb723cc62 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 @@ -77,10 +77,12 @@ import com.google.gson.reflect.TypeToken; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.iceberg.BaseTable; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.MetadataTableUtils; import org.apache.iceberg.PartitionData; @@ -105,6 +107,7 @@ import org.apache.iceberg.hive.HiveCatalog; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.TimestampType; @@ -179,6 +182,10 @@ public Integer initialValue() { public static final String IDENTITY = "identity"; public static final int PARTITION_DATA_ID_START = 1000; // org.apache.iceberg.PartitionSpec + public static final int ICEBERG_ROW_LINEAGE_MIN_VERSION = 3; + public static final String ICEBERG_ROW_ID_COL = "_row_id"; + public static final String ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL = "_last_updated_sequence_number"; + private static final Pattern SNAPSHOT_ID = Pattern.compile("\\d+"); public static Expression convertToIcebergExpr(Expr expr, Schema schema) { @@ -252,7 +259,10 @@ public static Expression convertToIcebergExpr(Expr expr, Schema schema) { return null; } String colName = slotRef.getColumnName(); - Types.NestedField nestedField = schema.caseInsensitiveFindField(colName); + Types.NestedField nestedField = getPushdownField(schema, colName); + if (nestedField == null) { + return null; + } colName = nestedField.name(); Object value = extractDorisLiteral(nestedField.type(), literalExpr); if (value == null) { @@ -298,7 +308,10 @@ public static Expression convertToIcebergExpr(Expr expr, Schema schema) { return null; } String colName = slotRef.getColumnName(); - Types.NestedField nestedField = schema.caseInsensitiveFindField(colName); + Types.NestedField nestedField = getPushdownField(schema, colName); + if (nestedField == null) { + return null; + } colName = nestedField.name(); List valueList = new ArrayList<>(); for (int i = 1; i < inExpr.getChildren().size(); ++i) { @@ -324,6 +337,14 @@ public static Expression convertToIcebergExpr(Expr expr, Schema schema) { return checkConversion(expression, schema); } + private static Types.NestedField getPushdownField(Schema schema, String colName) { + if (ICEBERG_ROW_ID_COL.equalsIgnoreCase(colName) + || ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL.equalsIgnoreCase(colName)) { + return null; + } + return schema.caseInsensitiveFindField(colName); + } + private static Expression checkConversion(Expression expression, Schema schema) { if (expression == null) { return null; @@ -1631,6 +1652,59 @@ private static Optional loadTableSchemaCacheValue(ExternalTabl return Optional.of(new IcebergSchemaCacheValue(schema, tmpColumns)); } + + public static boolean isIcebergRowLineageColumn(Column column) { + return column.nameEquals(IcebergUtils.ICEBERG_ROW_ID_COL, false) + || column.nameEquals(IcebergUtils.ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL, false); + } + + public static boolean isIcebergRowLineageColumn(String columnName) { + return IcebergUtils.ICEBERG_ROW_ID_COL.equalsIgnoreCase(columnName) + || IcebergUtils.ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL.equalsIgnoreCase(columnName); + } + + public static List appendRowLineageColumnsForV3(List schema, Table table) { + if (getFormatVersion(table) < ICEBERG_ROW_LINEAGE_MIN_VERSION) { + return schema; + } + List newSchema = Lists.newArrayList(schema); + + Column rowIdColumn = new Column(ICEBERG_ROW_ID_COL, Type.BIGINT, true); + rowIdColumn.setUniqueId(2147483540); + rowIdColumn.setIsVisible(false); + newSchema.add(rowIdColumn); + + Column lastUpdatedSequenceNumberColumn = + new Column(ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL, Type.BIGINT, true); + lastUpdatedSequenceNumberColumn.setUniqueId(2147483539); + lastUpdatedSequenceNumberColumn.setIsVisible(false); + newSchema.add(lastUpdatedSequenceNumberColumn); + + return newSchema; + } + + public static Schema appendRowLineageFieldsForV3(Schema schema) { + return TypeUtil.join(schema, new Schema( + MetadataColumns.ROW_ID, MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER)); + } + + public static int getFormatVersion(Table table) { + int formatVersion = 2; // default format version : 2 + if (table instanceof BaseTable) { + formatVersion = ((BaseTable) table).operations().current().formatVersion(); + } else if (table != null && table.properties() != null) { + String version = table.properties().get(TableProperties.FORMAT_VERSION); + if (version != null) { + try { + formatVersion = Integer.parseInt(version); + } catch (NumberFormatException ignored) { + // keep default value + } + } + } + return formatVersion; + } + public static String showCreateView(IcebergExternalTable icebergExternalTable) { return String.format("CREATE VIEW `%s` AS ", icebergExternalTable.getName()) + diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/helper/IcebergRewritableDeletePlan.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/helper/IcebergRewritableDeletePlan.java new file mode 100644 index 00000000000000..92aec74da11dcb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/helper/IcebergRewritableDeletePlan.java @@ -0,0 +1,54 @@ +// 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.iceberg.helper; + +import org.apache.doris.thrift.TIcebergRewritableDeleteFileSet; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.iceberg.DeleteFile; + +import java.util.List; +import java.util.Map; + +public final class IcebergRewritableDeletePlan { + private static final IcebergRewritableDeletePlan EMPTY = + new IcebergRewritableDeletePlan(ImmutableList.of(), ImmutableMap.of()); + + private final List thriftDeleteFileSets; + private final Map> deleteFilesByReferencedDataFile; + + public IcebergRewritableDeletePlan( + List thriftDeleteFileSets, + Map> deleteFilesByReferencedDataFile) { + this.thriftDeleteFileSets = thriftDeleteFileSets; + this.deleteFilesByReferencedDataFile = deleteFilesByReferencedDataFile; + } + + public static IcebergRewritableDeletePlan empty() { + return EMPTY; + } + + public List getThriftDeleteFileSets() { + return thriftDeleteFileSets; + } + + public Map> getDeleteFilesByReferencedDataFile() { + return deleteFilesByReferencedDataFile; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/helper/IcebergRewritableDeletePlanner.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/helper/IcebergRewritableDeletePlanner.java new file mode 100644 index 00000000000000..b1a5867c1ceb9b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/helper/IcebergRewritableDeletePlanner.java @@ -0,0 +1,87 @@ +// 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.iceberg.helper; + +import org.apache.doris.common.UserException; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.datasource.iceberg.IcebergUtils; +import org.apache.doris.datasource.iceberg.source.IcebergScanNode; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.thrift.TIcebergRewritableDeleteFileSet; + +import org.apache.iceberg.DeleteFile; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +public final class IcebergRewritableDeletePlanner { + private static final int ICEBERG_DELETION_VECTOR_MIN_VERSION = 3; + + private IcebergRewritableDeletePlanner() { + } + + public static IcebergRewritableDeletePlan collectForDelete( + IcebergExternalTable table, NereidsPlanner planner) throws UserException { + return collect(table, planner); + } + + public static IcebergRewritableDeletePlan collectForMerge( + IcebergExternalTable table, NereidsPlanner planner) throws UserException { + return collect(table, planner); + } + + private static IcebergRewritableDeletePlan collect( + IcebergExternalTable table, NereidsPlanner planner) { + if (table == null + || planner == null + || IcebergUtils.getFormatVersion(table.getIcebergTable()) < ICEBERG_DELETION_VECTOR_MIN_VERSION) { + return IcebergRewritableDeletePlan.empty(); + } + + List thriftDeleteFileSets = new ArrayList<>(); + Map> deleteFilesByReferencedDataFile = new LinkedHashMap<>(); + + for (ScanNode scanNode : planner.getScanNodes()) { + if (!(scanNode instanceof IcebergScanNode)) { + continue; + } + IcebergScanNode icebergScanNode = (IcebergScanNode) scanNode; + + deleteFilesByReferencedDataFile.putAll(icebergScanNode.deleteFilesByReferencedDataFile); + icebergScanNode.deleteFilesDescByReferencedDataFile.forEach( + (key, value) -> { + TIcebergRewritableDeleteFileSet deleteFileSet = new TIcebergRewritableDeleteFileSet(); + deleteFileSet.setReferencedDataFilePath(key); + deleteFileSet.setDeleteFiles(value); + thriftDeleteFileSets.add(deleteFileSet); + } + ); + } + + if (thriftDeleteFileSets.isEmpty()) { + return IcebergRewritableDeletePlan.empty(); + } + return new IcebergRewritableDeletePlan( + Collections.unmodifiableList(thriftDeleteFileSets), + Collections.unmodifiableMap(deleteFilesByReferencedDataFile)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/helper/IcebergWriterHelper.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/helper/IcebergWriterHelper.java index 41537f6ca76cf5..b67a5911b64384 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/helper/IcebergWriterHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/helper/IcebergWriterHelper.java @@ -206,22 +206,38 @@ public static List convertToDeleteFiles( String deleteFilePath = commitData.getFilePath(); long fileSize = commitData.getFileSize(); long recordCount = commitData.getRowCount(); + boolean isDeletionVector = commitData.isSetContentOffset() + && commitData.isSetContentSizeInBytes(); + FileFormat effectiveFormat = isDeletionVector ? FileFormat.PUFFIN : format; // Build delete file metadata FileMetadata.Builder deleteBuilder = FileMetadata.deleteFileBuilder(spec) .withPath(deleteFilePath) - .withFormat(format) + .withFormat(effectiveFormat) .withFileSizeInBytes(fileSize) .withRecordCount(recordCount); // Set delete file content type if (commitData.getFileContent() == TFileContent.POSITION_DELETES) { deleteBuilder.ofPositionDeletes(); + } else if (commitData.getFileContent() == TFileContent.DELETION_VECTOR) { + deleteBuilder.ofPositionDeletes(); } else { throw new VerifyException("Iceberg delete only supports position deletes, but got " + commitData.getFileContent()); } + if (isDeletionVector) { + deleteBuilder.withContentOffset(commitData.getContentOffset()); + deleteBuilder.withContentSizeInBytes(commitData.getContentSizeInBytes()); + } + + if (commitData.isSetReferencedDataFilePath() + && commitData.getReferencedDataFilePath() != null + && !commitData.getReferencedDataFilePath().isEmpty()) { + deleteBuilder.withReferencedDataFile(commitData.getReferencedDataFilePath()); + } + // Add partition information if table is partitioned if (spec.isPartitioned()) { PartitionData partitionData; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteGroupTask.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteGroupTask.java index 13b7ee3ff680ce..eee1a7eb60256b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteGroupTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteGroupTask.java @@ -216,8 +216,7 @@ private RewriteTableCommand buildRewriteLogicalPlan() { DMLCommandType.INSERT, Optional.empty(), // labelName Optional.empty(), // branchName - sourceRelation); - + sourceRelation, true); // Create RewriteTableCommand for rewrite operation return new RewriteTableCommand( tableSink, 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 1060b3558181a9..e8763a09ca0761 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 @@ -69,9 +69,11 @@ import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.iceberg.BaseFileScanTask; import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.DeleteFileIndex; import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; @@ -158,6 +160,10 @@ public class IcebergScanNode extends FileQueryScanNode { private Boolean isBatchMode = null; + // ReferencedDataFile path -> List / List (exclude equal delete) + public Map> deleteFilesByReferencedDataFile = new HashMap<>(); + public Map> deleteFilesDescByReferencedDataFile = new HashMap<>(); + // for test @VisibleForTesting public IcebergScanNode(PlanNodeId id, TupleDescriptor desc, SessionVariable sv, ScanContext scanContext) { @@ -280,14 +286,20 @@ private void setIcebergParams(TFileRangeDesc rangeDesc, IcebergSplit icebergSpli if (icebergSplit.getPartitionDataJson() != null) { fileDesc.setPartitionDataJson(icebergSplit.getPartitionDataJson()); } + if (formatVersion >= 3) { + fileDesc.setFirstRowId(icebergSplit.getFirstRowId()); + fileDesc.setLastUpdatedSequenceNumber(icebergSplit.getLastUpdatedSequenceNumber()); + } if (formatVersion < MIN_DELETE_FILE_SUPPORT_VERSION) { fileDesc.setContent(FileContent.DATA.id()); } else { + fileDesc.setDeleteFiles(new ArrayList<>()); for (IcebergDeleteFileFilter filter : icebergSplit.getDeleteFileFilters()) { TIcebergDeleteFileDesc deleteFileDesc = new TIcebergDeleteFileDesc(); String deleteFilePath = filter.getDeleteFilePath(); LocationPath locationPath = LocationPath.of(deleteFilePath, icebergSplit.getConfig()); deleteFileDesc.setPath(locationPath.toStorageLocation().toString()); + setDeleteFileFormat(deleteFileDesc, filter.getFileformat()); if (filter instanceof IcebergDeleteFileFilter.PositionDelete) { IcebergDeleteFileFilter.PositionDelete positionDelete = (IcebergDeleteFileFilter.PositionDelete) filter; @@ -303,8 +315,8 @@ private void setIcebergParams(TFileRangeDesc rangeDesc, IcebergSplit icebergSpli if (filter instanceof IcebergDeleteFileFilter.DeletionVector) { IcebergDeleteFileFilter.DeletionVector dv = (IcebergDeleteFileFilter.DeletionVector) filter; - deleteFileDesc.setContentOffset((int) dv.getContentOffset()); - deleteFileDesc.setContentSizeInBytes((int) dv.getContentLength()); + deleteFileDesc.setContentOffset(dv.getContentOffset()); + deleteFileDesc.setContentSizeInBytes(dv.getContentLength()); deleteFileDesc.setContent(IcebergDeleteFileFilter.DeletionVector.type()); } } else { @@ -315,6 +327,22 @@ private void setIcebergParams(TFileRangeDesc rangeDesc, IcebergSplit icebergSpli } fileDesc.addToDeleteFiles(deleteFileDesc); } + + // Filter out equality delete files from deleteFilesByReferencedDataFile as well. + List nonEqualityDeleteFiles = new ArrayList<>(); + for (DeleteFile df : icebergSplit.getDeleteFiles()) { + if (df.content() != FileContent.EQUALITY_DELETES) { + nonEqualityDeleteFiles.add(df); + } + } + deleteFilesByReferencedDataFile.put(icebergSplit.getOriginalPath(), nonEqualityDeleteFiles); + List nonEqualityDeleteFileDesc = new ArrayList<>(); + for (TIcebergDeleteFileDesc df : fileDesc.getDeleteFiles()) { + if (df.getContent() != EqualityDelete.type()) { + nonEqualityDeleteFileDesc.add(df); + } + } + deleteFilesDescByReferencedDataFile.put(icebergSplit.getOriginalPath(), nonEqualityDeleteFileDesc); } tableFormatFileDesc.setIcebergParams(fileDesc); Map partitionValues = icebergSplit.getIcebergPartitionValues(); @@ -360,6 +388,14 @@ protected List getDeleteFiles(TFileRangeDesc rangeDesc) { return deleteFiles; } + private void setDeleteFileFormat(TIcebergDeleteFileDesc deleteFileDesc, FileFormat fileFormat) { + if (fileFormat == FileFormat.PARQUET) { + deleteFileDesc.setFileFormat(TFileFormatType.FORMAT_PARQUET); + } else if (fileFormat == FileFormat.ORC) { + deleteFileDesc.setFileFormat(TFileFormatType.FORMAT_ORC); + } + } + private String getDeleteFileContentType(int content) { // Iceberg file type: 0: data, 1: position delete, 2: equality delete, 3: deletion vector switch (content) { @@ -764,20 +800,29 @@ private LocationPath createLocationPathWithCache(String path) { } private Split createIcebergSplit(FileScanTask fileScanTask) { - String originalPath = fileScanTask.file().path().toString(); + DataFile dataFile = fileScanTask.file(); + String originalPath = dataFile.path().toString(); LocationPath locationPath = createLocationPathWithCache(originalPath); IcebergSplit split = new IcebergSplit( locationPath, fileScanTask.start(), fileScanTask.length(), - fileScanTask.file().fileSizeInBytes(), + dataFile.fileSizeInBytes(), new String[0], formatVersion, storagePropertiesMap, new ArrayList<>(), originalPath); + if (formatVersion >= 3) { + // -1 means that this table was just upgraded from v2 to v3. + // _row_id and _last_updated_sequence_number column is NULL. + split.setFirstRowId(dataFile.firstRowId() != null ? dataFile.firstRowId() : -1); + split.setLastUpdatedSequenceNumber( + dataFile.fileSequenceNumber() != null && dataFile.firstRowId() != null + ? dataFile.fileSequenceNumber() : -1); + } if (!fileScanTask.deletes().isEmpty()) { - split.setDeleteFileFilters(getDeleteFileFilters(fileScanTask)); + split.setDeleteFileFilters(fileScanTask.deletes(), getDeleteFileFilters(fileScanTask)); } split.setTableFormatType(TableFormatType.ICEBERG); split.setTargetSplitSize(targetSplitSize); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergSplit.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergSplit.java index 345bbf58c8c2fc..2596df5ad372b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergSplit.java @@ -22,6 +22,7 @@ import org.apache.doris.datasource.property.storage.StorageProperties; import lombok.Data; +import org.apache.iceberg.DeleteFile; import java.util.ArrayList; import java.util.List; @@ -37,6 +38,7 @@ public class IcebergSplit extends FileSplit { // but the original datafile path must be used. private final String originalPath; private Integer formatVersion; + private List deleteFiles = new ArrayList<>(); private List deleteFileFilters = new ArrayList<>(); private Map config; // tableLevelRowCount will be set only table-level count push down opt is available. @@ -45,6 +47,8 @@ public class IcebergSplit extends FileSplit { private Map icebergPartitionValues = null; private Integer partitionSpecId = null; private String partitionDataJson = null; + private Long firstRowId = null; + private Long lastUpdatedSequenceNumber = null; // File path will be changed if the file is modified, so there's no need to get modification time. public IcebergSplit(LocationPath file, long start, long length, long fileLength, String[] hosts, @@ -57,7 +61,8 @@ public IcebergSplit(LocationPath file, long start, long length, long fileLength, this.selfSplitWeight = length; } - public void setDeleteFileFilters(List deleteFileFilters) { + public void setDeleteFileFilters(List deleteFiles, List deleteFileFilters) { + this.deleteFiles = deleteFiles; this.deleteFileFilters = deleteFileFilters; this.selfSplitWeight += deleteFileFilters.stream().mapToLong(IcebergDeleteFileFilter::getFilesize).sum(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundIcebergTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundIcebergTableSink.java index 765b56de1b9798..213baccafb2688 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundIcebergTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundIcebergTableSink.java @@ -37,14 +37,21 @@ * Represent an iceberg table sink plan node that has not been bound. */ public class UnboundIcebergTableSink extends UnboundBaseExternalTableSink { + private boolean rewrite = false; + // Static partition key-value pairs for INSERT OVERWRITE ... PARTITION // (col='val', ...) private final Map staticPartitionKeyValues; public UnboundIcebergTableSink(List nameParts, List colNames, List hints, List partitions, CHILD_TYPE child) { + this(nameParts, colNames, hints, partitions, child, false); + } + + public UnboundIcebergTableSink(List nameParts, List colNames, List hints, + List partitions, CHILD_TYPE child, boolean rewrite) { this(nameParts, colNames, hints, partitions, DMLCommandType.NONE, - Optional.empty(), Optional.empty(), child, null); + Optional.empty(), Optional.empty(), child, null, rewrite); } /** @@ -59,7 +66,22 @@ public UnboundIcebergTableSink(List nameParts, Optional logicalProperties, CHILD_TYPE child) { this(nameParts, colNames, hints, partitions, dmlCommandType, - groupExpression, logicalProperties, child, null); + groupExpression, logicalProperties, child, false); + } + + /** + * constructor + */ + public UnboundIcebergTableSink(List nameParts, + List colNames, + List hints, + List partitions, + DMLCommandType dmlCommandType, + Optional groupExpression, + Optional logicalProperties, + CHILD_TYPE child, boolean rewrite) { + this(nameParts, colNames, hints, partitions, dmlCommandType, + groupExpression, logicalProperties, child, null, rewrite); } /** @@ -73,12 +95,14 @@ public UnboundIcebergTableSink(List nameParts, Optional groupExpression, Optional logicalProperties, CHILD_TYPE child, - Map staticPartitionKeyValues) { + Map staticPartitionKeyValues, + boolean rewrite) { super(nameParts, PlanType.LOGICAL_UNBOUND_ICEBERG_TABLE_SINK, ImmutableList.of(), groupExpression, logicalProperties, colNames, dmlCommandType, child, hints, partitions); this.staticPartitionKeyValues = staticPartitionKeyValues != null ? ImmutableMap.copyOf(staticPartitionKeyValues) : null; + this.rewrite = rewrite; } public Map getStaticPartitionKeyValues() { @@ -94,7 +118,7 @@ public Plan withChildren(List children) { Preconditions.checkArgument(children.size() == 1, "UnboundIcebergTableSink only accepts one child"); return new UnboundIcebergTableSink<>(nameParts, colNames, hints, partitions, - dmlCommandType, groupExpression, Optional.empty(), children.get(0), staticPartitionKeyValues); + dmlCommandType, groupExpression, Optional.empty(), children.get(0), staticPartitionKeyValues, rewrite); } @Override @@ -106,13 +130,17 @@ public R accept(PlanVisitor visitor, C context) { public Plan withGroupExpression(Optional groupExpression) { return new UnboundIcebergTableSink<>(nameParts, colNames, hints, partitions, dmlCommandType, groupExpression, Optional.of(getLogicalProperties()), child(), - staticPartitionKeyValues); + staticPartitionKeyValues, rewrite); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { return new UnboundIcebergTableSink<>(nameParts, colNames, hints, partitions, - dmlCommandType, groupExpression, logicalProperties, children.get(0), staticPartitionKeyValues); + dmlCommandType, groupExpression, logicalProperties, children.get(0), staticPartitionKeyValues, rewrite); + } + + public boolean isRewrite() { + return rewrite; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSinkCreator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSinkCreator.java index a1411cf885347d..28651f86299653 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSinkCreator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSinkCreator.java @@ -98,7 +98,7 @@ public static LogicalSink createUnboundTableSink(List na dmlCommandType, Optional.empty(), Optional.empty(), plan); } else if (curCatalog instanceof IcebergExternalCatalog) { return new UnboundIcebergTableSink<>(nameParts, colNames, hints, partitions, - dmlCommandType, Optional.empty(), Optional.empty(), plan, staticPartitionKeyValues); + dmlCommandType, Optional.empty(), Optional.empty(), plan, staticPartitionKeyValues, false); } else if (curCatalog instanceof JdbcExternalCatalog) { return new UnboundJdbcTableSink<>(nameParts, colNames, hints, partitions, dmlCommandType, Optional.empty(), Optional.empty(), plan); @@ -139,7 +139,7 @@ public static LogicalSink createUnboundTableSinkMaybeOverwrite(L dmlCommandType, Optional.empty(), Optional.empty(), plan); } else if (curCatalog instanceof IcebergExternalCatalog && !isAutoDetectPartition) { return new UnboundIcebergTableSink<>(nameParts, colNames, hints, partitions, - dmlCommandType, Optional.empty(), Optional.empty(), plan, staticPartitionKeyValues); + dmlCommandType, Optional.empty(), Optional.empty(), plan, staticPartitionKeyValues, false); } else if (curCatalog instanceof JdbcExternalCatalog) { return new UnboundJdbcTableSink<>(nameParts, colNames, hints, partitions, dmlCommandType, Optional.empty(), Optional.empty(), plan); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index 39150d69f64493..35c4516fabb65a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.FunctionRegistry; import org.apache.doris.common.Pair; import org.apache.doris.datasource.iceberg.IcebergMergeOperation; +import org.apache.doris.datasource.iceberg.IcebergUtils; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.SqlCacheContext; import org.apache.doris.nereids.StatementContext; @@ -351,7 +352,10 @@ private boolean isIcebergMergeMetaColumn(String name) { if (IcebergMergeOperation.OPERATION_COLUMN.equalsIgnoreCase(name)) { return true; } - return Column.ICEBERG_ROWID_COL.equalsIgnoreCase(name); + if (Column.ICEBERG_ROWID_COL.equalsIgnoreCase(name)) { + return true; + } + return IcebergUtils.isIcebergRowLineageColumn(name); } private static boolean hasUnboundPlan(Plan plan) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java index e154004aff37ac..fd501a53c6892b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java @@ -35,6 +35,7 @@ import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.iceberg.IcebergExternalDatabase; import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.datasource.iceberg.IcebergUtils; import org.apache.doris.datasource.jdbc.JdbcExternalDatabase; import org.apache.doris.datasource.jdbc.JdbcExternalTable; import org.apache.doris.datasource.maxcompute.MaxComputeExternalDatabase; @@ -711,9 +712,17 @@ private Plan bindIcebergTableSink(MatchingContext> List bindColumns; if (sink.getColNames().isEmpty()) { // When no column names specified, include all non-static-partition columns - bindColumns = table.getBaseSchema(true).stream() - .filter(col -> !staticPartitionColNames.contains(col.getName())) - .collect(ImmutableList.toImmutableList()); + if (sink.isRewrite()) { + bindColumns = table.getBaseSchema(true).stream() + .filter(col -> !staticPartitionColNames.contains(col.getName())) + .filter(col -> col.isVisible() || IcebergUtils.isIcebergRowLineageColumn(col)) + .collect(ImmutableList.toImmutableList()); + } else { + bindColumns = table.getBaseSchema(true).stream() + .filter(col -> !staticPartitionColNames.contains(col.getName())) + .filter(Column::isVisible) + .collect(ImmutableList.toImmutableList()); + } } else { bindColumns = sink.getColNames().stream().map(cn -> { Column column = table.getColumn(cn); @@ -721,6 +730,10 @@ private Plan bindIcebergTableSink(MatchingContext> throw new AnalysisException(String.format("column %s is not found in table %s", cn, table.getName())); } + if (IcebergUtils.isIcebergRowLineageColumn(column)) { + throw new AnalysisException(String.format( + "Cannot specify row lineage column '%s' in INSERT statement", cn)); + } return column; }).collect(ImmutableList.toImmutableList()); } @@ -763,7 +776,13 @@ private Plan bindIcebergTableSink(MatchingContext> } } - LogicalProject fullOutputProject = getOutputProjectByCoercion(table.getFullSchema(), child, columnToOutput); + List insertSchema = table.getFullSchema(); + if (!sink.isRewrite()) { + insertSchema = insertSchema.stream() + .filter(Column::isVisible) + .collect(Collectors.toList()); + } + LogicalProject fullOutputProject = getOutputProjectByCoercion(insertSchema, child, columnToOutput); return boundSink.withChildAndUpdateOutput(fullOutputProject); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergDeleteCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergDeleteCommand.java index f7da7f5dc4f1dc..ce48e03be9975c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergDeleteCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergDeleteCommand.java @@ -53,10 +53,12 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import java.util.List; import java.util.Optional; +import java.util.concurrent.Callable; /** * DELETE command for Iceberg tables. @@ -125,53 +127,71 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { long previousTargetTableId = ctx.getIcebergRowIdTargetTableId(); ctx.setIcebergRowIdTargetTableId(icebergTable.getId()); try { - // Build query plan with DELETE sink LogicalPlan deleteQueryPlan = completeQueryPlan(ctx, logicalQuery, icebergTable); - - // Create planner and plan the delete operation - NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); - LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(deleteQueryPlan, ctx.getStatementContext()); - - // Plan the delete query to generate physical plan and distributed plan - planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); - - // Set planner in executor for later use - executor.setPlanner(planner); - executor.checkBlockRules(); - Optional conflictFilter = - IcebergConflictDetectionFilterUtils.buildConflictDetectionFilter( - planner.getAnalyzedPlan(), icebergTable); - - PhysicalSink physicalSink = getPhysicalSink(planner); - PlanFragment fragment = planner.getFragments().get(0); - DataSink dataSink = fragment.getSink(); - boolean emptyInsert = childIsEmptyRelation(physicalSink); - String label = String.format("iceberg_delete_%x_%x", ctx.queryId().hi, ctx.queryId().lo); - - // Create IcebergDeleteExecutor and execute - IcebergDeleteExecutor deleteExecutor = new IcebergDeleteExecutor( - ctx, - icebergTable, - label, - planner, - emptyInsert, - -1L); - deleteExecutor.setConflictDetectionFilter(conflictFilter); - - if (deleteExecutor.isEmptyInsert()) { - return; - } - - deleteExecutor.beginTransaction(); - deleteExecutor.finalizeSinkForDelete(fragment, dataSink, physicalSink); - deleteExecutor.getCoordinator().setTxnId(deleteExecutor.getTxnId()); - executor.setCoord(deleteExecutor.getCoordinator()); - deleteExecutor.executeSingleInsert(executor); + executeWithExternalTableBatchModeDisabled(ctx, () -> { + // Create planner and plan the delete operation + NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); + LogicalPlanAdapter logicalPlanAdapter = + new LogicalPlanAdapter(deleteQueryPlan, ctx.getStatementContext()); + + // Plan the delete query to generate physical plan and distributed plan + planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); + + // Set planner in executor for later use + executor.setPlanner(planner); + executor.checkBlockRules(); + Optional conflictFilter = + IcebergConflictDetectionFilterUtils.buildConflictDetectionFilter( + planner.getAnalyzedPlan(), icebergTable); + + PhysicalSink physicalSink = getPhysicalSink(planner); + PlanFragment fragment = planner.getFragments().get(0); + DataSink dataSink = fragment.getSink(); + boolean emptyInsert = childIsEmptyRelation(physicalSink); + String label = String.format("iceberg_delete_%x_%x", ctx.queryId().hi, ctx.queryId().lo); + + // Create IcebergDeleteExecutor and execute + IcebergDeleteExecutor deleteExecutor = new IcebergDeleteExecutor( + ctx, + icebergTable, + label, + planner, + emptyInsert, + -1L); + deleteExecutor.setConflictDetectionFilter(conflictFilter); + + if (deleteExecutor.isEmptyInsert()) { + return null; + } + + deleteExecutor.beginTransaction(); + deleteExecutor.finalizeSinkForDelete(fragment, dataSink, physicalSink); + deleteExecutor.getCoordinator().setTxnId(deleteExecutor.getTxnId()); + executor.setCoord(deleteExecutor.getCoordinator()); + deleteExecutor.executeSingleInsert(executor); + return null; + }); } finally { ctx.setIcebergRowIdTargetTableId(previousTargetTableId); } } + @VisibleForTesting + static T executeWithExternalTableBatchModeDisabled( + ConnectContext ctx, Callable action) throws Exception { + boolean previousEnableExternalTableBatchMode = + ctx.getSessionVariable().enableExternalTableBatchMode; + // disable batch mode for iceberg scan node get all splits. + // IcebergRewritableDeletePlanner.collect for map list> + ctx.getSessionVariable().enableExternalTableBatchMode = false; + try { + return action.call(); + } finally { + ctx.getSessionVariable().enableExternalTableBatchMode = + previousEnableExternalTableBatchMode; + } + } + /** * Complete the query plan by adding necessary columns for position delete operation. * Select $row_id (file_path, row_position, partition info). diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergMergeCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergMergeCommand.java index c54d6a8889bf34..7770d147812682 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergMergeCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergMergeCommand.java @@ -27,6 +27,7 @@ import org.apache.doris.datasource.iceberg.IcebergMergeOperation; import org.apache.doris.datasource.iceberg.IcebergNereidsUtils; import org.apache.doris.datasource.iceberg.IcebergRowId; +import org.apache.doris.datasource.iceberg.IcebergUtils; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.analyzer.UnboundAlias; import org.apache.doris.nereids.analyzer.UnboundRelation; @@ -77,6 +78,7 @@ import org.apache.doris.qe.QueryState; import org.apache.doris.qe.StmtExecutor; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; @@ -87,6 +89,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.concurrent.Callable; /** * MERGE INTO command for Iceberg tables. @@ -234,7 +237,7 @@ private List buildDeleteProjection(Expression rowIdExpr, List nameParts = Lists.newArrayList(targetNameInPlan); @@ -259,6 +262,12 @@ private List buildUpdateProjection(MergeMatchedClause clause, Expres projection.add(new TinyIntLiteral(IcebergMergeOperation.UPDATE_OPERATION_NUMBER)); projection.add(rowIdExpr); for (Column column : columns) { + if (IcebergUtils.isIcebergRowLineageColumn(column)) { + List nameParts = Lists.newArrayList(targetNameInPlan); + nameParts.add(column.getName()); + projection.add(new UnboundSlot(nameParts)); + continue; + } if (!column.isVisible()) { continue; } @@ -312,6 +321,10 @@ private List buildInsertProjection(MergeNotMatchedClause clause, int visibleIndex = 0; for (Column column : columns) { + if (IcebergUtils.isIcebergRowLineageColumn(column)) { + projection.add(new NullLiteral(DataType.fromCatalogType(column.getType()))); + continue; + } if (!column.isVisible()) { continue; } @@ -384,10 +397,15 @@ private LogicalPlan buildMergeProjectPlan(ConnectContext ctx, IcebergExternalTab rowIdExpr = rowIdSlot.get(); } } + boolean hasRowLineageColumns = columns.stream().anyMatch(IcebergUtils::isIcebergRowLineageColumn); List outputProjections = new ArrayList<>(); outputProjections.add(new UnboundStar(ImmutableList.of())); if (!Util.showHiddenColumns()) { outputProjections.add((NamedExpression) rowIdExpr); + if (hasRowLineageColumns) { + outputProjections.add(getTargetRowLineageSlot(IcebergUtils.ICEBERG_ROW_ID_COL)); + outputProjections.add(getTargetRowLineageSlot(IcebergUtils.ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL)); + } } outputProjections.add(generateBranchLabel(rowIdExpr)); plan = new LogicalProject<>(outputProjections, plan); @@ -412,7 +430,7 @@ private LogicalPlan buildMergeProjectPlan(ConnectContext ctx, IcebergExternalTab colNames.add(IcebergMergeOperation.OPERATION_COLUMN); colNames.add(Column.ICEBERG_ROWID_COL); for (Column column : columns) { - if (column.isVisible()) { + if (column.isVisible() || IcebergUtils.isIcebergRowLineageColumn(column)) { colNames.add(column.getName()); } } @@ -452,35 +470,54 @@ private LogicalPlan buildMergePlan(ConnectContext ctx, IcebergExternalTable iceb private boolean executeMergePlan(ConnectContext ctx, StmtExecutor executor, IcebergExternalTable icebergTable, LogicalPlan logicalPlan) throws Exception { - LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(logicalPlan, ctx.getStatementContext()); - NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); - planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); - executor.setPlanner(planner); - executor.checkBlockRules(); - Optional conflictFilter = - IcebergConflictDetectionFilterUtils.buildConflictDetectionFilter( - planner.getAnalyzedPlan(), icebergTable); - - PhysicalSink physicalSink = getPhysicalMergeSink(planner); - PlanFragment fragment = planner.getFragments().get(0); - DataSink dataSink = fragment.getSink(); - boolean emptyInsert = childIsEmptyRelation(physicalSink); - String label = String.format("iceberg_merge_into_%x_%x", ctx.queryId().hi, ctx.queryId().lo); - - IcebergMergeExecutor insertExecutor = - new IcebergMergeExecutor(ctx, icebergTable, label, planner, emptyInsert, -1L); - insertExecutor.setConflictDetectionFilter(conflictFilter); - - if (insertExecutor.isEmptyInsert()) { - return true; - } + return executeWithExternalTableBatchModeDisabled(ctx, () -> { + LogicalPlanAdapter logicalPlanAdapter = + new LogicalPlanAdapter(logicalPlan, ctx.getStatementContext()); + NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); + planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); + executor.setPlanner(planner); + executor.checkBlockRules(); + Optional conflictFilter = + IcebergConflictDetectionFilterUtils.buildConflictDetectionFilter( + planner.getAnalyzedPlan(), icebergTable); + + PhysicalSink physicalSink = getPhysicalMergeSink(planner); + PlanFragment fragment = planner.getFragments().get(0); + DataSink dataSink = fragment.getSink(); + boolean emptyInsert = childIsEmptyRelation(physicalSink); + String label = String.format("iceberg_merge_into_%x_%x", ctx.queryId().hi, ctx.queryId().lo); + + IcebergMergeExecutor insertExecutor = + new IcebergMergeExecutor(ctx, icebergTable, label, planner, emptyInsert, -1L); + insertExecutor.setConflictDetectionFilter(conflictFilter); + + if (insertExecutor.isEmptyInsert()) { + return true; + } - insertExecutor.beginTransaction(); - insertExecutor.finalizeSinkForMerge(fragment, dataSink, physicalSink); - insertExecutor.getCoordinator().setTxnId(insertExecutor.getTxnId()); - executor.setCoord(insertExecutor.getCoordinator()); - insertExecutor.executeSingleInsert(executor); - return ctx.getState().getStateType() != QueryState.MysqlStateType.ERR; + insertExecutor.beginTransaction(); + insertExecutor.finalizeSinkForMerge(fragment, dataSink, physicalSink); + insertExecutor.getCoordinator().setTxnId(insertExecutor.getTxnId()); + executor.setCoord(insertExecutor.getCoordinator()); + insertExecutor.executeSingleInsert(executor); + return ctx.getState().getStateType() != QueryState.MysqlStateType.ERR; + }); + } + + @VisibleForTesting + static T executeWithExternalTableBatchModeDisabled( + ConnectContext ctx, Callable action) throws Exception { + boolean previousEnableExternalTableBatchMode = + ctx.getSessionVariable().enableExternalTableBatchMode; + // disable batch mode for iceberg scan node get all splits. + // IcebergRewritableDeletePlanner.collect for map list> + ctx.getSessionVariable().enableExternalTableBatchMode = false; + try { + return action.call(); + } finally { + ctx.getSessionVariable().enableExternalTableBatchMode = + previousEnableExternalTableBatchMode; + } } private PhysicalSink getPhysicalMergeSink(NereidsPlanner planner) { @@ -512,6 +549,12 @@ private Expression getTargetRowIdSlot() { return new UnboundSlot(Column.ICEBERG_ROWID_COL); } + private NamedExpression getTargetRowLineageSlot(String columnName) { + List nameParts = Lists.newArrayList(targetNameInPlan); + nameParts.add(columnName); + return new UnboundSlot(nameParts); + } + private static Column getRowIdColumn(IcebergExternalTable table) { return IcebergNereidsUtils.getRowIdColumn(table); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergUpdateCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergUpdateCommand.java index 8e488862a82f9a..2ed7e0ed49b6f7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergUpdateCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/IcebergUpdateCommand.java @@ -26,6 +26,7 @@ import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.datasource.iceberg.IcebergMergeOperation; import org.apache.doris.datasource.iceberg.IcebergNereidsUtils; +import org.apache.doris.datasource.iceberg.IcebergUtils; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.analyzer.UnboundAlias; import org.apache.doris.nereids.analyzer.UnboundSlot; @@ -64,6 +65,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.Callable; import java.util.stream.Collectors; /** @@ -136,35 +138,54 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { private boolean executeMergePlan(ConnectContext ctx, StmtExecutor executor, IcebergExternalTable icebergTable, LogicalPlan logicalPlan) throws Exception { - LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(logicalPlan, ctx.getStatementContext()); - NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); - planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); - executor.setPlanner(planner); - executor.checkBlockRules(); - Optional conflictFilter = - IcebergConflictDetectionFilterUtils.buildConflictDetectionFilter( - planner.getAnalyzedPlan(), icebergTable); + return executeWithExternalTableBatchModeDisabled(ctx, () -> { + LogicalPlanAdapter logicalPlanAdapter = + new LogicalPlanAdapter(logicalPlan, ctx.getStatementContext()); + NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); + planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); + executor.setPlanner(planner); + executor.checkBlockRules(); + Optional conflictFilter = + IcebergConflictDetectionFilterUtils.buildConflictDetectionFilter( + planner.getAnalyzedPlan(), icebergTable); - PhysicalSink physicalSink = getPhysicalMergeSink(planner); - PlanFragment fragment = planner.getFragments().get(0); - DataSink dataSink = fragment.getSink(); - boolean emptyInsert = childIsEmptyRelation(physicalSink); - String label = String.format("iceberg_update_merge_%x_%x", ctx.queryId().hi, ctx.queryId().lo); + PhysicalSink physicalSink = getPhysicalMergeSink(planner); + PlanFragment fragment = planner.getFragments().get(0); + DataSink dataSink = fragment.getSink(); + boolean emptyInsert = childIsEmptyRelation(physicalSink); + String label = String.format("iceberg_update_merge_%x_%x", ctx.queryId().hi, ctx.queryId().lo); - IcebergMergeExecutor insertExecutor = - new IcebergMergeExecutor(ctx, icebergTable, label, planner, emptyInsert, -1L); - insertExecutor.setConflictDetectionFilter(conflictFilter); + IcebergMergeExecutor insertExecutor = + new IcebergMergeExecutor(ctx, icebergTable, label, planner, emptyInsert, -1L); + insertExecutor.setConflictDetectionFilter(conflictFilter); - if (insertExecutor.isEmptyInsert()) { - return true; - } + if (insertExecutor.isEmptyInsert()) { + return true; + } - insertExecutor.beginTransaction(); - insertExecutor.finalizeSinkForMerge(fragment, dataSink, physicalSink); - insertExecutor.getCoordinator().setTxnId(insertExecutor.getTxnId()); - executor.setCoord(insertExecutor.getCoordinator()); - insertExecutor.executeSingleInsert(executor); - return ctx.getState().getStateType() != QueryState.MysqlStateType.ERR; + insertExecutor.beginTransaction(); + insertExecutor.finalizeSinkForMerge(fragment, dataSink, physicalSink); + insertExecutor.getCoordinator().setTxnId(insertExecutor.getTxnId()); + executor.setCoord(insertExecutor.getCoordinator()); + insertExecutor.executeSingleInsert(executor); + return ctx.getState().getStateType() != QueryState.MysqlStateType.ERR; + }); + } + + @VisibleForTesting + static T executeWithExternalTableBatchModeDisabled( + ConnectContext ctx, Callable action) throws Exception { + boolean previousEnableExternalTableBatchMode = + ctx.getSessionVariable().enableExternalTableBatchMode; + // disable batch mode for iceberg scan node get all splits. + // IcebergRewritableDeletePlanner.collect for map list> + ctx.getSessionVariable().enableExternalTableBatchMode = false; + try { + return action.call(); + } finally { + ctx.getSessionVariable().enableExternalTableBatchMode = + previousEnableExternalTableBatchMode; + } } @VisibleForTesting @@ -182,11 +203,15 @@ LogicalPlan buildMergeProjectPlan(ConnectContext ctx, LogicalPlan logicalQuery, NamedExpression operationColumn = new UnboundAlias( new TinyIntLiteral(IcebergMergeOperation.UPDATE_OPERATION_NUMBER), IcebergMergeOperation.OPERATION_COLUMN); - List projectItems = new ArrayList<>(2 + updateColumns.size()); projectItems.add(operationColumn); projectItems.add(rowIdColumn); projectItems.addAll(updateColumns); + for (Column col : columns) { + if (IcebergUtils.isIcebergRowLineageColumn(col)) { + projectItems.add(new UnboundSlot(tableName, col.getName())); + } + } return new LogicalProject<>(projectItems, planWithRowId); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergDeleteExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergDeleteExecutor.java index c761993583aa8a..0bc25dc15519b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergDeleteExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergDeleteExecutor.java @@ -20,9 +20,13 @@ import org.apache.doris.common.UserException; import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.datasource.iceberg.IcebergTransaction; +import org.apache.doris.datasource.iceberg.helper.IcebergRewritableDeletePlan; +import org.apache.doris.datasource.iceberg.helper.IcebergRewritableDeletePlanner; import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; import org.apache.doris.planner.DataSink; +import org.apache.doris.planner.IcebergDeleteSink; import org.apache.doris.planner.PlanFragment; import org.apache.doris.qe.ConnectContext; import org.apache.doris.transaction.TransactionType; @@ -46,7 +50,9 @@ */ public class IcebergDeleteExecutor extends BaseExternalTableInsertExecutor { private static final Logger LOG = LogManager.getLogger(IcebergDeleteExecutor.class); + private final NereidsPlanner nereidsPlanner; private Optional conflictDetectionFilter = Optional.empty(); + private IcebergRewritableDeletePlan rewritableDeletePlan = IcebergRewritableDeletePlan.empty(); public IcebergDeleteExecutor(ConnectContext ctx, IcebergExternalTable table, String labelName, NereidsPlanner planner, @@ -54,10 +60,22 @@ public IcebergDeleteExecutor(ConnectContext ctx, IcebergExternalTable table, // BaseExternalTableInsertExecutor requires Optional // For DELETE operations, we pass Optional.empty(). super(ctx, table, labelName, planner, Optional.empty(), emptyInsert, jobId); + this.nereidsPlanner = planner; } + /** Finalize delete sink and attach rewritable delete-file metadata for BE. */ public void finalizeSinkForDelete(PlanFragment fragment, DataSink sink, PhysicalSink physicalSink) { super.finalizeSink(fragment, sink, physicalSink); + if (!(sink instanceof IcebergDeleteSink)) { + return; + } + try { + rewritableDeletePlan = IcebergRewritableDeletePlanner.collectForDelete( + (IcebergExternalTable) table, nereidsPlanner); + } catch (UserException e) { + throw new AnalysisException(e.getMessage(), e); + } + ((IcebergDeleteSink) sink).setRewritableDeleteFileSets(rewritableDeletePlan.getThriftDeleteFileSets()); } public void setConflictDetectionFilter(Optional filter) { @@ -68,6 +86,8 @@ public void setConflictDetectionFilter(Optional filter) { protected void beforeExec() throws UserException { IcebergTransaction transaction = (IcebergTransaction) transactionManager.getTransaction(txnId); transaction.beginDelete((IcebergExternalTable) table); + transaction.setRewrittenDeleteFilesByReferencedDataFile( + rewritableDeletePlan.getDeleteFilesByReferencedDataFile()); if (conflictDetectionFilter.isPresent()) { transaction.setConflictDetectionFilter(conflictDetectionFilter.get()); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertCommandContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertCommandContext.java index 45bcb5092c8584..13f704c78cfb41 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertCommandContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertCommandContext.java @@ -30,6 +30,7 @@ public class IcebergInsertCommandContext extends BaseExternalTableInsertCommandC // Static partition key-value pairs for INSERT OVERWRITE ... PARTITION // (col='val', ...) private Map staticPartitionValues = Maps.newHashMap(); + private boolean rewriting = false; public Optional getBranchName() { return branchName; @@ -55,4 +56,12 @@ public void setStaticPartitionValues(Map staticPartitionValues) public boolean isStaticPartitionOverwrite() { return isOverwrite() && !staticPartitionValues.isEmpty(); } + + public boolean isRewriting() { + return rewriting; + } + + public void setRewriting(boolean rewriting) { + this.rewriting = rewriting; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergMergeExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergMergeExecutor.java index be82a2f82fbdbb..a96dba696eafe2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergMergeExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergMergeExecutor.java @@ -21,9 +21,13 @@ import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.datasource.iceberg.IcebergTransaction; +import org.apache.doris.datasource.iceberg.helper.IcebergRewritableDeletePlan; +import org.apache.doris.datasource.iceberg.helper.IcebergRewritableDeletePlanner; import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; import org.apache.doris.planner.DataSink; +import org.apache.doris.planner.IcebergMergeSink; import org.apache.doris.planner.PlanFragment; import org.apache.doris.qe.ConnectContext; import org.apache.doris.transaction.TransactionType; @@ -36,16 +40,30 @@ * Executor for Iceberg UPDATE merge operations (single scan + merge sink). */ public class IcebergMergeExecutor extends BaseExternalTableInsertExecutor { + private final NereidsPlanner nereidsPlanner; private Optional conflictDetectionFilter = Optional.empty(); + private IcebergRewritableDeletePlan rewritableDeletePlan = IcebergRewritableDeletePlan.empty(); public IcebergMergeExecutor(ConnectContext ctx, IcebergExternalTable table, String labelName, NereidsPlanner planner, boolean emptyInsert, long jobId) { super(ctx, table, labelName, planner, Optional.empty(), emptyInsert, jobId); + this.nereidsPlanner = planner; } + /** Finalize merge sink and attach rewritable delete-file metadata for BE. */ public void finalizeSinkForMerge(PlanFragment fragment, DataSink sink, PhysicalSink physicalSink) { super.finalizeSink(fragment, sink, physicalSink); + if (!(sink instanceof IcebergMergeSink)) { + return; + } + try { + rewritableDeletePlan = IcebergRewritableDeletePlanner.collectForMerge( + (IcebergExternalTable) table, nereidsPlanner); + } catch (UserException e) { + throw new AnalysisException(e.getMessage(), e); + } + ((IcebergMergeSink) sink).setRewritableDeleteFileSets(rewritableDeletePlan.getThriftDeleteFileSets()); } public void setConflictDetectionFilter(Optional filter) { @@ -56,6 +74,8 @@ public void setConflictDetectionFilter(Optional filter) { protected void beforeExec() throws UserException { IcebergTransaction transaction = (IcebergTransaction) transactionManager.getTransaction(txnId); transaction.beginMerge((IcebergExternalTable) table); + transaction.setRewrittenDeleteFilesByReferencedDataFile( + rewritableDeletePlan.getDeleteFilesByReferencedDataFile()); if (conflictDetectionFilter.isPresent()) { transaction.setConflictDetectionFilter(conflictDetectionFilter.get()); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/RewriteTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/RewriteTableCommand.java index 3a8edf2d849387..127d5955dfcf00 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/RewriteTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/RewriteTableCommand.java @@ -191,6 +191,7 @@ private ExecutorFactory selectInsertExecutorFactory(NereidsPlanner planner, Conn IcebergInsertCommandContext icebergInsertCtx = insertCtx .map(c -> (IcebergInsertCommandContext) c) .orElseGet(IcebergInsertCommandContext::new); + icebergInsertCtx.setRewriting(true); branchName.ifPresent(notUsed -> icebergInsertCtx.setBranchName(branchName)); return ExecutorFactory.from(planner, dataSink, physicalSink, () -> new IcebergRewriteExecutor(ctx, icebergExternalTable, label, planner, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java index 61f928657ca488..ef917b63a8d42e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java @@ -20,6 +20,7 @@ import org.apache.doris.analysis.TableScanParams; import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.common.IdGenerator; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.iceberg.IcebergExternalTable; @@ -27,8 +28,12 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.trees.TableSample; +import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; +import org.apache.doris.nereids.trees.plans.AbstractPlan; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.RelationId; @@ -40,6 +45,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableList.Builder; import com.google.common.collect.ImmutableMap; import java.util.Collection; @@ -167,7 +173,27 @@ public List computeOutput() { if (cachedOutputs.isPresent()) { return cachedOutputs.get(); } - return super.computeOutput(); + + if (table instanceof IcebergExternalTable) { + // iceberg v3 need append row lineage columns + return computeIcebergOutput((IcebergExternalTable) table); + } else { + return super.computeOutput(); + } + } + + private List computeIcebergOutput(IcebergExternalTable iceTable) { + IdGenerator exprIdGenerator = StatementScopeIdGenerator.getExprIdGenerator(); + Builder slots = ImmutableList.builder(); + table.getFullSchema() + .stream() + .map(col -> SlotReference.fromColumn(exprIdGenerator.getNextId(), table, col, qualified())) + .forEach(slots::add); + // add virtual slots + for (NamedExpression virtualColumn : virtualColumns) { + slots.add(virtualColumn.toSlot()); + } + return slots.build(); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergDeleteSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergDeleteSink.java index 7c1c152c459ab3..19bc1af6f14252 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergDeleteSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergDeleteSink.java @@ -32,11 +32,14 @@ import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TFileType; import org.apache.doris.thrift.TIcebergDeleteSink; +import org.apache.doris.thrift.TIcebergRewritableDeleteFileSet; import org.apache.iceberg.Table; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -49,6 +52,7 @@ public class IcebergDeleteSink extends BaseExternalTableDataSink { private final IcebergExternalTable targetTable; private final DeleteCommandContext deleteContext; + private List rewritableDeleteFileSets = Collections.emptyList(); private static final HashSet supportedTypes = new HashSet() {{ add(TFileFormatType.FORMAT_PARQUET); @@ -73,6 +77,13 @@ public IcebergDeleteSink(IcebergExternalTable targetTable, DeleteCommandContext targetTable.getIcebergTable()); } + public void setRewritableDeleteFileSets(List deleteFileSets) { + rewritableDeleteFileSets = deleteFileSets != null ? deleteFileSets : Collections.emptyList(); + if (tDataSink != null && tDataSink.isSetIcebergDeleteSink()) { + tDataSink.getIcebergDeleteSink().setRewritableDeleteFileSets(rewritableDeleteFileSets); + } + } + @Override protected Set supportedFileFormatTypes() { return supportedTypes; @@ -130,7 +141,12 @@ public void bindDataSink(Optional insertCtx) // Partition information if (icebergTable.spec().isPartitioned()) { tSink.setPartitionSpecId(icebergTable.spec().specId()); - // Partition data JSON will be set by BE based on actual data + } + + int formatVersion = IcebergUtils.getFormatVersion(icebergTable); + tSink.setFormatVersion(formatVersion); + if (formatVersion >= 3 && !rewritableDeleteFileSets.isEmpty()) { + tSink.setRewritableDeleteFileSets(rewritableDeleteFileSets); } tDataSink = new TDataSink(TDataSinkType.ICEBERG_DELETE_SINK); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergMergeSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergMergeSink.java index 99eb828918f62c..4af4ba17e18578 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergMergeSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergMergeSink.java @@ -32,6 +32,7 @@ import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TFileType; import org.apache.doris.thrift.TIcebergMergeSink; +import org.apache.doris.thrift.TIcebergRewritableDeleteFileSet; import org.apache.doris.thrift.TSortField; import com.google.common.collect.ImmutableList; @@ -39,6 +40,7 @@ import com.google.common.collect.Maps; import org.apache.iceberg.NullOrder; import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.SortDirection; import org.apache.iceberg.SortField; @@ -46,8 +48,10 @@ import org.apache.iceberg.Table; import org.apache.iceberg.types.Types; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -60,6 +64,7 @@ public class IcebergMergeSink extends BaseExternalTableDataSink { private final IcebergExternalTable targetTable; private final DeleteCommandContext deleteContext; + private List rewritableDeleteFileSets = Collections.emptyList(); private static final HashSet supportedTypes = new HashSet() {{ add(TFileFormatType.FORMAT_PARQUET); @@ -84,6 +89,13 @@ public IcebergMergeSink(IcebergExternalTable targetTable, DeleteCommandContext d targetTable.getIcebergTable()); } + public void setRewritableDeleteFileSets(List deleteFileSets) { + rewritableDeleteFileSets = deleteFileSets != null ? deleteFileSets : Collections.emptyList(); + if (tDataSink != null && tDataSink.isSetIcebergMergeSink()) { + tDataSink.getIcebergMergeSink().setRewritableDeleteFileSets(rewritableDeleteFileSets); + } + } + @Override protected Set supportedFileFormatTypes() { return supportedTypes; @@ -112,8 +124,13 @@ public void bindDataSink(Optional insertCtx) tSink.setDbName(targetTable.getDbName()); tSink.setTbName(targetTable.getName()); - // schema - tSink.setSchemaJson(SchemaParser.toJson(icebergTable.schema())); + Schema schema = icebergTable.schema(); + int formatVersion = IcebergUtils.getFormatVersion(icebergTable); + if (formatVersion >= 3) { + schema = IcebergUtils.appendRowLineageFieldsForV3(schema); + } + tSink.setFormatVersion(formatVersion); + tSink.setSchemaJson(SchemaParser.toJson(schema)); // partition spec if (icebergTable.spec().isPartitioned()) { @@ -173,6 +190,9 @@ public void bindDataSink(Optional insertCtx) tSink.setPartitionSpecIdForDelete(icebergTable.spec().specId()); } + if (formatVersion >= 3 && !rewritableDeleteFileSets.isEmpty()) { + tSink.setRewritableDeleteFileSets(rewritableDeleteFileSets); + } tDataSink = new TDataSink(TDataSinkType.ICEBERG_MERGE_SINK); tDataSink.setIcebergMergeSink(tSink); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergTableSink.java index 581eee08a5c541..0f3b1bb24d26bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/IcebergTableSink.java @@ -34,11 +34,13 @@ import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TFileType; import org.apache.doris.thrift.TIcebergTableSink; +import org.apache.doris.thrift.TIcebergWriteType; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.iceberg.NullOrder; import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.SortDirection; import org.apache.iceberg.SortField; @@ -117,8 +119,22 @@ public void bindDataSink(Optional insertCtx) tSink.setDbName(targetTable.getDbName()); tSink.setTbName(targetTable.getName()); - // schema - tSink.setSchemaJson(SchemaParser.toJson(icebergTable.schema())); + boolean isRewriting = false; + if (insertCtx.isPresent() && insertCtx.get() instanceof IcebergInsertCommandContext) { + IcebergInsertCommandContext context = (IcebergInsertCommandContext) insertCtx.get(); + isRewriting = context.isRewriting(); + if (isRewriting) { + tSink.setWriteType(TIcebergWriteType.REWRITE); + } + } + + Schema schema = icebergTable.schema(); + if (isRewriting + && IcebergUtils.getFormatVersion(icebergTable) >= IcebergUtils.ICEBERG_ROW_LINEAGE_MIN_VERSION) { + // iceberg v3 format requires additional row lineage fields when rewrite data files. + schema = IcebergUtils.appendRowLineageFieldsForV3(schema); + } + tSink.setSchemaJson(SchemaParser.toJson(schema)); // partition spec if (icebergTable.spec().isPartitioned()) { @@ -172,7 +188,7 @@ public void bindDataSink(Optional insertCtx) tSink.setBrokerAddresses(getBrokerAddresses(targetTable.getCatalog().bindBrokerName())); } - if (insertCtx.isPresent()) { + if (insertCtx.isPresent() && insertCtx.get() instanceof IcebergInsertCommandContext) { IcebergInsertCommandContext context = (IcebergInsertCommandContext) insertCtx.get(); tSink.setOverwrite(context.isOverwrite()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergTransactionTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergTransactionTest.java index 00e0bc69ec4c21..71c2eb8794c94f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergTransactionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergTransactionTest.java @@ -22,14 +22,19 @@ import org.apache.doris.common.util.SerializationUtils; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.iceberg.helper.IcebergWriterHelper; import org.apache.doris.nereids.trees.plans.commands.insert.IcebergInsertCommandContext; import org.apache.doris.thrift.TFileContent; import org.apache.doris.thrift.TIcebergCommitData; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Namespace; @@ -56,6 +61,8 @@ import java.nio.file.Path; import java.time.Instant; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -454,4 +461,154 @@ public void testUnpartitionedTableOverwriteWithoutData() throws UserException { checkSnapshotTotalProperties(table.currentSnapshot().summary(), "0", "0", "0"); } + + @Test + public void testFinishDeleteDoesNotRewritePreviousDeleteFilesForV2() throws UserException { + verifyFinishDeleteRewriteBehavior(2, false); + } + + @Test + public void testFinishDeleteRewritesAllSharedPuffinDeleteFilesForV3() throws UserException { + String referencedDataFile = "s3a://warehouse/wh/db3/tbWithoutPartition/data/data-file.parquet"; + + Table icebergTable = Mockito.mock(Table.class); + org.apache.iceberg.Transaction icebergTxn = Mockito.mock(org.apache.iceberg.Transaction.class); + RowDelta rowDelta = Mockito.mock(RowDelta.class, Mockito.RETURNS_SELF); + DeleteFile newDeleteFile = Mockito.mock(DeleteFile.class); + DeleteFile oldDeleteFile1 = buildDeletionVectorDeleteFile( + "s3a://warehouse/wh/db3/tbWithoutPartition/data/delete-shared.puffin", + referencedDataFile, 4L, 21L); + DeleteFile oldDeleteFile2 = buildDeletionVectorDeleteFile( + "s3a://warehouse/wh/db3/tbWithoutPartition/data/delete-shared.puffin", + referencedDataFile, 25L, 19L); + IcebergExternalTable icebergExternalTable = Mockito.mock(IcebergExternalTable.class); + + PartitionSpec spec = PartitionSpec.unpartitioned(); + Mockito.when(icebergTable.newTransaction()).thenReturn(icebergTxn); + Mockito.when(icebergTable.currentSnapshot()).thenReturn(null); + Mockito.when(icebergTable.spec()).thenReturn(spec); + Mockito.when(icebergTable.specs()).thenReturn(Collections.singletonMap(spec.specId(), spec)); + Mockito.when(icebergTable.properties()).thenReturn(Collections.emptyMap()); + Mockito.when(icebergTable.name()).thenReturn(tbWithoutPartition); + Mockito.when(icebergTxn.table()).thenReturn(icebergTable); + Mockito.when(icebergTxn.newRowDelta()).thenReturn(rowDelta); + Mockito.when(newDeleteFile.path()).thenReturn("s3a://warehouse/wh/db3/tbWithoutPartition/data/delete-new.puffin"); + + Mockito.when(icebergExternalTable.getCatalog()).thenReturn(spyExternalCatalog); + Mockito.when(icebergExternalTable.getName()).thenReturn(tbWithoutPartition); + + TIcebergCommitData commitData = new TIcebergCommitData(); + commitData.setFilePath("delete-dv-shared.puffin"); + commitData.setFileContent(TFileContent.POSITION_DELETES); + commitData.setRowCount(3); + commitData.setFileSize(44); + commitData.setContentOffset(4); + commitData.setContentSizeInBytes(21); + commitData.setReferencedDataFilePath(referencedDataFile); + + IcebergTransaction txn = getTxn(); + txn.updateIcebergCommitData(Collections.singletonList(commitData)); + + try (MockedStatic mockedUtils = Mockito.mockStatic(IcebergUtils.class); + MockedStatic mockedWriterHelper = + Mockito.mockStatic(IcebergWriterHelper.class)) { + mockedUtils.when(() -> IcebergUtils.getIcebergTable(ArgumentMatchers.any(ExternalTable.class))) + .thenReturn(icebergTable); + mockedUtils.when(() -> IcebergUtils.getFileFormat(icebergTable)).thenReturn(FileFormat.PARQUET); + mockedUtils.when(() -> IcebergUtils.getFormatVersion(icebergTable)).thenReturn(3); + mockedWriterHelper.when(() -> IcebergWriterHelper.convertToDeleteFiles( + ArgumentMatchers.any(FileFormat.class), + ArgumentMatchers.eq(spec), + ArgumentMatchers.anyList())) + .thenReturn(Collections.singletonList(newDeleteFile)); + + txn.beginDelete(icebergExternalTable); + txn.setRewrittenDeleteFilesByReferencedDataFile( + Collections.singletonMap(referencedDataFile, Arrays.asList(oldDeleteFile1, oldDeleteFile2))); + txn.finishDelete(NameMapping.createForTest(dbName, tbWithoutPartition)); + } + + Mockito.verify(rowDelta).addDeletes(newDeleteFile); + Mockito.verify(rowDelta).removeDeletes(oldDeleteFile1); + Mockito.verify(rowDelta).removeDeletes(oldDeleteFile2); + Mockito.verify(rowDelta).commit(); + } + + private void verifyFinishDeleteRewriteBehavior(int formatVersion, boolean expectRewrite) + throws UserException { + String referencedDataFile = "s3a://warehouse/wh/db3/tbWithoutPartition/data/data-file.parquet"; + + Table icebergTable = Mockito.mock(Table.class); + org.apache.iceberg.Transaction icebergTxn = Mockito.mock(org.apache.iceberg.Transaction.class); + RowDelta rowDelta = Mockito.mock(RowDelta.class, Mockito.RETURNS_SELF); + DeleteFile newDeleteFile = Mockito.mock(DeleteFile.class); + DeleteFile oldDeleteFile = Mockito.mock(DeleteFile.class); + IcebergExternalTable icebergExternalTable = Mockito.mock(IcebergExternalTable.class); + + PartitionSpec spec = PartitionSpec.unpartitioned(); + Mockito.when(icebergTable.newTransaction()).thenReturn(icebergTxn); + Mockito.when(icebergTable.currentSnapshot()).thenReturn(null); + Mockito.when(icebergTable.spec()).thenReturn(spec); + Mockito.when(icebergTable.specs()).thenReturn(Collections.singletonMap(spec.specId(), spec)); + Mockito.when(icebergTable.properties()).thenReturn(Collections.emptyMap()); + Mockito.when(icebergTable.name()).thenReturn(tbWithoutPartition); + Mockito.when(icebergTxn.table()).thenReturn(icebergTable); + Mockito.when(icebergTxn.newRowDelta()).thenReturn(rowDelta); + Mockito.when(newDeleteFile.path()).thenReturn("s3a://warehouse/wh/db3/tbWithoutPartition/data/delete-new.puffin"); + Mockito.when(oldDeleteFile.path()).thenReturn("s3a://warehouse/wh/db3/tbWithoutPartition/data/delete-old.parquet"); + + Mockito.when(icebergExternalTable.getCatalog()).thenReturn(spyExternalCatalog); + Mockito.when(icebergExternalTable.getName()).thenReturn(tbWithoutPartition); + + TIcebergCommitData commitData = new TIcebergCommitData(); + commitData.setFilePath("delete-dv.puffin"); + commitData.setFileContent(TFileContent.POSITION_DELETES); + commitData.setRowCount(3); + commitData.setFileSize(33); + commitData.setReferencedDataFilePath(referencedDataFile); + + IcebergTransaction txn = getTxn(); + txn.updateIcebergCommitData(Collections.singletonList(commitData)); + + try (MockedStatic mockedUtils = Mockito.mockStatic(IcebergUtils.class); + MockedStatic mockedWriterHelper = + Mockito.mockStatic(IcebergWriterHelper.class)) { + mockedUtils.when(() -> IcebergUtils.getIcebergTable(ArgumentMatchers.any(ExternalTable.class))) + .thenReturn(icebergTable); + mockedUtils.when(() -> IcebergUtils.getFileFormat(icebergTable)).thenReturn(FileFormat.PARQUET); + mockedUtils.when(() -> IcebergUtils.getFormatVersion(icebergTable)).thenReturn(formatVersion); + mockedWriterHelper.when(() -> IcebergWriterHelper.convertToDeleteFiles( + ArgumentMatchers.any(FileFormat.class), + ArgumentMatchers.eq(spec), + ArgumentMatchers.anyList())) + .thenReturn(Collections.singletonList(newDeleteFile)); + + txn.beginDelete(icebergExternalTable); + txn.setRewrittenDeleteFilesByReferencedDataFile( + Collections.singletonMap(referencedDataFile, Collections.singletonList(oldDeleteFile))); + txn.finishDelete(NameMapping.createForTest(dbName, tbWithoutPartition)); + } + + Mockito.verify(rowDelta).addDeletes(newDeleteFile); + if (expectRewrite) { + Mockito.verify(rowDelta).removeDeletes(oldDeleteFile); + } else { + Mockito.verify(rowDelta, Mockito.never()).removeDeletes(ArgumentMatchers.any(DeleteFile.class)); + } + Mockito.verify(rowDelta).commit(); + } + + private DeleteFile buildDeletionVectorDeleteFile(String puffinPath, String referencedDataFile, + long contentOffset, long contentLength) { + return FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath(puffinPath) + .withFormat(FileFormat.PUFFIN) + .withFileSizeInBytes(128) + .withRecordCount(2) + .withContentOffset(contentOffset) + .withContentSizeInBytes(contentLength) + .withReferencedDataFile(referencedDataFile) + .build(); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergUtilsTest.java index 40e3e08935a7b1..1fcde27aa95416 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergUtilsTest.java @@ -19,6 +19,8 @@ import org.apache.doris.analysis.TableScanParams; import org.apache.doris.analysis.TableSnapshot; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Type; import org.apache.doris.common.UserException; import org.apache.doris.datasource.iceberg.source.IcebergTableQueryInfo; @@ -28,6 +30,7 @@ import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFile.PartitionFieldSummary; +import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -103,6 +106,59 @@ private boolean getListAllTables(HiveCatalog hiveCatalog) throws IllegalAccessEx return declaredField.getBoolean(hiveCatalog); } + @Test + public void testIsIcebergRowLineageColumn() { + Column rowIdColumn = new Column(IcebergUtils.ICEBERG_ROW_ID_COL, Type.BIGINT, true); + Column sequenceColumn = new Column(IcebergUtils.ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL, Type.BIGINT, true); + Column normalColumn = new Column("id", Type.INT, true); + + Assert.assertTrue(IcebergUtils.isIcebergRowLineageColumn(rowIdColumn)); + Assert.assertTrue(IcebergUtils.isIcebergRowLineageColumn(sequenceColumn)); + Assert.assertTrue(IcebergUtils.isIcebergRowLineageColumn("_ROW_ID")); + Assert.assertFalse(IcebergUtils.isIcebergRowLineageColumn(normalColumn)); + Assert.assertFalse(IcebergUtils.isIcebergRowLineageColumn("id")); + } + + @Test + public void testAppendRowLineageColumnsForV3AddsInvisibleColumns() { + List schema = new ArrayList<>(); + schema.add(new Column("id", Type.INT, true)); + Table table = Mockito.mock(Table.class); + Mockito.when(table.properties()).thenReturn(ImmutableMap.of("format-version", "3")); + + List schemaWithRowLineage = IcebergUtils.appendRowLineageColumnsForV3(schema, table); + + Assert.assertEquals(3, schemaWithRowLineage.size()); + Assert.assertEquals(IcebergUtils.ICEBERG_ROW_ID_COL, schemaWithRowLineage.get(1).getName()); + Assert.assertEquals(IcebergUtils.ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL, + schemaWithRowLineage.get(2).getName()); + Assert.assertFalse(schemaWithRowLineage.get(1).isVisible()); + Assert.assertFalse(schemaWithRowLineage.get(2).isVisible()); + } + + @Test + public void testAppendRowLineageColumnsForV2ReturnsOriginalSchema() { + List schema = new ArrayList<>(); + schema.add(new Column("id", Type.INT, true)); + Table table = Mockito.mock(Table.class); + Mockito.when(table.properties()).thenReturn(ImmutableMap.of("format-version", "2")); + + List schemaWithRowLineage = IcebergUtils.appendRowLineageColumnsForV3(schema, table); + + Assert.assertSame(schema, schemaWithRowLineage); + Assert.assertEquals(1, schemaWithRowLineage.size()); + } + + @Test + public void testAppendRowLineageFieldsForV3AddsMetadataFields() { + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + + Schema schemaWithRowLineage = IcebergUtils.appendRowLineageFieldsForV3(schema); + + Assert.assertNotNull(schemaWithRowLineage.findField(MetadataColumns.ROW_ID.fieldId())); + Assert.assertNotNull(schemaWithRowLineage.findField(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId())); + } + @Test public void testGetPartitionInfoMapSkipBinaryIdentityPartition() { Schema schema = new Schema( diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/helper/IcebergRewritableDeletePlannerTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/helper/IcebergRewritableDeletePlannerTest.java new file mode 100644 index 00000000000000..4e797f75807bac --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/helper/IcebergRewritableDeletePlannerTest.java @@ -0,0 +1,144 @@ +// 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.iceberg.helper; + +import org.apache.doris.analysis.TupleDescriptor; +import org.apache.doris.analysis.TupleId; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.datasource.iceberg.source.IcebergScanNode; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.planner.PlanNodeId; +import org.apache.doris.planner.ScanContext; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.qe.SessionVariable; +import org.apache.doris.thrift.TIcebergDeleteFileDesc; +import org.apache.doris.thrift.TIcebergRewritableDeleteFileSet; + +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class IcebergRewritableDeletePlannerTest { + + private static class TestIcebergScanNode extends IcebergScanNode { + TestIcebergScanNode() { + super(new PlanNodeId(0), new TupleDescriptor(new TupleId(0)), new SessionVariable(), ScanContext.EMPTY); + } + } + + @Test + public void testCollectForDeleteReturnsEmptyWhenTableFormatVersionIsLessThanThree() throws Exception { + IcebergExternalTable table = mockIcebergExternalTable(2); + NereidsPlanner planner = Mockito.mock(NereidsPlanner.class); + Mockito.when(planner.getScanNodes()).thenReturn(Collections.singletonList(buildScanNode( + "file:///tmp/data.parquet", + buildDeleteFile("file:///tmp/delete.parquet", "file:///tmp/data.parquet"), + buildDeleteFileDesc("file:///tmp/delete.parquet")))); + + IcebergRewritableDeletePlan plan = IcebergRewritableDeletePlanner.collectForDelete(table, planner); + + Assertions.assertTrue(plan.getThriftDeleteFileSets().isEmpty()); + Assertions.assertTrue(plan.getDeleteFilesByReferencedDataFile().isEmpty()); + } + + @Test + public void testCollectForMergeAggregatesDeleteFilesAcrossIcebergScanNodes() throws Exception { + String firstDataFile = "file:///tmp/data-1.parquet"; + String secondDataFile = "file:///tmp/data-2.parquet"; + DeleteFile firstDeleteFile = buildDeleteFile("file:///tmp/delete-1.puffin", firstDataFile); + DeleteFile secondDeleteFile = buildDeleteFile("file:///tmp/delete-2.puffin", secondDataFile); + TestIcebergScanNode firstScanNode = buildScanNode( + firstDataFile, firstDeleteFile, buildDeleteFileDesc("file:///tmp/delete-1.puffin")); + TestIcebergScanNode secondScanNode = buildScanNode( + secondDataFile, secondDeleteFile, buildDeleteFileDesc("file:///tmp/delete-2.puffin")); + ScanNode otherScanNode = Mockito.mock(ScanNode.class); + + IcebergExternalTable table = mockIcebergExternalTable(3); + NereidsPlanner planner = Mockito.mock(NereidsPlanner.class); + Mockito.when(planner.getScanNodes()).thenReturn(Arrays.asList(firstScanNode, otherScanNode, secondScanNode)); + + IcebergRewritableDeletePlan plan = IcebergRewritableDeletePlanner.collectForMerge(table, planner); + + Assertions.assertEquals(2, plan.getThriftDeleteFileSets().size()); + Assertions.assertEquals(2, plan.getDeleteFilesByReferencedDataFile().size()); + Assertions.assertSame(firstDeleteFile, plan.getDeleteFilesByReferencedDataFile().get(firstDataFile).get(0)); + Assertions.assertSame(secondDeleteFile, plan.getDeleteFilesByReferencedDataFile().get(secondDataFile).get(0)); + + List referencedDataFiles = plan.getThriftDeleteFileSets().stream() + .map(TIcebergRewritableDeleteFileSet::getReferencedDataFilePath) + .sorted() + .collect(Collectors.toList()); + Assertions.assertEquals(Arrays.asList(firstDataFile, secondDataFile), referencedDataFiles); + + Assertions.assertThrows(UnsupportedOperationException.class, + () -> plan.getDeleteFilesByReferencedDataFile().put("new", Collections.emptyList())); + } + + private static TestIcebergScanNode buildScanNode( + String referencedDataFile, + DeleteFile deleteFile, + TIcebergDeleteFileDesc deleteFileDesc) { + TestIcebergScanNode scanNode = new TestIcebergScanNode(); + scanNode.deleteFilesByReferencedDataFile.put(referencedDataFile, Collections.singletonList(deleteFile)); + scanNode.deleteFilesDescByReferencedDataFile.put(referencedDataFile, Collections.singletonList(deleteFileDesc)); + return scanNode; + } + + private static DeleteFile buildDeleteFile(String deleteFilePath, String referencedDataFilePath) { + return FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath(deleteFilePath) + .withFormat(FileFormat.PUFFIN) + .withFileSizeInBytes(128L) + .withRecordCount(4L) + .withReferencedDataFile(referencedDataFilePath) + .withContentOffset(16L) + .withContentSizeInBytes(64L) + .build(); + } + + private static TIcebergDeleteFileDesc buildDeleteFileDesc(String deleteFilePath) { + TIcebergDeleteFileDesc deleteFileDesc = new TIcebergDeleteFileDesc(); + deleteFileDesc.setPath(deleteFilePath); + return deleteFileDesc; + } + + private static IcebergExternalTable mockIcebergExternalTable(int formatVersion) { + Table icebergTable = Mockito.mock(Table.class); + Map properties = new HashMap<>(); + properties.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + Mockito.when(icebergTable.properties()).thenReturn(properties); + + IcebergExternalTable table = Mockito.mock(IcebergExternalTable.class); + Mockito.when(table.getIcebergTable()).thenReturn(icebergTable); + return table; + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/helper/IcebergWriterHelperTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/helper/IcebergWriterHelperTest.java index 545dacf1ceea05..77d318518f326e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/helper/IcebergWriterHelperTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/helper/IcebergWriterHelperTest.java @@ -108,6 +108,32 @@ public void testConvertToDeleteFiles_PositionDelete() { Assertions.assertEquals(org.apache.iceberg.FileContent.POSITION_DELETES, deleteFile.content()); } + @Test + public void testConvertToDeleteFiles_DeletionVectorUsesPuffinMetadata() { + List commitDataList = new ArrayList<>(); + + TIcebergCommitData commitData = new TIcebergCommitData(); + commitData.setFilePath("/path/to/delete.puffin"); + commitData.setRowCount(7); + commitData.setFileSize(2048); + commitData.setFileContent(TFileContent.DELETION_VECTOR); + commitData.setContentOffset(128L); + commitData.setContentSizeInBytes(64L); + commitData.setReferencedDataFilePath("/path/to/data.parquet"); + commitDataList.add(commitData); + + List deleteFiles = IcebergWriterHelper.convertToDeleteFiles( + format, unpartitionedSpec, commitDataList); + + Assertions.assertEquals(1, deleteFiles.size()); + DeleteFile deleteFile = deleteFiles.get(0); + Assertions.assertEquals(FileFormat.PUFFIN, deleteFile.format()); + Assertions.assertEquals(128L, deleteFile.contentOffset()); + Assertions.assertEquals(64L, deleteFile.contentSizeInBytes()); + Assertions.assertEquals("/path/to/data.parquet", deleteFile.referencedDataFile()); + Assertions.assertEquals(org.apache.iceberg.FileContent.POSITION_DELETES, deleteFile.content()); + } + @Test public void testConvertToDeleteFiles_UnsupportedDeleteContent() { List commitDataList = new ArrayList<>(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/source/IcebergScanNodeTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/source/IcebergScanNodeTest.java index e16ee803dcaef0..f55b4f6f8e8027 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/source/IcebergScanNodeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/source/IcebergScanNodeTest.java @@ -19,9 +19,13 @@ import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.analysis.TupleId; +import org.apache.doris.common.util.LocationPath; +import org.apache.doris.datasource.TableFormatType; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.planner.ScanContext; import org.apache.doris.qe.SessionVariable; +import org.apache.doris.thrift.TFileRangeDesc; +import org.apache.doris.thrift.TIcebergDeleteFileDesc; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileScanTask; @@ -30,7 +34,9 @@ import org.junit.Test; import org.mockito.Mockito; +import java.lang.reflect.Field; import java.lang.reflect.Method; +import java.util.ArrayList; import java.util.Collections; public class IcebergScanNodeTest { @@ -70,4 +76,71 @@ public void testDetermineTargetFileSplitSizeHonorsMaxFileSplitNum() throws Excep Assert.assertEquals(100 * MB, target); } } + + @Test + public void testSetIcebergParamsKeepsDeletionVectorOffsetAsLong() throws Exception { + SessionVariable sv = new SessionVariable(); + TestIcebergScanNode node = new TestIcebergScanNode(sv); + + Field formatVersionField = IcebergScanNode.class.getDeclaredField("formatVersion"); + formatVersionField.setAccessible(true); + formatVersionField.set(node, 3); + + String dataPath = "file:///tmp/data-file.parquet"; + String deletePath = "file:///tmp/delete-shared.puffin"; + IcebergSplit split = new IcebergSplit(LocationPath.of(dataPath), 0, 128, 128, new String[0], + 3, Collections.emptyMap(), new ArrayList<>(), dataPath); + split.setTableFormatType(TableFormatType.ICEBERG); + split.setFirstRowId(10L); + split.setLastUpdatedSequenceNumber(20L); + split.setDeleteFileFilters(Collections.emptyList(), Collections.singletonList( + new IcebergDeleteFileFilter.DeletionVector(deletePath, -1L, -1L, 256L, + (long) Integer.MAX_VALUE + 5L, (long) Integer.MAX_VALUE + 7L))); + + Method method = IcebergScanNode.class.getDeclaredMethod("setIcebergParams", + TFileRangeDesc.class, IcebergSplit.class); + method.setAccessible(true); + + TFileRangeDesc rangeDesc = new TFileRangeDesc(); + method.invoke(node, rangeDesc, split); + + TIcebergDeleteFileDesc deleteFileDesc = rangeDesc.getTableFormatParams() + .getIcebergParams() + .getDeleteFiles() + .get(0); + Assert.assertEquals((long) Integer.MAX_VALUE + 5L, deleteFileDesc.getContentOffset()); + Assert.assertEquals((long) Integer.MAX_VALUE + 7L, deleteFileDesc.getContentSizeInBytes()); + } + + @Test + public void testSetIcebergParamsPropagatesPositionDeleteFileFormat() throws Exception { + SessionVariable sv = new SessionVariable(); + TestIcebergScanNode node = new TestIcebergScanNode(sv); + + Field formatVersionField = IcebergScanNode.class.getDeclaredField("formatVersion"); + formatVersionField.setAccessible(true); + formatVersionField.set(node, 2); + + String dataPath = "file:///tmp/data-file.parquet"; + String deletePath = "file:///tmp/delete-file.orc"; + IcebergSplit split = new IcebergSplit(LocationPath.of(dataPath), 0, 128, 128, new String[0], + 2, Collections.emptyMap(), new ArrayList<>(), dataPath); + split.setTableFormatType(TableFormatType.ICEBERG); + split.setDeleteFileFilters(Collections.emptyList(), Collections.singletonList( + new IcebergDeleteFileFilter.PositionDelete(deletePath, -1L, -1L, 256L, + org.apache.iceberg.FileFormat.ORC))); + + Method method = IcebergScanNode.class.getDeclaredMethod("setIcebergParams", + TFileRangeDesc.class, IcebergSplit.class); + method.setAccessible(true); + + TFileRangeDesc rangeDesc = new TFileRangeDesc(); + method.invoke(node, rangeDesc, split); + + TIcebergDeleteFileDesc deleteFileDesc = rangeDesc.getTableFormatParams() + .getIcebergParams() + .getDeleteFiles() + .get(0); + Assert.assertEquals(org.apache.doris.thrift.TFileFormatType.FORMAT_ORC, deleteFileDesc.getFileFormat()); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/IcebergDeleteCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/IcebergDeleteCommandTest.java index 348a1333cc1f84..f6874c2705d334 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/IcebergDeleteCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/IcebergDeleteCommandTest.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.StructType; import org.apache.doris.datasource.iceberg.IcebergRowId; +import org.apache.doris.qe.ConnectContext; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -59,4 +60,32 @@ public void testRowIdStructFields() { StructType structType = (StructType) IcebergRowId.getRowIdType(); Assertions.assertEquals(4, structType.getFields().size()); } + + @Test + public void testExecuteWithExternalTableBatchModeDisabledRestoresValueOnSuccess() throws Exception { + ConnectContext ctx = new ConnectContext(); + ctx.getSessionVariable().enableExternalTableBatchMode = true; + + IcebergDeleteCommand.executeWithExternalTableBatchModeDisabled(ctx, () -> { + Assertions.assertFalse(ctx.getSessionVariable().enableExternalTableBatchMode); + return null; + }); + + Assertions.assertTrue(ctx.getSessionVariable().enableExternalTableBatchMode); + } + + @Test + public void testExecuteWithExternalTableBatchModeDisabledRestoresValueOnException() { + ConnectContext ctx = new ConnectContext(); + ctx.getSessionVariable().enableExternalTableBatchMode = false; + + RuntimeException exception = Assertions.assertThrows(RuntimeException.class, + () -> IcebergDeleteCommand.executeWithExternalTableBatchModeDisabled(ctx, () -> { + Assertions.assertFalse(ctx.getSessionVariable().enableExternalTableBatchMode); + throw new RuntimeException("expected"); + })); + + Assertions.assertEquals("expected", exception.getMessage()); + Assertions.assertFalse(ctx.getSessionVariable().enableExternalTableBatchMode); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/IcebergMergeCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/IcebergMergeCommandTest.java new file mode 100644 index 00000000000000..531e8e30855a2e --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/IcebergMergeCommandTest.java @@ -0,0 +1,55 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.qe.ConnectContext; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class IcebergMergeCommandTest { + + @Test + public void testExecuteWithExternalTableBatchModeDisabledRestoresValueOnSuccess() throws Exception { + ConnectContext ctx = new ConnectContext(); + ctx.getSessionVariable().enableExternalTableBatchMode = true; + + Boolean result = IcebergMergeCommand.executeWithExternalTableBatchModeDisabled(ctx, () -> { + Assertions.assertFalse(ctx.getSessionVariable().enableExternalTableBatchMode); + return Boolean.TRUE; + }); + + Assertions.assertTrue(result); + Assertions.assertTrue(ctx.getSessionVariable().enableExternalTableBatchMode); + } + + @Test + public void testExecuteWithExternalTableBatchModeDisabledRestoresValueOnException() { + ConnectContext ctx = new ConnectContext(); + ctx.getSessionVariable().enableExternalTableBatchMode = false; + + RuntimeException exception = Assertions.assertThrows(RuntimeException.class, + () -> IcebergMergeCommand.executeWithExternalTableBatchModeDisabled(ctx, () -> { + Assertions.assertFalse(ctx.getSessionVariable().enableExternalTableBatchMode); + throw new RuntimeException("expected"); + })); + + Assertions.assertEquals("expected", exception.getMessage()); + Assertions.assertFalse(ctx.getSessionVariable().enableExternalTableBatchMode); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/IcebergUpdateCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/IcebergUpdateCommandTest.java index 19460993b971e7..44482530e9684d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/IcebergUpdateCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/IcebergUpdateCommandTest.java @@ -134,4 +134,33 @@ public void testBuildUpdateSelectItemsSkipsHiddenColumns() { slot.getNameParts().get(slot.getNameParts().size() - 1))); Assertions.assertTrue(hasC2Slot); } + + @Test + public void testExecuteWithExternalTableBatchModeDisabledRestoresValueOnSuccess() throws Exception { + ConnectContext ctx = new ConnectContext(); + ctx.getSessionVariable().enableExternalTableBatchMode = true; + + Boolean result = IcebergUpdateCommand.executeWithExternalTableBatchModeDisabled(ctx, () -> { + Assertions.assertFalse(ctx.getSessionVariable().enableExternalTableBatchMode); + return Boolean.TRUE; + }); + + Assertions.assertTrue(result); + Assertions.assertTrue(ctx.getSessionVariable().enableExternalTableBatchMode); + } + + @Test + public void testExecuteWithExternalTableBatchModeDisabledRestoresValueOnException() { + ConnectContext ctx = new ConnectContext(); + ctx.getSessionVariable().enableExternalTableBatchMode = false; + + RuntimeException exception = Assertions.assertThrows(RuntimeException.class, + () -> IcebergUpdateCommand.executeWithExternalTableBatchModeDisabled(ctx, () -> { + Assertions.assertFalse(ctx.getSessionVariable().enableExternalTableBatchMode); + throw new RuntimeException("expected"); + })); + + Assertions.assertEquals("expected", exception.getMessage()); + Assertions.assertFalse(ctx.getSessionVariable().enableExternalTableBatchMode); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergDeleteExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergDeleteExecutorTest.java new file mode 100644 index 00000000000000..8b2f186d1f2ac1 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergDeleteExecutorTest.java @@ -0,0 +1,175 @@ +// 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.nereids.trees.plans.commands.insert; + +import org.apache.doris.analysis.DescriptorTable; +import org.apache.doris.analysis.TupleDescriptor; +import org.apache.doris.analysis.TupleId; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.datasource.CatalogProperty; +import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.datasource.iceberg.IcebergTransaction; +import org.apache.doris.datasource.iceberg.source.IcebergScanNode; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.planner.BaseExternalTableDataSink; +import org.apache.doris.planner.IcebergDeleteSink; +import org.apache.doris.planner.PlanNodeId; +import org.apache.doris.planner.ScanContext; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.SessionVariable; +import org.apache.doris.thrift.TDataSink; +import org.apache.doris.thrift.TIcebergDeleteFileDesc; +import org.apache.doris.thrift.TUniqueId; +import org.apache.doris.transaction.TransactionManager; + +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.lang.reflect.Field; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class IcebergDeleteExecutorTest { + + private static class TestIcebergScanNode extends IcebergScanNode { + TestIcebergScanNode() { + super(new PlanNodeId(0), new TupleDescriptor(new TupleId(0)), new SessionVariable(), ScanContext.EMPTY); + } + } + + @Test + public void testFinalizeSinkAndBeforeExecPropagateRewritableDeleteMetadata() throws Exception { + String referencedDataFile = "file:///tmp/data.parquet"; + DeleteFile deleteFile = FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath("file:///tmp/delete.puffin") + .withFormat(FileFormat.PUFFIN) + .withFileSizeInBytes(128L) + .withRecordCount(3L) + .withReferencedDataFile(referencedDataFile) + .withContentOffset(32L) + .withContentSizeInBytes(64L) + .build(); + TIcebergDeleteFileDesc deleteFileDesc = new TIcebergDeleteFileDesc(); + deleteFileDesc.setPath("file:///tmp/delete.puffin"); + + TestIcebergScanNode scanNode = new TestIcebergScanNode(); + scanNode.deleteFilesByReferencedDataFile.put(referencedDataFile, Collections.singletonList(deleteFile)); + scanNode.deleteFilesDescByReferencedDataFile.put(referencedDataFile, Collections.singletonList(deleteFileDesc)); + + IcebergTransaction transaction = Mockito.mock(IcebergTransaction.class); + TransactionManager transactionManager = Mockito.mock(TransactionManager.class); + Mockito.when(transactionManager.getTransaction(10L)).thenReturn(transaction); + + IcebergExternalTable table = mockIcebergExternalTable(3, transactionManager); + NereidsPlanner planner = mockPlanner(Collections.singletonList(scanNode)); + ConnectContext ctx = new ConnectContext(); + ctx.setQueryId(new TUniqueId(1L, 2L)); + ctx.getSessionVariable().setEnableNereidsDistributePlanner(false); + ctx.setThreadLocalInfo(); + + IcebergDeleteExecutor executor = new IcebergDeleteExecutor(ctx, table, "label", planner, false, -1L); + IcebergDeleteSink sink = new IcebergDeleteSink(table, new org.apache.doris.nereids.trees.plans.commands.delete.DeleteCommandContext()); + + executor.finalizeSinkForDelete(null, sink, null); + TDataSink tDataSink = getTDataSink(sink); + Assertions.assertEquals(1, tDataSink.getIcebergDeleteSink().getRewritableDeleteFileSetsSize()); + Assertions.assertEquals(referencedDataFile, + tDataSink.getIcebergDeleteSink().getRewritableDeleteFileSets().get(0).getReferencedDataFilePath()); + + executor.txnId = 10L; + executor.beforeExec(); + + Mockito.verify(transaction).beginDelete(table); + ArgumentCaptor>> deleteFilesCaptor = ArgumentCaptor.forClass(Map.class); + Mockito.verify(transaction).setRewrittenDeleteFilesByReferencedDataFile(deleteFilesCaptor.capture()); + Assertions.assertSame(deleteFile, deleteFilesCaptor.getValue().get(referencedDataFile).get(0)); + Mockito.verify(transaction).clearConflictDetectionFilter(); + } + + private static NereidsPlanner mockPlanner(List scanNodes) { + NereidsPlanner planner = Mockito.mock(NereidsPlanner.class); + Mockito.when(planner.getFragments()).thenReturn(Collections.emptyList()); + Mockito.when(planner.getScanNodes()).thenReturn(scanNodes); + Mockito.when(planner.getDescTable()).thenReturn(new DescriptorTable()); + Mockito.when(planner.getRuntimeFilters()).thenReturn(Collections.emptyList()); + Mockito.when(planner.getTopnFilters()).thenReturn(Collections.emptyList()); + return planner; + } + + private static IcebergExternalTable mockIcebergExternalTable(int formatVersion, + TransactionManager transactionManager) { + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map properties = new HashMap<>(); + properties.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + properties.put(TableProperties.DEFAULT_FILE_FORMAT, "parquet"); + properties.put(TableProperties.PARQUET_COMPRESSION, "snappy"); + properties.put(TableProperties.WRITE_DATA_LOCATION, "file:///tmp/iceberg_tbl/data"); + + Table icebergTable = Mockito.mock(Table.class); + Mockito.when(icebergTable.properties()).thenReturn(properties); + Mockito.when(icebergTable.spec()).thenReturn(spec); + Mockito.when(icebergTable.specs()).thenReturn(Collections.singletonMap(spec.specId(), spec)); + Mockito.when(icebergTable.location()).thenReturn("file:///tmp/iceberg_tbl"); + Mockito.when(icebergTable.schema()).thenReturn(schema); + Mockito.when(icebergTable.sortOrder()).thenReturn(SortOrder.unsorted()); + Mockito.when(icebergTable.name()).thenReturn("db.tbl"); + + CatalogProperty catalogProperty = Mockito.mock(CatalogProperty.class); + Mockito.when(catalogProperty.getMetastoreProperties()).thenReturn(null); + Mockito.when(catalogProperty.getStoragePropertiesMap()).thenReturn(Collections.emptyMap()); + + IcebergExternalCatalog catalog = Mockito.mock(IcebergExternalCatalog.class); + Mockito.when(catalog.getCatalogProperty()).thenReturn(catalogProperty); + Mockito.when(catalog.getTransactionManager()).thenReturn(transactionManager); + Mockito.when(catalog.getName()).thenReturn("iceberg"); + + DatabaseIf database = Mockito.mock(DatabaseIf.class); + Mockito.when(database.getId()).thenReturn(1L); + + IcebergExternalTable table = Mockito.mock(IcebergExternalTable.class); + Mockito.when(table.isView()).thenReturn(false); + Mockito.when(table.getCatalog()).thenReturn(catalog); + Mockito.when(table.getDatabase()).thenReturn(database); + Mockito.when(table.getDbName()).thenReturn("db"); + Mockito.when(table.getName()).thenReturn("tbl"); + Mockito.when(table.getIcebergTable()).thenReturn(icebergTable); + return table; + } + + private static TDataSink getTDataSink(BaseExternalTableDataSink sink) throws ReflectiveOperationException { + Field field = BaseExternalTableDataSink.class.getDeclaredField("tDataSink"); + field.setAccessible(true); + return (TDataSink) field.get(sink); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergMergeExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergMergeExecutorTest.java new file mode 100644 index 00000000000000..8015cf5feb3bb3 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergMergeExecutorTest.java @@ -0,0 +1,178 @@ +// 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.nereids.trees.plans.commands.insert; + +import org.apache.doris.analysis.DescriptorTable; +import org.apache.doris.analysis.TupleDescriptor; +import org.apache.doris.analysis.TupleId; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.datasource.CatalogProperty; +import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.datasource.iceberg.IcebergTransaction; +import org.apache.doris.datasource.iceberg.source.IcebergScanNode; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.planner.BaseExternalTableDataSink; +import org.apache.doris.planner.IcebergMergeSink; +import org.apache.doris.planner.PlanNodeId; +import org.apache.doris.planner.ScanContext; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.SessionVariable; +import org.apache.doris.thrift.TDataSink; +import org.apache.doris.thrift.TIcebergDeleteFileDesc; +import org.apache.doris.thrift.TUniqueId; +import org.apache.doris.transaction.TransactionManager; + +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.lang.reflect.Field; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class IcebergMergeExecutorTest { + + private static class TestIcebergScanNode extends IcebergScanNode { + TestIcebergScanNode() { + super(new PlanNodeId(0), new TupleDescriptor(new TupleId(0)), new SessionVariable(), ScanContext.EMPTY); + } + } + + @Test + public void testFinalizeSinkAndBeforeExecPropagateRewritableDeleteMetadata() throws Exception { + String referencedDataFile = "file:///tmp/data.parquet"; + DeleteFile deleteFile = FileMetadata.deleteFileBuilder(org.apache.iceberg.PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath("file:///tmp/delete.puffin") + .withFormat(FileFormat.PUFFIN) + .withFileSizeInBytes(128L) + .withRecordCount(3L) + .withReferencedDataFile(referencedDataFile) + .withContentOffset(32L) + .withContentSizeInBytes(64L) + .build(); + TIcebergDeleteFileDesc deleteFileDesc = new TIcebergDeleteFileDesc(); + deleteFileDesc.setPath("file:///tmp/delete.puffin"); + + TestIcebergScanNode scanNode = new TestIcebergScanNode(); + scanNode.deleteFilesByReferencedDataFile.put(referencedDataFile, Collections.singletonList(deleteFile)); + scanNode.deleteFilesDescByReferencedDataFile.put(referencedDataFile, Collections.singletonList(deleteFileDesc)); + + IcebergTransaction transaction = Mockito.mock(IcebergTransaction.class); + TransactionManager transactionManager = Mockito.mock(TransactionManager.class); + Mockito.when(transactionManager.getTransaction(11L)).thenReturn(transaction); + + IcebergExternalTable table = mockIcebergExternalTable(3, transactionManager); + NereidsPlanner planner = mockPlanner(Collections.singletonList(scanNode)); + ConnectContext ctx = new ConnectContext(); + ctx.setQueryId(new TUniqueId(3L, 4L)); + ctx.getSessionVariable().setEnableNereidsDistributePlanner(false); + ctx.setThreadLocalInfo(); + + IcebergMergeExecutor executor = new IcebergMergeExecutor(ctx, table, "label", planner, false, -1L); + IcebergMergeSink sink = new IcebergMergeSink(table, new org.apache.doris.nereids.trees.plans.commands.delete.DeleteCommandContext()); + + executor.finalizeSinkForMerge(null, sink, null); + TDataSink tDataSink = getTDataSink(sink); + Assertions.assertEquals(1, tDataSink.getIcebergMergeSink().getRewritableDeleteFileSetsSize()); + Assertions.assertEquals(referencedDataFile, + tDataSink.getIcebergMergeSink().getRewritableDeleteFileSets().get(0).getReferencedDataFilePath()); + + Expression conflictFilter = Expressions.equal("id", 1); + executor.setConflictDetectionFilter(java.util.Optional.of(conflictFilter)); + executor.txnId = 11L; + executor.beforeExec(); + + Mockito.verify(transaction).beginMerge(table); + ArgumentCaptor>> deleteFilesCaptor = ArgumentCaptor.forClass(Map.class); + Mockito.verify(transaction).setRewrittenDeleteFilesByReferencedDataFile(deleteFilesCaptor.capture()); + Assertions.assertSame(deleteFile, deleteFilesCaptor.getValue().get(referencedDataFile).get(0)); + Mockito.verify(transaction).setConflictDetectionFilter(conflictFilter); + } + + private static NereidsPlanner mockPlanner(List scanNodes) { + NereidsPlanner planner = Mockito.mock(NereidsPlanner.class); + Mockito.when(planner.getFragments()).thenReturn(Collections.emptyList()); + Mockito.when(planner.getScanNodes()).thenReturn(scanNodes); + Mockito.when(planner.getDescTable()).thenReturn(new DescriptorTable()); + Mockito.when(planner.getRuntimeFilters()).thenReturn(Collections.emptyList()); + Mockito.when(planner.getTopnFilters()).thenReturn(Collections.emptyList()); + return planner; + } + + private static IcebergExternalTable mockIcebergExternalTable(int formatVersion, + TransactionManager transactionManager) { + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + org.apache.iceberg.PartitionSpec spec = org.apache.iceberg.PartitionSpec.unpartitioned(); + Map properties = new HashMap<>(); + properties.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + properties.put(TableProperties.DEFAULT_FILE_FORMAT, "parquet"); + properties.put(TableProperties.PARQUET_COMPRESSION, "snappy"); + properties.put(TableProperties.WRITE_DATA_LOCATION, "file:///tmp/iceberg_tbl/data"); + + Table icebergTable = Mockito.mock(Table.class); + Mockito.when(icebergTable.properties()).thenReturn(properties); + Mockito.when(icebergTable.spec()).thenReturn(spec); + Mockito.when(icebergTable.specs()).thenReturn(Collections.singletonMap(spec.specId(), spec)); + Mockito.when(icebergTable.location()).thenReturn("file:///tmp/iceberg_tbl"); + Mockito.when(icebergTable.schema()).thenReturn(schema); + Mockito.when(icebergTable.sortOrder()).thenReturn(SortOrder.unsorted()); + Mockito.when(icebergTable.name()).thenReturn("db.tbl"); + + CatalogProperty catalogProperty = Mockito.mock(CatalogProperty.class); + Mockito.when(catalogProperty.getMetastoreProperties()).thenReturn(null); + Mockito.when(catalogProperty.getStoragePropertiesMap()).thenReturn(Collections.emptyMap()); + + IcebergExternalCatalog catalog = Mockito.mock(IcebergExternalCatalog.class); + Mockito.when(catalog.getCatalogProperty()).thenReturn(catalogProperty); + Mockito.when(catalog.getTransactionManager()).thenReturn(transactionManager); + Mockito.when(catalog.getName()).thenReturn("iceberg"); + + DatabaseIf database = Mockito.mock(DatabaseIf.class); + Mockito.when(database.getId()).thenReturn(1L); + + IcebergExternalTable table = Mockito.mock(IcebergExternalTable.class); + Mockito.when(table.isView()).thenReturn(false); + Mockito.when(table.getCatalog()).thenReturn(catalog); + Mockito.when(table.getDatabase()).thenReturn(database); + Mockito.when(table.getDbName()).thenReturn("db"); + Mockito.when(table.getName()).thenReturn("tbl"); + Mockito.when(table.getIcebergTable()).thenReturn(icebergTable); + return table; + } + + private static TDataSink getTDataSink(BaseExternalTableDataSink sink) throws ReflectiveOperationException { + Field field = BaseExternalTableDataSink.class.getDeclaredField("tDataSink"); + field.setAccessible(true); + return (TDataSink) field.get(sink); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScanTest.java new file mode 100644 index 00000000000000..865bba61e1f3ad --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScanTest.java @@ -0,0 +1,67 @@ +// 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.nereids.trees.plans.logical; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Type; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.datasource.iceberg.IcebergUtils; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +public class LogicalFileScanTest { + + @Test + public void testComputeOutputIncludesInvisibleRowLineageColumnsForIcebergTable() { + Column rowIdColumn = new Column(IcebergUtils.ICEBERG_ROW_ID_COL, Type.BIGINT, true); + rowIdColumn.setIsVisible(false); + Column lastUpdatedSequenceNumberColumn = + new Column(IcebergUtils.ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL, Type.BIGINT, true); + lastUpdatedSequenceNumberColumn.setIsVisible(false); + List schema = Arrays.asList( + new Column("id", Type.INT, true), + rowIdColumn, + lastUpdatedSequenceNumberColumn); + + IcebergExternalTable table = Mockito.mock(IcebergExternalTable.class); + Mockito.when(table.initSelectedPartitions(Mockito.any())).thenReturn(SelectedPartitions.NOT_PRUNED); + Mockito.when(table.getFullSchema()).thenReturn(schema); + Mockito.when(table.getName()).thenReturn("iceberg_tbl"); + + LogicalFileScan scan = new LogicalFileScan(new RelationId(1), table, + Collections.singletonList("db"), Collections.emptyList(), + Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); + + List outputNames = scan.computeOutput().stream().map(slot -> slot.getName()) + .collect(Collectors.toList()); + Assertions.assertEquals(Arrays.asList( + "id", + IcebergUtils.ICEBERG_ROW_ID_COL, + IcebergUtils.ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL), outputNames); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/IcebergDeleteSinkTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/IcebergDeleteSinkTest.java new file mode 100644 index 00000000000000..7a9ed79725377e --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/IcebergDeleteSinkTest.java @@ -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. + +package org.apache.doris.planner; + +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.datasource.CatalogProperty; +import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.nereids.trees.plans.commands.delete.DeleteCommandContext; +import org.apache.doris.thrift.TIcebergDeleteFileDesc; +import org.apache.doris.thrift.TIcebergDeleteSink; +import org.apache.doris.thrift.TIcebergRewritableDeleteFileSet; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +public class IcebergDeleteSinkTest { + + @Test + public void testBindDataSinkIncludesRewritableDeleteFileSetsForV3() throws Exception { + IcebergDeleteSink sink = new IcebergDeleteSink(mockIcebergExternalTable(3), new DeleteCommandContext()); + sink.setRewritableDeleteFileSets(Collections.singletonList(buildDeleteFileSet())); + + sink.bindDataSink(Optional.empty()); + + TIcebergDeleteSink thriftSink = sink.tDataSink.getIcebergDeleteSink(); + Assertions.assertEquals(3, thriftSink.getFormatVersion()); + Assertions.assertEquals(1, thriftSink.getRewritableDeleteFileSetsSize()); + Assertions.assertEquals("file:///tmp/data.parquet", + thriftSink.getRewritableDeleteFileSets().get(0).getReferencedDataFilePath()); + } + + @Test + public void testBindDataSinkSkipsRewritableDeleteFileSetsForV2() throws Exception { + IcebergDeleteSink sink = new IcebergDeleteSink(mockIcebergExternalTable(2), new DeleteCommandContext()); + sink.setRewritableDeleteFileSets(Collections.singletonList(buildDeleteFileSet())); + + sink.bindDataSink(Optional.empty()); + + TIcebergDeleteSink thriftSink = sink.tDataSink.getIcebergDeleteSink(); + Assertions.assertEquals(2, thriftSink.getFormatVersion()); + Assertions.assertFalse(thriftSink.isSetRewritableDeleteFileSets()); + } + + private static TIcebergRewritableDeleteFileSet buildDeleteFileSet() { + TIcebergDeleteFileDesc deleteFileDesc = new TIcebergDeleteFileDesc(); + deleteFileDesc.setPath("file:///tmp/delete.puffin"); + TIcebergRewritableDeleteFileSet deleteFileSet = new TIcebergRewritableDeleteFileSet(); + deleteFileSet.setReferencedDataFilePath("file:///tmp/data.parquet"); + deleteFileSet.setDeleteFiles(Collections.singletonList(deleteFileDesc)); + return deleteFileSet; + } + + private static IcebergExternalTable mockIcebergExternalTable(int formatVersion) { + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map properties = new HashMap<>(); + properties.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + properties.put(TableProperties.DEFAULT_FILE_FORMAT, "parquet"); + properties.put(TableProperties.PARQUET_COMPRESSION, "snappy"); + properties.put(TableProperties.WRITE_DATA_LOCATION, "file:///tmp/iceberg_tbl/data"); + + Table icebergTable = Mockito.mock(Table.class); + Mockito.when(icebergTable.properties()).thenReturn(properties); + Mockito.when(icebergTable.spec()).thenReturn(spec); + Mockito.when(icebergTable.specs()).thenReturn(Collections.singletonMap(spec.specId(), spec)); + Mockito.when(icebergTable.location()).thenReturn("file:///tmp/iceberg_tbl"); + Mockito.when(icebergTable.schema()).thenReturn(schema); + Mockito.when(icebergTable.sortOrder()).thenReturn(SortOrder.unsorted()); + Mockito.when(icebergTable.name()).thenReturn("db.tbl"); + + CatalogProperty catalogProperty = Mockito.mock(CatalogProperty.class); + Mockito.when(catalogProperty.getMetastoreProperties()).thenReturn(null); + Mockito.when(catalogProperty.getStoragePropertiesMap()).thenReturn(Collections.emptyMap()); + + IcebergExternalCatalog catalog = Mockito.mock(IcebergExternalCatalog.class); + Mockito.when(catalog.getCatalogProperty()).thenReturn(catalogProperty); + + DatabaseIf database = Mockito.mock(DatabaseIf.class); + IcebergExternalTable table = Mockito.mock(IcebergExternalTable.class); + Mockito.when(table.isView()).thenReturn(false); + Mockito.when(table.getCatalog()).thenReturn(catalog); + Mockito.when(table.getDatabase()).thenReturn(database); + Mockito.when(table.getDbName()).thenReturn("db"); + Mockito.when(table.getName()).thenReturn("tbl"); + Mockito.when(table.getIcebergTable()).thenReturn(icebergTable); + return table; + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/IcebergMergeSinkTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/IcebergMergeSinkTest.java new file mode 100644 index 00000000000000..23dcb4403ce731 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/IcebergMergeSinkTest.java @@ -0,0 +1,121 @@ +// 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.planner; + +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.datasource.CatalogProperty; +import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.datasource.iceberg.IcebergUtils; +import org.apache.doris.nereids.trees.plans.commands.delete.DeleteCommandContext; +import org.apache.doris.thrift.TIcebergDeleteFileDesc; +import org.apache.doris.thrift.TIcebergMergeSink; +import org.apache.doris.thrift.TIcebergRewritableDeleteFileSet; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +public class IcebergMergeSinkTest { + + @Test + public void testBindDataSinkIncludesRowLineageSchemaAndRewritableDeleteFileSetsForV3() throws Exception { + IcebergMergeSink sink = new IcebergMergeSink(mockIcebergExternalTable(3), new DeleteCommandContext()); + sink.setRewritableDeleteFileSets(Collections.singletonList(buildDeleteFileSet())); + + sink.bindDataSink(Optional.empty()); + + TIcebergMergeSink thriftSink = sink.tDataSink.getIcebergMergeSink(); + Assertions.assertEquals(3, thriftSink.getFormatVersion()); + Assertions.assertTrue(thriftSink.getSchemaJson().contains(IcebergUtils.ICEBERG_ROW_ID_COL)); + Assertions.assertTrue(thriftSink.getSchemaJson().contains( + IcebergUtils.ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL)); + Assertions.assertEquals(1, thriftSink.getRewritableDeleteFileSetsSize()); + } + + @Test + public void testBindDataSinkSkipsRewritableDeleteFileSetsAndRowLineageSchemaForV2() throws Exception { + IcebergMergeSink sink = new IcebergMergeSink(mockIcebergExternalTable(2), new DeleteCommandContext()); + sink.setRewritableDeleteFileSets(Collections.singletonList(buildDeleteFileSet())); + + sink.bindDataSink(Optional.empty()); + + TIcebergMergeSink thriftSink = sink.tDataSink.getIcebergMergeSink(); + Assertions.assertEquals(2, thriftSink.getFormatVersion()); + Assertions.assertFalse(thriftSink.isSetRewritableDeleteFileSets()); + Assertions.assertFalse(thriftSink.getSchemaJson().contains(IcebergUtils.ICEBERG_ROW_ID_COL)); + Assertions.assertFalse(thriftSink.getSchemaJson().contains( + IcebergUtils.ICEBERG_LAST_UPDATED_SEQUENCE_NUMBER_COL)); + } + + private static TIcebergRewritableDeleteFileSet buildDeleteFileSet() { + TIcebergDeleteFileDesc deleteFileDesc = new TIcebergDeleteFileDesc(); + deleteFileDesc.setPath("file:///tmp/delete.puffin"); + TIcebergRewritableDeleteFileSet deleteFileSet = new TIcebergRewritableDeleteFileSet(); + deleteFileSet.setReferencedDataFilePath("file:///tmp/data.parquet"); + deleteFileSet.setDeleteFiles(Collections.singletonList(deleteFileDesc)); + return deleteFileSet; + } + + private static IcebergExternalTable mockIcebergExternalTable(int formatVersion) { + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map properties = new HashMap<>(); + properties.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + properties.put(TableProperties.DEFAULT_FILE_FORMAT, "parquet"); + properties.put(TableProperties.PARQUET_COMPRESSION, "snappy"); + properties.put(TableProperties.WRITE_DATA_LOCATION, "file:///tmp/iceberg_tbl/data"); + + Table icebergTable = Mockito.mock(Table.class); + Mockito.when(icebergTable.properties()).thenReturn(properties); + Mockito.when(icebergTable.spec()).thenReturn(spec); + Mockito.when(icebergTable.specs()).thenReturn(Collections.singletonMap(spec.specId(), spec)); + Mockito.when(icebergTable.location()).thenReturn("file:///tmp/iceberg_tbl"); + Mockito.when(icebergTable.schema()).thenReturn(schema); + Mockito.when(icebergTable.sortOrder()).thenReturn(SortOrder.unsorted()); + Mockito.when(icebergTable.name()).thenReturn("db.tbl"); + + CatalogProperty catalogProperty = Mockito.mock(CatalogProperty.class); + Mockito.when(catalogProperty.getMetastoreProperties()).thenReturn(null); + Mockito.when(catalogProperty.getStoragePropertiesMap()).thenReturn(Collections.emptyMap()); + + IcebergExternalCatalog catalog = Mockito.mock(IcebergExternalCatalog.class); + Mockito.when(catalog.getCatalogProperty()).thenReturn(catalogProperty); + + DatabaseIf database = Mockito.mock(DatabaseIf.class); + IcebergExternalTable table = Mockito.mock(IcebergExternalTable.class); + Mockito.when(table.isView()).thenReturn(false); + Mockito.when(table.getCatalog()).thenReturn(catalog); + Mockito.when(table.getDatabase()).thenReturn(database); + Mockito.when(table.getDbName()).thenReturn("db"); + Mockito.when(table.getName()).thenReturn("tbl"); + Mockito.when(table.getIcebergTable()).thenReturn(icebergTable); + return table; + } +} diff --git a/gensrc/thrift/DataSinks.thrift b/gensrc/thrift/DataSinks.thrift index 7edfba9ab67924..89897fd4cf012f 100644 --- a/gensrc/thrift/DataSinks.thrift +++ b/gensrc/thrift/DataSinks.thrift @@ -397,7 +397,8 @@ struct THivePartitionUpdate { enum TFileContent { DATA = 0, POSITION_DELETES = 1, - EQUALITY_DELETES = 2 + EQUALITY_DELETES = 2, + DELETION_VECTOR = 3 } struct TIcebergColumnStats { @@ -411,6 +412,7 @@ struct TIcebergColumnStats { struct TIcebergCommitData { 1: optional string file_path + // File-level record count used to construct Iceberg data/delete file metadata. 2: optional i64 row_count 3: optional i64 file_size 4: optional TFileContent file_content @@ -425,6 +427,12 @@ struct TIcebergCommitData { 10: optional i32 partition_spec_id // Partition data JSON for delete files 11: optional string partition_data_json + // For deletion vector (V3): offset of the DV blob within the Puffin file + 12: optional i64 content_offset + // For deletion vector (V3): size of the DV blob in bytes + 13: optional i64 content_size_in_bytes + // Rows newly affected by the current statement for this commit item. + 14: optional i64 affected_rows } struct TSortField { @@ -433,6 +441,16 @@ struct TSortField { 3: optional bool null_first } +// Iceberg table sink write type: normal insert vs rewrite/compaction +enum TIcebergWriteType { + // Normal INSERT INTO + INSERT = 0, + // rewrite_data_files / compaction + REWRITE = 1, + // update / merge into + UPDATE = 2 +} + struct TIcebergTableSink { 1: optional string db_name 2: optional string tb_name @@ -453,12 +471,21 @@ struct TIcebergTableSink { // When set, BE should use these values directly instead of computing from data 15: optional map static_partition_values; 16: optional PlanNodes.TSortInfo sort_info; + 17: optional TIcebergWriteType write_type = TIcebergWriteType.INSERT; } +struct TIcebergRewritableDeleteFileSet { + // Data file being modified by the current statement + 1: optional string referenced_data_file_path + // old position delete file and old deletion vector that need to be merged into the new DV file. + 2: optional list delete_files +} + + struct TIcebergDeleteSink { 1: optional string db_name 2: optional string tb_name - 3: optional TFileContent delete_type // POSITION_DELETES or EQUALITY_DELETES + 3: optional TFileContent delete_type // POSITION_DELETES or EQUALITY_DELETES or DELETION_VECTOR 4: optional list equality_field_ids // For equality delete 5: optional PlanNodes.TFileFormatType file_format 6: optional PlanNodes.TFileCompressType compress_type @@ -469,6 +496,10 @@ struct TIcebergDeleteSink { 11: optional i32 partition_spec_id 12: optional string partition_data_json 13: optional list broker_addresses; + // Iceberg table format version (2 or 3), determines whether to write DVs + 14: optional i32 format_version + // Only for format_version >= 3. Existing live delete files that should be merged into the new DV. + 15: optional list rewritable_delete_file_sets } // Merge sink for Iceberg UPDATE: mix of position delete + data insert @@ -493,6 +524,10 @@ struct TIcebergMergeSink { 21: optional string table_location 22: optional i32 partition_spec_id_for_delete 23: optional string partition_data_json_for_delete + // Iceberg table format version (2 or 3), determines whether to write DVs + 24: optional i32 format_version + // Only for format_version >= 3. Existing live delete files that should be merged into the new DV. + 25: optional list rewritable_delete_file_sets } enum TDictLayoutType { diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index e32adb337da844..15545ac4f017a2 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -294,8 +294,9 @@ struct TIcebergDeleteFileDesc { 5: optional i32 content; // 6 & 7 : iceberg v3 deletion vector. // The content_offset and content_size_in_bytes fields are used to reference a specific blob for direct access to a deletion vector. - 6: optional i32 content_offset; - 7: optional i32 content_size_in_bytes; + 6: optional i64 content_offset; + 7: optional i64 content_size_in_bytes; + 8: optional TFileFormatType file_format; } struct TIcebergFileDesc { @@ -314,6 +315,10 @@ struct TIcebergFileDesc { 7: optional i64 row_count; 8: optional i32 partition_spec_id; 9: optional string partition_data_json; + // Only for format_version >= 3, the starting _row_id to assign to rows added by ADDED data files. + 10: optional i64 first_row_id; + // Only for format_version >= 3, the sequence number which last updated this file. + 11: optional i64 last_updated_sequence_number; } struct TPaimonDeletionFileDesc { diff --git a/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_v3_row_lineage_rewrite_data_files.groovy b/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_v3_row_lineage_rewrite_data_files.groovy new file mode 100644 index 00000000000000..438276c6946950 --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_v3_row_lineage_rewrite_data_files.groovy @@ -0,0 +1,244 @@ +// 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_v3_row_lineage_rewrite_data_files", "p0,external,iceberg,external_docker,external_docker_iceberg") { + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("Iceberg test is disabled") + return + } + + String catalogName = "test_iceberg_v3_row_lineage_rewrite_data_files" + String dbName = "test_row_lineage_rewrite_db" + String restPort = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minioPort = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String endpoint = "http://${externalEnvIp}:${minioPort}" + + def formats = ["parquet", "orc"] + + def schemaContainsField = { schemaRows, fieldName -> + String target = fieldName.toLowerCase() + return schemaRows.any { row -> row.toString().toLowerCase().contains(target) } + } + + def fileSchemaRows = { filePath, format -> + return sql(""" + desc function s3( + "uri" = "${filePath}", + "format" = "${format}", + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "${endpoint}", + "s3.region" = "us-east-1" + ) + """) + } + + def assertCurrentFilesContainRowLineageColumns = { tableName, format -> + def files = sql("""select file_path, lower(file_format) from ${tableName}\$files order by file_path""") + log.info("Checking rewritten files for physical row lineage columns in ${tableName}: ${files}") + assertTrue(files.size() > 0, "Current files should exist for ${tableName}") + files.each { row -> + assertEquals(format, row[1].toString()) + assertTrue(row[0].toString().endsWith(format == "parquet" ? ".parquet" : ".orc"), + "Current data file should match ${format} for ${tableName}, file=${row[0]}") + def schemaRows = fileSchemaRows(row[0].toString(), format) + log.info("Rewritten ${format} schema for ${tableName}, file=${row[0]} -> ${schemaRows}") + assertTrue(schemaContainsField(schemaRows, "_row_id"), + "Rewritten file should physically contain _row_id for ${tableName}, schema=${schemaRows}") + assertTrue(schemaContainsField(schemaRows, "_last_updated_sequence_number"), + "Rewritten file should physically contain _last_updated_sequence_number for ${tableName}, schema=${schemaRows}") + } + } + + def assertCurrentFilesDoNotContainRowLineageColumns = { tableName, format -> + def files = sql("""select file_path, lower(file_format) from ${tableName}\$files order by file_path""") + log.info("Checking regular INSERT files for absence of physical row lineage columns in ${tableName}: ${files}") + assertTrue(files.size() > 0, "Current files should exist for ${tableName}") + files.each { row -> + assertEquals(format, row[1].toString()) + assertTrue(row[0].toString().endsWith(format == "parquet" ? ".parquet" : ".orc"), + "Current data file should match ${format} for ${tableName}, file=${row[0]}") + def schemaRows = fileSchemaRows(row[0].toString(), format) + log.info("Regular INSERT ${format} schema for ${tableName}, file=${row[0]} -> ${schemaRows}") + assertTrue(!schemaContainsField(schemaRows, "_row_id"), + "Normal INSERT file should not contain _row_id for ${tableName}, schema=${schemaRows}") + assertTrue(!schemaContainsField(schemaRows, "_last_updated_sequence_number"), + "Normal INSERT file should not contain _last_updated_sequence_number for ${tableName}, schema=${schemaRows}") + } + } + + def lineageMap = { tableName -> + def rows = sql(""" + select id, _row_id, _last_updated_sequence_number + from ${tableName} + order by id + """) + Map> result = [:] + rows.each { row -> + result[row[0].toString().toInteger()] = [row[1].toString(), row[2].toString()] + } + log.info("Built lineage map for ${tableName}: ${result}") + return result + } + + def assertLineageMapEquals = { expected, actual, tableName -> + log.info("Comparing lineage maps for ${tableName}: expected=${expected}, actual=${actual}") + assertEquals(expected.size(), actual.size()) + expected.each { key, value -> + assertTrue(actual.containsKey(key), "Missing id=${key} after rewrite for ${tableName}") + assertEquals(value[0], actual[key][0]) + assertEquals(value[1], actual[key][1]) + } + } + + def runRewriteAndAssert = { tableName, format, expectedCount -> + def filesBefore = sql("""select file_path from ${tableName}\$files order by file_path""") + def snapshotsBefore = sql("""select snapshot_id from ${tableName}\$snapshots order by committed_at""") + log.info("Checking rewrite preconditions for ${tableName}: filesBefore=${filesBefore}, snapshotsBefore=${snapshotsBefore}") + assertTrue(filesBefore.size() >= 2, + "Rewrite test requires at least 2 input files for ${tableName}, but got ${filesBefore.size()}") + + def visibleBefore = sql("""select * from ${tableName} order by id""") + def rowLineageBefore = lineageMap(tableName) + log.info("Visible rows before rewrite for ${tableName}: ${visibleBefore}") + + assertCurrentFilesDoNotContainRowLineageColumns(tableName, format) + + def rewriteResult = sql(""" + alter table ${catalogName}.${dbName}.${tableName} + execute rewrite_data_files( + "target-file-size-bytes" = "10485760", + "min-input-files" = "1" + ) + """) + log.info("rewrite_data_files result for ${tableName}: ${rewriteResult}") + assertTrue(rewriteResult.size() > 0, "rewrite_data_files should return summary rows for ${tableName}") + int rewrittenFiles = rewriteResult[0][0] as int + assertTrue(rewrittenFiles > 0, "rewrite_data_files should rewrite at least one file for ${tableName}") + + def visibleAfter = sql("""select * from ${tableName} order by id""") + log.info("Visible rows after rewrite for ${tableName}: ${visibleAfter}") + assertEquals(visibleBefore, visibleAfter) + + def rowLineageAfter = lineageMap(tableName) + assertLineageMapEquals(rowLineageBefore, rowLineageAfter, tableName) + + def countAfter = sql("""select count(*) from ${tableName}""") + log.info("Checking row count after rewrite for ${tableName}: ${countAfter}") + assertEquals(expectedCount, countAfter[0][0].toString().toInteger()) + + def snapshotsAfter = sql("""select snapshot_id from ${tableName}\$snapshots order by committed_at""") + log.info("Snapshots after rewrite for ${tableName}: ${snapshotsAfter}") + assertTrue(snapshotsAfter.size() > snapshotsBefore.size(), + "rewrite_data_files should create a new snapshot for ${tableName}") + + assertCurrentFilesContainRowLineageColumns(tableName, format) + + def sampleRowId = rowLineageAfter.entrySet().iterator().next().value[0] + def sampleQuery = sql("""select count(*) from ${tableName} where _row_id = ${sampleRowId}""") + log.info("Checking sample _row_id predicate after rewrite for ${tableName}: sampleRowId=${sampleRowId}, result=${sampleQuery}") + assertEquals(1, sampleQuery[0][0].toString().toInteger()) + } + + sql """drop catalog if exists ${catalogName}""" + sql """ + create catalog if not exists ${catalogName} properties ( + "type" = "iceberg", + "iceberg.catalog.type" = "rest", + "uri" = "http://${externalEnvIp}:${restPort}", + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "${endpoint}", + "s3.region" = "us-east-1" + ) + """ + + sql """switch ${catalogName}""" + sql """create database if not exists ${dbName}""" + sql """use ${dbName}""" + sql """set enable_fallback_to_original_planner = false""" + sql """set show_hidden_columns = false""" + + try { + formats.each { format -> + String rewriteTable = "test_row_lineage_rewrite_unpartitioned_${format}" + String rewritePartitionTable = "test_row_lineage_rewrite_partitioned_${format}" + log.info("Run rewrite_data_files row lineage test with format ${format}") + + try { + sql """drop table if exists ${rewriteTable}""" + sql """ + create table ${rewriteTable} ( + id int, + name string, + score int + ) engine=iceberg + properties ( + "format-version" = "3", + "write.format.default" = "${format}" + ) + """ + + sql """insert into ${rewriteTable} values (1, 'A', 10), (2, 'B', 20)""" + sql """insert into ${rewriteTable} values (3, 'C', 30), (4, 'D', 40)""" + sql """insert into ${rewriteTable} values (5, 'E', 50), (6, 'F', 60)""" + log.info("Inserted three batches into ${rewriteTable} to prepare rewrite_data_files input files") + + // Assert baseline: + // 1. Data files from regular INSERT do not physically contain the two row lineage columns. + // 2. After rewrite_data_files, every current data file should contain both row lineage columns. + // 3. Visible query results stay unchanged before and after rewrite. + // 4. _row_id and _last_updated_sequence_number stay stable for every row across rewrite. + runRewriteAndAssert(rewriteTable, format, 6) + + sql """drop table if exists ${rewritePartitionTable}""" + sql """ + create table ${rewritePartitionTable} ( + id int, + name string, + score int, + dt date + ) engine=iceberg + partition by list (day(dt)) () + properties ( + "format-version" = "3", + "write.format.default" = "${format}" + ) + """ + + sql """insert into ${rewritePartitionTable} values (11, 'P1', 10, '2024-01-01'), (12, 'P2', 20, '2024-01-01')""" + sql """insert into ${rewritePartitionTable} values (13, 'P3', 30, '2024-01-01'), (14, 'P4', 40, '2024-02-01')""" + sql """insert into ${rewritePartitionTable} values (15, 'P5', 50, '2024-02-01'), (16, 'P6', 60, '2024-01-01')""" + log.info("Inserted three partitioned batches into ${rewritePartitionTable} to prepare rewrite_data_files input files") + + // Assert baseline: + // 1. Partitioned tables also write row lineage columns physically only during rewrite. + // 2. Business data and row lineage values stay stable before and after rewrite. + // 3. _row_id predicate queries remain available after rewrite. + runRewriteAndAssert(rewritePartitionTable, format, 6) + } finally { + sql """drop table if exists ${rewritePartitionTable}""" + sql """drop table if exists ${rewriteTable}""" + } + } + } finally { + sql """drop database if exists ${dbName} force""" + sql """drop catalog if exists ${catalogName}""" + } +} diff --git a/regression-test/suites/external_table_p0/iceberg/test_iceberg_v2_to_v3_doris_spark_compare.groovy b/regression-test/suites/external_table_p0/iceberg/test_iceberg_v2_to_v3_doris_spark_compare.groovy new file mode 100644 index 00000000000000..df6d1bbea20087 --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/test_iceberg_v2_to_v3_doris_spark_compare.groovy @@ -0,0 +1,223 @@ +// 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_v2_to_v3_doris_spark_compare", "p0,external,iceberg,external_docker,external_docker_iceberg") { + def enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("Iceberg test is disabled") + return + } + + def catalogName = "test_iceberg_v2_to_v3_doris_spark_compare" + def dbName = "test_v2_to_v3_doris_spark_compare_db" + def restPort = context.config.otherConfigs.get("iceberg_rest_uri_port") + def minioPort = context.config.otherConfigs.get("iceberg_minio_port") + def externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + def formats = ["parquet", "orc"] + + def tableNameForFormat = { baseName, format -> + return format == "parquet" ? baseName : "${baseName}_orc" + } + + sql """drop catalog if exists ${catalogName}""" + sql """ + create catalog if not exists ${catalogName} properties ( + "type" = "iceberg", + "iceberg.catalog.type" = "rest", + "uri" = "http://${externalEnvIp}:${restPort}", + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minioPort}", + "s3.region" = "us-east-1" + ) + """ + + sql """switch ${catalogName}""" + sql """use ${dbName}""" + sql """set enable_fallback_to_original_planner = false""" + + try { + def assertV2RowsAreNullAfterUpgrade = { tableName -> + def rows = sql """ + select id, _row_id, _last_updated_sequence_number + from ${tableName} + order by id + """ + assertEquals(2, rows.size()) + rows.each { row -> + assertTrue(row[1] == null, + "_row_id should be null for v2 rows after upgrade in ${tableName}, row=${row}") + assertTrue(row[2] == null, + "_last_updated_sequence_number should be null for v2 rows after upgrade in ${tableName}, row=${row}") + } + } + + def assertV23RowsNotNullAfterUpd = { tableName -> + def rows = sql """ + select id, _row_id, _last_updated_sequence_number + from ${tableName} + order by id + """ + rows.each { row -> + assertTrue(row[1] != null, + "_row_id should be non-null after Doris operator for ${tableName}") + assertTrue(row[2] != null, + "_last_updated_sequence_number should be non-null after Doris operator for ${tableName}") + + } + } + + def upgradeV3DorisOperationInsert = { tableName -> + assertV2RowsAreNullAfterUpgrade(tableName) + + sql """ + insert into ${tableName} values + (4, 'post_v3_i', 400, date '2024-01-04') + """ + + def rows = sql """ + select id, tag, score, _row_id, _last_updated_sequence_number + from ${tableName} + order by id + """ + assertEquals(3, rows.size()) + assertEquals(4, rows[2][0].toString().toInteger()) + assertEquals("post_v3_i", rows[2][1]) + assertV23RowsNotNullAfterUpd(tableName) + } + + def upgradeV3DorisOperationDelete = { tableName -> + assertV2RowsAreNullAfterUpgrade(tableName) + + sql """ + delete from ${tableName} + where id = 3 + """ + + def rows = sql """ + select id, tag, score + from ${tableName} + order by id + """ + assertEquals(1, rows.size()) + assertEquals(1, rows[0][0].toString().toInteger()) + assertV23RowsNotNullAfterUpd(tableName) + + } + + def upgradeV3DorisOperationUpdate = { tableName -> + assertV2RowsAreNullAfterUpgrade(tableName) + + sql """ + update ${tableName} + set tag = 'post_v3_u', score = score + 20 + where id = 1 + """ + + def rows = sql """ + select id, tag, score + from ${tableName} + order by id + """ + assertEquals(2, rows.size()) + assertEquals(1, rows[0][0].toString().toInteger()) + assertEquals("post_v3_u", rows[0][1]) + assertV23RowsNotNullAfterUpd(tableName) + } + + def upgradeV3DorisOperationRewrite = { tableName -> + assertV2RowsAreNullAfterUpgrade(tableName) + + def rewriteResult = sql(""" + alter table ${catalogName}.${dbName}.${tableName} + execute rewrite_data_files( + "target-file-size-bytes" = "10485760", + "min-input-files" = "1" + ) + """) + assertTrue(rewriteResult.size() > 0, + "rewrite_data_files should return summary rows for ${tableName}") + + def rowCount = sql """ + select count(*) + from ${tableName} + """ + assertEquals(2, rowCount[0][0].toString().toInteger()) + assertV23RowsNotNullAfterUpd(tableName) + } + + formats.each { format -> + def rowLineageNullTable = tableNameForFormat("v2v3_row_lineage_null_after_upgrade", format) + def sparkReferenceTable = tableNameForFormat("v2v3_spark_ops_reference", format) + def dorisTargetTable = tableNameForFormat("v2v3_doris_ops_target", format) + log.info("Run v2-to-v3 Doris/Spark compare test with format ${format}") + + def scenario1Rows = sql """ + select id, _row_id, _last_updated_sequence_number + from ${rowLineageNullTable} + order by id + """ + assertEquals(3, scenario1Rows.size()) + scenario1Rows.each { row -> + assertTrue(row[1] == null, + "_row_id should be null for rows written before v3 upgrade, row=${row}") + assertTrue(row[2] == null, + "_last_updated_sequence_number should be null for rows written before v3 upgrade, row=${row}") + } + + sql """ + update ${dorisTargetTable} + set tag = 'post_v3_u', score = score + 20 + where id = 2 + """ + + sql """ + insert into ${dorisTargetTable} values + (4, 'post_v3_i', 400, date '2024-02-04') + """ + + def dorisRewriteResult = sql(""" + alter table ${catalogName}.${dbName}.${dorisTargetTable} + execute rewrite_data_files( + "target-file-size-bytes" = "10485760", + "min-input-files" = "1" + ) + """) + assertTrue(dorisRewriteResult.size() > 0, + "Doris rewrite_data_files should return summary rows") + + check_sqls_result_equal """ + select * + from ${dorisTargetTable} + order by id + """, """ + select * + from ${sparkReferenceTable} + order by id + """ + + upgradeV3DorisOperationInsert(tableNameForFormat("v2v3_doris_upd_case1", format)) + upgradeV3DorisOperationDelete(tableNameForFormat("v2v3_doris_upd_case2", format)) + upgradeV3DorisOperationUpdate(tableNameForFormat("v2v3_doris_upd_case3", format)) + upgradeV3DorisOperationRewrite(tableNameForFormat("v2v3_doris_upd_case4", format)) + } + + } finally { + sql """drop catalog if exists ${catalogName}""" + } +} diff --git a/regression-test/suites/external_table_p0/iceberg/test_iceberg_v3_row_lineage_query_insert.groovy b/regression-test/suites/external_table_p0/iceberg/test_iceberg_v3_row_lineage_query_insert.groovy new file mode 100644 index 00000000000000..7276fadba76b2c --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/test_iceberg_v3_row_lineage_query_insert.groovy @@ -0,0 +1,304 @@ +// 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_v3_row_lineage_query_insert", "p0,external,iceberg,external_docker,external_docker_iceberg") { + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("Iceberg test is disabled") + return + } + + String catalogName = "test_iceberg_v3_row_lineage_query_insert" + String dbName = "test_row_lineage_query_insert_db" + String restPort = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minioPort = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String endpoint = "http://${externalEnvIp}:${minioPort}" + + def formats = ["parquet", "orc"] + + def collectDescColumns = { rows -> + return rows.collect { row -> row[0].toString().toLowerCase() } + } + + def schemaContainsField = { schemaRows, fieldName -> + String target = fieldName.toLowerCase() + return schemaRows.any { row -> row.toString().toLowerCase().contains(target) } + } + + def fileSchemaRows = { filePath, format -> + return sql(""" + desc function s3( + "uri" = "${filePath}", + "format" = "${format}", + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "${endpoint}", + "s3.region" = "us-east-1" + ) + """) + } + + def assertCurrentFilesDoNotContainRowLineageColumns = { tableName, format, messagePrefix -> + def files = sql("""select file_path, lower(file_format) from ${tableName}\$files order by file_path""") + log.info("${messagePrefix}: checking ${files.size()} current data files for ${tableName}: ${files}") + assertTrue(files.size() > 0, "Current data files should exist for ${tableName}") + files.each { row -> + assertEquals(format, row[1].toString()) + assertTrue(row[0].toString().endsWith(format == "parquet" ? ".parquet" : ".orc"), + "${messagePrefix} should write ${format} files for ${tableName}, file=${row[0]}") + def schemaRows = fileSchemaRows(row[0].toString(), format) + log.info("${messagePrefix}: ${format} schema for ${tableName}, file=${row[0]} -> ${schemaRows}") + assertTrue(!schemaContainsField(schemaRows, "_row_id"), + "${messagePrefix} should not physically write _row_id, schema=${schemaRows}") + assertTrue(!schemaContainsField(schemaRows, "_last_updated_sequence_number"), + "${messagePrefix} should not physically write _last_updated_sequence_number, schema=${schemaRows}") + } + } + + def assertRowLineageHiddenColumns = { tableName, visibleColumnCount -> + sql("""set show_hidden_columns = false""") + def descDefault = sql("""desc ${tableName}""") + def defaultColumns = collectDescColumns(descDefault) + log.info("Checking hidden-column default visibility for ${tableName}: desc=${descDefault}") + assertTrue(!defaultColumns.contains("_row_id"), + "DESC default should hide _row_id for ${tableName}, got ${defaultColumns}") + assertTrue(!defaultColumns.contains("_last_updated_sequence_number"), + "DESC default should hide _last_updated_sequence_number for ${tableName}, got ${defaultColumns}") + + def selectVisible = sql("""select * from ${tableName} order by id""") + log.info("Checking visible SELECT * layout for ${tableName}: rowCount=${selectVisible.size()}, firstRow=${selectVisible ? selectVisible[0] : 'EMPTY'}") + assertTrue(selectVisible.size() > 0, "SELECT * should return rows for ${tableName}") + assertEquals(visibleColumnCount, selectVisible[0].size()) + + sql("""set show_hidden_columns = true""") + def descHidden = sql("""desc ${tableName}""") + def hiddenColumns = collectDescColumns(descHidden) + log.info("Checking hidden-column enabled visibility for ${tableName}: desc=${descHidden}") + assertTrue(hiddenColumns.contains("_row_id"), + "DESC with show_hidden_columns=true should expose _row_id for ${tableName}, got ${hiddenColumns}") + assertTrue(hiddenColumns.contains("_last_updated_sequence_number"), + "DESC with show_hidden_columns=true should expose _last_updated_sequence_number for ${tableName}, got ${hiddenColumns}") + + def selectHidden = sql("""select * from ${tableName} order by id""") + log.info("Checking hidden SELECT * layout for ${tableName}: rowCount=${selectHidden.size()}, firstRow=${selectHidden ? selectHidden[0] : 'EMPTY'}") + assertTrue(selectHidden.size() > 0, "SELECT * with hidden columns should return rows for ${tableName}") + assertEquals(visibleColumnCount + 2 + 1, selectHidden[0].size()) // _row_id + _last_updated_sequence_number + __DORIS_ICEBERG_ROWID_COL__ + + sql("""set show_hidden_columns = false""") + } + + def assertExplicitRowLineageReadable = { tableName, expectedIds -> + def rowLineageRows = sql(""" + select id, _row_id, _last_updated_sequence_number + from ${tableName} + order by id + """) + log.info("Checking explicit row lineage projection for ${tableName}: rows=${rowLineageRows}") + assertEquals(expectedIds.size(), rowLineageRows.size()) + for (int i = 0; i < expectedIds.size(); i++) { + assertEquals(expectedIds[i], rowLineageRows[i][0].toString().toInteger()) + assertTrue(rowLineageRows[i][1] != null, + "_row_id should be non-null for ${tableName}, row=${rowLineageRows[i]}") + assertTrue(rowLineageRows[i][2] != null, + "_last_updated_sequence_number should be non-null for ${tableName}, row=${rowLineageRows[i]}") + } + + long firstRowId = rowLineageRows[0][1].toString().toLong() + long secondRowId = rowLineageRows[1][1].toString().toLong() + assertTrue(firstRowId < secondRowId, + "Row lineage ids should increase with row position for ${tableName}, rows=${rowLineageRows}") + + def byRowId = sql("""select id from ${tableName} where _row_id = ${firstRowId} order by id""") + log.info("Checking single _row_id predicate for ${tableName}: rowId=${firstRowId}, result=${byRowId}") + assertEquals(1, byRowId.size()) + assertEquals(expectedIds[0], byRowId[0][0].toString().toInteger()) + + def combinedPredicate = sql(""" + select id + from ${tableName} + where id >= ${expectedIds[1]} and _row_id in (${rowLineageRows[1][1]}, ${rowLineageRows[2][1]}) + order by id + """) + log.info("Checking combined business + _row_id predicate for ${tableName}: result=${combinedPredicate}") + assertEquals(2, combinedPredicate.size()) + assertEquals(expectedIds[1], combinedPredicate[0][0].toString().toInteger()) + assertEquals(expectedIds[2], combinedPredicate[1][0].toString().toInteger()) + } + + sql """drop catalog if exists ${catalogName}""" + sql """ + create catalog if not exists ${catalogName} properties ( + "type" = "iceberg", + "iceberg.catalog.type" = "rest", + "uri" = "http://${externalEnvIp}:${restPort}", + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "${endpoint}", + "s3.region" = "us-east-1" + ) + """ + + sql """switch ${catalogName}""" + sql """create database if not exists ${dbName}""" + sql """use ${dbName}""" + sql """set enable_fallback_to_original_planner = false""" + sql """set show_hidden_columns = false""" + + try { + formats.each { format -> + String unpartitionedTable = "test_row_lineage_query_insert_unpartitioned_${format}" + String partitionedTable = "test_row_lineage_query_insert_partitioned_${format}" + log.info("Run row lineage query/insert test with format ${format}") + + try { + sql """drop table if exists ${unpartitionedTable}""" + sql """ + create table ${unpartitionedTable} ( + id int, + name string, + age int + ) engine=iceberg + properties ( + "format-version" = "3", + "write.format.default" = "${format}" + ) + """ + + sql """ + insert into ${unpartitionedTable} values(1, 'Alice', 25); + """ + sql """ insert into ${unpartitionedTable} values(2, 'Bob', 30) """ + sql """ insert into ${unpartitionedTable} values(3, 'Charlie', 35) """ + + log.info("Inserted initial rows into ${unpartitionedTable}") + + // Assert baseline: + // 1. DESC and SELECT * hide row lineage columns by default. + // 2. show_hidden_columns=true exposes both hidden columns in DESC and SELECT *. + // 3. Explicit SELECT on row lineage columns returns non-null values. + assertRowLineageHiddenColumns(unpartitionedTable, 3) + assertExplicitRowLineageReadable(unpartitionedTable, [1, 2, 3]) + + test { + sql """insert into ${unpartitionedTable}(_row_id, id, name, age) values (1, 9, 'BadRow', 99)""" + exception "Cannot specify row lineage column '_row_id' in INSERT statement" + } + + test { + sql """ + insert into ${unpartitionedTable}(_last_updated_sequence_number, id, name, age) + values (1, 10, 'BadSeq', 100) + """ + exception "Cannot specify row lineage column '_last_updated_sequence_number' in INSERT statement" + } + + sql """insert into ${unpartitionedTable}(id, name, age) values (4, 'Doris', 40)""" + def unpartitionedCount = sql """select count(*) from ${unpartitionedTable}""" + log.info("Checking row count after regular INSERT for ${unpartitionedTable}: result=${unpartitionedCount}") + assertEquals(4, unpartitionedCount[0][0].toString().toInteger()) + + assertCurrentFilesDoNotContainRowLineageColumns( + unpartitionedTable, + format, + "Unpartitioned normal INSERT") + + sql """drop table if exists ${partitionedTable}""" + sql """ + create table ${partitionedTable} ( + id int, + name string, + age int, + dt date + ) engine=iceberg + partition by list (day(dt)) () + properties ( + "format-version" = "3", + "write.format.default" = "${format}" + ) + """ + + sql """ insert into ${partitionedTable} values(11, 'Penny', 21, '2024-01-01')""" + sql """ insert into ${partitionedTable} values(12, 'Quinn', 22, '2024-01-02')""" + sql """ insert into ${partitionedTable} values(13, 'Rita', 23, '2024-01-03')""" + + log.info("Inserted initial rows into ${partitionedTable}") + + // Assert baseline: + // 1. Partitioned tables follow the same row lineage semantics as unpartitioned tables. + // 2. Explicit SELECT on _row_id remains readable under partition predicates. + // 3. Regular INSERT still rejects hidden columns and does not write them physically. + assertRowLineageHiddenColumns(partitionedTable, 4) + + def partitionLineageRows = sql """ + select id, _row_id, _last_updated_sequence_number + from ${partitionedTable} + where dt >= '2024-01-01' + order by id + """ + log.info("Checking partitioned row lineage projection for ${partitionedTable}: rows=${partitionLineageRows}") + assertEquals(3, partitionLineageRows.size()) + partitionLineageRows.each { row -> + assertTrue(row[1] != null, "_row_id should be non-null for partitioned table row=${row}") + assertTrue(row[2] != null, "_last_updated_sequence_number should be non-null for partitioned table row=${row}") + } + + def exactPartitionPredicate = sql """ + select id + from ${partitionedTable} + where dt = '2024-01-02' and _row_id = ${partitionLineageRows[1][1]} + """ + log.info("Checking exact partition + _row_id predicate for ${partitionedTable}: result=${exactPartitionPredicate}") + assertEquals(1, exactPartitionPredicate.size()) + assertEquals(12, exactPartitionPredicate[0][0].toString().toInteger()) + + test { + sql """ + insert into ${partitionedTable}(_row_id, id, name, age, dt) + values (1, 14, 'BadPartitionRow', 24, '2024-01-04') + """ + exception "Cannot specify row lineage column '_row_id' in INSERT statement" + } + + test { + sql """ + insert into ${partitionedTable}(_last_updated_sequence_number, id, name, age, dt) + values (1, 15, 'BadPartitionSeq', 25, '2024-01-05') + """ + exception "Cannot specify row lineage column '_last_updated_sequence_number' in INSERT statement" + } + + sql """insert into ${partitionedTable}(id, name, age, dt) values (14, 'Sara', 24, '2024-01-04')""" + def partitionedCount = sql """select count(*) from ${partitionedTable}""" + log.info("Checking row count after regular INSERT for ${partitionedTable}: result=${partitionedCount}") + assertEquals(4, partitionedCount[0][0].toString().toInteger()) + + assertCurrentFilesDoNotContainRowLineageColumns( + partitionedTable, + format, + "Partitioned normal INSERT") + } finally { + sql """drop table if exists ${partitionedTable}""" + sql """drop table if exists ${unpartitionedTable}""" + } + } + } finally { + sql """set show_hidden_columns = false""" + sql """drop database if exists ${dbName} force""" + sql """drop catalog if exists ${catalogName}""" + } +} diff --git a/regression-test/suites/external_table_p0/iceberg/test_iceberg_v3_row_lineage_update_delete_merge.groovy b/regression-test/suites/external_table_p0/iceberg/test_iceberg_v3_row_lineage_update_delete_merge.groovy new file mode 100644 index 00000000000000..4bce7387f864bb --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/test_iceberg_v3_row_lineage_update_delete_merge.groovy @@ -0,0 +1,292 @@ +// 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_v3_row_lineage_update_delete_merge", "p0,external,iceberg,external_docker,external_docker_iceberg") { + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("Iceberg test is disabled") + return + } + + String catalogName = "test_iceberg_v3_row_lineage_update_delete_merge" + String dbName = "test_row_lineage_update_delete_merge_db" + String restPort = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minioPort = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String endpoint = "http://${externalEnvIp}:${minioPort}" + + def formats = ["parquet", "orc"] + + def schemaContainsField = { schemaRows, fieldName -> + String target = fieldName.toLowerCase() + return schemaRows.any { row -> row.toString().toLowerCase().contains(target) } + } + + def fileSchemaRows = { filePath, format -> + return sql(""" + desc function s3( + "uri" = "${filePath}", + "format" = "${format}", + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "${endpoint}", + "s3.region" = "us-east-1" + ) + """) + } + + def assertDeleteFilesArePuffin = { tableName -> + def deleteFiles = sql(""" + select file_path, lower(file_format) + from ${tableName}\$delete_files + order by file_path + """) + log.info("Checking delete files for ${tableName}: ${deleteFiles}") + assertTrue(deleteFiles.size() > 0, "V3 table ${tableName} should produce delete files") + deleteFiles.each { row -> + assertTrue(row[0].toString().endsWith(".puffin"), + "V3 delete file should be Puffin: ${row}") + assertEquals("puffin", row[1].toString()) + } + } + + def assertAtLeastOneCurrentDataFileHasRowLineageColumns = { tableName, format -> + def currentFiles = sql("""select file_path, lower(file_format) from ${tableName}\$data_files order by file_path""") + log.info("Checking current data files for physical row lineage columns in ${tableName}: ${currentFiles}") + assertTrue(currentFiles.size() > 0, "Current data files should exist for ${tableName}") + + boolean found = false + currentFiles.each { row -> + assertEquals(format, row[1].toString()) + assertTrue(row[0].toString().endsWith(format == "parquet" ? ".parquet" : ".orc"), + "Current data file should match ${format} for ${tableName}, file=${row[0]}") + def schemaRows = fileSchemaRows(row[0].toString(), format) + log.info("${format} schema for ${tableName}, file=${row[0]} -> ${schemaRows}") + if (schemaContainsField(schemaRows, "_row_id") + && schemaContainsField(schemaRows, "_last_updated_sequence_number")) { + found = true + } + } + assertTrue(found, "At least one current data file should physically contain row lineage columns for ${tableName}") + } + + def assertExplicitRowLineageNonNull = { tableName, expectedRowCount -> + def rows = sql(""" + select id, _row_id, _last_updated_sequence_number + from ${tableName} + order by id + """) + log.info("Checking explicit row lineage projection for ${tableName}: rows=${rows}") + assertEquals(expectedRowCount, rows.size()) + rows.each { row -> + assertTrue(row[1] != null, "_row_id should be non-null for ${tableName}, row=${row}") + assertTrue(row[2] != null, "_last_updated_sequence_number should be non-null for ${tableName}, row=${row}") + } + } + + def lineageMap = { tableName -> + def rows = sql(""" + select id, _row_id, _last_updated_sequence_number + from ${tableName} + order by id + """) + Map> result = [:] + rows.each { row -> + result[row[0].toString().toInteger()] = [row[1].toString(), row[2].toString()] + } + log.info("Built lineage map for ${tableName}: ${result}") + return result + } + + sql """drop catalog if exists ${catalogName}""" + sql """ + create catalog if not exists ${catalogName} properties ( + "type" = "iceberg", + "iceberg.catalog.type" = "rest", + "uri" = "http://${externalEnvIp}:${restPort}", + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "${endpoint}", + "s3.region" = "us-east-1" + ) + """ + + sql """switch ${catalogName}""" + sql """create database if not exists ${dbName}""" + sql """use ${dbName}""" + sql """set enable_fallback_to_original_planner = false""" + sql """set show_hidden_columns = false""" + + try { + formats.each { format -> + String updateDeleteTable = "test_row_lineage_v3_update_delete_${format}" + String mergeTable = "test_row_lineage_v3_merge_${format}" + log.info("Run row lineage update/delete/merge test with format ${format}") + + try { + sql """drop table if exists ${updateDeleteTable}""" + sql """ + create table ${updateDeleteTable} ( + id int, + name string, + age int + ) engine=iceberg + properties ( + "format-version" = "3", + "write.format.default" = "${format}" + ) + """ + + sql """insert into ${updateDeleteTable} values (1, 'Alice', 25) """ + sql """insert into ${updateDeleteTable} values (2, 'Bob', 30) """ + sql """insert into ${updateDeleteTable} values (3, 'Charlie', 35)""" + + def updateDeleteLineageBefore = lineageMap(updateDeleteTable) + log.info("Lineage before UPDATE/DELETE on ${updateDeleteTable}: ${updateDeleteLineageBefore}") + sql """update ${updateDeleteTable} set name = 'Alice_u', age = 26 where id = 1""" + sql """delete from ${updateDeleteTable} where id = 2""" + + // Assert baseline: + // 1. UPDATE keeps rows readable and applies the new values. + // 2. DELETE removes the target row. + // 3. V3 delete files use Puffin deletion vectors instead of delete_pos parquet/orc files. + // 4. Explicit row lineage reads remain non-null after DML. + def updateDeleteRows = sql """select * from ${updateDeleteTable} order by id""" + log.info("Checking table rows after UPDATE/DELETE on ${updateDeleteTable}: ${updateDeleteRows}") + assertEquals(2, updateDeleteRows.size()) + assertEquals(1, updateDeleteRows[0][0].toString().toInteger()) + assertEquals("Alice_u", updateDeleteRows[0][1]) + assertEquals(26, updateDeleteRows[0][2].toString().toInteger()) + assertEquals(3, updateDeleteRows[1][0].toString().toInteger()) + assertEquals("Charlie", updateDeleteRows[1][1]) + assertEquals(35, updateDeleteRows[1][2].toString().toInteger()) + + assertExplicitRowLineageNonNull(updateDeleteTable, 2) + def updateDeleteLineageAfter = lineageMap(updateDeleteTable) + log.info("Lineage after UPDATE/DELETE on ${updateDeleteTable}: ${updateDeleteLineageAfter}") + assertEquals(updateDeleteLineageBefore[1][0], updateDeleteLineageAfter[1][0]) + assertTrue(updateDeleteLineageBefore[1][1] != updateDeleteLineageAfter[1][1], + "UPDATE should change _last_updated_sequence_number for id=1") + assertTrue(updateDeleteLineageAfter[1][1].toLong() > updateDeleteLineageBefore[1][1].toLong(), + "UPDATE should advance _last_updated_sequence_number for id=1") + assertEquals(updateDeleteLineageBefore[3][0], updateDeleteLineageAfter[3][0]) + assertEquals(updateDeleteLineageBefore[3][1], updateDeleteLineageAfter[3][1]) + assertTrue(!updateDeleteLineageAfter.containsKey(2), "Deleted row id=2 should not remain after DELETE") + assertDeleteFilesArePuffin(updateDeleteTable) + assertAtLeastOneCurrentDataFileHasRowLineageColumns(updateDeleteTable, format) + + def minRowIdAfterUpdate = sql """ + select min(_row_id) + from ${updateDeleteTable} + """ + def rowIdFilterResult = sql """ + select count(*) + from ${updateDeleteTable} + where _row_id = ${minRowIdAfterUpdate[0][0]} + """ + log.info("Checking _row_id filter after UPDATE/DELETE on ${updateDeleteTable}: minRowId=${minRowIdAfterUpdate}, result=${rowIdFilterResult}") + assertEquals(1, rowIdFilterResult[0][0].toString().toInteger()) + + sql """drop table if exists ${mergeTable}""" + sql """ + create table ${mergeTable} ( + id int, + name string, + age int, + dt date + ) engine=iceberg + partition by list (day(dt)) () + properties ( + "format-version" = "3", + "write.format.default" = "${format}" + ) + """ + + sql """ insert into ${mergeTable} values (1, 'Penny', 21, '2024-01-01') """ + sql """ insert into ${mergeTable} values (2, 'Quinn', 22, '2024-01-02') """ + sql """ insert into ${mergeTable} values (3, 'Rita', 23, '2024-01-03') """ + + def mergeLineageBefore = lineageMap(mergeTable) + log.info("Lineage before MERGE on ${mergeTable}: ${mergeLineageBefore}") + sql """ + merge into ${mergeTable} t + using ( + select 1 as id, 'Penny_u' as name, 31 as age, date '2024-01-01' as dt, 'U' as flag + union all + select 2, 'Quinn', 22, date '2024-01-02', 'D' + union all + select 4, 'Sara', 24, date '2024-01-04', 'I' + ) s + on t.id = s.id + when matched and s.flag = 'D' then delete + when matched then update set + name = s.name, + age = s.age + when not matched then insert (id, name, age, dt) + values (s.id, s.name, s.age, s.dt) + """ + + // Assert baseline: + // 1. MERGE applies DELETE, UPDATE, and INSERT actions in one statement. + // 2. The partitioned MERGE still writes Puffin deletion vectors. + // 3. At least one current data file written by MERGE contains physical row lineage columns. + def mergeRows = sql """select * from ${mergeTable} order by id""" + log.info("Checking table rows after MERGE on ${mergeTable}: ${mergeRows}") + assertEquals(3, mergeRows.size()) + assertEquals(1, mergeRows[0][0].toString().toInteger()) + assertEquals("Penny_u", mergeRows[0][1]) + assertEquals(31, mergeRows[0][2].toString().toInteger()) + assertEquals(3, mergeRows[1][0].toString().toInteger()) + assertEquals("Rita", mergeRows[1][1]) + assertEquals(23, mergeRows[1][2].toString().toInteger()) + assertEquals(4, mergeRows[2][0].toString().toInteger()) + assertEquals("Sara", mergeRows[2][1]) + assertEquals(24, mergeRows[2][2].toString().toInteger()) + + assertExplicitRowLineageNonNull(mergeTable, 3) + def mergeLineageAfter = lineageMap(mergeTable) + log.info("Lineage after MERGE on ${mergeTable}: ${mergeLineageAfter}") + assertEquals(mergeLineageBefore[1][0], mergeLineageAfter[1][0]) + assertTrue(mergeLineageBefore[1][1] != mergeLineageAfter[1][1], + "MERGE UPDATE should change _last_updated_sequence_number for id=1") + assertTrue(mergeLineageAfter[1][1].toLong() > mergeLineageBefore[1][1].toLong(), + "MERGE UPDATE should advance _last_updated_sequence_number for id=1") + assertEquals(mergeLineageBefore[3][0], mergeLineageAfter[3][0]) + assertEquals(mergeLineageBefore[3][1], mergeLineageAfter[3][1]) + assertTrue(!mergeLineageAfter.containsKey(2), "MERGE DELETE should remove id=2") + assertDeleteFilesArePuffin(mergeTable) + assertAtLeastOneCurrentDataFileHasRowLineageColumns(mergeTable, format) + + def insertedRowLineage = sql """ + select _row_id, _last_updated_sequence_number + from ${mergeTable} + where id = 4 + """ + log.info("Checking inserted MERGE row lineage for ${mergeTable}: ${insertedRowLineage}") + assertEquals(1, insertedRowLineage.size()) + assertTrue(insertedRowLineage[0][0] != null, "Inserted MERGE row should get generated _row_id") + assertTrue(insertedRowLineage[0][1] != null, "Inserted MERGE row should get generated _last_updated_sequence_number") + } finally { + sql """drop table if exists ${mergeTable}""" + sql """drop table if exists ${updateDeleteTable}""" + } + } + } finally { + sql """drop database if exists ${dbName} force""" + sql """drop catalog if exists ${catalogName}""" + } +}