From 0a017665d4ec0b2fd81a91fc82f264ac4f2e5706 Mon Sep 17 00:00:00 2001 From: Letian Jiang Date: Thu, 2 Nov 2023 21:34:45 +0800 Subject: [PATCH 1/6] [Feature] Support dynamic partition pruning (#30319) Signed-off-by: Letian Jiang --- be/src/connector/connector.h | 6 +- be/src/connector/hive_connector.cpp | 86 ++++++++++------- be/src/connector/hive_connector.h | 2 + be/src/exec/hdfs_scanner.cpp | 19 ++-- be/src/exec/hdfs_scanner.h | 6 +- be/src/exec/jni_scanner.h | 2 +- .../pipeline/scan/connector_scan_operator.h | 2 +- be/src/exprs/runtime_filter_bank.cpp | 74 +++++++++++++++ be/src/exprs/runtime_filter_bank.h | 3 + be/src/formats/parquet/file_reader.cpp | 16 ---- be/test/exec/hdfs_scanner_test.cpp | 94 +++++++++---------- .../com/starrocks/qe/SessionVariable.java | 7 ++ gensrc/thrift/InternalService.thrift | 2 +- 13 files changed, 210 insertions(+), 109 deletions(-) diff --git a/be/src/connector/connector.h b/be/src/connector/connector.h index da0f19ffbaf99..cb88aaf48d0c5 100644 --- a/be/src/connector/connector.h +++ b/be/src/connector/connector.h @@ -20,6 +20,7 @@ #include "exec/pipeline/scan/morsel.h" #include "gen_cpp/InternalService_types.h" +#include "exprs/runtime_filter_bank.h" #include "gen_cpp/PlanNodes_types.h" #include "runtime/runtime_state.h" #include "storage/chunk_helper.h" @@ -69,7 +70,7 @@ class DataSource { _runtime_profile->add_info_string("DataSourceType", name()); } void set_predicates(const std::vector& predicates) { _conjunct_ctxs = predicates; } - void set_runtime_filters(const RuntimeFilterProbeCollector* runtime_filters) { _runtime_filters = runtime_filters; } + void set_runtime_filters(RuntimeFilterProbeCollector* runtime_filters) { _runtime_filters = runtime_filters; } void set_read_limit(const uint64_t limit) { _read_limit = limit; } void set_split_context(pipeline::ScanSplitContext* split_context) { _split_context = split_context; } Status parse_runtime_filters(RuntimeState* state); @@ -87,7 +88,8 @@ class DataSource { int64_t _read_limit = -1; // no limit bool _has_any_predicate = false; std::vector _conjunct_ctxs; - const RuntimeFilterProbeCollector* _runtime_filters = nullptr; + RuntimeFilterProbeCollector* _runtime_filters = nullptr; + RuntimeBloomFilterEvalContext runtime_bloom_filter_eval_context; RuntimeProfile* _runtime_profile = nullptr; TupleDescriptor* _tuple_desc = nullptr; pipeline::ScanSplitContext* _split_context = nullptr; diff --git a/be/src/connector/hive_connector.cpp b/be/src/connector/hive_connector.cpp index d873bf8b1f426..f8303f1b75ce8 100644 --- a/be/src/connector/hive_connector.cpp +++ b/be/src/connector/hive_connector.cpp @@ -111,8 +111,10 @@ Status HiveDataSource::open(RuntimeState* state) { } if (state->query_options().__isset.enable_connector_split_io_tasks) { _enable_split_tasks = state->query_options().enable_connector_split_io_tasks; + if (state->query_options().__isset.enable_dynamic_prune_scan_range) { + _enable_dynamic_prune_scan_range = state->query_options().enable_dynamic_prune_scan_range; } - + RETURN_IF_ERROR(_init_conjunct_ctxs(state)); _init_tuples_and_slots(state); _init_counter(state); @@ -171,44 +173,46 @@ Status HiveDataSource::_init_partition_values() { } const auto& partition_values = partition_desc->partition_key_value_evals(); - _partition_values = partition_values; - - if (_has_partition_conjuncts || _has_scan_range_indicate_const_column) { - ChunkPtr partition_chunk = ChunkHelper::new_chunk(_partition_slots, 1); - // append partition data - for (int i = 0; i < _partition_slots.size(); i++) { - SlotId slot_id = _partition_slots[i]->id(); - int partition_col_idx = _partition_index_in_hdfs_partition_columns[i]; - ASSIGN_OR_RETURN(auto partition_value_col, partition_values[partition_col_idx]->evaluate(nullptr)); - assert(partition_value_col->is_constant()); - auto* const_column = ColumnHelper::as_raw_column(partition_value_col); - const ColumnPtr& data_column = const_column->data_column(); - ColumnPtr& chunk_part_column = partition_chunk->get_column_by_slot_id(slot_id); - if (data_column->is_nullable()) { - chunk_part_column->append_default(); - } else { - chunk_part_column->append(*data_column, 0, 1); + _partition_values = partition_desc->partition_key_value_evals(); + + // init partition chunk + auto partition_chunk = std::make_shared(); + for (int i = 0; i < _partition_slots.size(); i++) { + SlotId slot_id = _partition_slots[i]->id(); + int partition_col_idx = _partition_index_in_hdfs_partition_columns[i]; + ASSIGN_OR_RETURN(auto partition_value_col, partition_values[partition_col_idx]->evaluate(nullptr)); + DCHECK(partition_value_col->is_constant()); + partition_chunk->append_column(partition_value_col, slot_id); + } + + // eval conjuncts and skip if no rows. + if (_has_scan_range_indicate_const_column) { + std::vector ctxs; + for (SlotId slotId : _scan_range.identity_partition_slot_ids) { + if (_conjunct_ctxs_by_slot.find(slotId) != _conjunct_ctxs_by_slot.end()) { + ctxs.insert(ctxs.end(), _conjunct_ctxs_by_slot.at(slotId).begin(), + _conjunct_ctxs_by_slot.at(slotId).end()); } } + RETURN_IF_ERROR(ExecNode::eval_conjuncts(ctxs, partition_chunk.get())); + } else if (_has_partition_conjuncts) { + RETURN_IF_ERROR(ExecNode::eval_conjuncts(_partition_conjunct_ctxs, partition_chunk.get())); + } - // eval conjuncts and skip if no rows. - if (_has_scan_range_indicate_const_column) { - std::vector ctxs; - for (SlotId slotId : _scan_range.identity_partition_slot_ids) { - if (_conjunct_ctxs_by_slot.find(slotId) != _conjunct_ctxs_by_slot.end()) { - ctxs.insert(ctxs.end(), _conjunct_ctxs_by_slot.at(slotId).begin(), - _conjunct_ctxs_by_slot.at(slotId).end()); - } - } - RETURN_IF_ERROR(ExecNode::eval_conjuncts(ctxs, partition_chunk.get())); - } else { - RETURN_IF_ERROR(ExecNode::eval_conjuncts(_partition_conjunct_ctxs, partition_chunk.get())); - } + if (!partition_chunk->has_rows()) { + _filter_by_eval_partition_conjuncts = true; + return Status::OK(); + } + if (_enable_dynamic_prune_scan_range && _runtime_filters) { + _init_rf_counters(); + _runtime_filters->evaluate_partial_chunk(partition_chunk.get(), runtime_bloom_filter_eval_context); if (!partition_chunk->has_rows()) { _filter_by_eval_partition_conjuncts = true; + return Status::OK(); } } + return Status::OK(); } @@ -453,6 +457,24 @@ void HiveDataSource::_init_counter(RuntimeState* state) { } } +void HiveDataSource::_init_rf_counters() { + auto* root = _runtime_profile; + if (runtime_bloom_filter_eval_context.join_runtime_filter_timer == nullptr) { + static const char* prefix = "DynamicPruneScanRange"; + ADD_COUNTER(root, prefix, TUnit::NONE); + runtime_bloom_filter_eval_context.join_runtime_filter_timer = + ADD_CHILD_TIMER(root, "JoinRuntimeFilterTime", prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_hash_timer = + ADD_CHILD_TIMER(root, "JoinRuntimeFilterHashTime", prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_input_counter = + ADD_CHILD_COUNTER(root, "JoinRuntimeFilterInputScanRanges", TUnit::UNIT, prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_output_counter = + ADD_CHILD_COUNTER(root, "JoinRuntimeFilterOutputScanRanges", TUnit::UNIT, prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_eval_counter = + ADD_CHILD_COUNTER(root, "JoinRuntimeFilterEvaluate", TUnit::UNIT, prefix); + } +} + Status HiveDataSource::_init_scanner(RuntimeState* state) { SCOPED_TIMER(_profile.open_file_timer); @@ -603,7 +625,7 @@ void HiveDataSource::close(RuntimeState* state) { if (!_scanner->has_split_tasks()) { COUNTER_UPDATE(_profile.scan_ranges_counter, 1); } - _scanner->close(state); + _scanner->close(); } Expr::close(_min_max_conjunct_ctxs, state); Expr::close(_partition_conjunct_ctxs, state); diff --git a/be/src/connector/hive_connector.h b/be/src/connector/hive_connector.h index c6c2035844a85..40b17168ff5ee 100644 --- a/be/src/connector/hive_connector.h +++ b/be/src/connector/hive_connector.h @@ -90,6 +90,7 @@ class HiveDataSource final : public DataSource { Status _decompose_conjunct_ctxs(RuntimeState* state); void _init_tuples_and_slots(RuntimeState* state); void _init_counter(RuntimeState* state); + void _init_rf_counters(); Status _init_partition_values(); Status _init_scanner(RuntimeState* state); @@ -111,6 +112,7 @@ class HiveDataSource final : public DataSource { int32_t _datacache_evict_probability = 0; bool _use_file_metacache = false; bool _enable_split_tasks = false; + bool _enable_dynamic_prune_scan_range = true; // ============ conjuncts ================= std::vector _min_max_conjunct_ctxs; diff --git a/be/src/exec/hdfs_scanner.cpp b/be/src/exec/hdfs_scanner.cpp index d07a83226461f..fd0cbfaa0bb56 100644 --- a/be/src/exec/hdfs_scanner.cpp +++ b/be/src/exec/hdfs_scanner.cpp @@ -193,24 +193,31 @@ Status HdfsScanner::open(RuntimeState* runtime_state) { return Status::OK(); } -void HdfsScanner::close(RuntimeState* runtime_state) noexcept { +void HdfsScanner::close() noexcept { VLOG_FILE << "close file success: " << _scanner_params.path << ", scan range = [" << _scanner_params.scan_range->offset << "," << (_scanner_params.scan_range->length + _scanner_params.scan_range->offset) << "], rows = " << _app_stats.rows_read; + if (!_runtime_state) { + return; + } + + DCHECK(!has_pending_token()); bool expect = false; if (!_closed.compare_exchange_strong(expect, true)) return; update_counter(); - do_close(runtime_state); + do_close(_runtime_state); _file.reset(nullptr); _mor_processor->close(_runtime_state); } -void HdfsScanner::finalize() { - if (_runtime_state != nullptr) { - close(_runtime_state); - } +void HdfsScanner::enter_pending_queue() { + _pending_queue_sw.start(); +} + +uint64_t HdfsScanner::exit_pending_queue() { + return _pending_queue_sw.reset(); } Status HdfsScanner::open_random_access_file() { diff --git a/be/src/exec/hdfs_scanner.h b/be/src/exec/hdfs_scanner.h index fa1fa72d53394..3732c0b42835c 100644 --- a/be/src/exec/hdfs_scanner.h +++ b/be/src/exec/hdfs_scanner.h @@ -20,6 +20,7 @@ #include "exec/mor_processor.h" #include "exprs/expr.h" #include "exprs/expr_context.h" +#include "exprs/runtime_filter_bank.h" #include "fs/fs.h" #include "io/cache_input_stream.h" #include "io/shared_buffered_input_stream.h" @@ -330,11 +331,10 @@ class HdfsScanner { HdfsScanner() = default; virtual ~HdfsScanner() = default; + Status init(RuntimeState* runtime_state, const HdfsScannerParams& scanner_params); Status open(RuntimeState* runtime_state); - void close(RuntimeState* runtime_state) noexcept; Status get_next(RuntimeState* runtime_state, ChunkPtr* chunk); - Status init(RuntimeState* runtime_state, const HdfsScannerParams& scanner_params); - void finalize(); + void close() noexcept; int64_t num_bytes_read() const { return _app_stats.bytes_read; } int64_t raw_rows_read() const { return _app_stats.raw_rows_read; } diff --git a/be/src/exec/jni_scanner.h b/be/src/exec/jni_scanner.h index 547f91d553354..6f21f10854772 100644 --- a/be/src/exec/jni_scanner.h +++ b/be/src/exec/jni_scanner.h @@ -34,7 +34,7 @@ class JniScanner : public HdfsScanner { JniScanner(std::string factory_class, std::map params) : _jni_scanner_params(std::move(params)), _jni_scanner_factory_class(std::move(factory_class)) {} - ~JniScanner() override { finalize(); } + ~JniScanner() override { close(); } [[nodiscard]] Status do_open(RuntimeState* runtime_state) override; void do_update_counter(HdfsScanProfile* profile) override; diff --git a/be/src/exec/pipeline/scan/connector_scan_operator.h b/be/src/exec/pipeline/scan/connector_scan_operator.h index 515cd01b8722a..1dd8dfb7e0715 100644 --- a/be/src/exec/pipeline/scan/connector_scan_operator.h +++ b/be/src/exec/pipeline/scan/connector_scan_operator.h @@ -155,7 +155,7 @@ class ConnectorChunkSource : public ChunkSource { const int64_t _limit; // -1: no limit const std::vector& _runtime_in_filters; - const RuntimeFilterProbeCollector* _runtime_bloom_filters; + RuntimeFilterProbeCollector* _runtime_bloom_filters; // copied from scan node and merge predicates from runtime filter. std::vector _conjunct_ctxs; diff --git a/be/src/exprs/runtime_filter_bank.cpp b/be/src/exprs/runtime_filter_bank.cpp index 861a4f19c2a98..69916c7ea9b5f 100644 --- a/be/src/exprs/runtime_filter_bank.cpp +++ b/be/src/exprs/runtime_filter_bank.cpp @@ -421,6 +421,63 @@ void RuntimeFilterProbeCollector::do_evaluate(Chunk* chunk, RuntimeBloomFilterEv } } } + +void RuntimeFilterProbeCollector::do_evaluate_partial_chunk(Chunk* partial_chunk, + RuntimeBloomFilterEvalContext& eval_context) { + auto& selection = eval_context.running_context.selection; + eval_context.running_context.use_merged_selection = false; + eval_context.running_context.compatibility = + _runtime_state->func_version() <= 3 || !_runtime_state->enable_pipeline_engine(); + + // since partial chunk is currently very lightweight (a bunch of const columns), use every runtime filter if possible + // without computing each rf's selectivity + for (auto kv : _descriptors) { + RuntimeFilterProbeDescriptor* rf_desc = kv.second; + const JoinRuntimeFilter* filter = rf_desc->runtime_filter(); + if (filter == nullptr || filter->always_true()) { + continue; + } + + auto is_existent_slot_ref = [&](ExprContext* expr) { + auto* root = expr->root(); + if (!root->is_slotref()) { + return false; + } + + auto* col_ref = down_cast(root); + return partial_chunk->is_slot_exist(col_ref->slot_id()); + }; + + auto* probe_expr = rf_desc->probe_expr_ctx(); + auto* partition_by_exprs = rf_desc->partition_by_expr_contexts(); + + bool can_use_rf_on_partial_chunk = is_existent_slot_ref(probe_expr); + for (auto* part_by_expr : *partition_by_exprs) { + can_use_rf_on_partial_chunk &= is_existent_slot_ref(part_by_expr); + } + + // skip runtime filter that references a non-existent column for the partial chunk + if (!can_use_rf_on_partial_chunk) { + continue; + } + + ColumnPtr column = EVALUATE_NULL_IF_ERROR(probe_expr, probe_expr->root(), partial_chunk); + // for colocate grf + compute_hash_values(partial_chunk, column.get(), rf_desc, eval_context); + filter->evaluate(column.get(), &eval_context.running_context); + + auto true_count = SIMD::count_nonzero(selection); + eval_context.run_filter_nums += 1; + + if (true_count == 0) { + partial_chunk->set_num_rows(0); + return; + } else { + partial_chunk->filter(selection); + } + } +} + void RuntimeFilterProbeCollector::init_counter() { _eval_context.join_runtime_filter_timer = ADD_TIMER(_runtime_profile, "JoinRuntimeFilterTime"); _eval_context.join_runtime_filter_hash_timer = ADD_TIMER(_runtime_profile, "JoinRuntimeFilterHashTime"); @@ -456,6 +513,23 @@ void RuntimeFilterProbeCollector::evaluate(Chunk* chunk, RuntimeBloomFilterEvalC } } +void RuntimeFilterProbeCollector::evaluate_partial_chunk(Chunk* partial_chunk, + RuntimeBloomFilterEvalContext& eval_context) { + if (_descriptors.empty()) return; + size_t before = partial_chunk->num_rows(); + if (before == 0) return; + + { + SCOPED_TIMER(eval_context.join_runtime_filter_timer); + eval_context.join_runtime_filter_input_counter->update(before); + eval_context.run_filter_nums = 0; + do_evaluate_partial_chunk(partial_chunk, eval_context); + size_t after = partial_chunk->num_rows(); + eval_context.join_runtime_filter_output_counter->update(after); + eval_context.join_runtime_filter_eval_counter->update(eval_context.run_filter_nums); + } +} + void RuntimeFilterProbeCollector::compute_hash_values(Chunk* chunk, Column* column, RuntimeFilterProbeDescriptor* rf_desc, RuntimeBloomFilterEvalContext& eval_context) { diff --git a/be/src/exprs/runtime_filter_bank.h b/be/src/exprs/runtime_filter_bank.h index f5f3b47d9c4c7..0fca1c32e49d9 100644 --- a/be/src/exprs/runtime_filter_bank.h +++ b/be/src/exprs/runtime_filter_bank.h @@ -216,6 +216,8 @@ class RuntimeFilterProbeCollector { RuntimeBloomFilterEvalContext& eval_context); void evaluate(Chunk* chunk); void evaluate(Chunk* chunk, RuntimeBloomFilterEvalContext& eval_context); + // evaluate partial chunk that may not contain slots referenced by runtime filter + void evaluate_partial_chunk(Chunk* partial_chunk, RuntimeBloomFilterEvalContext& eval_context); void add_descriptor(RuntimeFilterProbeDescriptor* desc); // accept RuntimeFilterCollector from parent node // which means parent node to push down runtime filter. @@ -247,6 +249,7 @@ class RuntimeFilterProbeCollector { // TODO: return a funcion call status void do_evaluate(Chunk* chunk); void do_evaluate(Chunk* chunk, RuntimeBloomFilterEvalContext& eval_context); + void do_evaluate_partial_chunk(Chunk* partial_chunk, RuntimeBloomFilterEvalContext& eval_context); // mapping from filter id to runtime filter descriptor. std::map _descriptors; int _wait_timeout_ms = 0; diff --git a/be/src/formats/parquet/file_reader.cpp b/be/src/formats/parquet/file_reader.cpp index 2388d696dd8fe..552bf20bad7dc 100644 --- a/be/src/formats/parquet/file_reader.cpp +++ b/be/src/formats/parquet/file_reader.cpp @@ -404,22 +404,6 @@ StatusOr FileReader::_filter_group(const tparquet::RowGroup& row_group) { if (discard) { return true; } - - // skip topn runtime filter, because it has taken effect on min/max filtering - // if row-group contains exactly one value(i.e. min_value = max_value), use bloom filter to test - if (!filter->always_true() && min_chunk->columns()[0]->equals(0, *max_chunk->columns()[0], 0)) { - ColumnPtr& chunk_part_column = min_chunk->columns()[0]; - JoinRuntimeFilter::RunningContext ctx; - ctx.use_merged_selection = false; - ctx.compatibility = false; - auto& selection = ctx.selection; - selection.assign(chunk_part_column->size(), 1); - filter->compute_hash({chunk_part_column.get()}, &ctx); - filter->evaluate(chunk_part_column.get(), &ctx); - if (selection[0] == 0) { - return true; - } - } } } diff --git a/be/test/exec/hdfs_scanner_test.cpp b/be/test/exec/hdfs_scanner_test.cpp index 9244e49e262f4..2a19bfa21a48c 100644 --- a/be/test/exec/hdfs_scanner_test.cpp +++ b/be/test/exec/hdfs_scanner_test.cpp @@ -250,7 +250,7 @@ TEST_F(HdfsScannerTest, TestParquetGetNext) { status = scanner->get_next(_runtime_state, &chunk); ASSERT_TRUE(status.is_end_of_file()); - scanner->close(_runtime_state); + scanner->close(); } // ========================= ORC SCANNER ============================ @@ -438,7 +438,7 @@ TEST_F(HdfsScannerTest, TestOrcGetNext) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 100); EXPECT_EQ(scanner->raw_rows_read(), 100); - scanner->close(_runtime_state); + scanner->close(); } TEST_F(HdfsScannerTest, TestOrcSkipFile) { @@ -622,7 +622,7 @@ TEST_F(HdfsScannerTest, TestOrcGetNextWithMinMaxFilterNoRows) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 0); EXPECT_EQ(scanner->raw_rows_read(), 0); - scanner->close(_runtime_state); + scanner->close(); } TEST_F(HdfsScannerTest, TestOrcGetNextWithMinMaxFilterRows1) { @@ -653,7 +653,7 @@ TEST_F(HdfsScannerTest, TestOrcGetNextWithMinMaxFilterRows1) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 100); EXPECT_EQ(scanner->raw_rows_read(), 100); - scanner->close(_runtime_state); + scanner->close(); } TEST_F(HdfsScannerTest, TestOrcGetNextWithMinMaxFilterRows2) { @@ -684,7 +684,7 @@ TEST_F(HdfsScannerTest, TestOrcGetNextWithMinMaxFilterRows2) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 100); EXPECT_EQ(scanner->raw_rows_read(), 100); - scanner->close(_runtime_state); + scanner->close(); } // ==================================================================================================== @@ -761,7 +761,7 @@ TEST_F(HdfsScannerTest, TestOrcGetNextWithDictFilter) { // since we use dict filter eval cache, we can do filter on orc cvb // so actually read rows is 1000. EXPECT_EQ(scanner->raw_rows_read(), 1000); - scanner->close(_runtime_state); + scanner->close(); } // ==================================================================================================== @@ -847,7 +847,7 @@ TEST_F(HdfsScannerTest, TestOrcGetNextWithDiffEncodeDictFilter) { // since we use dict filter eval cache, we can do filter on orc cvb // so actually read rows is 200. EXPECT_EQ(scanner->raw_rows_read(), 200); - scanner->close(_runtime_state); + scanner->close(); } // ==================================================================================================== @@ -940,7 +940,7 @@ TEST_F(HdfsScannerTest, TestOrcGetNextWithDatetimeMinMaxFilter) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 4640); EXPECT_EQ(scanner->raw_rows_read(), 4640); - scanner->close(_runtime_state); + scanner->close(); } // ==================================================================================================== @@ -1044,7 +1044,7 @@ TEST_F(HdfsScannerTest, TestOrcGetNextWithPaddingCharDictFilter) { // since we use dict filter eval cache, we can do filter on orc cvb // so actually read rows is 1000. EXPECT_EQ(scanner->raw_rows_read(), 1000); - scanner->close(_runtime_state); + scanner->close(); } // ==================================================================================================== @@ -1161,7 +1161,7 @@ TEST_F(HdfsScannerTest, TestOrcDecodeMinMaxDateTime) { status = scanner->open(_runtime_state); EXPECT_TRUE(status.ok()) << status.to_string(); READ_SCANNER_ROWS(scanner, c.exp); - scanner->close(_runtime_state); + scanner->close(); } } @@ -1309,7 +1309,7 @@ TEST_F(HdfsScannerTest, TestOrcZeroSizeStream) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 1); EXPECT_EQ(scanner->raw_rows_read(), 1); - scanner->close(_runtime_state); + scanner->close(); } /** @@ -1371,7 +1371,7 @@ TEST_F(HdfsScannerTest, TestOrcLazyLoad) { // Should be end of file in next read. EXPECT_TRUE(status.is_end_of_file()); - scanner->close(_runtime_state); + scanner->close(); } /** @@ -1492,7 +1492,7 @@ TEST_F(HdfsScannerTest, TestOrcBooleanConjunct) { EXPECT_EQ("[1]", chunk->debug_row(0)); - scanner->close(_runtime_state); + scanner->close(); } // ============================================================================= @@ -1545,7 +1545,7 @@ TEST_F(HdfsScannerTest, TestParquetCoalesceReadAcrossRowGroup) { READ_SCANNER_ROWS(scanner, 100000); - scanner->close(_runtime_state); + scanner->close(); } TEST_F(HdfsScannerTest, TestParquetRuntimeFilter) { @@ -1592,9 +1592,9 @@ TEST_F(HdfsScannerTest, TestParquetRuntimeFilter) { auto c = ColumnHelper::cast_to_raw(column); c->append(tc.max_value); c->append(tc.min_value); - RuntimeFilterHelper::fill_runtime_bloom_filter(column, LogicalType::TYPE_BIGINT, f, 0, false); + ASSERT_OK(RuntimeFilterHelper::fill_runtime_bloom_filter(column, LogicalType::TYPE_BIGINT, f, 0, false)); - rf_probe_desc.init(0, &probe_expr_ctx); + ASSERT_OK(rf_probe_desc.init(0, &probe_expr_ctx)); rf_probe_desc.set_runtime_filter(f); rf_collector.add_descriptor(&rf_probe_desc); param->runtime_filter_collector = &rf_collector; @@ -1607,7 +1607,7 @@ TEST_F(HdfsScannerTest, TestParquetRuntimeFilter) { READ_SCANNER_ROWS(scanner, tc.exp_rows); - scanner->close(_runtime_state); + scanner->close(); probe_expr_ctx.close(_runtime_state); } } @@ -1676,7 +1676,7 @@ TEST_F(HdfsScannerTest, TestParqueTypeMismatchDecodeMinMax) { status = scanner->open(_runtime_state); EXPECT_TRUE(!status.ok()); - scanner->close(_runtime_state); + scanner->close(); } // ============================================================================= @@ -1718,7 +1718,7 @@ TEST_F(HdfsScannerTest, TestCSVCompressed) { ASSERT_TRUE(status.ok()) << status.get_error_msg(); READ_SCANNER_ROWS(scanner, 100); - scanner->close(_runtime_state); + scanner->close(); } { auto* range = _create_scan_range(compressed_file, 0, 0); @@ -1734,7 +1734,7 @@ TEST_F(HdfsScannerTest, TestCSVCompressed) { ASSERT_TRUE(status.ok()) << status.get_error_msg(); READ_SCANNER_ROWS(scanner, 100); - scanner->close(_runtime_state); + scanner->close(); } { // compressed file with offset != 0 @@ -1804,7 +1804,7 @@ TEST_F(HdfsScannerTest, TestCSVCompressed) { uint64_t records = 0; READ_SCANNER_RETURN_ROWS(scanner, records); EXPECT_NE(records, 100); - scanner->close(_runtime_state); + scanner->close(); } } @@ -1833,7 +1833,7 @@ TEST_F(HdfsScannerTest, TestCSVWithDifferentLineDelimiter) { ASSERT_TRUE(status.ok()) << status.get_error_msg(); READ_SCANNER_ROWS(scanner, 5); - scanner->close(_runtime_state); + scanner->close(); } { // test line delimiter = \r @@ -1851,7 +1851,7 @@ TEST_F(HdfsScannerTest, TestCSVWithDifferentLineDelimiter) { ASSERT_TRUE(status.ok()) << status.get_error_msg(); READ_SCANNER_ROWS(scanner, 5); - scanner->close(_runtime_state); + scanner->close(); } } @@ -1884,7 +1884,7 @@ TEST_F(HdfsScannerTest, TestCSVSmall) { ASSERT_TRUE(status.ok()) << status.get_error_msg(); READ_SCANNER_ROWS(scanner, 2); - scanner->close(_runtime_state); + scanner->close(); } for (int offset = 10; offset < 20; offset++) { @@ -1906,7 +1906,7 @@ TEST_F(HdfsScannerTest, TestCSVSmall) { READ_SCANNER_RETURN_ROWS(scanner, records); - scanner->close(_runtime_state); + scanner->close(); }; read_range(0, offset); @@ -1937,7 +1937,7 @@ TEST_F(HdfsScannerTest, TestCSVCaseIgnore) { ASSERT_TRUE(status.ok()) << status.get_error_msg(); READ_SCANNER_ROWS(scanner, 2); - scanner->close(_runtime_state); + scanner->close(); } } @@ -1973,7 +1973,7 @@ TEST_F(HdfsScannerTest, TestCSVWithoutEndDelemeter) { ASSERT_TRUE(status.ok()) << status.get_error_msg(); READ_SCANNER_ROWS(scanner, 3); - scanner->close(_runtime_state); + scanner->close(); } } @@ -2005,7 +2005,7 @@ TEST_F(HdfsScannerTest, TestCSVWithWindowsEndDelemeter) { EXPECT_EQ("['hello']", chunk->debug_row(0)); EXPECT_EQ("['world']", chunk->debug_row(1)); EXPECT_EQ("['starrocks']", chunk->debug_row(2)); - scanner->close(_runtime_state); + scanner->close(); } } @@ -2036,7 +2036,7 @@ TEST_F(HdfsScannerTest, TestCSVWithUTFBOM) { EXPECT_EQ("['5c3ffda0d1d7']", chunk->debug_row(0)); EXPECT_EQ("['62ef51eae5d8']", chunk->debug_row(1)); - scanner->close(_runtime_state); + scanner->close(); } { @@ -2122,7 +2122,7 @@ TEST_F(HdfsScannerTest, TestCSVNewlyAddColumn) { EXPECT_EQ("['hello', '5', 'smith', NULL]", chunk->debug_row(0)); EXPECT_EQ("['world', '6', 'cruise', NULL]", chunk->debug_row(1)); - scanner->close(_runtime_state); + scanner->close(); } } @@ -2159,7 +2159,7 @@ TEST_F(HdfsScannerTest, TestCSVDifferentOrderColumn) { EXPECT_EQ("['smith', '5', 'hello', NULL]", chunk->debug_row(0)); EXPECT_EQ("['cruise', '6', 'world', NULL]", chunk->debug_row(1)); - scanner->close(_runtime_state); + scanner->close(); } } @@ -2208,7 +2208,7 @@ TEST_F(HdfsScannerTest, TestCSVWithStructMap) { EXPECT_EQ("[1, [NULL,NULL], [NULL,NULL,NULL]]", chunk->debug_row(0)); EXPECT_EQ("[2, [NULL], [NULL,NULL]]", chunk->debug_row(1)); - scanner->close(_runtime_state); + scanner->close(); } } @@ -2300,7 +2300,7 @@ TEST_F(HdfsScannerTest, TestParqueTypeMismatchInt96String) { status = scanner->open(_runtime_state); // parquet column reader: not supported convert from parquet `INT96` to `VARCHAR` EXPECT_TRUE(!status.ok()) << status.get_error_msg(); - scanner->close(_runtime_state); + scanner->close(); } // ============================================================================= @@ -2331,7 +2331,7 @@ TEST_F(HdfsScannerTest, TestCSVSingleColumnNullAndEmpty) { ASSERT_TRUE(status.ok()) << status.get_error_msg(); READ_SCANNER_ROWS(scanner, 3); - scanner->close(_runtime_state); + scanner->close(); } } @@ -2396,7 +2396,7 @@ TEST_F(HdfsScannerTest, TestParquetUppercaseFiledPredicate) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 1); EXPECT_EQ(scanner->raw_rows_read(), 100); - scanner->close(_runtime_state); + scanner->close(); } // ============================================================================= @@ -2463,7 +2463,7 @@ TEST_F(HdfsScannerTest, TestParquetArrayDecode) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 1500); EXPECT_EQ(scanner->raw_rows_read(), 1500); - scanner->close(_runtime_state); + scanner->close(); } // ============================================================================= @@ -2562,7 +2562,7 @@ TEST_F(HdfsScannerTest, TestParquetDictTwoPage) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 50); EXPECT_EQ(scanner->raw_rows_read(), 200); - scanner->close(_runtime_state); + scanner->close(); } // Test min-max logic when parquet file contains complex types @@ -2607,7 +2607,7 @@ TEST_F(HdfsScannerTest, TestMinMaxFilterWhenContainsComplexTypes) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 0); EXPECT_EQ(scanner->raw_rows_read(), 0); - scanner->close(_runtime_state); + scanner->close(); } // ======================================================= @@ -2740,7 +2740,7 @@ TEST_F(HdfsScannerTest, TestHudiMORArrayMapStruct) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 3); EXPECT_EQ(scanner->raw_rows_read(), 3); - scanner->close(_runtime_state); + scanner->close(); EXPECT_EQ(_debug_row_output, "[0, 'hello', NULL, NULL, {a:NULL,b:{'key1':10},c:{a:[10,20],b:{a:10,b:'world'}}}]\n"); } @@ -2832,7 +2832,7 @@ TEST_F(HdfsScannerTest, TestHudiMORArrayMapStruct2) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 1); EXPECT_EQ(scanner->raw_rows_read(), 1); - scanner->close(_runtime_state); + scanner->close(); EXPECT_EQ(_debug_row_output, "[1, 'hello', [10,20,30], {'key1':1,'key2':2}, {a:10,b:'world'}]\n"); } @@ -2860,7 +2860,7 @@ TEST_F(HdfsScannerTest, TestHudiMORArrayMapStruct2) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 1); EXPECT_EQ(scanner->raw_rows_read(), 1); - scanner->close(_runtime_state); + scanner->close(); EXPECT_EQ(_debug_row_output, "[{b:'world',a:10}]\n"); } @@ -2888,7 +2888,7 @@ TEST_F(HdfsScannerTest, TestHudiMORArrayMapStruct2) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 1); EXPECT_EQ(scanner->raw_rows_read(), 1); - scanner->close(_runtime_state); + scanner->close(); EXPECT_EQ(_debug_row_output, "[{B:NULL,a:10}]\n"); } @@ -2914,7 +2914,7 @@ TEST_F(HdfsScannerTest, TestHudiMORArrayMapStruct2) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 1); EXPECT_EQ(scanner->raw_rows_read(), 1); - scanner->close(_runtime_state); + scanner->close(); EXPECT_EQ(_debug_row_output, "[{a:10}]\n"); } @@ -2941,7 +2941,7 @@ TEST_F(HdfsScannerTest, TestHudiMORArrayMapStruct2) { EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 1); EXPECT_EQ(scanner->raw_rows_read(), 1); - scanner->close(_runtime_state); + scanner->close(); EXPECT_EQ(_debug_row_output, "[{'key1':NULL,'key2':NULL}]\n"); } @@ -3019,7 +3019,7 @@ TEST_F(HdfsScannerTest, TestParquetTimestampToDatetime) { EXPECT_EQ(scanner->raw_rows_read(), 4); EXPECT_EQ(_debug_row_output, "[3023-01-01 00:00:00]\n[2023-01-01 00:00:00]\n[1000-01-01 00:00:00]\n[1900-01-01 00:00:00]\n"); - scanner->close(_runtime_state); + scanner->close(); } TEST_F(HdfsScannerTest, TestParquetIcebergCaseSensitive) { @@ -3052,7 +3052,7 @@ TEST_F(HdfsScannerTest, TestParquetIcebergCaseSensitive) { READ_SCANNER_ROWS(scanner, 1); EXPECT_EQ(scanner->raw_rows_read(), 1); EXPECT_EQ(_debug_row_output, "[1]\n"); - scanner->close(_runtime_state); + scanner->close(); } TEST_F(HdfsScannerTest, TestParquetLZOFormat) { @@ -3073,7 +3073,7 @@ TEST_F(HdfsScannerTest, TestParquetLZOFormat) { status = scanner->open(_runtime_state); EXPECT_TRUE(status.ok()); READ_SCANNER_ROWS(scanner, 100000); - scanner->close(_runtime_state); + scanner->close(); } } // namespace starrocks diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java b/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java index 527e5daa39fd7..4dfad1ba9c9fb 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java @@ -443,7 +443,11 @@ public class SessionVariable implements Serializable, Writable, Cloneable { public static final String ENABLE_FILE_METACACHE = "enable_file_metacache"; public static final String HUDI_MOR_FORCE_JNI_READER = "hudi_mor_force_jni_reader"; +<<<<<<< HEAD public static final String PAIMON_FORCE_JNI_READER = "paimon_force_jni_reader"; +======= + public static final String ENABLE_DYNAMIC_PRUNE_SCAN_RANGE = "enable_dynamic_prune_scan_range"; +>>>>>>> 3def3c1d3d ([Feature] Support dynamic partition pruning (#30319)) public static final String IO_TASKS_PER_SCAN_OPERATOR = "io_tasks_per_scan_operator"; public static final String CONNECTOR_IO_TASKS_PER_SCAN_OPERATOR = "connector_io_tasks_per_scan_operator"; public static final String ENABLE_CONNECTOR_ADAPTIVE_IO_TASKS = "enable_connector_adaptive_io_tasks"; @@ -1442,6 +1446,8 @@ public String getCatalog() { @VariableMgr.VarAttr(name = DATACACHE_EVICT_PROBABILITY, flag = VariableMgr.INVISIBLE) private int datacacheEvictProbability = 100; + @VariableMgr.VarAttr(name = ENABLE_DYNAMIC_PRUNE_SCAN_RANGE) + private boolean enableDynamicPruneScanRange = true; @VariableMgr.VarAttr(name = IO_TASKS_PER_SCAN_OPERATOR) private int ioTasksPerScanOperator = 4; @@ -3608,6 +3614,7 @@ public TQueryOptions toThrift() { tResult.setHudi_mor_force_jni_reader(hudiMORForceJNIReader); tResult.setIo_tasks_per_scan_operator(ioTasksPerScanOperator); tResult.setConnector_io_tasks_per_scan_operator(connectorIoTasksPerScanOperator); + tResult.setEnable_dynamic_prune_scan_range(enableDynamicPruneScanRange); tResult.setUse_page_cache(usePageCache); tResult.setEnable_connector_adaptive_io_tasks(enableConnectorAdaptiveIoTasks); diff --git a/gensrc/thrift/InternalService.thrift b/gensrc/thrift/InternalService.thrift index 5b975df53234f..4383ea55d2b29 100644 --- a/gensrc/thrift/InternalService.thrift +++ b/gensrc/thrift/InternalService.thrift @@ -217,7 +217,7 @@ struct TQueryOptions { 86: optional i32 io_tasks_per_scan_operator = 4; 87: optional i32 connector_io_tasks_per_scan_operator = 16; 88: optional double runtime_filter_early_return_selectivity = 0.05; - + 89: optional bool enable_dynamic_prune_scan_range = true; 90: optional i64 log_rejected_record_num = 0; From 74dff46f39b823cfd745d38b05fe076cbae3d852 Mon Sep 17 00:00:00 2001 From: Letian Jiang Date: Tue, 6 Aug 2024 15:11:08 +0800 Subject: [PATCH 2/6] Enable dynamic partition pruning for non-slot-ref probe expr Signed-off-by: Letian Jiang --- be/src/exprs/runtime_filter_bank.cpp | 21 ++++++++++++--------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/be/src/exprs/runtime_filter_bank.cpp b/be/src/exprs/runtime_filter_bank.cpp index 69916c7ea9b5f..a2a12102fa9c4 100644 --- a/be/src/exprs/runtime_filter_bank.cpp +++ b/be/src/exprs/runtime_filter_bank.cpp @@ -438,22 +438,25 @@ void RuntimeFilterProbeCollector::do_evaluate_partial_chunk(Chunk* partial_chunk continue; } - auto is_existent_slot_ref = [&](ExprContext* expr) { - auto* root = expr->root(); - if (!root->is_slotref()) { - return false; + auto only_reference_existent_slots = [&](ExprContext* expr) { + std::vector slot_ids; + int n = expr->root()->get_slot_ids(&slot_ids); + DCHECK(slot_ids.size() == n); + + for (auto slot_id : slot_ids) { + if (!partial_chunk->is_slot_exist(slot_id)) { + return false; + } } - - auto* col_ref = down_cast(root); - return partial_chunk->is_slot_exist(col_ref->slot_id()); + return true; }; auto* probe_expr = rf_desc->probe_expr_ctx(); auto* partition_by_exprs = rf_desc->partition_by_expr_contexts(); - bool can_use_rf_on_partial_chunk = is_existent_slot_ref(probe_expr); + bool can_use_rf_on_partial_chunk = only_reference_existent_slots(probe_expr); for (auto* part_by_expr : *partition_by_exprs) { - can_use_rf_on_partial_chunk &= is_existent_slot_ref(part_by_expr); + can_use_rf_on_partial_chunk &= only_reference_existent_slots(part_by_expr); } // skip runtime filter that references a non-existent column for the partial chunk From 15eb0697dd5add829890f2b68af37adf74ab5272 Mon Sep 17 00:00:00 2001 From: Letian Jiang Date: Tue, 6 Aug 2024 17:25:02 +0800 Subject: [PATCH 3/6] test Signed-off-by: Letian Jiang --- be/src/exprs/runtime_filter_bank.cpp | 1 + test/sql/test_dpp/R/test_dpp | 75 ++++++++++++++++++++++++++++ test/sql/test_dpp/T/test_dpp | 29 +++++++++++ 3 files changed, 105 insertions(+) create mode 100644 test/sql/test_dpp/R/test_dpp create mode 100644 test/sql/test_dpp/T/test_dpp diff --git a/be/src/exprs/runtime_filter_bank.cpp b/be/src/exprs/runtime_filter_bank.cpp index a2a12102fa9c4..448573fc688d2 100644 --- a/be/src/exprs/runtime_filter_bank.cpp +++ b/be/src/exprs/runtime_filter_bank.cpp @@ -448,6 +448,7 @@ void RuntimeFilterProbeCollector::do_evaluate_partial_chunk(Chunk* partial_chunk return false; } } + return true; }; diff --git a/test/sql/test_dpp/R/test_dpp b/test/sql/test_dpp/R/test_dpp new file mode 100644 index 0000000000000..624a9a850cfb0 --- /dev/null +++ b/test/sql/test_dpp/R/test_dpp @@ -0,0 +1,75 @@ +-- name: test_dpp +create external catalog hive_catalog_${uuid0} PROPERTIES ("type"="hive", "hive.metastore.uris"="${hive_metastore_uris}"); +-- result: +[] +-- !result +create database hive_catalog_${uuid0}.hive_db_${uuid0}; +-- result: +[] +-- !result +create table hive_catalog_${uuid0}.hive_db_${uuid0}.fact (payload string, dt date) partition by(dt); +-- result: +[] +-- !result +insert into hive_catalog_${uuid0}.hive_db_${uuid0}.fact values ('A', date '2020-01-01'), ('B', date '2020-01-02'), ('C', date '2020-01-03'); +-- result: +[] +-- !result +create table hive_catalog_${uuid0}.hive_db_${uuid0}.dim (dt string); +-- result: +[] +-- !result +insert into hive_catalog_${uuid0}.hive_db_${uuid0}.dim values ("2020-01-01"); +-- result: +[] +-- !result +set disable_join_reorder = true; +-- result: +[] +-- !result +set enable_dynamic_prune_scan_range = true; +-- result: +[] +-- !result +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt = d.dt; +-- result: +A 2020-01-01 2020-01-01 +-- !result +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt + interval 1 day = d.dt; +-- result: +-- !result +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt - interval 1 day = d.dt; +-- result: +B 2020-01-02 2020-01-01 +-- !result +set enable_dynamic_prune_scan_range = false; +-- result: +[] +-- !result +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt = d.dt; +-- result: +A 2020-01-01 2020-01-01 +-- !result +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt + interval 1 day = d.dt; +-- result: +-- !result +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt - interval 1 day = d.dt; +-- result: +B 2020-01-02 2020-01-01 +-- !result +drop table hive_catalog_${uuid0}.hive_db_${uuid0}.fact force; +-- result: +[] +-- !result +drop table hive_catalog_${uuid0}.hive_db_${uuid0}.dim force; +-- result: +[] +-- !result +drop database hive_catalog_${uuid0}.hive_db_${uuid0}; +-- result: +[] +-- !result +drop catalog hive_catalog_${uuid0}; +-- result: +[] +-- !result \ No newline at end of file diff --git a/test/sql/test_dpp/T/test_dpp b/test/sql/test_dpp/T/test_dpp new file mode 100644 index 0000000000000..aa4802650c634 --- /dev/null +++ b/test/sql/test_dpp/T/test_dpp @@ -0,0 +1,29 @@ +-- name: test_dpp + +create external catalog hive_catalog_${uuid0} PROPERTIES ("type"="hive", "hive.metastore.uris"="${hive_metastore_uris}"); +create database hive_catalog_${uuid0}.hive_db_${uuid0}; + +create table hive_catalog_${uuid0}.hive_db_${uuid0}.fact (payload string, dt date) partition by(dt); +insert into hive_catalog_${uuid0}.hive_db_${uuid0}.fact values ('A', date '2020-01-01'), ('B', date '2020-01-02'), ('C', date '2020-01-03'); + +create table hive_catalog_${uuid0}.hive_db_${uuid0}.dim (dt string); +insert into hive_catalog_${uuid0}.hive_db_${uuid0}.dim values ("2020-01-01"); + +set disable_join_reorder = true; +set enable_dynamic_prune_scan_range = true; + +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt = d.dt; +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt + interval 1 day = d.dt; +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt - interval 1 day = d.dt; + +set enable_dynamic_prune_scan_range = false; + +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt = d.dt; +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt + interval 1 day = d.dt; +select * from hive_catalog_${uuid0}.hive_db_${uuid0}.fact as f join hive_catalog_${uuid0}.hive_db_${uuid0}.dim as d where f.dt - interval 1 day = d.dt; + +drop table hive_catalog_${uuid0}.hive_db_${uuid0}.fact force; +drop table hive_catalog_${uuid0}.hive_db_${uuid0}.dim force; + +drop database hive_catalog_${uuid0}.hive_db_${uuid0}; +drop catalog hive_catalog_${uuid0}; From 00ac587581bdc4397174a2910399e35e561dd673 Mon Sep 17 00:00:00 2001 From: Letian Jiang Date: Tue, 6 Aug 2024 17:48:47 +0800 Subject: [PATCH 4/6] ban struct subfield Signed-off-by: Letian Jiang --- be/src/exprs/runtime_filter_bank.cpp | 5 +++++ be/src/exprs/subfield_expr.cpp | 6 ++++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/be/src/exprs/runtime_filter_bank.cpp b/be/src/exprs/runtime_filter_bank.cpp index 448573fc688d2..2791a2da66367 100644 --- a/be/src/exprs/runtime_filter_bank.cpp +++ b/be/src/exprs/runtime_filter_bank.cpp @@ -443,6 +443,11 @@ void RuntimeFilterProbeCollector::do_evaluate_partial_chunk(Chunk* partial_chunk int n = expr->root()->get_slot_ids(&slot_ids); DCHECK(slot_ids.size() == n); + // do not allow struct subfield + if (expr->root()->get_subfields(nullptr) > 0) { + return false; + } + for (auto slot_id : slot_ids) { if (!partial_chunk->is_slot_exist(slot_id)) { return false; diff --git a/be/src/exprs/subfield_expr.cpp b/be/src/exprs/subfield_expr.cpp index c3b48a5827691..3d0d3c6719eef 100644 --- a/be/src/exprs/subfield_expr.cpp +++ b/be/src/exprs/subfield_expr.cpp @@ -83,7 +83,9 @@ class SubfieldExpr final : public Expr { Expr* clone(ObjectPool* pool) const override { return pool->add(new SubfieldExpr(*this)); } int get_subfields(std::vector>* subfields) const override { - subfields->push_back(_used_subfield_names); + if (subfields != nullptr) { + subfields->push_back(_used_subfield_names); + } return 1; } @@ -97,4 +99,4 @@ Expr* SubfieldExprFactory::from_thrift(const TExprNode& node) { return new SubfieldExpr(node); } -} // namespace starrocks \ No newline at end of file +} // namespace starrocks From dceaad92b2148587d0c2f5428df8de2c4e8bb326 Mon Sep 17 00:00:00 2001 From: Letian Jiang Date: Tue, 6 Aug 2024 21:06:25 +0800 Subject: [PATCH 5/6] format Signed-off-by: Letian Jiang --- be/src/connector/connector.h | 2 +- be/src/connector/hive_connector.cpp | 1073 +++++++++-------- .../com/starrocks/qe/SessionVariable.java | 3 - 3 files changed, 539 insertions(+), 539 deletions(-) diff --git a/be/src/connector/connector.h b/be/src/connector/connector.h index cb88aaf48d0c5..42fea2cd97e3b 100644 --- a/be/src/connector/connector.h +++ b/be/src/connector/connector.h @@ -19,8 +19,8 @@ #include #include "exec/pipeline/scan/morsel.h" -#include "gen_cpp/InternalService_types.h" #include "exprs/runtime_filter_bank.h" +#include "gen_cpp/InternalService_types.h" #include "gen_cpp/PlanNodes_types.h" #include "runtime/runtime_state.h" #include "storage/chunk_helper.h" diff --git a/be/src/connector/hive_connector.cpp b/be/src/connector/hive_connector.cpp index f8303f1b75ce8..d3c0f5ac3ed4e 100644 --- a/be/src/connector/hive_connector.cpp +++ b/be/src/connector/hive_connector.cpp @@ -111,624 +111,627 @@ Status HiveDataSource::open(RuntimeState* state) { } if (state->query_options().__isset.enable_connector_split_io_tasks) { _enable_split_tasks = state->query_options().enable_connector_split_io_tasks; - if (state->query_options().__isset.enable_dynamic_prune_scan_range) { - _enable_dynamic_prune_scan_range = state->query_options().enable_dynamic_prune_scan_range; - } - - RETURN_IF_ERROR(_init_conjunct_ctxs(state)); - _init_tuples_and_slots(state); - _init_counter(state); - RETURN_IF_ERROR(_init_partition_values()); - if (_filter_by_eval_partition_conjuncts) { - _no_data = true; - return Status::OK(); - } - RETURN_IF_ERROR(_init_scanner(state)); - return Status::OK(); -} - -void HiveDataSource::_update_has_any_predicate() { - auto f = [&]() { - if (_runtime_filters != nullptr && _runtime_filters->size() > 0) return true; - return false; - }; - _has_any_predicate = f(); -} + if (state->query_options().__isset.enable_dynamic_prune_scan_range) { + _enable_dynamic_prune_scan_range = state->query_options().enable_dynamic_prune_scan_range; + } -Status HiveDataSource::_init_conjunct_ctxs(RuntimeState* state) { - const auto& hdfs_scan_node = _provider->_hdfs_scan_node; - if (hdfs_scan_node.__isset.min_max_conjuncts) { - RETURN_IF_ERROR( - Expr::create_expr_trees(&_pool, hdfs_scan_node.min_max_conjuncts, &_min_max_conjunct_ctxs, state)); + RETURN_IF_ERROR(_init_conjunct_ctxs(state)); + _init_tuples_and_slots(state); + _init_counter(state); + RETURN_IF_ERROR(_init_partition_values()); + if (_filter_by_eval_partition_conjuncts) { + _no_data = true; + return Status::OK(); + } + RETURN_IF_ERROR(_init_scanner(state)); + return Status::OK(); } - if (hdfs_scan_node.__isset.partition_conjuncts) { - RETURN_IF_ERROR( - Expr::create_expr_trees(&_pool, hdfs_scan_node.partition_conjuncts, &_partition_conjunct_ctxs, state)); - _has_partition_conjuncts = true; + void HiveDataSource::_update_has_any_predicate() { + auto f = [&]() { + if (_runtime_filters != nullptr && _runtime_filters->size() > 0) return true; + return false; + }; + _has_any_predicate = f(); } - if (hdfs_scan_node.__isset.case_sensitive) { - _case_sensitive = hdfs_scan_node.case_sensitive; - } + Status HiveDataSource::_init_conjunct_ctxs(RuntimeState * state) { + const auto& hdfs_scan_node = _provider->_hdfs_scan_node; + if (hdfs_scan_node.__isset.min_max_conjuncts) { + RETURN_IF_ERROR( + Expr::create_expr_trees(&_pool, hdfs_scan_node.min_max_conjuncts, &_min_max_conjunct_ctxs, state)); + } - RETURN_IF_ERROR(Expr::prepare(_min_max_conjunct_ctxs, state)); - RETURN_IF_ERROR(Expr::prepare(_partition_conjunct_ctxs, state)); - RETURN_IF_ERROR(Expr::open(_min_max_conjunct_ctxs, state)); - RETURN_IF_ERROR(Expr::open(_partition_conjunct_ctxs, state)); - _update_has_any_predicate(); + if (hdfs_scan_node.__isset.partition_conjuncts) { + RETURN_IF_ERROR(Expr::create_expr_trees(&_pool, hdfs_scan_node.partition_conjuncts, + &_partition_conjunct_ctxs, state)); + _has_partition_conjuncts = true; + } - RETURN_IF_ERROR(_decompose_conjunct_ctxs(state)); - return Status::OK(); -} + if (hdfs_scan_node.__isset.case_sensitive) { + _case_sensitive = hdfs_scan_node.case_sensitive; + } -Status HiveDataSource::_init_partition_values() { - if (!(_hive_table != nullptr && _has_partition_columns)) return Status::OK(); + RETURN_IF_ERROR(Expr::prepare(_min_max_conjunct_ctxs, state)); + RETURN_IF_ERROR(Expr::prepare(_partition_conjunct_ctxs, state)); + RETURN_IF_ERROR(Expr::open(_min_max_conjunct_ctxs, state)); + RETURN_IF_ERROR(Expr::open(_partition_conjunct_ctxs, state)); + _update_has_any_predicate(); - auto* partition_desc = _hive_table->get_partition(_scan_range.partition_id); - if (partition_desc == nullptr) { - return Status::InternalError( - fmt::format("Plan inconsistency. scan_range.partition_id = {} not found in partition description map", - _scan_range.partition_id)); + RETURN_IF_ERROR(_decompose_conjunct_ctxs(state)); + return Status::OK(); } - const auto& partition_values = partition_desc->partition_key_value_evals(); - _partition_values = partition_desc->partition_key_value_evals(); + Status HiveDataSource::_init_partition_values() { + if (!(_hive_table != nullptr && _has_partition_columns)) return Status::OK(); - // init partition chunk - auto partition_chunk = std::make_shared(); - for (int i = 0; i < _partition_slots.size(); i++) { - SlotId slot_id = _partition_slots[i]->id(); - int partition_col_idx = _partition_index_in_hdfs_partition_columns[i]; - ASSIGN_OR_RETURN(auto partition_value_col, partition_values[partition_col_idx]->evaluate(nullptr)); - DCHECK(partition_value_col->is_constant()); - partition_chunk->append_column(partition_value_col, slot_id); - } + auto* partition_desc = _hive_table->get_partition(_scan_range.partition_id); + if (partition_desc == nullptr) { + return Status::InternalError(fmt::format( + "Plan inconsistency. scan_range.partition_id = {} not found in partition description map", + _scan_range.partition_id)); + } - // eval conjuncts and skip if no rows. - if (_has_scan_range_indicate_const_column) { - std::vector ctxs; - for (SlotId slotId : _scan_range.identity_partition_slot_ids) { - if (_conjunct_ctxs_by_slot.find(slotId) != _conjunct_ctxs_by_slot.end()) { - ctxs.insert(ctxs.end(), _conjunct_ctxs_by_slot.at(slotId).begin(), - _conjunct_ctxs_by_slot.at(slotId).end()); - } + const auto& partition_values = partition_desc->partition_key_value_evals(); + _partition_values = partition_desc->partition_key_value_evals(); + + // init partition chunk + auto partition_chunk = std::make_shared(); + for (int i = 0; i < _partition_slots.size(); i++) { + SlotId slot_id = _partition_slots[i]->id(); + int partition_col_idx = _partition_index_in_hdfs_partition_columns[i]; + ASSIGN_OR_RETURN(auto partition_value_col, partition_values[partition_col_idx]->evaluate(nullptr)); + DCHECK(partition_value_col->is_constant()); + partition_chunk->append_column(partition_value_col, slot_id); } - RETURN_IF_ERROR(ExecNode::eval_conjuncts(ctxs, partition_chunk.get())); - } else if (_has_partition_conjuncts) { - RETURN_IF_ERROR(ExecNode::eval_conjuncts(_partition_conjunct_ctxs, partition_chunk.get())); - } - if (!partition_chunk->has_rows()) { - _filter_by_eval_partition_conjuncts = true; - return Status::OK(); - } + // eval conjuncts and skip if no rows. + if (_has_scan_range_indicate_const_column) { + std::vector ctxs; + for (SlotId slotId : _scan_range.identity_partition_slot_ids) { + if (_conjunct_ctxs_by_slot.find(slotId) != _conjunct_ctxs_by_slot.end()) { + ctxs.insert(ctxs.end(), _conjunct_ctxs_by_slot.at(slotId).begin(), + _conjunct_ctxs_by_slot.at(slotId).end()); + } + } + RETURN_IF_ERROR(ExecNode::eval_conjuncts(ctxs, partition_chunk.get())); + } else if (_has_partition_conjuncts) { + RETURN_IF_ERROR(ExecNode::eval_conjuncts(_partition_conjunct_ctxs, partition_chunk.get())); + } - if (_enable_dynamic_prune_scan_range && _runtime_filters) { - _init_rf_counters(); - _runtime_filters->evaluate_partial_chunk(partition_chunk.get(), runtime_bloom_filter_eval_context); if (!partition_chunk->has_rows()) { _filter_by_eval_partition_conjuncts = true; return Status::OK(); } - } - return Status::OK(); -} + if (_enable_dynamic_prune_scan_range && _runtime_filters) { + _init_rf_counters(); + _runtime_filters->evaluate_partial_chunk(partition_chunk.get(), runtime_bloom_filter_eval_context); + if (!partition_chunk->has_rows()) { + _filter_by_eval_partition_conjuncts = true; + return Status::OK(); + } + } -int32_t HiveDataSource::scan_range_indicate_const_column_index(SlotId id) const { - if (!_scan_range.__isset.identity_partition_slot_ids) { - return -1; - } - auto it = std::find(_scan_range.identity_partition_slot_ids.begin(), _scan_range.identity_partition_slot_ids.end(), - id); - if (it == _scan_range.identity_partition_slot_ids.end()) { - return -1; - } else { - return it - _scan_range.identity_partition_slot_ids.begin(); + return Status::OK(); } -} -void HiveDataSource::_init_tuples_and_slots(RuntimeState* state) { - const auto& hdfs_scan_node = _provider->_hdfs_scan_node; - if (hdfs_scan_node.__isset.min_max_tuple_id) { - _min_max_tuple_id = hdfs_scan_node.min_max_tuple_id; - _min_max_tuple_desc = state->desc_tbl().get_tuple_descriptor(_min_max_tuple_id); - } - - const auto& slots = _tuple_desc->slots(); - for (int i = 0; i < slots.size(); i++) { - if (_hive_table != nullptr && _hive_table->is_partition_col(slots[i])) { - _partition_slots.push_back(slots[i]); - _partition_index_in_chunk.push_back(i); - _partition_index_in_hdfs_partition_columns.push_back(_hive_table->get_partition_col_index(slots[i])); - _has_partition_columns = true; - } else if (int32_t index = scan_range_indicate_const_column_index(slots[i]->id()); index >= 0) { - _partition_slots.push_back(slots[i]); - _partition_index_in_chunk.push_back(i); - _partition_index_in_hdfs_partition_columns.push_back(index); - _has_partition_columns = true; - _has_scan_range_indicate_const_column = true; + int32_t HiveDataSource::scan_range_indicate_const_column_index(SlotId id) const { + if (!_scan_range.__isset.identity_partition_slot_ids) { + return -1; + } + auto it = std::find(_scan_range.identity_partition_slot_ids.begin(), + _scan_range.identity_partition_slot_ids.end(), id); + if (it == _scan_range.identity_partition_slot_ids.end()) { + return -1; } else { - _materialize_slots.push_back(slots[i]); - _materialize_index_in_chunk.push_back(i); + return it - _scan_range.identity_partition_slot_ids.begin(); } } - if (_scan_range.__isset.delete_column_slot_ids && !_scan_range.delete_column_slot_ids.empty()) { - std::map id_to_slots; - for (const auto& slot : _materialize_slots) { - id_to_slots.emplace(slot->id(), slot); + void HiveDataSource::_init_tuples_and_slots(RuntimeState * state) { + const auto& hdfs_scan_node = _provider->_hdfs_scan_node; + if (hdfs_scan_node.__isset.min_max_tuple_id) { + _min_max_tuple_id = hdfs_scan_node.min_max_tuple_id; + _min_max_tuple_desc = state->desc_tbl().get_tuple_descriptor(_min_max_tuple_id); } - int32_t delete_column_index = slots.size(); - _delete_column_tuple_desc = state->desc_tbl().get_tuple_descriptor(_provider->_hdfs_scan_node.mor_tuple_id); - - for (SlotDescriptor* d_slot_desc : _delete_column_tuple_desc->slots()) { - _equality_delete_slots.emplace_back(d_slot_desc); - if (!id_to_slots.contains(d_slot_desc->id())) { - _materialize_slots.push_back(d_slot_desc); - _materialize_index_in_chunk.push_back(delete_column_index++); + const auto& slots = _tuple_desc->slots(); + for (int i = 0; i < slots.size(); i++) { + if (_hive_table != nullptr && _hive_table->is_partition_col(slots[i])) { + _partition_slots.push_back(slots[i]); + _partition_index_in_chunk.push_back(i); + _partition_index_in_hdfs_partition_columns.push_back(_hive_table->get_partition_col_index(slots[i])); + _has_partition_columns = true; + } else if (int32_t index = scan_range_indicate_const_column_index(slots[i]->id()); index >= 0) { + _partition_slots.push_back(slots[i]); + _partition_index_in_chunk.push_back(i); + _partition_index_in_hdfs_partition_columns.push_back(index); + _has_partition_columns = true; + _has_scan_range_indicate_const_column = true; + } else { + _materialize_slots.push_back(slots[i]); + _materialize_index_in_chunk.push_back(i); } } - } - if (hdfs_scan_node.__isset.hive_column_names) { - _hive_column_names = hdfs_scan_node.hive_column_names; - } - if (hdfs_scan_node.__isset.case_sensitive) { - _case_sensitive = hdfs_scan_node.case_sensitive; - } - if (hdfs_scan_node.__isset.can_use_any_column) { - _can_use_any_column = hdfs_scan_node.can_use_any_column; - } - if (hdfs_scan_node.__isset.can_use_min_max_count_opt) { - _can_use_min_max_count_opt = hdfs_scan_node.can_use_min_max_count_opt; - } - if (hdfs_scan_node.__isset.use_partition_column_value_only) { - _use_partition_column_value_only = hdfs_scan_node.use_partition_column_value_only; - } + if (_scan_range.__isset.delete_column_slot_ids && !_scan_range.delete_column_slot_ids.empty()) { + std::map id_to_slots; + for (const auto& slot : _materialize_slots) { + id_to_slots.emplace(slot->id(), slot); + } - // The reason why we need double check here is for iceberg table. - // for some partitions, partition column maybe is not constant value. - // If partition column is not constant value, we can not use this optimization, - // And we can not use `can_use_any_column` either. - // So checks are: - // 1. can_use_any_column = true - // 2. only one materialized slot - // 3. besides that, all slots are partition slots. - // 4. scan iceberg data file without equality delete files. - auto check_opt_on_iceberg = [&]() { - if (!_can_use_any_column) { - return false; + int32_t delete_column_index = slots.size(); + _delete_column_tuple_desc = state->desc_tbl().get_tuple_descriptor(_provider->_hdfs_scan_node.mor_tuple_id); + + for (SlotDescriptor* d_slot_desc : _delete_column_tuple_desc->slots()) { + _equality_delete_slots.emplace_back(d_slot_desc); + if (!id_to_slots.contains(d_slot_desc->id())) { + _materialize_slots.push_back(d_slot_desc); + _materialize_index_in_chunk.push_back(delete_column_index++); + } + } } - if ((_partition_slots.size() + 1) != slots.size()) { - return false; + + if (hdfs_scan_node.__isset.hive_column_names) { + _hive_column_names = hdfs_scan_node.hive_column_names; } - if (_materialize_slots.size() != 1) { - return false; + if (hdfs_scan_node.__isset.case_sensitive) { + _case_sensitive = hdfs_scan_node.case_sensitive; } - if (!_scan_range.delete_column_slot_ids.empty()) { - return false; + if (hdfs_scan_node.__isset.can_use_any_column) { + _can_use_any_column = hdfs_scan_node.can_use_any_column; + } + if (hdfs_scan_node.__isset.can_use_min_max_count_opt) { + _can_use_min_max_count_opt = hdfs_scan_node.can_use_min_max_count_opt; + } + if (hdfs_scan_node.__isset.use_partition_column_value_only) { + _use_partition_column_value_only = hdfs_scan_node.use_partition_column_value_only; } - return true; - }; - if (!check_opt_on_iceberg()) { - _use_partition_column_value_only = false; - _can_use_any_column = false; - } -} - -Status HiveDataSource::_decompose_conjunct_ctxs(RuntimeState* state) { - if (_conjunct_ctxs.empty()) { - return Status::OK(); - } - std::unordered_map slot_by_id; - for (SlotDescriptor* slot : _tuple_desc->slots()) { - slot_by_id[slot->id()] = slot; + // The reason why we need double check here is for iceberg table. + // for some partitions, partition column maybe is not constant value. + // If partition column is not constant value, we can not use this optimization, + // And we can not use `can_use_any_column` either. + // So checks are: + // 1. can_use_any_column = true + // 2. only one materialized slot + // 3. besides that, all slots are partition slots. + // 4. scan iceberg data file without equality delete files. + auto check_opt_on_iceberg = [&]() { + if (!_can_use_any_column) { + return false; + } + if ((_partition_slots.size() + 1) != slots.size()) { + return false; + } + if (_materialize_slots.size() != 1) { + return false; + } + if (!_scan_range.delete_column_slot_ids.empty()) { + return false; + } + return true; + }; + if (!check_opt_on_iceberg()) { + _use_partition_column_value_only = false; + _can_use_any_column = false; + } } - std::vector cloned_conjunct_ctxs; - RETURN_IF_ERROR(Expr::clone_if_not_exists(state, &_pool, _conjunct_ctxs, &cloned_conjunct_ctxs)); - - for (ExprContext* ctx : cloned_conjunct_ctxs) { - const Expr* root_expr = ctx->root(); - std::vector slot_ids; - root_expr->get_slot_ids(&slot_ids); - for (SlotId slot_id : slot_ids) { - _slots_in_conjunct.insert(slot_id); + Status HiveDataSource::_decompose_conjunct_ctxs(RuntimeState * state) { + if (_conjunct_ctxs.empty()) { + return Status::OK(); } - // For some conjunct like (a < 1) or (a > 7) - // slot_ids = (a, a), but actually there is only one slot. - bool single_slot = true; - for (int i = 1; i < slot_ids.size(); i++) { - if (slot_ids[i] != slot_ids[0]) { - single_slot = false; - break; - } + std::unordered_map slot_by_id; + for (SlotDescriptor* slot : _tuple_desc->slots()) { + slot_by_id[slot->id()] = slot; } - if (!single_slot || slot_ids.empty()) { - _scanner_conjunct_ctxs.emplace_back(ctx); + + std::vector cloned_conjunct_ctxs; + RETURN_IF_ERROR(Expr::clone_if_not_exists(state, &_pool, _conjunct_ctxs, &cloned_conjunct_ctxs)); + + for (ExprContext* ctx : cloned_conjunct_ctxs) { + const Expr* root_expr = ctx->root(); + std::vector slot_ids; + root_expr->get_slot_ids(&slot_ids); for (SlotId slot_id : slot_ids) { - _slots_of_mutli_slot_conjunct.insert(slot_id); + _slots_in_conjunct.insert(slot_id); } - continue; - } - SlotId slot_id = slot_ids[0]; - if (slot_by_id.find(slot_id) != slot_by_id.end()) { - if (_conjunct_ctxs_by_slot.find(slot_id) == _conjunct_ctxs_by_slot.end()) { - _conjunct_ctxs_by_slot.insert({slot_id, std::vector()}); + // For some conjunct like (a < 1) or (a > 7) + // slot_ids = (a, a), but actually there is only one slot. + bool single_slot = true; + for (int i = 1; i < slot_ids.size(); i++) { + if (slot_ids[i] != slot_ids[0]) { + single_slot = false; + break; + } + } + if (!single_slot || slot_ids.empty()) { + _scanner_conjunct_ctxs.emplace_back(ctx); + for (SlotId slot_id : slot_ids) { + _slots_of_mutli_slot_conjunct.insert(slot_id); + } + continue; + } + + SlotId slot_id = slot_ids[0]; + if (slot_by_id.find(slot_id) != slot_by_id.end()) { + if (_conjunct_ctxs_by_slot.find(slot_id) == _conjunct_ctxs_by_slot.end()) { + _conjunct_ctxs_by_slot.insert({slot_id, std::vector()}); + } + _conjunct_ctxs_by_slot[slot_id].emplace_back(ctx); } - _conjunct_ctxs_by_slot[slot_id].emplace_back(ctx); } + return Status::OK(); } - return Status::OK(); -} -void HiveDataSource::_init_counter(RuntimeState* state) { - const auto& hdfs_scan_node = _provider->_hdfs_scan_node; + void HiveDataSource::_init_counter(RuntimeState * state) { + const auto& hdfs_scan_node = _provider->_hdfs_scan_node; + + _profile.runtime_profile = _runtime_profile; + _profile.raw_rows_read_counter = ADD_COUNTER(_runtime_profile, "RawRowsRead", TUnit::UNIT); + _profile.rows_read_counter = ADD_COUNTER(_runtime_profile, "RowsRead", TUnit::UNIT); + _profile.late_materialize_skip_rows_counter = + ADD_COUNTER(_runtime_profile, "LateMaterializeSkipRows", TUnit::UNIT); + _profile.scan_ranges_counter = ADD_COUNTER(_runtime_profile, "ScanRanges", TUnit::UNIT); + + _profile.reader_init_timer = ADD_TIMER(_runtime_profile, "ReaderInit"); + _profile.open_file_timer = ADD_TIMER(_runtime_profile, "OpenFile"); + _profile.expr_filter_timer = ADD_TIMER(_runtime_profile, "ExprFilterTime"); + + _profile.column_read_timer = ADD_TIMER(_runtime_profile, "ColumnReadTime"); + _profile.column_convert_timer = ADD_TIMER(_runtime_profile, "ColumnConvertTime"); + + { + static const char* prefix = "SharedBuffered"; + ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); + _profile.shared_buffered_shared_io_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "SharedIOBytes", TUnit::BYTES, prefix); + _profile.shared_buffered_shared_align_io_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "SharedAlignIOBytes", TUnit::BYTES, prefix); + _profile.shared_buffered_shared_io_count = + ADD_CHILD_COUNTER(_runtime_profile, "SharedIOCount", TUnit::UNIT, prefix); + _profile.shared_buffered_shared_io_timer = ADD_CHILD_TIMER(_runtime_profile, "SharedIOTime", prefix); + _profile.shared_buffered_direct_io_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DirectIOBytes", TUnit::BYTES, prefix); + _profile.shared_buffered_direct_io_count = + ADD_CHILD_COUNTER(_runtime_profile, "DirectIOCount", TUnit::UNIT, prefix); + _profile.shared_buffered_direct_io_timer = ADD_CHILD_TIMER(_runtime_profile, "DirectIOTime", prefix); + } + + if (_use_datacache) { + static const char* prefix = "DataCache"; + ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); + _profile.datacache_read_counter = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadCounter", TUnit::UNIT, prefix); + _profile.datacache_read_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBytes", TUnit::BYTES, prefix); + _profile.datacache_read_mem_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadMemBytes", TUnit::BYTES, "DataCacheReadBytes"); + _profile.datacache_read_disk_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadDiskBytes", TUnit::BYTES, "DataCacheReadBytes"); + _profile.datacache_skip_read_counter = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheSkipReadCounter", TUnit::UNIT, prefix); + _profile.datacache_skip_read_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheSkipReadBytes", TUnit::BYTES, prefix); + _profile.datacache_read_timer = ADD_CHILD_TIMER(_runtime_profile, "DataCacheReadTimer", prefix); + _profile.datacache_write_counter = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteCounter", TUnit::UNIT, prefix); + _profile.datacache_write_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteBytes", TUnit::BYTES, prefix); + _profile.datacache_write_timer = ADD_CHILD_TIMER(_runtime_profile, "DataCacheWriteTimer", prefix); + _profile.datacache_write_fail_counter = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteFailCounter", TUnit::UNIT, prefix); + _profile.datacache_write_fail_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteFailBytes", TUnit::BYTES, prefix); + _profile.datacache_read_block_buffer_counter = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBlockBufferCounter", TUnit::UNIT, prefix); + _profile.datacache_read_block_buffer_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBlockBufferBytes", TUnit::BYTES, prefix); + } + + { + static const char* prefix = "InputStream"; + ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); + _profile.app_io_bytes_read_counter = + ADD_CHILD_COUNTER(_runtime_profile, "AppIOBytesRead", TUnit::BYTES, prefix); + _profile.app_io_timer = ADD_CHILD_TIMER(_runtime_profile, "AppIOTime", prefix); + _profile.app_io_counter = ADD_CHILD_COUNTER(_runtime_profile, "AppIOCounter", TUnit::UNIT, prefix); + _profile.fs_bytes_read_counter = ADD_CHILD_COUNTER(_runtime_profile, "FSIOBytesRead", TUnit::BYTES, prefix); + _profile.fs_io_counter = ADD_CHILD_COUNTER(_runtime_profile, "FSIOCounter", TUnit::UNIT, prefix); + _profile.fs_io_timer = ADD_CHILD_TIMER(_runtime_profile, "FSIOTime", prefix); + } - _profile.runtime_profile = _runtime_profile; - _profile.raw_rows_read_counter = ADD_COUNTER(_runtime_profile, "RawRowsRead", TUnit::UNIT); - _profile.rows_read_counter = ADD_COUNTER(_runtime_profile, "RowsRead", TUnit::UNIT); - _profile.late_materialize_skip_rows_counter = ADD_COUNTER(_runtime_profile, "LateMaterializeSkipRows", TUnit::UNIT); - _profile.scan_ranges_counter = ADD_COUNTER(_runtime_profile, "ScanRanges", TUnit::UNIT); - - _profile.reader_init_timer = ADD_TIMER(_runtime_profile, "ReaderInit"); - _profile.open_file_timer = ADD_TIMER(_runtime_profile, "OpenFile"); - _profile.expr_filter_timer = ADD_TIMER(_runtime_profile, "ExprFilterTime"); - - _profile.column_read_timer = ADD_TIMER(_runtime_profile, "ColumnReadTime"); - _profile.column_convert_timer = ADD_TIMER(_runtime_profile, "ColumnConvertTime"); - - { - static const char* prefix = "SharedBuffered"; - ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); - _profile.shared_buffered_shared_io_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "SharedIOBytes", TUnit::BYTES, prefix); - _profile.shared_buffered_shared_align_io_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "SharedAlignIOBytes", TUnit::BYTES, prefix); - _profile.shared_buffered_shared_io_count = - ADD_CHILD_COUNTER(_runtime_profile, "SharedIOCount", TUnit::UNIT, prefix); - _profile.shared_buffered_shared_io_timer = ADD_CHILD_TIMER(_runtime_profile, "SharedIOTime", prefix); - _profile.shared_buffered_direct_io_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DirectIOBytes", TUnit::BYTES, prefix); - _profile.shared_buffered_direct_io_count = - ADD_CHILD_COUNTER(_runtime_profile, "DirectIOCount", TUnit::UNIT, prefix); - _profile.shared_buffered_direct_io_timer = ADD_CHILD_TIMER(_runtime_profile, "DirectIOTime", prefix); - } - - if (_use_datacache) { - static const char* prefix = "DataCache"; - ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); - _profile.datacache_read_counter = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadCounter", TUnit::UNIT, prefix); - _profile.datacache_read_bytes = ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBytes", TUnit::BYTES, prefix); - _profile.datacache_read_mem_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadMemBytes", TUnit::BYTES, "DataCacheReadBytes"); - _profile.datacache_read_disk_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadDiskBytes", TUnit::BYTES, "DataCacheReadBytes"); - _profile.datacache_skip_read_counter = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheSkipReadCounter", TUnit::UNIT, prefix); - _profile.datacache_skip_read_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheSkipReadBytes", TUnit::BYTES, prefix); - _profile.datacache_read_timer = ADD_CHILD_TIMER(_runtime_profile, "DataCacheReadTimer", prefix); - _profile.datacache_write_counter = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteCounter", TUnit::UNIT, prefix); - _profile.datacache_write_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteBytes", TUnit::BYTES, prefix); - _profile.datacache_write_timer = ADD_CHILD_TIMER(_runtime_profile, "DataCacheWriteTimer", prefix); - _profile.datacache_write_fail_counter = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteFailCounter", TUnit::UNIT, prefix); - _profile.datacache_write_fail_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteFailBytes", TUnit::BYTES, prefix); - _profile.datacache_read_block_buffer_counter = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBlockBufferCounter", TUnit::UNIT, prefix); - _profile.datacache_read_block_buffer_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBlockBufferBytes", TUnit::BYTES, prefix); - } - - { - static const char* prefix = "InputStream"; - ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); - _profile.app_io_bytes_read_counter = - ADD_CHILD_COUNTER(_runtime_profile, "AppIOBytesRead", TUnit::BYTES, prefix); - _profile.app_io_timer = ADD_CHILD_TIMER(_runtime_profile, "AppIOTime", prefix); - _profile.app_io_counter = ADD_CHILD_COUNTER(_runtime_profile, "AppIOCounter", TUnit::UNIT, prefix); - _profile.fs_bytes_read_counter = ADD_CHILD_COUNTER(_runtime_profile, "FSIOBytesRead", TUnit::BYTES, prefix); - _profile.fs_io_counter = ADD_CHILD_COUNTER(_runtime_profile, "FSIOCounter", TUnit::UNIT, prefix); - _profile.fs_io_timer = ADD_CHILD_TIMER(_runtime_profile, "FSIOTime", prefix); - } - - if (hdfs_scan_node.__isset.table_name) { - _runtime_profile->add_info_string("Table", hdfs_scan_node.table_name); - } - if (hdfs_scan_node.__isset.sql_predicates) { - _runtime_profile->add_info_string("Predicates", hdfs_scan_node.sql_predicates); - } - if (hdfs_scan_node.__isset.min_max_sql_predicates) { - _runtime_profile->add_info_string("PredicatesMinMax", hdfs_scan_node.min_max_sql_predicates); - } - if (hdfs_scan_node.__isset.partition_sql_predicates) { - _runtime_profile->add_info_string("PredicatesPartition", hdfs_scan_node.partition_sql_predicates); + if (hdfs_scan_node.__isset.table_name) { + _runtime_profile->add_info_string("Table", hdfs_scan_node.table_name); + } + if (hdfs_scan_node.__isset.sql_predicates) { + _runtime_profile->add_info_string("Predicates", hdfs_scan_node.sql_predicates); + } + if (hdfs_scan_node.__isset.min_max_sql_predicates) { + _runtime_profile->add_info_string("PredicatesMinMax", hdfs_scan_node.min_max_sql_predicates); + } + if (hdfs_scan_node.__isset.partition_sql_predicates) { + _runtime_profile->add_info_string("PredicatesPartition", hdfs_scan_node.partition_sql_predicates); + } } -} -void HiveDataSource::_init_rf_counters() { - auto* root = _runtime_profile; - if (runtime_bloom_filter_eval_context.join_runtime_filter_timer == nullptr) { - static const char* prefix = "DynamicPruneScanRange"; - ADD_COUNTER(root, prefix, TUnit::NONE); - runtime_bloom_filter_eval_context.join_runtime_filter_timer = - ADD_CHILD_TIMER(root, "JoinRuntimeFilterTime", prefix); - runtime_bloom_filter_eval_context.join_runtime_filter_hash_timer = - ADD_CHILD_TIMER(root, "JoinRuntimeFilterHashTime", prefix); - runtime_bloom_filter_eval_context.join_runtime_filter_input_counter = - ADD_CHILD_COUNTER(root, "JoinRuntimeFilterInputScanRanges", TUnit::UNIT, prefix); - runtime_bloom_filter_eval_context.join_runtime_filter_output_counter = - ADD_CHILD_COUNTER(root, "JoinRuntimeFilterOutputScanRanges", TUnit::UNIT, prefix); - runtime_bloom_filter_eval_context.join_runtime_filter_eval_counter = - ADD_CHILD_COUNTER(root, "JoinRuntimeFilterEvaluate", TUnit::UNIT, prefix); + void HiveDataSource::_init_rf_counters() { + auto* root = _runtime_profile; + if (runtime_bloom_filter_eval_context.join_runtime_filter_timer == nullptr) { + static const char* prefix = "DynamicPruneScanRange"; + ADD_COUNTER(root, prefix, TUnit::NONE); + runtime_bloom_filter_eval_context.join_runtime_filter_timer = + ADD_CHILD_TIMER(root, "JoinRuntimeFilterTime", prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_hash_timer = + ADD_CHILD_TIMER(root, "JoinRuntimeFilterHashTime", prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_input_counter = + ADD_CHILD_COUNTER(root, "JoinRuntimeFilterInputScanRanges", TUnit::UNIT, prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_output_counter = + ADD_CHILD_COUNTER(root, "JoinRuntimeFilterOutputScanRanges", TUnit::UNIT, prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_eval_counter = + ADD_CHILD_COUNTER(root, "JoinRuntimeFilterEvaluate", TUnit::UNIT, prefix); + } } -} -Status HiveDataSource::_init_scanner(RuntimeState* state) { - SCOPED_TIMER(_profile.open_file_timer); + Status HiveDataSource::_init_scanner(RuntimeState * state) { + SCOPED_TIMER(_profile.open_file_timer); - const auto& scan_range = _scan_range; - std::string native_file_path = scan_range.full_path; - if (_hive_table != nullptr && _hive_table->has_partition() && !_hive_table->has_base_path()) { - auto* partition_desc = _hive_table->get_partition(scan_range.partition_id); - if (partition_desc == nullptr) { - return Status::InternalError(fmt::format( - "Plan inconsistency. scan_range.partition_id = {} not found in partition description map", - scan_range.partition_id)); + const auto& scan_range = _scan_range; + std::string native_file_path = scan_range.full_path; + if (_hive_table != nullptr && _hive_table->has_partition() && !_hive_table->has_base_path()) { + auto* partition_desc = _hive_table->get_partition(scan_range.partition_id); + if (partition_desc == nullptr) { + return Status::InternalError(fmt::format( + "Plan inconsistency. scan_range.partition_id = {} not found in partition description map", + scan_range.partition_id)); + } + std::filesystem::path file_path(partition_desc->location()); + file_path /= scan_range.relative_path; + native_file_path = file_path.native(); + } + if (native_file_path.empty()) { + native_file_path = _hive_table->get_base_path() + scan_range.relative_path; } - std::filesystem::path file_path(partition_desc->location()); - file_path /= scan_range.relative_path; - native_file_path = file_path.native(); - } - if (native_file_path.empty()) { - native_file_path = _hive_table->get_base_path() + scan_range.relative_path; - } - const auto& hdfs_scan_node = _provider->_hdfs_scan_node; - auto fsOptions = - FSOptions(hdfs_scan_node.__isset.cloud_configuration ? &hdfs_scan_node.cloud_configuration : nullptr); - - ASSIGN_OR_RETURN(auto fs, FileSystem::CreateUniqueFromString(native_file_path, fsOptions)); - - HdfsScannerParams scanner_params; - scanner_params.runtime_filter_collector = _runtime_filters; - scanner_params.scan_range = &scan_range; - scanner_params.fs = _pool.add(fs.release()); - scanner_params.path = native_file_path; - scanner_params.file_size = _scan_range.file_length; - scanner_params.modification_time = _scan_range.modification_time; - scanner_params.tuple_desc = _tuple_desc; - scanner_params.materialize_slots = _materialize_slots; - scanner_params.materialize_index_in_chunk = _materialize_index_in_chunk; - scanner_params.partition_slots = _partition_slots; - scanner_params.partition_index_in_chunk = _partition_index_in_chunk; - scanner_params._partition_index_in_hdfs_partition_columns = _partition_index_in_hdfs_partition_columns; - scanner_params.partition_values = _partition_values; - scanner_params.conjunct_ctxs = _scanner_conjunct_ctxs; - scanner_params.conjunct_ctxs_by_slot = _conjunct_ctxs_by_slot; - scanner_params.slots_in_conjunct = _slots_in_conjunct; - scanner_params.slots_of_mutli_slot_conjunct = _slots_of_mutli_slot_conjunct; - scanner_params.min_max_conjunct_ctxs = _min_max_conjunct_ctxs; - scanner_params.min_max_tuple_desc = _min_max_tuple_desc; - scanner_params.hive_column_names = &_hive_column_names; - scanner_params.case_sensitive = _case_sensitive; - scanner_params.profile = &_profile; - scanner_params.lazy_column_coalesce_counter = get_lazy_column_coalesce_counter(); - scanner_params.split_context = down_cast(_split_context); - scanner_params.enable_split_tasks = _enable_split_tasks; - if (state->query_options().__isset.connector_max_split_size) { - scanner_params.connector_max_split_size = state->query_options().connector_max_split_size; - } - - if (!_equality_delete_slots.empty()) { - MORParams& mor_params = scanner_params.mor_params; - mor_params.tuple_desc = _tuple_desc; - mor_params.equality_slots = _equality_delete_slots; - mor_params.delete_column_tuple_desc = _delete_column_tuple_desc; - mor_params.mor_tuple_id = _provider->_hdfs_scan_node.mor_tuple_id; - mor_params.runtime_profile = _runtime_profile; - } - - for (const auto& delete_file : scan_range.delete_files) { - scanner_params.deletes.emplace_back(&delete_file); - } - - if (dynamic_cast(_hive_table)) { - auto tbl = dynamic_cast(_hive_table); - scanner_params.iceberg_schema = tbl->get_iceberg_schema(); - scanner_params.iceberg_equal_delete_schema = tbl->get_iceberg_equal_delete_schema(); - } - if (scan_range.__isset.paimon_deletion_file && !scan_range.paimon_deletion_file.path.empty()) { - scanner_params.paimon_deletion_file = std::make_shared(scan_range.paimon_deletion_file); - } - scanner_params.use_datacache = _use_datacache; - scanner_params.enable_populate_datacache = _enable_populate_datacache; - scanner_params.enable_datacache_async_populate_mode = _enable_datacache_aync_populate_mode; - scanner_params.enable_datacache_io_adaptor = _enable_datacache_io_adaptor; - scanner_params.datacache_evict_probability = _datacache_evict_probability; - scanner_params.can_use_any_column = _can_use_any_column; - scanner_params.can_use_min_max_count_opt = _can_use_min_max_count_opt; - scanner_params.use_file_metacache = _use_file_metacache; - - HdfsScanner* scanner = nullptr; - auto format = scan_range.file_format; - - bool use_hudi_jni_reader = false; - if (scan_range.__isset.use_hudi_jni_reader) { - use_hudi_jni_reader = scan_range.use_hudi_jni_reader; - } - bool use_paimon_jni_reader = false; - if (scan_range.__isset.use_paimon_jni_reader) { - use_paimon_jni_reader = scan_range.use_paimon_jni_reader; - } - bool use_odps_jni_reader = false; - if (scan_range.__isset.use_odps_jni_reader) { - use_odps_jni_reader = scan_range.use_odps_jni_reader; - } - - JniScanner::CreateOptions jni_scanner_create_options = { - .fs_options = &fsOptions, .hive_table = _hive_table, .scan_range = &scan_range}; - - if (_use_partition_column_value_only) { - DCHECK(_can_use_any_column); - scanner = new HdfsPartitionScanner(); - } else if (use_paimon_jni_reader) { - scanner = create_paimon_jni_scanner(jni_scanner_create_options).release(); - } else if (use_hudi_jni_reader) { - scanner = create_hudi_jni_scanner(jni_scanner_create_options).release(); - } else if (use_odps_jni_reader) { - scanner = create_odps_jni_scanner(jni_scanner_create_options).release(); - } else if (format == THdfsFileFormat::PARQUET) { - scanner = new HdfsParquetScanner(); - } else if (format == THdfsFileFormat::ORC) { - scanner_params.orc_use_column_names = state->query_options().orc_use_column_names; - scanner = new HdfsOrcScanner(); - } else if (format == THdfsFileFormat::TEXT) { - scanner = new HdfsTextScanner(); - } else if ((format == THdfsFileFormat::AVRO || format == THdfsFileFormat::RC_BINARY || - format == THdfsFileFormat::RC_TEXT || format == THdfsFileFormat::SEQUENCE_FILE) && - (dynamic_cast(_hive_table) != nullptr || - dynamic_cast(_hive_table) != nullptr)) { - scanner = create_hive_jni_scanner(jni_scanner_create_options).release(); - } else { - std::string msg = fmt::format("unsupported hdfs file format: {}", format); - LOG(WARNING) << msg; - return Status::NotSupported(msg); - } - if (scanner == nullptr) { - return Status::InternalError("create hdfs scanner failed"); - } - _pool.add(scanner); - - RETURN_IF_ERROR(scanner->init(state, scanner_params)); - Status st = scanner->open(state); - if (!st.ok()) { - return scanner->reinterpret_status(st); - } - _scanner = scanner; - return Status::OK(); -} + const auto& hdfs_scan_node = _provider->_hdfs_scan_node; + auto fsOptions = + FSOptions(hdfs_scan_node.__isset.cloud_configuration ? &hdfs_scan_node.cloud_configuration : nullptr); + + ASSIGN_OR_RETURN(auto fs, FileSystem::CreateUniqueFromString(native_file_path, fsOptions)); + + HdfsScannerParams scanner_params; + scanner_params.runtime_filter_collector = _runtime_filters; + scanner_params.scan_range = &scan_range; + scanner_params.fs = _pool.add(fs.release()); + scanner_params.path = native_file_path; + scanner_params.file_size = _scan_range.file_length; + scanner_params.modification_time = _scan_range.modification_time; + scanner_params.tuple_desc = _tuple_desc; + scanner_params.materialize_slots = _materialize_slots; + scanner_params.materialize_index_in_chunk = _materialize_index_in_chunk; + scanner_params.partition_slots = _partition_slots; + scanner_params.partition_index_in_chunk = _partition_index_in_chunk; + scanner_params._partition_index_in_hdfs_partition_columns = _partition_index_in_hdfs_partition_columns; + scanner_params.partition_values = _partition_values; + scanner_params.conjunct_ctxs = _scanner_conjunct_ctxs; + scanner_params.conjunct_ctxs_by_slot = _conjunct_ctxs_by_slot; + scanner_params.slots_in_conjunct = _slots_in_conjunct; + scanner_params.slots_of_mutli_slot_conjunct = _slots_of_mutli_slot_conjunct; + scanner_params.min_max_conjunct_ctxs = _min_max_conjunct_ctxs; + scanner_params.min_max_tuple_desc = _min_max_tuple_desc; + scanner_params.hive_column_names = &_hive_column_names; + scanner_params.case_sensitive = _case_sensitive; + scanner_params.profile = &_profile; + scanner_params.lazy_column_coalesce_counter = get_lazy_column_coalesce_counter(); + scanner_params.split_context = down_cast(_split_context); + scanner_params.enable_split_tasks = _enable_split_tasks; + if (state->query_options().__isset.connector_max_split_size) { + scanner_params.connector_max_split_size = state->query_options().connector_max_split_size; + } -void HiveDataSource::close(RuntimeState* state) { - if (_scanner != nullptr) { - if (!_scanner->has_split_tasks()) { - COUNTER_UPDATE(_profile.scan_ranges_counter, 1); + if (!_equality_delete_slots.empty()) { + MORParams& mor_params = scanner_params.mor_params; + mor_params.tuple_desc = _tuple_desc; + mor_params.equality_slots = _equality_delete_slots; + mor_params.delete_column_tuple_desc = _delete_column_tuple_desc; + mor_params.mor_tuple_id = _provider->_hdfs_scan_node.mor_tuple_id; + mor_params.runtime_profile = _runtime_profile; } - _scanner->close(); - } - Expr::close(_min_max_conjunct_ctxs, state); - Expr::close(_partition_conjunct_ctxs, state); - Expr::close(_scanner_conjunct_ctxs, state); - for (auto& it : _conjunct_ctxs_by_slot) { - Expr::close(it.second, state); + + for (const auto& delete_file : scan_range.delete_files) { + scanner_params.deletes.emplace_back(&delete_file); + } + + if (dynamic_cast(_hive_table)) { + auto tbl = dynamic_cast(_hive_table); + scanner_params.iceberg_schema = tbl->get_iceberg_schema(); + scanner_params.iceberg_equal_delete_schema = tbl->get_iceberg_equal_delete_schema(); + } + if (scan_range.__isset.paimon_deletion_file && !scan_range.paimon_deletion_file.path.empty()) { + scanner_params.paimon_deletion_file = + std::make_shared(scan_range.paimon_deletion_file); + } + scanner_params.use_datacache = _use_datacache; + scanner_params.enable_populate_datacache = _enable_populate_datacache; + scanner_params.enable_datacache_async_populate_mode = _enable_datacache_aync_populate_mode; + scanner_params.enable_datacache_io_adaptor = _enable_datacache_io_adaptor; + scanner_params.datacache_evict_probability = _datacache_evict_probability; + scanner_params.can_use_any_column = _can_use_any_column; + scanner_params.can_use_min_max_count_opt = _can_use_min_max_count_opt; + scanner_params.use_file_metacache = _use_file_metacache; + + HdfsScanner* scanner = nullptr; + auto format = scan_range.file_format; + + bool use_hudi_jni_reader = false; + if (scan_range.__isset.use_hudi_jni_reader) { + use_hudi_jni_reader = scan_range.use_hudi_jni_reader; + } + bool use_paimon_jni_reader = false; + if (scan_range.__isset.use_paimon_jni_reader) { + use_paimon_jni_reader = scan_range.use_paimon_jni_reader; + } + bool use_odps_jni_reader = false; + if (scan_range.__isset.use_odps_jni_reader) { + use_odps_jni_reader = scan_range.use_odps_jni_reader; + } + + JniScanner::CreateOptions jni_scanner_create_options = { + .fs_options = &fsOptions, .hive_table = _hive_table, .scan_range = &scan_range}; + + if (_use_partition_column_value_only) { + DCHECK(_can_use_any_column); + scanner = new HdfsPartitionScanner(); + } else if (use_paimon_jni_reader) { + scanner = create_paimon_jni_scanner(jni_scanner_create_options).release(); + } else if (use_hudi_jni_reader) { + scanner = create_hudi_jni_scanner(jni_scanner_create_options).release(); + } else if (use_odps_jni_reader) { + scanner = create_odps_jni_scanner(jni_scanner_create_options).release(); + } else if (format == THdfsFileFormat::PARQUET) { + scanner = new HdfsParquetScanner(); + } else if (format == THdfsFileFormat::ORC) { + scanner_params.orc_use_column_names = state->query_options().orc_use_column_names; + scanner = new HdfsOrcScanner(); + } else if (format == THdfsFileFormat::TEXT) { + scanner = new HdfsTextScanner(); + } else if ((format == THdfsFileFormat::AVRO || format == THdfsFileFormat::RC_BINARY || + format == THdfsFileFormat::RC_TEXT || format == THdfsFileFormat::SEQUENCE_FILE) && + (dynamic_cast(_hive_table) != nullptr || + dynamic_cast(_hive_table) != nullptr)) { + scanner = create_hive_jni_scanner(jni_scanner_create_options).release(); + } else { + std::string msg = fmt::format("unsupported hdfs file format: {}", format); + LOG(WARNING) << msg; + return Status::NotSupported(msg); + } + if (scanner == nullptr) { + return Status::InternalError("create hdfs scanner failed"); + } + _pool.add(scanner); + + RETURN_IF_ERROR(scanner->init(state, scanner_params)); + Status st = scanner->open(state); + if (!st.ok()) { + return scanner->reinterpret_status(st); + } + _scanner = scanner; + return Status::OK(); } -} -Status HiveDataSource::get_next(RuntimeState* state, ChunkPtr* chunk) { - if (_no_data) { - return Status::EndOfFile("no data"); + void HiveDataSource::close(RuntimeState * state) { + if (_scanner != nullptr) { + if (!_scanner->has_split_tasks()) { + COUNTER_UPDATE(_profile.scan_ranges_counter, 1); + } + _scanner->close(); + } + Expr::close(_min_max_conjunct_ctxs, state); + Expr::close(_partition_conjunct_ctxs, state); + Expr::close(_scanner_conjunct_ctxs, state); + for (auto& it : _conjunct_ctxs_by_slot) { + Expr::close(it.second, state); + } } - do { - RETURN_IF_ERROR(_init_chunk_if_needed(chunk, _runtime_state->chunk_size())); - RETURN_IF_ERROR(_scanner->get_next(state, chunk)); - } while ((*chunk)->num_rows() == 0); + Status HiveDataSource::get_next(RuntimeState * state, ChunkPtr * chunk) { + if (_no_data) { + return Status::EndOfFile("no data"); + } - // The column order of chunk is required to be invariable. In order to simplify the logic of each scanner, - // we force to reorder the columns of chunk, so scanner doesn't have to care about the column order anymore. - // The overhead of reorder is negligible because we only swap columns. - ChunkHelper::reorder_chunk(*_tuple_desc, chunk->get()); + do { + RETURN_IF_ERROR(_init_chunk_if_needed(chunk, _runtime_state->chunk_size())); + RETURN_IF_ERROR(_scanner->get_next(state, chunk)); + } while ((*chunk)->num_rows() == 0); - return Status::OK(); -} + // The column order of chunk is required to be invariable. In order to simplify the logic of each scanner, + // we force to reorder the columns of chunk, so scanner doesn't have to care about the column order anymore. + // The overhead of reorder is negligible because we only swap columns. + ChunkHelper::reorder_chunk(*_tuple_desc, chunk->get()); -Status HiveDataSource::_init_chunk_if_needed(ChunkPtr* chunk, size_t n) { - if ((*chunk) != nullptr && (*chunk)->num_columns() != 0) { return Status::OK(); } - *chunk = ChunkHelper::new_chunk(*_tuple_desc, n); - - if (!_equality_delete_slots.empty()) { - std::map id_to_slots; - for (const auto& slot : _tuple_desc->slots()) { - id_to_slots.emplace(slot->id(), slot); + Status HiveDataSource::_init_chunk_if_needed(ChunkPtr * chunk, size_t n) { + if ((*chunk) != nullptr && (*chunk)->num_columns() != 0) { + return Status::OK(); } - for (const auto& slot : _equality_delete_slots) { - if (!id_to_slots.contains(slot->id())) { - const auto column = ColumnHelper::create_column(slot->type(), slot->is_nullable()); - column->reserve(n); - (*chunk)->append_column(column, slot->id()); + *chunk = ChunkHelper::new_chunk(*_tuple_desc, n); + + if (!_equality_delete_slots.empty()) { + std::map id_to_slots; + for (const auto& slot : _tuple_desc->slots()) { + id_to_slots.emplace(slot->id(), slot); + } + + for (const auto& slot : _equality_delete_slots) { + if (!id_to_slots.contains(slot->id())) { + const auto column = ColumnHelper::create_column(slot->type(), slot->is_nullable()); + column->reserve(n); + (*chunk)->append_column(column, slot->id()); + } } } + return Status::OK(); } - return Status::OK(); -} -const std::string HiveDataSource::get_custom_coredump_msg() const { - const std::string path = !_scan_range.relative_path.empty() ? _scan_range.relative_path : _scan_range.full_path; - return strings::Substitute("Hive file path: $0, partition id: $1, length: $2, offset: $3", path, - _scan_range.partition_id, _scan_range.length, _scan_range.offset); -} + const std::string HiveDataSource::get_custom_coredump_msg() const { + const std::string path = !_scan_range.relative_path.empty() ? _scan_range.relative_path : _scan_range.full_path; + return strings::Substitute("Hive file path: $0, partition id: $1, length: $2, offset: $3", path, + _scan_range.partition_id, _scan_range.length, _scan_range.offset); + } -int64_t HiveDataSource::raw_rows_read() const { - if (_scanner == nullptr) return 0; - return _scanner->raw_rows_read(); -} -int64_t HiveDataSource::num_rows_read() const { - if (_scanner == nullptr) return 0; - return _scanner->num_rows_read(); -} -int64_t HiveDataSource::num_bytes_read() const { - if (_scanner == nullptr) return 0; - return _scanner->num_bytes_read(); -} -int64_t HiveDataSource::cpu_time_spent() const { - if (_scanner == nullptr) return 0; - return _scanner->cpu_time_spent(); -} + int64_t HiveDataSource::raw_rows_read() const { + if (_scanner == nullptr) return 0; + return _scanner->raw_rows_read(); + } + int64_t HiveDataSource::num_rows_read() const { + if (_scanner == nullptr) return 0; + return _scanner->num_rows_read(); + } + int64_t HiveDataSource::num_bytes_read() const { + if (_scanner == nullptr) return 0; + return _scanner->num_bytes_read(); + } + int64_t HiveDataSource::cpu_time_spent() const { + if (_scanner == nullptr) return 0; + return _scanner->cpu_time_spent(); + } -int64_t HiveDataSource::io_time_spent() const { - if (_scanner == nullptr) return 0; - return _scanner->io_time_spent(); -} + int64_t HiveDataSource::io_time_spent() const { + if (_scanner == nullptr) return 0; + return _scanner->io_time_spent(); + } -int64_t HiveDataSource::estimated_mem_usage() const { - if (_scanner == nullptr) return 0; - return _scanner->estimated_mem_usage(); -} + int64_t HiveDataSource::estimated_mem_usage() const { + if (_scanner == nullptr) return 0; + return _scanner->estimated_mem_usage(); + } -void HiveDataSourceProvider::peek_scan_ranges(const std::vector& scan_ranges) { - for (const auto& sc : scan_ranges) { - const TScanRange& x = sc.scan_range; - if (!x.__isset.hdfs_scan_range) continue; - const THdfsScanRange& y = x.hdfs_scan_range; - _max_file_length = std::max(_max_file_length, y.file_length); + void HiveDataSourceProvider::peek_scan_ranges(const std::vector& scan_ranges) { + for (const auto& sc : scan_ranges) { + const TScanRange& x = sc.scan_range; + if (!x.__isset.hdfs_scan_range) continue; + const THdfsScanRange& y = x.hdfs_scan_range; + _max_file_length = std::max(_max_file_length, y.file_length); + } } -} -void HiveDataSourceProvider::default_data_source_mem_bytes(int64_t* min_value, int64_t* max_value) { - DataSourceProvider::default_data_source_mem_bytes(min_value, max_value); - // here we compute as default mem bytes = max(MIN_SIZE, min(max_file_length, MAX_SIZE)) - int64_t size = std::max(*min_value, std::min(_max_file_length * 3 / 2, *max_value)); - *min_value = *max_value = size; -} + void HiveDataSourceProvider::default_data_source_mem_bytes(int64_t * min_value, int64_t * max_value) { + DataSourceProvider::default_data_source_mem_bytes(min_value, max_value); + // here we compute as default mem bytes = max(MIN_SIZE, min(max_file_length, MAX_SIZE)) + int64_t size = std::max(*min_value, std::min(_max_file_length * 3 / 2, *max_value)); + *min_value = *max_value = size; + } -void HiveDataSource::get_split_tasks(std::vector* split_tasks) { - if (_scanner == nullptr) return; - _scanner->move_split_tasks(split_tasks); -} + void HiveDataSource::get_split_tasks(std::vector * split_tasks) { + if (_scanner == nullptr) return; + _scanner->move_split_tasks(split_tasks); + } } // namespace starrocks::connector diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java b/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java index 4dfad1ba9c9fb..dac08113d7a79 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java @@ -443,11 +443,8 @@ public class SessionVariable implements Serializable, Writable, Cloneable { public static final String ENABLE_FILE_METACACHE = "enable_file_metacache"; public static final String HUDI_MOR_FORCE_JNI_READER = "hudi_mor_force_jni_reader"; -<<<<<<< HEAD public static final String PAIMON_FORCE_JNI_READER = "paimon_force_jni_reader"; -======= public static final String ENABLE_DYNAMIC_PRUNE_SCAN_RANGE = "enable_dynamic_prune_scan_range"; ->>>>>>> 3def3c1d3d ([Feature] Support dynamic partition pruning (#30319)) public static final String IO_TASKS_PER_SCAN_OPERATOR = "io_tasks_per_scan_operator"; public static final String CONNECTOR_IO_TASKS_PER_SCAN_OPERATOR = "connector_io_tasks_per_scan_operator"; public static final String ENABLE_CONNECTOR_ADAPTIVE_IO_TASKS = "enable_connector_adaptive_io_tasks"; From 34e633583df61a4011a70a9816a69c6aabba0de1 Mon Sep 17 00:00:00 2001 From: Letian Jiang Date: Tue, 6 Aug 2024 21:40:34 +0800 Subject: [PATCH 6/6] format Signed-off-by: Letian Jiang --- be/src/connector/hive_connector.cpp | 1072 +++++++++++++-------------- be/src/exec/hdfs_scanner.cpp | 9 - be/test/exec/hdfs_scanner_test.cpp | 18 +- 3 files changed, 544 insertions(+), 555 deletions(-) diff --git a/be/src/connector/hive_connector.cpp b/be/src/connector/hive_connector.cpp index d3c0f5ac3ed4e..93222fbf54c7e 100644 --- a/be/src/connector/hive_connector.cpp +++ b/be/src/connector/hive_connector.cpp @@ -111,627 +111,625 @@ Status HiveDataSource::open(RuntimeState* state) { } if (state->query_options().__isset.enable_connector_split_io_tasks) { _enable_split_tasks = state->query_options().enable_connector_split_io_tasks; - if (state->query_options().__isset.enable_dynamic_prune_scan_range) { - _enable_dynamic_prune_scan_range = state->query_options().enable_dynamic_prune_scan_range; - } + } + if (state->query_options().__isset.enable_dynamic_prune_scan_range) { + _enable_dynamic_prune_scan_range = state->query_options().enable_dynamic_prune_scan_range; + } - RETURN_IF_ERROR(_init_conjunct_ctxs(state)); - _init_tuples_and_slots(state); - _init_counter(state); - RETURN_IF_ERROR(_init_partition_values()); - if (_filter_by_eval_partition_conjuncts) { - _no_data = true; - return Status::OK(); - } - RETURN_IF_ERROR(_init_scanner(state)); + RETURN_IF_ERROR(_init_conjunct_ctxs(state)); + _init_tuples_and_slots(state); + _init_counter(state); + RETURN_IF_ERROR(_init_partition_values()); + if (_filter_by_eval_partition_conjuncts) { + _no_data = true; return Status::OK(); } + RETURN_IF_ERROR(_init_scanner(state)); + return Status::OK(); +} - void HiveDataSource::_update_has_any_predicate() { - auto f = [&]() { - if (_runtime_filters != nullptr && _runtime_filters->size() > 0) return true; - return false; - }; - _has_any_predicate = f(); +void HiveDataSource::_update_has_any_predicate() { + auto f = [&]() { + if (_runtime_filters != nullptr && _runtime_filters->size() > 0) return true; + return false; + }; + _has_any_predicate = f(); +} + +Status HiveDataSource::_init_conjunct_ctxs(RuntimeState* state) { + const auto& hdfs_scan_node = _provider->_hdfs_scan_node; + if (hdfs_scan_node.__isset.min_max_conjuncts) { + RETURN_IF_ERROR( + Expr::create_expr_trees(&_pool, hdfs_scan_node.min_max_conjuncts, &_min_max_conjunct_ctxs, state)); } - Status HiveDataSource::_init_conjunct_ctxs(RuntimeState * state) { - const auto& hdfs_scan_node = _provider->_hdfs_scan_node; - if (hdfs_scan_node.__isset.min_max_conjuncts) { - RETURN_IF_ERROR( - Expr::create_expr_trees(&_pool, hdfs_scan_node.min_max_conjuncts, &_min_max_conjunct_ctxs, state)); - } + if (hdfs_scan_node.__isset.partition_conjuncts) { + RETURN_IF_ERROR( + Expr::create_expr_trees(&_pool, hdfs_scan_node.partition_conjuncts, &_partition_conjunct_ctxs, state)); + _has_partition_conjuncts = true; + } - if (hdfs_scan_node.__isset.partition_conjuncts) { - RETURN_IF_ERROR(Expr::create_expr_trees(&_pool, hdfs_scan_node.partition_conjuncts, - &_partition_conjunct_ctxs, state)); - _has_partition_conjuncts = true; - } + if (hdfs_scan_node.__isset.case_sensitive) { + _case_sensitive = hdfs_scan_node.case_sensitive; + } - if (hdfs_scan_node.__isset.case_sensitive) { - _case_sensitive = hdfs_scan_node.case_sensitive; - } + RETURN_IF_ERROR(Expr::prepare(_min_max_conjunct_ctxs, state)); + RETURN_IF_ERROR(Expr::prepare(_partition_conjunct_ctxs, state)); + RETURN_IF_ERROR(Expr::open(_min_max_conjunct_ctxs, state)); + RETURN_IF_ERROR(Expr::open(_partition_conjunct_ctxs, state)); + _update_has_any_predicate(); - RETURN_IF_ERROR(Expr::prepare(_min_max_conjunct_ctxs, state)); - RETURN_IF_ERROR(Expr::prepare(_partition_conjunct_ctxs, state)); - RETURN_IF_ERROR(Expr::open(_min_max_conjunct_ctxs, state)); - RETURN_IF_ERROR(Expr::open(_partition_conjunct_ctxs, state)); - _update_has_any_predicate(); + RETURN_IF_ERROR(_decompose_conjunct_ctxs(state)); + return Status::OK(); +} - RETURN_IF_ERROR(_decompose_conjunct_ctxs(state)); - return Status::OK(); - } +Status HiveDataSource::_init_partition_values() { + if (!(_hive_table != nullptr && _has_partition_columns)) return Status::OK(); - Status HiveDataSource::_init_partition_values() { - if (!(_hive_table != nullptr && _has_partition_columns)) return Status::OK(); + auto* partition_desc = _hive_table->get_partition(_scan_range.partition_id); + if (partition_desc == nullptr) { + return Status::InternalError( + fmt::format("Plan inconsistency. scan_range.partition_id = {} not found in partition description map", + _scan_range.partition_id)); + } - auto* partition_desc = _hive_table->get_partition(_scan_range.partition_id); - if (partition_desc == nullptr) { - return Status::InternalError(fmt::format( - "Plan inconsistency. scan_range.partition_id = {} not found in partition description map", - _scan_range.partition_id)); - } + const auto& partition_values = partition_desc->partition_key_value_evals(); + _partition_values = partition_desc->partition_key_value_evals(); - const auto& partition_values = partition_desc->partition_key_value_evals(); - _partition_values = partition_desc->partition_key_value_evals(); - - // init partition chunk - auto partition_chunk = std::make_shared(); - for (int i = 0; i < _partition_slots.size(); i++) { - SlotId slot_id = _partition_slots[i]->id(); - int partition_col_idx = _partition_index_in_hdfs_partition_columns[i]; - ASSIGN_OR_RETURN(auto partition_value_col, partition_values[partition_col_idx]->evaluate(nullptr)); - DCHECK(partition_value_col->is_constant()); - partition_chunk->append_column(partition_value_col, slot_id); - } + // init partition chunk + auto partition_chunk = std::make_shared(); + for (int i = 0; i < _partition_slots.size(); i++) { + SlotId slot_id = _partition_slots[i]->id(); + int partition_col_idx = _partition_index_in_hdfs_partition_columns[i]; + ASSIGN_OR_RETURN(auto partition_value_col, partition_values[partition_col_idx]->evaluate(nullptr)); + DCHECK(partition_value_col->is_constant()); + partition_chunk->append_column(partition_value_col, slot_id); + } - // eval conjuncts and skip if no rows. - if (_has_scan_range_indicate_const_column) { - std::vector ctxs; - for (SlotId slotId : _scan_range.identity_partition_slot_ids) { - if (_conjunct_ctxs_by_slot.find(slotId) != _conjunct_ctxs_by_slot.end()) { - ctxs.insert(ctxs.end(), _conjunct_ctxs_by_slot.at(slotId).begin(), - _conjunct_ctxs_by_slot.at(slotId).end()); - } + // eval conjuncts and skip if no rows. + if (_has_scan_range_indicate_const_column) { + std::vector ctxs; + for (SlotId slotId : _scan_range.identity_partition_slot_ids) { + if (_conjunct_ctxs_by_slot.find(slotId) != _conjunct_ctxs_by_slot.end()) { + ctxs.insert(ctxs.end(), _conjunct_ctxs_by_slot.at(slotId).begin(), + _conjunct_ctxs_by_slot.at(slotId).end()); } - RETURN_IF_ERROR(ExecNode::eval_conjuncts(ctxs, partition_chunk.get())); - } else if (_has_partition_conjuncts) { - RETURN_IF_ERROR(ExecNode::eval_conjuncts(_partition_conjunct_ctxs, partition_chunk.get())); } + RETURN_IF_ERROR(ExecNode::eval_conjuncts(ctxs, partition_chunk.get())); + } else if (_has_partition_conjuncts) { + RETURN_IF_ERROR(ExecNode::eval_conjuncts(_partition_conjunct_ctxs, partition_chunk.get())); + } + if (!partition_chunk->has_rows()) { + _filter_by_eval_partition_conjuncts = true; + return Status::OK(); + } + + if (_enable_dynamic_prune_scan_range && _runtime_filters) { + _init_rf_counters(); + _runtime_filters->evaluate_partial_chunk(partition_chunk.get(), runtime_bloom_filter_eval_context); if (!partition_chunk->has_rows()) { _filter_by_eval_partition_conjuncts = true; return Status::OK(); } + } - if (_enable_dynamic_prune_scan_range && _runtime_filters) { - _init_rf_counters(); - _runtime_filters->evaluate_partial_chunk(partition_chunk.get(), runtime_bloom_filter_eval_context); - if (!partition_chunk->has_rows()) { - _filter_by_eval_partition_conjuncts = true; - return Status::OK(); - } - } + return Status::OK(); +} - return Status::OK(); +int32_t HiveDataSource::scan_range_indicate_const_column_index(SlotId id) const { + if (!_scan_range.__isset.identity_partition_slot_ids) { + return -1; + } + auto it = std::find(_scan_range.identity_partition_slot_ids.begin(), _scan_range.identity_partition_slot_ids.end(), + id); + if (it == _scan_range.identity_partition_slot_ids.end()) { + return -1; + } else { + return it - _scan_range.identity_partition_slot_ids.begin(); } +} - int32_t HiveDataSource::scan_range_indicate_const_column_index(SlotId id) const { - if (!_scan_range.__isset.identity_partition_slot_ids) { - return -1; - } - auto it = std::find(_scan_range.identity_partition_slot_ids.begin(), - _scan_range.identity_partition_slot_ids.end(), id); - if (it == _scan_range.identity_partition_slot_ids.end()) { - return -1; +void HiveDataSource::_init_tuples_and_slots(RuntimeState* state) { + const auto& hdfs_scan_node = _provider->_hdfs_scan_node; + if (hdfs_scan_node.__isset.min_max_tuple_id) { + _min_max_tuple_id = hdfs_scan_node.min_max_tuple_id; + _min_max_tuple_desc = state->desc_tbl().get_tuple_descriptor(_min_max_tuple_id); + } + + const auto& slots = _tuple_desc->slots(); + for (int i = 0; i < slots.size(); i++) { + if (_hive_table != nullptr && _hive_table->is_partition_col(slots[i])) { + _partition_slots.push_back(slots[i]); + _partition_index_in_chunk.push_back(i); + _partition_index_in_hdfs_partition_columns.push_back(_hive_table->get_partition_col_index(slots[i])); + _has_partition_columns = true; + } else if (int32_t index = scan_range_indicate_const_column_index(slots[i]->id()); index >= 0) { + _partition_slots.push_back(slots[i]); + _partition_index_in_chunk.push_back(i); + _partition_index_in_hdfs_partition_columns.push_back(index); + _has_partition_columns = true; + _has_scan_range_indicate_const_column = true; } else { - return it - _scan_range.identity_partition_slot_ids.begin(); + _materialize_slots.push_back(slots[i]); + _materialize_index_in_chunk.push_back(i); } } - void HiveDataSource::_init_tuples_and_slots(RuntimeState * state) { - const auto& hdfs_scan_node = _provider->_hdfs_scan_node; - if (hdfs_scan_node.__isset.min_max_tuple_id) { - _min_max_tuple_id = hdfs_scan_node.min_max_tuple_id; - _min_max_tuple_desc = state->desc_tbl().get_tuple_descriptor(_min_max_tuple_id); - } - - const auto& slots = _tuple_desc->slots(); - for (int i = 0; i < slots.size(); i++) { - if (_hive_table != nullptr && _hive_table->is_partition_col(slots[i])) { - _partition_slots.push_back(slots[i]); - _partition_index_in_chunk.push_back(i); - _partition_index_in_hdfs_partition_columns.push_back(_hive_table->get_partition_col_index(slots[i])); - _has_partition_columns = true; - } else if (int32_t index = scan_range_indicate_const_column_index(slots[i]->id()); index >= 0) { - _partition_slots.push_back(slots[i]); - _partition_index_in_chunk.push_back(i); - _partition_index_in_hdfs_partition_columns.push_back(index); - _has_partition_columns = true; - _has_scan_range_indicate_const_column = true; - } else { - _materialize_slots.push_back(slots[i]); - _materialize_index_in_chunk.push_back(i); - } + if (_scan_range.__isset.delete_column_slot_ids && !_scan_range.delete_column_slot_ids.empty()) { + std::map id_to_slots; + for (const auto& slot : _materialize_slots) { + id_to_slots.emplace(slot->id(), slot); } - if (_scan_range.__isset.delete_column_slot_ids && !_scan_range.delete_column_slot_ids.empty()) { - std::map id_to_slots; - for (const auto& slot : _materialize_slots) { - id_to_slots.emplace(slot->id(), slot); - } - - int32_t delete_column_index = slots.size(); - _delete_column_tuple_desc = state->desc_tbl().get_tuple_descriptor(_provider->_hdfs_scan_node.mor_tuple_id); + int32_t delete_column_index = slots.size(); + _delete_column_tuple_desc = state->desc_tbl().get_tuple_descriptor(_provider->_hdfs_scan_node.mor_tuple_id); - for (SlotDescriptor* d_slot_desc : _delete_column_tuple_desc->slots()) { - _equality_delete_slots.emplace_back(d_slot_desc); - if (!id_to_slots.contains(d_slot_desc->id())) { - _materialize_slots.push_back(d_slot_desc); - _materialize_index_in_chunk.push_back(delete_column_index++); - } + for (SlotDescriptor* d_slot_desc : _delete_column_tuple_desc->slots()) { + _equality_delete_slots.emplace_back(d_slot_desc); + if (!id_to_slots.contains(d_slot_desc->id())) { + _materialize_slots.push_back(d_slot_desc); + _materialize_index_in_chunk.push_back(delete_column_index++); } } + } - if (hdfs_scan_node.__isset.hive_column_names) { - _hive_column_names = hdfs_scan_node.hive_column_names; - } - if (hdfs_scan_node.__isset.case_sensitive) { - _case_sensitive = hdfs_scan_node.case_sensitive; - } - if (hdfs_scan_node.__isset.can_use_any_column) { - _can_use_any_column = hdfs_scan_node.can_use_any_column; + if (hdfs_scan_node.__isset.hive_column_names) { + _hive_column_names = hdfs_scan_node.hive_column_names; + } + if (hdfs_scan_node.__isset.case_sensitive) { + _case_sensitive = hdfs_scan_node.case_sensitive; + } + if (hdfs_scan_node.__isset.can_use_any_column) { + _can_use_any_column = hdfs_scan_node.can_use_any_column; + } + if (hdfs_scan_node.__isset.can_use_min_max_count_opt) { + _can_use_min_max_count_opt = hdfs_scan_node.can_use_min_max_count_opt; + } + if (hdfs_scan_node.__isset.use_partition_column_value_only) { + _use_partition_column_value_only = hdfs_scan_node.use_partition_column_value_only; + } + + // The reason why we need double check here is for iceberg table. + // for some partitions, partition column maybe is not constant value. + // If partition column is not constant value, we can not use this optimization, + // And we can not use `can_use_any_column` either. + // So checks are: + // 1. can_use_any_column = true + // 2. only one materialized slot + // 3. besides that, all slots are partition slots. + // 4. scan iceberg data file without equality delete files. + auto check_opt_on_iceberg = [&]() { + if (!_can_use_any_column) { + return false; } - if (hdfs_scan_node.__isset.can_use_min_max_count_opt) { - _can_use_min_max_count_opt = hdfs_scan_node.can_use_min_max_count_opt; + if ((_partition_slots.size() + 1) != slots.size()) { + return false; } - if (hdfs_scan_node.__isset.use_partition_column_value_only) { - _use_partition_column_value_only = hdfs_scan_node.use_partition_column_value_only; + if (_materialize_slots.size() != 1) { + return false; } - - // The reason why we need double check here is for iceberg table. - // for some partitions, partition column maybe is not constant value. - // If partition column is not constant value, we can not use this optimization, - // And we can not use `can_use_any_column` either. - // So checks are: - // 1. can_use_any_column = true - // 2. only one materialized slot - // 3. besides that, all slots are partition slots. - // 4. scan iceberg data file without equality delete files. - auto check_opt_on_iceberg = [&]() { - if (!_can_use_any_column) { - return false; - } - if ((_partition_slots.size() + 1) != slots.size()) { - return false; - } - if (_materialize_slots.size() != 1) { - return false; - } - if (!_scan_range.delete_column_slot_ids.empty()) { - return false; - } - return true; - }; - if (!check_opt_on_iceberg()) { - _use_partition_column_value_only = false; - _can_use_any_column = false; + if (!_scan_range.delete_column_slot_ids.empty()) { + return false; } + return true; + }; + if (!check_opt_on_iceberg()) { + _use_partition_column_value_only = false; + _can_use_any_column = false; } +} - Status HiveDataSource::_decompose_conjunct_ctxs(RuntimeState * state) { - if (_conjunct_ctxs.empty()) { - return Status::OK(); - } - - std::unordered_map slot_by_id; - for (SlotDescriptor* slot : _tuple_desc->slots()) { - slot_by_id[slot->id()] = slot; - } - - std::vector cloned_conjunct_ctxs; - RETURN_IF_ERROR(Expr::clone_if_not_exists(state, &_pool, _conjunct_ctxs, &cloned_conjunct_ctxs)); - - for (ExprContext* ctx : cloned_conjunct_ctxs) { - const Expr* root_expr = ctx->root(); - std::vector slot_ids; - root_expr->get_slot_ids(&slot_ids); - for (SlotId slot_id : slot_ids) { - _slots_in_conjunct.insert(slot_id); - } - - // For some conjunct like (a < 1) or (a > 7) - // slot_ids = (a, a), but actually there is only one slot. - bool single_slot = true; - for (int i = 1; i < slot_ids.size(); i++) { - if (slot_ids[i] != slot_ids[0]) { - single_slot = false; - break; - } - } - if (!single_slot || slot_ids.empty()) { - _scanner_conjunct_ctxs.emplace_back(ctx); - for (SlotId slot_id : slot_ids) { - _slots_of_mutli_slot_conjunct.insert(slot_id); - } - continue; - } - - SlotId slot_id = slot_ids[0]; - if (slot_by_id.find(slot_id) != slot_by_id.end()) { - if (_conjunct_ctxs_by_slot.find(slot_id) == _conjunct_ctxs_by_slot.end()) { - _conjunct_ctxs_by_slot.insert({slot_id, std::vector()}); - } - _conjunct_ctxs_by_slot[slot_id].emplace_back(ctx); - } - } +Status HiveDataSource::_decompose_conjunct_ctxs(RuntimeState* state) { + if (_conjunct_ctxs.empty()) { return Status::OK(); } - void HiveDataSource::_init_counter(RuntimeState * state) { - const auto& hdfs_scan_node = _provider->_hdfs_scan_node; - - _profile.runtime_profile = _runtime_profile; - _profile.raw_rows_read_counter = ADD_COUNTER(_runtime_profile, "RawRowsRead", TUnit::UNIT); - _profile.rows_read_counter = ADD_COUNTER(_runtime_profile, "RowsRead", TUnit::UNIT); - _profile.late_materialize_skip_rows_counter = - ADD_COUNTER(_runtime_profile, "LateMaterializeSkipRows", TUnit::UNIT); - _profile.scan_ranges_counter = ADD_COUNTER(_runtime_profile, "ScanRanges", TUnit::UNIT); - - _profile.reader_init_timer = ADD_TIMER(_runtime_profile, "ReaderInit"); - _profile.open_file_timer = ADD_TIMER(_runtime_profile, "OpenFile"); - _profile.expr_filter_timer = ADD_TIMER(_runtime_profile, "ExprFilterTime"); - - _profile.column_read_timer = ADD_TIMER(_runtime_profile, "ColumnReadTime"); - _profile.column_convert_timer = ADD_TIMER(_runtime_profile, "ColumnConvertTime"); - - { - static const char* prefix = "SharedBuffered"; - ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); - _profile.shared_buffered_shared_io_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "SharedIOBytes", TUnit::BYTES, prefix); - _profile.shared_buffered_shared_align_io_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "SharedAlignIOBytes", TUnit::BYTES, prefix); - _profile.shared_buffered_shared_io_count = - ADD_CHILD_COUNTER(_runtime_profile, "SharedIOCount", TUnit::UNIT, prefix); - _profile.shared_buffered_shared_io_timer = ADD_CHILD_TIMER(_runtime_profile, "SharedIOTime", prefix); - _profile.shared_buffered_direct_io_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DirectIOBytes", TUnit::BYTES, prefix); - _profile.shared_buffered_direct_io_count = - ADD_CHILD_COUNTER(_runtime_profile, "DirectIOCount", TUnit::UNIT, prefix); - _profile.shared_buffered_direct_io_timer = ADD_CHILD_TIMER(_runtime_profile, "DirectIOTime", prefix); - } + std::unordered_map slot_by_id; + for (SlotDescriptor* slot : _tuple_desc->slots()) { + slot_by_id[slot->id()] = slot; + } - if (_use_datacache) { - static const char* prefix = "DataCache"; - ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); - _profile.datacache_read_counter = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadCounter", TUnit::UNIT, prefix); - _profile.datacache_read_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBytes", TUnit::BYTES, prefix); - _profile.datacache_read_mem_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadMemBytes", TUnit::BYTES, "DataCacheReadBytes"); - _profile.datacache_read_disk_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadDiskBytes", TUnit::BYTES, "DataCacheReadBytes"); - _profile.datacache_skip_read_counter = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheSkipReadCounter", TUnit::UNIT, prefix); - _profile.datacache_skip_read_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheSkipReadBytes", TUnit::BYTES, prefix); - _profile.datacache_read_timer = ADD_CHILD_TIMER(_runtime_profile, "DataCacheReadTimer", prefix); - _profile.datacache_write_counter = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteCounter", TUnit::UNIT, prefix); - _profile.datacache_write_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteBytes", TUnit::BYTES, prefix); - _profile.datacache_write_timer = ADD_CHILD_TIMER(_runtime_profile, "DataCacheWriteTimer", prefix); - _profile.datacache_write_fail_counter = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteFailCounter", TUnit::UNIT, prefix); - _profile.datacache_write_fail_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteFailBytes", TUnit::BYTES, prefix); - _profile.datacache_read_block_buffer_counter = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBlockBufferCounter", TUnit::UNIT, prefix); - _profile.datacache_read_block_buffer_bytes = - ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBlockBufferBytes", TUnit::BYTES, prefix); - } + std::vector cloned_conjunct_ctxs; + RETURN_IF_ERROR(Expr::clone_if_not_exists(state, &_pool, _conjunct_ctxs, &cloned_conjunct_ctxs)); - { - static const char* prefix = "InputStream"; - ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); - _profile.app_io_bytes_read_counter = - ADD_CHILD_COUNTER(_runtime_profile, "AppIOBytesRead", TUnit::BYTES, prefix); - _profile.app_io_timer = ADD_CHILD_TIMER(_runtime_profile, "AppIOTime", prefix); - _profile.app_io_counter = ADD_CHILD_COUNTER(_runtime_profile, "AppIOCounter", TUnit::UNIT, prefix); - _profile.fs_bytes_read_counter = ADD_CHILD_COUNTER(_runtime_profile, "FSIOBytesRead", TUnit::BYTES, prefix); - _profile.fs_io_counter = ADD_CHILD_COUNTER(_runtime_profile, "FSIOCounter", TUnit::UNIT, prefix); - _profile.fs_io_timer = ADD_CHILD_TIMER(_runtime_profile, "FSIOTime", prefix); + for (ExprContext* ctx : cloned_conjunct_ctxs) { + const Expr* root_expr = ctx->root(); + std::vector slot_ids; + root_expr->get_slot_ids(&slot_ids); + for (SlotId slot_id : slot_ids) { + _slots_in_conjunct.insert(slot_id); } - if (hdfs_scan_node.__isset.table_name) { - _runtime_profile->add_info_string("Table", hdfs_scan_node.table_name); - } - if (hdfs_scan_node.__isset.sql_predicates) { - _runtime_profile->add_info_string("Predicates", hdfs_scan_node.sql_predicates); - } - if (hdfs_scan_node.__isset.min_max_sql_predicates) { - _runtime_profile->add_info_string("PredicatesMinMax", hdfs_scan_node.min_max_sql_predicates); + // For some conjunct like (a < 1) or (a > 7) + // slot_ids = (a, a), but actually there is only one slot. + bool single_slot = true; + for (int i = 1; i < slot_ids.size(); i++) { + if (slot_ids[i] != slot_ids[0]) { + single_slot = false; + break; + } } - if (hdfs_scan_node.__isset.partition_sql_predicates) { - _runtime_profile->add_info_string("PredicatesPartition", hdfs_scan_node.partition_sql_predicates); + if (!single_slot || slot_ids.empty()) { + _scanner_conjunct_ctxs.emplace_back(ctx); + for (SlotId slot_id : slot_ids) { + _slots_of_mutli_slot_conjunct.insert(slot_id); + } + continue; } - } - void HiveDataSource::_init_rf_counters() { - auto* root = _runtime_profile; - if (runtime_bloom_filter_eval_context.join_runtime_filter_timer == nullptr) { - static const char* prefix = "DynamicPruneScanRange"; - ADD_COUNTER(root, prefix, TUnit::NONE); - runtime_bloom_filter_eval_context.join_runtime_filter_timer = - ADD_CHILD_TIMER(root, "JoinRuntimeFilterTime", prefix); - runtime_bloom_filter_eval_context.join_runtime_filter_hash_timer = - ADD_CHILD_TIMER(root, "JoinRuntimeFilterHashTime", prefix); - runtime_bloom_filter_eval_context.join_runtime_filter_input_counter = - ADD_CHILD_COUNTER(root, "JoinRuntimeFilterInputScanRanges", TUnit::UNIT, prefix); - runtime_bloom_filter_eval_context.join_runtime_filter_output_counter = - ADD_CHILD_COUNTER(root, "JoinRuntimeFilterOutputScanRanges", TUnit::UNIT, prefix); - runtime_bloom_filter_eval_context.join_runtime_filter_eval_counter = - ADD_CHILD_COUNTER(root, "JoinRuntimeFilterEvaluate", TUnit::UNIT, prefix); + SlotId slot_id = slot_ids[0]; + if (slot_by_id.find(slot_id) != slot_by_id.end()) { + if (_conjunct_ctxs_by_slot.find(slot_id) == _conjunct_ctxs_by_slot.end()) { + _conjunct_ctxs_by_slot.insert({slot_id, std::vector()}); + } + _conjunct_ctxs_by_slot[slot_id].emplace_back(ctx); } } + return Status::OK(); +} - Status HiveDataSource::_init_scanner(RuntimeState * state) { - SCOPED_TIMER(_profile.open_file_timer); - - const auto& scan_range = _scan_range; - std::string native_file_path = scan_range.full_path; - if (_hive_table != nullptr && _hive_table->has_partition() && !_hive_table->has_base_path()) { - auto* partition_desc = _hive_table->get_partition(scan_range.partition_id); - if (partition_desc == nullptr) { - return Status::InternalError(fmt::format( - "Plan inconsistency. scan_range.partition_id = {} not found in partition description map", - scan_range.partition_id)); - } - std::filesystem::path file_path(partition_desc->location()); - file_path /= scan_range.relative_path; - native_file_path = file_path.native(); - } - if (native_file_path.empty()) { - native_file_path = _hive_table->get_base_path() + scan_range.relative_path; - } +void HiveDataSource::_init_counter(RuntimeState* state) { + const auto& hdfs_scan_node = _provider->_hdfs_scan_node; - const auto& hdfs_scan_node = _provider->_hdfs_scan_node; - auto fsOptions = - FSOptions(hdfs_scan_node.__isset.cloud_configuration ? &hdfs_scan_node.cloud_configuration : nullptr); - - ASSIGN_OR_RETURN(auto fs, FileSystem::CreateUniqueFromString(native_file_path, fsOptions)); - - HdfsScannerParams scanner_params; - scanner_params.runtime_filter_collector = _runtime_filters; - scanner_params.scan_range = &scan_range; - scanner_params.fs = _pool.add(fs.release()); - scanner_params.path = native_file_path; - scanner_params.file_size = _scan_range.file_length; - scanner_params.modification_time = _scan_range.modification_time; - scanner_params.tuple_desc = _tuple_desc; - scanner_params.materialize_slots = _materialize_slots; - scanner_params.materialize_index_in_chunk = _materialize_index_in_chunk; - scanner_params.partition_slots = _partition_slots; - scanner_params.partition_index_in_chunk = _partition_index_in_chunk; - scanner_params._partition_index_in_hdfs_partition_columns = _partition_index_in_hdfs_partition_columns; - scanner_params.partition_values = _partition_values; - scanner_params.conjunct_ctxs = _scanner_conjunct_ctxs; - scanner_params.conjunct_ctxs_by_slot = _conjunct_ctxs_by_slot; - scanner_params.slots_in_conjunct = _slots_in_conjunct; - scanner_params.slots_of_mutli_slot_conjunct = _slots_of_mutli_slot_conjunct; - scanner_params.min_max_conjunct_ctxs = _min_max_conjunct_ctxs; - scanner_params.min_max_tuple_desc = _min_max_tuple_desc; - scanner_params.hive_column_names = &_hive_column_names; - scanner_params.case_sensitive = _case_sensitive; - scanner_params.profile = &_profile; - scanner_params.lazy_column_coalesce_counter = get_lazy_column_coalesce_counter(); - scanner_params.split_context = down_cast(_split_context); - scanner_params.enable_split_tasks = _enable_split_tasks; - if (state->query_options().__isset.connector_max_split_size) { - scanner_params.connector_max_split_size = state->query_options().connector_max_split_size; - } + _profile.runtime_profile = _runtime_profile; + _profile.raw_rows_read_counter = ADD_COUNTER(_runtime_profile, "RawRowsRead", TUnit::UNIT); + _profile.rows_read_counter = ADD_COUNTER(_runtime_profile, "RowsRead", TUnit::UNIT); + _profile.late_materialize_skip_rows_counter = ADD_COUNTER(_runtime_profile, "LateMaterializeSkipRows", TUnit::UNIT); + _profile.scan_ranges_counter = ADD_COUNTER(_runtime_profile, "ScanRanges", TUnit::UNIT); + + _profile.reader_init_timer = ADD_TIMER(_runtime_profile, "ReaderInit"); + _profile.open_file_timer = ADD_TIMER(_runtime_profile, "OpenFile"); + _profile.expr_filter_timer = ADD_TIMER(_runtime_profile, "ExprFilterTime"); + + _profile.column_read_timer = ADD_TIMER(_runtime_profile, "ColumnReadTime"); + _profile.column_convert_timer = ADD_TIMER(_runtime_profile, "ColumnConvertTime"); + + { + static const char* prefix = "SharedBuffered"; + ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); + _profile.shared_buffered_shared_io_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "SharedIOBytes", TUnit::BYTES, prefix); + _profile.shared_buffered_shared_align_io_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "SharedAlignIOBytes", TUnit::BYTES, prefix); + _profile.shared_buffered_shared_io_count = + ADD_CHILD_COUNTER(_runtime_profile, "SharedIOCount", TUnit::UNIT, prefix); + _profile.shared_buffered_shared_io_timer = ADD_CHILD_TIMER(_runtime_profile, "SharedIOTime", prefix); + _profile.shared_buffered_direct_io_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DirectIOBytes", TUnit::BYTES, prefix); + _profile.shared_buffered_direct_io_count = + ADD_CHILD_COUNTER(_runtime_profile, "DirectIOCount", TUnit::UNIT, prefix); + _profile.shared_buffered_direct_io_timer = ADD_CHILD_TIMER(_runtime_profile, "DirectIOTime", prefix); + } + + if (_use_datacache) { + static const char* prefix = "DataCache"; + ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); + _profile.datacache_read_counter = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadCounter", TUnit::UNIT, prefix); + _profile.datacache_read_bytes = ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBytes", TUnit::BYTES, prefix); + _profile.datacache_read_mem_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadMemBytes", TUnit::BYTES, "DataCacheReadBytes"); + _profile.datacache_read_disk_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadDiskBytes", TUnit::BYTES, "DataCacheReadBytes"); + _profile.datacache_skip_read_counter = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheSkipReadCounter", TUnit::UNIT, prefix); + _profile.datacache_skip_read_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheSkipReadBytes", TUnit::BYTES, prefix); + _profile.datacache_read_timer = ADD_CHILD_TIMER(_runtime_profile, "DataCacheReadTimer", prefix); + _profile.datacache_write_counter = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteCounter", TUnit::UNIT, prefix); + _profile.datacache_write_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteBytes", TUnit::BYTES, prefix); + _profile.datacache_write_timer = ADD_CHILD_TIMER(_runtime_profile, "DataCacheWriteTimer", prefix); + _profile.datacache_write_fail_counter = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteFailCounter", TUnit::UNIT, prefix); + _profile.datacache_write_fail_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheWriteFailBytes", TUnit::BYTES, prefix); + _profile.datacache_read_block_buffer_counter = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBlockBufferCounter", TUnit::UNIT, prefix); + _profile.datacache_read_block_buffer_bytes = + ADD_CHILD_COUNTER(_runtime_profile, "DataCacheReadBlockBufferBytes", TUnit::BYTES, prefix); + } + + { + static const char* prefix = "InputStream"; + ADD_COUNTER(_runtime_profile, prefix, TUnit::NONE); + _profile.app_io_bytes_read_counter = + ADD_CHILD_COUNTER(_runtime_profile, "AppIOBytesRead", TUnit::BYTES, prefix); + _profile.app_io_timer = ADD_CHILD_TIMER(_runtime_profile, "AppIOTime", prefix); + _profile.app_io_counter = ADD_CHILD_COUNTER(_runtime_profile, "AppIOCounter", TUnit::UNIT, prefix); + _profile.fs_bytes_read_counter = ADD_CHILD_COUNTER(_runtime_profile, "FSIOBytesRead", TUnit::BYTES, prefix); + _profile.fs_io_counter = ADD_CHILD_COUNTER(_runtime_profile, "FSIOCounter", TUnit::UNIT, prefix); + _profile.fs_io_timer = ADD_CHILD_TIMER(_runtime_profile, "FSIOTime", prefix); + } + + if (hdfs_scan_node.__isset.table_name) { + _runtime_profile->add_info_string("Table", hdfs_scan_node.table_name); + } + if (hdfs_scan_node.__isset.sql_predicates) { + _runtime_profile->add_info_string("Predicates", hdfs_scan_node.sql_predicates); + } + if (hdfs_scan_node.__isset.min_max_sql_predicates) { + _runtime_profile->add_info_string("PredicatesMinMax", hdfs_scan_node.min_max_sql_predicates); + } + if (hdfs_scan_node.__isset.partition_sql_predicates) { + _runtime_profile->add_info_string("PredicatesPartition", hdfs_scan_node.partition_sql_predicates); + } +} - if (!_equality_delete_slots.empty()) { - MORParams& mor_params = scanner_params.mor_params; - mor_params.tuple_desc = _tuple_desc; - mor_params.equality_slots = _equality_delete_slots; - mor_params.delete_column_tuple_desc = _delete_column_tuple_desc; - mor_params.mor_tuple_id = _provider->_hdfs_scan_node.mor_tuple_id; - mor_params.runtime_profile = _runtime_profile; - } +void HiveDataSource::_init_rf_counters() { + auto* root = _runtime_profile; + if (runtime_bloom_filter_eval_context.join_runtime_filter_timer == nullptr) { + static const char* prefix = "DynamicPruneScanRange"; + ADD_COUNTER(root, prefix, TUnit::NONE); + runtime_bloom_filter_eval_context.join_runtime_filter_timer = + ADD_CHILD_TIMER(root, "JoinRuntimeFilterTime", prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_hash_timer = + ADD_CHILD_TIMER(root, "JoinRuntimeFilterHashTime", prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_input_counter = + ADD_CHILD_COUNTER(root, "JoinRuntimeFilterInputScanRanges", TUnit::UNIT, prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_output_counter = + ADD_CHILD_COUNTER(root, "JoinRuntimeFilterOutputScanRanges", TUnit::UNIT, prefix); + runtime_bloom_filter_eval_context.join_runtime_filter_eval_counter = + ADD_CHILD_COUNTER(root, "JoinRuntimeFilterEvaluate", TUnit::UNIT, prefix); + } +} - for (const auto& delete_file : scan_range.delete_files) { - scanner_params.deletes.emplace_back(&delete_file); - } +Status HiveDataSource::_init_scanner(RuntimeState* state) { + SCOPED_TIMER(_profile.open_file_timer); - if (dynamic_cast(_hive_table)) { - auto tbl = dynamic_cast(_hive_table); - scanner_params.iceberg_schema = tbl->get_iceberg_schema(); - scanner_params.iceberg_equal_delete_schema = tbl->get_iceberg_equal_delete_schema(); - } - if (scan_range.__isset.paimon_deletion_file && !scan_range.paimon_deletion_file.path.empty()) { - scanner_params.paimon_deletion_file = - std::make_shared(scan_range.paimon_deletion_file); - } - scanner_params.use_datacache = _use_datacache; - scanner_params.enable_populate_datacache = _enable_populate_datacache; - scanner_params.enable_datacache_async_populate_mode = _enable_datacache_aync_populate_mode; - scanner_params.enable_datacache_io_adaptor = _enable_datacache_io_adaptor; - scanner_params.datacache_evict_probability = _datacache_evict_probability; - scanner_params.can_use_any_column = _can_use_any_column; - scanner_params.can_use_min_max_count_opt = _can_use_min_max_count_opt; - scanner_params.use_file_metacache = _use_file_metacache; - - HdfsScanner* scanner = nullptr; - auto format = scan_range.file_format; - - bool use_hudi_jni_reader = false; - if (scan_range.__isset.use_hudi_jni_reader) { - use_hudi_jni_reader = scan_range.use_hudi_jni_reader; - } - bool use_paimon_jni_reader = false; - if (scan_range.__isset.use_paimon_jni_reader) { - use_paimon_jni_reader = scan_range.use_paimon_jni_reader; - } - bool use_odps_jni_reader = false; - if (scan_range.__isset.use_odps_jni_reader) { - use_odps_jni_reader = scan_range.use_odps_jni_reader; + const auto& scan_range = _scan_range; + std::string native_file_path = scan_range.full_path; + if (_hive_table != nullptr && _hive_table->has_partition() && !_hive_table->has_base_path()) { + auto* partition_desc = _hive_table->get_partition(scan_range.partition_id); + if (partition_desc == nullptr) { + return Status::InternalError(fmt::format( + "Plan inconsistency. scan_range.partition_id = {} not found in partition description map", + scan_range.partition_id)); } + std::filesystem::path file_path(partition_desc->location()); + file_path /= scan_range.relative_path; + native_file_path = file_path.native(); + } + if (native_file_path.empty()) { + native_file_path = _hive_table->get_base_path() + scan_range.relative_path; + } - JniScanner::CreateOptions jni_scanner_create_options = { - .fs_options = &fsOptions, .hive_table = _hive_table, .scan_range = &scan_range}; - - if (_use_partition_column_value_only) { - DCHECK(_can_use_any_column); - scanner = new HdfsPartitionScanner(); - } else if (use_paimon_jni_reader) { - scanner = create_paimon_jni_scanner(jni_scanner_create_options).release(); - } else if (use_hudi_jni_reader) { - scanner = create_hudi_jni_scanner(jni_scanner_create_options).release(); - } else if (use_odps_jni_reader) { - scanner = create_odps_jni_scanner(jni_scanner_create_options).release(); - } else if (format == THdfsFileFormat::PARQUET) { - scanner = new HdfsParquetScanner(); - } else if (format == THdfsFileFormat::ORC) { - scanner_params.orc_use_column_names = state->query_options().orc_use_column_names; - scanner = new HdfsOrcScanner(); - } else if (format == THdfsFileFormat::TEXT) { - scanner = new HdfsTextScanner(); - } else if ((format == THdfsFileFormat::AVRO || format == THdfsFileFormat::RC_BINARY || - format == THdfsFileFormat::RC_TEXT || format == THdfsFileFormat::SEQUENCE_FILE) && - (dynamic_cast(_hive_table) != nullptr || - dynamic_cast(_hive_table) != nullptr)) { - scanner = create_hive_jni_scanner(jni_scanner_create_options).release(); - } else { - std::string msg = fmt::format("unsupported hdfs file format: {}", format); - LOG(WARNING) << msg; - return Status::NotSupported(msg); - } - if (scanner == nullptr) { - return Status::InternalError("create hdfs scanner failed"); - } - _pool.add(scanner); + const auto& hdfs_scan_node = _provider->_hdfs_scan_node; + auto fsOptions = + FSOptions(hdfs_scan_node.__isset.cloud_configuration ? &hdfs_scan_node.cloud_configuration : nullptr); + + ASSIGN_OR_RETURN(auto fs, FileSystem::CreateUniqueFromString(native_file_path, fsOptions)); + + HdfsScannerParams scanner_params; + scanner_params.runtime_filter_collector = _runtime_filters; + scanner_params.scan_range = &scan_range; + scanner_params.fs = _pool.add(fs.release()); + scanner_params.path = native_file_path; + scanner_params.file_size = _scan_range.file_length; + scanner_params.modification_time = _scan_range.modification_time; + scanner_params.tuple_desc = _tuple_desc; + scanner_params.materialize_slots = _materialize_slots; + scanner_params.materialize_index_in_chunk = _materialize_index_in_chunk; + scanner_params.partition_slots = _partition_slots; + scanner_params.partition_index_in_chunk = _partition_index_in_chunk; + scanner_params._partition_index_in_hdfs_partition_columns = _partition_index_in_hdfs_partition_columns; + scanner_params.partition_values = _partition_values; + scanner_params.conjunct_ctxs = _scanner_conjunct_ctxs; + scanner_params.conjunct_ctxs_by_slot = _conjunct_ctxs_by_slot; + scanner_params.slots_in_conjunct = _slots_in_conjunct; + scanner_params.slots_of_mutli_slot_conjunct = _slots_of_mutli_slot_conjunct; + scanner_params.min_max_conjunct_ctxs = _min_max_conjunct_ctxs; + scanner_params.min_max_tuple_desc = _min_max_tuple_desc; + scanner_params.hive_column_names = &_hive_column_names; + scanner_params.case_sensitive = _case_sensitive; + scanner_params.profile = &_profile; + scanner_params.lazy_column_coalesce_counter = get_lazy_column_coalesce_counter(); + scanner_params.split_context = down_cast(_split_context); + scanner_params.enable_split_tasks = _enable_split_tasks; + if (state->query_options().__isset.connector_max_split_size) { + scanner_params.connector_max_split_size = state->query_options().connector_max_split_size; + } + + if (!_equality_delete_slots.empty()) { + MORParams& mor_params = scanner_params.mor_params; + mor_params.tuple_desc = _tuple_desc; + mor_params.equality_slots = _equality_delete_slots; + mor_params.delete_column_tuple_desc = _delete_column_tuple_desc; + mor_params.mor_tuple_id = _provider->_hdfs_scan_node.mor_tuple_id; + mor_params.runtime_profile = _runtime_profile; + } + + for (const auto& delete_file : scan_range.delete_files) { + scanner_params.deletes.emplace_back(&delete_file); + } + + if (dynamic_cast(_hive_table)) { + auto tbl = dynamic_cast(_hive_table); + scanner_params.iceberg_schema = tbl->get_iceberg_schema(); + scanner_params.iceberg_equal_delete_schema = tbl->get_iceberg_equal_delete_schema(); + } + if (scan_range.__isset.paimon_deletion_file && !scan_range.paimon_deletion_file.path.empty()) { + scanner_params.paimon_deletion_file = std::make_shared(scan_range.paimon_deletion_file); + } + scanner_params.use_datacache = _use_datacache; + scanner_params.enable_populate_datacache = _enable_populate_datacache; + scanner_params.enable_datacache_async_populate_mode = _enable_datacache_aync_populate_mode; + scanner_params.enable_datacache_io_adaptor = _enable_datacache_io_adaptor; + scanner_params.datacache_evict_probability = _datacache_evict_probability; + scanner_params.can_use_any_column = _can_use_any_column; + scanner_params.can_use_min_max_count_opt = _can_use_min_max_count_opt; + scanner_params.use_file_metacache = _use_file_metacache; + + HdfsScanner* scanner = nullptr; + auto format = scan_range.file_format; + + bool use_hudi_jni_reader = false; + if (scan_range.__isset.use_hudi_jni_reader) { + use_hudi_jni_reader = scan_range.use_hudi_jni_reader; + } + bool use_paimon_jni_reader = false; + if (scan_range.__isset.use_paimon_jni_reader) { + use_paimon_jni_reader = scan_range.use_paimon_jni_reader; + } + bool use_odps_jni_reader = false; + if (scan_range.__isset.use_odps_jni_reader) { + use_odps_jni_reader = scan_range.use_odps_jni_reader; + } + + JniScanner::CreateOptions jni_scanner_create_options = { + .fs_options = &fsOptions, .hive_table = _hive_table, .scan_range = &scan_range}; + + if (_use_partition_column_value_only) { + DCHECK(_can_use_any_column); + scanner = new HdfsPartitionScanner(); + } else if (use_paimon_jni_reader) { + scanner = create_paimon_jni_scanner(jni_scanner_create_options).release(); + } else if (use_hudi_jni_reader) { + scanner = create_hudi_jni_scanner(jni_scanner_create_options).release(); + } else if (use_odps_jni_reader) { + scanner = create_odps_jni_scanner(jni_scanner_create_options).release(); + } else if (format == THdfsFileFormat::PARQUET) { + scanner = new HdfsParquetScanner(); + } else if (format == THdfsFileFormat::ORC) { + scanner_params.orc_use_column_names = state->query_options().orc_use_column_names; + scanner = new HdfsOrcScanner(); + } else if (format == THdfsFileFormat::TEXT) { + scanner = new HdfsTextScanner(); + } else if ((format == THdfsFileFormat::AVRO || format == THdfsFileFormat::RC_BINARY || + format == THdfsFileFormat::RC_TEXT || format == THdfsFileFormat::SEQUENCE_FILE) && + (dynamic_cast(_hive_table) != nullptr || + dynamic_cast(_hive_table) != nullptr)) { + scanner = create_hive_jni_scanner(jni_scanner_create_options).release(); + } else { + std::string msg = fmt::format("unsupported hdfs file format: {}", format); + LOG(WARNING) << msg; + return Status::NotSupported(msg); + } + if (scanner == nullptr) { + return Status::InternalError("create hdfs scanner failed"); + } + _pool.add(scanner); + + RETURN_IF_ERROR(scanner->init(state, scanner_params)); + Status st = scanner->open(state); + if (!st.ok()) { + return scanner->reinterpret_status(st); + } + _scanner = scanner; + return Status::OK(); +} - RETURN_IF_ERROR(scanner->init(state, scanner_params)); - Status st = scanner->open(state); - if (!st.ok()) { - return scanner->reinterpret_status(st); +void HiveDataSource::close(RuntimeState* state) { + if (_scanner != nullptr) { + if (!_scanner->has_split_tasks()) { + COUNTER_UPDATE(_profile.scan_ranges_counter, 1); } - _scanner = scanner; - return Status::OK(); + _scanner->close(); } + Expr::close(_min_max_conjunct_ctxs, state); + Expr::close(_partition_conjunct_ctxs, state); + Expr::close(_scanner_conjunct_ctxs, state); + for (auto& it : _conjunct_ctxs_by_slot) { + Expr::close(it.second, state); + } +} - void HiveDataSource::close(RuntimeState * state) { - if (_scanner != nullptr) { - if (!_scanner->has_split_tasks()) { - COUNTER_UPDATE(_profile.scan_ranges_counter, 1); - } - _scanner->close(); - } - Expr::close(_min_max_conjunct_ctxs, state); - Expr::close(_partition_conjunct_ctxs, state); - Expr::close(_scanner_conjunct_ctxs, state); - for (auto& it : _conjunct_ctxs_by_slot) { - Expr::close(it.second, state); - } +Status HiveDataSource::get_next(RuntimeState* state, ChunkPtr* chunk) { + if (_no_data) { + return Status::EndOfFile("no data"); } - Status HiveDataSource::get_next(RuntimeState * state, ChunkPtr * chunk) { - if (_no_data) { - return Status::EndOfFile("no data"); - } + do { + RETURN_IF_ERROR(_init_chunk_if_needed(chunk, _runtime_state->chunk_size())); + RETURN_IF_ERROR(_scanner->get_next(state, chunk)); + } while ((*chunk)->num_rows() == 0); - do { - RETURN_IF_ERROR(_init_chunk_if_needed(chunk, _runtime_state->chunk_size())); - RETURN_IF_ERROR(_scanner->get_next(state, chunk)); - } while ((*chunk)->num_rows() == 0); + // The column order of chunk is required to be invariable. In order to simplify the logic of each scanner, + // we force to reorder the columns of chunk, so scanner doesn't have to care about the column order anymore. + // The overhead of reorder is negligible because we only swap columns. + ChunkHelper::reorder_chunk(*_tuple_desc, chunk->get()); - // The column order of chunk is required to be invariable. In order to simplify the logic of each scanner, - // we force to reorder the columns of chunk, so scanner doesn't have to care about the column order anymore. - // The overhead of reorder is negligible because we only swap columns. - ChunkHelper::reorder_chunk(*_tuple_desc, chunk->get()); + return Status::OK(); +} +Status HiveDataSource::_init_chunk_if_needed(ChunkPtr* chunk, size_t n) { + if ((*chunk) != nullptr && (*chunk)->num_columns() != 0) { return Status::OK(); } - Status HiveDataSource::_init_chunk_if_needed(ChunkPtr * chunk, size_t n) { - if ((*chunk) != nullptr && (*chunk)->num_columns() != 0) { - return Status::OK(); - } + *chunk = ChunkHelper::new_chunk(*_tuple_desc, n); - *chunk = ChunkHelper::new_chunk(*_tuple_desc, n); - - if (!_equality_delete_slots.empty()) { - std::map id_to_slots; - for (const auto& slot : _tuple_desc->slots()) { - id_to_slots.emplace(slot->id(), slot); - } + if (!_equality_delete_slots.empty()) { + std::map id_to_slots; + for (const auto& slot : _tuple_desc->slots()) { + id_to_slots.emplace(slot->id(), slot); + } - for (const auto& slot : _equality_delete_slots) { - if (!id_to_slots.contains(slot->id())) { - const auto column = ColumnHelper::create_column(slot->type(), slot->is_nullable()); - column->reserve(n); - (*chunk)->append_column(column, slot->id()); - } + for (const auto& slot : _equality_delete_slots) { + if (!id_to_slots.contains(slot->id())) { + const auto column = ColumnHelper::create_column(slot->type(), slot->is_nullable()); + column->reserve(n); + (*chunk)->append_column(column, slot->id()); } } - return Status::OK(); } + return Status::OK(); +} - const std::string HiveDataSource::get_custom_coredump_msg() const { - const std::string path = !_scan_range.relative_path.empty() ? _scan_range.relative_path : _scan_range.full_path; - return strings::Substitute("Hive file path: $0, partition id: $1, length: $2, offset: $3", path, - _scan_range.partition_id, _scan_range.length, _scan_range.offset); - } +const std::string HiveDataSource::get_custom_coredump_msg() const { + const std::string path = !_scan_range.relative_path.empty() ? _scan_range.relative_path : _scan_range.full_path; + return strings::Substitute("Hive file path: $0, partition id: $1, length: $2, offset: $3", path, + _scan_range.partition_id, _scan_range.length, _scan_range.offset); +} - int64_t HiveDataSource::raw_rows_read() const { - if (_scanner == nullptr) return 0; - return _scanner->raw_rows_read(); - } - int64_t HiveDataSource::num_rows_read() const { - if (_scanner == nullptr) return 0; - return _scanner->num_rows_read(); - } - int64_t HiveDataSource::num_bytes_read() const { - if (_scanner == nullptr) return 0; - return _scanner->num_bytes_read(); - } - int64_t HiveDataSource::cpu_time_spent() const { - if (_scanner == nullptr) return 0; - return _scanner->cpu_time_spent(); - } +int64_t HiveDataSource::raw_rows_read() const { + if (_scanner == nullptr) return 0; + return _scanner->raw_rows_read(); +} +int64_t HiveDataSource::num_rows_read() const { + if (_scanner == nullptr) return 0; + return _scanner->num_rows_read(); +} +int64_t HiveDataSource::num_bytes_read() const { + if (_scanner == nullptr) return 0; + return _scanner->num_bytes_read(); +} +int64_t HiveDataSource::cpu_time_spent() const { + if (_scanner == nullptr) return 0; + return _scanner->cpu_time_spent(); +} - int64_t HiveDataSource::io_time_spent() const { - if (_scanner == nullptr) return 0; - return _scanner->io_time_spent(); - } +int64_t HiveDataSource::io_time_spent() const { + if (_scanner == nullptr) return 0; + return _scanner->io_time_spent(); +} - int64_t HiveDataSource::estimated_mem_usage() const { - if (_scanner == nullptr) return 0; - return _scanner->estimated_mem_usage(); - } +int64_t HiveDataSource::estimated_mem_usage() const { + if (_scanner == nullptr) return 0; + return _scanner->estimated_mem_usage(); +} - void HiveDataSourceProvider::peek_scan_ranges(const std::vector& scan_ranges) { - for (const auto& sc : scan_ranges) { - const TScanRange& x = sc.scan_range; - if (!x.__isset.hdfs_scan_range) continue; - const THdfsScanRange& y = x.hdfs_scan_range; - _max_file_length = std::max(_max_file_length, y.file_length); - } +void HiveDataSourceProvider::peek_scan_ranges(const std::vector& scan_ranges) { + for (const auto& sc : scan_ranges) { + const TScanRange& x = sc.scan_range; + if (!x.__isset.hdfs_scan_range) continue; + const THdfsScanRange& y = x.hdfs_scan_range; + _max_file_length = std::max(_max_file_length, y.file_length); } +} - void HiveDataSourceProvider::default_data_source_mem_bytes(int64_t * min_value, int64_t * max_value) { - DataSourceProvider::default_data_source_mem_bytes(min_value, max_value); - // here we compute as default mem bytes = max(MIN_SIZE, min(max_file_length, MAX_SIZE)) - int64_t size = std::max(*min_value, std::min(_max_file_length * 3 / 2, *max_value)); - *min_value = *max_value = size; - } +void HiveDataSourceProvider::default_data_source_mem_bytes(int64_t* min_value, int64_t* max_value) { + DataSourceProvider::default_data_source_mem_bytes(min_value, max_value); + // here we compute as default mem bytes = max(MIN_SIZE, min(max_file_length, MAX_SIZE)) + int64_t size = std::max(*min_value, std::min(_max_file_length * 3 / 2, *max_value)); + *min_value = *max_value = size; +} - void HiveDataSource::get_split_tasks(std::vector * split_tasks) { - if (_scanner == nullptr) return; - _scanner->move_split_tasks(split_tasks); - } +void HiveDataSource::get_split_tasks(std::vector* split_tasks) { + if (_scanner == nullptr) return; + _scanner->move_split_tasks(split_tasks); +} } // namespace starrocks::connector diff --git a/be/src/exec/hdfs_scanner.cpp b/be/src/exec/hdfs_scanner.cpp index fd0cbfaa0bb56..d924abb50bcc9 100644 --- a/be/src/exec/hdfs_scanner.cpp +++ b/be/src/exec/hdfs_scanner.cpp @@ -203,7 +203,6 @@ void HdfsScanner::close() noexcept { return; } - DCHECK(!has_pending_token()); bool expect = false; if (!_closed.compare_exchange_strong(expect, true)) return; update_counter(); @@ -212,14 +211,6 @@ void HdfsScanner::close() noexcept { _mor_processor->close(_runtime_state); } -void HdfsScanner::enter_pending_queue() { - _pending_queue_sw.start(); -} - -uint64_t HdfsScanner::exit_pending_queue() { - return _pending_queue_sw.reset(); -} - Status HdfsScanner::open_random_access_file() { CHECK(_file == nullptr) << "File has already been opened"; ASSIGN_OR_RETURN(std::unique_ptr raw_file, diff --git a/be/test/exec/hdfs_scanner_test.cpp b/be/test/exec/hdfs_scanner_test.cpp index 2a19bfa21a48c..97f685d0f3b01 100644 --- a/be/test/exec/hdfs_scanner_test.cpp +++ b/be/test/exec/hdfs_scanner_test.cpp @@ -463,7 +463,7 @@ TEST_F(HdfsScannerTest, TestOrcSkipFile) { READ_SCANNER_ROWS(scanner, 0); EXPECT_EQ(scanner->raw_rows_read(), 0); - scanner->close(_runtime_state); + scanner->close(); } class BadOrcFileStream : public ORCHdfsFileStream { @@ -516,7 +516,7 @@ TEST_F(HdfsScannerTest, TestOrcReaderException) { status = scanner->open(_runtime_state); EXPECT_FALSE(status.ok()) << status.message(); EXPECT_EQ(status.code(), ec.ret_code); - scanner->close(_runtime_state); + scanner->close(); } } @@ -1212,7 +1212,7 @@ TEST_F(HdfsScannerTest, TestOrcDecodeMinMaxWithTypeMismatch) { status = scanner->open(_runtime_state); EXPECT_TRUE(status.ok()) << status.to_string(); READ_SCANNER_ROWS(scanner, 4); - scanner->close(_runtime_state); + scanner->close(); } // ==================================================================================================== @@ -1442,7 +1442,7 @@ TEST_F(HdfsScannerTest, TestOrcMapLazyLoadWithSubfieldSeleted) { ASSERT_EQ(0, chunk->num_rows()); ASSERT_TRUE(status.is_end_of_file()); - scanner->close(_runtime_state); + scanner->close(); } TEST_F(HdfsScannerTest, TestOrcBooleanConjunct) { @@ -1750,7 +1750,7 @@ TEST_F(HdfsScannerTest, TestCSVCompressed) { status = scanner->open(_runtime_state); ASSERT_TRUE(status.ok()) << status.message(); ASSERT_EQ(0, scanner->estimated_mem_usage()); - scanner->close(_runtime_state); + scanner->close(); } { // compressed file with skip_header_line_count @@ -1767,7 +1767,7 @@ TEST_F(HdfsScannerTest, TestCSVCompressed) { status = scanner->open(_runtime_state); ASSERT_TRUE(status.ok()) << status.message(); READ_SCANNER_ROWS(scanner, 50); - scanner->close(_runtime_state); + scanner->close(); } { // compressed file with skip_header_line_count > total line count @@ -1784,7 +1784,7 @@ TEST_F(HdfsScannerTest, TestCSVCompressed) { status = scanner->open(_runtime_state); ASSERT_TRUE(status.ok()) << status.message(); READ_SCANNER_ROWS(scanner, 0); - scanner->close(_runtime_state); + scanner->close(); } { auto* range = _create_scan_range(compressed_file, 0, 0); @@ -2061,7 +2061,7 @@ TEST_F(HdfsScannerTest, TestCSVWithUTFBOM) { EXPECT_EQ(1, chunk->num_rows()); EXPECT_EQ("['6358db89e73b']", chunk->debug_row(0)); - scanner->close(_runtime_state); + scanner->close(); } { @@ -2086,7 +2086,7 @@ TEST_F(HdfsScannerTest, TestCSVWithUTFBOM) { EXPECT_EQ(2, chunk->num_rows()); EXPECT_EQ("['62ef51eae5d8']", chunk->debug_row(0)); EXPECT_EQ("['6358db89e73b']", chunk->debug_row(1)); - scanner->close(_runtime_state); + scanner->close(); } }