diff --git a/be/src/io/fs/file_system.cpp b/be/src/io/fs/file_system.cpp index 3579a5323d92172..e6b5ef7df1a8f5d 100644 --- a/be/src/io/fs/file_system.cpp +++ b/be/src/io/fs/file_system.cpp @@ -25,58 +25,70 @@ namespace io { Status FileSystem::create_file(const Path& file, FileWriterPtr* writer, const FileWriterOptions* opts) { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(create_file_impl(path, writer, opts)); } Status FileSystem::open_file(const Path& file, FileReaderSPtr* reader, const FileReaderOptions* opts) { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(open_file_impl(path, reader, opts)); } Status FileSystem::create_directory(const Path& dir, bool failed_if_exists) { - auto path = absolute_path(dir); + Path path; + RETURN_IF_ERROR(absolute_path(dir, path)); FILESYSTEM_M(create_directory_impl(path, failed_if_exists)); } Status FileSystem::delete_file(const Path& file) { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(delete_file_impl(path)); } Status FileSystem::delete_directory(const Path& dir) { - auto path = absolute_path(dir); + Path path; + RETURN_IF_ERROR(absolute_path(dir, path)); FILESYSTEM_M(delete_directory_impl(path)); } Status FileSystem::batch_delete(const std::vector& files) { std::vector abs_files; for (auto& file : files) { - abs_files.push_back(absolute_path(file)); + Path abs_file; + RETURN_IF_ERROR(absolute_path(file, abs_file)); + abs_files.push_back(abs_file); } FILESYSTEM_M(batch_delete_impl(abs_files)); } Status FileSystem::exists(const Path& path, bool* res) const { - auto fs_path = absolute_path(path); + Path fs_path; + RETURN_IF_ERROR(absolute_path(path, fs_path)); FILESYSTEM_M(exists_impl(fs_path, res)); } Status FileSystem::file_size(const Path& file, int64_t* file_size) const { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(file_size_impl(path, file_size)); } Status FileSystem::list(const Path& dir, bool only_file, std::vector* files, bool* exists) { - auto path = absolute_path(dir); + Path path; + RETURN_IF_ERROR(absolute_path(dir, path)); FILESYSTEM_M(list_impl(path, only_file, files, exists)); } Status FileSystem::rename(const Path& orig_name, const Path& new_name) { - auto orig_path = absolute_path(orig_name); - auto new_path = absolute_path(new_name); + Path orig_path; + RETURN_IF_ERROR(absolute_path(orig_name, orig_path)); + Path new_path; + RETURN_IF_ERROR(absolute_path(new_name, new_path)); FILESYSTEM_M(rename_impl(orig_path, new_path)); } diff --git a/be/src/io/fs/file_system.h b/be/src/io/fs/file_system.h index a8ccc8756bb60a7..6baf07917d35cb4 100644 --- a/be/src/io/fs/file_system.h +++ b/be/src/io/fs/file_system.h @@ -163,7 +163,7 @@ class FileSystem { // FIMXE(plat1ko): The implementation and semantics of this function are not completely // consistent, which is confused. - virtual Path absolute_path(const Path& path) const = 0; + virtual Status absolute_path(const Path& path, Path& abs_path) const = 0; FileSystem(std::string id, FileSystemType type) : _id(std::move(id)), _type(type) {} diff --git a/be/src/io/fs/local_file_system.cpp b/be/src/io/fs/local_file_system.cpp index 0107ed57dc8fb16..9270d919a37519d 100644 --- a/be/src/io/fs/local_file_system.cpp +++ b/be/src/io/fs/local_file_system.cpp @@ -471,4 +471,54 @@ Status LocalFileSystem::permission_impl(const Path& file, std::filesystem::perms return Status::OK(); } +Status LocalFileSystem::convert_to_abs_path(const Path& input_path_str, Path& abs_path) { + // valid path include: + // 1. abc/def will return abc/def + // 2. /abc/def will return /abc/def + // 3. file:/abc/def will return /abc/def + // 4. file:///abc/def will return /abc/def + std::string path_str = input_path_str; + size_t slash = path_str.find('/'); + if (slash == 0) { + abs_path = input_path_str; + return Status::OK(); + } + + // Initialize scheme and authority + std::string scheme; + size_t start = 0; + + // Parse URI scheme + size_t colon = path_str.find(':'); + if (colon != std::string::npos && (slash == std::string::npos || colon < slash)) { + // Has a scheme + scheme = path_str.substr(0, colon); + if (scheme != "file") { + return Status::InternalError( + "Only supports `file` type scheme, like 'file:///path', 'file:/path'."); + } + start = colon + 1; + } + + // Parse URI authority, if any + if (path_str.compare(start, 2, "//") == 0 && path_str.length() - start > 2) { + // Has authority + // such as : path_str = "file://authority/abc/def" + // and now : start = 5 + size_t next_slash = path_str.find('/', start + 2); + // now : next_slash = 16 + if (next_slash == std::string::npos) { + return Status::InternalError( + "This input string only has authority, but has no path information"); + } + // We will skit authority + // now : start = 16 + start = next_slash; + } + + // URI path is the rest of the string + abs_path = path_str.substr(start); + return Status::OK(); +} + } // namespace doris::io diff --git a/be/src/io/fs/local_file_system.h b/be/src/io/fs/local_file_system.h index c6295b0bae1d6c5..4540df47c16d810 100644 --- a/be/src/io/fs/local_file_system.h +++ b/be/src/io/fs/local_file_system.h @@ -34,6 +34,8 @@ class LocalFileSystem final : public FileSystem { public: ~LocalFileSystem() override; + static Status convert_to_abs_path(const Path& path, Path& abs_path); + /// hard link dest file to src file Status link_file(const Path& src, const Path& dest); @@ -104,7 +106,9 @@ class LocalFileSystem final : public FileSystem { // `LocalFileSystem` always use absolute path as arguments // FIXME(plat1ko): Eliminate this method - Path absolute_path(const Path& path) const override { return path; } + Status absolute_path(const Path& path, Path& abs_path) const override { + return convert_to_abs_path(path, abs_path); + } friend const std::shared_ptr& global_local_filesystem(); }; diff --git a/be/src/io/fs/remote_file_system.cpp b/be/src/io/fs/remote_file_system.cpp index 2b6af2af046afca..fd793f60cdc5d8d 100644 --- a/be/src/io/fs/remote_file_system.cpp +++ b/be/src/io/fs/remote_file_system.cpp @@ -29,7 +29,8 @@ namespace doris::io { Status RemoteFileSystem::upload(const Path& local_file, const Path& dest_file) { - auto dest_path = absolute_path(dest_file); + Path dest_path; + RETURN_IF_ERROR(absolute_path(dest_file, dest_path)); FILESYSTEM_M(upload_impl(local_file, dest_path)); } @@ -37,13 +38,16 @@ Status RemoteFileSystem::batch_upload(const std::vector& local_files, const std::vector& remote_files) { std::vector remote_paths; for (auto& path : remote_files) { - remote_paths.push_back(absolute_path(path)); + Path abs_path; + RETURN_IF_ERROR(absolute_path(path, abs_path)); + remote_paths.push_back(abs_path); } FILESYSTEM_M(batch_upload_impl(local_files, remote_paths)); } Status RemoteFileSystem::download(const Path& remote_file, const Path& local) { - auto remote_path = absolute_path(remote_file); + Path remote_path; + RETURN_IF_ERROR(absolute_path(remote_file, remote_path)); FILESYSTEM_M(download_impl(remote_path, local)); } diff --git a/be/src/io/fs/remote_file_system.h b/be/src/io/fs/remote_file_system.h index e9472140ab7b08e..de0a1b71519a922 100644 --- a/be/src/io/fs/remote_file_system.h +++ b/be/src/io/fs/remote_file_system.h @@ -64,11 +64,13 @@ class RemoteFileSystem : public FileSystem { virtual Status open_file_internal(const Path& file, FileReaderSPtr* reader, const FileReaderOptions& opts) = 0; - Path absolute_path(const Path& path) const override { + Status absolute_path(const Path& path, Path& abs_path) const override { if (path.is_absolute()) { - return path; + abs_path = path; + } else { + abs_path = _root_path / path; } - return _root_path / path; + return Status::OK(); } Path _root_path; diff --git a/be/src/io/fs/s3_file_system.h b/be/src/io/fs/s3_file_system.h index d1e8b5b6e31a619..61967a63e443796 100644 --- a/be/src/io/fs/s3_file_system.h +++ b/be/src/io/fs/s3_file_system.h @@ -113,16 +113,17 @@ class S3FileSystem final : public RemoteFileSystem { const std::vector& remote_files) override; Status download_impl(const Path& remote_file, const Path& local_file) override; - Path absolute_path(const Path& path) const override { + Status absolute_path(const Path& path, Path& abs_path) const override { if (path.string().find("://") != std::string::npos) { // the path is with schema, which means this is a full path like: // s3://bucket/path/to/file.txt // so no need to concat with prefix - return path; + abs_path = path; } else { // path with no schema - return _root_path / path; + abs_path = _root_path / path; } + return Status::OK(); } private: diff --git a/be/src/olap/parallel_scanner_builder.cpp b/be/src/olap/parallel_scanner_builder.cpp index 33e2762aa446219..88c69ab5c9a584a 100644 --- a/be/src/olap/parallel_scanner_builder.cpp +++ b/be/src/olap/parallel_scanner_builder.cpp @@ -24,6 +24,7 @@ #include "cloud/config.h" #include "common/status.h" #include "olap/rowset/beta_rowset.h" +#include "olap/segment_loader.h" #include "pipeline/exec/olap_scan_operator.h" #include "vec/exec/scan/new_olap_scanner.h" @@ -63,21 +64,18 @@ Status ParallelScannerBuilder::_build_scanners_by_rowid(std::list& auto rowset = reader->rowset(); const auto rowset_id = rowset->rowset_id(); - DCHECK(_segment_cache_handles.contains(rowset_id)); - auto& segment_cache_handle = _segment_cache_handles[rowset_id]; + const auto& segments_rows = _all_segments_rows[rowset_id]; if (rowset->num_rows() == 0) { continue; } - const auto& segments = segment_cache_handle.get_segments(); int segment_start = 0; auto split = RowSetSplits(reader->clone()); - for (size_t i = 0; i != segments.size(); ++i) { - const auto& segment = segments[i]; + for (size_t i = 0; i != segments_rows.size(); ++i) { + const size_t rows_of_segment = segments_rows[i]; RowRanges row_ranges; - const size_t rows_of_segment = segment->num_rows(); int64_t offset_in_segment = 0; // try to split large segments into RowRanges @@ -125,7 +123,7 @@ Status ParallelScannerBuilder::_build_scanners_by_rowid(std::list& // The non-empty `row_ranges` means there are some rows left in this segment not added into `split`. if (!row_ranges.is_empty()) { DCHECK_GT(rows_collected, 0); - DCHECK_EQ(row_ranges.to(), segment->num_rows()); + DCHECK_EQ(row_ranges.to(), rows_of_segment); split.segment_row_ranges.emplace_back(std::move(row_ranges)); } } @@ -133,7 +131,7 @@ Status ParallelScannerBuilder::_build_scanners_by_rowid(std::list& DCHECK_LE(rows_collected, _rows_per_scanner); if (rows_collected > 0) { split.segment_offsets.first = segment_start; - split.segment_offsets.second = segments.size(); + split.segment_offsets.second = segments_rows.size(); DCHECK_GT(split.segment_offsets.second, split.segment_offsets.first); DCHECK_EQ(split.segment_row_ranges.size(), split.segment_offsets.second - split.segment_offsets.first); @@ -181,11 +179,15 @@ Status ParallelScannerBuilder::_load() { auto rowset = rs_split.rs_reader->rowset(); RETURN_IF_ERROR(rowset->load()); const auto rowset_id = rowset->rowset_id(); - auto& segment_cache_handle = _segment_cache_handles[rowset_id]; + SegmentCacheHandle segment_cache_handle; RETURN_IF_ERROR(SegmentLoader::instance()->load_segments( std::dynamic_pointer_cast(rowset), &segment_cache_handle, enable_segment_cache, false)); + + for (const auto& segment : segment_cache_handle.get_segments()) { + _all_segments_rows[rowset_id].emplace_back(segment->num_rows()); + } _total_rows += rowset->num_rows(); } } diff --git a/be/src/olap/parallel_scanner_builder.h b/be/src/olap/parallel_scanner_builder.h index 934d769ed59aa09..7c6b5648e89e043 100644 --- a/be/src/olap/parallel_scanner_builder.h +++ b/be/src/olap/parallel_scanner_builder.h @@ -83,7 +83,7 @@ class ParallelScannerBuilder { size_t _rows_per_scanner {_min_rows_per_scanner}; - std::map _segment_cache_handles; + std::map> _all_segments_rows; std::shared_ptr _scanner_profile; RuntimeState* _state; diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index bf177742ab35638..7d3f4da935099ee 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -112,23 +112,24 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu if (_closed) { return Status::OK(); } - auto p = _parent->cast(); Defer defer {[&]() { - if (_should_build_hash_table) { - // The build side hash key column maybe no need output, but we need to keep the column in block - // because it is used to compare with probe side hash key column - if (p._should_keep_hash_key_column && _build_col_ids.size() == 1) { - p._should_keep_column_flags[_build_col_ids[0]] = true; - } + if (!_should_build_hash_table) { + return; + } + // The build side hash key column maybe no need output, but we need to keep the column in block + // because it is used to compare with probe side hash key column + auto p = _parent->cast(); + if (p._should_keep_hash_key_column && _build_col_ids.size() == 1) { + p._should_keep_column_flags[_build_col_ids[0]] = true; + } - if (_shared_state->build_block) { - // release the memory of unused column in probe stage - _shared_state->build_block->clear_column_mem_not_keep( - p._should_keep_column_flags, bool(p._shared_hashtable_controller)); - } + if (_shared_state->build_block) { + // release the memory of unused column in probe stage + _shared_state->build_block->clear_column_mem_not_keep( + p._should_keep_column_flags, bool(p._shared_hashtable_controller)); } - if (_should_build_hash_table && p._shared_hashtable_controller) { + if (p._shared_hashtable_controller) { p._shared_hashtable_controller->signal_finish(p.node_id()); } }}; @@ -137,22 +138,22 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu return Base::close(state, exec_status); } - if (state->get_task()->wake_up_by_downstream()) { - RETURN_IF_ERROR(_runtime_filter_slots->send_filter_size(state, 0, _finish_dependency)); - RETURN_IF_ERROR(_runtime_filter_slots->ignore_all_filters()); - } else { - auto* block = _shared_state->build_block.get(); - uint64_t hash_table_size = block ? block->rows() : 0; - { - SCOPED_TIMER(_runtime_filter_init_timer); - if (_should_build_hash_table) { + if (_should_build_hash_table) { + if (state->get_task()->wake_up_by_downstream()) { + RETURN_IF_ERROR(_runtime_filter_slots->send_filter_size(state, 0, _finish_dependency)); + RETURN_IF_ERROR(_runtime_filter_slots->ignore_all_filters()); + } else { + auto* block = _shared_state->build_block.get(); + uint64_t hash_table_size = block ? block->rows() : 0; + { + SCOPED_TIMER(_runtime_filter_init_timer); RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size)); + RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); + } + if (hash_table_size > 1) { + SCOPED_TIMER(_runtime_filter_compute_timer); + _runtime_filter_slots->insert(block); } - RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); - } - if (_should_build_hash_table && hash_table_size > 1) { - SCOPED_TIMER(_runtime_filter_compute_timer); - _runtime_filter_slots->insert(block); } } SCOPED_TIMER(_publish_runtime_filter_timer); diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 813f0833ad40615..a27936f5f0d88b9 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -294,6 +294,7 @@ class ExecEnv { static void set_tracking_memory(bool tracking_memory) { _s_tracking_memory.store(tracking_memory, std::memory_order_release); } + void set_orc_memory_pool(orc::MemoryPool* pool) { _orc_memory_pool = pool; } #endif LoadStreamMapPool* load_stream_map_pool() { return _load_stream_map_pool.get(); } diff --git a/be/src/vec/exec/format/orc/vorc_reader.cpp b/be/src/vec/exec/format/orc/vorc_reader.cpp index 5d6ae4024c75610..70f3f6f003f6117 100644 --- a/be/src/vec/exec/format/orc/vorc_reader.cpp +++ b/be/src/vec/exec/format/orc/vorc_reader.cpp @@ -18,13 +18,14 @@ #include "vorc_reader.h" #include -#include #include +#include #include #include #include #include +#include // IWYU pragma: no_include #include // IWYU pragma: keep #include @@ -33,12 +34,10 @@ #include #include #include -#include #include "cctz/civil_time.h" #include "cctz/time_zone.h" #include "common/exception.h" -#include "exec/olap_utils.h" #include "exprs/create_predicate_function.h" #include "exprs/hybrid_set.h" #include "gutil/strings/substitute.h" @@ -55,6 +54,7 @@ #include "runtime/descriptors.h" #include "runtime/primitive_type.h" #include "runtime/thread_context.h" +#include "util/runtime_profile.h" #include "util/slice.h" #include "util/timezone_utils.h" #include "vec/columns/column.h" @@ -71,15 +71,13 @@ #include "vec/data_types/data_type_map.h" #include "vec/data_types/data_type_nullable.h" #include "vec/data_types/data_type_struct.h" -#include "vec/exec/format/orc/orc_memory_pool.h" #include "vec/exec/format/table/transactional_hive_common.h" #include "vec/exprs/vbloom_predicate.h" #include "vec/exprs/vdirect_in_predicate.h" #include "vec/exprs/vectorized_fn_call.h" +#include "vec/exprs/vexpr_fwd.h" #include "vec/exprs/vin_predicate.h" -#include "vec/exprs/vliteral.h" #include "vec/exprs/vruntimefilter_wrapper.h" -#include "vec/exprs/vslot_ref.h" #include "vec/runtime/vdatetime_value.h" namespace doris { @@ -237,6 +235,10 @@ void OrcReader::_init_profile() { ADD_CHILD_TIMER_WITH_LEVEL(_profile, "DecodeNullMapTime", orc_profile, 1); _orc_profile.filter_block_time = ADD_CHILD_TIMER_WITH_LEVEL(_profile, "FilterBlockTime", orc_profile, 1); + _orc_profile.selected_row_group_count = + ADD_COUNTER_WITH_LEVEL(_profile, "SelectedRowGroupCount", TUnit::UNIT, 1); + _orc_profile.evaluated_row_group_count = + ADD_COUNTER_WITH_LEVEL(_profile, "EvaluatedRowGroupCount", TUnit::UNIT, 1); } } @@ -259,6 +261,7 @@ Status OrcReader::_create_file_reader() { try { orc::ReaderOptions options; options.setMemoryPool(*ExecEnv::GetInstance()->orc_memory_pool()); + options.setReaderMetrics(&_reader_metrics); _reader = orc::createReader( std::unique_ptr(_file_input_stream.release()), options); } catch (std::exception& e) { @@ -388,6 +391,9 @@ Status OrcReader::_init_read_columns() { } _col_name_to_file_col_name[col_name] = read_col; + // TODO: refactor this + std::transform(read_col.begin(), read_col.end(), read_col.begin(), ::tolower); + _col_name_to_file_col_name_low_case[col_name] = read_col; } } return Status::OK(); @@ -461,8 +467,10 @@ static std::unordered_map TYPEKIND_TO_PRE {orc::TypeKind::BOOLEAN, orc::PredicateDataType::BOOLEAN}}; template -std::tuple convert_to_orc_literal(const orc::Type* type, const void* value, - int precision, int scale) { +std::tuple convert_to_orc_literal(const orc::Type* type, + StringRef& literal_data, int precision, + int scale) { + const auto* value = literal_data.data; try { switch (type->getKind()) { case orc::TypeKind::BOOLEAN: @@ -486,8 +494,7 @@ std::tuple convert_to_orc_literal(const orc::Type* type, con case orc::TypeKind::CHAR: [[fallthrough]]; case orc::TypeKind::VARCHAR: { - StringRef* string_value = (StringRef*)value; - return std::make_tuple(true, orc::Literal(string_value->data, string_value->size)); + return std::make_tuple(true, orc::Literal(literal_data.data, literal_data.size)); } case orc::TypeKind::DECIMAL: { int128_t decimal_value; @@ -558,190 +565,353 @@ std::tuple convert_to_orc_literal(const orc::Type* type, con } } -template -std::vector value_range_to_predicate( - const ColumnValueRange& col_val_range, const orc::Type* type, - std::vector* unsupported_pushdown_types) { - std::vector predicates; - - PrimitiveType src_type = OrcReader::convert_to_doris_type(type).type; - if (src_type != primitive_type) { - if (!(is_string_type(src_type) && is_string_type(primitive_type))) { - // not support schema change - return predicates; - } +std::tuple OrcReader::_make_orc_literal( + const VSlotRef* slot_ref, const VLiteral* literal) { + auto file_col_name_low_case = _col_name_to_file_col_name_low_case[slot_ref->expr_name()]; + if (!_type_map.contains(file_col_name_low_case)) { + // TODO: this is for acid table + LOG(WARNING) << "Column " << slot_ref->expr_name() << " not found in _type_map"; + return std::make_tuple(false, orc::Literal(false), orc::PredicateDataType::LONG); + } + const auto* orc_type = _type_map[file_col_name_low_case]; + if (!TYPEKIND_TO_PREDICATE_TYPE.contains(orc_type->getKind())) { + LOG(WARNING) << "Unsupported Push Down Orc Type [TypeKind=" << orc_type->getKind() << "]"; + return std::make_tuple(false, orc::Literal(false), orc::PredicateDataType::LONG); + } + const auto predicate_type = TYPEKIND_TO_PREDICATE_TYPE[orc_type->getKind()]; + if (literal == nullptr) { + // only get the predicate_type + return std::make_tuple(true, orc::Literal(true), predicate_type); + } + auto literal_data = literal->get_column_ptr()->get_data_at(0); + auto* slot = _tuple_descriptor->slots()[slot_ref->column_id()]; + auto slot_type = slot->type(); + switch (slot_type.type) { +#define M(NAME) \ + case TYPE_##NAME: { \ + auto [valid, orc_literal] = convert_to_orc_literal( \ + orc_type, literal_data, slot_type.precision, slot_type.scale); \ + return std::make_tuple(valid, orc_literal, predicate_type); \ + } +#define APPLY_FOR_PRIMITIVE_TYPE(M) \ + M(TINYINT) \ + M(SMALLINT) \ + M(INT) \ + M(BIGINT) \ + M(LARGEINT) \ + M(CHAR) \ + M(DATE) \ + M(DATETIME) \ + M(DATEV2) \ + M(DATETIMEV2) \ + M(VARCHAR) \ + M(STRING) \ + M(HLL) \ + M(DECIMAL32) \ + M(DECIMAL64) \ + M(DECIMAL128I) \ + M(DECIMAL256) \ + M(DECIMALV2) \ + M(BOOLEAN) \ + M(IPV4) \ + M(IPV6) + APPLY_FOR_PRIMITIVE_TYPE(M) +#undef M + default: { + VLOG_CRITICAL << "Unsupported Convert Orc Literal [ColName=" << slot->col_name() << "]"; + return std::make_tuple(false, orc::Literal(false), predicate_type); + } } +} - if (unsupported_pushdown_types != nullptr) { - for (vector::iterator it = unsupported_pushdown_types->begin(); - it != unsupported_pushdown_types->end(); ++it) { - if (*it == type->getKind()) { - // Unsupported type - return predicates; - } - } +// check if the slot of expr can be pushed down to orc reader +bool OrcReader::_check_slot_can_push_down(const VExprSPtr& expr) { + if (!expr->children()[0]->is_slot_ref()) { + return false; } + const auto* slot_ref = static_cast(expr->children()[0].get()); + // check if the slot exists in orc file and not partition column + return _col_name_to_file_col_name.contains(slot_ref->expr_name()) && + !_lazy_read_ctx.predicate_partition_columns.contains(slot_ref->expr_name()); +} - orc::PredicateDataType predicate_data_type; - auto type_it = TYPEKIND_TO_PREDICATE_TYPE.find(type->getKind()); - if (type_it == TYPEKIND_TO_PREDICATE_TYPE.end()) { - // Unsupported type - return predicates; - } else { - predicate_data_type = type_it->second; +// check if there are rest children of expr can be pushed down to orc reader +bool OrcReader::_check_rest_children_can_push_down(const VExprSPtr& expr) { + if (expr->children().size() < 2) { + return false; } - if (col_val_range.is_fixed_value_range()) { - OrcPredicate in_predicate; - in_predicate.col_name = col_val_range.column_name(); - in_predicate.data_type = predicate_data_type; - in_predicate.op = SQLFilterOp::FILTER_IN; - for (const auto& value : col_val_range.get_fixed_value_set()) { - auto [valid, literal] = convert_to_orc_literal( - type, &value, col_val_range.precision(), col_val_range.scale()); - if (valid) { - in_predicate.literals.push_back(literal); - } + for (size_t i = 1; i < expr->children().size(); ++i) { + if (!expr->children()[i]->is_literal()) { + return false; } - if (!in_predicate.literals.empty()) { - predicates.emplace_back(in_predicate); + } + return true; +} + +// check if the expr can be pushed down to orc reader +bool OrcReader::_check_expr_can_push_down(const VExprSPtr& expr) { + DCHECK(expr != nullptr); + switch (expr->op()) { + case TExprOpcode::COMPOUND_AND: + // at least one child can be pushed down + return std::ranges::any_of(expr->children(), [this](const auto& child) { + return _check_expr_can_push_down(child); + }); + case TExprOpcode::COMPOUND_OR: + // all children must be pushed down + return std::ranges::all_of(expr->children(), [this](const auto& child) { + return _check_expr_can_push_down(child); + }); + case TExprOpcode::COMPOUND_NOT: + DCHECK_EQ(expr->children().size(), 1); + return _check_expr_can_push_down(expr->children()[0]); + + case TExprOpcode::GE: + case TExprOpcode::GT: + case TExprOpcode::LE: + case TExprOpcode::LT: + case TExprOpcode::EQ: + case TExprOpcode::NE: + case TExprOpcode::FILTER_IN: + case TExprOpcode::FILTER_NOT_IN: + return _check_slot_can_push_down(expr) && _check_rest_children_can_push_down(expr); + + case TExprOpcode::INVALID_OPCODE: + if (expr->node_type() == TExprNodeType::FUNCTION_CALL) { + auto fn_name = expr->fn().name.function_name; + // only support is_null_pred and is_not_null_pred + if (fn_name == "is_null_pred" || fn_name == "is_not_null_pred") { + return _check_slot_can_push_down(expr); + } + VLOG_CRITICAL << "Unsupported function [funciton=" << fn_name << "]"; } - return predicates; + return false; + default: + VLOG_CRITICAL << "Unsupported Opcode [OpCode=" << expr->op() << "]"; + return false; + } +} + +bool OrcReader::_build_less_than(const VExprSPtr& expr, + std::unique_ptr& builder) { + DCHECK(expr->children().size() == 2); + DCHECK(expr->children()[0]->is_slot_ref()); + DCHECK(expr->children()[1]->is_literal()); + const auto* slot_ref = static_cast(expr->children()[0].get()); + const auto* literal = static_cast(expr->children()[1].get()); + auto [valid, orc_literal, predicate_type] = _make_orc_literal(slot_ref, literal); + if (!valid) { + return false; } + builder->lessThan(slot_ref->expr_name(), predicate_type, orc_literal); + return true; +} - const auto& high_value = col_val_range.get_range_max_value(); - const auto& low_value = col_val_range.get_range_min_value(); - const auto& high_op = col_val_range.get_range_high_op(); - const auto& low_op = col_val_range.get_range_low_op(); +bool OrcReader::_build_less_than_equals(const VExprSPtr& expr, + std::unique_ptr& builder) { + DCHECK(expr->children().size() == 2); + DCHECK(expr->children()[0]->is_slot_ref()); + DCHECK(expr->children()[1]->is_literal()); + const auto* slot_ref = static_cast(expr->children()[0].get()); + const auto* literal = static_cast(expr->children()[1].get()); + auto [valid, orc_literal, predicate_type] = _make_orc_literal(slot_ref, literal); + if (!valid) { + return false; + } + builder->lessThanEquals(slot_ref->expr_name(), predicate_type, orc_literal); + return true; +} - // orc can only push down is_null. When col_value_range._contain_null = true, only indicating that - // value can be null, not equals null, so ignore _contain_null in col_value_range - if (col_val_range.is_high_value_maximum() && high_op == SQLFilterOp::FILTER_LESS_OR_EQUAL && - col_val_range.is_low_value_mininum() && low_op == SQLFilterOp::FILTER_LARGER_OR_EQUAL) { - return predicates; +bool OrcReader::_build_equals(const VExprSPtr& expr, + std::unique_ptr& builder) { + DCHECK(expr->children().size() == 2); + DCHECK(expr->children()[0]->is_slot_ref()); + DCHECK(expr->children()[1]->is_literal()); + const auto* slot_ref = static_cast(expr->children()[0].get()); + const auto* literal = static_cast(expr->children()[1].get()); + auto [valid, orc_literal, predicate_type] = _make_orc_literal(slot_ref, literal); + if (!valid) { + return false; } + builder->equals(slot_ref->expr_name(), predicate_type, orc_literal); + return true; +} - if (low_value < high_value) { - if (!col_val_range.is_low_value_mininum() || - SQLFilterOp::FILTER_LARGER_OR_EQUAL != low_op) { - auto [valid, low_literal] = convert_to_orc_literal( - type, &low_value, col_val_range.precision(), col_val_range.scale()); - if (valid) { - OrcPredicate low_predicate; - low_predicate.col_name = col_val_range.column_name(); - low_predicate.data_type = predicate_data_type; - low_predicate.op = low_op; - low_predicate.literals.emplace_back(low_literal); - predicates.emplace_back(low_predicate); - } - } - if (!col_val_range.is_high_value_maximum() || - SQLFilterOp::FILTER_LESS_OR_EQUAL != high_op) { - auto [valid, high_literal] = convert_to_orc_literal( - type, &high_value, col_val_range.precision(), col_val_range.scale()); - if (valid) { - OrcPredicate high_predicate; - high_predicate.col_name = col_val_range.column_name(); - high_predicate.data_type = predicate_data_type; - high_predicate.op = high_op; - high_predicate.literals.emplace_back(high_literal); - predicates.emplace_back(high_predicate); - } +bool OrcReader::_build_filter_in(const VExprSPtr& expr, + std::unique_ptr& builder) { + DCHECK(expr->children().size() >= 2); + DCHECK(expr->children()[0]->is_slot_ref()); + const auto* slot_ref = static_cast(expr->children()[0].get()); + std::vector literals; + orc::PredicateDataType predicate_type = orc::PredicateDataType::LONG; + for (size_t i = 1; i < expr->children().size(); ++i) { + DCHECK(expr->children()[i]->is_literal()); + const auto* literal = static_cast(expr->children()[i].get()); + auto [valid, orc_literal, type] = _make_orc_literal(slot_ref, literal); + if (!valid) { + return false; } + literals.emplace_back(orc_literal); + predicate_type = type; } - return predicates; + DCHECK(!literals.empty()); + builder->in(slot_ref->expr_name(), predicate_type, literals); + return true; } -bool static build_search_argument(std::vector& predicates, int index, - std::unique_ptr& builder) { - if (index >= predicates.size()) { +bool OrcReader::_build_is_null(const VExprSPtr& expr, + std::unique_ptr& builder) { + DCHECK(expr->children().size() == 1); + DCHECK(expr->children()[0]->is_slot_ref()); + const auto* slot_ref = static_cast(expr->children()[0].get()); + auto [valid, _, predicate_type] = _make_orc_literal(slot_ref, nullptr); + builder->isNull(slot_ref->expr_name(), predicate_type); + return true; +} + +bool OrcReader::_build_search_argument(const VExprSPtr& expr, + std::unique_ptr& builder) { + if (expr == nullptr) { return false; } - if (index < predicates.size() - 1) { - builder->startAnd(); + + // if expr can not be pushed down, skip it and continue to next expr + if (!_check_expr_can_push_down(expr)) { + return false; } - OrcPredicate& predicate = predicates[index]; - switch (predicate.op) { - case SQLFilterOp::FILTER_IN: { - if (predicate.literals.size() == 1) { - builder->equals(predicate.col_name, predicate.data_type, predicate.literals[0]); - } else { - builder->in(predicate.col_name, predicate.data_type, predicate.literals); + + switch (expr->op()) { + case TExprOpcode::COMPOUND_AND: { + bool at_least_one_can_push_down = false; + builder->startAnd(); + for (const auto& child : expr->children()) { + if (_build_search_argument(child, builder)) { + at_least_one_can_push_down = true; + } + } + if (!at_least_one_can_push_down) { + // if all exprs can not be pushed down, builder->end() will throw exception + return false; } + builder->end(); break; } - case SQLFilterOp::FILTER_LESS: - builder->lessThan(predicate.col_name, predicate.data_type, predicate.literals[0]); + case TExprOpcode::COMPOUND_OR: + builder->startOr(); + for (const auto& child : expr->children()) { + if (!_build_search_argument(child, builder)) { + return false; + } + } + builder->end(); + break; + case TExprOpcode::COMPOUND_NOT: + builder->startNot(); + DCHECK_EQ(expr->children().size(), 1); + if (!_build_search_argument(expr->children()[0], builder)) { + return false; + } + builder->end(); break; - case SQLFilterOp::FILTER_LESS_OR_EQUAL: - builder->lessThanEquals(predicate.col_name, predicate.data_type, predicate.literals[0]); + case TExprOpcode::GE: + builder->startNot(); + if (!_build_less_than(expr, builder)) { + return false; + } + builder->end(); break; - case SQLFilterOp::FILTER_LARGER: { + case TExprOpcode::GT: builder->startNot(); - builder->lessThanEquals(predicate.col_name, predicate.data_type, predicate.literals[0]); + if (!_build_less_than_equals(expr, builder)) { + return false; + } builder->end(); break; - } - case SQLFilterOp::FILTER_LARGER_OR_EQUAL: { + case TExprOpcode::LE: + if (!_build_less_than_equals(expr, builder)) { + return false; + } + break; + case TExprOpcode::LT: + if (!_build_less_than(expr, builder)) { + return false; + } + break; + case TExprOpcode::EQ: + if (!_build_equals(expr, builder)) { + return false; + } + break; + case TExprOpcode::NE: builder->startNot(); - builder->lessThan(predicate.col_name, predicate.data_type, predicate.literals[0]); + if (!_build_equals(expr, builder)) { + return false; + } builder->end(); break; - } - default: - return false; - } - if (index < predicates.size() - 1) { - bool can_build = build_search_argument(predicates, index + 1, builder); - if (!can_build) { + case TExprOpcode::FILTER_IN: + if (!_build_filter_in(expr, builder)) { + return false; + } + break; + case TExprOpcode::FILTER_NOT_IN: + builder->startNot(); + if (!_build_filter_in(expr, builder)) { return false; } builder->end(); + break; + // is null and is not null is represented as function call + case TExprOpcode::INVALID_OPCODE: { + DCHECK(expr->node_type() == TExprNodeType::FUNCTION_CALL); + if (expr->fn().name.function_name == "is_null_pred") { + if (!_build_is_null(expr, builder)) { + return false; + } + } else if (expr->fn().name.function_name == "is_not_null_pred") { + builder->startNot(); + if (!_build_is_null(expr, builder)) { + return false; + } + builder->end(); + } else { + __builtin_unreachable(); + } + break; + } + default: { + // should not reach here, because _check_expr_can_push_down has already checked + __builtin_unreachable(); + } } return true; } -bool OrcReader::_init_search_argument( - std::unordered_map* colname_to_value_range) { - if ((!_enable_filter_by_min_max) || colname_to_value_range->empty()) { +bool OrcReader::_init_search_argument(const VExprContextSPtrs& conjuncts) { + if (!_enable_filter_by_min_max) { return false; } - std::vector predicates; - auto& root_type = _reader->getType(); - std::unordered_map type_map; - for (int i = 0; i < root_type.getSubtypeCount(); ++i) { - type_map.emplace(get_field_name_lower_case(&root_type, i), root_type.getSubtype(i)); - } - for (auto& col_name : _lazy_read_ctx.all_read_columns) { - auto iter = colname_to_value_range->find(col_name); - if (iter == colname_to_value_range->end()) { - continue; - } - auto type_it = type_map.find(_col_name_to_file_col_name[col_name]); - if (type_it == type_map.end()) { - continue; + + // build search argument, if any expr can not be pushed down, return false + auto builder = orc::SearchArgumentFactory::newBuilder(); + bool at_least_one_can_push_down = false; + builder->startAnd(); + for (const auto& expr_ctx : conjuncts) { + if (_build_search_argument(expr_ctx->root(), builder)) { + at_least_one_can_push_down = true; } - std::visit( - [&](auto& range) { - std::vector value_predicates = value_range_to_predicate( - range, type_it->second, _unsupported_pushdown_types); - for (auto& range_predicate : value_predicates) { - predicates.emplace_back(range_predicate); - } - }, - iter->second); } - if (predicates.empty()) { - return false; - } - std::unique_ptr builder = orc::SearchArgumentFactory::newBuilder(); - if (build_search_argument(predicates, 0, builder)) { - std::unique_ptr sargs = builder->build(); - _row_reader_options.searchArgument(std::move(sargs)); - return true; - } else { + if (!at_least_one_can_push_down) { + // if all exprs can not be pushed down, builder->end() will throw exception return false; } + builder->end(); + + auto sargs = builder->build(); + _profile->add_info_string("OrcReader SearchArgument: ", sargs->toString()); + _row_reader_options.searchArgument(std::move(sargs)); + return true; } Status OrcReader::set_fill_columns( @@ -854,7 +1024,7 @@ Status OrcReader::set_fill_columns( _lazy_read_ctx.can_lazy_read = true; } - if (_colname_to_value_range == nullptr || !_init_search_argument(_colname_to_value_range)) { + if (_lazy_read_ctx.conjuncts.empty() || !_init_search_argument(_lazy_read_ctx.conjuncts)) { _lazy_read_ctx.can_lazy_read = false; } try { @@ -1619,6 +1789,12 @@ std::string OrcReader::get_field_name_lower_case(const orc::Type* orc_type, int Status OrcReader::get_next_block(Block* block, size_t* read_rows, bool* eof) { RETURN_IF_ERROR(get_next_block_impl(block, read_rows, eof)); + if (*eof) { + COUNTER_UPDATE(_orc_profile.selected_row_group_count, + _reader_metrics.SelectedRowGroupCount); + COUNTER_UPDATE(_orc_profile.evaluated_row_group_count, + _reader_metrics.EvaluatedRowGroupCount); + } if (_orc_filter) { RETURN_IF_ERROR(_orc_filter->get_status()); } diff --git a/be/src/vec/exec/format/orc/vorc_reader.h b/be/src/vec/exec/format/orc/vorc_reader.h index 0807f4949e58508..8c73957e79e4e0e 100644 --- a/be/src/vec/exec/format/orc/vorc_reader.h +++ b/be/src/vec/exec/format/orc/vorc_reader.h @@ -18,9 +18,9 @@ #pragma once #include -#include -#include +#include +#include #include #include #include @@ -51,6 +51,8 @@ #include "vec/exec/format/format_common.h" #include "vec/exec/format/generic_reader.h" #include "vec/exec/format/table/transactional_hive_reader.h" +#include "vec/exprs/vliteral.h" +#include "vec/exprs/vslot_ref.h" namespace doris { class RuntimeState; @@ -80,13 +82,6 @@ namespace doris::vectorized { class ORCFileInputStream; -struct OrcPredicate { - std::string col_name; - orc::PredicateDataType data_type; - std::vector literals; - SQLFilterOp op; -}; - struct LazyReadContext { VExprContextSPtrs conjuncts; bool can_lazy_read = false; @@ -228,6 +223,8 @@ class OrcReader : public GenericReader { RuntimeProfile::Counter* decode_value_time = nullptr; RuntimeProfile::Counter* decode_null_map_time = nullptr; RuntimeProfile::Counter* filter_block_time = nullptr; + RuntimeProfile::Counter* selected_row_group_count = nullptr; + RuntimeProfile::Counter* evaluated_row_group_count = nullptr; }; class ORCFilterImpl : public orc::ORCFilter { @@ -291,8 +288,23 @@ class OrcReader : public GenericReader { bool* is_hive1_orc); static bool _check_acid_schema(const orc::Type& type); static const orc::Type& _remove_acid(const orc::Type& type); - bool _init_search_argument( - std::unordered_map* colname_to_value_range); + std::tuple _make_orc_literal( + const VSlotRef* slot_ref, const VLiteral* literal); + bool _check_slot_can_push_down(const VExprSPtr& expr); + bool _check_rest_children_can_push_down(const VExprSPtr& expr); + bool _check_expr_can_push_down(const VExprSPtr& expr); + bool _build_less_than(const VExprSPtr& expr, + std::unique_ptr& builder); + bool _build_less_than_equals(const VExprSPtr& expr, + std::unique_ptr& builder); + bool _build_equals(const VExprSPtr& expr, std::unique_ptr& builder); + bool _build_filter_in(const VExprSPtr& expr, + std::unique_ptr& builder); + bool _build_is_null(const VExprSPtr& expr, + std::unique_ptr& builder); + bool _build_search_argument(const VExprSPtr& expr, + std::unique_ptr& builder); + bool _init_search_argument(const VExprContextSPtrs& conjuncts); void _init_bloom_filter( std::unordered_map* colname_to_value_range); void _init_system_properties(); @@ -578,11 +590,14 @@ class OrcReader : public GenericReader { bool _is_hive1_orc_or_use_idx = false; std::unordered_map _col_name_to_file_col_name; + // TODO: check if we can remove _col_name_to_file_col_name_low_case + std::unordered_map _col_name_to_file_col_name_low_case; std::unordered_map _type_map; std::vector _col_orc_type; std::unique_ptr _file_input_stream; Statistics _statistics; OrcProfile _orc_profile; + orc::ReaderMetrics _reader_metrics; std::unique_ptr _batch; std::unique_ptr _reader; diff --git a/be/src/vec/exec/format/parquet/schema_desc.cpp b/be/src/vec/exec/format/parquet/schema_desc.cpp index de8797062640287..1eae65b1a4db19b 100644 --- a/be/src/vec/exec/format/parquet/schema_desc.cpp +++ b/be/src/vec/exec/format/parquet/schema_desc.cpp @@ -137,6 +137,9 @@ Status FieldDescriptor::parse_from_thrift(const std::vectorsecond.data()}; +} + Status FieldDescriptor::parse_node_field(const std::vector& t_schemas, size_t curr_pos, FieldSchema* node_field) { if (curr_pos >= t_schemas.size()) { @@ -172,6 +183,7 @@ Status FieldDescriptor::parse_node_field(const std::vectortype.add_sub_type(child->type); node_field->is_nullable = false; _next_schema_pos = curr_pos + 1; + node_field->field_id = t_schema.__isset.field_id ? t_schema.field_id : -1; } else { bool is_optional = is_optional_node(t_schema); if (is_optional) { @@ -194,6 +206,7 @@ void FieldDescriptor::parse_physical_field(const tparquet::SchemaElement& physic auto type = get_doris_type(physical_schema); physical_field->type = type.first; physical_field->is_type_compatibility = type.second; + physical_field->field_id = physical_schema.__isset.field_id ? physical_schema.field_id : -1; } std::pair FieldDescriptor::get_doris_type( @@ -465,6 +478,7 @@ Status FieldDescriptor::parse_group_field(const std::vectortype.type = TYPE_ARRAY; group_field->type.add_sub_type(struct_field->type); group_field->is_nullable = false; + group_field->field_id = group_schema.__isset.field_id ? group_schema.field_id : -1; } else { RETURN_IF_ERROR(parse_struct_field(t_schemas, curr_pos, group_field)); } @@ -533,6 +547,7 @@ Status FieldDescriptor::parse_list_field(const std::vectortype.type = TYPE_ARRAY; list_field->type.add_sub_type(list_field->children[0].type); list_field->is_nullable = is_optional; + list_field->field_id = first_level.__isset.field_id ? first_level.field_id : -1; return Status::OK(); } @@ -597,6 +612,7 @@ Status FieldDescriptor::parse_map_field(const std::vectortype.add_sub_type(map_kv_field->type.children[0]); map_field->type.add_sub_type(map_kv_field->type.children[1]); map_field->is_nullable = is_optional; + map_field->field_id = map_schema.__isset.field_id ? map_schema.field_id : -1; return Status::OK(); } @@ -619,6 +635,7 @@ Status FieldDescriptor::parse_struct_field(const std::vectorname = to_lower(struct_schema.name); struct_field->is_nullable = is_optional; struct_field->type.type = TYPE_STRUCT; + struct_field->field_id = struct_schema.__isset.field_id ? struct_schema.field_id : -1; for (int i = 0; i < num_children; ++i) { struct_field->type.add_sub_type(struct_field->children[i].type, struct_field->children[i].name); diff --git a/be/src/vec/exec/format/parquet/schema_desc.h b/be/src/vec/exec/format/parquet/schema_desc.h index ca726ef1b575900..2593da837c3da66 100644 --- a/be/src/vec/exec/format/parquet/schema_desc.h +++ b/be/src/vec/exec/format/parquet/schema_desc.h @@ -28,6 +28,7 @@ #include "common/status.h" #include "runtime/types.h" +#include "util/slice.h" namespace doris::vectorized { @@ -56,6 +57,8 @@ struct FieldSchema { ~FieldSchema() = default; FieldSchema(const FieldSchema& fieldSchema) = default; std::string debug_string() const; + + int32_t field_id; }; class FieldDescriptor { @@ -68,6 +71,7 @@ class FieldDescriptor { std::unordered_map _name_to_field; // Used in from_thrift, marking the next schema position that should be parsed size_t _next_schema_pos; + std::unordered_map _field_id_name_mapping; void parse_physical_field(const tparquet::SchemaElement& physical_schema, bool is_nullable, FieldSchema* physical_field); @@ -128,6 +132,10 @@ class FieldDescriptor { std::string debug_string() const; int32_t size() const { return _fields.size(); } + + bool has_parquet_field_id() const { return _field_id_name_mapping.size() > 0; } + + const doris::Slice get_column_name_from_field_id(int32_t id) const; }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/parquet/vparquet_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_reader.cpp index 1a3ae2f885e4a06..47209dbb332cb8d 100644 --- a/be/src/vec/exec/format/parquet/vparquet_reader.cpp +++ b/be/src/vec/exec/format/parquet/vparquet_reader.cpp @@ -253,10 +253,8 @@ Status ParquetReader::_open_file() { return Status::OK(); } -// Get iceberg col id to col name map stored in parquet metadata key values. -// This is for iceberg schema evolution. -std::vector ParquetReader::get_metadata_key_values() { - return _t_metadata->key_value_metadata; +const FieldDescriptor ParquetReader::get_file_metadata_schema() { + return _file_metadata->schema(); } Status ParquetReader::open() { diff --git a/be/src/vec/exec/format/parquet/vparquet_reader.h b/be/src/vec/exec/format/parquet/vparquet_reader.h index a7768c8bd6c4f85..b2aa156a79d0887 100644 --- a/be/src/vec/exec/format/parquet/vparquet_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_reader.h @@ -148,7 +148,7 @@ class ParquetReader : public GenericReader { partition_columns, const std::unordered_map& missing_columns) override; - std::vector get_metadata_key_values(); + const FieldDescriptor get_file_metadata_schema(); void set_table_to_file_col_map(std::unordered_map& map) { _table_col_to_file_col = map; } diff --git a/be/src/vec/exec/format/table/iceberg_reader.cpp b/be/src/vec/exec/format/table/iceberg_reader.cpp index 295a3a405441b61..8f130ca6002d5d4 100644 --- a/be/src/vec/exec/format/table/iceberg_reader.cpp +++ b/be/src/vec/exec/format/table/iceberg_reader.cpp @@ -53,6 +53,7 @@ #include "vec/exec/format/format_common.h" #include "vec/exec/format/generic_reader.h" #include "vec/exec/format/orc/vorc_reader.h" +#include "vec/exec/format/parquet/schema_desc.h" #include "vec/exec/format/table/table_format_reader.h" namespace cctz { @@ -546,8 +547,8 @@ Status IcebergParquetReader::init_reader( _col_id_name_map = col_id_name_map; _file_col_names = file_col_names; _colname_to_value_range = colname_to_value_range; - auto parquet_meta_kv = parquet_reader->get_metadata_key_values(); - RETURN_IF_ERROR(_gen_col_name_maps(parquet_meta_kv)); + FieldDescriptor field_desc = parquet_reader->get_file_metadata_schema(); + RETURN_IF_ERROR(_gen_col_name_maps(field_desc)); _gen_file_col_names(); _gen_new_colname_to_value_range(); parquet_reader->set_table_to_file_col_map(_table_col_to_file_col); @@ -672,39 +673,20 @@ Status IcebergOrcReader::_read_position_delete_file(const TFileRangeDesc* delete * 1. col1_new -> col1 * 2. col1 -> col1_new */ -Status IcebergParquetReader::_gen_col_name_maps(std::vector parquet_meta_kv) { - for (int i = 0; i < parquet_meta_kv.size(); ++i) { - tparquet::KeyValue kv = parquet_meta_kv[i]; - if (kv.key == "iceberg.schema") { - _has_iceberg_schema = true; - std::string schema = kv.value; - rapidjson::Document json; - json.Parse(schema.c_str()); - - if (json.HasMember("fields")) { - rapidjson::Value& fields = json["fields"]; - if (fields.IsArray()) { - for (int j = 0; j < fields.Size(); j++) { - rapidjson::Value& e = fields[j]; - rapidjson::Value& id = e["id"]; - rapidjson::Value& name = e["name"]; - std::string name_string = name.GetString(); - transform(name_string.begin(), name_string.end(), name_string.begin(), - ::tolower); - auto iter = _col_id_name_map.find(id.GetInt()); - if (iter != _col_id_name_map.end()) { - _table_col_to_file_col.emplace(iter->second, name_string); - _file_col_to_table_col.emplace(name_string, iter->second); - if (name_string != iter->second) { - _has_schema_change = true; - } - } else { - _has_schema_change = true; - } - } +Status IcebergParquetReader::_gen_col_name_maps(const FieldDescriptor& field_desc) { + if (field_desc.has_parquet_field_id()) { + for (const auto& pair : _col_id_name_map) { + auto name_slice = field_desc.get_column_name_from_field_id(pair.first); + if (name_slice.get_size() == 0) { + _has_schema_change = true; + } else { + auto name_string = name_slice.to_string(); + _table_col_to_file_col.emplace(pair.second, name_string); + _file_col_to_table_col.emplace(name_string, pair.second); + if (name_string != pair.second) { + _has_schema_change = true; } } - break; } } return Status::OK(); diff --git a/be/src/vec/exec/format/table/iceberg_reader.h b/be/src/vec/exec/format/table/iceberg_reader.h index 04f64aad518a9fb..2e240f465b6a2cd 100644 --- a/be/src/vec/exec/format/table/iceberg_reader.h +++ b/be/src/vec/exec/format/table/iceberg_reader.h @@ -218,7 +218,7 @@ class IcebergParquetReader final : public IcebergTableReader { parquet_reader->set_delete_rows(&_iceberg_delete_rows); } - Status _gen_col_name_maps(std::vector parquet_meta_kv); + Status _gen_col_name_maps(const FieldDescriptor& field_desc); protected: std::unique_ptr _create_equality_reader( diff --git a/be/test/exec/test_data/orc_scanner/orders.orc b/be/test/exec/test_data/orc_scanner/orders.orc new file mode 100644 index 000000000000000..6fad5043288d430 Binary files /dev/null and b/be/test/exec/test_data/orc_scanner/orders.orc differ diff --git a/be/test/io/fs/local_file_system_test.cpp b/be/test/io/fs/local_file_system_test.cpp index 0fd18445beac4c8..c930ba72eabf867 100644 --- a/be/test/io/fs/local_file_system_test.cpp +++ b/be/test/io/fs/local_file_system_test.cpp @@ -417,4 +417,54 @@ TEST_F(LocalFileSystemTest, TestGlob) { EXPECT_TRUE(io::global_local_filesystem()->delete_directory(path).ok()); } +TEST_F(LocalFileSystemTest, TestConvertToAbsPath) { + io::Path abs_path; + Status st; + + // suppurt path: + st = doris::io::LocalFileSystem::convert_to_abs_path("/abc/def", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/abc/def", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file:/def/hij", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/def/hij", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file://host:80/hij/abc", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/hij/abc", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file://host/abc/def", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/abc/def", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file:///def", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/def", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file:///", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file://auth/", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("abc", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("abc", abs_path); + + // not support path: + st = doris::io::LocalFileSystem::convert_to_abs_path("file://auth", abs_path); + ASSERT_TRUE(!st.ok()); + + st = doris::io::LocalFileSystem::convert_to_abs_path("fileee:/abc", abs_path); + ASSERT_TRUE(!st.ok()); + + st = doris::io::LocalFileSystem::convert_to_abs_path("hdfs:///abc", abs_path); + ASSERT_TRUE(!st.ok()); + + st = doris::io::LocalFileSystem::convert_to_abs_path("hdfs:/abc", abs_path); + ASSERT_TRUE(!st.ok()); +} } // namespace doris diff --git a/be/test/testutil/desc_tbl_builder.cpp b/be/test/testutil/desc_tbl_builder.cpp index 4cba9a44a4b0d1b..6404d1c5449165d 100644 --- a/be/test/testutil/desc_tbl_builder.cpp +++ b/be/test/testutil/desc_tbl_builder.cpp @@ -17,20 +17,9 @@ #include "testutil/desc_tbl_builder.h" -#include -#include -#include +#include -#include - -#include "common/object_pool.h" #include "common/status.h" -#include "gtest/gtest_pred_impl.h" -#include "runtime/define_primitive_type.h" -#include "runtime/descriptors.h" -#include "util/bit_util.h" - -using std::vector; namespace doris { @@ -44,7 +33,7 @@ TupleDescBuilder& DescriptorTblBuilder::declare_tuple() { // item_id of -1 indicates no itemTupleId static TSlotDescriptor make_slot_descriptor(int id, int parent_id, const TypeDescriptor& type, - int slot_idx, int item_id) { + const std::string& name, int slot_idx, int item_id) { int null_byte = slot_idx / 8; int null_bit = slot_idx % 8; TSlotDescriptor slot_desc; @@ -58,6 +47,7 @@ static TSlotDescriptor make_slot_descriptor(int id, int parent_id, const TypeDes slot_desc.__set_nullIndicatorBit(null_bit); slot_desc.__set_slotIdx(slot_idx); slot_desc.__set_isMaterialized(true); + slot_desc.__set_colName(name); // if (item_id != -1) { // slot_desc.__set_itemTupleId(item_id); // } @@ -78,8 +68,9 @@ DescriptorTbl* DescriptorTblBuilder::build() { int tuple_id = 0; int slot_id = 0; - for (int i = 0; i < _tuples_descs.size(); ++i) { - build_tuple(_tuples_descs[i]->slot_types(), &thrift_desc_tbl, &tuple_id, &slot_id); + for (auto& _tuples_desc : _tuples_descs) { + build_tuple(_tuples_desc->slot_types(), _tuples_desc->slot_names(), &thrift_desc_tbl, + &tuple_id, &slot_id); } Status status = DescriptorTbl::create(_obj_pool, thrift_desc_tbl, &desc_tbl); @@ -87,7 +78,8 @@ DescriptorTbl* DescriptorTblBuilder::build() { return desc_tbl; } -TTupleDescriptor DescriptorTblBuilder::build_tuple(const vector& slot_types, +TTupleDescriptor DescriptorTblBuilder::build_tuple(const std::vector& slot_types, + const std::vector& slot_names, TDescriptorTable* thrift_desc_tbl, int* next_tuple_id, int* slot_id) { // We never materialize struct slots (there's no in-memory representation of structs, @@ -95,7 +87,8 @@ TTupleDescriptor DescriptorTblBuilder::build_tuple(const vector& // still have a struct item type. In this case, the array item tuple contains the // "inlined" struct fields. if (slot_types.size() == 1 && slot_types[0].type == TYPE_STRUCT) { - return build_tuple(slot_types[0].children, thrift_desc_tbl, next_tuple_id, slot_id); + return build_tuple(slot_types[0].children, slot_types[0].field_names, thrift_desc_tbl, + next_tuple_id, slot_id); } int tuple_id = *next_tuple_id; @@ -111,7 +104,7 @@ TTupleDescriptor DescriptorTblBuilder::build_tuple(const vector& // } thrift_desc_tbl->slotDescriptors.push_back( - make_slot_descriptor(*slot_id, tuple_id, slot_types[i], i, item_id)); + make_slot_descriptor(*slot_id, tuple_id, slot_types[i], slot_names[i], i, item_id)); thrift_desc_tbl->__isset.slotDescriptors = true; ++(*slot_id); } diff --git a/be/test/testutil/desc_tbl_builder.h b/be/test/testutil/desc_tbl_builder.h index c29ef9acd43eb51..968b29bd00143cc 100644 --- a/be/test/testutil/desc_tbl_builder.h +++ b/be/test/testutil/desc_tbl_builder.h @@ -20,15 +20,16 @@ #include +#include #include +#include "common/object_pool.h" +#include "runtime/descriptors.h" #include "runtime/types.h" namespace doris { -class ObjectPool; class TupleDescBuilder; -class DescriptorTbl; // Aids in the construction of a DescriptorTbl by declaring tuples and slots // associated with those tuples. @@ -40,6 +41,7 @@ class DescriptorTbl; // DescriptorTblBuilder builder; // builder.declare_tuple() << TYPE_TINYINT << TYPE_TIMESTAMP; // gets TupleId 0 // builder.declare_tuple() << TYPE_FLOAT; // gets TupleId 1 +// builder.declare_tuple() << std::make_tuple(TYPE_INT, "col1") << std::make_tuple(TYPE_STRING, "col2"); // gets Tuple with type and name // DescriptorTbl desc_tbl = builder.build(); class DescriptorTblBuilder { public: @@ -57,20 +59,31 @@ class DescriptorTblBuilder { std::vector _tuples_descs; TTupleDescriptor build_tuple(const std::vector& slot_types, + const std::vector& slot_names, TDescriptorTable* thrift_desc_tbl, int* tuple_id, int* slot_id); }; class TupleDescBuilder { public: + using SlotType = std::tuple; + TupleDescBuilder& operator<<(const SlotType& slot) { + _slot_types.push_back(std::get<0>(slot)); + _slot_names.push_back(std::get<1>(slot)); + return *this; + } + TupleDescBuilder& operator<<(const TypeDescriptor& slot_type) { _slot_types.push_back(slot_type); + _slot_names.emplace_back(""); return *this; } std::vector slot_types() const { return _slot_types; } + std::vector slot_names() const { return _slot_names; } private: std::vector _slot_types; + std::vector _slot_names; }; } // end namespace doris diff --git a/be/test/vec/exec/orc_reader_test.cpp b/be/test/vec/exec/orc_reader_test.cpp new file mode 100644 index 000000000000000..ec5bd9b519de633 --- /dev/null +++ b/be/test/vec/exec/orc_reader_test.cpp @@ -0,0 +1,155 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include + +#include +#include +#include +#include + +#include "orc/sargs/SearchArgument.hh" +#include "runtime/define_primitive_type.h" +#include "runtime/exec_env.h" +#include "runtime/runtime_state.h" +#include "testutil/desc_tbl_builder.h" +#include "vec/exec/format/orc/orc_memory_pool.h" +#include "vec/exec/format/orc/vorc_reader.h" +#include "vec/exprs/vexpr_context.h" +#include "vec/exprs/vexpr_fwd.h" +#include "vec/utils/util.hpp" +namespace doris::vectorized { +class OrcReaderTest : public testing::Test { +public: + OrcReaderTest() = default; + ~OrcReaderTest() override = default; + +private: + static constexpr const char* CANNOT_PUSH_DOWN_ERROR = "can't push down"; + std::string build_search_argument(const std::string& expr) { + // build orc_reader for table orders + std::vector column_names = { + "o_orderkey", "o_custkey", "o_orderstatus", "o_totalprice", "o_orderdate", + "o_orderpriority", "o_clerk", "o_shippriority", "o_comment"}; + ObjectPool object_pool; + DescriptorTblBuilder builder(&object_pool); + builder.declare_tuple() << std::make_tuple(TYPE_INT, "o_orderkey") + << std::make_tuple(TYPE_INT, "o_custkey") + << std::make_tuple(TYPE_STRING, "o_orderstatus") + << std::make_tuple(TYPE_DOUBLE, "o_totalprice") + << std::make_tuple(TYPE_DATE, "o_orderdate") + << std::make_tuple(TYPE_STRING, "o_orderpriority") + << std::make_tuple(TYPE_STRING, "o_clerk") + << std::make_tuple(TYPE_INT, "o_shippriority") + << std::make_tuple(TYPE_STRING, "o_comment"); + DescriptorTbl* desc_tbl = builder.build(); + auto* tuple_desc = const_cast(desc_tbl->get_tuple_descriptor(0)); + RowDescriptor row_desc(tuple_desc, false); + TFileScanRangeParams params; + TFileRangeDesc range; + range.path = "./be/test/exec/test_data/orc_scanner/orders.orc"; + range.start_offset = 0; + range.size = 1293; + auto reader = OrcReader::create_unique(params, range, "", nullptr, true); + auto status = reader->init_reader(&column_names, nullptr, {}, false, tuple_desc, &row_desc, + nullptr, nullptr); + EXPECT_TRUE(status.ok()); + + // deserialize expr + auto exprx = apache::thrift::from_json_string(expr); + VExprContextSPtr context; + status = VExpr::create_expr_tree(exprx, context); + EXPECT_TRUE(status.ok()); + + // prepare expr context + RuntimeState state; + state.set_desc_tbl(desc_tbl); + status = context->prepare(&state, row_desc); + EXPECT_TRUE(status.ok()); + + // build search argument + auto sarg_builder = orc::SearchArgumentFactory::newBuilder(); + auto res = reader->_build_search_argument(context->root(), sarg_builder); + if (!res) { + return CANNOT_PUSH_DOWN_ERROR; + } + return sarg_builder->build()->toString(); + } +}; + +TEST_F(OrcReaderTest, test_build_search_argument) { + ExecEnv::GetInstance()->set_orc_memory_pool(new ORCMemoryPool()); + std:: + vector + exprs = + { + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 100 or o_orderkey > 5999900 or o_orderkey in (1000000, 2000000, 3000000); + R"|({"1":{"lst":["rec",13,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":13},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"gt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"gt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":5999900}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":5},"4":{"i32":4},"11":{"rec":{"1":{"tf":0}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":3000000}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 100 and 1000 and o_orderkey not in (200, 300, 400)); + R"|({"1":{"lst":["rec",16,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"is_null_pred"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"is_null_pred(int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":6},"4":{"i32":4},"11":{"rec":{"1":{"tf":1}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":200}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":300}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":400}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 1000000 and 1200000 and o_orderkey != 1100000); + R"|({"1":{"lst":["rec",14,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"is_null_pred"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"is_null_pred(int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1200000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":10},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ne"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ne(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1100000}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_orderkey IN (1000000, 2000000, 3000000) OR (o_orderdate >= '1994-01-01' AND o_orderdate <= '1994-12-31'); + R"|({"1":{"lst":["rec",13,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":5},"4":{"i32":4},"11":{"rec":{"1":{"tf":0}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":3000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(datev2, datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":26},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"1994-01-01"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(datev2, datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":26},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"1994-12-31"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 2 or (o_comment like '%delayed%' and o_orderpriority = '1-URGENT'); + R"|({"1":{"lst":["rec",11,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"like"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":9}}}}]},"3":{"i64":9}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"like(text, varchar(9))"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":8},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_comment"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"%delayed%"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(text, varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":23},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-URGENT"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 1 + 1; + R"|({"1":{"lst":["rec",3,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_comment LIKE '%delayed%' OR o_orderpriority = '1-URGENT'; + R"|({"1":{"lst":["rec",7,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"like"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":9}}}}]},"3":{"i64":9}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"like(text, varchar(9))"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":8},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_comment"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"%delayed%"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(text, varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":23},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-URGENT"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey between 1 and 100 or random() > 0.5; + R"|({"1":{"lst":["rec",11,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":13},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"gt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"gt(double, double)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":8},"29":{"tf":0}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"random"}}},"2":{"i32":0},"3":{"lst":["rec",0]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"random()"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":8},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"9":{"rec":{"1":{"dbl":0.5}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where lower(o_orderpriority) = '1-urgent'; + R"|({"1":{"lst":["rec",4,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(varchar(65533), varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":15},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lower"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lower(text)"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-urgent"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey * 2 < 60; + R"|({"1":{"lst":["rec",5,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(bigint, bigint)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":6},"29":{"tf":1}},{"1":{"i32":1},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"3":{"i32":55},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"multiply"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":3}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"multiply(int, tinyint)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":3}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":60}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderdate is not null; + R"|({"1":{"lst":["rec",4,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(datetimev2(0), datetimev2(0))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":27},"29":{"tf":1}},{"1":{"i32":5},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"3":{"i32":4},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"casttodatetimev2"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"casttodatetimev2(datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"2024-11-12 21:13:02"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + }; + std::vector result_search_arguments = { + "leaf-0 = (o_orderkey < 100), leaf-1 = (o_orderkey <= 5999900), leaf-2 " + "= (o_orderkey " + "in " + "[1000000, 2000000, 3000000]), expr = (or leaf-0 (not leaf-1) leaf-2)", + "leaf-0 = (o_orderkey is null), leaf-1 = (o_orderkey < 100), leaf-2 = " + "(o_orderkey <= " + "1000), leaf-3 = (o_orderkey in [200, 300, 400]), expr = (and (or " + "leaf-0 (not leaf-1)) " + "(or leaf-0 leaf-2) (or leaf-0 (not leaf-3)))", + "leaf-0 = (o_orderkey is null), leaf-1 = (o_orderkey < 1000000), leaf-2 = (o_orderkey " + "<= 1200000), leaf-3 = (o_orderkey = 1100000), expr = (and (or leaf-0 (not leaf-1)) " + "(or leaf-0 leaf-2) (or leaf-0 (not leaf-3)))", + "leaf-0 = (o_orderkey in [1000000, 2000000, 3000000]), leaf-1 = (o_orderdate < " + "17121205), leaf-2 = (o_orderdate <= 17121205), expr = (and (or leaf-0 (not leaf-1)) " + "(or leaf-0 leaf-2))", + "leaf-0 = (o_orderkey < 2), leaf-1 = (o_orderpriority = 1-URGENT), expr = (or leaf-0 " + "leaf-1)", + "leaf-0 = (o_orderkey < 2), expr = leaf-0", + CANNOT_PUSH_DOWN_ERROR, + CANNOT_PUSH_DOWN_ERROR, + CANNOT_PUSH_DOWN_ERROR, + CANNOT_PUSH_DOWN_ERROR, + CANNOT_PUSH_DOWN_ERROR, + }; + for (int i = 0; i < exprs.size(); i++) { + auto search_argument = build_search_argument(exprs[i]); + ASSERT_EQ(search_argument, result_search_arguments[i]); + } +} + +} // namespace doris::vectorized diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 38c49c355c03a34..b4848237a1768c2 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -224,6 +224,7 @@ supportedShowStatement | SHOW CREATE MATERIALIZED VIEW mvName=identifier ON tableName=multipartIdentifier #showCreateMaterializedView | SHOW BACKENDS #showBackends + | SHOW FULL? TRIGGERS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTriggers | SHOW FRONTENDS name=identifier? #showFrontends | SHOW TABLE tableId=INTEGER_VALUE #showTableId | SHOW WHITELIST #showWhitelist @@ -270,7 +271,6 @@ unsupportedShowStatement | SHOW FULL? VIEWS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showViews | SHOW FULL? PROCESSLIST #showProcessList | SHOW (GLOBAL | SESSION | LOCAL)? STATUS wildWhere? #showStatus - | SHOW FULL? TRIGGERS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTriggers | SHOW EVENTS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showEvents | SHOW CREATE VIEW name=multipartIdentifier #showCreateView | SHOW CREATE MATERIALIZED VIEW name=multipartIdentifier #showMaterializedView diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index ebb194ed6a6262b..a2ed437349850c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -701,11 +701,6 @@ private void modifyViewDef(Database db, View view, String inlineViewDef, long sq view.writeLockOrDdlException(); try { view.setInlineViewDefWithSqlMode(inlineViewDef, sqlMode); - try { - view.init(); - } catch (UserException e) { - throw new DdlException("failed to init view stmt, reason=" + e.getMessage()); - } view.setNewFullSchema(newFullSchema); String viewName = view.getName(); db.unregisterTable(viewName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index b7a0fa5cfbc7462..6eaf7d5522c96d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -434,9 +434,12 @@ private boolean processDropColumn(DropColumnClause alterClause, OlapTable olapTa // drop bloom filter column Set bfCols = olapTable.getCopiedBfColumns(); if (bfCols != null) { - Set newBfCols = new HashSet<>(); + Set newBfCols = null; for (String bfCol : bfCols) { if (!bfCol.equalsIgnoreCase(dropColName)) { + if (newBfCols == null) { + newBfCols = Sets.newHashSet(); + } newBfCols.add(bfCol); } } @@ -2953,6 +2956,25 @@ public void modifyTableLightSchemaChange(String rawSql, Database db, OlapTable o LOG.info("finished modify table's add or drop or modify columns. table: {}, job: {}, is replay: {}", olapTable.getName(), jobId, isReplay); } + // for bloom filter, rebuild bloom filter info by table schema in replay + if (isReplay) { + Set bfCols = olapTable.getCopiedBfColumns(); + if (bfCols != null) { + List columns = olapTable.getBaseSchema(); + Set newBfCols = null; + for (String bfCol : bfCols) { + for (Column column : columns) { + if (column.getName().equalsIgnoreCase(bfCol)) { + if (newBfCols == null) { + newBfCols = Sets.newHashSet(); + } + newBfCols.add(column.getName()); + } + } + } + olapTable.setBloomFilterInfo(newBfCols, olapTable.getBfFpp()); + } + } } public void replayModifyTableLightSchemaChange(TableAddOrDropColumnsInfo info) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java index e265703128dff70..d33c234af113bf7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java @@ -21,20 +21,24 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.SqlParserUtils; import org.apache.doris.common.util.ToSqlContext; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.StringReader; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -147,6 +151,7 @@ protected void createColumnAndViewDefs(Analyzer analyzer) throws AnalysisExcepti // we don't need the slot id info, so using ToSqlContext to remove it. toSqlContext.setNeedSlotRefId(false); inlineViewDef = viewDefStmt.toSql(); + checkInlineViewDef(); } return; } @@ -161,6 +166,7 @@ protected void createColumnAndViewDefs(Analyzer analyzer) throws AnalysisExcepti // we don't need the slot id info, so using ToSqlContext to remove it. toSqlContext.setNeedSlotRefId(false); inlineViewDef = cloneStmt.toSql(); + checkInlineViewDef(); } } @@ -172,4 +178,24 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { throw new AnalysisException("Not support OUTFILE clause in CREATE VIEW statement"); } } + + private void checkInlineViewDef() throws UserException { + Preconditions.checkNotNull(inlineViewDef); + SqlScanner input = new SqlScanner(new StringReader(inlineViewDef), + ConnectContext.get().getSessionVariable().getSqlMode()); + SqlParser parser = new SqlParser(input); + ParseNode node; + try { + node = SqlParserUtils.getFirstStmt(parser); + } catch (Exception e) { + throw new DdlException( + String.format("Failed to parse view-definition statement of view: %s, stmt is %s, reason is %s", + tableName, inlineViewDef, e.getMessage())); + } + // Make sure the view definition parses to a query statement. + if (!(node instanceof QueryStmt)) { + throw new DdlException(String.format("View definition of %s " + + "is not a query statement", tableName)); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java index 2715bd1f6da2f9e..47fdfdce4e23c88 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java @@ -88,6 +88,9 @@ public void analyze(Analyzer analyzer) throws UserException { public String toSql() { StringBuilder stringBuilder = new StringBuilder(); stringBuilder.append("DROP DATABASE ").append("`").append(dbName).append("`"); + if (forceDrop) { + stringBuilder.append(" FORCE"); + } return stringBuilder.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java index 5e06fce75ee0740..d6a19e81f8e3f47 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java @@ -100,6 +100,9 @@ public void analyze(Analyzer analyzer) throws UserException { public String toSql() { StringBuilder stringBuilder = new StringBuilder(); stringBuilder.append("DROP TABLE ").append(tableName.toSql()); + if (forceDrop) { + stringBuilder.append(" FORCE"); + } return stringBuilder.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index fa406faf9d34d69..85e041b88b52fb0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -5663,13 +5663,6 @@ public void createView(CreateViewStmt stmt) throws DdlException { newView.setComment(stmt.getComment()); newView.setInlineViewDefWithSqlMode(stmt.getInlineViewDef(), ConnectContext.get().getSessionVariable().getSqlMode()); - // init here in case the stmt string from view.toSql() has some syntax error. - try { - newView.init(); - } catch (UserException e) { - throw new DdlException("failed to init view stmt, reason=" + e.getMessage()); - } - if (!((Database) db).createTableWithLock(newView, false, stmt.isSetIfNotExists()).first) { throw new DdlException("Failed to create view[" + tableName + "]."); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index 955bfd4279fd5c5..825bdef9f09819f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -59,6 +59,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -363,7 +364,7 @@ public MTMVRefreshSnapshot getRefreshSnapshot() { * @return mvPartitionName ==> mvPartitionKeyDesc */ public Map generateMvPartitionDescs() throws AnalysisException { - Map mtmvItems = getAndCopyPartitionItems(); + Map mtmvItems = getAndCopyPartitionItems(OptionalLong.empty()); Map result = Maps.newHashMap(); for (Entry entry : mtmvItems.entrySet()) { result.put(entry.getKey(), entry.getValue().toPartitionKeyDesc()); @@ -392,7 +393,7 @@ public Pair>, Map> calculateDoublyPartit Map baseToMv = Maps.newHashMap(); Map> relatedPartitionDescs = MTMVPartitionUtil .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItems(); + Map mvPartitionItems = getAndCopyPartitionItems(OptionalLong.empty()); for (Entry entry : mvPartitionItems.entrySet()) { Set basePartitionNames = relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet()); @@ -425,7 +426,7 @@ public Map> calculatePartitionMappings() throws AnalysisExce Map> res = Maps.newHashMap(); Map> relatedPartitionDescs = MTMVPartitionUtil .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItems(); + Map mvPartitionItems = getAndCopyPartitionItems(OptionalLong.empty()); for (Entry entry : mvPartitionItems.entrySet()) { res.put(entry.getKey(), relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 8a9ce4a2a2a5452..547a520a061e63c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -110,6 +110,7 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; @@ -3260,7 +3261,7 @@ public PartitionType getPartitionType() { } @Override - public Map getAndCopyPartitionItems() throws AnalysisException { + public Map getAndCopyPartitionItems(OptionalLong snapshotId) throws AnalysisException { if (!tryReadLock(1, TimeUnit.MINUTES)) { throw new AnalysisException("get table read lock timeout, database=" + getDBName() + ",table=" + getName()); } @@ -3284,7 +3285,8 @@ public List getPartitionColumns() { } @Override - public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context) + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + OptionalLong snapshotId) throws AnalysisException { Map partitionVersions = context.getBaseVersions().getPartitionVersions(); long partitionId = getPartitionOrAnalysisException(partitionName).getId(); @@ -3294,7 +3296,7 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont } @Override - public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context) { + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) { Map tableVersions = context.getBaseVersions().getTableVersions(); long visibleVersion = tableVersions.containsKey(id) ? tableVersions.get(id) : getVisibleVersion(); return new MTMVVersionSnapshot(visibleVersion, id); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java index 345254108345068..cde08113373aee4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java @@ -39,6 +39,7 @@ import org.apache.doris.common.Version; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; +import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.es.EsExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalCatalog; @@ -149,6 +150,7 @@ public abstract class ExternalCatalog protected Optional useMetaCache = Optional.empty(); protected MetaCache> metaCache; + protected PreExecutionAuthenticator preExecutionAuthenticator; public ExternalCatalog() { } @@ -418,6 +420,7 @@ private void refreshOnlyCatalogCache(boolean invalidCache) { if (useMetaCache.get() && metaCache != null) { metaCache.invalidateAll(); } else if (!useMetaCache.get()) { + this.initialized = false; for (ExternalDatabase db : idToDb.values()) { db.setUnInitialized(invalidCache); } @@ -934,4 +937,8 @@ public void setAutoAnalyzePolicy(String dbName, String tableName, String policy) tableAutoAnalyzePolicy.put(key, policy); } } + + public PreExecutionAuthenticator getPreExecutionAuthenticator() { + return preExecutionAuthenticator; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java index 20b9482041df02f..85b999f11110478 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java @@ -27,6 +27,7 @@ import org.apache.doris.common.ThreadPoolManager; import org.apache.doris.common.security.authentication.AuthenticationConfig; import org.apache.doris.common.security.authentication.HadoopAuthenticator; +import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.CatalogProperty; import org.apache.doris.datasource.ExternalCatalog; @@ -34,6 +35,7 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.InitCatalogLog; import org.apache.doris.datasource.SessionContext; +import org.apache.doris.datasource.iceberg.IcebergMetadataOps; import org.apache.doris.datasource.iceberg.IcebergUtils; import org.apache.doris.datasource.jdbc.client.JdbcClientConfig; import org.apache.doris.datasource.operations.ExternalMetadataOperations; @@ -88,7 +90,7 @@ public class HMSExternalCatalog extends ExternalCatalog { private boolean enableHmsEventsIncrementalSync = false; //for "type" = "hms" , but is iceberg table. - private HiveCatalog icebergHiveCatalog; + private IcebergMetadataOps icebergMetadataOps; @VisibleForTesting public HMSExternalCatalog() { @@ -168,6 +170,7 @@ public void checkProperties() throws DdlException { @Override protected void initLocalObjectsImpl() { + preExecutionAuthenticator = new PreExecutionAuthenticator(); if (authenticator == null) { AuthenticationConfig config = AuthenticationConfig.getKerberosConfig(getConfiguration()); authenticator = HadoopAuthenticator.getHadoopAuthenticator(config); @@ -199,8 +202,6 @@ protected void initLocalObjectsImpl() { transactionManager = TransactionManagerFactory.createHiveTransactionManager(hiveOps, fileSystemProvider, fileSystemExecutor); metadataOps = hiveOps; - - icebergHiveCatalog = IcebergUtils.createIcebergHiveCatalog(this, getName()); } @Override @@ -337,10 +338,6 @@ public boolean isEnableHmsEventsIncrementalSync() { return enableHmsEventsIncrementalSync; } - public HiveCatalog getIcebergHiveCatalog() { - return icebergHiveCatalog; - } - /** * Enum for meta tables in hive catalog. * eg: tbl$partitions @@ -393,5 +390,14 @@ public TableValuedFunctionRef createFunctionRef(String ctlName, String dbName, S } } } + + public IcebergMetadataOps getIcebergMetadataOps() { + makeSureInitialized(); + if (icebergMetadataOps == null) { + HiveCatalog icebergHiveCatalog = IcebergUtils.createIcebergHiveCatalog(this, getName()); + icebergMetadataOps = ExternalMetadataOperations.newIcebergMetadataOps(this, icebergHiveCatalog); + } + return icebergMetadataOps; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index aacd9268ae35cfc..98984467d75b5c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -84,6 +84,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.stream.Collectors; @@ -748,7 +749,7 @@ public Set getPartitionColumnNames() { } @Override - public Map getAndCopyPartitionItems() { + public Map getAndCopyPartitionItems(OptionalLong snapshotId) { HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) getCatalog()); HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( @@ -763,8 +764,8 @@ public Map getAndCopyPartitionItems() { } @Override - public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context) - throws AnalysisException { + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + OptionalLong snapshotId) throws AnalysisException { HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) getCatalog()); HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( @@ -776,7 +777,8 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont } @Override - public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context) throws AnalysisException { + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) + throws AnalysisException { if (getPartitionType() == PartitionType.UNPARTITIONED) { return new MTMVMaxTimestampSnapshot(getName(), getLastDdlTime()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java index 5e76996bb120ee6..7981a0b4f261ff2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java @@ -79,7 +79,7 @@ public COWIncrementalRelation(Map optParams, Configuration confi if (!metaClient.getTableConfig().populateMetaFields()) { throw new HoodieException("Incremental queries are not supported when meta fields are disabled"); } - HoodieInstant lastInstant = commitTimeline.lastInstant().get(); + String startInstantTime = optParams.get("hoodie.datasource.read.begin.instanttime"); if (startInstantTime == null) { throw new HoodieException("Specify the begin instant time to pull from using " @@ -89,16 +89,18 @@ public COWIncrementalRelation(Map optParams, Configuration confi startInstantTime = "000"; } String endInstantTime = optParams.getOrDefault("hoodie.datasource.read.end.instanttime", - lastInstant.getTimestamp()); + hollowCommitHandling == HollowCommitHandling.USE_TRANSITION_TIME + ? commitTimeline.lastInstant().get().getStateTransitionTime() + : commitTimeline.lastInstant().get().getTimestamp()); startInstantArchived = commitTimeline.isBeforeTimelineStarts(startInstantTime); endInstantArchived = commitTimeline.isBeforeTimelineStarts(endInstantTime); HoodieTimeline commitsTimelineToReturn; if (hollowCommitHandling == HollowCommitHandling.USE_TRANSITION_TIME) { commitsTimelineToReturn = commitTimeline.findInstantsInRangeByStateTransitionTime(startInstantTime, - lastInstant.getStateTransitionTime()); + endInstantTime); } else { - commitsTimelineToReturn = commitTimeline.findInstantsInRange(startInstantTime, lastInstant.getTimestamp()); + commitsTimelineToReturn = commitTimeline.findInstantsInRange(startInstantTime, endInstantTime); } List commitsToReturn = commitsTimelineToReturn.getInstants(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java index d8dfd1c128f162e..0fa69825a01ef49 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java @@ -40,11 +40,10 @@ public abstract class IcebergExternalCatalog extends ExternalCatalog { public static final String ICEBERG_HADOOP = "hadoop"; public static final String ICEBERG_GLUE = "glue"; public static final String ICEBERG_DLF = "dlf"; + public static final String EXTERNAL_CATALOG_NAME = "external_catalog.name"; protected String icebergCatalogType; protected Catalog catalog; - protected PreExecutionAuthenticator preExecutionAuthenticator; - public IcebergExternalCatalog(long catalogId, String name, String comment) { super(catalogId, name, InitCatalogLog.Type.ICEBERG, comment); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java index c1ac2a79754b796..ad347ca78f2a4f3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java @@ -36,8 +36,6 @@ import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; import java.util.HashMap; @@ -104,18 +102,16 @@ private List loadSnapshots(IcebergMetadataCacheKey key) { @NotNull private Table loadTable(IcebergMetadataCacheKey key) { - Catalog icebergCatalog; + IcebergMetadataOps ops; if (key.catalog instanceof HMSExternalCatalog) { - icebergCatalog = ((HMSExternalCatalog) key.catalog).getIcebergHiveCatalog(); + ops = ((HMSExternalCatalog) key.catalog).getIcebergMetadataOps(); } else if (key.catalog instanceof IcebergExternalCatalog) { - icebergCatalog = ((IcebergExternalCatalog) key.catalog).getCatalog(); + ops = (IcebergMetadataOps) (((IcebergExternalCatalog) key.catalog).getMetadataOps()); } else { throw new RuntimeException("Only support 'hms' and 'iceberg' type for iceberg table"); } - Table icebergTable = HiveMetaStoreClientHelper.ugiDoAs(((ExternalCatalog) key.catalog).getConfiguration(), - () -> icebergCatalog.loadTable(TableIdentifier.of(key.dbName, key.tableName))); - initIcebergTableFileIO(icebergTable, key.catalog.getProperties()); - return icebergTable; + return HiveMetaStoreClientHelper.ugiDoAs(((ExternalCatalog) key.catalog).getConfiguration(), + () -> ops.loadTable(key.dbName, key.tableName)); } public void invalidateCatalogCache(long catalogId) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java index 59729ddb47a6a42..970814b7acdc85e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java @@ -36,6 +36,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; @@ -46,29 +47,40 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; public class IcebergMetadataOps implements ExternalMetadataOps { private static final Logger LOG = LogManager.getLogger(IcebergMetadataOps.class); protected Catalog catalog; - protected IcebergExternalCatalog dorisCatalog; + protected ExternalCatalog dorisCatalog; protected SupportsNamespaces nsCatalog; private PreExecutionAuthenticator preExecutionAuthenticator; + // Generally, there should be only two levels under the catalog, namely ., + // but the REST type catalog is obtained from an external server, + // and the level provided by the external server may be three levels, ..
. + // Therefore, if the external server provides a catalog, + // the catalog needs to be recorded here to ensure semantic consistency. + private Optional externalCatalogName = Optional.empty(); - public IcebergMetadataOps(IcebergExternalCatalog dorisCatalog, Catalog catalog) { + public IcebergMetadataOps(ExternalCatalog dorisCatalog, Catalog catalog) { this.dorisCatalog = dorisCatalog; this.catalog = catalog; nsCatalog = (SupportsNamespaces) catalog; - this.preExecutionAuthenticator = dorisCatalog.preExecutionAuthenticator; + this.preExecutionAuthenticator = dorisCatalog.getPreExecutionAuthenticator(); + if (dorisCatalog.getProperties().containsKey(IcebergExternalCatalog.EXTERNAL_CATALOG_NAME)) { + externalCatalogName = + Optional.of(dorisCatalog.getProperties().get(IcebergExternalCatalog.EXTERNAL_CATALOG_NAME)); + } } public Catalog getCatalog() { return catalog; } - public IcebergExternalCatalog getExternalCatalog() { + public ExternalCatalog getExternalCatalog() { return dorisCatalog; } @@ -78,17 +90,18 @@ public void close() { @Override public boolean tableExist(String dbName, String tblName) { - return catalog.tableExists(TableIdentifier.of(dbName, tblName)); + return catalog.tableExists(getTableIdentifier(dbName, tblName)); } public boolean databaseExist(String dbName) { - return nsCatalog.namespaceExists(Namespace.of(dbName)); + return nsCatalog.namespaceExists(getNamespace(dbName)); } public List listDatabaseNames() { try { - return preExecutionAuthenticator.execute(() -> nsCatalog.listNamespaces().stream() - .map(Namespace::toString) + return preExecutionAuthenticator.execute(() -> nsCatalog.listNamespaces(getNamespace()) + .stream() + .map(n -> n.level(n.length() - 1)) .collect(Collectors.toList())); } catch (Exception e) { throw new RuntimeException("Failed to list database names, error message is: " + e.getMessage()); @@ -98,7 +111,7 @@ public List listDatabaseNames() { @Override public List listTableNames(String dbName) { - List tableIdentifiers = catalog.listTables(Namespace.of(dbName)); + List tableIdentifiers = catalog.listTables(getNamespace(dbName)); return tableIdentifiers.stream().map(TableIdentifier::name).collect(Collectors.toList()); } @@ -128,12 +141,14 @@ private void performCreateDb(CreateDbStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, dbName); } } - String icebergCatalogType = dorisCatalog.getIcebergCatalogType(); - if (!properties.isEmpty() && !IcebergExternalCatalog.ICEBERG_HMS.equals(icebergCatalogType)) { - throw new DdlException( + if (!properties.isEmpty() && dorisCatalog instanceof IcebergExternalCatalog) { + String icebergCatalogType = ((IcebergExternalCatalog) dorisCatalog).getIcebergCatalogType(); + if (!IcebergExternalCatalog.ICEBERG_HMS.equals(icebergCatalogType)) { + throw new DdlException( "Not supported: create database with properties for iceberg catalog type: " + icebergCatalogType); + } } - nsCatalog.createNamespace(Namespace.of(dbName), properties); + nsCatalog.createNamespace(getNamespace(dbName), properties); dorisCatalog.onRefreshCache(true); } @@ -159,8 +174,7 @@ private void preformDropDb(DropDbStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_DB_DROP_EXISTS, dbName); } } - SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; - nsCatalog.dropNamespace(Namespace.of(dbName)); + nsCatalog.dropNamespace(getNamespace(dbName)); dorisCatalog.onRefreshCache(true); } @@ -200,7 +214,7 @@ public boolean performCreateTable(CreateTableStmt stmt) throws UserException { Map properties = stmt.getProperties(); properties.put(ExternalCatalog.DORIS_VERSION, ExternalCatalog.DORIS_VERSION_VALUE); PartitionSpec partitionSpec = IcebergUtils.solveIcebergPartitionSpec(stmt.getPartitionDesc(), schema); - catalog.createTable(TableIdentifier.of(dbName, tableName), schema, partitionSpec, properties); + catalog.createTable(getTableIdentifier(dbName, tableName), schema, partitionSpec, properties); db.setUnInitialized(true); return false; } @@ -238,7 +252,7 @@ private void performDropTable(DropTableStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, tableName, dbName); } } - catalog.dropTable(TableIdentifier.of(dbName, tableName), true); + catalog.dropTable(getTableIdentifier(dbName, tableName), true); db.setUnInitialized(true); } @@ -250,4 +264,25 @@ public void truncateTable(String dbName, String tblName, List partitions public PreExecutionAuthenticator getPreExecutionAuthenticator() { return preExecutionAuthenticator; } + + @Override + public Table loadTable(String dbName, String tblName) { + return catalog.loadTable(getTableIdentifier(dbName, tblName)); + } + + private TableIdentifier getTableIdentifier(String dbName, String tblName) { + return externalCatalogName + .map(s -> TableIdentifier.of(s, dbName, tblName)) + .orElseGet(() -> TableIdentifier.of(dbName, tblName)); + } + + private Namespace getNamespace(String dbName) { + return externalCatalogName + .map(s -> Namespace.of(s, dbName)) + .orElseGet(() -> Namespace.of(dbName)); + } + + private Namespace getNamespace() { + return externalCatalogName.map(Namespace::of).orElseGet(() -> Namespace.empty()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java index 908a4fa9e3f2710..b92d2c91f9630e4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java @@ -26,7 +26,6 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.aws.AwsClientProperties; -import org.apache.iceberg.aws.s3.S3FileIO; import org.apache.iceberg.aws.s3.S3FileIOProperties; import java.util.HashMap; @@ -71,7 +70,6 @@ private Map convertToRestCatalogProperties() { Map props = catalogProperty.getProperties(); Map restProperties = new HashMap<>(props); - restProperties.put(CatalogProperties.FILE_IO_IMPL, S3FileIO.class.getName()); restProperties.put(CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_TYPE_REST); String restUri = props.getOrDefault(CatalogProperties.URI, ""); restProperties.put(CatalogProperties.URI, restUri); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java index 685915025d665e3..d0cca11b0af2bb9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java @@ -22,6 +22,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.info.SimpleTableInfo; +import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.iceberg.helper.IcebergWriterHelper; import org.apache.doris.nereids.trees.plans.commands.insert.BaseExternalTableInsertCommandContext; import org.apache.doris.nereids.trees.plans.commands.insert.InsertCommandContext; @@ -140,7 +141,7 @@ public long getUpdateCnt() { private synchronized Table getNativeTable(SimpleTableInfo tableInfo) { Objects.requireNonNull(tableInfo); - IcebergExternalCatalog externalCatalog = ops.getExternalCatalog(); + ExternalCatalog externalCatalog = ops.getExternalCatalog(); return IcebergUtils.getRemoteTable(externalCatalog, tableInfo); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java index 4a2757f918f2946..50166fe83051131 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java @@ -17,9 +17,9 @@ package org.apache.doris.datasource.operations; +import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HiveMetadataOps; -import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergMetadataOps; import org.apache.doris.datasource.jdbc.client.JdbcClientConfig; @@ -34,7 +34,7 @@ public static HiveMetadataOps newHiveMetadataOps(HiveConf hiveConf, JdbcClientCo return new HiveMetadataOps(hiveConf, jdbcClientConfig, catalog); } - public static IcebergMetadataOps newIcebergMetadataOps(IcebergExternalCatalog dorisCatalog, Catalog catalog) { + public static IcebergMetadataOps newIcebergMetadataOps(ExternalCatalog dorisCatalog, Catalog catalog) { return new IcebergMetadataOps(dorisCatalog, catalog); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java index 0333124b35294c5..e5ed129c679ffe3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java @@ -91,6 +91,10 @@ public interface ExternalMetadataOps { boolean databaseExist(String dbName); + default Object loadTable(String dbName, String tblName) { + throw new UnsupportedOperationException("Load table is not supported."); + } + /** * close the connection, eg, to hms */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java index 5645c4e89e726c5..632a0da0ebd316e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java @@ -67,6 +67,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.stream.Collectors; @@ -312,7 +313,7 @@ public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { } @Override - public Map getAndCopyPartitionItems() { + public Map getAndCopyPartitionItems(OptionalLong snapshotId) { return Maps.newHashMap(getPartitionInfoFromCache().getNameToPartitionItem()); } @@ -333,7 +334,8 @@ public List getPartitionColumns() { } @Override - public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context) + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + OptionalLong snapshotId) throws AnalysisException { PaimonPartition paimonPartition = getPartitionInfoFromCache().getNameToPartition().get(partitionName); if (paimonPartition == null) { @@ -343,7 +345,8 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont } @Override - public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context) throws AnalysisException { + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) + throws AnalysisException { return new MTMVVersionSnapshot(getLatestSnapshotIdFromCache()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java index d1a425aeaf78382..c997ebcd30e7757 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.common.Status; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.common.util.Util; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.task.AbstractTask; import org.apache.doris.load.FailMsg; @@ -31,6 +32,7 @@ import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.QueryState; import org.apache.doris.qe.StmtExecutor; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; @@ -195,10 +197,13 @@ public void run() throws JobException { return; } command.runWithUpdateInfo(ctx, stmtExecutor, loadStatistic); + if (ctx.getState().getStateType() != QueryState.MysqlStateType.OK) { + throw new JobException(ctx.getState().getErrorMessage()); + } } catch (Exception e) { log.warn("execute insert task error, job id is {}, task id is {},sql is {}", getJobId(), getTaskId(), sql, e); - throw new JobException(e); + throw new JobException(Util.getRootCauseMessage(e)); } } @@ -237,15 +242,7 @@ public TRow getTvfInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal(jobName)); trow.addToColumnValue(new TCell().setStringVal(getJobId() + LABEL_SPLITTER + getTaskId())); trow.addToColumnValue(new TCell().setStringVal(jobInfo.getState().name())); - // err msg - String errorMsg = ""; - if (failMsg != null) { - errorMsg = failMsg.getMsg(); - } - if (StringUtils.isNotBlank(getErrMsg())) { - errorMsg = getErrMsg(); - } - trow.addToColumnValue(new TCell().setStringVal(errorMsg)); + trow.addToColumnValue(new TCell().setStringVal(getErrorMsg())); // create time trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getCreateTimeMs()))); trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? "" @@ -275,7 +272,7 @@ private TRow getPendingTaskTVFInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal(jobName)); trow.addToColumnValue(new TCell().setStringVal(getJobId() + LABEL_SPLITTER + getTaskId())); trow.addToColumnValue(new TCell().setStringVal(getStatus().name())); - trow.addToColumnValue(new TCell().setStringVal("")); + trow.addToColumnValue(new TCell().setStringVal(getErrorMsg())); trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getCreateTimeMs()))); trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? "" : TimeUtils.longToTimeString(getStartTimeMs()))); @@ -287,4 +284,15 @@ private TRow getPendingTaskTVFInfo(String jobName) { return trow; } + private String getErrorMsg() { + // err msg + String errorMsg = ""; + if (failMsg != null) { + errorMsg = failMsg.getMsg(); + } + if (StringUtils.isNotBlank(getErrMsg())) { + errorMsg = getErrMsg(); + } + return errorMsg; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java index 1cfb5e021a5309c..1bbc51fb004c573 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java @@ -50,6 +50,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; +import java.util.OptionalLong; import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -329,7 +330,7 @@ public static boolean isSyncWithPartitions(MTMVRefreshContext context, String mt } for (String relatedPartitionName : relatedPartitionNames) { MTMVSnapshotIf relatedPartitionCurrentSnapshot = relatedTable - .getPartitionSnapshot(relatedPartitionName, context); + .getPartitionSnapshot(relatedPartitionName, context, OptionalLong.empty()); if (!mtmv.getRefreshSnapshot() .equalsWithRelatedPartition(mtmvPartitionName, relatedPartitionName, relatedPartitionCurrentSnapshot)) { @@ -446,7 +447,7 @@ private static boolean isSyncWithBaseTable(MTMVRefreshContext context, String mt if (!baseTable.needAutoRefresh()) { return true; } - MTMVSnapshotIf baseTableCurrentSnapshot = baseTable.getTableSnapshot(context); + MTMVSnapshotIf baseTableCurrentSnapshot = baseTable.getTableSnapshot(context, OptionalLong.empty()); return mtmv.getRefreshSnapshot() .equalsWithBaseTable(mtmvPartitionName, new BaseTableInfo(baseTable), baseTableCurrentSnapshot); } @@ -482,7 +483,7 @@ private static MTMVRefreshPartitionSnapshot generatePartitionSnapshot(MTMVRefres MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); for (String relatedPartitionName : relatedPartitionNames) { MTMVSnapshotIf partitionSnapshot = relatedTable - .getPartitionSnapshot(relatedPartitionName, context); + .getPartitionSnapshot(relatedPartitionName, context, OptionalLong.empty()); refreshPartitionSnapshot.getPartitions() .put(relatedPartitionName, partitionSnapshot); } @@ -497,7 +498,7 @@ private static MTMVRefreshPartitionSnapshot generatePartitionSnapshot(MTMVRefres continue; } refreshPartitionSnapshot.addTableSnapshot(baseTableInfo, - ((MTMVRelatedTableIf) table).getTableSnapshot(context)); + ((MTMVRelatedTableIf) table).getTableSnapshot(context, OptionalLong.empty())); } return refreshPartitionSnapshot; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java index 13b58239376116e..ef3100dec4c7327 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java @@ -20,6 +20,7 @@ import org.apache.doris.common.AnalysisException; import java.util.Map; +import java.util.OptionalLong; /** * get all related partition descs @@ -29,6 +30,6 @@ public class MTMVRelatedPartitionDescInitGenerator implements MTMVRelatedPartiti @Override public void apply(MTMVPartitionInfo mvPartitionInfo, Map mvProperties, RelatedPartitionDescResult lastResult) throws AnalysisException { - lastResult.setItems(mvPartitionInfo.getRelatedTable().getAndCopyPartitionItems()); + lastResult.setItems(mvPartitionInfo.getRelatedTable().getAndCopyPartitionItems(OptionalLong.empty())); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java index 4a8b14603ce4d67..e18784ae253a0fe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Map; +import java.util.OptionalLong; import java.util.Set; /** @@ -38,9 +39,10 @@ public interface MTMVRelatedTableIf extends TableIf { * Note: This method is called every time there is a refresh and transparent rewrite, * so if this method is slow, it will significantly reduce query performance * + * @param snapshotId * @return partitionName->PartitionItem */ - Map getAndCopyPartitionItems() throws AnalysisException; + Map getAndCopyPartitionItems(OptionalLong snapshotId) throws AnalysisException; /** * getPartitionType LIST/RANGE/UNPARTITIONED @@ -70,12 +72,14 @@ public interface MTMVRelatedTableIf extends TableIf { * If snapshots have already been obtained in bulk in the context, * the results should be obtained directly from the context * + * @param snapshotId * @param partitionName * @param context * @return partition snapshot at current time * @throws AnalysisException */ - MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context) throws AnalysisException; + MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, OptionalLong snapshotId) + throws AnalysisException; /** * getTableSnapshot @@ -83,11 +87,12 @@ public interface MTMVRelatedTableIf extends TableIf { * If snapshots have already been obtained in bulk in the context, * the results should be obtained directly from the context * + * @param snapshotId * @param context * @return table snapshot at current time * @throws AnalysisException */ - MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context) throws AnalysisException; + MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) throws AnalysisException; /** * Does the current type of table allow timed triggering diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index bb28191d77f0280..e203c9428fb303b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -225,6 +225,7 @@ import org.apache.doris.nereids.DorisParser.ShowTableIdContext; import org.apache.doris.nereids.DorisParser.ShowTabletsBelongContext; import org.apache.doris.nereids.DorisParser.ShowTrashContext; +import org.apache.doris.nereids.DorisParser.ShowTriggersContext; import org.apache.doris.nereids.DorisParser.ShowVariablesContext; import org.apache.doris.nereids.DorisParser.ShowViewContext; import org.apache.doris.nereids.DorisParser.ShowWhitelistContext; @@ -489,6 +490,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTriggersCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowWhiteListCommand; @@ -4233,6 +4235,7 @@ public LogicalPlan visitShowBackends(ShowBackendsContext ctx) { return new ShowBackendsCommand(); } + @Override public LogicalPlan visitShowPlugins(ShowPluginsContext ctx) { return new ShowPluginsCommand(); } @@ -4246,6 +4249,11 @@ public LogicalPlan visitShowSqlBlockRule(ShowSqlBlockRuleContext ctx) { return new ShowSqlBlockRuleCommand(ruleName); } + @Override + public LogicalPlan visitShowTriggers(ShowTriggersContext ctx) { + return new ShowTriggersCommand(); + } + @Override public LogicalPlan visitShowRepositories(ShowRepositoriesContext ctx) { return new ShowRepositoriesCommand(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 2e5aa3cfd38622b..9a639d1a023cb33 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -203,6 +203,7 @@ public enum PlanType { SHOW_STORAGE_ENGINES_COMMAND, SHOW_TABLE_ID_COMMAND, SHOW_TRASH_COMMAND, + SHOW_TRIGGERS_COMMAND, SHOW_VARIABLES_COMMAND, SHOW_AUTHORS_COMMAND, SHOW_VIEW_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTriggersCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTriggersCommand.java new file mode 100644 index 000000000000000..dd9d2c18c577ee6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTriggersCommand.java @@ -0,0 +1,66 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * show triggers command + */ +public class ShowTriggersCommand extends ShowCommand { + private static final ShowResultSetMetaData META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("Trigger", ScalarType.createVarchar(64))) + .addColumn(new Column("Event", ScalarType.createVarchar(10))) + .addColumn(new Column("Table", ScalarType.createVarchar(80))) + .addColumn(new Column("Statement", ScalarType.createVarchar(64))) + .addColumn(new Column("Timing", ScalarType.createVarchar(80))) + .addColumn(new Column("Created", ScalarType.createVarchar(80))) + .addColumn(new Column("sql_mode", ScalarType.createVarchar(80))) + .addColumn(new Column("Definer", ScalarType.createVarchar(80))) + .addColumn(new Column("character_set_client", ScalarType.createVarchar(80))) + .addColumn(new Column("collation_connection", ScalarType.createVarchar(80))) + .addColumn(new Column("Database Collation", ScalarType.createVarchar(80))) + .build(); + + public ShowTriggersCommand() { + super(PlanType.SHOW_TRIGGERS_COMMAND); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowTriggersCommand(this, context); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + List> rowSet = Lists.newArrayList(); + return new ShowResultSet(META_DATA, rowSet); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java index eea1afc2844a8d5..eb8af115ac0c14b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java @@ -78,18 +78,16 @@ public void init(ConnectContext ctx) throws UserException { /**translateToLegacyStmt*/ public AlterViewStmt translateToLegacyStmt(ConnectContext ctx) { - List cols = Lists.newArrayList(); - for (SimpleColumnDefinition def : simpleColumnDefinitions) { - cols.add(def.translateToColWithComment()); - } - AlterViewStmt alterViewStmt = new AlterViewStmt(viewName.transferToTableName(), cols, - null); // expand star(*) in project list and replace table name with qualifier String rewrittenSql = rewriteSql(ctx.getStatementContext().getIndexInSqlToString(), querySql); - // rewrite project alias rewrittenSql = rewriteProjectsToUserDefineAlias(rewrittenSql); - + checkViewSql(rewrittenSql); + List cols = Lists.newArrayList(); + for (SimpleColumnDefinition def : simpleColumnDefinitions) { + cols.add(def.translateToColWithComment()); + } + AlterViewStmt alterViewStmt = new AlterViewStmt(viewName.transferToTableName(), cols, null); alterViewStmt.setInlineViewDef(rewrittenSql); alterViewStmt.setFinalColumns(finalCols); return alterViewStmt; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java index 9c4b8670bb6e868..8df0f83b0b641c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java @@ -389,4 +389,8 @@ public Void visitSlot(Slot slot, StatementContext ctx) { return null; } } + + protected void checkViewSql(String viewSql) { + new NereidsParser().parseSingle(viewSql); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java index c0260ea682fe8ee..d3cd4293f4160eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java @@ -73,18 +73,17 @@ public void init(ConnectContext ctx) throws UserException { /**translateToLegacyStmt*/ public CreateViewStmt translateToLegacyStmt(ConnectContext ctx) { + // expand star(*) in project list and replace table name with qualifier + String rewrittenSql = rewriteSql(ctx.getStatementContext().getIndexInSqlToString(), querySql); + // rewrite project alias + rewrittenSql = rewriteProjectsToUserDefineAlias(rewrittenSql); + checkViewSql(rewrittenSql); List cols = Lists.newArrayList(); for (SimpleColumnDefinition def : simpleColumnDefinitions) { cols.add(def.translateToColWithComment()); } CreateViewStmt createViewStmt = new CreateViewStmt(ifNotExists, orReplace, viewName.transferToTableName(), cols, comment, null); - // expand star(*) in project list and replace table name with qualifier - String rewrittenSql = rewriteSql(ctx.getStatementContext().getIndexInSqlToString(), querySql); - - // rewrite project alias - rewrittenSql = rewriteProjectsToUserDefineAlias(rewrittenSql); - createViewStmt.setInlineViewDef(rewrittenSql); createViewStmt.setFinalColumns(finalCols); return createViewStmt; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 9393ca3b170d444..a5cc568c14344a4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -85,6 +85,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTriggersCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowWhiteListCommand; @@ -338,6 +339,10 @@ default R visitShowPluginsCommand(ShowPluginsCommand showPluginsCommand, C conte return visitCommand(showPluginsCommand, context); } + default R visitShowTriggersCommand(ShowTriggersCommand showTriggersCommand, C context) { + return visitCommand(showTriggersCommand, context); + } + default R visitShowRepositoriesCommand(ShowRepositoriesCommand showRepositoriesCommand, C context) { return visitCommand(showRepositoriesCommand, context); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java index 67b44adc565534e..f14f5113d8feea0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java @@ -45,7 +45,7 @@ public void setUp() { @Test public void testNormal() throws UserException, AnalysisException { - DropDbStmt stmt = new DropDbStmt(false, new DbName("test", "test"), true); + DropDbStmt stmt = new DropDbStmt(false, new DbName("test", "test"), false); stmt.analyze(analyzer); Assert.assertEquals("test", stmt.getCtlName()); @@ -53,6 +53,16 @@ public void testNormal() throws UserException, AnalysisException { Assert.assertEquals("DROP DATABASE `test`", stmt.toString()); } + @Test + public void testForce() throws UserException, AnalysisException { + DropDbStmt stmt = new DropDbStmt(false, new DbName("test", "test"), true); + + stmt.analyze(analyzer); + Assert.assertEquals("test", stmt.getCtlName()); + Assert.assertEquals("test", stmt.getDbName()); + Assert.assertEquals("DROP DATABASE `test` FORCE", stmt.toString()); + } + @Test(expected = AnalysisException.class) public void testFailed() throws UserException, AnalysisException { DropDbStmt stmt = new DropDbStmt(false, new DbName("", ""), true); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java index da6d5b8d4c44d0a..437e54f58f20e6d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java @@ -72,12 +72,13 @@ public void testNormal() throws UserException, AnalysisException { stmt.analyze(analyzer); Assert.assertEquals("db1", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); - Assert.assertEquals("DROP TABLE `db1`.`table1`", stmt.toString()); + // one with force. + Assert.assertEquals("DROP TABLE `db1`.`table1` FORCE", stmt.toString()); } @Test public void testDefaultNormal() throws UserException, AnalysisException { - DropTableStmt stmt = new DropTableStmt(false, noDbTbl, true); + DropTableStmt stmt = new DropTableStmt(false, noDbTbl, false); stmt.analyze(analyzer); Assert.assertEquals("testDb", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java index 997385742dc09a1..96ac59b81216bca 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java @@ -35,6 +35,7 @@ import org.junit.Test; import java.util.List; +import java.util.OptionalLong; import java.util.Set; public class MTMVPartitionUtilTest { @@ -112,7 +113,7 @@ public void setUp() throws NoSuchMethodException, SecurityException, AnalysisExc minTimes = 0; result = true; - baseOlapTable.getTableSnapshot((MTMVRefreshContext) any); + baseOlapTable.getTableSnapshot((MTMVRefreshContext) any, (OptionalLong) any); minTimes = 0; result = baseSnapshotIf; @@ -132,7 +133,7 @@ public void setUp() throws NoSuchMethodException, SecurityException, AnalysisExc minTimes = 0; result = true; - baseOlapTable.getPartitionSnapshot(anyString, (MTMVRefreshContext) any); + baseOlapTable.getPartitionSnapshot(anyString, (MTMVRefreshContext) any, (OptionalLong) any); minTimes = 0; result = baseSnapshotIf; diff --git a/regression-test/conf/regression-conf.groovy b/regression-test/conf/regression-conf.groovy index 2ee288d489487d8..ab9bb0beb918697 100644 --- a/regression-test/conf/regression-conf.groovy +++ b/regression-test/conf/regression-conf.groovy @@ -205,6 +205,9 @@ extEsPort = 9200 extEsUser = "*******" extEsPassword = "***********" +enableExternalHudiTest = false +hudiEmrCatalog = "***********" + enableObjStorageTest=false enableMaxComputeTest=false aliYunAk="***********" diff --git a/regression-test/data/bloom_filter_p0/test_bloom_filter_drop_column.out b/regression-test/data/bloom_filter_p0/test_bloom_filter_drop_column.out index 2c6ca8d224b7282..14334dfb4b5c48e 100644 --- a/regression-test/data/bloom_filter_p0/test_bloom_filter_drop_column.out +++ b/regression-test/data/bloom_filter_p0/test_bloom_filter_drop_column.out @@ -1,6 +1,6 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -1 1 +1 1 1 -- !select -- 1 \N diff --git a/regression-test/data/ddl_p0/create_view_nereids/create_view_use_mv.out b/regression-test/data/ddl_p0/create_view_nereids/create_view_use_mv.out new file mode 100644 index 000000000000000..c4df7484022baae --- /dev/null +++ b/regression-test/data/ddl_p0/create_view_nereids/create_view_use_mv.out @@ -0,0 +1,15 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !create_view_from_mv -- +\N 99.50 99.50 99.50 1 \N \N \N \N 1 \N \N +1 208.70 109.20 99.50 3 1 3 1 1 3 \N \N +2 218.40 109.20 109.20 2 2 2 2 2 1 \N \N +3 298.50 99.50 99.50 3 3 6 3 3 2 \N \N +4 99.50 99.50 99.50 1 \N 4 4 4 1 \N \N + +-- !alter_view_from_mv -- +\N 99.50 99.50 99.50 1 \N \N \N \N 1 \N \N +1 208.70 109.20 99.50 3 1 3 1 1 3 \N \N +2 218.40 109.20 109.20 2 2 2 2 2 1 \N \N +3 298.50 99.50 99.50 3 3 6 3 3 2 \N \N +4 99.50 99.50 99.50 1 \N 4 4 4 1 \N \N + diff --git a/regression-test/data/ddl_p0/create_view_nereids/test_create_view_nereids.out b/regression-test/data/ddl_p0/create_view_nereids/test_create_view_nereids.out new file mode 100644 index 000000000000000..5bdfe66d5aa672e --- /dev/null +++ b/regression-test/data/ddl_p0/create_view_nereids/test_create_view_nereids.out @@ -0,0 +1,397 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !test_view_1 -- +1 [1, 2, 3] +2 [10, -2, 8] +3 [-1, 20, 0] + +-- !test_view_2 -- +1 [1, 2, 3] [1, 1, 1] +2 [10, -2, 8] [1, 0, 1] +3 [-1, 20, 0] [0, 1, 0] + +-- !test_view_3 -- +1 [1, 2, 3] [1, 2, 3] [1, 2, 3] +2 [10, -2, 8] [10, 8] [10, 8] +3 [-1, 20, 0] [20] [20] + +-- !test_view_4 -- +1 [1, 2, 3] [1, 2, 3] [1, 2, 3] +2 [10, -2, 8] [10, 8] [10, 8] +3 [-1, 20, 0] [20] [20] + +-- !test_view_5 -- +1 [1, 2, 3] [1, 1, 1] +2 [10, -2, 8] [1, 0, 1] +3 [-1, 20, 0] [0, 1, 0] + +-- !test_view_6 -- +v1 CREATE VIEW `v1` AS SELECT\n `internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`error_code`, \n 1, \n 'string', \n now(), \n dayofyear(`internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`op_time`), \n cast (`internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`source` AS BIGINT), \n min(`internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`timestamp`) OVER (\n ORDER BY \n `internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`op_time` DESC ROWS BETWEEN UNBOUNDED PRECEDING\n AND 1 FOLLOWING\n ), \n 1 > 2,\n 2 + 3,\n 1 IN (1, 2, 3, 4), \n `internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`remark` LIKE '%like', \n CASE WHEN `internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`remark` = 's' THEN 1 ELSE 2 END,\n TRUE | FALSE \n FROM \n `internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`; + +-- !test_with_as -- +1 1 2 +1 1 4 +1 3 6 +2 1 3 +2 1 4 +2 1 7 +2 3 5 +2 3 9 +2 4 2 +3 2 8 +3 5 \N +3 5 6 +3 5 6 +3 5 8 +4 5 6 +6 \N 6 +6 7 1 + +-- !test_with_as_sql -- +test_view_with_as CREATE VIEW `test_view_with_as` AS (\n with `t1` as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`), `t2` as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`), \n `t3` as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`) SELECT `t1`.`pk`, `t1`.`a`, `t1`.`b` FROM t1); utf8mb4 utf8mb4_0900_bin + +-- !test_union -- +1 1 2 +1 1 2 +1 1 4 +1 1 4 +1 3 6 +1 3 6 +2 1 3 +2 1 3 +2 1 4 +2 1 4 +2 1 7 +2 1 7 +2 3 5 +2 3 5 +2 3 9 +2 3 9 +2 4 2 +2 4 2 +3 2 8 +3 2 8 +3 5 \N +3 5 \N +3 5 6 +3 5 6 +3 5 6 +3 5 6 +3 5 8 +3 5 8 +4 5 6 +4 5 6 +6 \N 6 +6 \N 6 +6 7 1 +6 7 1 + +-- !test_union_sql -- +test_view_union CREATE VIEW `test_view_union` AS (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk` AS `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a` AS `c2`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` AS `c3` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view` Union all SELECT `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` FROM `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`); utf8mb4 utf8mb4_0900_bin + +-- !test_count_star -- +17 + +-- !test_count_star_sql -- +test_view_count_star CREATE VIEW `test_view_count_star` AS (select count(*) AS `c1` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view` having count(*) > 0); utf8mb4 utf8mb4_0900_bin + +-- !test_expression -- +\N \N 6 +2 4 2 +2 4 3 +2 4 4 +2 4 4 +2 4 7 +3 5 8 +4 6 5 +4 6 6 +4 6 9 +5 7 2 +6 8 \N +6 8 6 +6 8 6 +6 8 6 +6 8 8 +8 10 1 + +-- !test_expression_sql -- +test_view_expression CREATE VIEW `test_view_expression` AS (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`+1 AS `c1`, abs(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`)+2+1 AS `c2`, cast(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` as varchar(10)) AS `c3` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`); utf8mb4 utf8mb4_0900_bin + +-- !test_alias -- +\N \N 6 +2 4 2 +2 4 3 +2 4 4 +2 4 4 +2 4 7 +3 5 8 +4 6 5 +4 6 6 +4 6 9 +5 7 2 +6 8 \N +6 8 6 +6 8 6 +6 8 6 +6 8 8 +8 10 1 + +-- !test_alias_sql -- +test_view_alias CREATE VIEW `test_view_alias` AS (\n select `t`.`c8` AS `c1`, `t`.`c2` AS `c2`, `t`.`c1` AS `c3` from (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`+1 `c8`,abs(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`)+2+1 as `c2`, cast(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` as varchar(10)) as `c1` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`) `t`); utf8mb4 utf8mb4_0900_bin + +-- !test_star_except -- +\N 6 +1 2 +1 3 +1 4 +1 4 +1 7 +2 8 +3 5 +3 6 +3 9 +4 2 +5 \N +5 6 +5 6 +5 6 +5 8 +7 1 + +-- !test_star_except_sql -- +test_view_star_except CREATE VIEW `test_view_star_except` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin + +-- !test_create_view_from_view -- +1 1 2 +1 1 4 +1 3 6 +2 1 3 +2 1 4 +2 1 7 +2 3 5 +2 3 9 +2 4 2 +3 2 8 +3 5 \N +3 5 6 +3 5 6 +3 5 8 +4 5 6 +6 \N 6 +6 7 1 + +-- !test_create_view_from_view_sql -- +test_view_from_view CREATE VIEW `test_view_from_view` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`test_view_with_as`.`pk` AS `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`test_view_with_as`.`a` AS `c2`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`test_view_with_as`.`b` AS `c3` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`test_view_with_as`; utf8mb4 utf8mb4_0900_bin + +-- !test_backquote_in_view_define -- +\N 6 +1 2 +1 3 +1 4 +1 4 +1 7 +2 8 +3 5 +3 6 +3 9 +4 2 +5 \N +5 6 +5 6 +5 6 +5 8 +7 1 + +-- !test_backquote_in_view_define_sql -- +test_backquote_in_view_define CREATE VIEW `test_backquote_in_view_define` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a` AS `abc`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` AS `c2` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin + +-- !test_backquote_in_table_alias -- +\N 6 +1 2 +1 3 +1 4 +1 4 +1 7 +2 8 +3 5 +3 6 +3 9 +4 2 +5 \N +5 6 +5 6 +5 6 +5 8 +7 1 + +-- !test_backquote_in_table_alias_sql -- +test_backquote_in_table_alias CREATE VIEW `test_backquote_in_table_alias` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`ab``c`.`a` AS `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`ab``c`.`b` AS `c2` from (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`,`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`) `ab``c`; utf8mb4 utf8mb4_0900_bin + +-- !test_invalid_column_name_in_table -- +\N 6 +1 2 +1 3 +1 4 +1 4 +1 7 +2 8 +3 5 +3 6 +3 9 +4 2 +5 \N +5 6 +5 6 +5 6 +5 8 +7 1 + +-- !test_invalid_column_name_in_table_define_sql -- +test_invalid_column_name_in_table CREATE VIEW `test_invalid_column_name_in_table` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a` ,`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin + +-- !test_generate -- +1 10 A 30 +1 10 A 60 +2 20 B 30 +2 20 B 60 +3 30 C 30 +3 30 C 60 +4 40 D 30 +4 40 D 60 + +-- !test_generate_sql -- +test_view_generate CREATE VIEW `test_view_generate` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value2`, `t1`.`age` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1` lateral view EXPLODE(ARRAY(30,60)) `t1` as `age`; utf8mb4 utf8mb4_0900_bin + +-- !test_generate_with_column -- +1 10 A 0 +2 20 B 0 +2 20 B 1 +3 30 C 0 +3 30 C 1 +3 30 C 2 +4 40 D 0 +4 40 D 1 +4 40 D 2 +4 40 D 3 + +-- !test_generate_with_column_sql -- +test_view_generate_with_column CREATE VIEW `test_view_generate_with_column` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value2`, `t1`.`age` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1` lateral view EXPLODE_numbers(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) `t1` as `age`; utf8mb4 utf8mb4_0900_bin + +-- !test_col_alias -- +1 10 +2 20 +3 30 +4 40 + +-- !test_col_alias_sql -- +test_view_col_alias CREATE VIEW `test_view_col_alias` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id` as `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`; utf8mb4 utf8mb4_0900_bin + +-- !test_col_alias_with_specific_name -- +1 10 +2 20 +3 30 +4 40 + +-- !test_col_alias_with_specific_name_sql -- +test_view_col_alias_specific_name CREATE VIEW `test_view_col_alias_specific_name` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id` AS `col1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value1` AS `col2` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`; utf8mb4 utf8mb4_0900_bin + +-- !test_table_alias -- +1 10 +2 20 +3 30 +4 40 + +-- !test_table_alias_sql -- +test_view_table_alias CREATE VIEW `test_view_table_alias` AS select `t`.`c1`, `t`.`c2` from (\n select `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id` as `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1` limit 10) as `t`; utf8mb4 utf8mb4_0900_bin + +-- !test_join_table_alias -- +1 10 +2 20 +3 30 +4 40 + +-- !test_join_table_alias_sql -- +test_view_join_table_alias CREATE VIEW `test_view_join_table_alias` AS select `t`.`c1`, `t`.`c2` from (\n select `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id` as `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1` `t1` inner join `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2` `t2` on `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id`=`internal`.`regression_test_ddl_p0_create_view_nereids`.`t2`.`id` limit 10) as `t`; utf8mb4 utf8mb4_0900_bin + +-- !test_alias_udf -- +1****1 1 +2****2 2 +3****3 3 +4****4 4 + +-- !test_alias_udf_sql -- +test_view_alias_udf CREATE VIEW `test_view_alias_udf` AS (select `regression_test_ddl_p0_create_view_nereids`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) as `c1`,abs(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin + +-- !test_alias_with_db_udf -- +1****1 1 +2****2 2 +3****3 3 +4****4 4 + +-- !test_alias_with_db_udf_sql -- +test_view_alias_udf_with_db CREATE VIEW `test_view_alias_udf_with_db` AS (select `regression_test_ddl_p0_create_view_nereids`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) as `c1`,abs(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin + +-- !test_udf_sql -- +test_view_udf CREATE VIEW `test_view_udf` AS (select `regression_test_ddl_p0_create_view_nereids`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) as `c1`, `regression_test_ddl_p0_create_view_nereids`.`java_udf_create_view_test`('2011-01-01','2011-01-02'),\n `regression_test_ddl_p0_create_view_nereids`.`java_udf_create_view_test`('2011-01-01','2011-01-03') from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin + +-- !test_udf -- +1****1 false false +2****2 false false +3****3 false false +4****4 false false + +-- !test_with_as_with_columns -- +1 1 2 +1 1 4 +1 3 6 +2 1 3 +2 1 4 +2 1 7 +2 3 5 +2 3 9 +2 4 2 +3 2 8 +3 5 \N +3 5 6 +3 5 6 +3 5 8 +4 5 6 +6 \N 6 +6 7 1 + +-- !test_with_as_with_columns_sql -- +test_view_with_as_with_columns CREATE VIEW `test_view_with_as_with_columns` AS (\n with `t1`(`c1`,`c2`,`c3`) as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`), `t2` as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`), \n `t3` as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`) SELECT `t1`.`c1`, `t1`.`c2`, `t1`.`c3` FROM t1); utf8mb4 utf8mb4_0900_bin + +-- !test_having -- +2 2 +3 3 +4 2 +4 4 +5 5 +6 3 +7 7 +8 4 +9 3 +10 5 +14 7 +15 5 + +-- !test_having_sql -- +test_having CREATE VIEW `test_having` AS select sum(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`) over(partition by `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a` order by `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`) as `c1` , `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view` group by grouping sets((`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`),(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b`),(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`,`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`)) having `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`>1; utf8mb4 utf8mb4_0900_bin + +-- !complicated_view1 -- +1 100 1 +2 200 1 + +-- !complicated_view1_sql -- +test_view_complicated CREATE VIEW `test_view_complicated` AS SELECT `internal`.`regression_test_ddl_p0_create_view_nereids`.`t`.`id`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`t`.`value3`, `t`.`row_num` FROM (\n SELECT `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`tt`.`value3`, ROW_NUMBER() OVER (PARTITION BY `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id` ORDER BY `internal`.`regression_test_ddl_p0_create_view_nereids`.`tt`.`value3` DESC) as `row_num`\n FROM (SELECT `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id` FROM `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1` GROUP BY `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) `t1`\n FULL OUTER JOIN (SELECT `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2`.`value3`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2`.`id`, MAX(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2`.`value4`) FROM `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2` GROUP BY `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2`.`value3`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2`.`id`) `tt`\n ON `internal`.`regression_test_ddl_p0_create_view_nereids`.`tt`.`id` = `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id`\n ORDER BY `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id`\n ) `t`\n WHERE `internal`.`regression_test_ddl_p0_create_view_nereids`.`t`.`value3` < 280 AND (`internal`.`regression_test_ddl_p0_create_view_nereids`.`t`.`id` < 3 or `internal`.`regression_test_ddl_p0_create_view_nereids`.`t`.`id` >8); utf8mb4 utf8mb4_0900_bin + +-- !nullable -- +1 小区A 10 1 2024-09-01T09:00 2024-09-01T10:00 +2 小区B 11 1 2024-09-01T09:00 2024-09-01T10:00 +3 小区C \N 1 2024-09-01T09:00 2024-09-01T10:00 + +-- !nullable_view_with_cols -- +1 小区A 10 1 2024-09-01T09:00 2024-09-01T10:00 +2 小区B 11 1 2024-09-01T09:00 2024-09-01T10:00 +3 小区C \N 1 2024-09-01T09:00 2024-09-01T10:00 + diff --git a/regression-test/data/ddl_p0/test_create_view_nereids.out b/regression-test/data/ddl_p0/test_create_view_nereids.out deleted file mode 100644 index 1b2ba19d93761d6..000000000000000 --- a/regression-test/data/ddl_p0/test_create_view_nereids.out +++ /dev/null @@ -1,397 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !test_view_1 -- -1 [1, 2, 3] -2 [10, -2, 8] -3 [-1, 20, 0] - --- !test_view_2 -- -1 [1, 2, 3] [1, 1, 1] -2 [10, -2, 8] [1, 0, 1] -3 [-1, 20, 0] [0, 1, 0] - --- !test_view_3 -- -1 [1, 2, 3] [1, 2, 3] [1, 2, 3] -2 [10, -2, 8] [10, 8] [10, 8] -3 [-1, 20, 0] [20] [20] - --- !test_view_4 -- -1 [1, 2, 3] [1, 2, 3] [1, 2, 3] -2 [10, -2, 8] [10, 8] [10, 8] -3 [-1, 20, 0] [20] [20] - --- !test_view_5 -- -1 [1, 2, 3] [1, 1, 1] -2 [10, -2, 8] [1, 0, 1] -3 [-1, 20, 0] [0, 1, 0] - --- !test_view_6 -- -v1 CREATE VIEW `v1` AS SELECT\n `internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`error_code`, \n 1, \n 'string', \n now(), \n dayofyear(`internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`op_time`), \n cast (`internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`source` AS BIGINT), \n min(`internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`timestamp`) OVER (\n ORDER BY \n `internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`op_time` DESC ROWS BETWEEN UNBOUNDED PRECEDING\n AND 1 FOLLOWING\n ), \n 1 > 2,\n 2 + 3,\n 1 IN (1, 2, 3, 4), \n `internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`remark` LIKE '%like', \n CASE WHEN `internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`remark` = 's' THEN 1 ELSE 2 END,\n TRUE | FALSE \n FROM \n `internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`; - --- !test_with_as -- -1 1 2 -1 1 4 -1 3 6 -2 1 3 -2 1 4 -2 1 7 -2 3 5 -2 3 9 -2 4 2 -3 2 8 -3 5 \N -3 5 6 -3 5 6 -3 5 8 -4 5 6 -6 \N 6 -6 7 1 - --- !test_with_as_sql -- -test_view_with_as CREATE VIEW `test_view_with_as` AS (\n with `t1` as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`), `t2` as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`), \n `t3` as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`) SELECT `t1`.`pk`, `t1`.`a`, `t1`.`b` FROM t1); utf8mb4 utf8mb4_0900_bin - --- !test_union -- -1 1 2 -1 1 2 -1 1 4 -1 1 4 -1 3 6 -1 3 6 -2 1 3 -2 1 3 -2 1 4 -2 1 4 -2 1 7 -2 1 7 -2 3 5 -2 3 5 -2 3 9 -2 3 9 -2 4 2 -2 4 2 -3 2 8 -3 2 8 -3 5 \N -3 5 \N -3 5 6 -3 5 6 -3 5 6 -3 5 6 -3 5 8 -3 5 8 -4 5 6 -4 5 6 -6 \N 6 -6 \N 6 -6 7 1 -6 7 1 - --- !test_union_sql -- -test_view_union CREATE VIEW `test_view_union` AS (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk` AS `c1`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a` AS `c2`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` AS `c3` from `internal`.`regression_test_ddl_p0`.`mal_test_view` Union all SELECT `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` FROM `internal`.`regression_test_ddl_p0`.`mal_test_view`); utf8mb4 utf8mb4_0900_bin - --- !test_count_star -- -17 - --- !test_count_star_sql -- -test_view_count_star CREATE VIEW `test_view_count_star` AS (select count(*) AS `c1` from `internal`.`regression_test_ddl_p0`.`mal_test_view` having count(*) > 0); utf8mb4 utf8mb4_0900_bin - --- !test_expression -- -\N \N 6 -2 4 2 -2 4 3 -2 4 4 -2 4 4 -2 4 7 -3 5 8 -4 6 5 -4 6 6 -4 6 9 -5 7 2 -6 8 \N -6 8 6 -6 8 6 -6 8 6 -6 8 8 -8 10 1 - --- !test_expression_sql -- -test_view_expression CREATE VIEW `test_view_expression` AS (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`+1 AS `c1`, abs(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`)+2+1 AS `c2`, cast(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` as varchar(10)) AS `c3` from `internal`.`regression_test_ddl_p0`.`mal_test_view`); utf8mb4 utf8mb4_0900_bin - --- !test_alias -- -\N \N 6 -2 4 2 -2 4 3 -2 4 4 -2 4 4 -2 4 7 -3 5 8 -4 6 5 -4 6 6 -4 6 9 -5 7 2 -6 8 \N -6 8 6 -6 8 6 -6 8 6 -6 8 8 -8 10 1 - --- !test_alias_sql -- -test_view_alias CREATE VIEW `test_view_alias` AS (\n select `t`.`c8` AS `c1`, `t`.`c2` AS `c2`, `t`.`c1` AS `c3` from (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`+1 `c8`,abs(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`)+2+1 as `c2`, cast(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` as varchar(10)) as `c1` from `internal`.`regression_test_ddl_p0`.`mal_test_view`) `t`); utf8mb4 utf8mb4_0900_bin - --- !test_star_except -- -\N 6 -1 2 -1 3 -1 4 -1 4 -1 7 -2 8 -3 5 -3 6 -3 9 -4 2 -5 \N -5 6 -5 6 -5 6 -5 8 -7 1 - --- !test_star_except_sql -- -test_view_star_except CREATE VIEW `test_view_star_except` AS select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin - --- !test_create_view_from_view -- -1 1 2 -1 1 4 -1 3 6 -2 1 3 -2 1 4 -2 1 7 -2 3 5 -2 3 9 -2 4 2 -3 2 8 -3 5 \N -3 5 6 -3 5 6 -3 5 8 -4 5 6 -6 \N 6 -6 7 1 - --- !test_create_view_from_view_sql -- -test_view_from_view CREATE VIEW `test_view_from_view` AS select `internal`.`regression_test_ddl_p0`.`test_view_with_as`.`pk` AS `c1`, `internal`.`regression_test_ddl_p0`.`test_view_with_as`.`a` AS `c2`, `internal`.`regression_test_ddl_p0`.`test_view_with_as`.`b` AS `c3` from `internal`.`regression_test_ddl_p0`.`test_view_with_as`; utf8mb4 utf8mb4_0900_bin - --- !test_backquote_in_view_define -- -\N 6 -1 2 -1 3 -1 4 -1 4 -1 7 -2 8 -3 5 -3 6 -3 9 -4 2 -5 \N -5 6 -5 6 -5 6 -5 8 -7 1 - --- !test_backquote_in_view_define_sql -- -test_backquote_in_view_define CREATE VIEW `test_backquote_in_view_define` AS select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a` AS `abc`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` AS `c2` from `internal`.`regression_test_ddl_p0`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin - --- !test_backquote_in_table_alias -- -\N 6 -1 2 -1 3 -1 4 -1 4 -1 7 -2 8 -3 5 -3 6 -3 9 -4 2 -5 \N -5 6 -5 6 -5 6 -5 8 -7 1 - --- !test_backquote_in_table_alias_sql -- -test_backquote_in_table_alias CREATE VIEW `test_backquote_in_table_alias` AS select `internal`.`regression_test_ddl_p0`.`ab``c`.`a` AS `c1`, `internal`.`regression_test_ddl_p0`.`ab``c`.`b` AS `c2` from (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`,`internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`) `ab``c`; utf8mb4 utf8mb4_0900_bin - --- !test_invalid_column_name_in_table -- -\N 6 -1 2 -1 3 -1 4 -1 4 -1 7 -2 8 -3 5 -3 6 -3 9 -4 2 -5 \N -5 6 -5 6 -5 6 -5 8 -7 1 - --- !test_invalid_column_name_in_table_define_sql -- -test_invalid_column_name_in_table CREATE VIEW `test_invalid_column_name_in_table` AS select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a` ,`internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin - --- !test_generate -- -1 10 A 30 -1 10 A 60 -2 20 B 30 -2 20 B 60 -3 30 C 30 -3 30 C 60 -4 40 D 30 -4 40 D 60 - --- !test_generate_sql -- -test_view_generate CREATE VIEW `test_view_generate` AS select `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value1`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value2`, `t1`.`age` from `internal`.`regression_test_ddl_p0`.`create_view_table1` lateral view EXPLODE(ARRAY(30,60)) `t1` as `age`; utf8mb4 utf8mb4_0900_bin - --- !test_generate_with_column -- -1 10 A 0 -2 20 B 0 -2 20 B 1 -3 30 C 0 -3 30 C 1 -3 30 C 2 -4 40 D 0 -4 40 D 1 -4 40 D 2 -4 40 D 3 - --- !test_generate_with_column_sql -- -test_view_generate_with_column CREATE VIEW `test_view_generate_with_column` AS select `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value1`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value2`, `t1`.`age` from `internal`.`regression_test_ddl_p0`.`create_view_table1` lateral view EXPLODE_numbers(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) `t1` as `age`; utf8mb4 utf8mb4_0900_bin - --- !test_col_alias -- -1 10 -2 20 -3 30 -4 40 - --- !test_col_alias_sql -- -test_view_col_alias CREATE VIEW `test_view_col_alias` AS select `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id` as `c1`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0`.`create_view_table1`; utf8mb4 utf8mb4_0900_bin - --- !test_col_alias_with_specific_name -- -1 10 -2 20 -3 30 -4 40 - --- !test_col_alias_with_specific_name_sql -- -test_view_col_alias_specific_name CREATE VIEW `test_view_col_alias_specific_name` AS select `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id` AS `col1`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value1` AS `col2` from `internal`.`regression_test_ddl_p0`.`create_view_table1`; utf8mb4 utf8mb4_0900_bin - --- !test_table_alias -- -1 10 -2 20 -3 30 -4 40 - --- !test_table_alias_sql -- -test_view_table_alias CREATE VIEW `test_view_table_alias` AS select `t`.`c1`, `t`.`c2` from (\n select `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id` as `c1`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0`.`create_view_table1` limit 10) as `t`; utf8mb4 utf8mb4_0900_bin - --- !test_join_table_alias -- -1 10 -2 20 -3 30 -4 40 - --- !test_join_table_alias_sql -- -test_view_join_table_alias CREATE VIEW `test_view_join_table_alias` AS select `t`.`c1`, `t`.`c2` from (\n select `internal`.`regression_test_ddl_p0`.`t1`.`id` as `c1`, `internal`.`regression_test_ddl_p0`.`t1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0`.`create_view_table1` `t1` inner join `internal`.`regression_test_ddl_p0`.`create_view_table2` `t2` on `internal`.`regression_test_ddl_p0`.`t1`.`id`=`internal`.`regression_test_ddl_p0`.`t2`.`id` limit 10) as `t`; utf8mb4 utf8mb4_0900_bin - --- !test_alias_udf -- -1****1 1 -2****2 2 -3****3 3 -4****4 4 - --- !test_alias_udf_sql -- -test_view_alias_udf CREATE VIEW `test_view_alias_udf` AS (select `regression_test_ddl_p0`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) as `c1`,abs(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) from `internal`.`regression_test_ddl_p0`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin - --- !test_alias_with_db_udf -- -1****1 1 -2****2 2 -3****3 3 -4****4 4 - --- !test_alias_with_db_udf_sql -- -test_view_alias_udf_with_db CREATE VIEW `test_view_alias_udf_with_db` AS (select `regression_test_ddl_p0`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) as `c1`,abs(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) from `internal`.`regression_test_ddl_p0`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin - --- !test_udf_sql -- -test_view_udf CREATE VIEW `test_view_udf` AS (select `regression_test_ddl_p0`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) as `c1`, `regression_test_ddl_p0`.`java_udf_create_view_test`('2011-01-01','2011-01-02'),\n `regression_test_ddl_p0`.`java_udf_create_view_test`('2011-01-01','2011-01-03') from `internal`.`regression_test_ddl_p0`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin - --- !test_udf -- -1****1 false false -2****2 false false -3****3 false false -4****4 false false - --- !test_with_as_with_columns -- -1 1 2 -1 1 4 -1 3 6 -2 1 3 -2 1 4 -2 1 7 -2 3 5 -2 3 9 -2 4 2 -3 2 8 -3 5 \N -3 5 6 -3 5 6 -3 5 8 -4 5 6 -6 \N 6 -6 7 1 - --- !test_with_as_with_columns_sql -- -test_view_with_as_with_columns CREATE VIEW `test_view_with_as_with_columns` AS (\n with `t1`(`c1`,`c2`,`c3`) as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`), `t2` as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`), \n `t3` as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`) SELECT `t1`.`c1`, `t1`.`c2`, `t1`.`c3` FROM t1); utf8mb4 utf8mb4_0900_bin - --- !test_having -- -2 2 -3 3 -4 2 -4 4 -5 5 -6 3 -7 7 -8 4 -9 3 -10 5 -14 7 -15 5 - --- !test_having_sql -- -test_having CREATE VIEW `test_having` AS select sum(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`) over(partition by `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a` order by `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`) as `c1` , `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a` from `internal`.`regression_test_ddl_p0`.`mal_test_view` group by grouping sets((`internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`),(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`b`),(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`,`internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`)) having `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`>1; utf8mb4 utf8mb4_0900_bin - --- !complicated_view1 -- -1 100 1 -2 200 1 - --- !complicated_view1_sql -- -test_view_complicated CREATE VIEW `test_view_complicated` AS SELECT `internal`.`regression_test_ddl_p0`.`t`.`id`, `internal`.`regression_test_ddl_p0`.`t`.`value3`, `t`.`row_num` FROM (\n SELECT `internal`.`regression_test_ddl_p0`.`t1`.`id`, `internal`.`regression_test_ddl_p0`.`tt`.`value3`, ROW_NUMBER() OVER (PARTITION BY `internal`.`regression_test_ddl_p0`.`t1`.`id` ORDER BY `internal`.`regression_test_ddl_p0`.`tt`.`value3` DESC) as `row_num`\n FROM (SELECT `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id` FROM `internal`.`regression_test_ddl_p0`.`create_view_table1` GROUP BY `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) `t1`\n FULL OUTER JOIN (SELECT `internal`.`regression_test_ddl_p0`.`create_view_table2`.`value3`, `internal`.`regression_test_ddl_p0`.`create_view_table2`.`id`, MAX(`internal`.`regression_test_ddl_p0`.`create_view_table2`.`value4`) FROM `internal`.`regression_test_ddl_p0`.`create_view_table2` GROUP BY `internal`.`regression_test_ddl_p0`.`create_view_table2`.`value3`, `internal`.`regression_test_ddl_p0`.`create_view_table2`.`id`) `tt`\n ON `internal`.`regression_test_ddl_p0`.`tt`.`id` = `internal`.`regression_test_ddl_p0`.`t1`.`id`\n ORDER BY `internal`.`regression_test_ddl_p0`.`t1`.`id`\n ) `t`\n WHERE `internal`.`regression_test_ddl_p0`.`t`.`value3` < 280 AND (`internal`.`regression_test_ddl_p0`.`t`.`id` < 3 or `internal`.`regression_test_ddl_p0`.`t`.`id` >8); utf8mb4 utf8mb4_0900_bin - --- !nullable -- -1 小区A 10 1 2024-09-01T09:00 2024-09-01T10:00 -2 小区B 11 1 2024-09-01T09:00 2024-09-01T10:00 -3 小区C \N 1 2024-09-01T09:00 2024-09-01T10:00 - --- !nullable_view_with_cols -- -1 小区A 10 1 2024-09-01T09:00 2024-09-01T10:00 -2 小区B 11 1 2024-09-01T09:00 2024-09-01T10:00 -3 小区C \N 1 2024-09-01T09:00 2024-09-01T10:00 - diff --git a/regression-test/data/external_table_p0/hive/test_hive_orc.out b/regression-test/data/external_table_p0/hive/test_hive_orc.out index 066c5d4b4d3b5f9..b34f276020c7cc1 100644 --- a/regression-test/data/external_table_p0/hive/test_hive_orc.out +++ b/regression-test/data/external_table_p0/hive/test_hive_orc.out @@ -110,6 +110,30 @@ tablets tinyint_col 179 182 182 187 183 181 177 183 177 187 183 202 202 186 528 -- !string_col_dict_plain_mixed3 -- 10240 +-- !predicate_pushdown1 -- +55 + +-- !predicate_pushdown2 -- +228 + +-- !predicate_pushdown3 -- +53 + +-- !predicate_pushdown4 -- +50000 + +-- !predicate_pushdown5 -- +90425 + +-- !predicate_pushdown6 -- +279428 + +-- !predicate_pushdown7 -- +300343 + +-- !predicate_pushdown8 -- +1533 + -- !select_top50 -- 4 55 999742610 400899305488827731 false 6.5976813E8 7.8723304616937395E17 \N base tennis pit vertical friday 2022-08-19T07:29:58 \N tablets smallint_col 2019-02-07 [7.53124931825377e+17] ["NbSSBtwzpxNSkkwga"] tablets smallint_col 2 49 999613702 105493714032727452 \N 6.3322381E8 9.8642324410240179E17 Unveil bright recruit participate. Suspect impression camera mathematical revelation. Fault live2 elbow debt west hydrogen current. how literary 2022-09-03T17:20:21 481707.1065 tablets boolean_col 2020-01-12 [] ["HoMrAnn", "wteEFvIwoZsVpVQdscMb", null, "zcGFmv", "kGEBBckbMtX", "hrEtCGFdPWZK"] tablets boolean_col @@ -273,6 +297,30 @@ tablets tinyint_col 179 182 182 187 183 181 177 183 177 187 183 202 202 186 528 -- !string_col_dict_plain_mixed3 -- 10240 +-- !predicate_pushdown1 -- +55 + +-- !predicate_pushdown2 -- +228 + +-- !predicate_pushdown3 -- +53 + +-- !predicate_pushdown4 -- +50000 + +-- !predicate_pushdown5 -- +90425 + +-- !predicate_pushdown6 -- +279428 + +-- !predicate_pushdown7 -- +300343 + +-- !predicate_pushdown8 -- +1533 + -- !select_top50 -- 4 55 999742610 400899305488827731 false 6.5976813E8 7.8723304616937395E17 \N base tennis pit vertical friday 2022-08-19T07:29:58 \N tablets smallint_col 2019-02-07 [7.53124931825377e+17] ["NbSSBtwzpxNSkkwga"] tablets smallint_col 2 49 999613702 105493714032727452 \N 6.3322381E8 9.8642324410240179E17 Unveil bright recruit participate. Suspect impression camera mathematical revelation. Fault live2 elbow debt west hydrogen current. how literary 2022-09-03T17:20:21 481707.1065 tablets boolean_col 2020-01-12 [] ["HoMrAnn", "wteEFvIwoZsVpVQdscMb", null, "zcGFmv", "kGEBBckbMtX", "hrEtCGFdPWZK"] tablets boolean_col diff --git a/regression-test/data/external_table_p0/hive/test_hive_use_meta_cache.out b/regression-test/data/external_table_p0/hive/test_hive_use_meta_cache.out index a5fc7ede82b8bec..4d54c601c876a24 100644 --- a/regression-test/data/external_table_p0/hive/test_hive_use_meta_cache.out +++ b/regression-test/data/external_table_p0/hive/test_hive_use_meta_cache.out @@ -28,15 +28,20 @@ test_use_meta_cache_db_hive test_use_meta_cache_tbl_hive -- !sql08 -- -test_use_meta_cache_tbl_hive -- !sql09 -- +p1=part1 +p1=part2 -- !sql10 -- -test_use_meta_cache_db_hive -- !sql11 -- +-- !sql12 -- +test_use_meta_cache_db_hive + +-- !sql13 -- + -- !sql01 -- -- !sql02 -- @@ -61,17 +66,114 @@ test_use_meta_cache_db_hive -- !sql05 -- -- !sql06 -- +test_use_meta_cache_tbl_hive -- !sql07 -- test_use_meta_cache_tbl_hive -- !sql08 -- + +-- !sql09 -- +p1=part1 +p1=part2 + +-- !sql10 -- +test_use_meta_cache_partitioned_tbl_hive +test_use_meta_cache_tbl_hive + +-- !sql11 -- + +-- !sql12 -- +test_use_meta_cache_db_hive + +-- !sql13 -- + +-- !sql01 -- + +-- !sql02 -- +test_use_meta_cache_db + +-- !sql03 -- +test_use_meta_cache_tbl + +-- !sql04 -- + +-- !sql05 -- + +-- !sql01 -- + +-- !sql02 -- + +-- !sql03 -- +test_use_meta_cache_db_hive + +-- !sql04 -- + +-- !sql05 -- + +-- !sql06 -- + +-- !sql07 -- test_use_meta_cache_tbl_hive +-- !sql08 -- + -- !sql09 -- +p1=part1 +p1=part2 -- !sql10 -- + +-- !sql11 -- + +-- !sql12 -- +test_use_meta_cache_db_hive + +-- !sql13 -- + +-- !sql01 -- + +-- !sql02 -- +test_use_meta_cache_db + +-- !sql03 -- +test_use_meta_cache_tbl + +-- !sql04 -- + +-- !sql05 -- + +-- !sql01 -- + +-- !sql02 -- + +-- !sql03 -- test_use_meta_cache_db_hive +-- !sql04 -- + +-- !sql05 -- + +-- !sql06 -- +test_use_meta_cache_tbl_hive + +-- !sql07 -- +test_use_meta_cache_tbl_hive + +-- !sql08 -- + +-- !sql09 -- +p1=part1 +p1=part2 + +-- !sql10 -- +test_use_meta_cache_partitioned_tbl_hive +test_use_meta_cache_tbl_hive + -- !sql11 -- +-- !sql12 -- +test_use_meta_cache_db_hive + +-- !sql13 -- + diff --git a/regression-test/data/external_table_p0/iceberg/iceberg_read_unitycatalog_table.out b/regression-test/data/external_table_p0/iceberg/iceberg_read_unitycatalog_table.out new file mode 100644 index 000000000000000..42414c3654930c5 --- /dev/null +++ b/regression-test/data/external_table_p0/iceberg/iceberg_read_unitycatalog_table.out @@ -0,0 +1,40 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q1 -- +1 nWYHawtqUw 930 +2 uvOzzthsLV 166 +3 WIAehuXWkv 170 +4 wYCSvnJKTo 709 +5 VsslXsUIDZ 993 +6 ZLsACYYTFy 813 +7 BtDDvLeBpK 52 +8 YISVtrPfGr 8 +9 PBPJHDFjjC 45 +10 qbDuUJzJMO 756 +11 EjqqWoaLJn 712 +12 jpZLMdKXpn 847 +13 acpjQXpJCp 649 +14 nOKqHhRwao 133 +15 kxUUZEUoKv 398 + +-- !q2 -- +7 +8 +9 +10 +11 +12 +13 +14 +15 + +-- !q3 -- +nWYHawtqUw 930 +wYCSvnJKTo 709 +VsslXsUIDZ 993 +ZLsACYYTFy 813 +qbDuUJzJMO 756 +EjqqWoaLJn 712 +jpZLMdKXpn 847 +acpjQXpJCp 649 +kxUUZEUoKv 398 + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_incremental.out b/regression-test/data/external_table_p2/hudi/test_hudi_incremental.out new file mode 100644 index 000000000000000..b1bdad85013bfce --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/test_hudi_incremental.out @@ -0,0 +1,349 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !incremental_1_end -- +9000 + +-- !incremental_earliest_1 -- +1000 + +-- !incremental_2_end -- +8000 + +-- !incremental_earliest_2 -- +2000 + +-- !incremental_1_2 -- +1000 + +-- !incremental_3_end -- +7000 + +-- !incremental_earliest_3 -- +3000 + +-- !incremental_2_3 -- +1000 + +-- !incremental_4_end -- +6000 + +-- !incremental_earliest_4 -- +4000 + +-- !incremental_3_4 -- +1000 + +-- !incremental_5_end -- +5000 + +-- !incremental_earliest_5 -- +5000 + +-- !incremental_4_5 -- +1000 + +-- !incremental_6_end -- +4000 + +-- !incremental_earliest_6 -- +6000 + +-- !incremental_5_6 -- +1000 + +-- !incremental_7_end -- +3000 + +-- !incremental_earliest_7 -- +7000 + +-- !incremental_6_7 -- +1000 + +-- !incremental_8_end -- +2000 + +-- !incremental_earliest_8 -- +8000 + +-- !incremental_7_8 -- +1000 + +-- !incremental_9_end -- +1000 + +-- !incremental_earliest_9 -- +9000 + +-- !incremental_8_9 -- +1000 + +-- !incremental_10_end -- +0 + +-- !incremental_earliest_10 -- +10000 + +-- !incremental_9_10 -- +1000 + +-- !incremental_1_end -- +9000 + +-- !incremental_earliest_1 -- +1000 + +-- !incremental_2_end -- +8000 + +-- !incremental_earliest_2 -- +2000 + +-- !incremental_1_2 -- +1000 + +-- !incremental_3_end -- +7000 + +-- !incremental_earliest_3 -- +3000 + +-- !incremental_2_3 -- +1000 + +-- !incremental_4_end -- +6000 + +-- !incremental_earliest_4 -- +4000 + +-- !incremental_3_4 -- +1000 + +-- !incremental_5_end -- +5000 + +-- !incremental_earliest_5 -- +5000 + +-- !incremental_4_5 -- +1000 + +-- !incremental_6_end -- +4000 + +-- !incremental_earliest_6 -- +6000 + +-- !incremental_5_6 -- +1000 + +-- !incremental_7_end -- +3000 + +-- !incremental_earliest_7 -- +7000 + +-- !incremental_6_7 -- +1000 + +-- !incremental_8_end -- +2000 + +-- !incremental_earliest_8 -- +8000 + +-- !incremental_7_8 -- +1000 + +-- !incremental_9_end -- +1000 + +-- !incremental_earliest_9 -- +9000 + +-- !incremental_8_9 -- +1000 + +-- !incremental_10_end -- +0 + +-- !incremental_earliest_10 -- +10000 + +-- !incremental_9_10 -- +1000 + +-- !incremental_1_end -- +9000 + +-- !incremental_earliest_1 -- +1000 + +-- !incremental_2_end -- +8000 + +-- !incremental_earliest_2 -- +2000 + +-- !incremental_1_2 -- +1000 + +-- !incremental_3_end -- +7000 + +-- !incremental_earliest_3 -- +3000 + +-- !incremental_2_3 -- +1000 + +-- !incremental_4_end -- +6000 + +-- !incremental_earliest_4 -- +4000 + +-- !incremental_3_4 -- +1000 + +-- !incremental_5_end -- +5000 + +-- !incremental_earliest_5 -- +5000 + +-- !incremental_4_5 -- +1000 + +-- !incremental_6_end -- +4000 + +-- !incremental_earliest_6 -- +6000 + +-- !incremental_5_6 -- +1000 + +-- !incremental_7_end -- +3000 + +-- !incremental_earliest_7 -- +7000 + +-- !incremental_6_7 -- +1000 + +-- !incremental_8_end -- +2000 + +-- !incremental_earliest_8 -- +8000 + +-- !incremental_7_8 -- +1000 + +-- !incremental_9_end -- +1000 + +-- !incremental_earliest_9 -- +9000 + +-- !incremental_8_9 -- +1000 + +-- !incremental_10_end -- +0 + +-- !incremental_earliest_10 -- +10000 + +-- !incremental_9_10 -- +1000 + +-- !incremental_1_end -- +9000 + +-- !incremental_earliest_1 -- +1000 + +-- !incremental_2_end -- +8000 + +-- !incremental_earliest_2 -- +2000 + +-- !incremental_1_2 -- +1000 + +-- !incremental_3_end -- +7000 + +-- !incremental_earliest_3 -- +3000 + +-- !incremental_2_3 -- +1000 + +-- !incremental_4_end -- +6000 + +-- !incremental_earliest_4 -- +4000 + +-- !incremental_3_4 -- +1000 + +-- !incremental_5_end -- +5000 + +-- !incremental_earliest_5 -- +5000 + +-- !incremental_4_5 -- +1000 + +-- !incremental_6_end -- +4000 + +-- !incremental_earliest_6 -- +6000 + +-- !incremental_5_6 -- +1000 + +-- !incremental_7_end -- +3000 + +-- !incremental_earliest_7 -- +7000 + +-- !incremental_6_7 -- +1000 + +-- !incremental_8_end -- +2000 + +-- !incremental_earliest_8 -- +8000 + +-- !incremental_7_8 -- +1000 + +-- !incremental_9_end -- +1000 + +-- !incremental_earliest_9 -- +9000 + +-- !incremental_8_9 -- +1000 + +-- !incremental_10_end -- +0 + +-- !incremental_earliest_10 -- +10000 + +-- !incremental_9_10 -- +1000 + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_schema_evolution.out b/regression-test/data/external_table_p2/hudi/test_hudi_schema_evolution.out new file mode 100644 index 000000000000000..12dd0cf086d3f04 --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/test_hudi_schema_evolution.out @@ -0,0 +1,33 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !adding_simple_columns_table -- +20241118012126237 20241118012126237_0_1 1 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 1 Alice \N +20241118012126237 20241118012126237_0_0 2 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 2 Bob \N +20241118012126237 20241118012126237_0_2 3 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 3 Cathy \N +20241118012132306 20241118012132306_0_3 4 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 4 David 25 +20241118012132306 20241118012132306_0_4 5 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 5 Eva 30 +20241118012132306 20241118012132306_0_5 6 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 6 Frank 28 + +-- !altering_simple_columns_table -- +20241118012136512 20241118012136512_0_0 1 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 1 Alice 25.0 +20241118012136512 20241118012136512_0_2 2 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 2 Bob 30.0 +20241118012136512 20241118012136512_0_1 3 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 3 Cathy 28.0 +20241118012138287 20241118012138287_0_3 4 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 4 David 26.0 +20241118012138287 20241118012138287_0_4 5 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 5 Eva 31.5 +20241118012138287 20241118012138287_0_5 6 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 6 Frank 29.2 + +-- !adding_complex_columns_table -- +20241118012144831 20241118012144831_0_1 1 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 1 Alice {"age":25, "address":"Guangzhou", "email":null} +20241118012144831 20241118012144831_0_0 2 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 2 Bob {"age":30, "address":"Shanghai", "email":null} +20241118012144831 20241118012144831_0_2 3 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 3 Cathy {"age":28, "address":"Beijing", "email":null} +20241118012146150 20241118012146150_0_3 4 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 4 David {"age":25, "address":"Shenzhen", "email":"david@example.com"} +20241118012146150 20241118012146150_0_4 5 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 5 Eva {"age":30, "address":"Chengdu", "email":"eva@example.com"} +20241118012146150 20241118012146150_0_5 6 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 6 Frank {"age":28, "address":"Wuhan", "email":"frank@example.com"} + +-- !altering_complex_columns_table -- +20241118012147879 20241118012147879_0_0 1 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 1 Alice {"age":25, "address":"Guangzhou"} +20241118012147879 20241118012147879_0_2 2 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 2 Bob {"age":30, "address":"Shanghai"} +20241118012147879 20241118012147879_0_1 3 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 3 Cathy {"age":28, "address":"Beijing"} +20241118012149007 20241118012149007_0_3 4 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 4 David {"age":26, "address":"Shenzhen"} +20241118012149007 20241118012149007_0_4 5 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 5 Eva {"age":31.5, "address":"Chengdu"} +20241118012149007 20241118012149007_0_5 6 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 6 Frank {"age":29.2, "address":"Wuhan"} + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_snapshot.out b/regression-test/data/external_table_p2/hudi/test_hudi_snapshot.out new file mode 100644 index 000000000000000..efad67ffbfa8c40 Binary files /dev/null and b/regression-test/data/external_table_p2/hudi/test_hudi_snapshot.out differ diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_timestamp.out b/regression-test/data/external_table_p2/hudi/test_hudi_timestamp.out new file mode 100644 index 000000000000000..dc47ff86d90a8d7 --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/test_hudi_timestamp.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !timestamp -- +20241115015956800 20241115015956800_0_2 1 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 1 Alice 2024-10-25T08:00 +20241115015956800 20241115015956800_0_0 2 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 2 Bob 2024-10-25T09:30:00 +20241115015956800 20241115015956800_0_1 3 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 3 Charlie 2024-10-25T11:00:00 + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out b/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out index 38b6ff7846f49ae..a9b5d23595a8e3d 100644 --- a/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out +++ b/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out @@ -1,125 +1,121 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !q00 -- -20240724195843565 20240724195843565_0_0 20240724195843565_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 1 a b para para -20240724195845718 20240724195845718_0_0 20240724195845718_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 2 a b para parb -20240724195848377 20240724195848377_0_1 20240724195848377_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 3 a b para para -20240724195850799 20240724195850799_0_1 20240724195850799_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 4 a b para parb +-- !timetravel1 -- +1000 --- !q01 -- +-- !timetravel2 -- +2000 --- !q02 -- +-- !timetravel3 -- +3000 --- !q01 -- -20240724195843565 20240724195843565_0_0 20240724195843565_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 1 a b para para -20240724195845718 20240724195845718_0_0 20240724195845718_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 2 a b para parb -20240724195848377 20240724195848377_0_1 20240724195848377_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 3 a b para para -20240724195850799 20240724195850799_0_1 20240724195850799_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 4 a b para parb - --- !q02 -- -20240724195843565 20240724195843565_0_0 20240724195843565_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 1 a b para para -20240724195845718 20240724195845718_0_0 20240724195845718_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 2 a b para parb -20240724195848377 20240724195848377_0_1 20240724195848377_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 3 a b para para -20240724195850799 20240724195850799_0_1 20240724195850799_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 4 a b para parb - --- !q03 -- - --- !q04 -- - --- !q05 -- -1 a b para para - --- !q06 -- -1 a b para para - --- !q07 -- -1 a b para para -2 a b para parb - --- !q08 -- -1 a b para para -2 a b para parb - --- !q09 -- -1 a b para para -2 a b para parb -3 a b para para - --- !q10 -- -1 a b para para -2 a b para parb -3 a b para para - --- !q11 -- -1 a b para para -2 a b para parb -3 a b para para -4 a b para parb - --- !q12 -- -1 a b para para -2 a b para parb -3 a b para para -4 a b para parb - --- !q50 -- -20240724195853736 20240724195853736_0_0 Id:1 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 1 a b para para -20240724195856338 20240724195856338_0_0 Id:2 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 2 a b para parb -20240724195858450 20240724195858450_0_1 Id:3 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 3 a b para para -20240724195902682 20240724195902682_0_1 Id:4 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 4 a b para parb - --- !q51 -- - --- !q52 -- - --- !q51 -- -20240724195853736 20240724195853736_0_0 Id:1 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 1 a b para para -20240724195856338 20240724195856338_0_0 Id:2 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 2 a b para parb -20240724195858450 20240724195858450_0_1 Id:3 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 3 a b para para -20240724195902682 20240724195902682_0_1 Id:4 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 4 a b para parb - --- !q52 -- -20240724195853736 20240724195853736_0_0 Id:1 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 1 a b para para -20240724195856338 20240724195856338_0_0 Id:2 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 2 a b para parb -20240724195858450 20240724195858450_0_1 Id:3 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 3 a b para para -20240724195902682 20240724195902682_0_1 Id:4 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 4 a b para parb - --- !q53 -- - --- !q54 -- - --- !q55 -- -1 a b para para - --- !q56 -- -1 a b para para - --- !q57 -- -1 a b para para -2 a b para parb - --- !q58 -- -1 a b para para -2 a b para parb - --- !q59 -- -1 a b para para -2 a b para parb -3 a b para para - --- !q60 -- -1 a b para para -2 a b para parb -3 a b para para - --- !q61 -- -1 a b para para -2 a b para parb -3 a b para para -4 a b para parb - --- !q62 -- -1 a b para para -2 a b para parb -3 a b para para -4 a b para parb +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 + +-- !timetravel1 -- +1000 + +-- !timetravel2 -- +2000 + +-- !timetravel3 -- +3000 + +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 + +-- !timetravel1 -- +1000 + +-- !timetravel2 -- +2000 + +-- !timetravel3 -- +3000 + +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 + +-- !timetravel1 -- +1000 + +-- !timetravel2 -- +2000 + +-- !timetravel3 -- +3000 + +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 diff --git a/regression-test/data/insert_p0/test_insert_tablet_sink.out b/regression-test/data/insert_p0/test_insert_tablet_sink.out new file mode 100644 index 000000000000000..90774639a55b70b --- /dev/null +++ b/regression-test/data/insert_p0/test_insert_tablet_sink.out @@ -0,0 +1,21 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +1 170141183460469231731687303715884105727 +2 -170141183460469231731687303715884105728 +3 333 +4 444 +5 555 +6 666 + +-- !select -- +1 170141183460469231731687303715884105727 +2 -170141183460469231731687303715884105728 +3 333 +3 333 +4 444 +4 444 +5 555 +5 555 +6 666 +6 666 + diff --git a/regression-test/suites/auth_call/test_account_management_user_auth.groovy b/regression-test/suites/auth_call/test_account_management_user_auth.groovy index e6b0c203dd367e9..936071b1e9d9ef6 100644 --- a/regression-test/suites/auth_call/test_account_management_user_auth.groovy +++ b/regression-test/suites/auth_call/test_account_management_user_auth.groovy @@ -23,7 +23,7 @@ suite("test_account_management_user_auth","p0,auth_call") { String user = 'test_account_management_user_auth_user' String pwd = 'C123_567p' String dbName = 'test_account_management_user_auth_db' - String user_derive = 'test_account_management_user_derive_role' + String user_derive = 'test_account_management_user_derive_user' //cloud-mode if (isCloudMode()) { @@ -68,6 +68,7 @@ suite("test_account_management_user_auth","p0,auth_call") { } } sql """grant grant_priv on *.*.* to '${user}'""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { sql """CREATE USER ${user_derive} IDENTIFIED BY '${pwd}';""" sql """ALTER USER ${user_derive} IDENTIFIED BY "${pwd}";""" diff --git a/regression-test/suites/auth_call/test_grant_show_view_priv_auth.groovy b/regression-test/suites/auth_call/test_grant_show_view_priv_auth.groovy new file mode 100644 index 000000000000000..59a7dc1efa8b5ef --- /dev/null +++ b/regression-test/suites/auth_call/test_grant_show_view_priv_auth.groovy @@ -0,0 +1,94 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.junit.Assert; +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite("test_grant_show_view_priv_auth","p0,auth_call") { + + String user = 'test_grant_show_view_priv_user' + String pwd = 'C123_567p' + String dbName = 'test_grant_show_view_priv_auth_db' + String user_derive = 'test_grant_show_view_priv_derive_user' + String tableName = 'test_grant_show_view_priv_auth_tb' + String viewName = 'test_grant_show_view_priv_auth_view' + + //cloud-mode + if (isCloudMode()) { + def clusters = sql " SHOW CLUSTERS; " + assertTrue(!clusters.isEmpty()) + def validCluster = clusters[0][0] + sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}"""; + } + + try_sql("DROP USER ${user}") + try_sql("DROP USER ${user_derive}") + try_sql """drop database if exists ${dbName}""" + + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + sql """create database ${dbName}""" + sql """create table ${dbName}.${tableName} ( + id BIGINT, + username VARCHAR(20) + ) + DISTRIBUTED BY HASH(id) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + );""" + sql """ + INSERT INTO ${dbName}.${tableName} (id, username) + VALUES (1, "111"), + (2, "222"), + (3, "333") + """ + sql """CREATE VIEW ${dbName}.${viewName} (k1, v1) + AS + SELECT id as k1, SUM(id) FROM ${dbName}.${tableName} + WHERE id = 1 GROUP BY k1; + """ + + sql """grant grant_priv on *.*.* to '${user}'""" + sql """grant SHOW_VIEW_PRIV on ${dbName}.${viewName} to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + sql """CREATE USER ${user_derive} IDENTIFIED BY '${pwd}';""" + sql """grant select_priv on regression_test to ${user_derive}""" + + connect(user=user_derive, password="${pwd}", url=context.config.jdbcUrl) { + test { + sql """show create table ${dbName}.${viewName};""" + exception "denied" + } + test { + sql """select * from ${dbName}.${viewName};""" + exception "denied" + } + } + sql """grant SHOW_VIEW_PRIV on ${dbName}.${viewName} to ${user_derive}""" + connect(user=user_derive, password="${pwd}", url=context.config.jdbcUrl) { + sql """show create table ${dbName}.${viewName};""" + test { + sql """select * from ${dbName}.${viewName};""" + exception "denied" + } + } + } + + sql """drop database if exists ${dbName}""" + try_sql("DROP USER ${user}") + try_sql("DROP role ${user_derive}") +} diff --git a/regression-test/suites/bloom_filter_p0/test_bloom_filter_drop_column.groovy b/regression-test/suites/bloom_filter_p0/test_bloom_filter_drop_column.groovy index f426d4fca10a79a..d83c70af30c709c 100644 --- a/regression-test/suites/bloom_filter_p0/test_bloom_filter_drop_column.groovy +++ b/regression-test/suites/bloom_filter_p0/test_bloom_filter_drop_column.groovy @@ -21,13 +21,14 @@ suite("test_bloom_filter_drop_column") { sql """CREATE TABLE IF NOT EXISTS ${table_name} ( `a` varchar(150) NULL, - `c1` varchar(10) + `c1` varchar(10), + `c2` varchar(10) ) ENGINE=OLAP DUPLICATE KEY(`a`) DISTRIBUTED BY HASH(`a`) BUCKETS 1 PROPERTIES ( "replication_allocation" = "tag.location.default: 1", - "bloom_filter_columns" = "c1", + "bloom_filter_columns" = "c1, c2", "in_memory" = "false", "storage_format" = "V2" )""" @@ -51,12 +52,12 @@ suite("test_bloom_filter_drop_column") { assertTrue(useTime <= OpTimeout, "wait_for_latest_op_on_table_finish timeout") } - def assertShowCreateTableWithRetry = { tableName, expectedCondition, maxRetries, waitSeconds -> + def assertShowCreateTableWithRetry = { tableName, expectedCondition, contains, maxRetries, waitSeconds -> int attempt = 0 while (attempt < maxRetries) { def res = sql """SHOW CREATE TABLE ${tableName}""" log.info("Attempt ${attempt + 1}: show table: ${res}") - if (res && res.size() > 0 && res[0][1].contains(expectedCondition)) { + if (res && res.size() > 0 && ((contains && res[0][1].contains(expectedCondition)) || (!contains && !res[0][1].contains(expectedCondition)))) { logger.info("Attempt ${attempt + 1}: Condition met.") return } else { @@ -70,21 +71,34 @@ suite("test_bloom_filter_drop_column") { def finalRes = sql """SHOW CREATE TABLE ${tableName}""" log.info("Final attempt: show table: ${finalRes}") assertTrue(finalRes && finalRes.size() > 0, "SHOW CREATE TABLE return empty or null") - assertTrue(finalRes[0][1].contains(expectedCondition), "expected\"${expectedCondition}\",actural: ${finalRes[0][1]}") + if (contains) { + assertTrue(finalRes[0][1].contains(expectedCondition), "expected to contain \"${expectedCondition}\", actual: ${finalRes[0][1]}") + } else { + assertTrue(!finalRes[0][1].contains(expectedCondition), "expected not to contain \"${expectedCondition}\", actual: ${finalRes[0][1]}") + } } - sql """INSERT INTO ${table_name} values ('1', '1')""" + sql """INSERT INTO ${table_name} values ('1', '1', '1')""" sql "sync" qt_select """select * from ${table_name} order by a""" + assertShowCreateTableWithRetry(table_name, "\"bloom_filter_columns\" = \"c1, c2\"", true, 3, 30) // drop column c1 sql """ALTER TABLE ${table_name} DROP COLUMN c1""" wait_for_latest_op_on_table_finish(table_name, timeout) sql "sync" // show create table with retry logic - assertShowCreateTableWithRetry(table_name, "\"bloom_filter_columns\" = \"\"", 3, 30) + assertShowCreateTableWithRetry(table_name, "\"bloom_filter_columns\" = \"c2\"", true, 3, 30) + + // drop column c2 + sql """ALTER TABLE ${table_name} DROP COLUMN c2""" + wait_for_latest_op_on_table_finish(table_name, timeout) + sql "sync" + + // show create table with retry logic + assertShowCreateTableWithRetry(table_name, "\"bloom_filter_columns\" = \"\"", false, 3, 30) // add new column c1 sql """ALTER TABLE ${table_name} ADD COLUMN c1 ARRAY""" diff --git a/regression-test/suites/ddl_p0/create_view_nereids/create_view_use_mv.groovy b/regression-test/suites/ddl_p0/create_view_nereids/create_view_use_mv.groovy new file mode 100644 index 000000000000000..295b195aa589547 --- /dev/null +++ b/regression-test/suites/ddl_p0/create_view_nereids/create_view_use_mv.groovy @@ -0,0 +1,95 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("create_view_use_mv") { + sql "drop table if exists orders" + sql """create table orders ( o_orderkey bigint null, o_custkey int null, o_orderstatus varchar(1) null, + o_totalprice decimal(15,2) null, o_orderpriority varchar(15) null, o_clerk varchar(15) null, o_shippriority int null, + o_comment varchar(79) null, o_orderdate date not null) engine=olap duplicate key(o_orderkey,o_custkey) + comment 'olap' distributed by hash(o_orderkey) buckets 96 properties("replication_num"="1")""" + + sql "drop table if exists lineitem" + sql """create table lineitem ( + l_orderkey bigint null, l_linenumber int null, l_partkey int null, l_suppkey int null, l_quantity decimal(15,2) null,l_extendedprice decimal(15,2) null, + l_discount decimal(15,2) null, l_tax decimal(15,2) null, l_returnflag varchar(1) null, l_linestatus varchar(1) null, l_commitdate date null, l_receiptdate date null, + l_shipnstruct varchar(25) null, l_shipmode varchar(10) null, l_comment varchar(44) null,l_shipdate date not null) engine=olap + duplicate key(l_orderkey, l_linenumber,l_partkey, l_suppkey) + distributed by hash(l_orderkey) buckets 96 + properties("replication_num"="1");""" + + sql """insert into orders values (null, 1, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'),(1, null, 'o', 109.2, 'c','d',2, 'mm', '2023-10-17'), + (3, 3, null, 99.5, 'a', 'b', 1, 'yy', '2023-10-19'), + (1, 2, 'o', null, 'a', 'b', 1, 'yy', '2023-10-20'), + (2, 3, 'k', 109.2, null,'d',2, 'mm', '2023-10-21'), + (3, 1, 'k', 99.5, 'a', null, 1, 'yy', '2023-10-22'), + (1, 3, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'o', 109.2, 'c','d',2, null, '2023-10-18'), + (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (4, 5, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-19');""" + + sql """insert into lineitem values(null, 1, 2, 3, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'),(1, null, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (3, 3, null, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 2, 3, null, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (2, 3, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), + (3, 1, 1, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17');""" + + createMV(""" + CREATE MATERIALIZED VIEW t_mv_mv AS select + o_orderkey, + sum(o_totalprice) as sum_total, + max(o_totalprice) as max_total, + min(o_totalprice) as min_total, + count(*) as count_all, + bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, + bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 + from orders group by o_orderkey + """) + + sql "drop view if exists t_mv_v_view" + sql """CREATE VIEW t_mv_v_view (k1, k2, k3, k4, k5, k6, v1, v2, v3, v4, v5, v6) as + select `mv_o_orderkey` as k1, `mva_SUM__``o_totalprice``` as k2, `mva_MAX__``o_totalprice``` as k3, + `mva_MIN__``o_totalprice``` as k4, `mva_SUM__CASE WHEN 1 IS NULL THEN 0 ELSE 1 END` as k5, l_orderkey, + sum(`mv_o_orderkey`) as sum_total, + max(`mv_o_orderkey`) as max_total, + min(`mv_o_orderkey`) as min_total, + count(`mva_SUM__``o_totalprice```) as count_all, + bitmap_union(to_bitmap(case when mv_o_orderkey > 1 then `mva_SUM__``o_totalprice``` else null end)) cnt_1, + bitmap_union(to_bitmap(case when mv_o_orderkey > 2 then `mva_MAX__``o_totalprice``` else null end)) as cnt_2 + from orders index t_mv_mv + left join lineitem on lineitem.l_orderkey = orders.mv_o_orderkey + group by + k1, k2, k3, k4, k5, l_orderkey, mv_o_orderkey""" + qt_create_view_from_mv "select * from t_mv_v_view order by 1" + + sql "drop view if exists v_for_alter" + sql "CREATE VIEW v_for_alter AS SELECT * FROM orders" + sql """ALTER VIEW v_for_alter as + select `mv_o_orderkey` as k1, `mva_SUM__``o_totalprice``` as k2, `mva_MAX__``o_totalprice``` as k3, + `mva_MIN__``o_totalprice``` as k4, `mva_SUM__CASE WHEN 1 IS NULL THEN 0 ELSE 1 END` as k5, l_orderkey, + sum(`mv_o_orderkey`) as sum_total, + max(`mv_o_orderkey`) as max_total, + min(`mv_o_orderkey`) as min_total, + count(`mva_SUM__``o_totalprice```) as count_all, + bitmap_union(to_bitmap(case when mv_o_orderkey > 1 then `mva_SUM__``o_totalprice``` else null end)) cnt_1, + bitmap_union(to_bitmap(case when mv_o_orderkey > 2 then `mva_MAX__``o_totalprice``` else null end)) as cnt_2 + from orders index t_mv_mv + left join lineitem on lineitem.l_orderkey = orders.mv_o_orderkey + group by + k1, k2, k3, k4, k5, l_orderkey, mv_o_orderkey""" + qt_alter_view_from_mv "select * from v_for_alter order by 1" +} \ No newline at end of file diff --git a/regression-test/suites/ddl_p0/test_create_view_nereids.groovy b/regression-test/suites/ddl_p0/create_view_nereids/test_create_view_nereids.groovy similarity index 100% rename from regression-test/suites/ddl_p0/test_create_view_nereids.groovy rename to regression-test/suites/ddl_p0/create_view_nereids/test_create_view_nereids.groovy diff --git a/regression-test/suites/external_table_p0/hive/test_hive_orc.groovy b/regression-test/suites/external_table_p0/hive/test_hive_orc.groovy index 0f837c0abd3088a..6457d2b3edd5d82 100644 --- a/regression-test/suites/external_table_p0/hive/test_hive_orc.groovy +++ b/regression-test/suites/external_table_p0/hive/test_hive_orc.groovy @@ -81,6 +81,17 @@ suite("test_hive_orc", "all_types,p0,external,hive,external_docker,external_dock qt_string_col_dict_plain_mixed3 """select count(col2) from string_col_dict_plain_mixed_orc where col1 like '%Test%';""" } + def predicate_pushdown = { + qt_predicate_pushdown1 """ select count(o_orderkey) from tpch1_orc.orders where o_orderkey is not null and (o_orderkey < 100 or o_orderkey > 5999900 or o_orderkey in (1000000, 2000000, 3000000)); """ + qt_predicate_pushdown2 """ select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 100 and 1000 and o_orderkey not in (200, 300, 400)); """ + qt_predicate_pushdown3 """ select count(o_orderkey) from tpch1_orc.orders where o_orderkey is not null and (o_orderkey < 100 or o_orderkey > 5999900 or o_orderkey = 3000000); """ + qt_predicate_pushdown4 """ select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 1000000 and 1200000 and o_orderkey != 1100000); """ + qt_predicate_pushdown5 """ SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE (o_orderdate >= '1994-01-01' AND o_orderdate <= '1994-12-31') AND (o_orderpriority = '5-LOW' OR o_orderpriority = '3-MEDIUM') AND o_totalprice > 2000;""" + qt_predicate_pushdown6 """ SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_orderstatus <> 'F' AND o_custkey < 54321; """ + qt_predicate_pushdown7 """ SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_comment LIKE '%delayed%' OR o_orderpriority = '1-URGENT'; """ + qt_predicate_pushdown8 """ SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_orderkey IN (1000000, 2000000, 3000000) OR o_clerk = 'Clerk#000000470'; """ + } + String enabled = context.config.otherConfigs.get("enableHiveTest") if (enabled == null || !enabled.equalsIgnoreCase("true")) { logger.info("diable Hive test.") @@ -108,6 +119,7 @@ suite("test_hive_orc", "all_types,p0,external,hive,external_docker,external_dock only_partition_col() decimals() string_col_dict_plain_mixed() + predicate_pushdown() sql """drop catalog if exists ${catalog_name}""" diff --git a/regression-test/suites/external_table_p0/hive/test_hive_use_meta_cache.groovy b/regression-test/suites/external_table_p0/hive/test_hive_use_meta_cache.groovy index 3562ce3126725de..df12fc74898f06f 100644 --- a/regression-test/suites/external_table_p0/hive/test_hive_use_meta_cache.groovy +++ b/regression-test/suites/external_table_p0/hive/test_hive_use_meta_cache.groovy @@ -23,80 +23,113 @@ suite("test_hive_use_meta_cache", "p0,external,hive,external_docker,external_doc return; } - for (String hivePrefix : ["hive3", "hive3"]) { + for (String hivePrefix : ["hive2", "hive3"]) { setHivePrefix(hivePrefix) try { - String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") - String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") - String catalog = "test_${hivePrefix}_use_meta_cache" - String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + def test_use_meta_cache = { Boolean use_meta_cache -> + String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") + String use_meta_cache_string = use_meta_cache ? "true" : "false" + String catalog = "test_${hivePrefix}_use_meta_cache_${use_meta_cache}" + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") - sql """drop catalog if exists ${catalog}""" - sql """create catalog if not exists ${catalog} properties ( - 'type'='hms', - 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', - 'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}', - 'use_meta_cache' = 'true' - );""" + sql """drop catalog if exists ${catalog}""" + sql """create catalog if not exists ${catalog} properties ( + 'type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}', + 'use_meta_cache' = '${use_meta_cache_string}' + );""" + + // create from Doris, the cache will be filled immediately + String database= "test_use_meta_cache_db" + String table = "test_use_meta_cache_tbl" + String database_hive = "test_use_meta_cache_db_hive" + String table_hive = "test_use_meta_cache_tbl_hive" + String partitioned_table_hive = "test_use_meta_cache_partitioned_tbl_hive" + + sql "switch ${catalog}" + sql "drop database if exists ${database}" + sql "drop database if exists ${database_hive}" + order_qt_sql01 "show databases like '%${database}%'"; + sql "drop database if exists ${database}" + sql "create database ${database}" + order_qt_sql02 "show databases like '%${database}%'"; + sql "use ${database}" + sql "create table ${table} (k1 int)" + order_qt_sql03 "show tables" + sql "drop table ${table}" + order_qt_sql04 "show tables" + sql "drop database ${database}" + order_qt_sql05 "show databases like '%${database}%'"; - // create from Doris, the cache will be filled immediately - String database= "test_use_meta_cache_db" - String table = "test_use_meta_cache_tbl" - String database_hive = "test_use_meta_cache_db_hive" - String table_hive = "test_use_meta_cache_tbl_hive" - sql "switch ${catalog}" - sql "drop database if exists ${database}" - sql "drop database if exists ${database_hive}" - order_qt_sql01 "show databases like '%${database}%'"; - sql "drop database if exists ${database}" - sql "create database ${database}" - order_qt_sql02 "show databases like '%${database}%'"; - sql "use ${database}" - sql "create table ${table} (k1 int)" - order_qt_sql03 "show tables" - sql "drop table ${table}" - order_qt_sql04 "show tables" - sql "drop database ${database}" - order_qt_sql05 "show databases like '%${database}%'"; - - // create from Hive, the cache has different behavior - order_qt_sql01 "show databases like '%${database_hive}%'"; - hive_docker "drop database if exists ${database_hive}" - hive_docker "create database ${database_hive}" - // not see - order_qt_sql02 "show databases like '%${database_hive}%'"; - // but can use - sql "use ${database_hive}" - sql "refresh catalog ${catalog}" - // can see - order_qt_sql03 "show databases like '%${database_hive}%'"; - // show tables first to fill cache - order_qt_sql04 "show tables" - hive_docker "create table ${database_hive}.${table_hive} (k1 int)" - // not see - order_qt_sql05 "show tables" - // but can select - sql "select * from ${table_hive}" - // still not see - order_qt_sql06 "show tables" - sql "refresh database ${database_hive}" - // can see - order_qt_sql07 "show tables" - hive_docker "drop table ${database_hive}.${table_hive}" - // still can see - order_qt_sql08 "show tables" - sql "refresh database ${database_hive}" - // can not see - order_qt_sql09 "show tables" - hive_docker "drop database ${database_hive}" - // still can see - order_qt_sql10 "show databases like '%${database_hive}%'"; - sql "refresh catalog ${catalog}" - // can not see - order_qt_sql11 "show databases like '%${database_hive}%'"; + // create from Hive, the cache has different behavior + order_qt_sql01 "show databases like '%${database_hive}%'"; + hive_docker "drop database if exists ${database_hive}" + hive_docker "create database ${database_hive}" + // not see + order_qt_sql02 "show databases like '%${database_hive}%'"; + if (use_meta_cache) { + // if use meta cache, can use + sql "use ${database_hive}" + sql "refresh catalog ${catalog}" + } else { + // if not use meta cache, can not use + sql "refresh catalog ${catalog}" + sql "use ${database_hive}" + } + + // can see + order_qt_sql03 "show databases like '%${database_hive}%'"; + // show tables first to fill cache + order_qt_sql04 "show tables" + hive_docker "create table ${database_hive}.${table_hive} (k1 int)" + // not see + order_qt_sql05 "show tables" + if (use_meta_cache) { + // but can select + sql "select * from ${table_hive}" + // still not see + order_qt_sql06 "show tables" + sql "refresh database ${database_hive}" + } else { + // if not use meta cache, can not select + sql "refresh database ${database_hive}" + sql "select * from ${table_hive}" + order_qt_sql06 "show tables" + } + // can see + order_qt_sql07 "show tables" + + // test Hive Metastore table partition file listing + hive_docker "create table ${database_hive}.${partitioned_table_hive} (k1 int) partitioned by (p1 string)" + sql "refresh catalog ${catalog}" + order_qt_sql08 "show partitions from ${partitioned_table_hive}" + hive_docker "alter table ${database_hive}.${partitioned_table_hive} add partition (p1='part1')" + hive_docker "alter table ${database_hive}.${partitioned_table_hive} add partition (p1='part2')" + // can see because partition file listing is not cached + order_qt_sql09 "show partitions from ${partitioned_table_hive}" + + // drop tables + hive_docker "drop table ${database_hive}.${partitioned_table_hive}" + hive_docker "drop table ${database_hive}.${table_hive}" + // still can see + order_qt_sql10 "show tables" + sql "refresh database ${database_hive}" + // can not see + order_qt_sql11 "show tables" + + // drop database + hive_docker "drop database ${database_hive}" + // still can see + order_qt_sql12 "show databases like '%${database_hive}%'"; + sql "refresh catalog ${catalog}" + // can not see + order_qt_sql13 "show databases like '%${database_hive}%'"; + } + test_use_meta_cache(true) + test_use_meta_cache(false) } finally { } } } - - diff --git a/regression-test/suites/external_table_p0/iceberg/iceberg_read_unitycatalog_table.groovy b/regression-test/suites/external_table_p0/iceberg/iceberg_read_unitycatalog_table.groovy new file mode 100644 index 000000000000000..48b8b6559ca82e1 --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/iceberg_read_unitycatalog_table.groovy @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("iceberg_read_unitycatalog_table", "p0,external,doris,external_docker,external_docker_doris") { + + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable iceberg test.") + return + } + + String catalog_name = "iceberg_read_unitycatalog_table" + String rest_port = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + sql """drop catalog if exists ${catalog_name}""" + sql """ + CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://${externalEnvIp}:${rest_port}', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + logger.info("catalog " + catalog_name + " created") + sql """ use ${catalog_name}.test_db """ + String tb = "unitycatalog_marksheet_uniform" + + qt_q1 """ select * from ${tb} order by c1 """ + qt_q2 """ select c1 from ${tb} where c1 > 6 order by c1 """ + qt_q3 """ select c2, c3 from ${tb} where c3 > 200 order by c1 """ + +} + +/* + +spark-sql: + 1. create table marksheet_uniform (c1 int, c2 string, c3 int); + 2. get parquet file from marksheet_uniform; (ref: https://docs.unitycatalog.io/usage/tables/uniform/) + 3. put parquet file to hdfs: hdfs dfs -put hdfs://xxxxx + 4. CALL .system.add_files( + table => '.unitycatalog_db.marksheet_uniform', + source_table => '`parquet`.`hdfs://172.20.32.136:8020/user/doris/preinstalled_data/iceberg_hadoop_warehouse/unitycatalog_db/marksheet_uniform_data/part-00000-5af50cc4-3218-465b-a3a4-eb4fc709421d-c000.snappy.parquet`' + ); +*/ \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_catalog.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_catalog.groovy new file mode 100644 index 000000000000000..f2082ef89c7a50f --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_catalog.groovy @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hudi_catalog", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_catalog" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + def tables = sql """ show tables; """ + assertTrue(tables.size() > 0) + sql """drop catalog if exists ${catalog_name};""" +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_incremental.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_incremental.groovy new file mode 100644 index 000000000000000..8cc1d2a852b8c4b --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_incremental.groovy @@ -0,0 +1,111 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hudi_incremental", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_incremental" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + + def test_hudi_incremental_querys = { table_name, timestamps -> + timestamps.eachWithIndex { timestamp, index -> + def query_name = "qt_incremental_${index + 1}_end" + "${query_name}" """ select count(user_id) from ${table_name}@incr('beginTime' = '${timestamp}'); """ + query_name = "qt_incremental_earliest_${index + 1}" + "${query_name}" """ select count(user_id) from ${table_name}@incr('beginTime' = 'earliest', 'endTime' = '${timestamp}'); """ + if (index > 0) { + query_name = "qt_incremental_${index}_${index + 1}" + "${query_name}" """ select count(user_id) from ${table_name}@incr('beginTime' = '${timestamps[index - 1]}', 'endTime' = '${timestamp}'); """ + } + } + } + + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_cow_non_partition order by _hoodie_commit_time;" + def timestamps_cow_non_partition = [ + "20241114151946599", + "20241114151952471", + "20241114151956317", + "20241114151958164", + "20241114152000425", + "20241114152004116", + "20241114152005954", + "20241114152007945", + "20241114152009764", + "20241114152011901", + ] + test_hudi_incremental_querys("user_activity_log_cow_non_partition", timestamps_cow_non_partition) + + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_cow_partition order by _hoodie_commit_time;" + def timestamps_cow_partition = [ + "20241114152034850", + "20241114152042944", + "20241114152052682", + "20241114152101650", + "20241114152110650", + "20241114152120030", + "20241114152128871", + "20241114152137714", + "20241114152147114", + "20241114152156417", + ] + test_hudi_incremental_querys("user_activity_log_cow_partition", timestamps_cow_partition) + + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_non_partition order by _hoodie_commit_time;" + def timestamps_mor_non_partition = [ + "20241114152014186", + "20241114152015753", + "20241114152017539", + "20241114152019371", + "20241114152020915", + "20241114152022911", + "20241114152024706", + "20241114152026873", + "20241114152028770", + "20241114152030746", + ] + test_hudi_incremental_querys("user_activity_log_mor_non_partition", timestamps_mor_non_partition) + + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_partition order by _hoodie_commit_time;" + def timestamps_mor_partition = [ + "20241114152207700", + "20241114152214609", + "20241114152223933", + "20241114152232579", + "20241114152241610", + "20241114152252244", + "20241114152302763", + "20241114152313010", + "20241114152323587", + "20241114152334111", + ] + test_hudi_incremental_querys("user_activity_log_mor_partition", timestamps_mor_partition) + + sql """drop catalog if exists ${catalog_name};""" +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_schema_evolution.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_schema_evolution.groovy new file mode 100644 index 000000000000000..b247aaf492400d4 --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_schema_evolution.groovy @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hudi_schema_evolution", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_schema_evolution" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + + qt_adding_simple_columns_table """ select * from adding_simple_columns_table order by id """ + qt_altering_simple_columns_table """ select * from altering_simple_columns_table order by id """ + // qt_deleting_simple_columns_table """ select * from deleting_simple_columns_table order by id """ + // qt_renaming_simple_columns_table """ select * from renaming_simple_columns_table order by id """ + + qt_adding_complex_columns_table """ select * from adding_complex_columns_table order by id """ + qt_altering_complex_columns_table """ select * from altering_complex_columns_table order by id """ + // qt_deleting_complex_columns_table """ select * from deleting_complex_columns_table order by id """ + // qt_renaming_complex_columns_table """ select * from renaming_complex_columns_table order by id """ + + sql """drop catalog if exists ${catalog_name};""" +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_snapshot.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_snapshot.groovy new file mode 100644 index 000000000000000..53c09e6d5a90312 --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_snapshot.groovy @@ -0,0 +1,91 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hudi_snapshot", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_snapshot" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + + // 创建groovy函数,接收table_name为参数 + def test_hudi_snapshot_querys = { table_name -> + // Query users by event_time in descending order and limit output + qt_q01 """SELECT * FROM ${table_name} ORDER BY event_time DESC LIMIT 10;""" + + // Query all active user records and limit output + qt_q02 """SELECT * FROM ${table_name} WHERE is_active = TRUE ORDER BY event_time LIMIT 10;""" + + // Query specific user's activity records and limit output + qt_q03 """SELECT * FROM ${table_name} WHERE user_id = '62785e0e-ad44-4321-8b20-9ee4c4daca4a' ORDER BY event_time LIMIT 5;""" + + // Query events within a specific time range and limit output + qt_q04 """SELECT * FROM ${table_name} WHERE event_time BETWEEN '2024-01-01 00:00:00' AND '2024-12-31 23:59:59' ORDER BY event_time LIMIT 10;""" + + // Count users by age group and limit output + qt_q05 """SELECT age, COUNT(*) AS user_count FROM ${table_name} GROUP BY age ORDER BY user_count DESC LIMIT 5;""" + + // Query users with purchase records and limit output + qt_q06 """SELECT user_id, purchases FROM ${table_name} WHERE array_size(purchases) > 0 ORDER BY user_id LIMIT 5;""" + + // Query users with a specific tag and limit output + qt_q07 """SELECT * FROM ${table_name} WHERE array_contains(tags, 'others') ORDER BY event_time LIMIT 5;""" + + // Query users living in a specific city and limit output + qt_q08 """SELECT * FROM ${table_name} WHERE struct_element(address, 'city') = 'North Rachelview' ORDER BY event_time LIMIT 5;""" + + // Query users within a specific coordinate range and limit output + qt_q09 """SELECT * FROM ${table_name} WHERE struct_element(struct_element(address, 'coordinates'), 'latitude') BETWEEN 0 AND 100 AND struct_element(struct_element(address, 'coordinates'), 'longitude') BETWEEN 0 AND 100 ORDER BY event_time LIMIT 5;""" + + // Query records with ratings above a specific value and limit output + qt_q10 """SELECT * FROM ${table_name} WHERE rating > 4.5 ORDER BY rating DESC LIMIT 5;""" + + // Query all users' signup dates and limit output + qt_q11 """SELECT user_id, signup_date FROM ${table_name} ORDER BY signup_date DESC LIMIT 10;""" + + // Query users with a specific postal code and limit output + qt_q12 """SELECT * FROM ${table_name} WHERE struct_element(address, 'postal_code') = '80312' ORDER BY event_time LIMIT 5;""" + + // Query users with profile pictures and limit output + qt_q13 """SELECT user_id, profile_picture FROM ${table_name} WHERE profile_picture IS NOT NULL ORDER BY user_id LIMIT 5;""" + + // Query users by signup date and limit output + qt_q14 """SELECT * FROM ${table_name} WHERE signup_date = '2024-01-15' ORDER BY user_id LIMIT 5;""" + + // Query the total count of purchases for each user and limit output + qt_q15 """SELECT user_id, array_size(purchases) AS purchase_count FROM ${table_name} ORDER BY purchase_count DESC LIMIT 5;""" + } + + test_hudi_snapshot_querys("user_activity_log_cow_non_partition") + test_hudi_snapshot_querys("user_activity_log_cow_partition") + test_hudi_snapshot_querys("user_activity_log_mor_non_partition") + test_hudi_snapshot_querys("user_activity_log_mor_partition") + + sql """drop catalog if exists ${catalog_name};""" +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_timestamp.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_timestamp.groovy new file mode 100644 index 000000000000000..c1ba630e4a7d01e --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_timestamp.groovy @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hudi_timestamp", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_timestamp" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + + // TODO: fix hudi timezone issue and enable this + // qt_timestamp """ select * from hudi_table_with_timestamp order by id; """ + + sql """drop catalog if exists ${catalog_name};""" +} + +// DROP TABLE IF EXISTS hudi_table_with_timestamp; + +// -- create table +// CREATE TABLE hudi_table_with_timestamp ( +// id STRING, +// name STRING, +// event_time TIMESTAMP +// ) USING HUDI +// OPTIONS ( +// type = 'cow', +// primaryKey = 'id', +// preCombineField = 'event_time' +// ); + +// SET TIME ZONE 'America/Los_Angeles'; + +// INSERT OVERWRITE hudi_table_with_timestamp VALUES +// ('1', 'Alice', timestamp('2024-10-25 08:00:00')), +// ('2', 'Bob', timestamp('2024-10-25 09:30:00')), +// ('3', 'Charlie', timestamp('2024-10-25 11:00:00')); \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy index db535e3517987eb..4d458dc4381dcf7 100644 --- a/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy @@ -16,13 +16,6 @@ // under the License. suite("test_hudi_timetravel", "p2,external,hudi,external_remote,external_remote_hudi") { - - Boolean ignoreP2 = true; - if (ignoreP2) { - logger.info("disable p2 test"); - return; - } - String enabled = context.config.otherConfigs.get("enableExternalHudiTest") if (enabled == null || !enabled.equalsIgnoreCase("true")) { logger.info("disable hudi test") @@ -37,77 +30,76 @@ suite("test_hudi_timetravel", "p2,external,hudi,external_remote,external_remote_ ); """ - sql """switch ${catalog_name};""" + sql """ switch ${catalog_name};""" sql """ use regression_hudi;""" sql """ set enable_fallback_to_original_planner=false """ - qt_q00 """select * from timetravel_cow order by id""" - qt_q01 """select * from timetravel_cow FOR TIME AS OF "2024-07-24" order by id""" // no data - qt_q02 """select * from timetravel_cow FOR TIME AS OF "20240724" order by id""" // no data - qt_q01 """select * from timetravel_cow FOR TIME AS OF "2024-07-25" order by id""" - qt_q02 """select * from timetravel_cow FOR TIME AS OF "20240725" order by id""" - qt_q03 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "2024-07-24 19:58:43" order by id """ // no data - qt_q04 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "20240724195843" order by id """ // no data - qt_q05 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "2024-07-24 19:58:44" order by id """ // one - qt_q06 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "20240724195844" order by id """ //one - qt_q07 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "2024-07-24 19:58:48" order by id """ // two - qt_q08 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "20240724195848" order by id """ // two - qt_q09 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "2024-07-24 19:58:49" order by id """ // three - qt_q10 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "20240724195849" order by id """ // three - qt_q11 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "2024-07-24 19:58:51" order by id """ // four - qt_q12 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "20240724195851" order by id """ // four - - qt_q50 """select * from timetravel_mor order by id""" - qt_q51 """select * from timetravel_mor FOR TIME AS OF "2024-07-24" order by id""" // no data - qt_q52 """select * from timetravel_mor FOR TIME AS OF "20240724" order by id""" // no data - qt_q51 """select * from timetravel_mor FOR TIME AS OF "2024-07-25" order by id""" - qt_q52 """select * from timetravel_mor FOR TIME AS OF "20240725" order by id""" - qt_q53 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "2024-07-24 19:58:53" order by id """ // no data - qt_q54 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "20240724195853" order by id """ // no data - qt_q55 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "2024-07-24 19:58:54" order by id """ // one - qt_q56 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "20240724195854" order by id """ //one - qt_q57 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "2024-07-24 19:58:58" order by id """ // two - qt_q58 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "20240724195858" order by id """ // two - qt_q59 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "2024-07-24 19:58:59" order by id """ // three - qt_q60 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "20240724195859" order by id """ // three - qt_q61 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "2024-07-24 19:59:03" order by id """ // four - qt_q62 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "20240724195903" order by id """ // four -} - - -/* + def test_hudi_timetravel_querys = { table_name, timestamps -> + timestamps.eachWithIndex { timestamp, index -> + def query_name = "qt_timetravel${index + 1}" + "${query_name}" """ select count(user_id) from ${table_name} for time as of "${timestamp}"; """ + } + } -create table timetravel_cow ( - Id int, - VAL1 string, - val2 string, - PAR1 string, - par2 string -) using hudi -partitioned by (par1, par2) -TBLPROPERTIES ( - 'type' = 'cow'); + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_cow_non_partition order by _hoodie_commit_time;" + def timestamps_cow_non_partition = [ + "20241114151946599", + "20241114151952471", + "20241114151956317", + "20241114151958164", + "20241114152000425", + "20241114152004116", + "20241114152005954", + "20241114152007945", + "20241114152009764", + "20241114152011901", + ] + test_hudi_timetravel_querys("user_activity_log_cow_non_partition", timestamps_cow_non_partition) -create table timetravel_mor ( - Id int, - VAL1 string, - val2 string, - PAR1 string, - par2 string -) using hudi -partitioned by (par1, par2) -TBLPROPERTIES ( - 'primaryKey' = 'Id', - 'type' = 'mor'); + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_cow_partition order by _hoodie_commit_time;" + def timestamps_cow_partition = [ + "20241114152034850", + "20241114152042944", + "20241114152052682", + "20241114152101650", + "20241114152110650", + "20241114152120030", + "20241114152128871", + "20241114152137714", + "20241114152147114", + "20241114152156417", + ] + test_hudi_timetravel_querys("user_activity_log_cow_partition", timestamps_cow_partition) -insert into timetravel_cow values (1, 'a','b','para','para'); -insert into timetravel_cow values (2, 'a','b','para','parb'); -insert into timetravel_cow values (3, 'a','b','para','para'); -insert into timetravel_cow values (4, 'a','b','para','parb'); + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_non_partition order by _hoodie_commit_time;" + def timestamps_mor_non_partition = [ + "20241114152014186", + "20241114152015753", + "20241114152017539", + "20241114152019371", + "20241114152020915", + "20241114152022911", + "20241114152024706", + "20241114152026873", + "20241114152028770", + "20241114152030746", + ] + test_hudi_timetravel_querys("user_activity_log_mor_non_partition", timestamps_mor_non_partition) -insert into timetravel_mor values (1, 'a','b','para','para'); -insert into timetravel_mor values (2, 'a','b','para','parb'); -insert into timetravel_mor values (3, 'a','b','para','para'); -insert into timetravel_mor values (4, 'a','b','para','parb'); + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_partition order by _hoodie_commit_time;" + def timestamps_mor_partition = [ + "20241114152207700", + "20241114152214609", + "20241114152223933", + "20241114152232579", + "20241114152241610", + "20241114152252244", + "20241114152302763", + "20241114152313010", + "20241114152323587", + "20241114152334111", + ] + test_hudi_timetravel_querys("user_activity_log_mor_partition", timestamps_mor_partition) -*/ + sql """drop catalog if exists ${catalog_name};""" +} \ No newline at end of file diff --git a/regression-test/suites/insert_p0/test_insert_tablet_sink.groovy b/regression-test/suites/insert_p0/test_insert_tablet_sink.groovy new file mode 100644 index 000000000000000..e601362a4cde597 --- /dev/null +++ b/regression-test/suites/insert_p0/test_insert_tablet_sink.groovy @@ -0,0 +1,70 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_insert_tablet_sink") { + sql """drop table if exists `table_largeint`;""" + sql """drop table if exists `tmp_varchar`;""" + + sql """ + CREATE TABLE `tmp_varchar` ( + `k1` bigint(20) not NULL, + `c_varchar` varchar(65533) not NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`, c_varchar) + COMMENT 'OLAP' + AUTO PARTITION BY list (c_varchar) () + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + CREATE TABLE `table_largeint` ( + `k1` bigint(20) not NULL, + `c_largeint` largeint not NULL, + str string null + ) ENGINE=OLAP + UNIQUE KEY(`k1`, c_largeint) + COMMENT 'OLAP' + AUTO PARTITION BY list (c_largeint) () + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + + """ + + sql """insert into tmp_varchar values(1, "170141183460469231731687303715884105727")""" + sql """insert into tmp_varchar values(2, "-170141183460469231731687303715884105728")""" + sql """insert into tmp_varchar values(3,'333');""" + sql """insert into tmp_varchar values(4,'444');""" + sql """insert into tmp_varchar values(5,'555');""" + sql """insert into tmp_varchar values(6,'666');""" + + qt_select """select * from tmp_varchar order by 1;""" + + sql """ set skip_delete_bitmap = true; """ + sql """ set enable_memtable_on_sink_node = true; """ + sql """ set parallel_pipeline_task_num = 2; """ + + + sql """ insert into table_largeint select k1,c_varchar,cast(rand() * 50000000 as bigint) from tmp_varchar where k1>=3; """ + explain { + sql "insert into table_largeint select k1,c_varchar,cast(rand() * 50000000 as bigint) from tmp_varchar;" + contains "TABLET_SINK_SHUFFLE_PARTITIONED" + } + + sql """ insert into table_largeint select k1,c_varchar,cast(rand() * 50000000 as bigint) from tmp_varchar; """ + qt_select """select k1,c_largeint from table_largeint order by 1;""" +} diff --git a/regression-test/suites/mtmv_p0/test_mtmv_property.groovy b/regression-test/suites/mtmv_p0/test_mtmv_property.groovy index e0ffbca8dbf63c2..4104f1480e88b7f 100644 --- a/regression-test/suites/mtmv_p0/test_mtmv_property.groovy +++ b/regression-test/suites/mtmv_p0/test_mtmv_property.groovy @@ -60,7 +60,8 @@ suite("test_mtmv_property","mtmv") { def showCreateTableResult = sql """show create materialized view ${mvName}""" logger.info("showCreateTableResult: " + showCreateTableResult.toString()) - assertTrue(showCreateTableResult.toString().contains('tag.location.default: 1')) + // Cannot compare the number of replicas to 1, as the pipeline may force the number of replicas to be set + assertTrue(showCreateTableResult.toString().contains('tag.location.default:')) assertTrue(showCreateTableResult.toString().contains('"min_load_replica_num" = "-1"')) assertTrue(showCreateTableResult.toString().contains('"storage_medium" = "hdd"')) assertTrue(showCreateTableResult.toString().contains('"store_row_column" = "true"')) diff --git a/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy b/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy index 7721130d5586f0d..e994e22ec997096 100644 --- a/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy +++ b/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy @@ -25,4 +25,6 @@ suite("test_show_commands_nereids") { checkNereidsExecute("""show frontends;""") checkNereidsExecute("""show backends;""") checkNereidsExecute("""show whitelist;""") + checkNereidsExecute("""show triggers;""") + } diff --git a/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/SimpleDemo.java b/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/SimpleDemo.java new file mode 100644 index 000000000000000..0b7f51372cc4f3b --- /dev/null +++ b/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/SimpleDemo.java @@ -0,0 +1,88 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.udf; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.logging.Logger; + +public class SimpleDemo { + + Logger log = Logger.getLogger("SimpleDemo"); + + //Need an inner class to store data + /*required*/ + public static class State { + /*some variables if you need */ + public int sum = 0; + } + + /*required*/ + public State create() { + /* here could do some init work if needed */ + return new State(); + } + + /*required*/ + public void destroy(State state) { + /* here could do some destroy work if needed */ + } + + /*Not Required*/ + public void reset(State state) { + /*if you want this udaf function can work with window function.*/ + /*Must impl this, it will be reset to init state after calculate every window frame*/ + state.sum = 0; + } + + /*required*/ + //first argument is State, then other types your input + public void add(State state, Integer val) throws Exception { + /* here doing update work when input data*/ + if (val != null) { + state.sum += val; + } + } + + /*required*/ + public void serialize(State state, DataOutputStream out) throws IOException { + /* serialize some data into buffer */ + out.writeInt(state.sum); + } + + /*required*/ + public void deserialize(State state, DataInputStream in) throws IOException { + /* deserialize get data from buffer before you put */ + int val = in.readInt(); + state.sum = val; + } + + /*required*/ + public void merge(State state, State rhs) throws Exception { + /* merge data from state */ + state.sum += rhs.sum; + } + + /*required*/ + //return Type you defined + public Integer getValue(State state) throws Exception { + /* return finally result */ + return state.sum; + } +} \ No newline at end of file diff --git a/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/UDTFStringTest.java b/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/UDTFStringTest.java new file mode 100644 index 000000000000000..cb2eb45c9c1f922 --- /dev/null +++ b/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/UDTFStringTest.java @@ -0,0 +1,31 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.udf; + +import java.util.ArrayList; +import java.util.Arrays; + +public class UDTFStringTest { + public ArrayList evaluate(String value, String separator) { + if (value == null || separator == null) { + return null; + } else { + return new ArrayList<>(Arrays.asList(value.split(separator))); + } + } +}