diff --git a/.github/workflows/build-extension.yml b/.github/workflows/build-extension.yml index d58e21a75dc0ea..7c7dc5fb8b4f39 100644 --- a/.github/workflows/build-extension.yml +++ b/.github/workflows/build-extension.yml @@ -61,7 +61,7 @@ jobs: uses: actions/setup-java@v2 with: distribution: adopt - java-version: '8' + java-version: '17' - name: Setup thrift run: | diff --git a/be/src/agent/heartbeat_server.cpp b/be/src/agent/heartbeat_server.cpp index dfeb05a932d389..e6d893a4a2d9ba 100644 --- a/be/src/agent/heartbeat_server.cpp +++ b/be/src/agent/heartbeat_server.cpp @@ -30,6 +30,7 @@ #include "common/status.h" #include "olap/storage_engine.h" #include "runtime/exec_env.h" +#include "runtime/fragment_mgr.h" #include "runtime/heartbeat_flags.h" #include "service/backend_options.h" #include "util/debug_util.h" @@ -83,6 +84,10 @@ void HeartbeatServer::heartbeat(THeartbeatResult& heartbeat_result, heartbeat_result.backend_info.__set_be_node_role(config::be_node_role); // If be is gracefully stop, then k_doris_exist is set to true heartbeat_result.backend_info.__set_is_shutdown(doris::k_doris_exit); + heartbeat_result.backend_info.__set_fragment_executing_count( + get_fragment_executing_count()); + heartbeat_result.backend_info.__set_fragment_last_active_time( + get_fragment_last_active_time()); } watch.stop(); if (watch.elapsed_time() > 1000L * 1000L * 1000L) { diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index be4d3b137e6453..5f24d8ce054aa1 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -76,6 +76,7 @@ #include "olap/txn_manager.h" #include "olap/utils.h" #include "runtime/exec_env.h" +#include "runtime/fragment_mgr.h" #include "runtime/snapshot_loader.h" #include "service/backend_options.h" #include "util/doris_metrics.h" @@ -446,7 +447,6 @@ void add_task_count(const TAgentTaskRequest& task, int n) { ADD_TASK_COUNT(PUBLISH_VERSION) ADD_TASK_COUNT(CLEAR_TRANSACTION_TASK) ADD_TASK_COUNT(UPDATE_TABLET_META_INFO) - ADD_TASK_COUNT(ALTER) ADD_TASK_COUNT(CLONE) ADD_TASK_COUNT(STORAGE_MEDIUM_MIGRATE) ADD_TASK_COUNT(GC_BINLOG) @@ -459,6 +459,17 @@ void add_task_count(const TAgentTaskRequest& task, int n) { DELETE_count << n; } return; + case TTaskType::ALTER: + { + ALTER_count << n; + // cloud auto stop need sc jobs, a tablet's sc can also be considered a fragment + doris::g_fragment_executing_count << 1; + int64 now = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); + g_fragment_last_active_time.set_value(now); + return; + } default: return; } @@ -1387,12 +1398,14 @@ void update_s3_resource(const TStorageResource& param, io::RemoteFileSystemSPtr void update_hdfs_resource(const TStorageResource& param, io::RemoteFileSystemSPtr existed_fs) { Status st; io::RemoteFileSystemSPtr fs; + std::string root_path = + param.hdfs_storage_param.__isset.root_path ? param.hdfs_storage_param.root_path : ""; if (!existed_fs) { // No such FS instance on BE - auto res = io::HdfsFileSystem::create(param.hdfs_storage_param, - param.hdfs_storage_param.fs_name, - std::to_string(param.id), nullptr); + auto res = io::HdfsFileSystem::create( + param.hdfs_storage_param, param.hdfs_storage_param.fs_name, + std::to_string(param.id), nullptr, std::move(root_path)); if (!res.has_value()) { st = std::move(res).error(); } else { @@ -1410,7 +1423,8 @@ void update_hdfs_resource(const TStorageResource& param, io::RemoteFileSystemSPt } else { LOG_INFO("successfully update hdfs resource") .tag("resource_id", param.id) - .tag("resource_name", param.name); + .tag("resource_name", param.name) + .tag("root_path", fs->root_path().string()); put_storage_resource(param.id, {std::move(fs), param.version}); } } @@ -1851,6 +1865,11 @@ void alter_tablet_callback(StorageEngine& engine, const TAgentTaskRequest& req) alter_tablet(engine, req, signature, task_type, &finish_task_request); finish_task(finish_task_request); } + doris::g_fragment_executing_count << -1; + int64 now = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); + g_fragment_last_active_time.set_value(now); remove_task_info(req.task_type, req.signature); } @@ -1872,6 +1891,11 @@ void alter_cloud_tablet_callback(CloudStorageEngine& engine, const TAgentTaskReq alter_cloud_tablet(engine, req, signature, task_type, &finish_task_request); finish_task(finish_task_request); } + doris::g_fragment_executing_count << -1; + int64 now = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); + g_fragment_last_active_time.set_value(now); remove_task_info(req.task_type, req.signature); } diff --git a/be/src/cloud/cloud_rowset_builder.cpp b/be/src/cloud/cloud_rowset_builder.cpp index 72010fa6e9c53c..d8bb50dc5f2a00 100644 --- a/be/src/cloud/cloud_rowset_builder.cpp +++ b/be/src/cloud/cloud_rowset_builder.cpp @@ -65,6 +65,7 @@ Status CloudRowsetBuilder::init() { context.mow_context = mow_context; context.write_file_cache = _req.write_file_cache; context.partial_update_info = _partial_update_info; + context.file_cache_ttl_sec = _tablet->ttl_seconds(); // New loaded data is always written to latest shared storage // TODO(AlexYue): use the passed resource id to retrive the corresponding // fs to pass to the RowsetWriterContext diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index 92e6699ceec4d6..08ed09d90f5055 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -95,11 +95,6 @@ Status CloudTablet::capture_rs_readers(const Version& spec_version, return capture_rs_readers_unlocked(version_path, rs_splits); } -Status CloudTablet::sync_meta() { - // TODO(lightman): FileCache - return Status::NotSupported("CloudTablet::sync_meta is not implemented"); -} - // There are only two tablet_states RUNNING and NOT_READY in cloud mode // This function will erase the tablet from `CloudTabletMgr` when it can't find this tablet in MS. Status CloudTablet::sync_rowsets(int64_t query_version, bool warmup_delta_data) { @@ -618,4 +613,82 @@ Status CloudTablet::calc_delete_bitmap_for_compaciton( return Status::OK(); } +Status CloudTablet::sync_meta() { + if (!config::enable_file_cache) { + return Status::OK(); + } + + TabletMetaSharedPtr tablet_meta; + auto st = _engine.meta_mgr().get_tablet_meta(tablet_id(), &tablet_meta); + if (!st.ok()) { + if (st.is()) { + // TODO(Lchangliang): recycle_resources_by_self(); + } + return st; + } + if (tablet_meta->tablet_state() != TABLET_RUNNING) { // impossible + return Status::InternalError("invalid tablet state. tablet_id={}", tablet_id()); + } + + auto new_ttl_seconds = tablet_meta->ttl_seconds(); + if (_tablet_meta->ttl_seconds() != new_ttl_seconds) { + _tablet_meta->set_ttl_seconds(new_ttl_seconds); + int64_t cur_time = UnixSeconds(); + std::shared_lock rlock(_meta_lock); + for (auto& [_, rs] : _rs_version_map) { + for (int seg_id = 0; seg_id < rs->num_segments(); ++seg_id) { + int64_t new_expiration_time = + new_ttl_seconds + rs->rowset_meta()->newest_write_timestamp(); + new_expiration_time = new_expiration_time > cur_time ? new_expiration_time : 0; + auto file_key = io::BlockFileCache::hash( + io::Path(rs->segment_file_path(seg_id)).filename().native()); + auto* file_cache = io::FileCacheFactory::instance()->get_by_path(file_key); + file_cache->modify_expiration_time(file_key, new_expiration_time); + } + } + } + + auto new_compaction_policy = tablet_meta->compaction_policy(); + if (_tablet_meta->compaction_policy() != new_compaction_policy) { + _tablet_meta->set_compaction_policy(new_compaction_policy); + } + auto new_time_series_compaction_goal_size_mbytes = + tablet_meta->time_series_compaction_goal_size_mbytes(); + if (_tablet_meta->time_series_compaction_goal_size_mbytes() != + new_time_series_compaction_goal_size_mbytes) { + _tablet_meta->set_time_series_compaction_goal_size_mbytes( + new_time_series_compaction_goal_size_mbytes); + } + auto new_time_series_compaction_file_count_threshold = + tablet_meta->time_series_compaction_file_count_threshold(); + if (_tablet_meta->time_series_compaction_file_count_threshold() != + new_time_series_compaction_file_count_threshold) { + _tablet_meta->set_time_series_compaction_file_count_threshold( + new_time_series_compaction_file_count_threshold); + } + auto new_time_series_compaction_time_threshold_seconds = + tablet_meta->time_series_compaction_time_threshold_seconds(); + if (_tablet_meta->time_series_compaction_time_threshold_seconds() != + new_time_series_compaction_time_threshold_seconds) { + _tablet_meta->set_time_series_compaction_time_threshold_seconds( + new_time_series_compaction_time_threshold_seconds); + } + auto new_time_series_compaction_empty_rowsets_threshold = + tablet_meta->time_series_compaction_empty_rowsets_threshold(); + if (_tablet_meta->time_series_compaction_empty_rowsets_threshold() != + new_time_series_compaction_empty_rowsets_threshold) { + _tablet_meta->set_time_series_compaction_empty_rowsets_threshold( + new_time_series_compaction_empty_rowsets_threshold); + } + auto new_time_series_compaction_level_threshold = + tablet_meta->time_series_compaction_level_threshold(); + if (_tablet_meta->time_series_compaction_level_threshold() != + new_time_series_compaction_level_threshold) { + _tablet_meta->set_time_series_compaction_level_threshold( + new_time_series_compaction_level_threshold); + } + + return Status::OK(); +} + } // namespace doris diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index ddf393d315c5ce..40335a3195620f 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -603,7 +603,7 @@ DEFINE_mInt32(result_buffer_cancelled_interval_time, "300"); DEFINE_mInt32(priority_queue_remaining_tasks_increased_frequency, "512"); // sync tablet_meta when modifying meta -DEFINE_mBool(sync_tablet_meta, "true"); +DEFINE_mBool(sync_tablet_meta, "false"); // default thrift rpc timeout ms DEFINE_mInt32(thrift_rpc_timeout_ms, "60000"); @@ -617,9 +617,9 @@ DEFINE_Bool(enable_metric_calculator, "true"); // max consumer num in one data consumer group, for routine load DEFINE_mInt32(max_consumer_num_per_group, "3"); -// the size of thread pool for routine load task. +// the max size of thread pool for routine load task. // this should be larger than FE config 'max_routine_load_task_num_per_be' (default 5) -DEFINE_Int32(routine_load_thread_pool_size, "10"); +DEFINE_Int32(max_routine_load_thread_pool_size, "1024"); // max external scan cache batch count, means cache max_memory_cache_batch_count * batch_size row // default is 20, batch_size's default value is 1024 means 20 * 1024 rows will be cached @@ -979,6 +979,7 @@ DEFINE_String(inverted_index_searcher_cache_limit, "10%"); DEFINE_Bool(enable_write_index_searcher_cache, "true"); DEFINE_Bool(enable_inverted_index_cache_check_timestamp, "true"); DEFINE_Int32(inverted_index_fd_number_limit_percent, "40"); // 40% +DEFINE_Int32(inverted_index_query_cache_shards, "256"); // inverted index match bitmap cache size DEFINE_String(inverted_index_query_cache_limit, "10%"); diff --git a/be/src/common/config.h b/be/src/common/config.h index e7b258ddbbdd02..b7c0d315c51e65 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -667,9 +667,9 @@ DECLARE_Bool(enable_metric_calculator); // max consumer num in one data consumer group, for routine load DECLARE_mInt32(max_consumer_num_per_group); -// the size of thread pool for routine load task. +// the max size of thread pool for routine load task. // this should be larger than FE config 'max_routine_load_task_num_per_be' (default 5) -DECLARE_Int32(routine_load_thread_pool_size); +DECLARE_Int32(max_routine_load_thread_pool_size); // max external scan cache batch count, means cache max_memory_cache_batch_count * batch_size row // default is 20, batch_size's default value is 1024 means 20 * 1024 rows will be cached @@ -1027,6 +1027,7 @@ DECLARE_String(inverted_index_searcher_cache_limit); DECLARE_Bool(enable_write_index_searcher_cache); DECLARE_Bool(enable_inverted_index_cache_check_timestamp); DECLARE_Int32(inverted_index_fd_number_limit_percent); // 50% +DECLARE_Int32(inverted_index_query_cache_shards); // inverted index match bitmap cache size DECLARE_String(inverted_index_query_cache_limit); diff --git a/be/src/io/cache/block_file_cache.h b/be/src/io/cache/block_file_cache.h index 36aa94acacfc30..4de78f228e6f86 100644 --- a/be/src/io/cache/block_file_cache.h +++ b/be/src/io/cache/block_file_cache.h @@ -21,6 +21,7 @@ #include #include +#include #include "io/cache/file_block.h" #include "io/cache/file_cache_common.h" diff --git a/be/src/io/fs/hdfs_file_system.cpp b/be/src/io/fs/hdfs_file_system.cpp index 892ee836ae6e10..f988bdf4904928 100644 --- a/be/src/io/fs/hdfs_file_system.cpp +++ b/be/src/io/fs/hdfs_file_system.cpp @@ -53,14 +53,15 @@ namespace doris::io { Result> HdfsFileSystem::create( const std::map& properties, std::string fs_name, std::string id, - RuntimeProfile* profile) { + RuntimeProfile* profile, std::string root_path) { return HdfsFileSystem::create(parse_properties(properties), std::move(fs_name), std::move(id), - profile); + profile, std::move(root_path)); } Result> HdfsFileSystem::create(const THdfsParams& hdfs_params, std::string fs_name, std::string id, - RuntimeProfile* profile) { + RuntimeProfile* profile, + std::string root_path) { #ifdef USE_HADOOP_HDFS if (!config::enable_java_support) { return ResultError(Status::InternalError( @@ -68,15 +69,15 @@ Result> HdfsFileSystem::create(const THdfsParams "true.")); } #endif - std::shared_ptr fs( - new HdfsFileSystem(hdfs_params, std::move(fs_name), std::move(id), profile)); + std::shared_ptr fs(new HdfsFileSystem( + hdfs_params, std::move(fs_name), std::move(id), profile, std::move(root_path))); RETURN_IF_ERROR_RESULT(fs->init()); return fs; } HdfsFileSystem::HdfsFileSystem(const THdfsParams& hdfs_params, std::string fs_name, std::string id, - RuntimeProfile* profile) - : RemoteFileSystem("", std::move(id), FileSystemType::HDFS), + RuntimeProfile* profile, std::string root_path) + : RemoteFileSystem(root_path, std::move(id), FileSystemType::HDFS), _hdfs_params(hdfs_params), _fs_name(std::move(fs_name)), _profile(profile) { diff --git a/be/src/io/fs/hdfs_file_system.h b/be/src/io/fs/hdfs_file_system.h index ec6401964eea45..23ae65b0820ef4 100644 --- a/be/src/io/fs/hdfs_file_system.h +++ b/be/src/io/fs/hdfs_file_system.h @@ -48,11 +48,12 @@ class HdfsFileSystem final : public RemoteFileSystem { public: static Result> create(const THdfsParams& hdfs_params, std::string fs_name, std::string id, - RuntimeProfile* profile); + RuntimeProfile* profile, + std::string root_path = ""); static Result> create( const std::map& properties, std::string fs_name, - std::string id, RuntimeProfile* profile); + std::string id, RuntimeProfile* profile, std::string root_path = ""); ~HdfsFileSystem() override; @@ -84,7 +85,7 @@ class HdfsFileSystem final : public RemoteFileSystem { private: friend class HdfsFileWriter; HdfsFileSystem(const THdfsParams& hdfs_params, std::string fs_name, std::string id, - RuntimeProfile* profile); + RuntimeProfile* profile, std::string root_path); const THdfsParams& _hdfs_params; // Only used in init, so we can use reference here std::string _fs_name; // do not use std::shared_ptr or std::unique_ptr diff --git a/be/src/io/fs/s3_file_writer.cpp b/be/src/io/fs/s3_file_writer.cpp index b7a8eb043115dc..388aed1ea34eec 100644 --- a/be/src/io/fs/s3_file_writer.cpp +++ b/be/src/io/fs/s3_file_writer.cpp @@ -508,11 +508,12 @@ void S3FileWriter::_put_object(UploadFileBuffer& buf) { } std::string S3FileWriter::_dump_completed_part() const { - std::string view; + std::stringstream ss; + ss << "part_numbers:"; for (const auto& part : _completed_parts) { - view.append(fmt::format("part {}, ", view, part->GetPartNumber())); + ss << " " << part->GetPartNumber(); } - return view; + return ss.str(); } } // namespace doris::io diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index f32139b989bdfa..347f960841c41f 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -60,6 +60,7 @@ class BaseTablet { int32_t schema_hash() const { return _tablet_meta->schema_hash(); } KeysType keys_type() const { return _tablet_meta->tablet_schema()->keys_type(); } size_t num_key_columns() const { return _tablet_meta->tablet_schema()->num_key_columns(); } + int64_t ttl_seconds() const { return _tablet_meta->ttl_seconds(); } std::mutex& get_schema_change_lock() { return _schema_change_lock; } bool enable_unique_key_merge_on_write() const { #ifdef BE_TEST diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 141cdce24aea14..2947a7a5fc2845 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -617,7 +617,8 @@ Status CompactionMixin::do_inverted_index_compaction() { // we choose the first destination segment name as the temporary index writer path // Used to distinguish between different index compaction - auto index_tmp_path = tablet_path + "/" + dest_rowset_id.to_string() + "_" + "tmp"; + auto tmp_file_dir = ExecEnv::GetInstance()->get_tmp_file_dirs()->get_tmp_file_dir(); + auto index_tmp_path = tmp_file_dir / dest_rowset_id.to_string(); LOG(INFO) << "start index compaction" << ". tablet=" << _tablet->tablet_id() << ", source index size=" << src_segment_num << ", destination index size=" << dest_segment_num << "."; diff --git a/be/src/olap/delete_handler.cpp b/be/src/olap/delete_handler.cpp index d2126bbdf3e432..ca28d07254e909 100644 --- a/be/src/olap/delete_handler.cpp +++ b/be/src/olap/delete_handler.cpp @@ -80,7 +80,15 @@ Status DeleteHandler::generate_delete_predicate(const TabletSchema& schema, << "condition size=" << in_pred->values().size(); } else { // write sub predicate v1 for compactbility - del_pred->add_sub_predicates(construct_sub_predicate(condition)); + std::string condition_str = construct_sub_predicate(condition); + if (TCondition tmp; !DeleteHandler::parse_condition(condition_str, &tmp)) { + LOG(WARNING) << "failed to parse condition_str, condtion=" + << ThriftDebugString(condition); + return Status::Error( + "failed to parse condition_str, condtion={}", ThriftDebugString(condition)); + } + VLOG_NOTICE << __PRETTY_FUNCTION__ << " condition_str: " << condition_str; + del_pred->add_sub_predicates(condition_str); DeleteSubPredicatePB* sub_predicate = del_pred->add_sub_predicates_v2(); if (condition.__isset.column_unique_id) { sub_predicate->set_column_unique_id(condition.column_unique_id); @@ -127,8 +135,9 @@ std::string DeleteHandler::construct_sub_predicate(const TCondition& condition) } string condition_str; if ("IS" == op) { - condition_str = condition.column_name + " " + op + " " + condition.condition_values[0]; - } else { + // ATTN: tricky! Surround IS with spaces to make it "special" + condition_str = condition.column_name + " IS " + condition.condition_values[0]; + } else { // multi-elements IN expr has been processed with InPredicatePB if (op == "*=") { op = "="; } else if (op == "!*=") { @@ -286,44 +295,50 @@ Status DeleteHandler::parse_condition(const DeleteSubPredicatePB& sub_cond, TCon return Status::OK(); } +// clang-format off +// Condition string format, the format is (column_name)(op)(value) +// eg: condition_str="c1 = 1597751948193618247 and length(source)<1;\n;\n" +// column_name: matches "c1", must include FeNameFormat.java COLUMN_NAME_REGEX +// and compactible with any the lagacy +// operator: matches "=" +// value: matches "1597751948193618247 and length(source)<1;\n;\n" +// +// For more info, see DeleteHandler::construct_sub_predicates +// FIXME(gavin): support unicode. And this is a tricky implementation, it should +// not be the final resolution, refactor it. +const char* const CONDITION_STR_PATTERN = + // .----------------- column-name ----------------. .----------------------- operator ------------------------. .------------ value ----------. + R"(([_a-zA-Z@0-9\s/][.a-zA-Z0-9_+-/?@#$%^&*"\s,:]*)\s*((?:=)|(?:!=)|(?:>>)|(?:<<)|(?:>=)|(?:<=)|(?:\*=)|(?: IS ))\s*('((?:[\s\S]+)?)'|(?:[\s\S]+)?))"; + // '----------------- group 1 --------------------' '--------------------- group 2 ---------------------------' | '-- group 4--' | + // match any of: = != >> << >= <= *= " IS " '----------- group 3 ---------' + // match **ANY THING** without(4) + // or with(3) single quote +boost::regex DELETE_HANDLER_REGEX(CONDITION_STR_PATTERN); +// clang-format on + Status DeleteHandler::parse_condition(const std::string& condition_str, TCondition* condition) { - bool matched = true; + bool matched = false; boost::smatch what; - try { - // Condition string format, the format is (column_name)(op)(value) - // eg: condition_str="c1 = 1597751948193618247 and length(source)<1;\n;\n" - // group1: (\w+) matches "c1" - // group2: ((?:=)|(?:!=)|(?:>>)|(?:<<)|(?:>=)|(?:<=)|(?:\*=)|(?:IS)) matches "=" - // group3: ((?:[\s\S]+)?) matches "1597751948193618247 and length(source)<1;\n;\n" - const char* const CONDITION_STR_PATTERN = - R"(([\w$#%]+)\s*((?:=)|(?:!=)|(?:>>)|(?:<<)|(?:>=)|(?:<=)|(?:\*=)|(?:IS))\s*('((?:[\s\S]+)?)'|(?:[\s\S]+)?))"; - boost::regex ex(CONDITION_STR_PATTERN); - if (boost::regex_match(condition_str, what, ex)) { - if (condition_str.size() != what[0].str().size()) { - matched = false; - } - } else { - matched = false; - } + VLOG_NOTICE << "condition_str: " << condition_str; + matched = boost::regex_match(condition_str, what, DELETE_HANDLER_REGEX) && + condition_str.size() == what[0].str().size(); // exact match } catch (boost::regex_error& e) { VLOG_NOTICE << "fail to parse expr. [expr=" << condition_str << "; error=" << e.what() << "]"; - matched = false; } - if (!matched) { return Status::Error("fail to sub condition. condition={}", condition_str); } - condition->column_name = what[1].str(); - condition->condition_op = what[2].str(); - if (what[4].matched) { // match string with single quotes, eg. a = 'b' - condition->condition_values.push_back(what[4].str()); - } else { // match string without quote, compat with old conditions, eg. a = b - condition->condition_values.push_back(what[3].str()); - } + condition->column_name = what[1].str(); + condition->condition_op = what[2].str() == " IS " ? "IS" : what[2].str(); + // match string with single quotes, a = b or a = 'b' + condition->condition_values.push_back(what[3 + !!what[4].matched].str()); + VLOG_NOTICE << "parsed condition_str: col_name={" << condition->column_name << "} op={" + << condition->condition_op << "} val={" << condition->condition_values.back() + << "}"; return Status::OK(); } diff --git a/be/src/olap/delete_handler.h b/be/src/olap/delete_handler.h index 45be5d73ffefff..bce18669c58e9f 100644 --- a/be/src/olap/delete_handler.h +++ b/be/src/olap/delete_handler.h @@ -66,6 +66,15 @@ class DeleteHandler { static void convert_to_sub_pred_v2(DeletePredicatePB* delete_pred, TabletSchemaSPtr schema); + /** + * Use regular expression to extract 'column_name', 'op' and 'operands' + * + * @param condition_str input predicate string in form of `X OP Y` + * @param condition output param + * @return OK if matched and extracted correctly otherwise DELETE_INVALID_PARAMETERS + */ + static Status parse_condition(const std::string& condition_str, TCondition* condition); + private: // Validate the condition on the schema. static Status check_condition_valid(const TabletSchema& tablet_schema, const TCondition& cond); @@ -87,7 +96,6 @@ class DeleteHandler { // extract 'column_name', 'op' and 'operands' to condition static Status parse_condition(const DeleteSubPredicatePB& sub_cond, TCondition* condition); - static Status parse_condition(const std::string& condition_str, TCondition* condition); public: DeleteHandler() = default; diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index c08705861df7ad..27a71a56b148fe 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -369,6 +369,9 @@ struct OlapReaderStatistics { io::FileCacheStatistics file_cache_stats; int64_t load_segments_timer = 0; + + int64_t collect_iterator_merge_next_timer = 0; + int64_t collect_iterator_normal_next_timer = 0; }; using ColumnId = uint32_t; diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp index c288d3b42e118d..d0cd43172a0917 100644 --- a/be/src/olap/olap_server.cpp +++ b/be/src/olap/olap_server.cpp @@ -17,6 +17,7 @@ #include #include +#include #include #include @@ -1403,22 +1404,25 @@ void StorageEngine::_cold_data_compaction_producer_callback() { for (auto& [tablet, score] : tablet_to_follow) { LOG(INFO) << "submit to follow cooldown meta. tablet_id=" << tablet->tablet_id() << " score=" << score; - static_cast( - _cold_data_compaction_thread_pool->submit_func([&, t = std::move(tablet)]() { - { - std::lock_guard lock(tablet_submitted_mtx); - tablet_submitted.insert(t->tablet_id()); - } - auto st = t->cooldown(); - { - std::lock_guard lock(tablet_submitted_mtx); - tablet_submitted.erase(t->tablet_id()); - } - if (!st.ok()) { - LOG(WARNING) << "failed to cooldown. tablet_id=" << t->tablet_id() - << " err=" << st; - } - })); + static_cast(_cold_data_compaction_thread_pool->submit_func([&, + t = std::move( + tablet)]() { + { + std::lock_guard lock(tablet_submitted_mtx); + tablet_submitted.insert(t->tablet_id()); + } + auto st = t->cooldown(); + { + std::lock_guard lock(tablet_submitted_mtx); + tablet_submitted.erase(t->tablet_id()); + } + if (!st.ok()) { + // The cooldown of the replica may be relatively slow + // resulting in a short period of time where following cannot be successful + LOG_EVERY_N(WARNING, 5) + << "failed to cooldown. tablet_id=" << t->tablet_id() << " err=" << st; + } + })); } } } diff --git a/be/src/olap/rowset/beta_rowset.h b/be/src/olap/rowset/beta_rowset.h index 46f16f4d50230f..41cecadf783747 100644 --- a/be/src/olap/rowset/beta_rowset.h +++ b/be/src/olap/rowset/beta_rowset.h @@ -49,7 +49,7 @@ class BetaRowset final : public Rowset { Status create_reader(RowsetReaderSharedPtr* result) override; - std::string segment_file_path(int segment_id) const; + std::string segment_file_path(int segment_id) const override; static std::string segment_file_path(const std::string& rowset_dir, const RowsetId& rowset_id, int segment_id); diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp b/be/src/olap/rowset/beta_rowset_reader.cpp index a0dff2613aec0a..f5e0b8e5c62b49 100644 --- a/be/src/olap/rowset/beta_rowset_reader.cpp +++ b/be/src/olap/rowset/beta_rowset_reader.cpp @@ -238,6 +238,14 @@ Status BetaRowsetReader::get_segment_iterators(RowsetReaderContext* read_context _read_context->runtime_state->query_options().disable_file_cache; } + _read_options.io_ctx.expiration_time = + read_context->ttl_seconds == 0 + ? 0 + : _rowset->rowset_meta()->newest_write_timestamp() + read_context->ttl_seconds; + if (_read_options.io_ctx.expiration_time <= UnixSeconds()) { + _read_options.io_ctx.expiration_time = 0; + } + // load segments bool should_use_cache = use_cache || _read_context->reader_type == ReaderType::READER_QUERY; RETURN_IF_ERROR(SegmentLoader::instance()->load_segments(_rowset, &_segment_cache_handle, diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h index fd5264f3ac048e..f15527ddd6b1bd 100644 --- a/be/src/olap/rowset/rowset.h +++ b/be/src/olap/rowset/rowset.h @@ -169,6 +169,8 @@ class Rowset : public std::enable_shared_from_this { // TODO should we rename the method to remove_files() to be more specific? virtual Status remove() = 0; + virtual std::string segment_file_path(int segment_id) const = 0; + // close to clear the resource owned by rowset // including: open files, indexes and so on // NOTICE: can not call this function in multithreads diff --git a/be/src/olap/rowset/rowset_reader_context.h b/be/src/olap/rowset/rowset_reader_context.h index d5683924a9ec0c..8bfdeda60a8568 100644 --- a/be/src/olap/rowset/rowset_reader_context.h +++ b/be/src/olap/rowset/rowset_reader_context.h @@ -83,6 +83,7 @@ struct RowsetReaderContext { RowsetId rowset_id; // slots that cast may be eliminated in storage layer std::map target_cast_type_for_variants; + int64_t ttl_seconds = 0; }; } // namespace doris diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index 322ff03625100e..a15e809b8b8ad9 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -529,6 +529,7 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { _creation_time = tablet_meta_pb.creation_time(); _cumulative_layer_point = tablet_meta_pb.cumulative_layer_point(); _tablet_uid = TabletUid(tablet_meta_pb.tablet_uid()); + _ttl_seconds = tablet_meta_pb.ttl_seconds(); if (tablet_meta_pb.has_tablet_type()) { _tablet_type = tablet_meta_pb.tablet_type(); } else { @@ -647,6 +648,7 @@ void TabletMeta::to_meta_pb(TabletMetaPB* tablet_meta_pb) { tablet_meta_pb->set_cumulative_layer_point(cumulative_layer_point()); *(tablet_meta_pb->mutable_tablet_uid()) = tablet_uid().to_proto(); tablet_meta_pb->set_tablet_type(_tablet_type); + tablet_meta_pb->set_ttl_seconds(_ttl_seconds); switch (tablet_state()) { case TABLET_NOTREADY: tablet_meta_pb->set_tablet_state(PB_NOTREADY); diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index 58201c1c1f1d54..6c5233eac53e61 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -273,6 +273,16 @@ class TabletMeta { return _time_series_compaction_level_threshold; } + int64_t ttl_seconds() const { + std::shared_lock rlock(_meta_lock); + return _ttl_seconds; + } + + void set_ttl_seconds(int64_t ttl_seconds) { + std::lock_guard wlock(_meta_lock); + _ttl_seconds = ttl_seconds; + } + private: Status _save_meta(DataDir* data_dir); @@ -328,6 +338,9 @@ class TabletMeta { int64_t _time_series_compaction_empty_rowsets_threshold = 0; int64_t _time_series_compaction_level_threshold = 0; + // cloud + int64_t _ttl_seconds = 0; + mutable std::shared_mutex _meta_lock; }; diff --git a/be/src/olap/tablet_reader.cpp b/be/src/olap/tablet_reader.cpp index 7b40ff4eae1238..b2afd1360d4e4f 100644 --- a/be/src/olap/tablet_reader.cpp +++ b/be/src/olap/tablet_reader.cpp @@ -262,6 +262,7 @@ Status TabletReader::_capture_rs_readers(const ReaderParams& read_params) { _reader_context.common_expr_ctxs_push_down = read_params.common_expr_ctxs_push_down; _reader_context.output_columns = &read_params.output_columns; _reader_context.push_down_agg_type_opt = read_params.push_down_agg_type_opt; + _reader_context.ttl_seconds = _tablet->ttl_seconds(); return Status::OK(); } diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h index 372e3a5a0ca6cf..4c15194362e193 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h @@ -97,7 +97,7 @@ class DistinctStreamingAggOperatorX final bool need_more_input_data(RuntimeState* state) const override; DataDistribution required_data_distribution() const override { - if (_needs_finalize || !_probe_expr_ctxs.empty()) { + if (_needs_finalize || (!_probe_expr_ctxs.empty() && !_is_streaming_preagg)) { return _is_colocate ? DataDistribution(ExchangeType::BUCKET_HASH_SHUFFLE, _partition_exprs) : DataDistribution(ExchangeType::HASH_SHUFFLE, _partition_exprs); diff --git a/be/src/pipeline/exec/join_probe_operator.cpp b/be/src/pipeline/exec/join_probe_operator.cpp index 48607309eca42d..5c89075e8b9a98 100644 --- a/be/src/pipeline/exec/join_probe_operator.cpp +++ b/be/src/pipeline/exec/join_probe_operator.cpp @@ -86,7 +86,12 @@ Status JoinProbeLocalState::_build_output_block( // In previous versions, the join node had a separate set of project structures, // and you could see a 'todo' in the Thrift definition. // Here, we have refactored it, but considering upgrade compatibility, we still need to retain the old code. - *output_block = *origin_block; + if (!output_block->mem_reuse()) { + vectorized::MutableBlock tmp( + vectorized::VectorizedUtils::create_columns_with_type_and_name(p.row_desc())); + output_block->swap(tmp.to_block()); + } + output_block->swap(*origin_block); return Status::OK(); } SCOPED_TIMER(_build_output_block_timer); diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp b/be/src/pipeline/exec/olap_scan_operator.cpp index e33293c721506f..9e94d2118eedc1 100644 --- a/be/src/pipeline/exec/olap_scan_operator.cpp +++ b/be/src/pipeline/exec/olap_scan_operator.cpp @@ -67,6 +67,8 @@ Status OlapScanLocalState::_init_profile() { _block_conditions_filtered_timer = ADD_TIMER(_segment_profile, "BlockConditionsFilteredTime"); _block_conditions_filtered_bf_timer = ADD_TIMER(_segment_profile, "BlockConditionsFilteredBloomFilterTime"); + _collect_iterator_merge_next_timer = ADD_TIMER(_segment_profile, "CollectIteratorMergeTime"); + _collect_iterator_normal_next_timer = ADD_TIMER(_segment_profile, "CollectIteratorNormalTime"); _block_conditions_filtered_zonemap_timer = ADD_TIMER(_segment_profile, "BlockConditionsFilteredZonemapTime"); _block_conditions_filtered_zonemap_rp_timer = diff --git a/be/src/pipeline/exec/olap_scan_operator.h b/be/src/pipeline/exec/olap_scan_operator.h index 233283a59aa3cd..cc516780c09b8c 100644 --- a/be/src/pipeline/exec/olap_scan_operator.h +++ b/be/src/pipeline/exec/olap_scan_operator.h @@ -144,6 +144,8 @@ class OlapScanLocalState final : public ScanLocalState { RuntimeProfile::Counter* _block_init_seek_counter = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_timer = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_bf_timer = nullptr; + RuntimeProfile::Counter* _collect_iterator_merge_next_timer = nullptr; + RuntimeProfile::Counter* _collect_iterator_normal_next_timer = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_zonemap_timer = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_zonemap_rp_timer = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_dict_timer = nullptr; diff --git a/be/src/pipeline/exec/union_source_operator.cpp b/be/src/pipeline/exec/union_source_operator.cpp index 9ddf726700c311..de0348de508b65 100644 --- a/be/src/pipeline/exec/union_source_operator.cpp +++ b/be/src/pipeline/exec/union_source_operator.cpp @@ -165,8 +165,10 @@ Status UnionSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* b Defer set_eos {[&]() { //have executing const expr, queue have no data anymore, and child could be closed *eos = (_child_size == 0 && !local_state._need_read_for_const_expr) || - (_child_size > 0 && local_state._shared_state->data_queue.is_all_finish() && - !_has_data(state)); + // here should check `_has_data` first, or when `is_all_finish` is false, + // the data queue will have no chance to change the `_flag_queue_idx`. + (!_has_data(state) && _child_size > 0 && + local_state._shared_state->data_queue.is_all_finish()); }}; SCOPED_TIMER(local_state.exec_time_counter()); diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 2c46e4ab244e74..795cecd1337f42 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -234,6 +234,7 @@ Status ExecEnv::_init(const std::vector& store_paths, _function_client_cache = new BrpcClientCache(); _stream_load_executor = StreamLoadExecutor::create_shared(this); _routine_load_task_executor = new RoutineLoadTaskExecutor(this); + RETURN_IF_ERROR(_routine_load_task_executor->init()); _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); _block_spill_mgr = new BlockSpillManager(store_paths); _group_commit_mgr = new GroupCommitMgr(this); @@ -501,8 +502,8 @@ Status ExecEnv::_init_mem_env() { // Reason same as buffer_pool_limit inverted_index_query_cache_limit = inverted_index_query_cache_limit / 2; } - _inverted_index_query_cache = - InvertedIndexQueryCache::create_global_cache(inverted_index_query_cache_limit, 256); + _inverted_index_query_cache = InvertedIndexQueryCache::create_global_cache( + inverted_index_query_cache_limit, config::inverted_index_query_cache_shards); LOG(INFO) << "Inverted index query match cache memory limit: " << PrettyPrinter::print(inverted_index_cache_limit, TUnit::BYTES) << ", origin config value: " << config::inverted_index_query_cache_limit; diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index d858737d780350..190c0f5b0e1912 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -99,6 +99,19 @@ DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(fragment_thread_pool_queue_size, MetricUnit:: bvar::LatencyRecorder g_fragmentmgr_prepare_latency("doris_FragmentMgr", "prepare"); bvar::Adder g_pipeline_fragment_instances_count("doris_pipeline_fragment_instances_count"); +bvar::Adder g_fragment_executing_count("fragment_executing_count"); +bvar::Status g_fragment_last_active_time( + "fragment_last_active_time", duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count()); + +uint64_t get_fragment_executing_count() { + return g_fragment_executing_count.get_value(); +} +uint64_t get_fragment_last_active_time() { + return g_fragment_last_active_time.get_value(); +} + std::string to_load_error_http_path(const std::string& file_name) { if (file_name.empty()) { return ""; @@ -470,9 +483,15 @@ void FragmentMgr::_exec_actual(std::shared_ptr fragment_ex // remove exec state after this fragment finished { + int64 now = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); std::lock_guard lock(_lock); _fragment_instance_map.erase(fragment_executor->fragment_instance_id()); + g_fragment_executing_count << -1; + g_fragment_last_active_time.set_value(now); + LOG_INFO("Instance {} finished", print_id(fragment_executor->fragment_instance_id())); if (all_done && query_ctx) { @@ -584,6 +603,11 @@ void FragmentMgr::remove_pipeline_context( std::vector ins_ids; f_context->instance_ids(ins_ids); bool all_done = q_context->countdown(ins_ids.size()); + int64 now = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); + g_fragment_executing_count << -1; + g_fragment_last_active_time.set_value(now); for (const auto& ins_id : ins_ids) { LOG_INFO("Removing query {} instance {}, all done? {}", print_id(query_id), print_id(ins_id), all_done); @@ -733,7 +757,12 @@ Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params, static_cast(_runtimefilter_controller.add_entity( params.params, params.params.query_id, params.query_options, &handler, RuntimeFilterParamsContext::create(fragment_executor->runtime_state()))); + int64 now = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); { + g_fragment_executing_count << 1; + g_fragment_last_active_time.set_value(now); std::lock_guard lock(_lock); if (handler) { query_ctx->set_merge_controller_handler(handler); @@ -753,6 +782,8 @@ Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params, // Remove the exec state added std::lock_guard lock(_lock); _fragment_instance_map.erase(params.params.fragment_instance_id); + g_fragment_executing_count << -1; + g_fragment_last_active_time.set_value(now); } fragment_executor->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "push plan fragment to thread pool failed"); @@ -844,7 +875,12 @@ Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, query_ctx->set_ready_to_execute_only(); } } + int64 now = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); { + g_fragment_executing_count << 1; + g_fragment_last_active_time.set_value(now); std::lock_guard lock(_lock); std::vector ins_ids; reinterpret_cast(context.get()) @@ -905,7 +941,12 @@ Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, if (i == 0 && handler) { query_ctx->set_merge_controller_handler(handler); } + int64 now = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); { + g_fragment_executing_count << 1; + g_fragment_last_active_time.set_value(now); std::lock_guard lock(_lock); _pipeline_map.insert(std::make_pair(fragment_instance_id, context)); } diff --git a/be/src/runtime/fragment_mgr.h b/be/src/runtime/fragment_mgr.h index 3435d1f4f64e8b..5aef45954d3f9c 100644 --- a/be/src/runtime/fragment_mgr.h +++ b/be/src/runtime/fragment_mgr.h @@ -44,6 +44,8 @@ class IOBufAsZeroCopyInputStream; } namespace doris { +extern bvar::Adder g_fragment_executing_count; +extern bvar::Status g_fragment_last_active_time; namespace pipeline { class PipelineFragmentContext; @@ -202,4 +204,7 @@ class FragmentMgr : public RestMonitorIface { nullptr; // used for pipeliine context report }; +uint64_t get_fragment_executing_count(); +uint64_t get_fragment_last_active_time(); + } // namespace doris diff --git a/be/src/runtime/routine_load/routine_load_task_executor.cpp b/be/src/runtime/routine_load/routine_load_task_executor.cpp index 3e5eb48afbca80..2e8303ca4cc5b7 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.cpp +++ b/be/src/runtime/routine_load/routine_load_task_executor.cpp @@ -62,10 +62,7 @@ using namespace ErrorCode; DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(routine_load_task_count, MetricUnit::NOUNIT); RoutineLoadTaskExecutor::RoutineLoadTaskExecutor(ExecEnv* exec_env) - : _exec_env(exec_env), - _thread_pool(config::routine_load_thread_pool_size, config::routine_load_thread_pool_size, - "routine_load"), - _data_consumer_pool(config::routine_load_consumer_pool_size) { + : _exec_env(exec_env), _data_consumer_pool(config::routine_load_consumer_pool_size) { REGISTER_HOOK_METRIC(routine_load_task_count, [this]() { // std::lock_guard l(_lock); return _task_map.size(); @@ -79,10 +76,19 @@ RoutineLoadTaskExecutor::~RoutineLoadTaskExecutor() { _task_map.clear(); } +Status RoutineLoadTaskExecutor::init() { + return ThreadPoolBuilder("routine_load") + .set_min_threads(0) + .set_max_threads(config::max_routine_load_thread_pool_size) + .set_max_queue_size(config::max_routine_load_thread_pool_size) + .build(&_thread_pool); +} + void RoutineLoadTaskExecutor::stop() { DEREGISTER_HOOK_METRIC(routine_load_task_count); - _thread_pool.shutdown(); - _thread_pool.join(); + if (_thread_pool) { + _thread_pool->shutdown(); + } _data_consumer_pool.stop(); } @@ -180,10 +186,10 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { return Status::OK(); } - if (_task_map.size() >= config::routine_load_thread_pool_size) { + if (_task_map.size() >= config::max_routine_load_thread_pool_size) { LOG(INFO) << "too many tasks in thread pool. reject task: " << UniqueId(task.id) << ", job id: " << task.job_id - << ", queue size: " << _thread_pool.get_queue_size() + << ", queue size: " << _thread_pool->get_queue_size() << ", current tasks num: " << _task_map.size(); return Status::TooManyTasks("{}_{}", UniqueId(task.id).to_string(), BackendOptions::get_localhost()); @@ -259,7 +265,7 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { _task_map[ctx->id] = ctx; // offer the task to thread pool - if (!_thread_pool.offer(std::bind( + if (!_thread_pool->submit_func(std::bind( &RoutineLoadTaskExecutor::exec_task, this, ctx, &_data_consumer_pool, [this](std::shared_ptr ctx) { std::unique_lock l(_lock); diff --git a/be/src/runtime/routine_load/routine_load_task_executor.h b/be/src/runtime/routine_load/routine_load_task_executor.h index e4ad8be59214fa..b2a61612fe248e 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.h +++ b/be/src/runtime/routine_load/routine_load_task_executor.h @@ -27,8 +27,8 @@ #include #include "runtime/routine_load/data_consumer_pool.h" +#include "util/threadpool.h" #include "util/uid_util.h" -#include "util/work_thread_pool.hpp" namespace doris { @@ -51,6 +51,8 @@ class RoutineLoadTaskExecutor { ~RoutineLoadTaskExecutor(); + Status init(); + void stop(); // submit a routine load task @@ -81,7 +83,7 @@ class RoutineLoadTaskExecutor { private: ExecEnv* _exec_env = nullptr; - PriorityThreadPool _thread_pool; + std::unique_ptr _thread_pool; DataConsumerPool _data_consumer_pool; std::mutex _lock; diff --git a/be/src/vec/core/types.h b/be/src/vec/core/types.h index 8899b6ce017735..fd3e16e0a2b886 100644 --- a/be/src/vec/core/types.h +++ b/be/src/vec/core/types.h @@ -422,6 +422,13 @@ std::string decimal_to_string(const T& value, UInt32 scale) { return str; } +template +std::string decimal_to_string(const T& orig_value, UInt32 trunc_precision, UInt32 scale) { + T multiplier = decimal_scale_multiplier(trunc_precision); + T value = orig_value % multiplier; + return decimal_to_string(value, scale); +} + template size_t decimal_to_string(const T& value, char* dst, UInt32 scale, const T& scale_multiplier) { if (UNLIKELY(value == std::numeric_limits::min())) { @@ -621,6 +628,12 @@ struct Decimal { std::string to_string(UInt32 scale) const { return decimal_to_string(value, scale); } + // truncate to specified precision and scale, + // used by runtime filter only for now. + std::string to_string(UInt32 precision, UInt32 scale) const { + return decimal_to_string(value, precision, scale); + } + /** * Got the string representation of a decimal. * @param dst Store the result, should be pre-allocated. diff --git a/be/src/vec/exec/join/vjoin_node_base.cpp b/be/src/vec/exec/join/vjoin_node_base.cpp index 4697be19b84ee6..9b954811ee9a02 100644 --- a/be/src/vec/exec/join/vjoin_node_base.cpp +++ b/be/src/vec/exec/join/vjoin_node_base.cpp @@ -184,7 +184,11 @@ Status VJoinNodeBase::_build_output_block(Block* origin_block, Block* output_blo // In previous versions, the join node had a separate set of project structures, // and you could see a 'todo' in the Thrift definition. // Here, we have refactored it, but considering upgrade compatibility, we still need to retain the old code. - *output_block = *origin_block; + if (!output_block->mem_reuse()) { + MutableBlock tmp(VectorizedUtils::create_columns_with_type_and_name(row_desc())); + output_block->swap(tmp.to_block()); + } + output_block->swap(*origin_block); return Status::OK(); } auto is_mem_reuse = output_block->mem_reuse(); diff --git a/be/src/vec/exec/scan/new_olap_scan_node.cpp b/be/src/vec/exec/scan/new_olap_scan_node.cpp index 0e47bb2b027895..96316c98bd7822 100644 --- a/be/src/vec/exec/scan/new_olap_scan_node.cpp +++ b/be/src/vec/exec/scan/new_olap_scan_node.cpp @@ -115,6 +115,8 @@ Status NewOlapScanNode::_init_profile() { _block_conditions_filtered_timer = ADD_TIMER(_segment_profile, "BlockConditionsFilteredTime"); _block_conditions_filtered_bf_timer = ADD_TIMER(_segment_profile, "BlockConditionsFilteredBloomFilterTime"); + _collect_iterator_merge_next_timer = ADD_TIMER(_segment_profile, "CollectIteratorMergeTime"); + _collect_iterator_normal_next_timer = ADD_TIMER(_segment_profile, "CollectIteratorNormalTime"); _block_conditions_filtered_zonemap_timer = ADD_TIMER(_segment_profile, "BlockConditionsFilteredZonemapTime"); _block_conditions_filtered_zonemap_rp_timer = diff --git a/be/src/vec/exec/scan/new_olap_scan_node.h b/be/src/vec/exec/scan/new_olap_scan_node.h index e1861a3b619df1..0094b0e78b99ca 100644 --- a/be/src/vec/exec/scan/new_olap_scan_node.h +++ b/be/src/vec/exec/scan/new_olap_scan_node.h @@ -169,6 +169,8 @@ class NewOlapScanNode : public VScanNode { RuntimeProfile::Counter* _block_init_seek_counter = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_timer = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_bf_timer = nullptr; + RuntimeProfile::Counter* _collect_iterator_merge_next_timer = nullptr; + RuntimeProfile::Counter* _collect_iterator_normal_next_timer = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_zonemap_timer = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_zonemap_rp_timer = nullptr; RuntimeProfile::Counter* _block_conditions_filtered_dict_timer = nullptr; diff --git a/be/src/vec/exec/scan/new_olap_scanner.cpp b/be/src/vec/exec/scan/new_olap_scanner.cpp index b6e7b9da3f2401..c4b7f68160985d 100644 --- a/be/src/vec/exec/scan/new_olap_scanner.cpp +++ b/be/src/vec/exec/scan/new_olap_scanner.cpp @@ -218,6 +218,9 @@ Status NewOlapScanner::init() { Status NewOlapScanner::open(RuntimeState* state) { RETURN_IF_ERROR(VScanner::open(state)); + auto* timer = _parent ? ((NewOlapScanNode*)_parent)->_reader_init_timer + : ((pipeline::OlapScanLocalState*)_local_state)->_reader_init_timer; + SCOPED_TIMER(timer); auto res = _tablet_reader->init(_tablet_reader_params); if (!res.ok()) { @@ -585,6 +588,8 @@ void NewOlapScanner::_collect_profile_before_close() { COUNTER_UPDATE(Parent->_block_conditions_filtered_timer, stats.block_conditions_filtered_ns); \ COUNTER_UPDATE(Parent->_block_conditions_filtered_bf_timer, \ stats.block_conditions_filtered_bf_ns); \ + COUNTER_UPDATE(Parent->_collect_iterator_merge_next_timer, \ + stats.collect_iterator_merge_next_timer); \ COUNTER_UPDATE(Parent->_block_conditions_filtered_zonemap_timer, \ stats.block_conditions_filtered_zonemap_ns); \ COUNTER_UPDATE(Parent->_block_conditions_filtered_zonemap_rp_timer, \ diff --git a/be/src/vec/exec/scan/vscanner.cpp b/be/src/vec/exec/scan/vscanner.cpp index 152547e8a0286b..39a9059d1d37c8 100644 --- a/be/src/vec/exec/scan/vscanner.cpp +++ b/be/src/vec/exec/scan/vscanner.cpp @@ -91,7 +91,7 @@ Status VScanner::get_block(RuntimeState* state, Block* block, bool* eof) { SCOPED_RAW_TIMER(&_per_scanner_timer); int64_t rows_read_threshold = _num_rows_read + config::doris_scanner_row_num; if (!block->mem_reuse()) { - for (const auto slot_desc : _output_tuple_desc->slots()) { + for (auto* const slot_desc : _output_tuple_desc->slots()) { if (!slot_desc->need_materialize()) { // should be ignore from reading continue; @@ -112,7 +112,7 @@ Status VScanner::get_block(RuntimeState* state, Block* block, bool* eof) { // 1. Get input block from scanner { // get block time - auto timer = _parent ? _parent->_scan_timer : _local_state->_scan_timer; + auto* timer = _parent ? _parent->_scan_timer : _local_state->_scan_timer; SCOPED_TIMER(timer); RETURN_IF_ERROR(_get_block_impl(state, block, eof)); if (*eof) { @@ -125,7 +125,7 @@ Status VScanner::get_block(RuntimeState* state, Block* block, bool* eof) { // 2. Filter the output block finally. { - auto timer = _parent ? _parent->_filter_timer : _local_state->_filter_timer; + auto* timer = _parent ? _parent->_filter_timer : _local_state->_filter_timer; SCOPED_TIMER(timer); RETURN_IF_ERROR(_filter_output_block(block)); } diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h index 4c4f0aa6740c13..42a46d8a8f3b13 100644 --- a/be/src/vec/exprs/vexpr.h +++ b/be/src/vec/exprs/vexpr.h @@ -374,28 +374,35 @@ Status create_texpr_literal_node(const void* data, TExprNode* node, int precisio const auto* origin_value = reinterpret_cast*>(data); (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); TDecimalLiteral decimal_literal; - decimal_literal.__set_value(origin_value->to_string(scale)); + decimal_literal.__set_value(origin_value->to_string(precision, scale)); (*node).__set_decimal_literal(decimal_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL32, precision, scale)); } else if constexpr (T == TYPE_DECIMAL64) { const auto* origin_value = reinterpret_cast*>(data); (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); TDecimalLiteral decimal_literal; - decimal_literal.__set_value(origin_value->to_string(scale)); + decimal_literal.__set_value(origin_value->to_string(precision, scale)); (*node).__set_decimal_literal(decimal_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL64, precision, scale)); } else if constexpr (T == TYPE_DECIMAL128I) { const auto* origin_value = reinterpret_cast*>(data); (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); TDecimalLiteral decimal_literal; - decimal_literal.__set_value(origin_value->to_string(scale)); + // e.g. For a decimal(26,6) column, the initial value of the _min of the MinMax RF + // on the RF producer side is an int128 value with 38 digits of 9, and this is the + // final min value of the MinMax RF if the fragment instance has no data. + // Need to truncate the value to the right precision and scale here, to avoid + // error when casting string back to decimal later. + // TODO: this is a temporary solution, the best solution is to produce the + // right min max value at the producer side. + decimal_literal.__set_value(origin_value->to_string(precision, scale)); (*node).__set_decimal_literal(decimal_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL128I, precision, scale)); } else if constexpr (T == TYPE_DECIMAL256) { const auto* origin_value = reinterpret_cast*>(data); (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); TDecimalLiteral decimal_literal; - decimal_literal.__set_value(origin_value->to_string(scale)); + decimal_literal.__set_value(origin_value->to_string(precision, scale)); (*node).__set_decimal_literal(decimal_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL256, precision, scale)); } else if constexpr (T == TYPE_FLOAT) { diff --git a/be/src/vec/olap/vcollect_iterator.cpp b/be/src/vec/olap/vcollect_iterator.cpp index d8728cfa8126a2..90a38eef51e96e 100644 --- a/be/src/vec/olap/vcollect_iterator.cpp +++ b/be/src/vec/olap/vcollect_iterator.cpp @@ -22,6 +22,7 @@ #include #include +#include #include #include #include @@ -38,6 +39,7 @@ #include "runtime/query_context.h" #include "runtime/runtime_predicate.h" #include "runtime/runtime_state.h" +#include "util/runtime_profile.h" #include "vec/columns/column.h" #include "vec/core/column_with_type_and_name.h" #include "vec/core/field.h" @@ -467,7 +469,7 @@ Status VCollectIterator::Level0Iterator::init(bool get_data_by_ref) { } auto st = refresh_current_row(); - if (_get_data_by_ref && _block_view.size()) { + if (_get_data_by_ref && !_block_view.empty()) { _ref = _block_view[0]; } else { _ref = {_block, 0, false}; @@ -664,7 +666,7 @@ Status VCollectIterator::Level1Iterator::init(bool get_data_by_ref) { break; } } - _heap.reset(new MergeHeap {LevelIteratorComparator(sequence_loc, _is_reverse)}); + _heap = std::make_unique(LevelIteratorComparator(sequence_loc, _is_reverse)); for (auto&& child : _children) { DCHECK(child != nullptr); //DCHECK(child->current_row().ok()); @@ -770,6 +772,7 @@ Status VCollectIterator::Level1Iterator::_normal_next(IteratorRowRef* ref) { } Status VCollectIterator::Level1Iterator::_merge_next(Block* block) { + SCOPED_RAW_TIMER(&_reader->_stats.collect_iterator_merge_next_timer); int target_block_row = 0; auto target_columns = block->mutate_columns(); size_t column_count = target_columns.size(); @@ -851,6 +854,7 @@ Status VCollectIterator::Level1Iterator::_merge_next(Block* block) { } Status VCollectIterator::Level1Iterator::_normal_next(Block* block) { + SCOPED_RAW_TIMER(&_reader->_stats.collect_iterator_normal_next_timer); auto res = _cur_child->next(block); if (LIKELY(res.ok())) { return Status::OK(); diff --git a/be/src/vec/olap/vcollect_iterator.h b/be/src/vec/olap/vcollect_iterator.h index 06b17329163388..e83f56559f4971 100644 --- a/be/src/vec/olap/vcollect_iterator.h +++ b/be/src/vec/olap/vcollect_iterator.h @@ -187,7 +187,7 @@ class VCollectIterator { Status init(bool get_data_by_ref = false) override; - virtual void init_for_union(bool get_data_by_ref) override; + void init_for_union(bool get_data_by_ref) override; /* For unique and agg, rows is aggregated in block_reader, which access * first row so we need prepare the first row ref while duplicated diff --git a/be/src/vec/runtime/vparquet_transformer.cpp b/be/src/vec/runtime/vparquet_transformer.cpp index 94ff92de76eb9c..b831b1e2dc5ce8 100644 --- a/be/src/vec/runtime/vparquet_transformer.cpp +++ b/be/src/vec/runtime/vparquet_transformer.cpp @@ -200,9 +200,9 @@ void ParquetBuildHelper::build_version(parquet::WriterProperties::Builder& build VParquetTransformer::VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer, const VExprContextSPtrs& output_vexpr_ctxs, const std::vector& parquet_schemas, - const TParquetCompressionType::type& compression_type, - const bool& parquet_disable_dictionary, - const TParquetVersion::type& parquet_version, + TParquetCompressionType::type compression_type, + bool parquet_disable_dictionary, + TParquetVersion::type parquet_version, bool output_object_data) : VFileFormatTransformer(state, output_vexpr_ctxs, output_object_data), _parquet_schemas(parquet_schemas), diff --git a/be/src/vec/runtime/vparquet_transformer.h b/be/src/vec/runtime/vparquet_transformer.h index f825dcf1cc4656..c9d2de59a514d7 100644 --- a/be/src/vec/runtime/vparquet_transformer.h +++ b/be/src/vec/runtime/vparquet_transformer.h @@ -92,9 +92,9 @@ class VParquetTransformer final : public VFileFormatTransformer { VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer, const VExprContextSPtrs& output_vexpr_ctxs, const std::vector& parquet_schemas, - const TParquetCompressionType::type& compression_type, - const bool& parquet_disable_dictionary, - const TParquetVersion::type& parquet_version, bool output_object_data); + TParquetCompressionType::type compression_type, + bool parquet_disable_dictionary, TParquetVersion::type parquet_version, + bool output_object_data); ~VParquetTransformer() override = default; @@ -118,9 +118,9 @@ class VParquetTransformer final : public VFileFormatTransformer { std::shared_ptr _arrow_schema; const std::vector& _parquet_schemas; - const TParquetCompressionType::type& _compression_type; - const bool& _parquet_disable_dictionary; - const TParquetVersion::type& _parquet_version; + const TParquetCompressionType::type _compression_type; + const bool _parquet_disable_dictionary; + const TParquetVersion::type _parquet_version; }; } // namespace doris::vectorized diff --git a/be/src/vec/sink/writer/vhive_partition_writer.cpp b/be/src/vec/sink/writer/vhive_partition_writer.cpp index 6ce80679561e80..1a8cc083bc070e 100644 --- a/be/src/vec/sink/writer/vhive_partition_writer.cpp +++ b/be/src/vec/sink/writer/vhive_partition_writer.cpp @@ -49,8 +49,10 @@ Status VHivePartitionWriter::open(RuntimeState* state, RuntimeProfile* profile) io::FSPropertiesRef fs_properties(_write_info.file_type); fs_properties.properties = &_hadoop_conf; - _fs = DORIS_TRY(FileFactory::create_fs(fs_properties, - {.path = _write_info.write_path + '/' + _file_name})); + io::FileDescription file_description = { + .path = fmt::format("{}/{}", _write_info.write_path, _file_name)}; + _fs = DORIS_TRY(FileFactory::create_fs(fs_properties, file_description)); + RETURN_IF_ERROR(_fs->create_file(file_description.path, &_file_writer)); switch (_file_format_type) { case TFileFormatType::FORMAT_PARQUET: { @@ -75,17 +77,18 @@ Status VHivePartitionWriter::open(RuntimeState* state, RuntimeProfile* profile) } } std::vector parquet_schemas; + parquet_schemas.reserve(_columns.size()); for (int i = 0; i < _columns.size(); i++) { VExprSPtr column_expr = _vec_output_expr_ctxs[i]->root(); TParquetSchema parquet_schema; parquet_schema.schema_column_name = _columns[i].name; parquet_schemas.emplace_back(std::move(parquet_schema)); } - _vfile_writer.reset(new VParquetTransformer( - state, _file_writer_impl.get(), _vec_output_expr_ctxs, parquet_schemas, + _file_format_transformer.reset(new VParquetTransformer( + state, _file_writer.get(), _vec_output_expr_ctxs, parquet_schemas, parquet_compression_type, parquet_disable_dictionary, TParquetVersion::PARQUET_1_0, false)); - return _vfile_writer->open(); + return _file_format_transformer->open(); } case TFileFormatType::FORMAT_ORC: { orc::CompressionKind orc_compression_type; @@ -122,10 +125,10 @@ Status VHivePartitionWriter::open(RuntimeState* state, RuntimeProfile* profile) } } - _vfile_writer.reset(new VOrcTransformer(state, _file_writer_impl.get(), - _vec_output_expr_ctxs, std::move(root_schema), - false, orc_compression_type)); - return _vfile_writer->open(); + _file_format_transformer.reset( + new VOrcTransformer(state, _file_writer.get(), _vec_output_expr_ctxs, + std::move(root_schema), false, orc_compression_type)); + return _file_format_transformer->open(); } default: { return Status::InternalError("Unsupported file format type {}", @@ -135,13 +138,14 @@ Status VHivePartitionWriter::open(RuntimeState* state, RuntimeProfile* profile) } Status VHivePartitionWriter::close(const Status& status) { - if (_vfile_writer != nullptr) { - Status st = _vfile_writer->close(); + if (_file_format_transformer != nullptr) { + Status st = _file_format_transformer->close(); if (!st.ok()) { - LOG(WARNING) << fmt::format("_vfile_writer close failed, reason: {}", st.to_string()); + LOG(WARNING) << fmt::format("_file_format_transformer close failed, reason: {}", + st.to_string()); } } - if (!status.ok()) { + if (!status.ok() && _fs != nullptr) { auto path = fmt::format("{}/{}", _write_info.write_path, _file_name); Status st = _fs->delete_file(path); if (!st.ok()) { @@ -155,7 +159,7 @@ Status VHivePartitionWriter::close(const Status& status) { Status VHivePartitionWriter::write(vectorized::Block& block, vectorized::IColumn::Filter* filter) { Block output_block; RETURN_IF_ERROR(_projection_and_filter_block(block, filter, &output_block)); - RETURN_IF_ERROR(_vfile_writer->write(output_block)); + RETURN_IF_ERROR(_file_format_transformer->write(output_block)); _row_count += output_block.rows(); _input_size_in_bytes += output_block.bytes(); return Status::OK(); @@ -244,7 +248,7 @@ Status VHivePartitionWriter::_projection_and_filter_block(doris::vectorized::Blo return status; } RETURN_IF_ERROR(vectorized::VExprContext::get_output_block_after_execute_exprs( - _vec_output_expr_ctxs, input_block, output_block)); + _vec_output_expr_ctxs, input_block, output_block, true)); materialize_block_inplace(*output_block); if (filter == nullptr) { @@ -278,4 +282,4 @@ THivePartitionUpdate VHivePartitionWriter::_build_partition_update() { } } // namespace vectorized -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/vec/sink/writer/vhive_partition_writer.h b/be/src/vec/sink/writer/vhive_partition_writer.h index e46d0e8cd96177..e6b7059e900789 100644 --- a/be/src/vec/sink/writer/vhive_partition_writer.h +++ b/be/src/vec/sink/writer/vhive_partition_writer.h @@ -62,7 +62,7 @@ class VHivePartitionWriter { Status close(const Status& status); - inline size_t written_len() { return _vfile_writer->written_len(); } + inline size_t written_len() { return _file_format_transformer->written_len(); } private: std::unique_ptr _build_orc_type(const TypeDescriptor& type_descriptor); @@ -91,13 +91,13 @@ class VHivePartitionWriter { TFileCompressType::type _hive_compress_type; const std::map& _hadoop_conf; - std::shared_ptr _fs; + std::shared_ptr _fs = nullptr; // If the result file format is plain text, like CSV, this _file_writer is owned by this FileResultWriter. // If the result file format is Parquet, this _file_writer is owned by _parquet_writer. - std::unique_ptr _file_writer_impl = nullptr; + std::unique_ptr _file_writer = nullptr; // convert block to parquet/orc/csv format - std::unique_ptr _vfile_writer = nullptr; + std::unique_ptr _file_format_transformer = nullptr; RuntimeState* _state; }; diff --git a/be/test/io/cache/block_file_cache_test.cpp b/be/test/io/cache/block_file_cache_test.cpp index 497bd217cd25cc..6b1139d1b41c02 100644 --- a/be/test/io/cache/block_file_cache_test.cpp +++ b/be/test/io/cache/block_file_cache_test.cpp @@ -1143,7 +1143,7 @@ int get_disk_info(const char* const (&argv)[N], int* percent) { std::cerr << "Error creating pipe" << std::endl; return -1; } - pid_t pid = ::vfork(); + pid_t pid = ::fork(); if (pid == -1) { std::cerr << "Error forking process" << std::endl; return -1; diff --git a/be/test/olap/delete_handler_test.cpp b/be/test/olap/delete_handler_test.cpp index 3a22f317535f0c..c67d410f493bc8 100644 --- a/be/test/olap/delete_handler_test.cpp +++ b/be/test/olap/delete_handler_test.cpp @@ -1190,4 +1190,44 @@ TEST_F(TestDeleteHandler, FilterDataVersion) { EXPECT_EQ(Status::OK(), res); } +// clang-format off +TEST_F(TestDeleteHandler, TestParseDeleteCondition) { + auto test = [](const std::tuple& in) { + auto& [cond_str, exp_succ, exp_cond] = in; + TCondition parsed_cond; + EXPECT_EQ(DeleteHandler::parse_condition(cond_str, &parsed_cond), exp_succ) << " unexpected result, cond_str: " << cond_str; + if (exp_succ) EXPECT_EQ(parsed_cond, exp_cond) << " unexpected result, cond_str: " << cond_str; + }; + + auto gen_cond = [](const std::string& col, const std::string& op, const std::string& val) { + TCondition cond; + cond.__set_column_name(col); + cond.__set_condition_op(op); + cond.__set_condition_values(std::vector{val}); + return cond; + }; + + // > + std::vector> test_input { + {R"(abc=b)" , true, gen_cond(R"(abc)" , "=" , R"(b)" )}, // normal case + {R"(abc!=b)" , true, gen_cond(R"(abc)" , "!=", R"(b)" )}, // normal case + {R"(abc<=b)" , true, gen_cond(R"(abc)" , "<=", R"(b)" )}, // normal case + {R"(abc>=b)" , true, gen_cond(R"(abc)" , ">=", R"(b)" )}, // normal case + {R"(abc>>b)" , true, gen_cond(R"(abc)" , ">>", R"(b)" )}, // normal case + {R"(abc<>WTF(10086))" , true, gen_cond(R"(a*a)" , ">>", R"(WTF(10086))")}, // function + {R"(a-b IS NULL)" , true, gen_cond(R"(a-b)" , "IS", R"(NULL)" )}, // - in col name and test IS NULL + {R"(@a*-b IS NOT NULL)" , true, gen_cond(R"(@a*-b)" , "IS", R"(NOT NULL)" )}, // test IS NOT NULL + {R"(a IS b IS NOT NULL)", true, gen_cond(R"(a IS b)", "IS", R"(NOT NULL)" )}, // test " IS " in column name + {R"(_a-zA-Z@0-9 /.a-zA-Z0-9_+-/?@#$%^&*" ,:=hell)", true, gen_cond(R"(_a-zA-Z@0-9 /.a-zA-Z0-9_+-/?@#$%^&*" ,:)", "=", R"(hell)")}, // hellbound column name + {R"(this is a col very loooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooon colum name=long)", true, gen_cond(R"(this is a col very loooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooon colum name)", "=", R"(long)")}, // test " IS " in column name + }; + for (auto& i : test_input) { test(i); } +} +// clang-format on + } // namespace doris diff --git a/be/test/runtime/routine_load_task_executor_test.cpp b/be/test/runtime/routine_load_task_executor_test.cpp index 8a8dcc4d677441..f95fdcfdadfb8a 100644 --- a/be/test/runtime/routine_load_task_executor_test.cpp +++ b/be/test/runtime/routine_load_task_executor_test.cpp @@ -59,7 +59,7 @@ class RoutineLoadTaskExecutorTest : public testing::Test { _env.set_new_load_stream_mgr(NewLoadStreamMgr::create_unique()); _env.set_stream_load_executor(StreamLoadExecutor::create_unique(&_env)); - config::routine_load_thread_pool_size = 5; + config::max_routine_load_thread_pool_size = 1024; config::max_consumer_num_per_group = 3; } @@ -93,8 +93,10 @@ TEST_F(RoutineLoadTaskExecutorTest, exec_task) { task.__set_kafka_load_info(k_info); RoutineLoadTaskExecutor executor(&_env); - // submit task Status st; + st = executor.init(); + EXPECT_TRUE(st.ok()); + // submit task st = executor.submit_task(task); EXPECT_TRUE(st.ok()); diff --git a/be/test/testutil/mock_rowset.h b/be/test/testutil/mock_rowset.h index 50065ebe6b43d4..89fbb8cac3b5d7 100644 --- a/be/test/testutil/mock_rowset.h +++ b/be/test/testutil/mock_rowset.h @@ -50,6 +50,8 @@ class MockRowset : public Rowset { return Status::NotSupported("MockRowset not support this method."); } + std::string segment_file_path(int segment_id) const override { return ""; } + Status get_segments_key_bounds(std::vector* segments_key_bounds) override { // TODO(zhangchen): remove this after we implemented memrowset. if (is_mem_rowset_) { diff --git a/be/test/vec/data_types/decimal_test.cpp b/be/test/vec/data_types/decimal_test.cpp index 0f4b95020140dd..7f12876e148d1b 100644 --- a/be/test/vec/data_types/decimal_test.cpp +++ b/be/test/vec/data_types/decimal_test.cpp @@ -23,6 +23,7 @@ #include #include "gtest/gtest_pred_impl.h" +#include "runtime/define_primitive_type.h" #include "runtime/raw_value.h" #include "runtime/type_limit.h" #include "util/string_parser.hpp" @@ -209,4 +210,107 @@ TEST(DecimalTest, hash) { EXPECT_EQ(hash_val, 12344); } } + +TEST(DecimalTest, to_string) { + { + Decimal32 dec(999999999); + auto dec_str = dec.to_string(9, 0); + EXPECT_EQ(dec_str, "999999999"); + dec_str = dec.to_string(9, 6); + EXPECT_EQ(dec_str, "999.999999"); + dec_str = dec.to_string(9, 9); + EXPECT_EQ(dec_str, "0.999999999"); + + dec_str = dec.to_string(8, 0); + EXPECT_EQ(dec_str, "99999999"); + dec_str = dec.to_string(8, 6); + EXPECT_EQ(dec_str, "99.999999"); + dec_str = dec.to_string(8, 8); + EXPECT_EQ(dec_str, "0.99999999"); + + dec_str = dec.to_string(10, 0); + EXPECT_EQ(dec_str, "999999999"); + dec_str = dec.to_string(10, 6); + EXPECT_EQ(dec_str, "999.999999"); + dec_str = dec.to_string(10, 9); + EXPECT_EQ(dec_str, "0.999999999"); + } + { + Decimal32 dec(-999999999); + auto dec_str = dec.to_string(9, 0); + EXPECT_EQ(dec_str, "-999999999"); + dec_str = dec.to_string(9, 6); + EXPECT_EQ(dec_str, "-999.999999"); + dec_str = dec.to_string(9, 9); + EXPECT_EQ(dec_str, "-0.999999999"); + + dec_str = dec.to_string(8, 0); + EXPECT_EQ(dec_str, "-99999999"); + dec_str = dec.to_string(8, 6); + EXPECT_EQ(dec_str, "-99.999999"); + dec_str = dec.to_string(8, 8); + EXPECT_EQ(dec_str, "-0.99999999"); + + dec_str = dec.to_string(10, 0); + EXPECT_EQ(dec_str, "-999999999"); + dec_str = dec.to_string(10, 6); + EXPECT_EQ(dec_str, "-999.999999"); + dec_str = dec.to_string(10, 9); + EXPECT_EQ(dec_str, "-0.999999999"); + } + { + std::string val_str("999999999999999999999999999999"); // 30 digits + StringParser::ParseResult parse_result; + Decimal128V3 dec = StringParser::string_to_decimal( + val_str.data(), val_str.size(), val_str.size(), 0, &parse_result); + EXPECT_EQ(parse_result, StringParser::ParseResult::PARSE_SUCCESS); + auto dec_str = dec.to_string(30, 0); + EXPECT_EQ(dec_str, "999999999999999999999999999999"); + dec_str = dec.to_string(30, 6); + EXPECT_EQ(dec_str, "999999999999999999999999.999999"); + dec_str = dec.to_string(30, 30); + EXPECT_EQ(dec_str, "0.999999999999999999999999999999"); + + dec_str = dec.to_string(20, 0); + EXPECT_EQ(dec_str, "99999999999999999999"); + dec_str = dec.to_string(20, 6); + EXPECT_EQ(dec_str, "99999999999999.999999"); + dec_str = dec.to_string(20, 20); + EXPECT_EQ(dec_str, "0.99999999999999999999"); + } + { + std::string val_str("-999999999999999999999999999999"); // 30 digits + StringParser::ParseResult parse_result; + Decimal128V3 dec = StringParser::string_to_decimal( + val_str.data(), val_str.size(), val_str.size(), 0, &parse_result); + EXPECT_EQ(parse_result, StringParser::ParseResult::PARSE_SUCCESS); + auto dec_str = dec.to_string(30, 0); + EXPECT_EQ(dec_str, "-999999999999999999999999999999"); + dec_str = dec.to_string(30, 6); + EXPECT_EQ(dec_str, "-999999999999999999999999.999999"); + dec_str = dec.to_string(30, 30); + EXPECT_EQ(dec_str, "-0.999999999999999999999999999999"); + + dec_str = dec.to_string(20, 0); + EXPECT_EQ(dec_str, "-99999999999999999999"); + dec_str = dec.to_string(20, 6); + EXPECT_EQ(dec_str, "-99999999999999.999999"); + dec_str = dec.to_string(20, 20); + EXPECT_EQ(dec_str, "-0.99999999999999999999"); + } + + { + Decimal256 dec(type_limit::max()); + // Decimal256 dec_min(type_limit::min()); + auto dec_str = dec.to_string(76, 0); + EXPECT_EQ(dec_str, + "9999999999999999999999999999999999999999999999999999999999999999999999999999"); + dec_str = dec.to_string(76, 6); + EXPECT_EQ(dec_str, + "9999999999999999999999999999999999999999999999999999999999999999999999.999999"); + dec_str = dec.to_string(76, 76); + EXPECT_EQ(dec_str, + "0.9999999999999999999999999999999999999999999999999999999999999999999999999999"); + } +} } // namespace doris::vectorized \ No newline at end of file diff --git a/bin/start_be.sh b/bin/start_be.sh index 0649d42385f4c8..87326abe8064d0 100755 --- a/bin/start_be.sh +++ b/bin/start_be.sh @@ -89,9 +89,13 @@ fi preload_jars=("preload-extensions") preload_jars+=("java-udf") +DORIS_PRELOAD_JAR= for preload_jar_dir in "${preload_jars[@]}"; do for f in "${DORIS_HOME}/lib/java_extensions/${preload_jar_dir}"/*.jar; do - if [[ -z "${DORIS_CLASSPATH}" ]]; then + if [[ "${f}" == *"preload-extensions-project.jar" ]]; then + DORIS_PRELOAD_JAR="${f}" + continue + elif [[ -z "${DORIS_CLASSPATH}" ]]; then export DORIS_CLASSPATH="${f}" else export DORIS_CLASSPATH="${DORIS_CLASSPATH}:${f}" @@ -122,6 +126,10 @@ if [[ -d "${DORIS_HOME}/custom_lib" ]]; then done fi +# make sure the preload-extensions-project.jar is at first order, so that some classed +# with same qualified name can be loaded priority from preload-extensions-project.jar. +DORIS_CLASSPATH="${DORIS_PRELOAD_JAR}:${DORIS_CLASSPATH}" + if [[ -n "${HADOOP_CONF_DIR}" ]]; then export DORIS_CLASSPATH="${DORIS_CLASSPATH}:${HADOOP_CONF_DIR}" fi @@ -132,6 +140,8 @@ export CLASSPATH="${DORIS_HOME}/conf/:${DORIS_CLASSPATH}:${CLASSPATH}" # DORIS_CLASSPATH is for self-managed jni export DORIS_CLASSPATH="-Djava.class.path=${DORIS_CLASSPATH}" +#echo ${DORIS_CLASSPATH} + export LD_LIBRARY_PATH="${DORIS_HOME}/lib/hadoop_hdfs/native:${LD_LIBRARY_PATH}" jdk_version() { diff --git a/build.sh b/build.sh index dbd89f4aa1e768..25b63610aab3b2 100755 --- a/build.sh +++ b/build.sh @@ -818,10 +818,14 @@ EOF rm -rf "${BE_JAVA_EXTENSIONS_DIR}" mkdir "${BE_JAVA_EXTENSIONS_DIR}" for extensions_module in "${extensions_modules[@]}"; do - module_path="${DORIS_HOME}/fe/be-java-extensions/${extensions_module}/target/${extensions_module}-jar-with-dependencies.jar" + module_jar="${DORIS_HOME}/fe/be-java-extensions/${extensions_module}/target/${extensions_module}-jar-with-dependencies.jar" + module_proj_jar="${DORIS_HOME}/fe/be-java-extensions/${extensions_module}/target/${extensions_module}-project.jar" mkdir "${BE_JAVA_EXTENSIONS_DIR}"/"${extensions_module}" - if [[ -f "${module_path}" ]]; then - cp "${module_path}" "${BE_JAVA_EXTENSIONS_DIR}"/"${extensions_module}" + if [[ -f "${module_jar}" ]]; then + cp "${module_jar}" "${BE_JAVA_EXTENSIONS_DIR}"/"${extensions_module}" + fi + if [[ -f "${module_proj_jar}" ]]; then + cp "${module_proj_jar}" "${BE_JAVA_EXTENSIONS_DIR}"/"${extensions_module}" fi done diff --git a/cloud/src/common/bvars.cpp b/cloud/src/common/bvars.cpp index 702dea865022b2..ab0b5934b50998 100644 --- a/cloud/src/common/bvars.cpp +++ b/cloud/src/common/bvars.cpp @@ -69,6 +69,7 @@ BvarLatencyRecorderWithTag g_bvar_ms_get_delete_bitmap("ms", "get_delete_bitmap" BvarLatencyRecorderWithTag g_bvar_ms_get_delete_bitmap_update_lock("ms", "get_delete_bitmap_update_lock"); BvarLatencyRecorderWithTag g_bvar_ms_get_instance("ms", "get_instance"); +BvarLatencyRecorderWithTag g_bvar_ms_get_rl_task_commit_attach("ms", "get_rl_task_commit_attach"); BvarLatencyRecorderWithTag g_bvar_ms_start_tablet_job("ms", "start_tablet_job"); BvarLatencyRecorderWithTag g_bvar_ms_finish_tablet_job("ms", "finish_tablet_job"); diff --git a/cloud/src/common/bvars.h b/cloud/src/common/bvars.h index 1c4c4f749b6fe1..dbdbfa834e9812 100644 --- a/cloud/src/common/bvars.h +++ b/cloud/src/common/bvars.h @@ -170,6 +170,7 @@ extern BvarLatencyRecorderWithTag g_bvar_ms_get_delete_bitmap_update_lock; extern BvarLatencyRecorderWithTag g_bvar_ms_get_cluster_status; extern BvarLatencyRecorderWithTag g_bvar_ms_set_cluster_status; extern BvarLatencyRecorderWithTag g_bvar_ms_get_instance; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_rl_task_commit_attach; // txn_kv's bvars extern bvar::LatencyRecorder g_bvar_txn_kv_get; diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h index 859271f6503fed..03ae47abe569c1 100644 --- a/cloud/src/common/config.h +++ b/cloud/src/common/config.h @@ -60,7 +60,7 @@ CONF_mInt64(dropped_partition_retention_seconds, "10800"); // 3h CONF_Strings(recycle_whitelist, ""); // Comma seprated list // These instances will not be recycled, only effective when whitelist is empty. CONF_Strings(recycle_blacklist, ""); // Comma seprated list -CONF_mInt32(instance_recycler_worker_pool_size, "10"); +CONF_mInt32(instance_recycler_worker_pool_size, "1"); CONF_Bool(enable_checker, "false"); // Currently only used for recycler test CONF_Bool(enable_inverted_check, "false"); @@ -69,6 +69,9 @@ CONF_mInt32(scan_instances_interval_seconds, "60"); // 1min // interval for check object CONF_mInt32(check_object_interval_seconds, "43200"); // 12hours +CONF_mInt64(check_recycle_task_interval_seconds, "600"); // 10min +CONF_mInt64(recycle_task_threshold_seconds, "10800"); // 3h + CONF_String(test_s3_ak, "ak"); CONF_String(test_s3_sk, "sk"); CONF_String(test_s3_endpoint, "endpoint"); diff --git a/cloud/src/meta-service/meta_service.h b/cloud/src/meta-service/meta_service.h index 85232de7bd9ca8..a8cd0f521487b2 100644 --- a/cloud/src/meta-service/meta_service.h +++ b/cloud/src/meta-service/meta_service.h @@ -251,6 +251,11 @@ class MetaServiceImpl : public cloud::MetaService { GetClusterStatusResponse* response, ::google::protobuf::Closure* done) override; + void get_rl_task_commit_attach(::google::protobuf::RpcController* controller, + const GetRLTaskCommitAttachRequest* request, + GetRLTaskCommitAttachResponse* response, + ::google::protobuf::Closure* done) override; + // ATTN: If you add a new method, please also add the corresponding implementation in `MetaServiceProxy`. std::pair get_instance_info(const std::string& instance_id, @@ -574,6 +579,13 @@ class MetaServiceProxy final : public MetaService { call_impl(&cloud::MetaService::get_cluster_status, controller, request, response, done); } + void get_rl_task_commit_attach(::google::protobuf::RpcController* controller, + const GetRLTaskCommitAttachRequest* request, + GetRLTaskCommitAttachResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_rl_task_commit_attach, controller, request, response, done); + } + private: template using MetaServiceMethod = void (cloud::MetaService::*)(::google::protobuf::RpcController*, @@ -622,11 +634,14 @@ class MetaServiceProxy final : public MetaService { TEST_SYNC_POINT("MetaServiceProxy::call_impl:2"); if (--retry_times < 0) { + // For KV_TXN_CONFLICT, we should return KV_TXN_CONFLICT_RETRY_EXCEEDED_MAX_TIMES, + // because BE will retries the KV_TXN_CONFLICT error. resp->mutable_status()->set_code( - code == MetaServiceCode::KV_TXN_STORE_COMMIT_RETRYABLE ? KV_TXN_COMMIT_ERR - : code == MetaServiceCode::KV_TXN_STORE_GET_RETRYABLE ? KV_TXN_GET_ERR - : code == MetaServiceCode::KV_TXN_STORE_CREATE_RETRYABLE ? KV_TXN_CREATE_ERR - : code); + code == MetaServiceCode::KV_TXN_STORE_COMMIT_RETRYABLE ? KV_TXN_COMMIT_ERR + : code == MetaServiceCode::KV_TXN_STORE_GET_RETRYABLE ? KV_TXN_GET_ERR + : code == MetaServiceCode::KV_TXN_STORE_CREATE_RETRYABLE + ? KV_TXN_CREATE_ERR + : KV_TXN_CONFLICT_RETRY_EXCEEDED_MAX_TIMES); return; } diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index 788663c2ceac3f..03251ee0f071a7 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -478,6 +478,163 @@ void MetaServiceImpl::precommit_txn(::google::protobuf::RpcController* controlle } } +void put_routine_load_progress(MetaServiceCode& code, std::string& msg, + const std::string& instance_id, + const CommitTxnRequest* request, + Transaction* txn, int64_t db_id) { + std::stringstream ss; + int64_t txn_id = request->txn_id(); + if (!request->has_commit_attachment()) { + ss << "failed to get commit attachment from req, db_id=" << db_id + << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + + TxnCommitAttachmentPB txn_commit_attachment = request->commit_attachment(); + RLTaskTxnCommitAttachmentPB commit_attachment = + txn_commit_attachment.rl_task_txn_commit_attachment(); + int64_t job_id = commit_attachment.job_id(); + + std::string rl_progress_key; + std::string rl_progress_val; + bool prev_progress_existed = true; + RLJobProgressKeyInfo rl_progress_key_info {instance_id, db_id, job_id}; + rl_job_progress_key_info(rl_progress_key_info, &rl_progress_key); + TxnErrorCode err = txn->get(rl_progress_key, &rl_progress_val); + if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + prev_progress_existed = false; + } else { + code = cast_as(err); + ss << "failed to get routine load progress, db_id=" << db_id << " txn_id=" << txn_id + << " err=" << err; + msg = ss.str(); + return; + } + } + + RoutineLoadProgressPB prev_progress_info; + if (prev_progress_existed) { + if (!prev_progress_info.ParseFromString(rl_progress_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse routine load progress, db_id=" << db_id + << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + } + + std::string new_progress_val; + RoutineLoadProgressPB new_progress_info; + new_progress_info.CopyFrom(commit_attachment.progress()); + for (auto const& elem : prev_progress_info.partition_to_offset()) { + auto it = new_progress_info.partition_to_offset().find(elem.first); + if (it == new_progress_info.partition_to_offset().end()) { + new_progress_info.mutable_partition_to_offset()->insert(elem); + } + } + + std::string new_statistic_val; + RoutineLoadJobStatisticPB* new_statistic_info = new_progress_info.mutable_stat(); + if (prev_progress_info.has_stat()) { + const RoutineLoadJobStatisticPB& prev_statistic_info = prev_progress_info.stat(); + + new_statistic_info->set_filtered_rows(prev_statistic_info.filtered_rows() + commit_attachment.filtered_rows()); + new_statistic_info->set_loaded_rows(prev_statistic_info.loaded_rows() + commit_attachment.loaded_rows()); + new_statistic_info->set_unselected_rows(prev_statistic_info.unselected_rows() + commit_attachment.unselected_rows()); + new_statistic_info->set_received_bytes(prev_statistic_info.received_bytes() + commit_attachment.received_bytes()); + new_statistic_info->set_task_execution_time_ms(prev_statistic_info.task_execution_time_ms() + commit_attachment.task_execution_time_ms()); + } else { + new_statistic_info->set_filtered_rows(commit_attachment.filtered_rows()); + new_statistic_info->set_loaded_rows(commit_attachment.loaded_rows()); + new_statistic_info->set_unselected_rows(commit_attachment.unselected_rows()); + new_statistic_info->set_received_bytes(commit_attachment.received_bytes()); + new_statistic_info->set_task_execution_time_ms(commit_attachment.task_execution_time_ms()); + } + + LOG(INFO) << "routine load new progress: " << new_progress_info.ShortDebugString(); + + if (!new_progress_info.SerializeToString(&new_progress_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize new progress val, txn_id=" << txn_id; + msg = ss.str(); + return; + } + + txn->put(rl_progress_key, new_progress_val); +} + +void MetaServiceImpl::get_rl_task_commit_attach(::google::protobuf::RpcController* controller, + const GetRLTaskCommitAttachRequest* request, + GetRLTaskCommitAttachResponse* response, + ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_rl_task_commit_attach); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(get_rl_task_commit_attach) + + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "filed to create txn, err=" << err; + msg = ss.str(); + return; + } + + if (!request->has_db_id() || !request->has_job_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty db_id or job_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + + int64_t db_id = request->db_id(); + int64_t job_id = request->job_id(); + std::string rl_progress_key; + std::string rl_progress_val; + RLJobProgressKeyInfo rl_progress_key_info {instance_id, db_id, job_id}; + rl_job_progress_key_info(rl_progress_key_info, &rl_progress_key); + err = txn->get(rl_progress_key, &rl_progress_val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = MetaServiceCode::ROUTINE_LOAD_PROGRESS_NOT_FOUND; + ss << "pregress info not found, db_id=" << db_id + << " job_id=" << job_id << " err=" << err; + msg = ss.str(); + return; + } else if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get pregress info, db_id=" << db_id + << " job_id=" << job_id << " err=" << err; + msg = ss.str(); + return; + } + + RLTaskTxnCommitAttachmentPB* commit_attach = response->mutable_commit_attach(); + RoutineLoadProgressPB* progress_info = commit_attach->mutable_progress(); + if (!progress_info->ParseFromString(rl_progress_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse progress info, db_id=" << db_id << " job_id=" << job_id; + msg = ss.str(); + return; + } + + if (progress_info->has_stat()) { + const RoutineLoadJobStatisticPB& statistic_info = progress_info->stat(); + commit_attach->set_filtered_rows(statistic_info.filtered_rows()); + commit_attach->set_loaded_rows(statistic_info.loaded_rows()); + commit_attach->set_unselected_rows(statistic_info.unselected_rows()); + commit_attach->set_received_bytes(statistic_info.received_bytes()); + commit_attach->set_task_execution_time_ms(statistic_info.task_execution_time_ms()); + } +} + /** * 0. Extract txn_id from request * 1. Get db id from TxnKv with txn_id @@ -977,86 +1134,7 @@ void MetaServiceImpl::commit_txn(::google::protobuf::RpcController* controller, if (txn_info.load_job_source_type() == LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_ROUTINE_LOAD_TASK) { - if (!request->has_commit_attachment()) { - ss << "failed to get commit attachment from req, db_id=" << db_id - << " txn_id=" << txn_id; - msg = ss.str(); - return; - } - - TxnCommitAttachmentPB txn_commit_attachment = request->commit_attachment(); - RLTaskTxnCommitAttachmentPB commit_attachment = - txn_commit_attachment.rl_task_txn_commit_attachment(); - int64_t job_id = commit_attachment.job_id(); - - std::string rl_progress_key; - std::string rl_progress_val; - bool prev_progress_existed = true; - RLJobProgressKeyInfo rl_progress_key_info {instance_id, db_id, job_id}; - rl_job_progress_key_info(rl_progress_key_info, &rl_progress_key); - TxnErrorCode err = txn->get(rl_progress_key, &rl_progress_val); - if (err != TxnErrorCode::TXN_OK) { - if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { - prev_progress_existed = false; - } else { - code = cast_as(err); - ss << "failed to get txn_info, db_id=" << db_id << " txn_id=" << txn_id - << " err=" << err; - msg = ss.str(); - return; - } - } - - RoutineLoadProgressPB prev_progress_info; - if (prev_progress_existed) { - if (!prev_progress_info.ParseFromString(rl_progress_val)) { - code = MetaServiceCode::PROTOBUF_PARSE_ERR; - ss << "failed to parse txn_info, db_id=" << db_id << " txn_id=" << txn_id; - msg = ss.str(); - return; - } - - int cal_row_num = 0; - for (auto const& elem : commit_attachment.progress().partition_to_offset()) { - if (elem.second >= 0) { - auto it = prev_progress_info.partition_to_offset().find(elem.first); - if (it != prev_progress_info.partition_to_offset().end() && it->second >= 0) { - cal_row_num += elem.second - it->second; - } else { - cal_row_num += elem.second + 1; - } - } - } - - LOG(INFO) << " calculated row num " << cal_row_num << " actual row num " - << commit_attachment.loaded_rows() << " prev progress " - << prev_progress_info.DebugString(); - - if (cal_row_num == 0) { - LOG(WARNING) << " repeated to load task in routine load, db_id=" << db_id - << " txn_id=" << txn_id << " calculated row num " << cal_row_num - << " actual row num " << commit_attachment.loaded_rows(); - return; - } - } - - std::string new_progress_val; - RoutineLoadProgressPB new_progress_info; - new_progress_info.CopyFrom(commit_attachment.progress()); - for (auto const& elem : prev_progress_info.partition_to_offset()) { - auto it = new_progress_info.partition_to_offset().find(elem.first); - if (it == new_progress_info.partition_to_offset().end()) { - new_progress_info.mutable_partition_to_offset()->insert(elem); - } - } - - if (!new_progress_info.SerializeToString(&new_progress_val)) { - code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; - ss << "failed to serialize new progress val, txn_id=" << txn_info.txn_id(); - msg = ss.str(); - return; - } - txn->put(rl_progress_key, new_progress_val); + put_routine_load_progress(code, msg, instance_id, request, txn.get(), db_id); } LOG(INFO) << "xxx commit_txn put recycle_key key=" << hex(recycle_key) << " txn_id=" << txn_id; diff --git a/cloud/src/recycler/recycler.cpp b/cloud/src/recycler/recycler.cpp index c765a58d0fcfbc..1ca1e05f741e57 100644 --- a/cloud/src/recycler/recycler.cpp +++ b/cloud/src/recycler/recycler.cpp @@ -49,6 +49,8 @@ namespace doris::cloud { +using namespace std::chrono; + // return 0 for success get a key, 1 for key not found, negative for error [[maybe_unused]] static int txn_get(TxnKv* txn_kv, std::string_view key, std::string& val) { std::unique_ptr txn; @@ -143,6 +145,23 @@ static int txn_remove(TxnKv* txn_kv, std::vector keys) { } } +static inline void check_recycle_task(const std::string& instance_id, const std::string& task_name, + int64_t num_scanned, int64_t num_recycled, + int64_t start_time) { + if ((num_scanned % 10000) == 0 && (num_scanned > 0)) [[unlikely]] { + int64_t cost = + duration_cast(steady_clock::now().time_since_epoch()).count() - start_time; + if (cost > config::recycle_task_threshold_seconds) { + LOG_INFO("recycle task cost too much time cost={}s", cost) + .tag("instance_id", instance_id) + .tag("task", task_name) + .tag("num_scanned", num_scanned) + .tag("num_recycled", num_recycled); + } + } + return; +} + Recycler::Recycler(std::shared_ptr txn_kv) : txn_kv_(std::move(txn_kv)) { ip_port_ = std::string(butil::my_ip_cstr()) + ":" + std::to_string(config::brpc_listen_port); } @@ -221,7 +240,6 @@ void Recycler::recycle_callback() { } if (stopped()) return; LOG_INFO("begin to recycle instance").tag("instance_id", instance_id); - using namespace std::chrono; auto ctime_ms = duration_cast(system_clock::now().time_since_epoch()).count(); ret = instance_recycler->do_recycle(); // If instance recycler has been aborted, don't finish this job @@ -268,6 +286,23 @@ void Recycler::lease_recycle_jobs() { } } +void Recycler::check_recycle_tasks() { + while (!stopped()) { + std::unordered_map> recycling_instance_map; + { + std::lock_guard lock(mtx_); + recycling_instance_map = recycling_instance_map_; + } + for (auto& entry : recycling_instance_map) { + entry.second->check_recycle_tasks(); + } + + std::unique_lock lock(mtx_); + notifier_.wait_for(lock, std::chrono::seconds(config::check_recycle_task_interval_seconds), + [&]() { return stopped(); }); + } +} + int Recycler::start(brpc::Server* server) { instance_filter_.reset(config::recycle_whitelist, config::recycle_blacklist); @@ -298,6 +333,7 @@ int Recycler::start(brpc::Server* server) { } workers_.push_back(std::thread(std::mem_fn(&Recycler::lease_recycle_jobs), this)); + workers_.push_back(std::thread(std::mem_fn(&Recycler::check_recycle_tasks), this)); return 0; } @@ -470,7 +506,6 @@ int InstanceRecycler::recycle_deleted_instance() { LOG_INFO("begin to recycle deleted instance").tag("instance_id", instance_id_); int ret = 0; - using namespace std::chrono; auto start_time = steady_clock::now(); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { @@ -560,6 +595,7 @@ int InstanceRecycler::recycle_deleted_instance() { } int InstanceRecycler::recycle_indexes() { + const std::string task_name = "recycle_indexes"; int num_scanned = 0; int num_expired = 0; int num_recycled = 0; @@ -573,11 +609,13 @@ int InstanceRecycler::recycle_indexes() { LOG_INFO("begin to recycle indexes").tag("instance_id", instance_id_); - using namespace std::chrono; - auto start_time = steady_clock::now(); + int64_t start_time = duration_cast(steady_clock::now().time_since_epoch()).count(); + register_recycle_task(task_name, start_time); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { - auto cost = duration(steady_clock::now() - start_time).count(); + unregister_recycle_task(task_name); + int64_t cost = + duration_cast(steady_clock::now().time_since_epoch()).count() - start_time; LOG_INFO("recycle indexes finished, cost={}s", cost) .tag("instance_id", instance_id_) .tag("num_scanned", num_scanned) @@ -659,6 +697,7 @@ int InstanceRecycler::recycle_indexes() { return -1; } ++num_recycled; + check_recycle_task(instance_id_, task_name, num_scanned, num_recycled, start_time); index_keys.push_back(k); return 0; }; @@ -678,6 +717,7 @@ int InstanceRecycler::recycle_indexes() { } int InstanceRecycler::recycle_partitions() { + const std::string task_name = "recycle_partitions"; int num_scanned = 0; int num_expired = 0; int num_recycled = 0; @@ -691,11 +731,13 @@ int InstanceRecycler::recycle_partitions() { LOG_INFO("begin to recycle partitions").tag("instance_id", instance_id_); - using namespace std::chrono; - auto start_time = steady_clock::now(); + int64_t start_time = duration_cast(steady_clock::now().time_since_epoch()).count(); + register_recycle_task(task_name, start_time); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { - auto cost = duration(steady_clock::now() - start_time).count(); + unregister_recycle_task(task_name); + int64_t cost = + duration_cast(steady_clock::now().time_since_epoch()).count() - start_time; LOG_INFO("recycle partitions finished, cost={}s", cost) .tag("instance_id", instance_id_) .tag("num_scanned", num_scanned) @@ -786,6 +828,7 @@ int InstanceRecycler::recycle_partitions() { } if (ret == 0) { ++num_recycled; + check_recycle_task(instance_id_, task_name, num_scanned, num_recycled, start_time); partition_keys.push_back(k); if (part_pb.db_id() > 0) { version_keys.push_back(version_key( @@ -831,7 +874,6 @@ int InstanceRecycler::recycle_versions() { LOG_INFO("begin to recycle partition versions").tag("instance_id", instance_id_); - using namespace std::chrono; auto start_time = steady_clock::now(); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { @@ -928,7 +970,6 @@ int InstanceRecycler::recycle_tablets(int64_t table_id, int64_t index_id, int64_ .tag("index_id", index_id) .tag("partition_id", partition_id); - using namespace std::chrono; auto start_time = steady_clock::now(); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { @@ -1199,7 +1240,6 @@ int InstanceRecycler::recycle_tablet(int64_t tablet_id) { .tag("instance_id", instance_id_) .tag("tablet_id", tablet_id); - using namespace std::chrono; auto start_time = steady_clock::now(); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { @@ -1261,6 +1301,7 @@ int InstanceRecycler::recycle_tablet(int64_t tablet_id) { } int InstanceRecycler::recycle_rowsets() { + const std::string task_name = "recycle_rowsets"; int num_scanned = 0; int num_expired = 0; int num_prepare = 0; @@ -1277,11 +1318,13 @@ int InstanceRecycler::recycle_rowsets() { LOG_INFO("begin to recycle rowsets").tag("instance_id", instance_id_); - using namespace std::chrono; - auto start_time = steady_clock::now(); + int64_t start_time = duration_cast(steady_clock::now().time_since_epoch()).count(); + register_recycle_task(task_name, start_time); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { - auto cost = duration(steady_clock::now() - start_time).count(); + unregister_recycle_task(task_name); + int64_t cost = + duration_cast(steady_clock::now().time_since_epoch()).count() - start_time; LOG_INFO("recycle rowsets finished, cost={}s", cost) .tag("instance_id", instance_id_) .tag("num_scanned", num_scanned) @@ -1325,6 +1368,8 @@ int InstanceRecycler::recycle_rowsets() { << instance_id_; } else { num_recycled.fetch_add(keys.size(), std::memory_order_relaxed); + check_recycle_task(instance_id_, "recycle_rowsets", num_scanned, + num_recycled, start_time); } }, 0); @@ -1472,6 +1517,7 @@ int InstanceRecycler::recycle_rowsets() { } int InstanceRecycler::recycle_tmp_rowsets() { + const std::string task_name = "recycle_tmp_rowsets"; int num_scanned = 0; int num_expired = 0; int num_recycled = 0; @@ -1487,11 +1533,13 @@ int InstanceRecycler::recycle_tmp_rowsets() { LOG_INFO("begin to recycle tmp rowsets").tag("instance_id", instance_id_); - using namespace std::chrono; - auto start_time = steady_clock::now(); + int64_t start_time = duration_cast(steady_clock::now().time_since_epoch()).count(); + register_recycle_task(task_name, start_time); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { - auto cost = duration(steady_clock::now() - start_time).count(); + unregister_recycle_task(task_name); + int64_t cost = + duration_cast(steady_clock::now().time_since_epoch()).count() - start_time; LOG_INFO("recycle tmp rowsets finished, cost={}s", cost) .tag("instance_id", instance_id_) .tag("num_scanned", num_scanned) @@ -1616,6 +1664,7 @@ int InstanceRecycler::scan_and_recycle( } int InstanceRecycler::abort_timeout_txn() { + const std::string task_name = "abort_timeout_txn"; int num_scanned = 0; int num_timeout = 0; int num_abort = 0; @@ -1629,11 +1678,13 @@ int InstanceRecycler::abort_timeout_txn() { LOG_INFO("begin to abort timeout txn").tag("instance_id", instance_id_); - using namespace std::chrono; - auto start_time = steady_clock::now(); + int64_t start_time = duration_cast(steady_clock::now().time_since_epoch()).count(); + register_recycle_task(task_name, start_time); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { - auto cost = duration(steady_clock::now() - start_time).count(); + unregister_recycle_task(task_name); + int64_t cost = + duration_cast(steady_clock::now().time_since_epoch()).count() - start_time; LOG_INFO("end to abort timeout txn, cost={}s", cost) .tag("instance_id", instance_id_) .tag("num_scanned", num_scanned) @@ -1731,6 +1782,7 @@ int InstanceRecycler::abort_timeout_txn() { } int InstanceRecycler::recycle_expired_txn_label() { + const std::string task_name = "recycle_expired_txn_label"; int num_scanned = 0; int num_expired = 0; int num_recycled = 0; @@ -1744,11 +1796,12 @@ int InstanceRecycler::recycle_expired_txn_label() { LOG_INFO("begin to recycle expire txn").tag("instance_id", instance_id_); - using namespace std::chrono; - auto start_time = steady_clock::now(); - + int64_t start_time = duration_cast(steady_clock::now().time_since_epoch()).count(); + register_recycle_task(task_name, start_time); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { - auto cost = duration(steady_clock::now() - start_time).count(); + unregister_recycle_task(task_name); + int64_t cost = + duration_cast(steady_clock::now().time_since_epoch()).count() - start_time; LOG_INFO("end to recycle expired txn, cost={}s", cost) .tag("instance_id", instance_id_) .tag("num_scanned", num_scanned) @@ -1970,14 +2023,17 @@ int InstanceRecycler::recycle_copy_jobs() { int num_recycled = 0; // Used for INTERNAL stage's copy jobs to tag each batch for log trace uint64_t batch_count = 0; + const std::string task_name = "recycle_copy_jobs"; LOG_INFO("begin to recycle copy jobs").tag("instance_id", instance_id_); - using namespace std::chrono; - auto start_time = steady_clock::now(); + int64_t start_time = duration_cast(steady_clock::now().time_since_epoch()).count(); + register_recycle_task(task_name, start_time); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { - auto cost = duration(steady_clock::now() - start_time).count(); + unregister_recycle_task(task_name); + int64_t cost = + duration_cast(steady_clock::now().time_since_epoch()).count() - start_time; LOG_INFO("recycle copy jobs finished, cost={}s", cost) .tag("instance_id", instance_id_) .tag("num_scanned", num_scanned) @@ -1993,8 +2049,9 @@ int InstanceRecycler::recycle_copy_jobs() { copy_job_key(key_info0, &key0); copy_job_key(key_info1, &key1); std::unordered_map> stage_accessor_map; - auto recycle_func = [&num_scanned, &num_finished, &num_expired, &num_recycled, &batch_count, - &stage_accessor_map, this](std::string_view k, std::string_view v) -> int { + auto recycle_func = [&start_time, &num_scanned, &num_finished, &num_expired, &num_recycled, + &batch_count, &stage_accessor_map, &task_name, + this](std::string_view k, std::string_view v) -> int { ++num_scanned; CopyJobPB copy_job; if (!copy_job.ParseFromArray(v.data(), v.size())) { @@ -2099,6 +2156,7 @@ int InstanceRecycler::recycle_copy_jobs() { } ++num_recycled; + check_recycle_task(instance_id_, task_name, num_scanned, num_recycled, start_time); return 0; }; @@ -2222,14 +2280,17 @@ int InstanceRecycler::init_copy_job_accessor(const std::string& stage_id, int InstanceRecycler::recycle_stage() { int num_scanned = 0; int num_recycled = 0; + const std::string task_name = "recycle_stage"; LOG_INFO("begin to recycle stage").tag("instance_id", instance_id_); - using namespace std::chrono; - auto start_time = steady_clock::now(); + int64_t start_time = duration_cast(steady_clock::now().time_since_epoch()).count(); + register_recycle_task(task_name, start_time); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { - auto cost = duration(steady_clock::now() - start_time).count(); + unregister_recycle_task(task_name); + int64_t cost = + duration_cast(steady_clock::now().time_since_epoch()).count() - start_time; LOG_INFO("recycle stage, cost={}s", cost) .tag("instance_id", instance_id_) .tag("num_scanned", num_scanned) @@ -2245,7 +2306,7 @@ int InstanceRecycler::recycle_stage() { // Elements in `tmp_rowset_keys` has the same lifetime as `it` std::vector stage_keys; - auto recycle_func = [&num_scanned, &num_recycled, &stage_keys, this]( + auto recycle_func = [&start_time, &num_scanned, &num_recycled, &stage_keys, this]( std::string_view k, std::string_view v) -> int { ++num_scanned; RecycleStagePB recycle_stage; @@ -2304,6 +2365,7 @@ int InstanceRecycler::recycle_stage() { return -1; } ++num_recycled; + check_recycle_task(instance_id_, "recycle_stage", num_scanned, num_recycled, start_time); stage_keys.push_back(k); return 0; }; @@ -2325,11 +2387,11 @@ int InstanceRecycler::recycle_stage() { int InstanceRecycler::recycle_expired_stage_objects() { LOG_INFO("begin to recycle expired stage objects").tag("instance_id", instance_id_); - using namespace std::chrono; - auto start_time = steady_clock::now(); + int64_t start_time = duration_cast(steady_clock::now().time_since_epoch()).count(); std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { - auto cost = duration(steady_clock::now() - start_time).count(); + int64_t cost = + duration_cast(steady_clock::now().time_since_epoch()).count() - start_time; LOG_INFO("recycle expired stage objects, cost={}s", cost).tag("instance_id", instance_id_); }); int ret = 0; @@ -2391,4 +2453,37 @@ int InstanceRecycler::recycle_expired_stage_objects() { return ret; } +void InstanceRecycler::register_recycle_task(const std::string& task_name, int64_t start_time) { + std::lock_guard lock(recycle_tasks_mutex); + running_recycle_tasks[task_name] = start_time; +} + +void InstanceRecycler::unregister_recycle_task(const std::string& task_name) { + std::lock_guard lock(recycle_tasks_mutex); + DCHECK(running_recycle_tasks[task_name] > 0); + running_recycle_tasks.erase(task_name); +} + +bool InstanceRecycler::check_recycle_tasks() { + std::map tmp_running_recycle_tasks; + { + std::lock_guard lock(recycle_tasks_mutex); + tmp_running_recycle_tasks = running_recycle_tasks; + } + + bool found = false; + int64_t now = duration_cast(steady_clock::now().time_since_epoch()).count(); + for (auto& [task_name, start_time] : tmp_running_recycle_tasks) { + int64_t cost = now - start_time; + if (cost > config::recycle_task_threshold_seconds) [[unlikely]] { + LOG_INFO("recycle task cost too much time cost={}s", cost) + .tag("instance_id", instance_id_) + .tag("task", task_name); + found = true; + } + } + + return found; +} + } // namespace doris::cloud diff --git a/cloud/src/recycler/recycler.h b/cloud/src/recycler/recycler.h index 34139be6d1a3db..e745e815a79622 100644 --- a/cloud/src/recycler/recycler.h +++ b/cloud/src/recycler/recycler.h @@ -59,6 +59,8 @@ class Recycler { void lease_recycle_jobs(); + void check_recycle_tasks(); + private: friend class RecyclerServiceImpl; @@ -155,6 +157,8 @@ class InstanceRecycler { // returns 0 for success otherwise error int recycle_expired_stage_objects(); + bool check_recycle_tasks(); + private: /** * Scan key-value pairs between [`begin`, `end`), and perform `recycle_func` on each key-value pair. @@ -182,6 +186,10 @@ class InstanceRecycler { int init_copy_job_accessor(const std::string& stage_id, const StagePB::StageType& stage_type, std::shared_ptr* accessor); + void register_recycle_task(const std::string& task_name, int64_t start_time); + + void unregister_recycle_task(const std::string& task_name); + private: std::atomic_bool stopped_ {false}; std::shared_ptr txn_kv_; @@ -195,6 +203,10 @@ class InstanceRecycler { std::mutex recycled_tablets_mtx_; // Store recycled tablets, we can skip deleting rowset data of these tablets because these data has already been deleted. std::unordered_set recycled_tablets_; + + std::mutex recycle_tasks_mutex; + // > + std::map running_recycle_tasks; }; } // namespace doris::cloud diff --git a/cloud/test/recycler_test.cpp b/cloud/test/recycler_test.cpp index cf6d606585d457..87c70833a30d22 100644 --- a/cloud/test/recycler_test.cpp +++ b/cloud/test/recycler_test.cpp @@ -713,6 +713,7 @@ TEST(RecyclerTest, bench_recycle_rowsets) { obj_info->set_prefix("recycle_rowsets"); config::instance_recycler_worker_pool_size = 10; + config::recycle_task_threshold_seconds = 0; InstanceRecycler recycler(txn_kv, instance); ASSERT_EQ(recycler.init(), 0); @@ -723,8 +724,11 @@ TEST(RecyclerTest, bench_recycle_rowsets) { *((int*)limit) = 100; std::this_thread::sleep_for(std::chrono::milliseconds(5)); }); - sp->set_call_back("MockAccessor::delete_objects", - [&](void* p) { std::this_thread::sleep_for(std::chrono::milliseconds(20)); }); + sp->set_call_back("MockAccessor::delete_objects", [&](void* p) { + std::this_thread::sleep_for(std::chrono::seconds(1)); + bool found = recycler.check_recycle_tasks(); + ASSERT_EQ(found, true); + }); sp->set_call_back("MockAccessor::delete_objects_by_prefix", [&](void* p) { std::this_thread::sleep_for(std::chrono::milliseconds(20)); }); sp->enable_processing(); @@ -748,6 +752,7 @@ TEST(RecyclerTest, bench_recycle_rowsets) { } ASSERT_EQ(recycler.recycle_rowsets(), 0); + ASSERT_EQ(recycler.check_recycle_tasks(), false); // check rowset does not exist on obj store std::vector files; diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql index 345d62842d5dc5..730c8bda04d755 100644 --- a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql +++ b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql @@ -1937,3 +1937,506 @@ with serdeproperties ,'separatorChar'=','); insert into employee_gz values ('a', '1.1'), ('b', '2.2'); + +CREATE DATABASE write_test; +use write_test; + +CREATE TABLE `all_types_parquet_snappy_src`( + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double, + `decimal_col1` decimal(9,0), + `decimal_col2` decimal(8,4), + `decimal_col3` decimal(18,6), + `decimal_col4` decimal(38,12), + `string_col` string, + `binary_col` binary, + `date_col` date, + `timestamp_col1` timestamp, + `timestamp_col2` timestamp, + `timestamp_col3` timestamp, + `char_col1` char(50), + `char_col2` char(100), + `char_col3` char(255), + `varchar_col1` varchar(50), + `varchar_col2` varchar(100), + `varchar_col3` varchar(255), + `t_map_string` map, + `t_map_varchar` map, + `t_map_char` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` map, + `t_array_string` array, + `t_array_int` array, + `t_array_bigint` array, + `t_array_float` array, + `t_array_double` array, + `t_array_boolean` array, + `t_array_varchar` array, + `t_array_char` array, + `t_array_decimal_precision_2` array, + `t_array_decimal_precision_4` array, + `t_array_decimal_precision_8` array, + `t_array_decimal_precision_17` array, + `t_array_decimal_precision_18` array, + `t_array_decimal_precision_38` array, + `t_struct_bigint` struct, + `t_complex` map>>, + `t_struct_nested` struct>, + `t_struct_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array, + `dt` int) +stored as parquet +LOCATION + '/user/doris/preinstalled_data/parquet_table/all_types_parquet_snappy_src' +TBLPROPERTIES('parquet.compression'='SNAPPY'); + +CREATE TABLE `all_types_par_parquet_snappy_src`( + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double, + `decimal_col1` decimal(9,0), + `decimal_col2` decimal(8,4), + `decimal_col3` decimal(18,6), + `decimal_col4` decimal(38,12), + `string_col` string, + `binary_col` binary, + `date_col` date, + `timestamp_col1` timestamp, + `timestamp_col2` timestamp, + `timestamp_col3` timestamp, + `char_col1` char(50), + `char_col2` char(100), + `char_col3` char(255), + `varchar_col1` varchar(50), + `varchar_col2` varchar(100), + `varchar_col3` varchar(255), + `t_map_string` map, + `t_map_varchar` map, + `t_map_char` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` map, + `t_array_string` array, + `t_array_int` array, + `t_array_bigint` array, + `t_array_float` array, + `t_array_double` array, + `t_array_boolean` array, + `t_array_varchar` array, + `t_array_char` array, + `t_array_decimal_precision_2` array, + `t_array_decimal_precision_4` array, + `t_array_decimal_precision_8` array, + `t_array_decimal_precision_17` array, + `t_array_decimal_precision_18` array, + `t_array_decimal_precision_38` array, + `t_struct_bigint` struct, + `t_complex` map>>, + `t_struct_nested` struct>, + `t_struct_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array) +PARTITIONED BY ( + `dt` int) +stored as parquet +LOCATION + '/user/doris/preinstalled_data/parquet_table/all_types_par_parquet_snappy_src' +TBLPROPERTIES('parquet.compression'='SNAPPY'); +msck repair table all_types_par_parquet_snappy_src; + +CREATE TABLE `all_types_parquet_snappy`( + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double, + `decimal_col1` decimal(9,0), + `decimal_col2` decimal(8,4), + `decimal_col3` decimal(18,6), + `decimal_col4` decimal(38,12), + `string_col` string, + `binary_col` binary, + `date_col` date, + `timestamp_col1` timestamp, + `timestamp_col2` timestamp, + `timestamp_col3` timestamp, + `char_col1` char(50), + `char_col2` char(100), + `char_col3` char(255), + `varchar_col1` varchar(50), + `varchar_col2` varchar(100), + `varchar_col3` varchar(255), + `t_map_string` map, + `t_map_varchar` map, + `t_map_char` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` map, + `t_array_string` array, + `t_array_int` array, + `t_array_bigint` array, + `t_array_float` array, + `t_array_double` array, + `t_array_boolean` array, + `t_array_varchar` array, + `t_array_char` array, + `t_array_decimal_precision_2` array, + `t_array_decimal_precision_4` array, + `t_array_decimal_precision_8` array, + `t_array_decimal_precision_17` array, + `t_array_decimal_precision_18` array, + `t_array_decimal_precision_38` array, + `t_struct_bigint` struct, + `t_complex` map>>, + `t_struct_nested` struct>, + `t_struct_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array, + `dt` int) +stored as parquet +TBLPROPERTIES('parquet.compression'='SNAPPY'); + +CREATE TABLE `all_types_par_parquet_snappy`( + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double, + `decimal_col1` decimal(9,0), + `decimal_col2` decimal(8,4), + `decimal_col3` decimal(18,6), + `decimal_col4` decimal(38,12), + `string_col` string, + `binary_col` binary, + `date_col` date, + `timestamp_col1` timestamp, + `timestamp_col2` timestamp, + `timestamp_col3` timestamp, + `char_col1` char(50), + `char_col2` char(100), + `char_col3` char(255), + `varchar_col1` varchar(50), + `varchar_col2` varchar(100), + `varchar_col3` varchar(255), + `t_map_string` map, + `t_map_varchar` map, + `t_map_char` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` map, + `t_array_string` array, + `t_array_int` array, + `t_array_bigint` array, + `t_array_float` array, + `t_array_double` array, + `t_array_boolean` array, + `t_array_varchar` array, + `t_array_char` array, + `t_array_decimal_precision_2` array, + `t_array_decimal_precision_4` array, + `t_array_decimal_precision_8` array, + `t_array_decimal_precision_17` array, + `t_array_decimal_precision_18` array, + `t_array_decimal_precision_38` array, + `t_struct_bigint` struct, + `t_complex` map>>, + `t_struct_nested` struct>, + `t_struct_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array) +PARTITIONED BY ( + `dt` int) +stored as parquet +TBLPROPERTIES('parquet.compression'='SNAPPY'); + +CREATE TABLE `all_types_orc_zlib`( + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double, + `decimal_col1` decimal(9,0), + `decimal_col2` decimal(8,4), + `decimal_col3` decimal(18,6), + `decimal_col4` decimal(38,12), + `string_col` string, + `binary_col` binary, + `date_col` date, + `timestamp_col1` timestamp, + `timestamp_col2` timestamp, + `timestamp_col3` timestamp, + `char_col1` char(50), + `char_col2` char(100), + `char_col3` char(255), + `varchar_col1` varchar(50), + `varchar_col2` varchar(100), + `varchar_col3` varchar(255), + `t_map_string` map, + `t_map_varchar` map, + `t_map_char` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` map, + `t_array_string` array, + `t_array_int` array, + `t_array_bigint` array, + `t_array_float` array, + `t_array_double` array, + `t_array_boolean` array, + `t_array_varchar` array, + `t_array_char` array, + `t_array_decimal_precision_2` array, + `t_array_decimal_precision_4` array, + `t_array_decimal_precision_8` array, + `t_array_decimal_precision_17` array, + `t_array_decimal_precision_18` array, + `t_array_decimal_precision_38` array, + `t_struct_bigint` struct, + `t_complex` map>>, + `t_struct_nested` struct>, + `t_struct_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array, + `dt` int) +stored as orc +TBLPROPERTIES("orc.compress"="ZLIB"); + +CREATE TABLE `all_types_par_orc_zlib`( + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double, + `decimal_col1` decimal(9,0), + `decimal_col2` decimal(8,4), + `decimal_col3` decimal(18,6), + `decimal_col4` decimal(38,12), + `string_col` string, + `binary_col` binary, + `date_col` date, + `timestamp_col1` timestamp, + `timestamp_col2` timestamp, + `timestamp_col3` timestamp, + `char_col1` char(50), + `char_col2` char(100), + `char_col3` char(255), + `varchar_col1` varchar(50), + `varchar_col2` varchar(100), + `varchar_col3` varchar(255), + `t_map_string` map, + `t_map_varchar` map, + `t_map_char` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` map, + `t_array_string` array, + `t_array_int` array, + `t_array_bigint` array, + `t_array_float` array, + `t_array_double` array, + `t_array_boolean` array, + `t_array_varchar` array, + `t_array_char` array, + `t_array_decimal_precision_2` array, + `t_array_decimal_precision_4` array, + `t_array_decimal_precision_8` array, + `t_array_decimal_precision_17` array, + `t_array_decimal_precision_18` array, + `t_array_decimal_precision_38` array, + `t_struct_bigint` struct, + `t_complex` map>>, + `t_struct_nested` struct>, + `t_struct_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array) +PARTITIONED BY ( + `dt` int) +stored as orc +TBLPROPERTIES("orc.compress"="ZLIB"); + +CREATE TABLE `all_partition_types1_parquet_snappy_src`( + `id` int +) +PARTITIONED BY ( + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double) +stored as parquet +LOCATION + '/user/doris/preinstalled_data/parquet_table/all_partition_types1_parquet_snappy_src' +TBLPROPERTIES('parquet.compression'='SNAPPY'); +msck repair table all_partition_types1_parquet_snappy_src; + +CREATE TABLE `all_partition_types1_parquet_snappy`( + `id` int +) +PARTITIONED BY ( + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double) +stored as parquet +TBLPROPERTIES('parquet.compression'='SNAPPY'); + +CREATE TABLE `all_partition_types1_orc_zlib`( + `id` int +) +PARTITIONED BY ( + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double) +stored as orc +TBLPROPERTIES("orc.compress"="ZLIB"); + +CREATE TABLE `all_partition_types2_parquet_snappy_src`( + `id` int +) +PARTITIONED BY ( + `decimal_col` decimal(18,6), + `string_col` string, + `binary_col` binary, + `date_col` date, + `timestamp_col` timestamp, + `char_col` char(50), + `varchar_col` varchar(50)) +stored as parquet +LOCATION + '/user/doris/preinstalled_data/parquet_table/all_partition_types2_parquet_snappy_src' +TBLPROPERTIES('parquet.compression'='SNAPPY'); +msck repair table all_partition_types2_parquet_snappy_src; + +CREATE TABLE `all_partition_types2_parquet_snappy`( + `id` int +) +PARTITIONED BY ( + `decimal_col` decimal(18,6), + `string_col` string, + `binary_col` binary, + `date_col` date, + `timestamp_col` timestamp, + `char_col` char(50), + `varchar_col` varchar(50)) +stored as parquet +TBLPROPERTIES('parquet.compression'='SNAPPY'); + +CREATE TABLE `all_partition_types2_orc_zlib`( + `id` int +) +PARTITIONED BY ( + `decimal_col` decimal(18,6), + `string_col` string, + `binary_col` binary, + `date_col` date, + `timestamp_col` timestamp, + `char_col` char(50), + `varchar_col` varchar(50)) +stored as orc +TBLPROPERTIES("orc.compress"="ZLIB"); diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types1_parquet_snappy_src/boolean_col=false/tinyint_col=-128/smallint_col=-32768/int_col=-2147483648/bigint_col=-9223372036854775808/float_col=-123.45/double_col=-123456.789/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types1_parquet_snappy_src/boolean_col=false/tinyint_col=-128/smallint_col=-32768/int_col=-2147483648/bigint_col=-9223372036854775808/float_col=-123.45/double_col=-123456.789/000000_0 new file mode 100644 index 00000000000000..bec160f5b5bc55 Binary files /dev/null and b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types1_parquet_snappy_src/boolean_col=false/tinyint_col=-128/smallint_col=-32768/int_col=-2147483648/bigint_col=-9223372036854775808/float_col=-123.45/double_col=-123456.789/000000_0 differ diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types1_parquet_snappy_src/boolean_col=true/tinyint_col=127/smallint_col=32767/int_col=2147483647/bigint_col=9223372036854775807/float_col=123.45/double_col=123456.789/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types1_parquet_snappy_src/boolean_col=true/tinyint_col=127/smallint_col=32767/int_col=2147483647/bigint_col=9223372036854775807/float_col=123.45/double_col=123456.789/000000_0 new file mode 100644 index 00000000000000..35e86534a7ddbf Binary files /dev/null and b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types1_parquet_snappy_src/boolean_col=true/tinyint_col=127/smallint_col=32767/int_col=2147483647/bigint_col=9223372036854775807/float_col=123.45/double_col=123456.789/000000_0 differ diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types2_parquet_snappy_src/decimal_col=-123456.789012/string_col=string_value/binary_col=62 69 6e 61 72 79 5f 76 61 6c 75 65/date_col=2024-03-21/timestamp_col=2024-03-21 12%3A00%3A00/char_col=char_value1 /varchar_col=varchar_value1/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types2_parquet_snappy_src/decimal_col=-123456.789012/string_col=string_value/binary_col=62 69 6e 61 72 79 5f 76 61 6c 75 65/date_col=2024-03-21/timestamp_col=2024-03-21 12%3A00%3A00/char_col=char_value1 /varchar_col=varchar_value1/000000_0 new file mode 100644 index 00000000000000..958c6c3cef5275 Binary files /dev/null and b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types2_parquet_snappy_src/decimal_col=-123456.789012/string_col=string_value/binary_col=62 69 6e 61 72 79 5f 76 61 6c 75 65/date_col=2024-03-21/timestamp_col=2024-03-21 12%3A00%3A00/char_col=char_value1 /varchar_col=varchar_value1/000000_0 differ diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types2_parquet_snappy_src/decimal_col=-123456.789012/string_col=string_value/binary_col=62 69 6e 61 72 79 5f 76 61 6c 75 65/date_col=2024-03-22/timestamp_col=2024-03-22 12%3A00%3A00/char_col=char_value1 /varchar_col=varchar_value1/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types2_parquet_snappy_src/decimal_col=-123456.789012/string_col=string_value/binary_col=62 69 6e 61 72 79 5f 76 61 6c 75 65/date_col=2024-03-22/timestamp_col=2024-03-22 12%3A00%3A00/char_col=char_value1 /varchar_col=varchar_value1/000000_0 new file mode 100644 index 00000000000000..a17c9c584c712b Binary files /dev/null and b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types2_parquet_snappy_src/decimal_col=-123456.789012/string_col=string_value/binary_col=62 69 6e 61 72 79 5f 76 61 6c 75 65/date_col=2024-03-22/timestamp_col=2024-03-22 12%3A00%3A00/char_col=char_value1 /varchar_col=varchar_value1/000000_0 differ diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types2_parquet_snappy_src/decimal_col=123456.789012/string_col=string_value/binary_col=62 69 6e 61 72 79 5f 76 61 6c 75 65/date_col=2024-03-20/timestamp_col=2024-03-20 12%3A00%3A00/char_col=char_value1 /varchar_col=varchar_value1/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types2_parquet_snappy_src/decimal_col=123456.789012/string_col=string_value/binary_col=62 69 6e 61 72 79 5f 76 61 6c 75 65/date_col=2024-03-20/timestamp_col=2024-03-20 12%3A00%3A00/char_col=char_value1 /varchar_col=varchar_value1/000000_0 new file mode 100644 index 00000000000000..6d039cd8082f7e Binary files /dev/null and b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_partition_types2_parquet_snappy_src/decimal_col=123456.789012/string_col=string_value/binary_col=62 69 6e 61 72 79 5f 76 61 6c 75 65/date_col=2024-03-20/timestamp_col=2024-03-20 12%3A00%3A00/char_col=char_value1 /varchar_col=varchar_value1/000000_0 differ diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_par_parquet_snappy_src/dt=20240320/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_par_parquet_snappy_src/dt=20240320/000000_0 new file mode 100644 index 00000000000000..e80ee82fbf6ef6 Binary files /dev/null and b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_par_parquet_snappy_src/dt=20240320/000000_0 differ diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_par_parquet_snappy_src/dt=20240321/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_par_parquet_snappy_src/dt=20240321/000000_0 new file mode 100644 index 00000000000000..aeb0b590e00674 Binary files /dev/null and b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_par_parquet_snappy_src/dt=20240321/000000_0 differ diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_par_parquet_snappy_src/dt=20240322/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_par_parquet_snappy_src/dt=20240322/000000_0 new file mode 100644 index 00000000000000..3830d0a4f64f17 Binary files /dev/null and b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_par_parquet_snappy_src/dt=20240322/000000_0 differ diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_parquet_snappy_src/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_parquet_snappy_src/000000_0 new file mode 100644 index 00000000000000..69a2a8ec8c30a5 Binary files /dev/null and b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/all_types_parquet_snappy_src/000000_0 differ diff --git a/fe/be-java-extensions/hudi-scanner/src/main/resources/package.xml b/fe/be-java-extensions/hudi-scanner/src/main/resources/package.xml index 4bbb2610603363..206b7d3a7ca6d0 100644 --- a/fe/be-java-extensions/hudi-scanner/src/main/resources/package.xml +++ b/fe/be-java-extensions/hudi-scanner/src/main/resources/package.xml @@ -25,17 +25,13 @@ under the License. jar false - - + + + ${project.build.outputDirectory} / - true - true - runtime - - - **/Log4j2Plugins.dat - - - - + + **/*.class + + + diff --git a/fe/be-java-extensions/java-common/src/main/resources/package.xml b/fe/be-java-extensions/java-common/src/main/resources/package.xml index 4bbb2610603363..206b7d3a7ca6d0 100644 --- a/fe/be-java-extensions/java-common/src/main/resources/package.xml +++ b/fe/be-java-extensions/java-common/src/main/resources/package.xml @@ -25,17 +25,13 @@ under the License. jar false - - + + + ${project.build.outputDirectory} / - true - true - runtime - - - **/Log4j2Plugins.dat - - - - + + **/*.class + + + diff --git a/fe/be-java-extensions/jdbc-scanner/src/main/resources/package.xml b/fe/be-java-extensions/jdbc-scanner/src/main/resources/package.xml index 4bbb2610603363..206b7d3a7ca6d0 100644 --- a/fe/be-java-extensions/jdbc-scanner/src/main/resources/package.xml +++ b/fe/be-java-extensions/jdbc-scanner/src/main/resources/package.xml @@ -25,17 +25,13 @@ under the License. jar false - - + + + ${project.build.outputDirectory} / - true - true - runtime - - - **/Log4j2Plugins.dat - - - - + + **/*.class + + + diff --git a/fe/be-java-extensions/max-compute-scanner/src/main/resources/package.xml b/fe/be-java-extensions/max-compute-scanner/src/main/resources/package.xml index 4bbb2610603363..206b7d3a7ca6d0 100644 --- a/fe/be-java-extensions/max-compute-scanner/src/main/resources/package.xml +++ b/fe/be-java-extensions/max-compute-scanner/src/main/resources/package.xml @@ -25,17 +25,13 @@ under the License. jar false - - + + + ${project.build.outputDirectory} / - true - true - runtime - - - **/Log4j2Plugins.dat - - - - + + **/*.class + + + diff --git a/fe/be-java-extensions/paimon-scanner/src/main/resources/package.xml b/fe/be-java-extensions/paimon-scanner/src/main/resources/package.xml index 4bbb2610603363..206b7d3a7ca6d0 100644 --- a/fe/be-java-extensions/paimon-scanner/src/main/resources/package.xml +++ b/fe/be-java-extensions/paimon-scanner/src/main/resources/package.xml @@ -25,17 +25,13 @@ under the License. jar false - - + + + ${project.build.outputDirectory} / - true - true - runtime - - - **/Log4j2Plugins.dat - - - - + + **/*.class + + + diff --git a/fe/be-java-extensions/preload-extensions/pom.xml b/fe/be-java-extensions/preload-extensions/pom.xml index 281b18955a9827..4952824b118b87 100644 --- a/fe/be-java-extensions/preload-extensions/pom.xml +++ b/fe/be-java-extensions/preload-extensions/pom.xml @@ -252,23 +252,30 @@ under the License. org.apache.maven.plugins maven-assembly-plugin - - - src/main/resources/package.xml - - - - - - - - make-assembly + make-assembly-project package single + + + src/main/resources/package-proj.xml + + + + + make-assembly-deps + package + + single + + + + src/main/resources/package-deps.xml + + diff --git a/fe/be-java-extensions/preload-extensions/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/fe/be-java-extensions/preload-extensions/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java new file mode 100644 index 00000000000000..e5c7957d770d3a --- /dev/null +++ b/fe/be-java-extensions/preload-extensions/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -0,0 +1,3630 @@ +/* + * 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.hadoop.hive.metastore; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.common.StatsSetupConst; +import org.apache.hadoop.hive.common.ValidTxnList; +import org.apache.hadoop.hive.common.ValidWriteIdList; +import org.apache.hadoop.hive.metastore.HiveVersionUtil.HiveVersion; +import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME; +import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; +import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest; +import org.apache.hadoop.hive.metastore.api.AddCheckConstraintRequest; +import org.apache.hadoop.hive.metastore.api.AddDefaultConstraintRequest; +import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions; +import org.apache.hadoop.hive.metastore.api.AddForeignKeyRequest; +import org.apache.hadoop.hive.metastore.api.AddNotNullConstraintRequest; +import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest; +import org.apache.hadoop.hive.metastore.api.AddPartitionsResult; +import org.apache.hadoop.hive.metastore.api.AddPrimaryKeyRequest; +import org.apache.hadoop.hive.metastore.api.AddUniqueConstraintRequest; +import org.apache.hadoop.hive.metastore.api.AggrStats; +import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsRequest; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.hadoop.hive.metastore.api.AlterCatalogRequest; +import org.apache.hadoop.hive.metastore.api.AlterISchemaRequest; +import org.apache.hadoop.hive.metastore.api.CacheFileMetadataRequest; +import org.apache.hadoop.hive.metastore.api.CacheFileMetadataResult; +import org.apache.hadoop.hive.metastore.api.Catalog; +import org.apache.hadoop.hive.metastore.api.CheckConstraintsRequest; +import org.apache.hadoop.hive.metastore.api.CheckLockRequest; +import org.apache.hadoop.hive.metastore.api.ClearFileMetadataRequest; +import org.apache.hadoop.hive.metastore.api.ClientCapabilities; +import org.apache.hadoop.hive.metastore.api.ClientCapability; +import org.apache.hadoop.hive.metastore.api.CmRecycleRequest; +import org.apache.hadoop.hive.metastore.api.CmRecycleResponse; +import org.apache.hadoop.hive.metastore.api.ColumnStatistics; +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; +import org.apache.hadoop.hive.metastore.api.CommitTxnRequest; +import org.apache.hadoop.hive.metastore.api.CompactionRequest; +import org.apache.hadoop.hive.metastore.api.CompactionResponse; +import org.apache.hadoop.hive.metastore.api.CompactionType; +import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException; +import org.apache.hadoop.hive.metastore.api.CreateCatalogRequest; +import org.apache.hadoop.hive.metastore.api.CreationMetadata; +import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId; +import org.apache.hadoop.hive.metastore.api.DataOperationType; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.DefaultConstraintsRequest; +import org.apache.hadoop.hive.metastore.api.DropCatalogRequest; +import org.apache.hadoop.hive.metastore.api.DropConstraintRequest; +import org.apache.hadoop.hive.metastore.api.DropPartitionsExpr; +import org.apache.hadoop.hive.metastore.api.DropPartitionsRequest; +import org.apache.hadoop.hive.metastore.api.EnvironmentContext; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.FindSchemasByColsResp; +import org.apache.hadoop.hive.metastore.api.FindSchemasByColsRqst; +import org.apache.hadoop.hive.metastore.api.FireEventRequest; +import org.apache.hadoop.hive.metastore.api.FireEventResponse; +import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest; +import org.apache.hadoop.hive.metastore.api.Function; +import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse; +import org.apache.hadoop.hive.metastore.api.GetCatalogRequest; +import org.apache.hadoop.hive.metastore.api.GetCatalogResponse; +import org.apache.hadoop.hive.metastore.api.GetCatalogsResponse; +import org.apache.hadoop.hive.metastore.api.GetFileMetadataByExprRequest; +import org.apache.hadoop.hive.metastore.api.GetFileMetadataByExprResult; +import org.apache.hadoop.hive.metastore.api.GetFileMetadataRequest; +import org.apache.hadoop.hive.metastore.api.GetFileMetadataResult; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse; +import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest; +import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse; +import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest; +import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse; +import org.apache.hadoop.hive.metastore.api.GetRuntimeStatsRequest; +import org.apache.hadoop.hive.metastore.api.GetSerdeRequest; +import org.apache.hadoop.hive.metastore.api.GetTableRequest; +import org.apache.hadoop.hive.metastore.api.GetTablesRequest; +import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest; +import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse; +import org.apache.hadoop.hive.metastore.api.GrantRevokePrivilegeRequest; +import org.apache.hadoop.hive.metastore.api.GrantRevokePrivilegeResponse; +import org.apache.hadoop.hive.metastore.api.GrantRevokeRoleRequest; +import org.apache.hadoop.hive.metastore.api.GrantRevokeRoleResponse; +import org.apache.hadoop.hive.metastore.api.GrantRevokeType; +import org.apache.hadoop.hive.metastore.api.HeartbeatRequest; +import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeRequest; +import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse; +import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege; +import org.apache.hadoop.hive.metastore.api.HiveObjectRef; +import org.apache.hadoop.hive.metastore.api.ISchema; +import org.apache.hadoop.hive.metastore.api.ISchemaName; +import org.apache.hadoop.hive.metastore.api.InvalidObjectException; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.MapSchemaVersionToSerdeRequest; +import org.apache.hadoop.hive.metastore.api.Materialization; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.MetadataPpdResult; +import org.apache.hadoop.hive.metastore.api.NoSuchLockException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; +import org.apache.hadoop.hive.metastore.api.NotNullConstraintsRequest; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.api.NotificationEventRequest; +import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; +import org.apache.hadoop.hive.metastore.api.NotificationEventsCountRequest; +import org.apache.hadoop.hive.metastore.api.NotificationEventsCountResponse; +import org.apache.hadoop.hive.metastore.api.OpenTxnRequest; +import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.PartitionEventType; +import org.apache.hadoop.hive.metastore.api.PartitionValuesRequest; +import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse; +import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest; +import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult; +import org.apache.hadoop.hive.metastore.api.PartitionsStatsRequest; +import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest; +import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; +import org.apache.hadoop.hive.metastore.api.PrincipalType; +import org.apache.hadoop.hive.metastore.api.PrivilegeBag; +import org.apache.hadoop.hive.metastore.api.PutFileMetadataRequest; +import org.apache.hadoop.hive.metastore.api.ReplTblWriteIdStateRequest; +import org.apache.hadoop.hive.metastore.api.RequestPartsSpec; +import org.apache.hadoop.hive.metastore.api.Role; +import org.apache.hadoop.hive.metastore.api.RuntimeStat; +import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint; +import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint; +import org.apache.hadoop.hive.metastore.api.SQLForeignKey; +import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint; +import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey; +import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint; +import org.apache.hadoop.hive.metastore.api.SchemaVersion; +import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor; +import org.apache.hadoop.hive.metastore.api.SchemaVersionState; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest; +import org.apache.hadoop.hive.metastore.api.SetSchemaVersionStateRequest; +import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; +import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; +import org.apache.hadoop.hive.metastore.api.ShowLocksRequest; +import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.TableMeta; +import org.apache.hadoop.hive.metastore.api.TableStatsRequest; +import org.apache.hadoop.hive.metastore.api.TableValidWriteIds; +import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore; +import org.apache.hadoop.hive.metastore.api.TxnAbortedException; +import org.apache.hadoop.hive.metastore.api.TxnOpenException; +import org.apache.hadoop.hive.metastore.api.TxnToWriteId; +import org.apache.hadoop.hive.metastore.api.Type; +import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest; +import org.apache.hadoop.hive.metastore.api.UnknownDBException; +import org.apache.hadoop.hive.metastore.api.UnlockRequest; +import org.apache.hadoop.hive.metastore.api.WMAlterPoolRequest; +import org.apache.hadoop.hive.metastore.api.WMAlterResourcePlanRequest; +import org.apache.hadoop.hive.metastore.api.WMAlterResourcePlanResponse; +import org.apache.hadoop.hive.metastore.api.WMAlterTriggerRequest; +import org.apache.hadoop.hive.metastore.api.WMCreateOrDropTriggerToPoolMappingRequest; +import org.apache.hadoop.hive.metastore.api.WMCreateOrUpdateMappingRequest; +import org.apache.hadoop.hive.metastore.api.WMCreatePoolRequest; +import org.apache.hadoop.hive.metastore.api.WMCreateResourcePlanRequest; +import org.apache.hadoop.hive.metastore.api.WMCreateTriggerRequest; +import org.apache.hadoop.hive.metastore.api.WMDropMappingRequest; +import org.apache.hadoop.hive.metastore.api.WMDropPoolRequest; +import org.apache.hadoop.hive.metastore.api.WMDropResourcePlanRequest; +import org.apache.hadoop.hive.metastore.api.WMDropTriggerRequest; +import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan; +import org.apache.hadoop.hive.metastore.api.WMGetActiveResourcePlanRequest; +import org.apache.hadoop.hive.metastore.api.WMGetAllResourcePlanRequest; +import org.apache.hadoop.hive.metastore.api.WMGetResourcePlanRequest; +import org.apache.hadoop.hive.metastore.api.WMGetTriggersForResourePlanRequest; +import org.apache.hadoop.hive.metastore.api.WMMapping; +import org.apache.hadoop.hive.metastore.api.WMNullablePool; +import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan; +import org.apache.hadoop.hive.metastore.api.WMPool; +import org.apache.hadoop.hive.metastore.api.WMResourcePlan; +import org.apache.hadoop.hive.metastore.api.WMTrigger; +import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanRequest; +import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; +import org.apache.hadoop.hive.metastore.hooks.URIResolverHook; +import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy; +import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; +import org.apache.hadoop.hive.metastore.utils.JavaUtils; +import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; +import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog; +import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependCatalogToDbName; +import org.apache.hadoop.hive.metastore.utils.ObjectPair; +import org.apache.hadoop.hive.metastore.utils.SecurityUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import shade.doris.hive.org.apache.thrift.TApplicationException; +import shade.doris.hive.org.apache.thrift.TException; +import shade.doris.hive.org.apache.thrift.protocol.TBinaryProtocol; +import shade.doris.hive.org.apache.thrift.protocol.TCompactProtocol; +import shade.doris.hive.org.apache.thrift.protocol.TProtocol; +import shade.doris.hive.org.apache.thrift.transport.TFramedTransport; +import shade.doris.hive.org.apache.thrift.transport.TSocket; +import shade.doris.hive.org.apache.thrift.transport.TTransport; +import shade.doris.hive.org.apache.thrift.transport.TTransportException; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; +import java.net.InetAddress; +import java.net.URI; +import java.net.UnknownHostException; +import java.nio.ByteBuffer; +import java.security.PrivilegedExceptionAction; +import java.security.SecureRandom; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; +import java.util.Random; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import javax.annotation.Nullable; +import javax.security.auth.login.LoginException; + +/** + * Hive Metastore Client. + * The public implementation of IMetaStoreClient. Methods not inherited from IMetaStoreClient + * are not public and can change. Hence this is marked as unstable. + * For users who require retry mechanism when the connection between metastore and client is + * broken, RetryingMetaStoreClient class should be used. + * + * Copied From + * https://github.com/apache/hive/blob/rel/release-3.1.3/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java + * Doris Modification. + * To support different type of hive, copy this file from hive repo and modify some method based on hive version + * 1. getAllDatabases() + * 2. getAllTables() + * 3. tableExists() + * 4. listPartitionNames() + * 5. getPartition() + * 6. getTable() + * 7. getSchema() + * 8. getTableColumnStatistics() + * 9. getPartitionColumnStatistics() + * 10. getPartitionsByNames() + * 11. listPartitions() + * 12. alter_partition() + * 13. add_partitions() + * 14. dropPartition() + * 15. alter_table() + * 16. alter_table_with_environmentContext() + * 17. renamePartition() + * 18. truncateTable() + * 19. drop_table_with_environment_context() + * + * ATTN: There is a copy of this file in be-java-extensions. + * If you want to modify this file, please modify the file in be-java-extensions. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable { + /** + * Capabilities of the current client. If this client talks to a MetaStore server in a manner + * implying the usage of some expanded features that require client-side support that this client + * doesn't have (e.g. a getting a table of a new type), it will get back failures when the + * capability checking is enabled (the default). + */ + public final static ClientCapabilities VERSION = new ClientCapabilities( + Lists.newArrayList(ClientCapability.INSERT_ONLY_TABLES)); + // Test capability for tests. + public final static ClientCapabilities TEST_VERSION = new ClientCapabilities( + Lists.newArrayList(ClientCapability.INSERT_ONLY_TABLES, ClientCapability.TEST_CAPABILITY)); + + ThriftHiveMetastore.Iface client = null; + private TTransport transport = null; + private boolean isConnected = false; + private URI metastoreUris[]; + private final HiveMetaHookLoader hookLoader; + protected final Configuration conf; + // Keep a copy of HiveConf so if Session conf changes, we may need to get a new HMS client. + private String tokenStrForm; + private final boolean localMetaStore; + private final MetaStoreFilterHook filterHook; + private final URIResolverHook uriResolverHook; + private final int fileMetadataBatchSize; + + private Map currentMetaVars; + + private static final AtomicInteger connCount = new AtomicInteger(0); + + // for thrift connects + private int retries = 5; + private long retryDelaySeconds = 0; + private final ClientCapabilities version; + + private final HiveVersion hiveVersion; + + private static final Logger LOG = LogManager.getLogger(HiveMetaStoreClient.class); + + // copied from ErrorMsg.java + public static final String REPL_EVENTS_MISSING_IN_METASTORE = "Notification events are missing in the meta store."; + + public HiveMetaStoreClient(Configuration conf) throws MetaException { + this(conf, null, true); + } + + public HiveMetaStoreClient(Configuration conf, HiveMetaHookLoader hookLoader) throws MetaException { + this(conf, hookLoader, true); + } + + public HiveMetaStoreClient(Configuration conf, HiveMetaHookLoader hookLoader, Boolean allowEmbedded) + throws MetaException { + + this.hookLoader = hookLoader; + if (conf == null) { + conf = MetastoreConf.newMetastoreConf(); + this.conf = conf; + } else { + this.conf = new Configuration(conf); + } + + hiveVersion = HiveVersionUtil.getVersion(conf.get("hive.version")); + LOG.info("Loading Doris HiveMetaStoreClient. Hive version: " + conf.get("hive.version")); + + // For hive 2.3.7, there is no ClientCapability.INSERT_ONLY_TABLES + if (hiveVersion == HiveVersion.V1_0 || hiveVersion == HiveVersion.V2_0 || hiveVersion == HiveVersion.V2_3) { + version = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST) ? TEST_VERSION : null; + } else { + version = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST) ? TEST_VERSION : VERSION; + } + + filterHook = loadFilterHooks(); + uriResolverHook = loadUriResolverHook(); + fileMetadataBatchSize = MetastoreConf.getIntVar( + conf, ConfVars.BATCH_RETRIEVE_OBJECTS_MAX); + + String msUri = MetastoreConf.getVar(conf, ConfVars.THRIFT_URIS); + localMetaStore = MetastoreConf.isEmbeddedMetaStore(msUri); + if (localMetaStore) { + if (!allowEmbedded) { + throw new MetaException("Embedded metastore is not allowed here. Please configure " + + ConfVars.THRIFT_URIS.toString() + "; it is currently set to [" + msUri + "]"); + } + // instantiate the metastore server handler directly instead of connecting + // through the network + client = HiveMetaStore.newRetryingHMSHandler("hive client", this.conf, true); + isConnected = true; + snapshotActiveConf(); + return; + } + + // get the number retries + retries = MetastoreConf.getIntVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES); + retryDelaySeconds = MetastoreConf.getTimeVar(conf, + ConfVars.CLIENT_CONNECT_RETRY_DELAY, TimeUnit.SECONDS); + + // user wants file store based configuration + if (MetastoreConf.getVar(conf, ConfVars.THRIFT_URIS) != null) { + resolveUris(); + } else { + LOG.error("NOT getting uris from conf"); + throw new MetaException("MetaStoreURIs not found in conf file"); + } + + // If HADOOP_PROXY_USER is set in env or property, + // then need to create metastore client that proxies as that user. + String HADOOP_PROXY_USER = "HADOOP_PROXY_USER"; + String proxyUser = System.getenv(HADOOP_PROXY_USER); + if (proxyUser == null) { + proxyUser = System.getProperty(HADOOP_PROXY_USER); + } + // if HADOOP_PROXY_USER is set, create DelegationToken using real user + if (proxyUser != null) { + LOG.info(HADOOP_PROXY_USER + " is set. Using delegation " + + "token for HiveMetaStore connection."); + try { + UserGroupInformation.getLoginUser().getRealUser().doAs( + new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + open(); + return null; + } + }); + String delegationTokenPropString = "DelegationTokenForHiveMetaStoreServer"; + String delegationTokenStr = getDelegationToken(proxyUser, proxyUser); + SecurityUtils.setTokenStr(UserGroupInformation.getCurrentUser(), delegationTokenStr, + delegationTokenPropString); + MetastoreConf.setVar(this.conf, ConfVars.TOKEN_SIGNATURE, delegationTokenPropString); + close(); + } catch (Exception e) { + LOG.error("Error while setting delegation token for " + proxyUser, e); + if (e instanceof MetaException) { + throw (MetaException) e; + } else { + throw new MetaException(e.getMessage()); + } + } + } + // finally open the store + open(); + } + + private void resolveUris() throws MetaException { + String metastoreUrisString[] = MetastoreConf.getVar(conf, + ConfVars.THRIFT_URIS).split(","); + + List metastoreURIArray = new ArrayList(); + try { + int i = 0; + for (String s : metastoreUrisString) { + URI tmpUri = new URI(s); + if (tmpUri.getScheme() == null) { + throw new IllegalArgumentException("URI: " + s + + " does not have a scheme"); + } + if (uriResolverHook != null) { + metastoreURIArray.addAll(uriResolverHook.resolveURI(tmpUri)); + } else { + metastoreURIArray.add(new URI( + tmpUri.getScheme(), + tmpUri.getUserInfo(), + HadoopThriftAuthBridge.getBridge().getCanonicalHostName(tmpUri.getHost()), + tmpUri.getPort(), + tmpUri.getPath(), + tmpUri.getQuery(), + tmpUri.getFragment() + )); + } + } + metastoreUris = new URI[metastoreURIArray.size()]; + for (int j = 0; j < metastoreURIArray.size(); j++) { + metastoreUris[j] = metastoreURIArray.get(j); + } + + if (MetastoreConf.getVar(conf, ConfVars.THRIFT_URI_SELECTION).equalsIgnoreCase("RANDOM")) { + List uriList = Arrays.asList(metastoreUris); + Collections.shuffle(uriList); + metastoreUris = uriList.toArray(new URI[uriList.size()]); + } + } catch (IllegalArgumentException e) { + throw (e); + } catch (Exception e) { + MetaStoreUtils.logAndThrowMetaException(e); + } + } + + + private MetaStoreFilterHook loadFilterHooks() throws IllegalStateException { + Class authProviderClass = MetastoreConf. + getClass(conf, ConfVars.FILTER_HOOK, DefaultMetaStoreFilterHookImpl.class, + MetaStoreFilterHook.class); + String msg = "Unable to create instance of " + authProviderClass.getName() + ": "; + try { + Constructor constructor = + authProviderClass.getConstructor(Configuration.class); + return constructor.newInstance(conf); + } catch (NoSuchMethodException | SecurityException | IllegalAccessException | InstantiationException | + IllegalArgumentException | InvocationTargetException e) { + throw new IllegalStateException(msg + e.getMessage(), e); + } + } + + // multiple clients may initialize the hook at the same time + synchronized private URIResolverHook loadUriResolverHook() throws IllegalStateException { + + String uriResolverClassName = + MetastoreConf.getAsString(conf, ConfVars.URI_RESOLVER); + if (uriResolverClassName.equals("")) { + return null; + } else { + LOG.info("Loading uri resolver" + uriResolverClassName); + try { + Class uriResolverClass = Class.forName(uriResolverClassName, true, + JavaUtils.getClassLoader()); + return (URIResolverHook) ReflectionUtils.newInstance(uriResolverClass, null); + } catch (Exception e) { + LOG.error("Exception loading uri resolver hook" + e); + return null; + } + } + } + + /** + * Swaps the first element of the metastoreUris array with a random element from the + * remainder of the array. + */ + private void promoteRandomMetaStoreURI() { + if (metastoreUris.length <= 1) { + return; + } + Random rng = new SecureRandom(); + int index = rng.nextInt(metastoreUris.length - 1) + 1; + URI tmp = metastoreUris[0]; + metastoreUris[0] = metastoreUris[index]; + metastoreUris[index] = tmp; + } + + @VisibleForTesting + public TTransport getTTransport() { + return transport; + } + + @Override + public boolean isLocalMetaStore() { + return localMetaStore; + } + + @Override + public boolean isCompatibleWith(Configuration conf) { + // Make a copy of currentMetaVars, there is a race condition that + // currentMetaVars might be changed during the execution of the method + Map currentMetaVarsCopy = currentMetaVars; + if (currentMetaVarsCopy == null) { + return false; // recreate + } + boolean compatible = true; + for (ConfVars oneVar : MetastoreConf.metaVars) { + // Since metaVars are all of different types, use string for comparison + String oldVar = currentMetaVarsCopy.get(oneVar.getVarname()); + String newVar = MetastoreConf.getAsString(conf, oneVar); + if (oldVar == null || + (oneVar.isCaseSensitive() ? !oldVar.equals(newVar) : !oldVar.equalsIgnoreCase(newVar))) { + LOG.info("Mestastore configuration " + oneVar.toString() + + " changed from " + oldVar + " to " + newVar); + compatible = false; + } + } + return compatible; + } + + @Override + public void setHiveAddedJars(String addedJars) { + MetastoreConf.setVar(conf, ConfVars.ADDED_JARS, addedJars); + } + + @Override + public void reconnect() throws MetaException { + if (localMetaStore) { + // For direct DB connections we don't yet support reestablishing connections. + throw new MetaException("For direct MetaStore DB connections, we don't support retries" + + " at the client level."); + } else { + close(); + + if (uriResolverHook != null) { + // for dynamic uris, re-lookup if there are new metastore locations + resolveUris(); + } + + if (MetastoreConf.getVar(conf, ConfVars.THRIFT_URI_SELECTION).equalsIgnoreCase("RANDOM")) { + // Swap the first element of the metastoreUris[] with a random element from the rest + // of the array. Rationale being that this method will generally be called when the default + // connection has died and the default connection is likely to be the first array element. + promoteRandomMetaStoreURI(); + } + open(); + } + } + + @Override + public void alter_table(String dbname, String tbl_name, Table new_tbl) throws TException { + alter_table_with_environmentContext(dbname, tbl_name, new_tbl, null); + } + + @Override + public void alter_table(String defaultDatabaseName, String tblName, Table table, + boolean cascade) throws TException { + EnvironmentContext environmentContext = new EnvironmentContext(); + if (cascade) { + environmentContext.putToProperties(StatsSetupConst.CASCADE, StatsSetupConst.TRUE); + } + alter_table_with_environmentContext(defaultDatabaseName, tblName, table, environmentContext); + } + + @Override + public void alter_table_with_environmentContext(String dbname, String tbl_name, Table new_tbl, + EnvironmentContext envContext) throws InvalidOperationException, MetaException, TException { + HiveMetaHook hook = getHook(new_tbl); + if (hook != null) { + hook.preAlterTable(new_tbl, envContext); + } + client.alter_table_with_environment_context( + prependCatalogToDbNameByVersion(hiveVersion, null, dbname, conf), tbl_name, new_tbl, envContext); + } + + @Override + public void alter_table(String catName, String dbName, String tblName, Table newTable, + EnvironmentContext envContext) throws TException { + client.alter_table_with_environment_context(prependCatalogToDbNameByVersion(hiveVersion, catName, + dbName, conf), tblName, newTable, envContext); + } + + @Override + public void renamePartition(final String dbname, final String tableName, final List part_vals, + final Partition newPart) throws TException { + renamePartition(getDefaultCatalog(conf), dbname, tableName, part_vals, newPart); + } + + @Override + public void renamePartition(String catName, String dbname, String tableName, List part_vals, + Partition newPart) throws TException { + client.rename_partition(prependCatalogToDbNameByVersion(hiveVersion, catName, dbname, conf), + tableName, part_vals, newPart); + + } + + private void open() throws MetaException { + isConnected = false; + TTransportException tte = null; + MetaException lastException = null; + boolean useSSL = MetastoreConf.getBoolVar(conf, ConfVars.USE_SSL); + boolean useSasl = MetastoreConf.getBoolVar(conf, ConfVars.USE_THRIFT_SASL); + boolean useFramedTransport = MetastoreConf.getBoolVar(conf, ConfVars.USE_THRIFT_FRAMED_TRANSPORT); + boolean useCompactProtocol = MetastoreConf.getBoolVar(conf, ConfVars.USE_THRIFT_COMPACT_PROTOCOL); + int clientSocketTimeout = (int) MetastoreConf.getTimeVar(conf, + ConfVars.CLIENT_SOCKET_TIMEOUT, TimeUnit.MILLISECONDS); + + for (int attempt = 0; !isConnected && attempt < retries; ++attempt) { + for (URI store : metastoreUris) { + LOG.info("Trying to connect to metastore with URI " + store); + + try { + if (useSSL) { + try { + String trustStorePath = MetastoreConf.getVar(conf, ConfVars.SSL_TRUSTSTORE_PATH).trim(); + if (trustStorePath.isEmpty()) { + throw new IllegalArgumentException(ConfVars.SSL_TRUSTSTORE_PATH.toString() + + " Not configured for SSL connection"); + } + String trustStorePassword = + MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.SSL_TRUSTSTORE_PASSWORD); + + // Create an SSL socket and connect + transport = SecurityUtils.getSSLSocket(store.getHost(), store.getPort(), + clientSocketTimeout, + trustStorePath, trustStorePassword); + LOG.info("Opened an SSL connection to metastore, current connections: " + + connCount.incrementAndGet()); + } catch (IOException e) { + throw new IllegalArgumentException(e); + } catch (TTransportException e) { + tte = e; + throw new MetaException(e.toString()); + } + } else { + transport = new TSocket(store.getHost(), store.getPort(), clientSocketTimeout); + } + + if (useSasl) { + // Wrap thrift connection with SASL for secure connection. + try { + HadoopThriftAuthBridge.Client authBridge = + HadoopThriftAuthBridge.getBridge().createClient(); + + // check if we should use delegation tokens to authenticate + // the call below gets hold of the tokens if they are set up by hadoop + // this should happen on the map/reduce tasks if the client added the + // tokens into hadoop's credential store in the front end during job + // submission. + String tokenSig = MetastoreConf.getVar(conf, ConfVars.TOKEN_SIGNATURE); + // tokenSig could be null + tokenStrForm = SecurityUtils.getTokenStrForm(tokenSig); + + if (tokenStrForm != null) { + LOG.info( + "HMSC::open(): Found delegation token. Creating DIGEST-based thrift connection."); + // authenticate using delegation tokens via the "DIGEST" mechanism + transport = authBridge.createClientTransport(null, store.getHost(), + "DIGEST", tokenStrForm, transport, + MetaStoreUtils.getMetaStoreSaslProperties(conf, useSSL)); + } else { + LOG.info( + "HMSC::open(): Could not find delegation token. Creating KERBEROS-based thrift connection."); + String principalConfig = + MetastoreConf.getVar(conf, ConfVars.KERBEROS_PRINCIPAL); + transport = authBridge.createClientTransport( + principalConfig, store.getHost(), "KERBEROS", null, + transport, MetaStoreUtils.getMetaStoreSaslProperties(conf, useSSL)); + } + } catch (IOException ioe) { + LOG.error("Couldn't create client transport", ioe); + throw new MetaException(ioe.toString()); + } + } else { + if (useFramedTransport) { + transport = new TFramedTransport(transport); + } + } + + final TProtocol protocol; + if (useCompactProtocol) { + protocol = new TCompactProtocol(transport); + } else { + protocol = new TBinaryProtocol(transport); + } + client = new ThriftHiveMetastore.Client(protocol); + try { + if (!transport.isOpen()) { + transport.open(); + LOG.info("Opened a connection to metastore, current connections: " + + connCount.incrementAndGet()); + } + isConnected = true; + } catch (TTransportException e) { + tte = e; + if (LOG.isDebugEnabled()) { + LOG.warn("Failed to connect to the MetaStore Server...", e); + } else { + // Don't print full exception trace if DEBUG is not on. + LOG.warn("Failed to connect to the MetaStore Server..."); + } + } + + if (isConnected && !useSasl && MetastoreConf.getBoolVar(conf, ConfVars.EXECUTE_SET_UGI)) { + // Call set_ugi, only in unsecure mode. + try { + UserGroupInformation ugi = SecurityUtils.getUGI(); + client.set_ugi(ugi.getUserName(), Arrays.asList(ugi.getGroupNames())); + } catch (LoginException e) { + LOG.warn("Failed to do login. set_ugi() is not successful, " + + "Continuing without it.", e); + } catch (IOException e) { + LOG.warn("Failed to find ugi of client set_ugi() is not successful, " + + "Continuing without it.", e); + } catch (TException e) { + LOG.warn("set_ugi() not successful, Likely cause: new client talking to old server. " + + "Continuing without it.", e); + } + } + } catch (MetaException e) { + LOG.error("Unable to connect to metastore with URI " + store + + " in attempt " + attempt, e); + lastException = e; + } + if (isConnected) { + break; + } + } + // Wait before launching the next round of connection retries. + if (!isConnected && retryDelaySeconds > 0) { + try { + LOG.info("Waiting " + retryDelaySeconds + " seconds before next connection attempt."); + Thread.sleep(retryDelaySeconds * 1000); + } catch (InterruptedException ignore) { + } + } + } + + if (!isConnected) { + String msg = ""; + if (tte == null) { + if (lastException != null) { + msg = StringUtils.stringifyException(lastException); + } else { + msg = "unknown reason"; + } + } else { + msg = StringUtils.stringifyException(tte); + } + throw new MetaException("Could not connect to meta store using any of the URIs provided." + + " Most recent failure: " + msg); + } + + snapshotActiveConf(); + + LOG.info("Connected to metastore."); + } + + private void snapshotActiveConf() { + currentMetaVars = new HashMap<>(MetastoreConf.metaVars.length); + for (ConfVars oneVar : MetastoreConf.metaVars) { + currentMetaVars.put(oneVar.getVarname(), MetastoreConf.getAsString(conf, oneVar)); + } + } + + @Override + public String getTokenStrForm() throws IOException { + return tokenStrForm; + } + + @Override + public void close() { + isConnected = false; + currentMetaVars = null; + try { + if (null != client) { + client.shutdown(); + } + } catch (TException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("Unable to shutdown metastore client. Will try closing transport directly.", e); + } + } + // Transport would have got closed via client.shutdown(), so we dont need this, but + // just in case, we make this call. + if ((transport != null) && transport.isOpen()) { + transport.close(); + LOG.info("Closed a connection to metastore, current connections: " + connCount.decrementAndGet()); + } + } + + @Override + public void setMetaConf(String key, String value) throws TException { + client.setMetaConf(key, value); + } + + @Override + public String getMetaConf(String key) throws TException { + return client.getMetaConf(key); + } + + @Override + public void createCatalog(Catalog catalog) throws TException { + client.create_catalog(new CreateCatalogRequest(catalog)); + } + + @Override + public void alterCatalog(String catalogName, Catalog newCatalog) throws TException { + client.alter_catalog(new AlterCatalogRequest(catalogName, newCatalog)); + } + + @Override + public Catalog getCatalog(String catName) throws TException { + GetCatalogResponse rsp = client.get_catalog(new GetCatalogRequest(catName)); + return rsp == null ? null : filterHook.filterCatalog(rsp.getCatalog()); + } + + @Override + public List getCatalogs() throws TException { + GetCatalogsResponse rsp = client.get_catalogs(); + return rsp == null ? null : filterHook.filterCatalogs(rsp.getNames()); + } + + @Override + public void dropCatalog(String catName) throws TException { + client.drop_catalog(new DropCatalogRequest(catName)); + } + + /** + * @param new_part + * @return the added partition + * @throws InvalidObjectException + * @throws AlreadyExistsException + * @throws MetaException + * @throws TException + * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#add_partition(org.apache.hadoop.hive.metastore.api.Partition) + */ + @Override + public Partition add_partition(Partition new_part) throws TException { + return add_partition(new_part, null); + } + + public Partition add_partition(Partition new_part, EnvironmentContext envContext) + throws TException { + if (!new_part.isSetCatName()) { + new_part.setCatName(getDefaultCatalog(conf)); + } + Partition p = client.add_partition_with_environment_context(new_part, envContext); + return deepCopy(p); + } + + /** + * @param new_parts + * @throws InvalidObjectException + * @throws AlreadyExistsException + * @throws MetaException + * @throws TException + * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#add_partitions(List) + */ + @Override + public int add_partitions(List new_parts) throws TException { + if (new_parts != null && !new_parts.isEmpty() && !new_parts.get(0).isSetCatName()) { + if (hiveVersion == HiveVersion.V3_0) { + final String defaultCat = getDefaultCatalog(conf); + new_parts.forEach(p -> p.setCatName(defaultCat)); + } + } + return client.add_partitions(new_parts); + } + + @Override + public List add_partitions( + List parts, boolean ifNotExists, boolean needResults) throws TException { + if (parts.isEmpty()) { + return needResults ? new ArrayList<>() : null; + } + Partition part = parts.get(0); + // Have to set it for each partition too + if (!part.isSetCatName()) { + final String defaultCat = getDefaultCatalog(conf); + parts.forEach(p -> p.setCatName(defaultCat)); + } + AddPartitionsRequest req = new AddPartitionsRequest( + part.getDbName(), part.getTableName(), parts, ifNotExists); + req.setCatName(part.isSetCatName() ? part.getCatName() : getDefaultCatalog(conf)); + req.setNeedResult(needResults); + AddPartitionsResult result = client.add_partitions_req(req); + return needResults ? filterHook.filterPartitions(result.getPartitions()) : null; + } + + @Override + public int add_partitions_pspec(PartitionSpecProxy partitionSpec) throws TException { + if (partitionSpec.getCatName() == null) { + partitionSpec.setCatName(getDefaultCatalog(conf)); + } + return client.add_partitions_pspec(partitionSpec.toPartitionSpec()); + } + + @Override + public Partition appendPartition(String db_name, String table_name, + List part_vals) throws TException { + return appendPartition(getDefaultCatalog(conf), db_name, table_name, part_vals); + } + + @Override + public Partition appendPartition(String dbName, String tableName, String partName) + throws TException { + return appendPartition(getDefaultCatalog(conf), dbName, tableName, partName); + } + + @Override + public Partition appendPartition(String catName, String dbName, String tableName, + String name) throws TException { + Partition p = client.append_partition_by_name( + prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, conf), tableName, name); + return deepCopy(p); + } + + @Override + public Partition appendPartition(String catName, String dbName, String tableName, + List partVals) throws TException { + Partition p = client.append_partition( + prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, conf), tableName, partVals); + return deepCopy(p); + } + + @Deprecated + public Partition appendPartition(String dbName, String tableName, List partVals, + EnvironmentContext ec) throws TException { + return client.append_partition_with_environment_context( + prependCatalogToDbNameByVersion(hiveVersion, null, dbName, conf), tableName, partVals, ec).deepCopy(); + } + + /** + * Exchange the partition between two tables + * @param partitionSpecs partitions specs of the parent partition to be exchanged + * @param destDb the db of the destination table + * @param destinationTableName the destination table name + * @return new partition after exchanging + */ + @Override + public Partition exchange_partition(Map partitionSpecs, + String sourceDb, String sourceTable, String destDb, + String destinationTableName) throws TException { + return exchange_partition(partitionSpecs, getDefaultCatalog(conf), sourceDb, sourceTable, + getDefaultCatalog(conf), destDb, destinationTableName); + } + + @Override + public Partition exchange_partition(Map partitionSpecs, String sourceCat, + String sourceDb, String sourceTable, String destCat, + String destDb, String destTableName) throws TException { + return client.exchange_partition(partitionSpecs, prependCatalogToDbName(sourceCat, sourceDb, conf), + sourceTable, prependCatalogToDbName(destCat, destDb, conf), destTableName); + } + + /** + * Exchange the partitions between two tables + * @param partitionSpecs partitions specs of the parent partition to be exchanged + * @param destDb the db of the destination table + * @param destinationTableName the destination table name + * @return new partitions after exchanging + */ + @Override + public List exchange_partitions(Map partitionSpecs, + String sourceDb, String sourceTable, String destDb, + String destinationTableName) throws TException { + return exchange_partitions(partitionSpecs, getDefaultCatalog(conf), sourceDb, sourceTable, + getDefaultCatalog(conf), destDb, destinationTableName); + } + + @Override + public List exchange_partitions(Map partitionSpecs, String sourceCat, + String sourceDb, String sourceTable, String destCat, + String destDb, String destTableName) throws TException { + return client.exchange_partitions(partitionSpecs, prependCatalogToDbName(sourceCat, sourceDb, conf), + sourceTable, prependCatalogToDbName(destCat, destDb, conf), destTableName); + } + + @Override + public void validatePartitionNameCharacters(List partVals) + throws TException, MetaException { + client.partition_name_has_valid_characters(partVals, true); + } + + /** + * Create a new Database + * @param db + * @throws AlreadyExistsException + * @throws InvalidObjectException + * @throws MetaException + * @throws TException + * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#create_database(Database) + */ + @Override + public void createDatabase(Database db) + throws AlreadyExistsException, InvalidObjectException, MetaException, TException { + if (!db.isSetCatalogName()) { + db.setCatalogName(getDefaultCatalog(conf)); + } + client.create_database(db); + } + + /** + * @param tbl + * @throws MetaException + * @throws NoSuchObjectException + * @throws TException + * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#create_table(org.apache.hadoop.hive.metastore.api.Table) + */ + @Override + public void createTable(Table tbl) throws AlreadyExistsException, + InvalidObjectException, MetaException, NoSuchObjectException, TException { + createTable(tbl, null); + } + + public void createTable(Table tbl, EnvironmentContext envContext) throws AlreadyExistsException, + InvalidObjectException, MetaException, NoSuchObjectException, TException { + if (!tbl.isSetCatName()) { + tbl.setCatName(getDefaultCatalog(conf)); + } + HiveMetaHook hook = getHook(tbl); + if (hook != null) { + hook.preCreateTable(tbl); + } + boolean success = false; + try { + // Subclasses can override this step (for example, for temporary tables) + create_table_with_environment_context(tbl, envContext); + if (hook != null) { + hook.commitCreateTable(tbl); + } + success = true; + } finally { + if (!success && (hook != null)) { + try { + hook.rollbackCreateTable(tbl); + } catch (Exception e) { + LOG.error("Create rollback failed with", e); + } + } + } + } + + @Override + public void createTableWithConstraints(Table tbl, + List primaryKeys, List foreignKeys, + List uniqueConstraints, + List notNullConstraints, + List defaultConstraints, + List checkConstraints) + throws AlreadyExistsException, InvalidObjectException, + MetaException, NoSuchObjectException, TException { + + if (!tbl.isSetCatName()) { + String defaultCat = getDefaultCatalog(conf); + tbl.setCatName(defaultCat); + if (primaryKeys != null) { + primaryKeys.forEach(pk -> pk.setCatName(defaultCat)); + } + if (foreignKeys != null) { + foreignKeys.forEach(fk -> fk.setCatName(defaultCat)); + } + if (uniqueConstraints != null) { + uniqueConstraints.forEach(uc -> uc.setCatName(defaultCat)); + } + if (notNullConstraints != null) { + notNullConstraints.forEach(nn -> nn.setCatName(defaultCat)); + } + if (defaultConstraints != null) { + defaultConstraints.forEach(def -> def.setCatName(defaultCat)); + } + if (checkConstraints != null) { + checkConstraints.forEach(cc -> cc.setCatName(defaultCat)); + } + } + HiveMetaHook hook = getHook(tbl); + if (hook != null) { + hook.preCreateTable(tbl); + } + boolean success = false; + try { + // Subclasses can override this step (for example, for temporary tables) + client.create_table_with_constraints(tbl, primaryKeys, foreignKeys, + uniqueConstraints, notNullConstraints, defaultConstraints, checkConstraints); + if (hook != null) { + hook.commitCreateTable(tbl); + } + success = true; + } finally { + if (!success && (hook != null)) { + hook.rollbackCreateTable(tbl); + } + } + } + + @Override + public void dropConstraint(String dbName, String tableName, String constraintName) + throws TException { + dropConstraint(getDefaultCatalog(conf), dbName, tableName, constraintName); + } + + @Override + public void dropConstraint(String catName, String dbName, String tableName, String constraintName) + throws TException { + DropConstraintRequest rqst = new DropConstraintRequest(dbName, tableName, constraintName); + rqst.setCatName(catName); + client.drop_constraint(rqst); + } + + @Override + public void addPrimaryKey(List primaryKeyCols) throws TException { + if (!primaryKeyCols.isEmpty() && !primaryKeyCols.get(0).isSetCatName()) { + String defaultCat = getDefaultCatalog(conf); + primaryKeyCols.forEach(pk -> pk.setCatName(defaultCat)); + } + client.add_primary_key(new AddPrimaryKeyRequest(primaryKeyCols)); + } + + @Override + public void addForeignKey(List foreignKeyCols) throws TException { + if (!foreignKeyCols.isEmpty() && !foreignKeyCols.get(0).isSetCatName()) { + String defaultCat = getDefaultCatalog(conf); + foreignKeyCols.forEach(fk -> fk.setCatName(defaultCat)); + } + client.add_foreign_key(new AddForeignKeyRequest(foreignKeyCols)); + } + + @Override + public void addUniqueConstraint(List uniqueConstraintCols) throws + NoSuchObjectException, MetaException, TException { + if (!uniqueConstraintCols.isEmpty() && !uniqueConstraintCols.get(0).isSetCatName()) { + String defaultCat = getDefaultCatalog(conf); + uniqueConstraintCols.forEach(uc -> uc.setCatName(defaultCat)); + } + client.add_unique_constraint(new AddUniqueConstraintRequest(uniqueConstraintCols)); + } + + @Override + public void addNotNullConstraint(List notNullConstraintCols) throws + NoSuchObjectException, MetaException, TException { + if (!notNullConstraintCols.isEmpty() && !notNullConstraintCols.get(0).isSetCatName()) { + String defaultCat = getDefaultCatalog(conf); + notNullConstraintCols.forEach(nn -> nn.setCatName(defaultCat)); + } + client.add_not_null_constraint(new AddNotNullConstraintRequest(notNullConstraintCols)); + } + + @Override + public void addDefaultConstraint(List defaultConstraints) throws + NoSuchObjectException, MetaException, TException { + if (!defaultConstraints.isEmpty() && !defaultConstraints.get(0).isSetCatName()) { + String defaultCat = getDefaultCatalog(conf); + defaultConstraints.forEach(def -> def.setCatName(defaultCat)); + } + client.add_default_constraint(new AddDefaultConstraintRequest(defaultConstraints)); + } + + @Override + public void addCheckConstraint(List checkConstraints) throws + NoSuchObjectException, MetaException, TException { + if (!checkConstraints.isEmpty() && !checkConstraints.get(0).isSetCatName()) { + String defaultCat = getDefaultCatalog(conf); + checkConstraints.forEach(cc -> cc.setCatName(defaultCat)); + } + client.add_check_constraint(new AddCheckConstraintRequest(checkConstraints)); + } + + /** + * @param type + * @return true or false + * @throws AlreadyExistsException + * @throws InvalidObjectException + * @throws MetaException + * @throws TException + * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#create_type(org.apache.hadoop.hive.metastore.api.Type) + */ + public boolean createType(Type type) throws AlreadyExistsException, + InvalidObjectException, MetaException, TException { + return client.create_type(type); + } + + /** + * @param name + * @throws NoSuchObjectException + * @throws InvalidOperationException + * @throws MetaException + * @throws TException + * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_database(java.lang.String, boolean, boolean) + */ + @Override + public void dropDatabase(String name) + throws NoSuchObjectException, InvalidOperationException, MetaException, TException { + dropDatabase(getDefaultCatalog(conf), name, true, false, false); + } + + @Override + public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb) + throws NoSuchObjectException, InvalidOperationException, MetaException, TException { + dropDatabase(getDefaultCatalog(conf), name, deleteData, ignoreUnknownDb, false); + } + + @Override + public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb, boolean cascade) + throws NoSuchObjectException, InvalidOperationException, MetaException, TException { + dropDatabase(getDefaultCatalog(conf), name, deleteData, ignoreUnknownDb, cascade); + } + + @Override + public void dropDatabase(String catalogName, String dbName, boolean deleteData, + boolean ignoreUnknownDb, boolean cascade) + throws NoSuchObjectException, InvalidOperationException, MetaException, TException { + try { + getDatabase(catalogName, dbName); + } catch (NoSuchObjectException e) { + if (!ignoreUnknownDb) { + throw e; + } + return; + } + + if (cascade) { + // Note that this logic may drop some of the tables of the database + // even if the drop database fail for any reason + // TODO: Fix this + List materializedViews = getTables(dbName, ".*", TableType.MATERIALIZED_VIEW); + for (String table : materializedViews) { + // First we delete the materialized views + dropTable(dbName, table, deleteData, true); + } + List tableList = getAllTables(dbName); + for (String table : tableList) { + // Now we delete the rest of tables + try { + // Subclasses can override this step (for example, for temporary tables) + dropTable(dbName, table, deleteData, true); + } catch (UnsupportedOperationException e) { + // Ignore Index tables, those will be dropped with parent tables + } + } + } + client.drop_database(prependCatalogToDbNameByVersion(hiveVersion, catalogName, dbName, conf), deleteData, + cascade); + } + + @Override + public boolean dropPartition(String dbName, String tableName, String partName, boolean deleteData) + throws TException { + return dropPartition(getDefaultCatalog(conf), dbName, tableName, partName, deleteData); + } + + @Override + public boolean dropPartition(String catName, String db_name, String tbl_name, String name, + boolean deleteData) throws TException { + return client.drop_partition_by_name_with_environment_context(prependCatalogToDbNameByVersion(hiveVersion, + catName, db_name, conf), tbl_name, name, deleteData, null); + } + + private static EnvironmentContext getEnvironmentContextWithIfPurgeSet() { + Map warehouseOptions = new HashMap<>(); + warehouseOptions.put("ifPurge", "TRUE"); + return new EnvironmentContext(warehouseOptions); + } + + // A bunch of these are in HiveMetaStoreClient but not IMetaStoreClient. I have marked these + // as deprecated and not updated them for the catalogs. If we really want to support them we + // should add them to IMetaStoreClient. + + @Deprecated + public boolean dropPartition(String db_name, String tbl_name, List part_vals, + EnvironmentContext env_context) throws TException { + return client.drop_partition_with_environment_context( + prependCatalogToDbNameByVersion(hiveVersion, null, db_name, conf), tbl_name, part_vals, true, + env_context); + } + + @Deprecated + public boolean dropPartition(String dbName, String tableName, String partName, boolean dropData, + EnvironmentContext ec) throws TException { + return client.drop_partition_by_name_with_environment_context( + prependCatalogToDbNameByVersion(hiveVersion, null, dbName, conf), + tableName, partName, dropData, ec); + } + + @Deprecated + public boolean dropPartition(String dbName, String tableName, List partVals) + throws TException { + return client.drop_partition(prependCatalogToDbNameByVersion(hiveVersion, null, dbName, conf), + tableName, partVals, true); + } + + @Override + public boolean dropPartition(String db_name, String tbl_name, + List part_vals, boolean deleteData) throws TException { + return dropPartition(getDefaultCatalog(conf), db_name, tbl_name, part_vals, + PartitionDropOptions.instance().deleteData(deleteData)); + } + + @Override + public boolean dropPartition(String catName, String db_name, String tbl_name, + List part_vals, boolean deleteData) throws TException { + return dropPartition(catName, db_name, tbl_name, part_vals, PartitionDropOptions.instance() + .deleteData(deleteData)); + } + + @Override + public boolean dropPartition(String db_name, String tbl_name, + List part_vals, PartitionDropOptions options) throws TException { + return dropPartition(getDefaultCatalog(conf), db_name, tbl_name, part_vals, options); + } + + @Override + public boolean dropPartition(String catName, String db_name, String tbl_name, + List part_vals, PartitionDropOptions options) + throws TException { + if (options == null) { + options = PartitionDropOptions.instance(); + } + if (part_vals != null) { + for (String partVal : part_vals) { + if (partVal == null) { + throw new MetaException("The partition value must not be null."); + } + } + } + return client.drop_partition_with_environment_context(prependCatalogToDbNameByVersion(hiveVersion, + catName, db_name, conf), tbl_name, part_vals, options.deleteData, + options.purgeData ? getEnvironmentContextWithIfPurgeSet() : null); + } + + @Override + public List dropPartitions(String dbName, String tblName, + List> partExprs, + PartitionDropOptions options) + throws TException { + return dropPartitions(getDefaultCatalog(conf), dbName, tblName, partExprs, options); + } + + @Override + public List dropPartitions(String dbName, String tblName, + List> partExprs, boolean deleteData, + boolean ifExists, boolean needResult) throws NoSuchObjectException, MetaException, TException { + + return dropPartitions(getDefaultCatalog(conf), dbName, tblName, partExprs, + PartitionDropOptions.instance() + .deleteData(deleteData) + .ifExists(ifExists) + .returnResults(needResult)); + + } + + @Override + public List dropPartitions(String dbName, String tblName, + List> partExprs, boolean deleteData, + boolean ifExists) throws NoSuchObjectException, MetaException, TException { + // By default, we need the results from dropPartitions(); + return dropPartitions(getDefaultCatalog(conf), dbName, tblName, partExprs, + PartitionDropOptions.instance() + .deleteData(deleteData) + .ifExists(ifExists)); + } + + @Override + public List dropPartitions(String catName, String dbName, String tblName, + List> partExprs, + PartitionDropOptions options) throws TException { + RequestPartsSpec rps = new RequestPartsSpec(); + List exprs = new ArrayList<>(partExprs.size()); + for (ObjectPair partExpr : partExprs) { + DropPartitionsExpr dpe = new DropPartitionsExpr(); + dpe.setExpr(partExpr.getSecond()); + dpe.setPartArchiveLevel(partExpr.getFirst()); + exprs.add(dpe); + } + rps.setExprs(exprs); + DropPartitionsRequest req = new DropPartitionsRequest(dbName, tblName, rps); + if (hiveVersion == HiveVersion.V3_0) { + req.setCatName(catName); + } + req.setDeleteData(options.deleteData); + req.setNeedResult(options.returnResults); + req.setIfExists(options.ifExists); + if (options.purgeData) { + LOG.info("Dropped partitions will be purged!"); + req.setEnvironmentContext(getEnvironmentContextWithIfPurgeSet()); + } + return client.drop_partitions_req(req).getPartitions(); + } + + @Override + public void dropTable(String dbname, String name, boolean deleteData, + boolean ignoreUnknownTab) throws MetaException, TException, + NoSuchObjectException, UnsupportedOperationException { + dropTable(getDefaultCatalog(conf), dbname, name, deleteData, ignoreUnknownTab, null); + } + + @Override + public void dropTable(String dbname, String name, boolean deleteData, + boolean ignoreUnknownTab, boolean ifPurge) throws TException { + dropTable(getDefaultCatalog(conf), dbname, name, deleteData, ignoreUnknownTab, ifPurge); + } + + @Override + public void dropTable(String dbname, String name) throws TException { + dropTable(getDefaultCatalog(conf), dbname, name, true, true, null); + } + + @Override + public void dropTable(String catName, String dbName, String tableName, boolean deleteData, + boolean ignoreUnknownTable, boolean ifPurge) throws TException { + // build new environmentContext with ifPurge; + EnvironmentContext envContext = null; + if (ifPurge) { + Map warehouseOptions; + warehouseOptions = new HashMap<>(); + warehouseOptions.put("ifPurge", "TRUE"); + envContext = new EnvironmentContext(warehouseOptions); + } + dropTable(catName, dbName, tableName, deleteData, ignoreUnknownTable, envContext); + + } + + /** + * Drop the table and choose whether to: delete the underlying table data; + * throw if the table doesn't exist; save the data in the trash. + * + * @param catName catalog name + * @param dbname database name + * @param name table name + * @param deleteData delete the underlying data or just delete the table in metadata + * @param ignoreUnknownTab don't throw if the requested table doesn't exist + * @param envContext for communicating with thrift + * @throws MetaException could not drop table properly + * @throws NoSuchObjectException the table wasn't found + * @throws TException a thrift communication error occurred + * @throws UnsupportedOperationException dropping an index table is not allowed + * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String, + * java.lang.String, boolean) + */ + public void dropTable(String catName, String dbname, String name, boolean deleteData, + boolean ignoreUnknownTab, EnvironmentContext envContext) throws MetaException, TException, + NoSuchObjectException, UnsupportedOperationException { + Table tbl; + try { + tbl = getTable(catName, dbname, name); + } catch (NoSuchObjectException e) { + if (!ignoreUnknownTab) { + throw e; + } + return; + } + HiveMetaHook hook = getHook(tbl); + if (hook != null) { + hook.preDropTable(tbl); + } + boolean success = false; + try { + drop_table_with_environment_context(catName, dbname, name, deleteData, envContext); + if (hook != null) { + hook.commitDropTable(tbl, + deleteData || (envContext != null && "TRUE".equals(envContext.getProperties().get("ifPurge")))); + } + success = true; + } catch (NoSuchObjectException e) { + if (!ignoreUnknownTab) { + throw e; + } + } finally { + if (!success && (hook != null)) { + hook.rollbackDropTable(tbl); + } + } + } + + @Override + public void truncateTable(String dbName, String tableName, List partNames) throws TException { + truncateTable(getDefaultCatalog(conf), dbName, tableName, partNames); + } + + @Override + public void truncateTable(String catName, String dbName, String tableName, List partNames) + throws TException { + client.truncate_table(prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, conf), tableName, + partNames); + } + + /** + * Recycles the files recursively from the input path to the cmroot directory either by copying or moving it. + * + * @param request Inputs for path of the data files to be recycled to cmroot and + * isPurge flag when set to true files which needs to be recycled are not moved to Trash + * @return Response which is currently void + */ + @Override + public CmRecycleResponse recycleDirToCmPath(CmRecycleRequest request) throws MetaException, TException { + return client.cm_recycle(request); + } + + /** + * @param type + * @return true if the type is dropped + * @throws MetaException + * @throws TException + * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_type(java.lang.String) + */ + public boolean dropType(String type) throws NoSuchObjectException, MetaException, TException { + return client.drop_type(type); + } + + /** + * @param name + * @return map of types + * @throws MetaException + * @throws TException + * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_type_all(java.lang.String) + */ + public Map getTypeAll(String name) throws MetaException, + TException { + Map result = null; + Map fromClient = client.get_type_all(name); + if (fromClient != null) { + result = new LinkedHashMap<>(); + for (String key : fromClient.keySet()) { + result.put(key, deepCopy(fromClient.get(key))); + } + } + return result; + } + + @Override + public List getDatabases(String databasePattern) throws TException { + return getDatabases(getDefaultCatalog(conf), databasePattern); + } + + @Override + public List getDatabases(String catName, String databasePattern) throws TException { + return filterHook.filterDatabases(client.get_databases(prependCatalogToDbName( + catName, databasePattern, conf))); + } + + @Override + public List getAllDatabases() throws TException { + return getAllDatabases(getDefaultCatalog(conf)); + } + + @Override + public List getAllDatabases(String catName) throws TException { + if (hiveVersion == HiveVersion.V1_0 || hiveVersion == HiveVersion.V2_0 || hiveVersion == HiveVersion.V2_3) { + return filterHook.filterDatabases(client.get_all_databases()); + } else { + return filterHook.filterDatabases(client.get_databases(prependCatalogToDbName(catName, null, conf))); + } + } + + @Override + public List listPartitions(String db_name, String tbl_name, short max_parts) + throws TException { + return listPartitions(getDefaultCatalog(conf), db_name, tbl_name, max_parts); + } + + @Override + public List listPartitions(String catName, String db_name, String tbl_name, + int max_parts) throws TException { + List parts = client.get_partitions( + prependCatalogToDbNameByVersion(hiveVersion, catName, db_name, conf), + tbl_name, shrinkMaxtoShort(max_parts)); + return deepCopyPartitions(filterHook.filterPartitions(parts)); + } + + @Override + public PartitionSpecProxy listPartitionSpecs(String dbName, String tableName, int maxParts) throws TException { + return listPartitionSpecs(getDefaultCatalog(conf), dbName, tableName, maxParts); + } + + @Override + public PartitionSpecProxy listPartitionSpecs(String catName, String dbName, String tableName, + int maxParts) throws TException { + return PartitionSpecProxy.Factory.get(filterHook.filterPartitionSpecs( + client.get_partitions_pspec(prependCatalogToDbName(catName, dbName, conf), tableName, maxParts))); + } + + @Override + public List listPartitions(String db_name, String tbl_name, + List part_vals, short max_parts) throws TException { + return listPartitions(getDefaultCatalog(conf), db_name, tbl_name, part_vals, max_parts); + } + + @Override + public List listPartitions(String catName, String db_name, String tbl_name, + List part_vals, int max_parts) throws TException { + List parts = client.get_partitions_ps( + prependCatalogToDbNameByVersion(hiveVersion, catName, db_name, conf), + tbl_name, part_vals, shrinkMaxtoShort(max_parts)); + return deepCopyPartitions(filterHook.filterPartitions(parts)); + } + + @Override + public List listPartitionsWithAuthInfo(String db_name, String tbl_name, + short max_parts, String user_name, + List group_names) throws TException { + return listPartitionsWithAuthInfo(getDefaultCatalog(conf), db_name, tbl_name, max_parts, user_name, + group_names); + } + + @Override + public List listPartitionsWithAuthInfo(String catName, String dbName, String tableName, + int maxParts, String userName, + List groupNames) throws TException { + List parts = client.get_partitions_with_auth(prependCatalogToDbNameByVersion(hiveVersion, catName, + dbName, conf), tableName, shrinkMaxtoShort(maxParts), userName, groupNames); + return deepCopyPartitions(filterHook.filterPartitions(parts)); + } + + @Override + public List listPartitionsWithAuthInfo(String db_name, String tbl_name, + List part_vals, short max_parts, + String user_name, List group_names) + throws TException { + return listPartitionsWithAuthInfo(getDefaultCatalog(conf), db_name, tbl_name, part_vals, max_parts, + user_name, group_names); + } + + @Override + public List listPartitionsWithAuthInfo(String catName, String dbName, String tableName, + List partialPvals, int maxParts, + String userName, List groupNames) + throws TException { + List parts = client.get_partitions_ps_with_auth(prependCatalogToDbNameByVersion(hiveVersion, + catName, dbName, conf), tableName, partialPvals, shrinkMaxtoShort(maxParts), userName, groupNames); + return deepCopyPartitions(filterHook.filterPartitions(parts)); + } + + @Override + public List listPartitionsByFilter(String db_name, String tbl_name, + String filter, short max_parts) throws TException { + return listPartitionsByFilter(getDefaultCatalog(conf), db_name, tbl_name, filter, max_parts); + } + + @Override + public List listPartitionsByFilter(String catName, String db_name, String tbl_name, + String filter, int max_parts) throws TException { + List parts = client.get_partitions_by_filter(prependCatalogToDbName( + catName, db_name, conf), tbl_name, filter, shrinkMaxtoShort(max_parts)); + return deepCopyPartitions(filterHook.filterPartitions(parts)); + } + + @Override + public PartitionSpecProxy listPartitionSpecsByFilter(String db_name, String tbl_name, + String filter, int max_parts) + throws TException { + return listPartitionSpecsByFilter(getDefaultCatalog(conf), db_name, tbl_name, filter, max_parts); + } + + @Override + public PartitionSpecProxy listPartitionSpecsByFilter(String catName, String db_name, + String tbl_name, String filter, + int max_parts) throws TException { + return PartitionSpecProxy.Factory.get(filterHook.filterPartitionSpecs( + client.get_part_specs_by_filter(prependCatalogToDbName(catName, db_name, conf), tbl_name, filter, + max_parts))); + } + + @Override + public boolean listPartitionsByExpr(String db_name, String tbl_name, byte[] expr, + String default_partition_name, short max_parts, + List result) throws TException { + return listPartitionsByExpr(getDefaultCatalog(conf), db_name, tbl_name, expr, + default_partition_name, max_parts, result); + } + + @Override + public boolean listPartitionsByExpr(String catName, String db_name, String tbl_name, byte[] expr, + String default_partition_name, int max_parts, List result) + throws TException { + assert result != null; + PartitionsByExprRequest req = new PartitionsByExprRequest( + db_name, tbl_name, ByteBuffer.wrap(expr)); + if (default_partition_name != null) { + req.setDefaultPartitionName(default_partition_name); + } + if (max_parts >= 0) { + req.setMaxParts(shrinkMaxtoShort(max_parts)); + } + PartitionsByExprResult r; + try { + r = client.get_partitions_by_expr(req); + } catch (TApplicationException te) { + // TODO: backward compat for Hive <= 0.12. Can be removed later. + if (te.getType() != TApplicationException.UNKNOWN_METHOD + && te.getType() != TApplicationException.WRONG_METHOD_NAME) { + throw te; + } + throw new IncompatibleMetastoreException( + "Metastore doesn't support listPartitionsByExpr: " + te.getMessage()); + } + r.setPartitions(filterHook.filterPartitions(r.getPartitions())); + // TODO: in these methods, do we really need to deepcopy? + deepCopyPartitions(r.getPartitions(), result); + return !r.isSetHasUnknownPartitions() || r.isHasUnknownPartitions(); // Assume the worst. + } + + @Override + public Database getDatabase(String name) throws TException { + return getDatabase(getDefaultCatalog(conf), name); + } + + @Override + public Database getDatabase(String catalogName, String databaseName) throws TException { + if (hiveVersion == HiveVersion.V1_0 || hiveVersion == HiveVersion.V2_0 || hiveVersion == HiveVersion.V2_3) { + return deepCopy(client.get_database(databaseName)); + } else { + return deepCopy(client.get_database(prependCatalogToDbName(catalogName, databaseName, conf))); + } + } + + @Override + public Partition getPartition(String db_name, String tbl_name, List part_vals) + throws TException { + return getPartition(getDefaultCatalog(conf), db_name, tbl_name, part_vals); + } + + @Override + public Partition getPartition(String catName, String dbName, String tblName, + List partVals) throws TException { + Partition p = client.get_partition(prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, conf), tblName, + partVals); + return deepCopy(filterHook.filterPartition(p)); + } + + @Override + public List getPartitionsByNames(String db_name, String tbl_name, + List part_names) throws TException { + return getPartitionsByNames(getDefaultCatalog(conf), db_name, tbl_name, part_names); + } + + @Override + public List getPartitionsByNames(String catName, String db_name, String tbl_name, + List part_names) throws TException { + if (hiveVersion == HiveVersion.V1_0 || hiveVersion == HiveVersion.V2_0 || hiveVersion == HiveVersion.V2_3) { + return deepCopyPartitions( + filterHook.filterPartitions(client.get_partitions_by_names(db_name, tbl_name, part_names))); + } else { + return deepCopyPartitions(filterHook.filterPartitions( + client.get_partitions_by_names(prependCatalogToDbName(catName, db_name, conf), tbl_name, + part_names))); + } + } + + @Override + public PartitionValuesResponse listPartitionValues(PartitionValuesRequest request) + throws MetaException, TException, NoSuchObjectException { + if (!request.isSetCatName()) { + request.setCatName(getDefaultCatalog(conf)); + } + return client.get_partition_values(request); + } + + @Override + public Partition getPartitionWithAuthInfo(String db_name, String tbl_name, + List part_vals, String user_name, List group_names) + throws TException { + return getPartitionWithAuthInfo(getDefaultCatalog(conf), db_name, tbl_name, part_vals, + user_name, group_names); + } + + @Override + public Partition getPartitionWithAuthInfo(String catName, String dbName, String tableName, + List pvals, String userName, + List groupNames) throws TException { + Partition p = client.get_partition_with_auth(prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, + conf), + tableName, pvals, userName, groupNames); + return deepCopy(filterHook.filterPartition(p)); + } + + @Override + public Table getTable(String dbname, String name) throws TException { + return getTable(getDefaultCatalog(conf), dbname, name); + } + + @Override + public Table getTable(String catName, String dbName, String tableName) throws TException { + Table t; + if (hiveVersion == HiveVersion.V1_0 || hiveVersion == HiveVersion.V2_0) { + t = client.get_table(dbName, tableName); + } else if (hiveVersion == HiveVersion.V2_3) { + GetTableRequest req = new GetTableRequest(dbName, tableName); + req.setCapabilities(version); + t = client.get_table_req(req).getTable(); + } else { + GetTableRequest req = new GetTableRequest(dbName, tableName); + req.setCatName(catName); + req.setCapabilities(version); + t = client.get_table_req(req).getTable(); + } + return deepCopy(filterHook.filterTable(t)); + } + + @Override + public List getTableObjectsByName(String dbName, List tableNames) + throws TException { + return getTableObjectsByName(getDefaultCatalog(conf), dbName, tableNames); + } + + @Override + public List
getTableObjectsByName(String catName, String dbName, + List tableNames) throws TException { + List
tabs = new ArrayList<>(); + if (hiveVersion == HiveVersion.V1_0 || hiveVersion == HiveVersion.V2_0) { + for (String tableName : tableNames) { + tabs.add(client.get_table(dbName, tableName)); + } + } else { + GetTablesRequest req = new GetTablesRequest(dbName); + req.setCatName(catName); + req.setTblNames(tableNames); + req.setCapabilities(version); + tabs = client.get_table_objects_by_name_req(req).getTables(); + } + return deepCopyTables(filterHook.filterTables(tabs)); + } + + @Override + public Materialization getMaterializationInvalidationInfo(CreationMetadata cm, String validTxnList) + throws MetaException, InvalidOperationException, UnknownDBException, TException { + return client.get_materialization_invalidation_info(cm, validTxnList); + } + + @Override + public void updateCreationMetadata(String dbName, String tableName, CreationMetadata cm) + throws MetaException, InvalidOperationException, UnknownDBException, TException { + client.update_creation_metadata(getDefaultCatalog(conf), dbName, tableName, cm); + } + + @Override + public void updateCreationMetadata(String catName, String dbName, String tableName, + CreationMetadata cm) throws MetaException, TException { + client.update_creation_metadata(catName, dbName, tableName, cm); + + } + + /** + * {@inheritDoc} + */ + @Override + public List listTableNamesByFilter(String dbName, String filter, short maxTables) + throws TException { + return listTableNamesByFilter(getDefaultCatalog(conf), dbName, filter, maxTables); + } + + @Override + public List listTableNamesByFilter(String catName, String dbName, String filter, + int maxTables) throws TException { + return filterHook.filterTableNames(catName, dbName, + client.get_table_names_by_filter(prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, conf), + filter, + shrinkMaxtoShort(maxTables))); + } + + /** + * @param name + * @return the type + * @throws MetaException + * @throws TException + * @throws NoSuchObjectException + * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_type(java.lang.String) + */ + public Type getType(String name) throws NoSuchObjectException, MetaException, TException { + return deepCopy(client.get_type(name)); + } + + @Override + public List getTables(String dbname, String tablePattern) throws MetaException { + try { + return getTables(getDefaultCatalog(conf), dbname, tablePattern); + } catch (Exception e) { + MetaStoreUtils.logAndThrowMetaException(e); + } + return null; + } + + @Override + public List getTables(String catName, String dbName, String tablePattern) + throws TException { + return filterHook.filterTableNames(catName, dbName, + client.get_tables(prependCatalogToDbName(catName, dbName, conf), tablePattern)); + } + + @Override + public List getTables(String dbname, String tablePattern, TableType tableType) throws MetaException { + try { + return getTables(getDefaultCatalog(conf), dbname, tablePattern, tableType); + } catch (Exception e) { + MetaStoreUtils.logAndThrowMetaException(e); + } + return null; + } + + @Override + public List getTables(String catName, String dbName, String tablePattern, + TableType tableType) throws TException { + return filterHook.filterTableNames(catName, dbName, + client.get_tables_by_type(prependCatalogToDbName(catName, dbName, conf), tablePattern, + tableType.toString())); + } + + @Override + public List getMaterializedViewsForRewriting(String dbName) throws TException { + return getMaterializedViewsForRewriting(getDefaultCatalog(conf), dbName); + } + + @Override + public List getMaterializedViewsForRewriting(String catName, String dbname) + throws MetaException { + try { + return filterHook.filterTableNames(catName, dbname, + client.get_materialized_views_for_rewriting(prependCatalogToDbName(catName, dbname, conf))); + } catch (Exception e) { + MetaStoreUtils.logAndThrowMetaException(e); + } + return null; + } + + @Override + public List getTableMeta(String dbPatterns, String tablePatterns, List tableTypes) + throws MetaException { + try { + return getTableMeta(getDefaultCatalog(conf), dbPatterns, tablePatterns, tableTypes); + } catch (Exception e) { + MetaStoreUtils.logAndThrowMetaException(e); + } + return null; + } + + @Override + public List getTableMeta(String catName, String dbPatterns, String tablePatterns, + List tableTypes) throws TException { + return filterHook.filterTableMetas(client.get_table_meta(prependCatalogToDbName( + catName, dbPatterns, conf), tablePatterns, tableTypes)); + } + + @Override + public List getAllTables(String dbname) throws MetaException { + try { + return getAllTables(getDefaultCatalog(conf), dbname); + } catch (Exception e) { + MetaStoreUtils.logAndThrowMetaException(e); + } + return null; + } + + @Override + public List getAllTables(String catName, String dbName) throws TException { + if (hiveVersion == HiveVersion.V1_0 || hiveVersion == HiveVersion.V2_0 || hiveVersion == HiveVersion.V2_3) { + return filterHook.filterTableNames(null, dbName, client.get_all_tables(dbName)); + } else { + return filterHook.filterTableNames(catName, dbName, client.get_all_tables( + prependCatalogToDbName(catName, dbName, conf))); + } + } + + @Override + public boolean tableExists(String databaseName, String tableName) throws TException { + return tableExists(getDefaultCatalog(conf), databaseName, tableName); + } + + @Override + public boolean tableExists(String catName, String dbName, String tableName) throws TException { + try { + Table t; + if (hiveVersion == HiveVersion.V1_0 || hiveVersion == HiveVersion.V2_0) { + t = client.get_table(dbName, tableName); + } else if (hiveVersion == HiveVersion.V2_3) { + GetTableRequest req = new GetTableRequest(dbName, tableName); + req.setCapabilities(version); + t = client.get_table_req(req).getTable(); + } else { + GetTableRequest req = new GetTableRequest(dbName, tableName); + req.setCatName(catName); + req.setCapabilities(version); + t = client.get_table_req(req).getTable(); + } + return filterHook.filterTable(t) != null; + } catch (NoSuchObjectException e) { + return false; + } + } + + @Override + public List listPartitionNames(String dbName, String tblName, + short max) throws NoSuchObjectException, MetaException, TException { + return listPartitionNames(getDefaultCatalog(conf), dbName, tblName, max); + } + + @Override + public List listPartitionNames(String catName, String dbName, String tableName, + int maxParts) throws TException { + if (hiveVersion == HiveVersion.V1_0 || hiveVersion == HiveVersion.V2_0 || hiveVersion == HiveVersion.V2_3) { + return filterHook.filterPartitionNames(null, dbName, tableName, + client.get_partition_names(dbName, tableName, shrinkMaxtoShort(maxParts))); + } else { + return filterHook.filterPartitionNames(catName, dbName, tableName, + client.get_partition_names(prependCatalogToDbName(catName, dbName, conf), tableName, + shrinkMaxtoShort(maxParts))); + } + } + + @Override + public List listPartitionNames(String db_name, String tbl_name, + List part_vals, short max_parts) throws TException { + return listPartitionNames(getDefaultCatalog(conf), db_name, tbl_name, part_vals, max_parts); + } + + @Override + public List listPartitionNames(String catName, String db_name, String tbl_name, + List part_vals, int max_parts) throws TException { + if (hiveVersion == HiveVersion.V1_0 || hiveVersion == HiveVersion.V2_0 || hiveVersion == HiveVersion.V2_3) { + return filterHook.filterPartitionNames(null, db_name, tbl_name, + client.get_partition_names_ps(db_name, tbl_name, part_vals, shrinkMaxtoShort(max_parts))); + } else { + return filterHook.filterPartitionNames(catName, db_name, tbl_name, + client.get_partition_names_ps(prependCatalogToDbName(catName, db_name, conf), tbl_name, + part_vals, shrinkMaxtoShort(max_parts))); + } + } + + @Override + public int getNumPartitionsByFilter(String db_name, String tbl_name, + String filter) throws TException { + return getNumPartitionsByFilter(getDefaultCatalog(conf), db_name, tbl_name, filter); + } + + @Override + public int getNumPartitionsByFilter(String catName, String dbName, String tableName, + String filter) throws TException { + return client.get_num_partitions_by_filter(prependCatalogToDbName(catName, dbName, conf), tableName, + filter); + } + + @Override + public void alter_partition(String dbName, String tblName, Partition newPart) + throws InvalidOperationException, MetaException, TException { + alter_partition(getDefaultCatalog(conf), dbName, tblName, newPart, null); + } + + @Override + public void alter_partition(String dbName, String tblName, Partition newPart, EnvironmentContext environmentContext) + throws InvalidOperationException, MetaException, TException { + alter_partition(getDefaultCatalog(conf), dbName, tblName, newPart, environmentContext); + } + + @Override + public void alter_partition(String catName, String dbName, String tblName, Partition newPart, + EnvironmentContext environmentContext) throws TException { + client.alter_partition_with_environment_context(prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, + conf), + tblName, newPart, environmentContext); + } + + @Override + public void alter_partitions(String dbName, String tblName, List newParts) + throws TException { + alter_partitions(getDefaultCatalog(conf), dbName, tblName, newParts, null); + } + + @Override + public void alter_partitions(String dbName, String tblName, List newParts, + EnvironmentContext environmentContext) throws TException { + alter_partitions(getDefaultCatalog(conf), dbName, tblName, newParts, environmentContext); + } + + @Override + public void alter_partitions(String catName, String dbName, String tblName, + List newParts, + EnvironmentContext environmentContext) throws TException { + client.alter_partitions_with_environment_context( + prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, conf), + tblName, newParts, environmentContext); + } + + @Override + public void alterDatabase(String dbName, Database db) throws TException { + alterDatabase(getDefaultCatalog(conf), dbName, db); + } + + @Override + public void alterDatabase(String catName, String dbName, Database newDb) throws TException { + client.alter_database(prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, conf), newDb); + } + + @Override + public List getFields(String db, String tableName) throws TException { + return getFields(getDefaultCatalog(conf), db, tableName); + } + + @Override + public List getFields(String catName, String db, String tableName) + throws TException { + List fields = client.get_fields(prependCatalogToDbName(catName, db, conf), tableName); + return deepCopyFieldSchemas(fields); + } + + @Override + public List getPrimaryKeys(PrimaryKeysRequest req) throws TException { + if (!req.isSetCatName()) { + req.setCatName(getDefaultCatalog(conf)); + } + return client.get_primary_keys(req).getPrimaryKeys(); + } + + @Override + public List getForeignKeys(ForeignKeysRequest req) throws MetaException, + NoSuchObjectException, TException { + if (!req.isSetCatName()) { + req.setCatName(getDefaultCatalog(conf)); + } + return client.get_foreign_keys(req).getForeignKeys(); + } + + @Override + public List getUniqueConstraints(UniqueConstraintsRequest req) + throws MetaException, NoSuchObjectException, TException { + if (!req.isSetCatName()) { + req.setCatName(getDefaultCatalog(conf)); + } + return client.get_unique_constraints(req).getUniqueConstraints(); + } + + @Override + public List getNotNullConstraints(NotNullConstraintsRequest req) + throws MetaException, NoSuchObjectException, TException { + if (!req.isSetCatName()) { + req.setCatName(getDefaultCatalog(conf)); + } + return client.get_not_null_constraints(req).getNotNullConstraints(); + } + + @Override + public List getDefaultConstraints(DefaultConstraintsRequest req) + throws MetaException, NoSuchObjectException, TException { + if (!req.isSetCatName()) { + req.setCatName(getDefaultCatalog(conf)); + } + return client.get_default_constraints(req).getDefaultConstraints(); + } + + @Override + public List getCheckConstraints(CheckConstraintsRequest req) + throws MetaException, NoSuchObjectException, TException { + if (!req.isSetCatName()) { + req.setCatName(getDefaultCatalog(conf)); + } + return client.get_check_constraints(req).getCheckConstraints(); + } + + /** {@inheritDoc} */ + @Override + public boolean updateTableColumnStatistics(ColumnStatistics statsObj) throws TException { + if (!statsObj.getStatsDesc().isSetCatName()) { + statsObj.getStatsDesc().setCatName(getDefaultCatalog(conf)); + } + return client.update_table_column_statistics(statsObj); + } + + @Override + public boolean updatePartitionColumnStatistics(ColumnStatistics statsObj) throws TException { + if (!statsObj.getStatsDesc().isSetCatName()) { + statsObj.getStatsDesc().setCatName(getDefaultCatalog(conf)); + } + return client.update_partition_column_statistics(statsObj); + } + + @Override + public boolean setPartitionColumnStatistics(SetPartitionsStatsRequest request) throws TException { + String defaultCat = getDefaultCatalog(conf); + for (ColumnStatistics stats : request.getColStats()) { + if (!stats.getStatsDesc().isSetCatName()) { + stats.getStatsDesc().setCatName(defaultCat); + } + } + return client.set_aggr_stats_for(request); + } + + @Override + public void flushCache() { + try { + client.flushCache(); + } catch (TException e) { + // Not much we can do about it honestly + LOG.warn("Got error flushing the cache", e); + } + } + + @Override + public List getTableColumnStatistics(String dbName, String tableName, + List colNames) throws TException { + return getTableColumnStatistics(getDefaultCatalog(conf), dbName, tableName, colNames); + } + + @Override + public List getTableColumnStatistics(String catName, String dbName, + String tableName, + List colNames) throws TException { + TableStatsRequest rqst = new TableStatsRequest(dbName, tableName, colNames); + if (hiveVersion != HiveVersion.V1_0 && hiveVersion != HiveVersion.V2_0 && hiveVersion != HiveVersion.V2_3) { + rqst.setCatName(catName); + } + return client.get_table_statistics_req(rqst).getTableStats(); + } + + @Override + public Map> getPartitionColumnStatistics( + String dbName, String tableName, List partNames, List colNames) + throws TException { + return getPartitionColumnStatistics(getDefaultCatalog(conf), dbName, tableName, partNames, colNames); + } + + @Override + public Map> getPartitionColumnStatistics( + String catName, String dbName, String tableName, List partNames, + List colNames) throws TException { + PartitionsStatsRequest rqst = new PartitionsStatsRequest(dbName, tableName, colNames, + partNames); + if (hiveVersion != HiveVersion.V1_0 && hiveVersion != HiveVersion.V2_0 && hiveVersion != HiveVersion.V2_3) { + rqst.setCatName(catName); + } + return client.get_partitions_statistics_req(rqst).getPartStats(); + } + + @Override + public boolean deletePartitionColumnStatistics(String dbName, String tableName, String partName, + String colName) throws TException { + return deletePartitionColumnStatistics(getDefaultCatalog(conf), dbName, tableName, partName, + colName); + } + + @Override + public boolean deletePartitionColumnStatistics(String catName, String dbName, String tableName, + String partName, String colName) + throws TException { + return client.delete_partition_column_statistics( + prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, conf), + tableName, partName, colName); + } + + @Override + public boolean deleteTableColumnStatistics(String dbName, String tableName, String colName) + throws TException { + return deleteTableColumnStatistics(getDefaultCatalog(conf), dbName, tableName, colName); + } + + @Override + public boolean deleteTableColumnStatistics(String catName, String dbName, String tableName, + String colName) throws TException { + return client.delete_table_column_statistics( + prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, conf), + tableName, colName); + } + + @Override + public List getSchema(String db, String tableName) throws TException { + return getSchema(getDefaultCatalog(conf), db, tableName); + } + + @Override + public List getSchema(String catName, String db, String tableName) throws TException { + List fields; + if (hiveVersion == HiveVersion.V1_0) { + fields = client.get_schema(db, tableName); + } else { + EnvironmentContext envCxt = null; + String addedJars = MetastoreConf.getVar(conf, ConfVars.ADDED_JARS); + if (org.apache.commons.lang.StringUtils.isNotBlank(addedJars)) { + Map props = new HashMap<>(); + props.put("hive.added.jars.path", addedJars); + envCxt = new EnvironmentContext(props); + } + fields = client.get_schema_with_environment_context(prependCatalogToDbNameByVersion(hiveVersion, + catName, db, conf), tableName, envCxt); + } + return deepCopyFieldSchemas(fields); + } + + @Override + public String getConfigValue(String name, String defaultValue) + throws TException, ConfigValSecurityException { + return client.get_config_value(name, defaultValue); + } + + @Override + public Partition getPartition(String db, String tableName, String partName) throws TException { + return getPartition(getDefaultCatalog(conf), db, tableName, partName); + } + + @Override + public Partition getPartition(String catName, String dbName, String tblName, String name) + throws TException { + Partition p = client.get_partition_by_name(prependCatalogToDbNameByVersion(hiveVersion, catName, dbName, conf), + tblName, name); + return deepCopy(filterHook.filterPartition(p)); + } + + public Partition appendPartitionByName(String dbName, String tableName, String partName) + throws InvalidObjectException, AlreadyExistsException, MetaException, TException { + return appendPartitionByName(dbName, tableName, partName, null); + } + + public Partition appendPartitionByName(String dbName, String tableName, String partName, + EnvironmentContext envContext) throws InvalidObjectException, AlreadyExistsException, + MetaException, TException { + Partition p = client.append_partition_by_name_with_environment_context(dbName, tableName, + partName, envContext); + return deepCopy(p); + } + + public boolean dropPartitionByName(String dbName, String tableName, String partName, + boolean deleteData) throws NoSuchObjectException, MetaException, TException { + return dropPartitionByName(dbName, tableName, partName, deleteData, null); + } + + public boolean dropPartitionByName(String dbName, String tableName, String partName, + boolean deleteData, EnvironmentContext envContext) throws NoSuchObjectException, + MetaException, TException { + return client.drop_partition_by_name_with_environment_context(dbName, tableName, partName, + deleteData, envContext); + } + + private HiveMetaHook getHook(Table tbl) throws MetaException { + if (hookLoader == null) { + return null; + } + return hookLoader.getHook(tbl); + } + + @Override + public List partitionNameToVals(String name) throws MetaException, TException { + return client.partition_name_to_vals(name); + } + + @Override + public Map partitionNameToSpec(String name) throws MetaException, TException { + return client.partition_name_to_spec(name); + } + + /** + * @param partition + * @return + */ + protected Partition deepCopy(Partition partition) { + Partition copy = null; + if (partition != null) { + copy = new Partition(partition); + } + return copy; + } + + private Database deepCopy(Database database) { + Database copy = null; + if (database != null) { + copy = new Database(database); + } + return copy; + } + + protected Table deepCopy(Table table) { + Table copy = null; + if (table != null) { + copy = new Table(table); + } + return copy; + } + + private Type deepCopy(Type type) { + Type copy = null; + if (type != null) { + copy = new Type(type); + } + return copy; + } + + private FieldSchema deepCopy(FieldSchema schema) { + FieldSchema copy = null; + if (schema != null) { + copy = new FieldSchema(schema); + } + return copy; + } + + private Function deepCopy(Function func) { + Function copy = null; + if (func != null) { + copy = new Function(func); + } + return copy; + } + + protected PrincipalPrivilegeSet deepCopy(PrincipalPrivilegeSet pps) { + PrincipalPrivilegeSet copy = null; + if (pps != null) { + copy = new PrincipalPrivilegeSet(pps); + } + return copy; + } + + private List deepCopyPartitions(List partitions) { + return deepCopyPartitions(partitions, null); + } + + private List deepCopyPartitions( + Collection src, List dest) { + if (src == null) { + return dest; + } + if (dest == null) { + dest = new ArrayList(src.size()); + } + for (Partition part : src) { + dest.add(deepCopy(part)); + } + return dest; + } + + private List
deepCopyTables(List
tables) { + List
copy = null; + if (tables != null) { + copy = new ArrayList
(); + for (Table tab : tables) { + copy.add(deepCopy(tab)); + } + } + return copy; + } + + protected List deepCopyFieldSchemas(List schemas) { + List copy = null; + if (schemas != null) { + copy = new ArrayList(); + for (FieldSchema schema : schemas) { + copy.add(deepCopy(schema)); + } + } + return copy; + } + + @Override + public boolean grant_role(String roleName, String userName, + PrincipalType principalType, String grantor, PrincipalType grantorType, + boolean grantOption) throws MetaException, TException { + GrantRevokeRoleRequest req = new GrantRevokeRoleRequest(); + req.setRequestType(GrantRevokeType.GRANT); + req.setRoleName(roleName); + req.setPrincipalName(userName); + req.setPrincipalType(principalType); + req.setGrantor(grantor); + req.setGrantorType(grantorType); + req.setGrantOption(grantOption); + GrantRevokeRoleResponse res = client.grant_revoke_role(req); + if (!res.isSetSuccess()) { + throw new MetaException("GrantRevokeResponse missing success field"); + } + return res.isSuccess(); + } + + @Override + public boolean create_role(Role role) + throws MetaException, TException { + return client.create_role(role); + } + + @Override + public boolean drop_role(String roleName) throws MetaException, TException { + return client.drop_role(roleName); + } + + @Override + public List list_roles(String principalName, + PrincipalType principalType) throws MetaException, TException { + return client.list_roles(principalName, principalType); + } + + @Override + public List listRoleNames() throws MetaException, TException { + return client.get_role_names(); + } + + @Override + public GetPrincipalsInRoleResponse get_principals_in_role(GetPrincipalsInRoleRequest req) + throws MetaException, TException { + return client.get_principals_in_role(req); + } + + @Override + public GetRoleGrantsForPrincipalResponse get_role_grants_for_principal( + GetRoleGrantsForPrincipalRequest getRolePrincReq) throws MetaException, TException { + return client.get_role_grants_for_principal(getRolePrincReq); + } + + @Override + public boolean grant_privileges(PrivilegeBag privileges) + throws MetaException, TException { + String defaultCat = getDefaultCatalog(conf); + for (HiveObjectPrivilege priv : privileges.getPrivileges()) { + if (!priv.getHiveObject().isSetCatName()) { + priv.getHiveObject().setCatName(defaultCat); + } + } + GrantRevokePrivilegeRequest req = new GrantRevokePrivilegeRequest(); + req.setRequestType(GrantRevokeType.GRANT); + req.setPrivileges(privileges); + GrantRevokePrivilegeResponse res = client.grant_revoke_privileges(req); + if (!res.isSetSuccess()) { + throw new MetaException("GrantRevokePrivilegeResponse missing success field"); + } + return res.isSuccess(); + } + + @Override + public boolean revoke_role(String roleName, String userName, + PrincipalType principalType, boolean grantOption) throws MetaException, TException { + GrantRevokeRoleRequest req = new GrantRevokeRoleRequest(); + req.setRequestType(GrantRevokeType.REVOKE); + req.setRoleName(roleName); + req.setPrincipalName(userName); + req.setPrincipalType(principalType); + req.setGrantOption(grantOption); + GrantRevokeRoleResponse res = client.grant_revoke_role(req); + if (!res.isSetSuccess()) { + throw new MetaException("GrantRevokeResponse missing success field"); + } + return res.isSuccess(); + } + + @Override + public boolean revoke_privileges(PrivilegeBag privileges, boolean grantOption) throws MetaException, + TException { + String defaultCat = getDefaultCatalog(conf); + for (HiveObjectPrivilege priv : privileges.getPrivileges()) { + if (!priv.getHiveObject().isSetCatName()) { + priv.getHiveObject().setCatName(defaultCat); + } + } + GrantRevokePrivilegeRequest req = new GrantRevokePrivilegeRequest(); + req.setRequestType(GrantRevokeType.REVOKE); + req.setPrivileges(privileges); + req.setRevokeGrantOption(grantOption); + GrantRevokePrivilegeResponse res = client.grant_revoke_privileges(req); + if (!res.isSetSuccess()) { + throw new MetaException("GrantRevokePrivilegeResponse missing success field"); + } + return res.isSuccess(); + } + + @Override + public boolean refresh_privileges(HiveObjectRef objToRefresh, String authorizer, + PrivilegeBag grantPrivileges) throws MetaException, + TException { + String defaultCat = getDefaultCatalog(conf); + objToRefresh.setCatName(defaultCat); + + if (grantPrivileges.getPrivileges() != null) { + for (HiveObjectPrivilege priv : grantPrivileges.getPrivileges()) { + if (!priv.getHiveObject().isSetCatName()) { + priv.getHiveObject().setCatName(defaultCat); + } + } + } + GrantRevokePrivilegeRequest grantReq = new GrantRevokePrivilegeRequest(); + grantReq.setRequestType(GrantRevokeType.GRANT); + grantReq.setPrivileges(grantPrivileges); + + GrantRevokePrivilegeResponse res = client.refresh_privileges(objToRefresh, authorizer, grantReq); + if (!res.isSetSuccess()) { + throw new MetaException("GrantRevokePrivilegeResponse missing success field"); + } + return res.isSuccess(); + } + + @Override + public PrincipalPrivilegeSet get_privilege_set(HiveObjectRef hiveObject, + String userName, List groupNames) throws MetaException, + TException { + if (!hiveObject.isSetCatName()) { + hiveObject.setCatName(getDefaultCatalog(conf)); + } + return client.get_privilege_set(hiveObject, userName, groupNames); + } + + @Override + public List list_privileges(String principalName, + PrincipalType principalType, HiveObjectRef hiveObject) + throws MetaException, TException { + if (!hiveObject.isSetCatName()) { + hiveObject.setCatName(getDefaultCatalog(conf)); + } + return client.list_privileges(principalName, principalType, hiveObject); + } + + public String getDelegationToken(String renewerKerberosPrincipalName) throws + MetaException, TException, IOException { + // a convenience method that makes the intended owner for the delegation + // token request the current user + String owner = SecurityUtils.getUser(); + return getDelegationToken(owner, renewerKerberosPrincipalName); + } + + @Override + public String getDelegationToken(String owner, String renewerKerberosPrincipalName) throws + MetaException, TException { + // This is expected to be a no-op, so we will return null when we use local metastore. + if (localMetaStore) { + return null; + } + return client.get_delegation_token(owner, renewerKerberosPrincipalName); + } + + @Override + public long renewDelegationToken(String tokenStrForm) throws MetaException, TException { + if (localMetaStore) { + return 0; + } + return client.renew_delegation_token(tokenStrForm); + + } + + @Override + public void cancelDelegationToken(String tokenStrForm) throws MetaException, TException { + if (localMetaStore) { + return; + } + client.cancel_delegation_token(tokenStrForm); + } + + @Override + public boolean addToken(String tokenIdentifier, String delegationToken) throws TException { + return client.add_token(tokenIdentifier, delegationToken); + } + + @Override + public boolean removeToken(String tokenIdentifier) throws TException { + return client.remove_token(tokenIdentifier); + } + + @Override + public String getToken(String tokenIdentifier) throws TException { + return client.get_token(tokenIdentifier); + } + + @Override + public List getAllTokenIdentifiers() throws TException { + return client.get_all_token_identifiers(); + } + + @Override + public int addMasterKey(String key) throws MetaException, TException { + return client.add_master_key(key); + } + + @Override + public void updateMasterKey(Integer seqNo, String key) + throws NoSuchObjectException, MetaException, TException { + client.update_master_key(seqNo, key); + } + + @Override + public boolean removeMasterKey(Integer keySeq) throws TException { + return client.remove_master_key(keySeq); + } + + @Override + public String[] getMasterKeys() throws TException { + List keyList = client.get_master_keys(); + return keyList.toArray(new String[keyList.size()]); + } + + @Override + public ValidTxnList getValidTxns() throws TException { + return TxnUtils.createValidReadTxnList(client.get_open_txns(), 0); + } + + @Override + public ValidTxnList getValidTxns(long currentTxn) throws TException { + return TxnUtils.createValidReadTxnList(client.get_open_txns(), currentTxn); + } + + @Override + public ValidWriteIdList getValidWriteIds(String fullTableName) throws TException { + GetValidWriteIdsRequest rqst = new GetValidWriteIdsRequest(Collections.singletonList(fullTableName), null); + GetValidWriteIdsResponse validWriteIds = client.get_valid_write_ids(rqst); + return TxnUtils.createValidReaderWriteIdList(validWriteIds.getTblValidWriteIds().get(0)); + } + + @Override + public List getValidWriteIds( + List tablesList, String validTxnList) throws TException { + GetValidWriteIdsRequest rqst = new GetValidWriteIdsRequest(tablesList, validTxnList); + return client.get_valid_write_ids(rqst).getTblValidWriteIds(); + } + + @Override + public long openTxn(String user) throws TException { + OpenTxnsResponse txns = openTxnsIntr(user, 1, null, null); + return txns.getTxn_ids().get(0); + } + + @Override + public List replOpenTxn(String replPolicy, List srcTxnIds, String user) throws TException { + // As this is called from replication task, the user is the user who has fired the repl command. + // This is required for standalone metastore authentication. + OpenTxnsResponse txns = openTxnsIntr(user, srcTxnIds.size(), replPolicy, srcTxnIds); + return txns.getTxn_ids(); + } + + @Override + public OpenTxnsResponse openTxns(String user, int numTxns) throws TException { + return openTxnsIntr(user, numTxns, null, null); + } + + private OpenTxnsResponse openTxnsIntr(String user, int numTxns, String replPolicy, + List srcTxnIds) throws TException { + String hostname; + try { + hostname = InetAddress.getLocalHost().getHostName(); + } catch (UnknownHostException e) { + LOG.error("Unable to resolve my host name " + e.getMessage()); + throw new RuntimeException(e); + } + OpenTxnRequest rqst = new OpenTxnRequest(numTxns, user, hostname); + if (replPolicy != null) { + assert srcTxnIds != null; + assert numTxns == srcTxnIds.size(); + // need to set this only for replication tasks + rqst.setReplPolicy(replPolicy); + rqst.setReplSrcTxnIds(srcTxnIds); + } else { + assert srcTxnIds == null; + } + return client.open_txns(rqst); + } + + @Override + public void rollbackTxn(long txnid) throws NoSuchTxnException, TException { + client.abort_txn(new AbortTxnRequest(txnid)); + } + + @Override + public void replRollbackTxn(long srcTxnId, String replPolicy) throws NoSuchTxnException, TException { + AbortTxnRequest rqst = new AbortTxnRequest(srcTxnId); + rqst.setReplPolicy(replPolicy); + client.abort_txn(rqst); + } + + @Override + public void commitTxn(long txnid) + throws NoSuchTxnException, TxnAbortedException, TException { + client.commit_txn(new CommitTxnRequest(txnid)); + } + + @Override + public void replCommitTxn(long srcTxnId, String replPolicy) + throws NoSuchTxnException, TxnAbortedException, TException { + CommitTxnRequest rqst = new CommitTxnRequest(srcTxnId); + rqst.setReplPolicy(replPolicy); + client.commit_txn(rqst); + } + + @Override + public GetOpenTxnsInfoResponse showTxns() throws TException { + return client.get_open_txns_info(); + } + + @Override + public void abortTxns(List txnids) throws NoSuchTxnException, TException { + client.abort_txns(new AbortTxnsRequest(txnids)); + } + + @Override + public void replTableWriteIdState(String validWriteIdList, String dbName, String tableName, List partNames) + throws TException { + String user; + try { + user = UserGroupInformation.getCurrentUser().getUserName(); + } catch (IOException e) { + LOG.error("Unable to resolve current user name " + e.getMessage()); + throw new RuntimeException(e); + } + + String hostName; + try { + hostName = InetAddress.getLocalHost().getHostName(); + } catch (UnknownHostException e) { + LOG.error("Unable to resolve my host name " + e.getMessage()); + throw new RuntimeException(e); + } + + ReplTblWriteIdStateRequest rqst + = new ReplTblWriteIdStateRequest(validWriteIdList, user, hostName, dbName, tableName); + if (partNames != null) { + rqst.setPartNames(partNames); + } + client.repl_tbl_writeid_state(rqst); + } + + @Override + public long allocateTableWriteId(long txnId, String dbName, String tableName) throws TException { + return allocateTableWriteIdsBatch(Collections.singletonList(txnId), dbName, tableName).get(0).getWriteId(); + } + + @Override + public List allocateTableWriteIdsBatch(List txnIds, String dbName, String tableName) + throws TException { + AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest(dbName, tableName); + rqst.setTxnIds(txnIds); + return allocateTableWriteIdsBatchIntr(rqst); + } + + @Override + public List replAllocateTableWriteIdsBatch(String dbName, String tableName, + String replPolicy, List srcTxnToWriteIdList) throws TException { + AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest(dbName, tableName); + rqst.setReplPolicy(replPolicy); + rqst.setSrcTxnToWriteIdList(srcTxnToWriteIdList); + return allocateTableWriteIdsBatchIntr(rqst); + } + + private List allocateTableWriteIdsBatchIntr(AllocateTableWriteIdsRequest rqst) throws TException { + return client.allocate_table_write_ids(rqst).getTxnToWriteIds(); + } + + @Override + public LockResponse lock(LockRequest request) + throws NoSuchTxnException, TxnAbortedException, TException { + return client.lock(request); + } + + @Override + public LockResponse checkLock(long lockid) + throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, + TException { + return client.check_lock(new CheckLockRequest(lockid)); + } + + @Override + public void unlock(long lockid) + throws NoSuchLockException, TxnOpenException, TException { + client.unlock(new UnlockRequest(lockid)); + } + + @Override + @Deprecated + public ShowLocksResponse showLocks() throws TException { + return client.show_locks(new ShowLocksRequest()); + } + + @Override + public ShowLocksResponse showLocks(ShowLocksRequest request) throws TException { + return client.show_locks(request); + } + + @Override + public void heartbeat(long txnid, long lockid) + throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, + TException { + HeartbeatRequest hb = new HeartbeatRequest(); + hb.setLockid(lockid); + hb.setTxnid(txnid); + client.heartbeat(hb); + } + + @Override + public HeartbeatTxnRangeResponse heartbeatTxnRange(long min, long max) + throws NoSuchTxnException, TxnAbortedException, TException { + HeartbeatTxnRangeRequest rqst = new HeartbeatTxnRangeRequest(min, max); + return client.heartbeat_txn_range(rqst); + } + + @Override + @Deprecated + public void compact(String dbname, String tableName, String partitionName, CompactionType type) + throws TException { + CompactionRequest cr = new CompactionRequest(); + if (dbname == null) { + cr.setDbname(DEFAULT_DATABASE_NAME); + } else { + cr.setDbname(dbname); + } + cr.setTablename(tableName); + if (partitionName != null) { + cr.setPartitionname(partitionName); + } + cr.setType(type); + client.compact(cr); + } + + @Deprecated + @Override + public void compact(String dbname, String tableName, String partitionName, CompactionType type, + Map tblproperties) throws TException { + compact2(dbname, tableName, partitionName, type, tblproperties); + } + + @Override + public CompactionResponse compact2(String dbname, String tableName, String partitionName, CompactionType type, + Map tblproperties) throws TException { + CompactionRequest cr = new CompactionRequest(); + if (dbname == null) { + cr.setDbname(DEFAULT_DATABASE_NAME); + } else { + cr.setDbname(dbname); + } + cr.setTablename(tableName); + if (partitionName != null) { + cr.setPartitionname(partitionName); + } + cr.setType(type); + cr.setProperties(tblproperties); + return client.compact2(cr); + } + @Override + public ShowCompactResponse showCompactions() throws TException { + return client.show_compact(new ShowCompactRequest()); + } + + @Deprecated + @Override + public void addDynamicPartitions(long txnId, long writeId, String dbName, String tableName, + List partNames) throws TException { + client.add_dynamic_partitions(new AddDynamicPartitions(txnId, writeId, dbName, tableName, partNames)); + } + @Override + public void addDynamicPartitions(long txnId, long writeId, String dbName, String tableName, + List partNames, DataOperationType operationType) throws TException { + AddDynamicPartitions adp = new AddDynamicPartitions(txnId, writeId, dbName, tableName, partNames); + adp.setOperationType(operationType); + client.add_dynamic_partitions(adp); + } + + @Override + public void insertTable(Table table, boolean overwrite) throws MetaException { + boolean failed = true; + HiveMetaHook hook = getHook(table); + if (hook == null || !(hook instanceof DefaultHiveMetaHook)) { + return; + } + DefaultHiveMetaHook hiveMetaHook = (DefaultHiveMetaHook) hook; + try { + hiveMetaHook.commitInsertTable(table, overwrite); + failed = false; + } finally { + if (failed) { + hiveMetaHook.rollbackInsertTable(table, overwrite); + } + } + } + + @InterfaceAudience.LimitedPrivate({"HCatalog"}) + @Override + public NotificationEventResponse getNextNotification(long lastEventId, int maxEvents, + NotificationFilter filter) throws TException { + NotificationEventRequest rqst = new NotificationEventRequest(lastEventId); + rqst.setMaxEvents(maxEvents); + NotificationEventResponse rsp = client.get_next_notification(rqst); + if (LOG.isDebugEnabled()) { + LOG.debug("Got back " + rsp.getEventsSize() + " events"); + } + NotificationEventResponse filtered = new NotificationEventResponse(); + if (rsp != null && rsp.getEvents() != null) { + long nextEventId = lastEventId + 1; + for (NotificationEvent e : rsp.getEvents()) { + if (e.getEventId() != nextEventId) { + LOG.error("Requested events are found missing in NOTIFICATION_LOG table. Expected: {}, Actual: {}. " + + "Probably, cleaner would've cleaned it up. " + + "Try setting higher value for hive.metastore.event.db.listener.timetolive. " + + "Also, bootstrap the system again to get back the consistent replicated state.", + nextEventId, e.getEventId()); + throw new IllegalStateException(REPL_EVENTS_MISSING_IN_METASTORE); + } + if ((filter != null) && filter.accept(e)) { + filtered.addToEvents(e); + } + nextEventId++; + } + } + return (filter != null) ? filtered : rsp; + } + + @InterfaceAudience.LimitedPrivate({"HCatalog"}) + @Override + public CurrentNotificationEventId getCurrentNotificationEventId() throws TException { + return client.get_current_notificationEventId(); + } + + @InterfaceAudience.LimitedPrivate({"HCatalog"}) + @Override + public NotificationEventsCountResponse getNotificationEventsCount(NotificationEventsCountRequest rqst) + throws TException { + if (!rqst.isSetCatName()) { + rqst.setCatName(getDefaultCatalog(conf)); + } + return client.get_notification_events_count(rqst); + } + + @InterfaceAudience.LimitedPrivate({"Apache Hive, HCatalog"}) + @Override + public FireEventResponse fireListenerEvent(FireEventRequest rqst) throws TException { + if (!rqst.isSetCatName()) { + rqst.setCatName(getDefaultCatalog(conf)); + } + return client.fire_listener_event(rqst); + } + + /** + * Creates a synchronized wrapper for any {@link IMetaStoreClient}. + * This may be used by multi-threaded applications until we have + * fixed all reentrancy bugs. + * + * @param client unsynchronized client + * + * @return synchronized client + */ + public static IMetaStoreClient newSynchronizedClient( + IMetaStoreClient client) { + return (IMetaStoreClient) Proxy.newProxyInstance( + HiveMetaStoreClient.class.getClassLoader(), + new Class[] {IMetaStoreClient.class}, + new SynchronizedHandler(client)); + } + + private static class SynchronizedHandler implements InvocationHandler { + private final IMetaStoreClient client; + + SynchronizedHandler(IMetaStoreClient client) { + this.client = client; + } + + @Override + public synchronized Object invoke(Object proxy, Method method, Object[] args) + throws Throwable { + try { + return method.invoke(client, args); + } catch (InvocationTargetException e) { + throw e.getTargetException(); + } + } + } + + @Override + public void markPartitionForEvent(String db_name, String tbl_name, + Map partKVs, PartitionEventType eventType) + throws TException { + markPartitionForEvent(getDefaultCatalog(conf), db_name, tbl_name, partKVs, eventType); + } + + @Override + public void markPartitionForEvent(String catName, String db_name, String tbl_name, + Map partKVs, + PartitionEventType eventType) throws TException { + client.markPartitionForEvent(prependCatalogToDbName(catName, db_name, conf), tbl_name, partKVs, + eventType); + + } + + @Override + public boolean isPartitionMarkedForEvent(String db_name, String tbl_name, Map partKVs, + PartitionEventType eventType) + throws TException { + return isPartitionMarkedForEvent(getDefaultCatalog(conf), db_name, tbl_name, partKVs, eventType); + } + + @Override + public boolean isPartitionMarkedForEvent(String catName, String db_name, String tbl_name, + Map partKVs, + PartitionEventType eventType) throws TException { + return client.isPartitionMarkedForEvent(prependCatalogToDbName(catName, db_name, conf), tbl_name, + partKVs, eventType); + } + + @Override + public void createFunction(Function func) throws TException { + if (!func.isSetCatName()) { + func.setCatName(getDefaultCatalog(conf)); + } + client.create_function(func); + } + + @Override + public void alterFunction(String dbName, String funcName, Function newFunction) + throws TException { + alterFunction(getDefaultCatalog(conf), dbName, funcName, newFunction); + } + + @Override + public void alterFunction(String catName, String dbName, String funcName, + Function newFunction) throws TException { + client.alter_function(prependCatalogToDbName(catName, dbName, conf), funcName, newFunction); + } + + @Override + public void dropFunction(String dbName, String funcName) throws TException { + dropFunction(getDefaultCatalog(conf), dbName, funcName); + } + + @Override + public void dropFunction(String catName, String dbName, String funcName) throws TException { + client.drop_function(prependCatalogToDbName(catName, dbName, conf), funcName); + } + + @Override + public Function getFunction(String dbName, String funcName) throws TException { + return getFunction(getDefaultCatalog(conf), dbName, funcName); + } + + @Override + public Function getFunction(String catName, String dbName, String funcName) throws TException { + return deepCopy(client.get_function(prependCatalogToDbName(catName, dbName, conf), funcName)); + } + + @Override + public List getFunctions(String dbName, String pattern) throws TException { + return getFunctions(getDefaultCatalog(conf), dbName, pattern); + } + + @Override + public List getFunctions(String catName, String dbName, String pattern) throws TException { + return client.get_functions(prependCatalogToDbName(catName, dbName, conf), pattern); + } + + @Override + public GetAllFunctionsResponse getAllFunctions() throws TException { + return client.get_all_functions(); + } + + protected void create_table_with_environment_context(Table tbl, EnvironmentContext envContext) + throws AlreadyExistsException, InvalidObjectException, + MetaException, NoSuchObjectException, TException { + client.create_table_with_environment_context(tbl, envContext); + } + + protected void drop_table_with_environment_context(String catName, String dbname, String name, + boolean deleteData, EnvironmentContext envContext) throws TException { + client.drop_table_with_environment_context(prependCatalogToDbNameByVersion(hiveVersion, catName, dbname, conf), + name, deleteData, envContext); + } + + @Override + public AggrStats getAggrColStatsFor(String dbName, String tblName, + List colNames, List partNames) throws NoSuchObjectException, MetaException, TException { + return getAggrColStatsFor(getDefaultCatalog(conf), dbName, tblName, colNames, partNames); + } + + @Override + public AggrStats getAggrColStatsFor(String catName, String dbName, String tblName, + List colNames, List partNames) throws TException { + if (colNames.isEmpty() || partNames.isEmpty()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Columns is empty or partNames is empty : Short-circuiting stats eval on client side."); + } + return new AggrStats(new ArrayList<>(), 0); // Nothing to aggregate + } + PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partNames); + req.setCatName(catName); + return client.get_aggr_stats_for(req); + } + + @Override + public Iterable> getFileMetadata( + final List fileIds) throws TException { + return new MetastoreMapIterable() { + private int listIndex = 0; + @Override + protected Map fetchNextBatch() throws TException { + if (listIndex == fileIds.size()) { + return null; + } + int endIndex = Math.min(listIndex + fileMetadataBatchSize, fileIds.size()); + List subList = fileIds.subList(listIndex, endIndex); + GetFileMetadataResult resp = sendGetFileMetadataReq(subList); + // TODO: we could remember if it's unsupported and stop sending calls; although, it might + // be a bad idea for HS2+standalone metastore that could be updated with support. + // Maybe we should just remember this for some time. + if (!resp.isIsSupported()) { + return null; + } + listIndex = endIndex; + return resp.getMetadata(); + } + }; + } + + private GetFileMetadataResult sendGetFileMetadataReq(List fileIds) throws TException { + return client.get_file_metadata(new GetFileMetadataRequest(fileIds)); + } + + @Override + public Iterable> getFileMetadataBySarg( + final List fileIds, final ByteBuffer sarg, final boolean doGetFooters) + throws TException { + return new MetastoreMapIterable() { + private int listIndex = 0; + @Override + protected Map fetchNextBatch() throws TException { + if (listIndex == fileIds.size()) { + return null; + } + int endIndex = Math.min(listIndex + fileMetadataBatchSize, fileIds.size()); + List subList = fileIds.subList(listIndex, endIndex); + GetFileMetadataByExprResult resp = sendGetFileMetadataBySargReq( + sarg, subList, doGetFooters); + if (!resp.isIsSupported()) { + return null; + } + listIndex = endIndex; + return resp.getMetadata(); + } + }; + } + + private GetFileMetadataByExprResult sendGetFileMetadataBySargReq( + ByteBuffer sarg, List fileIds, boolean doGetFooters) throws TException { + GetFileMetadataByExprRequest req = new GetFileMetadataByExprRequest(fileIds, sarg); + req.setDoGetFooters(doGetFooters); // No need to get footers + return client.get_file_metadata_by_expr(req); + } + + public static abstract class MetastoreMapIterable + implements Iterable>, Iterator> { + private Iterator> currentIter; + + protected abstract Map fetchNextBatch() throws TException; + + @Override + public Iterator> iterator() { + return this; + } + + @Override + public boolean hasNext() { + ensureCurrentBatch(); + return currentIter != null; + } + + private void ensureCurrentBatch() { + if (currentIter != null && currentIter.hasNext()) { + return; + } + currentIter = null; + Map currentBatch; + do { + try { + currentBatch = fetchNextBatch(); + } catch (TException ex) { + throw new RuntimeException(ex); + } + if (currentBatch == null) { + return; // No more data. + } + } while (currentBatch.isEmpty()); + currentIter = currentBatch.entrySet().iterator(); + } + + @Override + public Entry next() { + ensureCurrentBatch(); + if (currentIter == null) { + throw new NoSuchElementException(); + } + return currentIter.next(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + } + + @Override + public void clearFileMetadata(List fileIds) throws TException { + ClearFileMetadataRequest req = new ClearFileMetadataRequest(); + req.setFileIds(fileIds); + client.clear_file_metadata(req); + } + + @Override + public void putFileMetadata(List fileIds, List metadata) throws TException { + PutFileMetadataRequest req = new PutFileMetadataRequest(); + req.setFileIds(fileIds); + req.setMetadata(metadata); + client.put_file_metadata(req); + } + + @Override + public boolean isSameConfObj(Configuration c) { + return conf == c; + } + + @Override + public boolean cacheFileMetadata( + String dbName, String tableName, String partName, boolean allParts) throws TException { + CacheFileMetadataRequest req = new CacheFileMetadataRequest(); + req.setDbName(dbName); + req.setTblName(tableName); + if (partName != null) { + req.setPartName(partName); + } else { + req.setIsAllParts(allParts); + } + CacheFileMetadataResult result = client.cache_file_metadata(req); + return result.isIsSupported(); + } + + @Override + public String getMetastoreDbUuid() throws TException { + return client.get_metastore_db_uuid(); + } + + @Override + public void createResourcePlan(WMResourcePlan resourcePlan, String copyFromName) + throws InvalidObjectException, MetaException, TException { + WMCreateResourcePlanRequest request = new WMCreateResourcePlanRequest(); + request.setResourcePlan(resourcePlan); + request.setCopyFrom(copyFromName); + client.create_resource_plan(request); + } + + @Override + public WMFullResourcePlan getResourcePlan(String resourcePlanName) + throws NoSuchObjectException, MetaException, TException { + WMGetResourcePlanRequest request = new WMGetResourcePlanRequest(); + request.setResourcePlanName(resourcePlanName); + return client.get_resource_plan(request).getResourcePlan(); + } + + @Override + public List getAllResourcePlans() + throws NoSuchObjectException, MetaException, TException { + WMGetAllResourcePlanRequest request = new WMGetAllResourcePlanRequest(); + return client.get_all_resource_plans(request).getResourcePlans(); + } + + @Override + public void dropResourcePlan(String resourcePlanName) + throws NoSuchObjectException, MetaException, TException { + WMDropResourcePlanRequest request = new WMDropResourcePlanRequest(); + request.setResourcePlanName(resourcePlanName); + client.drop_resource_plan(request); + } + + @Override + public WMFullResourcePlan alterResourcePlan(String resourcePlanName, WMNullableResourcePlan resourcePlan, + boolean canActivateDisabled, boolean isForceDeactivate, boolean isReplace) + throws NoSuchObjectException, InvalidObjectException, MetaException, TException { + WMAlterResourcePlanRequest request = new WMAlterResourcePlanRequest(); + request.setResourcePlanName(resourcePlanName); + request.setResourcePlan(resourcePlan); + request.setIsEnableAndActivate(canActivateDisabled); + request.setIsForceDeactivate(isForceDeactivate); + request.setIsReplace(isReplace); + WMAlterResourcePlanResponse resp = client.alter_resource_plan(request); + return resp.isSetFullResourcePlan() ? resp.getFullResourcePlan() : null; + } + + @Override + public WMFullResourcePlan getActiveResourcePlan() throws MetaException, TException { + return client.get_active_resource_plan(new WMGetActiveResourcePlanRequest()).getResourcePlan(); + } + + @Override + public WMValidateResourcePlanResponse validateResourcePlan(String resourcePlanName) + throws NoSuchObjectException, InvalidObjectException, MetaException, TException { + WMValidateResourcePlanRequest request = new WMValidateResourcePlanRequest(); + request.setResourcePlanName(resourcePlanName); + return client.validate_resource_plan(request); + } + + @Override + public void createWMTrigger(WMTrigger trigger) + throws InvalidObjectException, MetaException, TException { + WMCreateTriggerRequest request = new WMCreateTriggerRequest(); + request.setTrigger(trigger); + client.create_wm_trigger(request); + } + + @Override + public void alterWMTrigger(WMTrigger trigger) + throws NoSuchObjectException, InvalidObjectException, MetaException, TException { + WMAlterTriggerRequest request = new WMAlterTriggerRequest(); + request.setTrigger(trigger); + client.alter_wm_trigger(request); + } + + @Override + public void dropWMTrigger(String resourcePlanName, String triggerName) + throws NoSuchObjectException, MetaException, TException { + WMDropTriggerRequest request = new WMDropTriggerRequest(); + request.setResourcePlanName(resourcePlanName); + request.setTriggerName(triggerName); + client.drop_wm_trigger(request); + } + + @Override + public List getTriggersForResourcePlan(String resourcePlan) + throws NoSuchObjectException, MetaException, TException { + WMGetTriggersForResourePlanRequest request = new WMGetTriggersForResourePlanRequest(); + request.setResourcePlanName(resourcePlan); + return client.get_triggers_for_resourceplan(request).getTriggers(); + } + + @Override + public void createWMPool(WMPool pool) + throws NoSuchObjectException, InvalidObjectException, MetaException, TException { + WMCreatePoolRequest request = new WMCreatePoolRequest(); + request.setPool(pool); + client.create_wm_pool(request); + } + + @Override + public void alterWMPool(WMNullablePool pool, String poolPath) + throws NoSuchObjectException, InvalidObjectException, MetaException, TException { + WMAlterPoolRequest request = new WMAlterPoolRequest(); + request.setPool(pool); + request.setPoolPath(poolPath); + client.alter_wm_pool(request); + } + + @Override + public void dropWMPool(String resourcePlanName, String poolPath) + throws NoSuchObjectException, MetaException, TException { + WMDropPoolRequest request = new WMDropPoolRequest(); + request.setResourcePlanName(resourcePlanName); + request.setPoolPath(poolPath); + client.drop_wm_pool(request); + } + + @Override + public void createOrUpdateWMMapping(WMMapping mapping, boolean isUpdate) + throws NoSuchObjectException, InvalidObjectException, MetaException, TException { + WMCreateOrUpdateMappingRequest request = new WMCreateOrUpdateMappingRequest(); + request.setMapping(mapping); + request.setUpdate(isUpdate); + client.create_or_update_wm_mapping(request); + } + + @Override + public void dropWMMapping(WMMapping mapping) + throws NoSuchObjectException, MetaException, TException { + WMDropMappingRequest request = new WMDropMappingRequest(); + request.setMapping(mapping); + client.drop_wm_mapping(request); + } + + @Override + public void createOrDropTriggerToPoolMapping(String resourcePlanName, String triggerName, + String poolPath, boolean shouldDrop) throws AlreadyExistsException, NoSuchObjectException, + InvalidObjectException, MetaException, TException { + WMCreateOrDropTriggerToPoolMappingRequest request = new WMCreateOrDropTriggerToPoolMappingRequest(); + request.setResourcePlanName(resourcePlanName); + request.setTriggerName(triggerName); + request.setPoolPath(poolPath); + request.setDrop(shouldDrop); + client.create_or_drop_wm_trigger_to_pool_mapping(request); + } + + @Override + public void createISchema(ISchema schema) throws TException { + if (!schema.isSetCatName()) { + schema.setCatName(getDefaultCatalog(conf)); + } + client.create_ischema(schema); + } + + @Override + public void alterISchema(String catName, String dbName, String schemaName, ISchema newSchema) throws TException { + client.alter_ischema(new AlterISchemaRequest(new ISchemaName(catName, dbName, schemaName), newSchema)); + } + + @Override + public ISchema getISchema(String catName, String dbName, String name) throws TException { + return client.get_ischema(new ISchemaName(catName, dbName, name)); + } + + @Override + public void dropISchema(String catName, String dbName, String name) throws TException { + client.drop_ischema(new ISchemaName(catName, dbName, name)); + } + + @Override + public void addSchemaVersion(SchemaVersion schemaVersion) throws TException { + if (!schemaVersion.getSchema().isSetCatName()) { + schemaVersion.getSchema().setCatName(getDefaultCatalog(conf)); + } + client.add_schema_version(schemaVersion); + } + + @Override + public SchemaVersion getSchemaVersion(String catName, String dbName, String schemaName, int version) + throws TException { + return client.get_schema_version( + new SchemaVersionDescriptor(new ISchemaName(catName, dbName, schemaName), version)); + } + + @Override + public SchemaVersion getSchemaLatestVersion(String catName, String dbName, String schemaName) throws TException { + return client.get_schema_latest_version(new ISchemaName(catName, dbName, schemaName)); + } + + @Override + public List getSchemaAllVersions(String catName, String dbName, String schemaName) throws + TException { + return client.get_schema_all_versions(new ISchemaName(catName, dbName, schemaName)); + } + + @Override + public void dropSchemaVersion(String catName, String dbName, String schemaName, int version) throws TException { + client.drop_schema_version(new SchemaVersionDescriptor(new ISchemaName(catName, dbName, schemaName), version)); + } + + @Override + public FindSchemasByColsResp getSchemaByCols(FindSchemasByColsRqst rqst) throws TException { + return client.get_schemas_by_cols(rqst); + } + + @Override + public void mapSchemaVersionToSerde(String catName, String dbName, String schemaName, int version, String serdeName) + throws TException { + client.map_schema_version_to_serde(new MapSchemaVersionToSerdeRequest( + new SchemaVersionDescriptor(new ISchemaName(catName, dbName, schemaName), version), serdeName)); + } + + @Override + public void setSchemaVersionState(String catName, String dbName, String schemaName, int version, + SchemaVersionState state) + throws TException { + client.set_schema_version_state(new SetSchemaVersionStateRequest(new SchemaVersionDescriptor( + new ISchemaName(catName, dbName, schemaName), version), state)); + } + + @Override + public void addSerDe(SerDeInfo serDeInfo) throws TException { + client.add_serde(serDeInfo); + } + + @Override + public SerDeInfo getSerDe(String serDeName) throws TException { + return client.get_serde(new GetSerdeRequest(serDeName)); + } + + private short shrinkMaxtoShort(int max) { + if (max < 0) { + return -1; + } else if (max <= Short.MAX_VALUE) { + return (short)max; + } else { + return Short.MAX_VALUE; + } + } + + @Override + public LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId) throws TException { + return client.get_lock_materialization_rebuild(dbName, tableName, txnId); + } + + @Override + public boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId) throws TException { + return client.heartbeat_lock_materialization_rebuild(dbName, tableName, txnId); + } + + @Override + public void addRuntimeStat(RuntimeStat stat) throws TException { + client.add_runtime_stats(stat); + } + + @Override + public List getRuntimeStats(int maxWeight, int maxCreateTime) throws TException { + GetRuntimeStatsRequest req = new GetRuntimeStatsRequest(); + req.setMaxWeight(maxWeight); + req.setMaxCreateTime(maxCreateTime); + return client.get_runtime_stats(req); + } + + private static String prependCatalogToDbNameByVersion(HiveVersion version, @Nullable String catalogName, + @Nullable String dbName, Configuration conf) { + if (version == HiveVersion.V1_0 || version == HiveVersion.V2_0 || version == HiveVersion.V2_3) { + return dbName; + } + return prependCatalogToDbName(catalogName, dbName, conf); + } +} diff --git a/fe/be-java-extensions/preload-extensions/src/main/java/org/apache/hadoop/hive/metastore/HiveVersionUtil.java b/fe/be-java-extensions/preload-extensions/src/main/java/org/apache/hadoop/hive/metastore/HiveVersionUtil.java new file mode 100644 index 00000000000000..3cb111a92a693b --- /dev/null +++ b/fe/be-java-extensions/preload-extensions/src/main/java/org/apache/hadoop/hive/metastore/HiveVersionUtil.java @@ -0,0 +1,84 @@ +// 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.hadoop.hive.metastore; + +import com.google.common.base.Strings; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +/** + * For getting a compatible version of hive + * if user specified the version, it will parse it and return the compatible HiveVersion, + * otherwise, use DEFAULT_HIVE_VERSION + */ +public class HiveVersionUtil { + private static final Logger LOG = LogManager.getLogger(HiveVersionUtil.class); + + private static final HiveVersion DEFAULT_HIVE_VERSION = HiveVersion.V2_3; + + /** + * HiveVersion + */ + public enum HiveVersion { + V1_0, // [1.0.0 - 1.2.2] + V2_0, // [2.0.0 - 2.2.0] + V2_3, // [2.3.0 - 2.3.6] + V3_0 // [3.0.0 - 3.1.2] + } + + /** + * get the compatible HiveVersion + * + * @param version the version string + * @return HiveVersion + */ + public static HiveVersion getVersion(String version) { + if (Strings.isNullOrEmpty(version)) { + return DEFAULT_HIVE_VERSION; + } + String[] parts = version.split("\\."); + if (parts.length < 2) { + LOG.warn("invalid hive version: " + version); + return DEFAULT_HIVE_VERSION; + } + try { + int major = Integer.parseInt(parts[0]); + int minor = Integer.parseInt(parts[1]); + if (major == 1) { + return HiveVersion.V1_0; + } else if (major == 2) { + if (minor >= 0 && minor <= 2) { + return HiveVersion.V1_0; + } else if (minor >= 3) { + return HiveVersion.V2_3; + } else { + LOG.warn("invalid hive version: " + version); + return DEFAULT_HIVE_VERSION; + } + } else if (major >= 3) { + return HiveVersion.V2_3; + } else { + LOG.warn("invalid hive version: " + version); + return DEFAULT_HIVE_VERSION; + } + } catch (NumberFormatException e) { + LOG.warn("invalid hive version: " + version); + return DEFAULT_HIVE_VERSION; + } + } +} diff --git a/fe/be-java-extensions/preload-extensions/src/main/resources/package.xml b/fe/be-java-extensions/preload-extensions/src/main/resources/package-deps.xml similarity index 86% rename from fe/be-java-extensions/preload-extensions/src/main/resources/package.xml rename to fe/be-java-extensions/preload-extensions/src/main/resources/package-deps.xml index 4bbb2610603363..3ace22a7870027 100644 --- a/fe/be-java-extensions/preload-extensions/src/main/resources/package.xml +++ b/fe/be-java-extensions/preload-extensions/src/main/resources/package-deps.xml @@ -28,14 +28,9 @@ under the License. / - true + false true runtime - - - **/Log4j2Plugins.dat - - diff --git a/fe/be-java-extensions/preload-extensions/src/main/resources/package-proj.xml b/fe/be-java-extensions/preload-extensions/src/main/resources/package-proj.xml new file mode 100644 index 00000000000000..d3303fb4ec8a3c --- /dev/null +++ b/fe/be-java-extensions/preload-extensions/src/main/resources/package-proj.xml @@ -0,0 +1,37 @@ + + + + project + + jar + + false + + + ${project.build.outputDirectory} + / + + **/*.class + + + + diff --git a/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorPluginLoader.java b/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorPluginLoader.java index 134f315c333bc7..1f08e28b005fa6 100644 --- a/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorPluginLoader.java +++ b/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorPluginLoader.java @@ -30,6 +30,9 @@ import org.apache.logging.log4j.Logger; import java.io.File; +import java.util.logging.FileHandler; +import java.util.logging.Level; +import java.util.logging.SimpleFormatter; public class TrinoConnectorPluginLoader { private static final Logger LOG = LogManager.getLogger(TrinoConnectorPluginLoader.class); @@ -42,6 +45,18 @@ private static class TrinoConnectorPluginLoad { static { try { + // Trino uses jul as its own log system, so the attributes of JUL are configured here + System.setProperty("java.util.logging.SimpleFormatter.format", + "%1$tY-%1$tm-%1$td %1$tH:%1$tM:%1$tS %4$s: %5$s%6$s%n"); + java.util.logging.Logger logger = java.util.logging.Logger.getLogger(""); + logger.setUseParentHandlers(false); + FileHandler fileHandler = new FileHandler(EnvUtils.getDorisHome() + "/log/trinoconnector%g.log", + 500000000, 10, true); + fileHandler.setLevel(Level.INFO); + fileHandler.setFormatter(new SimpleFormatter()); + logger.addHandler(fileHandler); + java.util.logging.LogManager.getLogManager().addLogger(logger); + TypeOperators typeOperators = new TypeOperators(); featuresConfig = new FeaturesConfig(); TypeRegistry typeRegistry = new TypeRegistry(typeOperators, featuresConfig); diff --git a/fe/be-java-extensions/trino-connector-scanner/src/main/resources/package.xml b/fe/be-java-extensions/trino-connector-scanner/src/main/resources/package.xml index c4f838e54d1b7c..98fb269a547f8f 100644 --- a/fe/be-java-extensions/trino-connector-scanner/src/main/resources/package.xml +++ b/fe/be-java-extensions/trino-connector-scanner/src/main/resources/package.xml @@ -23,11 +23,20 @@ under the License. jar false - + + + ${project.build.outputDirectory} + / + + **/*.class + + + + + diff --git a/fe/check/checkstyle/suppressions.xml b/fe/check/checkstyle/suppressions.xml index d44bd6d3dbf691..84e6666c0c2b32 100644 --- a/fe/check/checkstyle/suppressions.xml +++ b/fe/check/checkstyle/suppressions.xml @@ -61,6 +61,7 @@ under the License. + diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 5f8b6c09624873..93b7df593fe188 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -1143,8 +1143,8 @@ public class Config extends ConfigBase { /** * the max concurrent routine load task num per BE. * This is to limit the num of routine load tasks sending to a BE, and it should also less - * than BE config 'routine_load_thread_pool_size'(default 10), - * which is the routine load task thread pool size on BE. + * than BE config 'max_routine_load_thread_pool_size'(default 1024), + * which is the routine load task thread pool max size on BE. */ @ConfField(mutable = true, masterOnly = true) public static int max_routine_load_task_num_per_be = 5; @@ -2530,11 +2530,22 @@ public class Config extends ConfigBase { }) public static boolean enable_proxy_protocol = false; + // Used to check compatibility when upgrading. + @ConfField + public static boolean enable_check_compatibility_mode = false; + + // Do checkpoint after replaying edit logs. + @ConfField + public static boolean checkpoint_after_check_compatibility = false; //========================================================================== // begin of cloud config //========================================================================== + @ConfField public static int info_sys_accumulated_file_size = 4; + @ConfField public static int warn_sys_accumulated_file_size = 2; + @ConfField public static int audit_sys_accumulated_file_size = 4; + @ConfField public static String cloud_unique_id = ""; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DeleteStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DeleteStmt.java index 6ad2c1285e8c61..fd7a50ea6caa8b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DeleteStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DeleteStmt.java @@ -34,6 +34,7 @@ import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; import org.apache.doris.common.util.Util; +import org.apache.doris.datasource.CatalogIf; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; @@ -307,9 +308,9 @@ void analyzePredicate(Expr predicate, Analyzer analyzer) throws AnalysisExceptio private void checkDeleteConditions() throws AnalysisException { // check condition column is key column and condition value // Here we use "getFullSchema()" to get all columns including VISIBLE and SHADOW columns - + CatalogIf catalog = getCatalog(); // we ensure the db and table exists. - Database db = (Database) Env.getCurrentEnv().getCurrentCatalog().getDb(getDbName()).get(); + Database db = (Database) catalog.getDb(getDbName()).get(); OlapTable table = ((OlapTable) db.getTable(getTableName()).get()); Map nameToColumn = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); @@ -438,6 +439,15 @@ private SlotRef getSlotRef(Predicate condition) { return slotRef; } + private CatalogIf getCatalog() { + Env env = Env.getCurrentEnv(); + if (null == tableName.getCtl()) { + return env.getCurrentCatalog(); + } else { + return env.getCatalogMgr().getCatalog(tableName.getCtl()); + } + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java index c2bc7bc7d0dea0..8a3543d3d36f8f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java @@ -299,6 +299,9 @@ public void analyze(Analyzer analyzer) throws AnalysisException { } this.needTableStable = false; this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_FILE_CACHE_TTL_SECONDS)) { + this.needTableStable = false; + this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; } else { throw new AnalysisException("Unknown table property: " + properties.keySet()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java index b3babe1e367994..490a521132119a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java @@ -17,142 +17,32 @@ package org.apache.doris.analysis; -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.ScalarType; -import org.apache.doris.common.AnalysisException; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; import org.apache.doris.common.UserException; import org.apache.doris.qe.ShowResultSetMetaData; -import com.google.common.base.Strings; - -// For stmt like: -// show load profile "/"; # list all saving load job ids -// show load profile "/e0f7390f5363419e-xxx" # show task ids of specified job -// show load profile "/e0f7390f5363419e-xxx/e0f7390f5363419e-yyy/" # show instance list of the task -// show load profile "/e0f7390f5363419e-xxx/e0f7390f5363419e-yyy/e0f7390f5363419e-zzz" # show instance's graph +// deprecated stmt, use will be guided to a specific url to get profile from +// web browser public class ShowLoadProfileStmt extends ShowStmt { - private static final ShowResultSetMetaData META_DATA_TASK_IDS = - ShowResultSetMetaData.builder() - .addColumn(new Column("TaskId", ScalarType.createVarchar(128))) - .addColumn(new Column("ActiveTime", ScalarType.createVarchar(64))) - .build(); - - public enum PathType { - QUERY_IDS, - TASK_IDS, - FRAGMENTS, - INSTANCES, - SINGLE_INSTANCE - } - - private String idPath; - private PathType pathType; - - private String jobId = ""; - private String taskId = ""; - private String fragmentId = ""; - private String instanceId = ""; - - public ShowLoadProfileStmt(String idPath) { - this.idPath = idPath; - } - - public PathType getPathType() { - return pathType; - } - - public String getJobId() { - return jobId; - } - - public String getTaskId() { - return taskId; - } + private String loadIdPath; - public String getFragmentId() { - return fragmentId; - } - - public String getInstanceId() { - return instanceId; + public ShowLoadProfileStmt(String path) { + this.loadIdPath = path; } @Override public void analyze(Analyzer analyzer) throws UserException { - super.analyze(analyzer); - if (Strings.isNullOrEmpty(idPath)) { - // list all query ids - pathType = PathType.QUERY_IDS; - return; - } - - if (!idPath.startsWith("/")) { - throw new AnalysisException("Path must starts with '/'"); - } - pathType = PathType.QUERY_IDS; - String[] parts = idPath.split("/"); - if (parts.length > 5) { - throw new AnalysisException("Path must in format '/jobId/taskId/fragmentId/instanceId'"); - } - - for (int i = 0; i < parts.length; i++) { - switch (i) { - case 0: - pathType = PathType.QUERY_IDS; - continue; - case 1: - jobId = parts[i]; - pathType = PathType.TASK_IDS; - break; - case 2: - taskId = parts[i]; - pathType = PathType.FRAGMENTS; - break; - case 3: - fragmentId = parts[i]; - pathType = PathType.INSTANCES; - break; - case 4: - instanceId = parts[i]; - pathType = PathType.SINGLE_INSTANCE; - break; - default: - break; - } - } - } - - @Override - public String toSql() { - StringBuilder sb = new StringBuilder("SHOW LOAD PROFILE ").append(idPath); - return sb.toString(); - } - - @Override - public String toString() { - return toSql(); + String selfHost = Env.getCurrentEnv().getSelfNode().getHost(); + int httpPort = Config.http_port; + String terminalMsg = String.format( + "try visit http://%s:%d/QueryProfile/%s, show query/load profile syntax is a deprecated feature", + selfHost, httpPort, this.loadIdPath); + throw new UserException(terminalMsg); } @Override public ShowResultSetMetaData getMetaData() { - switch (pathType) { - case QUERY_IDS: - return ShowQueryProfileStmt.META_DATA_QUERY_IDS; - case TASK_IDS: - return META_DATA_TASK_IDS; - case FRAGMENTS: - return ShowQueryProfileStmt.META_DATA_FRAGMENTS; - case INSTANCES: - return ShowQueryProfileStmt.META_DATA_INSTANCES; - case SINGLE_INSTANCE: - return ShowQueryProfileStmt.META_DATA_SINGLE_INSTANCE; - default: - return null; - } - } - - @Override - public RedirectStatus getRedirectStatus() { - return RedirectStatus.FORWARD_NO_SYNC; + return null; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcStmt.java index 239d41e1a93dd3..bc7c70cdc79257 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcStmt.java @@ -45,7 +45,8 @@ public ProcNodeInterface getNode() { @Override public void analyze(Analyzer analyzer) throws AnalysisException { - if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + if (!Env.getCurrentEnv().getAccessManager() + .checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN_OR_NODE)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); } node = ProcService.getInstance().open(path); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java index f27840faa81357..39d077438264bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java @@ -17,146 +17,32 @@ package org.apache.doris.analysis; -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.ScalarType; -import org.apache.doris.common.AnalysisException; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; import org.apache.doris.common.UserException; -import org.apache.doris.common.profile.SummaryProfile; import org.apache.doris.qe.ShowResultSetMetaData; -import com.google.common.base.Strings; - -// For stmt like: -// show query profile "/"; # list all saving query ids -// show query profile "/e0f7390f5363419e-b416a2a79996083e" # show graph of fragments of the query -// show query profile "/e0f7390f5363419e-b416a2a79996083e/0" # show instance list of the specified fragment -// show query profile "/e0f7390f5363419e-b416a2a79996083e/0/e0f7390f5363419e-b416a2a799960906" # show instance's graph +// deprecated stmt, use will be guided to a specific url to get profile from +// web browser public class ShowQueryProfileStmt extends ShowStmt { - // This should be same as ProfileManager.PROFILE_HEADERS - public static final ShowResultSetMetaData META_DATA_QUERY_IDS; - - public static final ShowResultSetMetaData META_DATA_FRAGMENTS = - ShowResultSetMetaData.builder() - .addColumn(new Column("Fragments", ScalarType.createVarchar(65535))) - .build(); - public static final ShowResultSetMetaData META_DATA_INSTANCES = - ShowResultSetMetaData.builder() - .addColumn(new Column("Instances", ScalarType.createVarchar(128))) - .addColumn(new Column("Host", ScalarType.createVarchar(64))) - .addColumn(new Column("ActiveTime", ScalarType.createVarchar(64))) - .build(); - public static final ShowResultSetMetaData META_DATA_SINGLE_INSTANCE = - ShowResultSetMetaData.builder() - .addColumn(new Column("Instance", ScalarType.createVarchar(65535))) - .build(); - - static { - ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); - for (String key : SummaryProfile.SUMMARY_KEYS) { - builder.addColumn(new Column(key, ScalarType.createStringType())); - } - META_DATA_QUERY_IDS = builder.build(); - } - - public enum PathType { - QUERY_IDS, - FRAGMENTS, - INSTANCES, - SINGLE_INSTANCE - } - private String queryIdPath; - private PathType pathType; - - private String queryId = ""; - private String fragmentId = ""; - private String instanceId = ""; public ShowQueryProfileStmt(String queryIdPath) { this.queryIdPath = queryIdPath; } - public PathType getPathType() { - return pathType; - } - - public String getQueryId() { - return queryId; - } - - public String getFragmentId() { - return fragmentId; - } - - public String getInstanceId() { - return instanceId; - } - @Override public void analyze(Analyzer analyzer) throws UserException { - super.analyze(analyzer); - if (Strings.isNullOrEmpty(queryIdPath)) { - // list all query ids - pathType = PathType.QUERY_IDS; - return; - } - - if (!queryIdPath.startsWith("/")) { - throw new AnalysisException("Query path must starts with '/'"); - } - pathType = PathType.QUERY_IDS; - String[] parts = queryIdPath.split("/"); - if (parts.length > 4) { - throw new AnalysisException("Query path must in format '/queryId/fragmentId/instanceId'"); - } - - for (int i = 0; i < parts.length; i++) { - switch (i) { - case 0: - pathType = PathType.QUERY_IDS; - continue; - case 1: - queryId = parts[i]; - pathType = PathType.FRAGMENTS; - break; - case 2: - fragmentId = parts[i]; - pathType = PathType.INSTANCES; - break; - case 3: - instanceId = parts[i]; - pathType = PathType.SINGLE_INSTANCE; - break; - default: - break; - } - } - } - - @Override - public String toSql() { - StringBuilder sb = new StringBuilder("SHOW QUERY PROFILE ").append(queryIdPath); - return sb.toString(); - } - - @Override - public String toString() { - return toSql(); + String selfHost = Env.getCurrentEnv().getSelfNode().getHost(); + int httpPort = Config.http_port; + String terminalMsg = String.format( + "try visit http://%s:%d/QueryProfile/%s, show query/load profile syntax is a deprecated feature", + selfHost, httpPort, this.queryIdPath); + throw new UserException(terminalMsg); } @Override public ShowResultSetMetaData getMetaData() { - switch (pathType) { - case QUERY_IDS: - return META_DATA_QUERY_IDS; - case FRAGMENTS: - return META_DATA_FRAGMENTS; - case INSTANCES: - return META_DATA_INSTANCES; - case SINGLE_INSTANCE: - return META_DATA_SINGLE_INSTANCE; - default: - return null; - } + return null; } } 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 e91ff7febe7197..1b479f364639b2 100755 --- 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 @@ -313,6 +313,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; + /** * A singleton class can also be seen as an entry point of Doris. * All manager classes can be obtained through this class. @@ -1000,7 +1001,13 @@ public void initialize(String[] args) throws Exception { auditEventProcessor.start(); // 2. get cluster id and role (Observer or Follower) - getClusterIdAndRole(); + if (!Config.enable_check_compatibility_mode) { + getClusterIdAndRole(); + } else { + role = FrontendNodeType.FOLLOWER; + nodeName = genFeNodeName(selfNode.getHost(), + selfNode.getPort(), false /* new style */); + } // 3. Load image first and replay edits this.editLog = new EditLog(nodeName); @@ -1009,6 +1016,10 @@ public void initialize(String[] args) throws Exception { this.globalTransactionMgr.setEditLog(editLog); this.idGenerator.setEditLog(editLog); + if (Config.enable_check_compatibility_mode) { + replayJournalsAndExit(); + } + // 4. create load and export job label cleaner thread createLabelCleaner(); @@ -1467,6 +1478,13 @@ private void transferToMaster() { long replayEndTime = System.currentTimeMillis(); LOG.info("finish replay in " + (replayEndTime - replayStartTime) + " msec"); + if (Config.enable_check_compatibility_mode) { + String msg = "check metadata compatibility successfully"; + LOG.info(msg); + System.out.println(msg); + System.exit(0); + } + checkCurrentNodeExist(); checkBeExecVersion(); @@ -1675,7 +1693,7 @@ protected void startMasterOnlyDaemonThreads() { } // start threads that should running on all FE - private void startNonMasterDaemonThreads() { + protected void startNonMasterDaemonThreads() { // start load manager thread loadManager.start(); tabletStatMgr.start(); @@ -4907,7 +4925,8 @@ public void modifyTableProperties(Database db, OlapTable table, Map properties) { for (Map.Entry property : properties.entrySet()) { if (property.getKey().equalsIgnoreCase(HADOOP_FS_NAME)) { tHdfsParams.setFsName(property.getValue()); + } else if (property.getKey().equalsIgnoreCase(HADOOP_FS_ROOT_PATH)) { + tHdfsParams.setRootPath(property.getValue()); } else if (property.getKey().equalsIgnoreCase(AuthenticationConfig.HADOOP_USER_NAME)) { tHdfsParams.setUser(property.getValue()); } else if (property.getKey().equalsIgnoreCase(AuthenticationConfig.HADOOP_KERBEROS_PRINCIPAL)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java index b227afdc142eab..3f11e9ffc4fd1b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java @@ -578,4 +578,9 @@ public JsonElement serialize(PartitionKey partitionKey, java.lang.reflect.Type r return result; } } + + // for test + public List getOriginHiveKeys() { + return originHiveKeys; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java index 751a572b2f30be..75d95cd12de0b5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java @@ -134,7 +134,7 @@ public TableProperty buildProperty(short opCode) { case OperationType.OP_MODIFY_REPLICATION_NUM: buildReplicaAllocation(); break; - case OperationType.OP_MODIFY_IN_MEMORY: + case OperationType.OP_MODIFY_TABLE_PROPERTIES: buildInMemory(); buildMinLoadReplicaNum(); buildStorageMedium(); @@ -149,6 +149,7 @@ public TableProperty buildProperty(short opCode) { buildDisableAutoCompaction(); buildTimeSeriesCompactionEmptyRowsetsThreshold(); buildTimeSeriesCompactionLevelThreshold(); + buildTTLSeconds(); break; default: break; diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java index 19bd102c84c171..613fef3be685f5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java @@ -54,19 +54,29 @@ public class CloudEnv extends Env { private static final Logger LOG = LogManager.getLogger(CloudEnv.class); + private CloudInstanceStatusChecker cloudInstanceStatusChecker; private CloudClusterChecker cloudClusterCheck; public CloudEnv(boolean isCheckpointCatalog) { super(isCheckpointCatalog); this.cloudClusterCheck = new CloudClusterChecker((CloudSystemInfoService) systemInfo); + this.cloudInstanceStatusChecker = new CloudInstanceStatusChecker((CloudSystemInfoService) systemInfo); } + @Override protected void startMasterOnlyDaemonThreads() { LOG.info("start cloud Master only daemon threads"); super.startMasterOnlyDaemonThreads(); cloudClusterCheck.start(); } + @Override + protected void startNonMasterDaemonThreads() { + LOG.info("start cloud Non Master only daemon threads"); + super.startNonMasterDaemonThreads(); + cloudInstanceStatusChecker.start(); + } + public static String genFeNodeNameFromMeta(String host, int port, long timeMs) { return host + "_" + port + "_" + timeMs; } @@ -382,7 +392,6 @@ public void checkCloudClusterPriv(String clusterName) throws DdlException { public void changeCloudCluster(String clusterName, ConnectContext ctx) throws DdlException { checkCloudClusterPriv(clusterName); - // TODO(merge-cloud): pick cloud auto start CloudSystemInfoService.waitForAutoStart(clusterName); try { ((CloudSystemInfoService) Env.getCurrentSystemInfo()).addCloudCluster(clusterName, ""); diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java index 49cb3c205903fc..19949ac73d3c0c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java @@ -314,4 +314,15 @@ public Cloud.GetInstanceResponse getInstance(Cloud.GetInstanceRequest request) { } return blockingStub.getInstance(request); } + + public Cloud.GetRLTaskCommitAttachResponse + getRLTaskCommitAttach(Cloud.GetRLTaskCommitAttachRequest request) { + if (!request.hasCloudUniqueId()) { + Cloud.GetRLTaskCommitAttachRequest.Builder builder = + Cloud.GetRLTaskCommitAttachRequest.newBuilder(); + builder.mergeFrom(request); + return blockingStub.getRlTaskCommitAttach(builder.setCloudUniqueId(Config.cloud_unique_id).build()); + } + return blockingStub.getRlTaskCommitAttach(request); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java index 9715d831e8f3ae..ccdb8006bb73d4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java @@ -442,4 +442,15 @@ public Cloud.AlterObjStoreInfoResponse alterObjStoreInfo(Cloud.AlterObjStoreInfo throw new RpcException("", e.getMessage(), e); } } + + public Cloud.GetRLTaskCommitAttachResponse + getRLTaskCommitAttach(Cloud.GetRLTaskCommitAttachRequest request) + throws RpcException { + try { + final MetaServiceClient client = getProxy(); + return client.getRLTaskCommitAttach(request); + } catch (Exception e) { + throw new RpcException("", e.getMessage(), e); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java index 8d852c1109714b..5eb74590d56292 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java @@ -38,9 +38,11 @@ import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TStorageMedium; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import org.apache.commons.lang3.time.StopWatch; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -578,9 +580,110 @@ public static void waitForAutoStartCurrentCluster() throws DdlException { } } - public static void waitForAutoStart(final String clusterName) throws DdlException { - // TODO(merge-cloud): merge from cloud. - // throw new DdlException("Env.waitForAutoStart unimplemented"); + public static String getClusterNameAutoStart(final String clusterName) { + if (!Strings.isNullOrEmpty(clusterName)) { + return clusterName; + } + + ConnectContext context = ConnectContext.get(); + if (context == null) { + LOG.warn("auto start cant get context so new it"); + context = new ConnectContext(); + } + ConnectContext.CloudClusterResult cloudClusterTypeAndName = context.getCloudClusterByPolicy(); + if (cloudClusterTypeAndName == null) { + LOG.warn("get cluster from ctx err"); + return null; + } + if (cloudClusterTypeAndName.comment + == ConnectContext.CloudClusterResult.Comment.DEFAULT_CLUSTER_SET_BUT_NOT_EXIST) { + LOG.warn("get default cluster from ctx err"); + return null; + } + + Preconditions.checkState(!Strings.isNullOrEmpty(cloudClusterTypeAndName.clusterName), + "get cluster name empty"); + LOG.info("get cluster to resume {}", cloudClusterTypeAndName); + return cloudClusterTypeAndName.clusterName; + } + + public static void waitForAutoStart(String clusterName) throws DdlException { + if (Config.isNotCloudMode()) { + return; + } + clusterName = getClusterNameAutoStart(clusterName); + if (Strings.isNullOrEmpty(clusterName)) { + LOG.warn("auto start in cloud mode, but clusterName empty {}", clusterName); + return; + } + String clusterStatus = ((CloudSystemInfoService) Env.getCurrentSystemInfo()).getCloudStatusByName(clusterName); + if (Strings.isNullOrEmpty(clusterStatus)) { + // for cluster rename or cluster dropped + LOG.warn("cant find clusterStatus in fe, clusterName {}", clusterName); + return; + } + // nofity ms -> wait for clusterStatus to normal + LOG.debug("auto start wait cluster {} status {}-{}", clusterName, clusterStatus, + Cloud.ClusterStatus.valueOf(clusterStatus)); + if (Cloud.ClusterStatus.valueOf(clusterStatus) != Cloud.ClusterStatus.NORMAL) { + Cloud.AlterClusterRequest.Builder builder = Cloud.AlterClusterRequest.newBuilder(); + builder.setCloudUniqueId(Config.cloud_unique_id); + builder.setOp(Cloud.AlterClusterRequest.Operation.SET_CLUSTER_STATUS); + + Cloud.ClusterPB.Builder clusterBuilder = Cloud.ClusterPB.newBuilder(); + clusterBuilder.setClusterId(((CloudSystemInfoService) + Env.getCurrentSystemInfo()).getCloudClusterIdByName(clusterName)); + clusterBuilder.setClusterStatus(Cloud.ClusterStatus.TO_RESUME); + builder.setCluster(clusterBuilder); + + Cloud.AlterClusterResponse response; + try { + response = MetaServiceProxy.getInstance().alterCluster(builder.build()); + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + LOG.warn("notify to resume cluster not ok, cluster {}, response: {}", clusterName, response); + } + LOG.info("notify to resume cluster {}, response: {} ", clusterName, response); + } catch (RpcException e) { + LOG.warn("failed to notify to resume cluster {}", clusterName, e); + throw new DdlException("notify to resume cluster not ok"); + } + } + // wait 15 mins? + int retryTimes = 15 * 60; + int retryTime = 0; + StopWatch stopWatch = new StopWatch(); + stopWatch.start(); + boolean hasAutoStart = false; + while (!String.valueOf(Cloud.ClusterStatus.NORMAL).equals(clusterStatus) + && retryTime < retryTimes) { + hasAutoStart = true; + ++retryTime; + // sleep random millis [0.5, 1] s + int randomSeconds = 500 + (int) (Math.random() * (1000 - 500)); + LOG.info("resume cluster {} retry times {}, wait randomMillis: {}, current status: {}", + clusterName, retryTime, randomSeconds, clusterStatus); + try { + if (retryTime > retryTimes / 2) { + // sleep random millis [1, 1.5] s + randomSeconds = 1000 + (int) (Math.random() * (1000 - 500)); + } + Thread.sleep(randomSeconds); + } catch (InterruptedException e) { + LOG.info("change cluster sleep wait InterruptedException: ", e); + } + clusterStatus = ((CloudSystemInfoService) Env.getCurrentSystemInfo()).getCloudStatusByName(clusterName); + } + if (retryTime >= retryTimes) { + // auto start timeout + stopWatch.stop(); + LOG.warn("auto start cluster {} timeout, wait {} ms", clusterName, stopWatch.getTime()); + throw new DdlException("auto start cluster timeout"); + } + + stopWatch.stop(); + if (hasAutoStart) { + LOG.info("auto start cluster {}, start cost {} ms", clusterName, stopWatch.getTime()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java b/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java index 4af8b682778d3e..7d479af62dc646 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java @@ -34,7 +34,7 @@ public class FeNameFormat { private static final String UNDERSCORE_COMMON_NAME_REGEX = "^[_a-zA-Z][a-zA-Z0-9-_]{0,63}$"; private static final String TABLE_NAME_REGEX = "^[a-zA-Z][a-zA-Z0-9-_]*$"; private static final String USER_NAME_REGEX = "^[a-zA-Z][a-zA-Z0-9.-_]*$"; - private static final String COLUMN_NAME_REGEX = "^[_a-zA-Z@0-9\\s<>/][.a-zA-Z0-9_+-/>\n" + " \n" + " \n" - + " \n" + + " \n" + + " \n" + + " \n" + " \n" + " \n" + " \n" @@ -74,7 +76,9 @@ public class Log4jConfig extends XmlConfiguration { + " \n" + " \n" + " \n" - + " \n" + + " \n" + + " \n" + + " \n" + " \n" + " \n" + " \n" @@ -89,7 +93,9 @@ public class Log4jConfig extends XmlConfiguration { + " \n" + " \n" + " \n" - + " \n" + + " \n" + + " \n" + + " \n" + " \n" + " \n" + " \n" @@ -215,6 +221,11 @@ private static void reconfig() throws IOException { properties.put("audit_roll_maxsize", auditRollMaxSize); properties.put("audit_roll_num", auditRollNum); properties.put("audit_log_delete_age", auditDeleteAge); + + properties.put("info_sys_accumulated_file_size", String.valueOf(Config.info_sys_accumulated_file_size)); + properties.put("warn_sys_accumulated_file_size", String.valueOf(Config.warn_sys_accumulated_file_size)); + properties.put("audit_sys_accumulated_file_size", String.valueOf(Config.audit_sys_accumulated_file_size)); + properties.put("include_location_flag", sysLogMode.equalsIgnoreCase("NORMAL") ? "true" : "false"); properties.put("immediate_flush_flag", sysLogMode.equalsIgnoreCase("ASYNC") ? "false" : "true"); properties.put("audit_file_postfix", compressAuditLog ? ".gz" : ""); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java index 4c002275bde15b..3ebce9667774bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java @@ -586,42 +586,89 @@ private static int findNextNestedField(String commaSplitFields) { * Convert doris type to hive type. */ public static String dorisTypeToHiveType(Type dorisType) { - if (dorisType.equals(Type.BOOLEAN)) { - return "boolean"; - } else if (dorisType.equals(Type.TINYINT)) { - return "tinyint"; - } else if (dorisType.equals(Type.SMALLINT)) { - return "smallint"; - } else if (dorisType.equals(Type.INT)) { - return "int"; - } else if (dorisType.equals(Type.BIGINT)) { - return "bigint"; - } else if (dorisType.equals(Type.DATE) || dorisType.equals(Type.DATEV2)) { - return "date"; - } else if (dorisType.equals(Type.DATETIME) || dorisType.equals(Type.DATETIMEV2)) { - return "timestamp"; - } else if (dorisType.equals(Type.FLOAT)) { - return "float"; - } else if (dorisType.equals(Type.DOUBLE)) { - return "double"; - } else if (dorisType.equals(Type.STRING)) { - return "string"; - } else if (dorisType.equals(Type.DEFAULT_DECIMALV3)) { - StringBuilder decimalType = new StringBuilder(); - decimalType.append("decimal"); - ScalarType scalarType = (ScalarType) dorisType; - int precision = scalarType.getScalarPrecision(); - if (precision == 0) { - precision = ScalarType.DEFAULT_PRECISION; + if (dorisType.isScalarType()) { + PrimitiveType primitiveType = dorisType.getPrimitiveType(); + switch (primitiveType) { + case BOOLEAN: + return "boolean"; + case TINYINT: + return "tinyint"; + case SMALLINT: + return "smallint"; + case INT: + return "int"; + case BIGINT: + return "bigint"; + case DATEV2: + case DATE: + return "date"; + case DATETIMEV2: + case DATETIME: + return "timestamp"; + case FLOAT: + return "float"; + case DOUBLE: + return "double"; + case CHAR: { + ScalarType scalarType = (ScalarType) dorisType; + return "char(" + scalarType.getLength() + ")"; + } + case VARCHAR: + case STRING: + return "string"; + case DECIMAL32: + case DECIMAL64: + case DECIMAL128: + case DECIMAL256: + case DECIMALV2: { + StringBuilder decimalType = new StringBuilder(); + decimalType.append("decimal"); + ScalarType scalarType = (ScalarType) dorisType; + int precision = scalarType.getScalarPrecision(); + if (precision == 0) { + precision = ScalarType.DEFAULT_PRECISION; + } + // decimal(precision, scale) + int scale = scalarType.getScalarScale(); + decimalType.append("("); + decimalType.append(precision); + decimalType.append(","); + decimalType.append(scale); + decimalType.append(")"); + return decimalType.toString(); + } + default: + throw new HMSClientException("Unsupported primitive type conversion of " + dorisType.toSql()); + } + } else if (dorisType.isArrayType()) { + ArrayType dorisArray = (ArrayType) dorisType; + Type itemType = dorisArray.getItemType(); + return "array<" + dorisTypeToHiveType(itemType) + ">"; + } else if (dorisType.isMapType()) { + MapType dorisMap = (MapType) dorisType; + Type keyType = dorisMap.getKeyType(); + Type valueType = dorisMap.getValueType(); + return "map<" + + dorisTypeToHiveType(keyType) + + "," + + dorisTypeToHiveType(valueType) + + ">"; + } else if (dorisType.isStructType()) { + StructType dorisStruct = (StructType) dorisType; + StringBuilder structType = new StringBuilder(); + structType.append("struct<"); + ArrayList fields = dorisStruct.getFields(); + for (int i = 0; i < fields.size(); i++) { + StructField field = fields.get(i); + structType.append(field.getName()); + structType.append(":"); + structType.append(dorisTypeToHiveType(field.getType())); + if (i != fields.size() - 1) { + structType.append(","); + } } - // decimal(precision, scale) - int scale = scalarType.getScalarScale(); - decimalType.append("("); - decimalType.append(precision); - decimalType.append(","); - decimalType.append(scale); - decimalType.append(")"); - return decimalType.toString(); + structType.append(">"); + return structType.toString(); } throw new HMSClientException("Unsupported type conversion of " + dorisType.toSql()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java index eb107464bfc532..cc8348940761db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java @@ -17,6 +17,8 @@ package org.apache.doris.datasource.hive; +import org.apache.doris.catalog.Column; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.fs.remote.BrokerFileSystem; import org.apache.doris.fs.remote.RemoteFileSystem; @@ -24,8 +26,13 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; import org.apache.hadoop.mapred.InputFormat; @@ -36,8 +43,13 @@ import java.io.UnsupportedEncodingException; import java.net.URLDecoder; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.Set; import java.util.stream.Collectors; /** @@ -157,4 +169,87 @@ public static Map convertToNamePartitionMap( } return resultBuilder.build(); } + + public static Table toHiveTable(HiveTableMetadata hiveTable) { + Objects.requireNonNull(hiveTable.getDbName(), "Hive database name should be not null"); + Objects.requireNonNull(hiveTable.getTableName(), "Hive table name should be not null"); + Table table = new Table(); + table.setDbName(hiveTable.getDbName()); + table.setTableName(hiveTable.getTableName()); + // table.setOwner(""); + int createTime = (int) System.currentTimeMillis() * 1000; + table.setCreateTime(createTime); + table.setLastAccessTime(createTime); + // table.setRetention(0); + String location = hiveTable.getProperties().get(HiveMetadataOps.LOCATION_URI_KEY); + Set partitionSet = new HashSet<>(hiveTable.getPartitionKeys()); + Pair, List> hiveSchema = toHiveSchema(hiveTable.getColumns(), partitionSet); + + table.setSd(toHiveStorageDesc(hiveSchema.first, hiveTable.getBucketCols(), hiveTable.getNumBuckets(), + hiveTable.getFileFormat(), location)); + table.setPartitionKeys(hiveSchema.second); + + // table.setViewOriginalText(hiveTable.getViewSql()); + // table.setViewExpandedText(hiveTable.getViewSql()); + table.setTableType("MANAGED_TABLE"); + table.setParameters(hiveTable.getProperties()); + return table; + } + + private static StorageDescriptor toHiveStorageDesc(List columns, + List bucketCols, int numBuckets, String fileFormat, String location) { + StorageDescriptor sd = new StorageDescriptor(); + sd.setCols(columns); + setFileFormat(fileFormat, sd); + if (StringUtils.isNotEmpty(location)) { + sd.setLocation(location); + } + sd.setBucketCols(bucketCols); + sd.setNumBuckets(numBuckets); + Map parameters = new HashMap<>(); + parameters.put("tag", "doris external hive talbe"); + sd.setParameters(parameters); + return sd; + } + + private static void setFileFormat(String fileFormat, StorageDescriptor sd) { + String inputFormat; + String outputFormat; + String serDe; + if (fileFormat.equalsIgnoreCase("orc")) { + inputFormat = "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"; + outputFormat = "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"; + serDe = "org.apache.hadoop.hive.ql.io.orc.OrcSerde"; + } else if (fileFormat.equalsIgnoreCase("parquet")) { + inputFormat = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; + outputFormat = "'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"; + serDe = "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"; + } else { + throw new IllegalArgumentException("Creating table with an unsupported file format: " + fileFormat); + } + SerDeInfo serDeInfo = new SerDeInfo(); + serDeInfo.setSerializationLib(serDe); + sd.setSerdeInfo(serDeInfo); + sd.setInputFormat(inputFormat); + sd.setOutputFormat(outputFormat); + } + + private static Pair, List> toHiveSchema(List columns, + Set partitionSet) { + List hiveCols = new ArrayList<>(); + List hiveParts = new ArrayList<>(); + for (Column column : columns) { + FieldSchema hiveFieldSchema = new FieldSchema(); + // TODO: add doc, just support doris type + hiveFieldSchema.setType(HiveMetaStoreClientHelper.dorisTypeToHiveType(column.getType())); + hiveFieldSchema.setName(column.getName()); + hiveFieldSchema.setComment(column.getComment()); + if (partitionSet.contains(column.getName())) { + hiveParts.add(hiveFieldSchema); + } else { + hiveCols.add(hiveFieldSchema); + } + } + return Pair.of(hiveCols, hiveParts); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveVersionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveVersionUtil.java index b93d5653add75f..59afad2d4be1f5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveVersionUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveVersionUtil.java @@ -31,6 +31,9 @@ public class HiveVersionUtil { private static final HiveVersion DEFAULT_HIVE_VERSION = HiveVersion.V2_3; + /** + * HiveVersion + */ public enum HiveVersion { V1_0, // [1.0.0 - 1.2.2] V2_0, // [2.0.0 - 2.2.0] @@ -38,6 +41,12 @@ public enum HiveVersion { V3_0 // [3.0.0 - 3.1.2] } + /** + * get the compatible HiveVersion + * + * @param version the version string + * @return HiveVersion + */ public static HiveVersion getVersion(String version) { if (Strings.isNullOrEmpty(version)) { return DEFAULT_HIVE_VERSION; @@ -72,26 +81,4 @@ public static HiveVersion getVersion(String version) { return DEFAULT_HIVE_VERSION; } } - - public static boolean isHive1(String version) { - if (Strings.isNullOrEmpty(version)) { - return false; - } - String[] parts = version.split("\\."); - if (parts.length < 2) { - LOG.warn("invalid hive version: " + version); - return false; - } - try { - int major = Integer.parseInt(parts[0]); - if (major == 1) { - return true; - } else { - return false; - } - } catch (NumberFormatException e) { - LOG.warn("invalid hive version: " + version); - return false; - } - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java index b5b1147447e183..1f3d188ac6b43f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java @@ -18,9 +18,7 @@ package org.apache.doris.datasource.hive; import org.apache.doris.analysis.TableName; -import org.apache.doris.catalog.Column; import org.apache.doris.common.Config; -import org.apache.doris.common.Pair; import org.apache.doris.datasource.DatabaseMetadata; import org.apache.doris.datasource.TableMetadata; import org.apache.doris.datasource.hive.event.MetastoreNotificationFetchException; @@ -68,18 +66,14 @@ import org.apache.logging.log4j.Logger; import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; import java.util.BitSet; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Queue; -import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -178,7 +172,7 @@ public void createTable(TableMetadata tbl, boolean ignoreIfExists) { // String location, if (tbl instanceof HiveTableMetadata) { ugiDoAs(() -> { - client.client.createTable(toHiveTable((HiveTableMetadata) tbl)); + client.client.createTable(HiveUtil.toHiveTable((HiveTableMetadata) tbl)); return null; }); } @@ -191,92 +185,6 @@ public void createTable(TableMetadata tbl, boolean ignoreIfExists) { } } - private static Table toHiveTable(HiveTableMetadata hiveTable) { - Objects.requireNonNull(hiveTable.getDbName(), "Hive database name should be not null"); - Objects.requireNonNull(hiveTable.getTableName(), "Hive table name should be not null"); - Table table = new Table(); - table.setDbName(hiveTable.getDbName()); - table.setTableName(hiveTable.getTableName()); - // table.setOwner(""); - int createTime = (int) System.currentTimeMillis() * 1000; - table.setCreateTime(createTime); - table.setLastAccessTime(createTime); - // table.setRetention(0); - String location = hiveTable.getProperties().get(HiveMetadataOps.LOCATION_URI_KEY); - Set partitionSet = new HashSet<>(hiveTable.getPartitionKeys()); - Pair, List> hiveSchema = toHiveSchema(hiveTable.getColumns(), partitionSet); - - table.setSd(toHiveStorageDesc(hiveSchema.first, hiveTable.getBucketCols(), hiveTable.getNumBuckets(), - hiveTable.getFileFormat(), location)); - table.setPartitionKeys(hiveSchema.second); - - // table.setViewOriginalText(hiveTable.getViewSql()); - // table.setViewExpandedText(hiveTable.getViewSql()); - table.setTableType("MANAGED_TABLE"); - table.setParameters(hiveTable.getProperties()); - return table; - } - - private static StorageDescriptor toHiveStorageDesc(List columns, - List bucketCols, - int numBuckets, - String fileFormat, - String location) { - StorageDescriptor sd = new StorageDescriptor(); - sd.setCols(columns); - setFileFormat(fileFormat, sd); - if (StringUtils.isNotEmpty(location)) { - sd.setLocation(location); - } - sd.setBucketCols(bucketCols); - sd.setNumBuckets(numBuckets); - Map parameters = new HashMap<>(); - parameters.put("tag", "doris external hive talbe"); - sd.setParameters(parameters); - return sd; - } - - private static void setFileFormat(String fileFormat, StorageDescriptor sd) { - String inputFormat; - String outputFormat; - String serDe; - if (fileFormat.equalsIgnoreCase("orc")) { - inputFormat = "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"; - outputFormat = "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"; - serDe = "org.apache.hadoop.hive.ql.io.orc.OrcSerde"; - } else if (fileFormat.equalsIgnoreCase("parquet")) { - inputFormat = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; - outputFormat = "'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"; - serDe = "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"; - } else { - throw new IllegalArgumentException("Creating table with an unsupported file format: " + fileFormat); - } - SerDeInfo serDeInfo = new SerDeInfo(); - serDeInfo.setSerializationLib(serDe); - sd.setSerdeInfo(serDeInfo); - sd.setInputFormat(inputFormat); - sd.setOutputFormat(outputFormat); - } - - private static Pair, List> toHiveSchema(List columns, - Set partitionSet) { - List hiveCols = new ArrayList<>(); - List hiveParts = new ArrayList<>(); - for (Column column : columns) { - FieldSchema hiveFieldSchema = new FieldSchema(); - // TODO: add doc, just support doris type - hiveFieldSchema.setType(HiveMetaStoreClientHelper.dorisTypeToHiveType(column.getType())); - hiveFieldSchema.setName(column.getName()); - hiveFieldSchema.setComment(column.getComment()); - if (partitionSet.contains(column.getName())) { - hiveParts.add(hiveFieldSchema); - } else { - hiveCols.add(hiveFieldSchema); - } - } - return Pair.of(hiveCols, hiveParts); - } - @Override public void dropDatabase(String dbName) { try (ThriftHMSClient client = getClient()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java index db98df8bc97785..3c224d384322a6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java @@ -309,8 +309,10 @@ private void testJdbcConnection(boolean isReplay) throws DdlException { testFeToJdbcConnection(); testBeToJdbcConnection(); } finally { - jdbcClient.closeClient(); - jdbcClient = null; + if (jdbcClient != null) { + jdbcClient.closeClient(); + jdbcClient = null; + } } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java index ec0a0586937128..0a273e5424cd64 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java @@ -23,6 +23,7 @@ import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.InitCatalogLog; import org.apache.doris.datasource.SessionContext; +import org.apache.doris.datasource.property.constants.HMSProperties; import org.apache.doris.datasource.property.constants.PaimonProperties; import com.google.common.collect.ImmutableList; @@ -33,9 +34,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; -import org.apache.paimon.catalog.FileSystemCatalog; import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.hive.HiveCatalog; import org.apache.paimon.options.Options; import java.util.ArrayList; @@ -65,15 +64,9 @@ protected void initLocalObjectsImpl() { for (Map.Entry propEntry : this.catalogProperty.getHadoopProperties().entrySet()) { conf.set(propEntry.getKey(), propEntry.getValue()); } - if (catalog instanceof FileSystemCatalog) { - authConf = AuthenticationConfig.getKerberosConfig(conf, - AuthenticationConfig.HADOOP_KERBEROS_PRINCIPAL, - AuthenticationConfig.HADOOP_KERBEROS_KEYTAB); - } else if (catalog instanceof HiveCatalog) { - authConf = AuthenticationConfig.getKerberosConfig(conf, - AuthenticationConfig.HIVE_KERBEROS_PRINCIPAL, - AuthenticationConfig.HIVE_KERBEROS_KEYTAB); - } + authConf = AuthenticationConfig.getKerberosConfig(conf, + AuthenticationConfig.HADOOP_KERBEROS_PRINCIPAL, + AuthenticationConfig.HADOOP_KERBEROS_KEYTAB); } public String getCatalogType() { @@ -133,6 +126,10 @@ protected Catalog createCatalog() { options.set(kv.getKey(), kv.getValue()); } CatalogContext context = CatalogContext.create(options, getConfiguration()); + return createCatalogImpl(context); + } + + protected Catalog createCatalogImpl(CatalogContext context) { return CatalogFactory.createCatalog(context); } @@ -153,6 +150,13 @@ protected void setPaimonExtraOptions(Map properties, Map primaryToForeign) { return true; } // There are some predicates in primary key but there is no predicate in foreign key - if (slotWithPredicates.containsKey(pf.getValue()) && slotWithPredicates.get(pf.getValue()).isEmpty()) { + if (!slotWithPredicates.containsKey(pf.getValue()) || slotWithPredicates.get(pf.getValue()).isEmpty()) { return false; } Set primaryPredicates = slotWithPredicates.get(pf.getKey()).stream() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java index f371a359e0a57f..67864187bb2a48 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java @@ -180,7 +180,7 @@ private static Expression convertRangePartitionToCompare(PartitionItem item, Slo Expression predicate = ExpressionUtils.and(expressions); // The partition without can be the first partition of LESS THAN PARTITIONS // The null value can insert into this partition, so we need to add or is null condition - if (!range.hasLowerBound()) { + if (!range.hasLowerBound() || range.lowerEndpoint().isMinValue()) { predicate = ExpressionUtils.or(predicate, new IsNull(col)); } return predicate; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java index 8a37bb71cc6d79..a670290a1630f9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java @@ -153,6 +153,7 @@ public PhysicalProperties getRequirePhysicalProperties() { columnIdx.add(i); } } + // mapping partition id List exprIds = columnIdx.stream() .map(idx -> child().getOutput().get(idx).getExprId()) .collect(Collectors.toList()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV3Type.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV3Type.java index f87f95db4a3b0f..aaef3775b34360 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV3Type.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV3Type.java @@ -42,7 +42,7 @@ public class DecimalV3Type extends FractionalType { public static final int MAX_DECIMAL256_PRECISION = 76; public static final DecimalV3Type WILDCARD = new DecimalV3Type(-1, -1); - public static final DecimalV3Type SYSTEM_DEFAULT = new DecimalV3Type(MAX_DECIMAL128_PRECISION, DEFAULT_SCALE); + public static final DecimalV3Type SYSTEM_DEFAULT = new DecimalV3Type(MAX_DECIMAL128_PRECISION, 9); public static final DecimalV3Type CATALOG_DEFAULT = new DecimalV3Type(MAX_DECIMAL32_PRECISION, DEFAULT_SCALE); private static final DecimalV3Type BOOLEAN_DECIMAL = new DecimalV3Type(1, 0); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index ffae17c6b1c6c9..94e789bfd02bfd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -806,7 +806,7 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { break; } case OperationType.OP_DYNAMIC_PARTITION: - case OperationType.OP_MODIFY_IN_MEMORY: + case OperationType.OP_MODIFY_TABLE_PROPERTIES: case OperationType.OP_UPDATE_BINLOG_CONFIG: case OperationType.OP_MODIFY_REPLICATION_NUM: { ModifyTablePropertyOperationLog log = (ModifyTablePropertyOperationLog) journal.getData(); @@ -1200,6 +1200,12 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { // TODO: implement this while statistics finished related work. break; } + case OperationType.OP_UPDATE_CLOUD_REPLICA: + case OperationType.OP_MODIFY_TTL_SECONDS: + case OperationType.OP_MODIFY_CLOUD_WARM_UP_JOB: { + // TODO: support cloud replated operation type. + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); @@ -1825,8 +1831,8 @@ public void logModifyDefaultDistributionBucketNum(ModifyTableDefaultDistribution logEdit(OperationType.OP_MODIFY_DISTRIBUTION_BUCKET_NUM, info); } - public long logModifyInMemory(ModifyTablePropertyOperationLog info) { - return logModifyTableProperty(OperationType.OP_MODIFY_IN_MEMORY, info); + public long logModifyTableProperties(ModifyTablePropertyOperationLog info) { + return logModifyTableProperty(OperationType.OP_MODIFY_TABLE_PROPERTIES, info); } public long logUpdateBinlogConfig(ModifyTablePropertyOperationLog info) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java index f5c71e58b72e63..8116bec8da7573 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java @@ -225,7 +225,7 @@ public class OperationType { // modify table properties: inMemory, StoragePolicy, IsBeingSynced, CompactionPolicy, // TimeSeriesCompactionFileCountThreshold, SeriesCompactionTimeThresholdSeconds, // SkipWriteIndexOnLoad, EnableSingleReplicaCompaction. - public static final short OP_MODIFY_IN_MEMORY = 267; + public static final short OP_MODIFY_TABLE_PROPERTIES = 267; // set table default distribution bucket num public static final short OP_MODIFY_DISTRIBUTION_BUCKET_NUM = 268; @@ -384,6 +384,11 @@ public class OperationType { public static final short OP_ALTER_ROLE = 475; + // For cloud. + public static final short OP_UPDATE_CLOUD_REPLICA = 1000; + public static final short OP_MODIFY_TTL_SECONDS = 1001; + public static final short OP_MODIFY_CLOUD_WARM_UP_JOB = 1002; + /** * Get opcode name by op code. **/ diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaReader.java b/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaReader.java index 8024105fe26211..8eb913d7f69b1f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaReader.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaReader.java @@ -100,6 +100,15 @@ public static void read(File imageFile, Env env) throws IOException, DdlExceptio LOG.info("Skip {} module since empty meta length in the end.", metaIndex.name); continue; } + // FIXME: pick cloudWarmUpJob and remove below codes. + if (metaIndex.name.equals("cloudWarmUpJob")) { + LOG.warn("meta modules {} is not supported yet, ignore and skip it", metaIndex.name); + // If this is the last module, nothing need to do. + if (i < metaFooter.metaIndices.size() - 1) { + IOUtils.skipFully(dis, metaFooter.metaIndices.get(i + 1).offset - metaIndex.offset); + } + continue; + } // skip deprecated modules if (PersistMetaModules.DEPRECATED_MODULE_NAMES.contains(metaIndex.name)) { LOG.warn("meta modules {} is deprecated, ignore and skip it", metaIndex.name); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java index a20e40cdc67585..2023395aa55a17 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java @@ -1070,6 +1070,85 @@ public String getCloudCluster() { return getCloudCluster(true); } + public static class CloudClusterResult { + public enum Comment { + FOUND_BY_DEFAULT_CLUSTER, + DEFAULT_CLUSTER_SET_BUT_NOT_EXIST, + FOUND_BY_FIRST_CLUSTER_WITH_ALIVE_BE, + FOUND_BY_FRIST_CLUSTER_HAS_AUTH, + } + + public String clusterName; + public Comment comment; + + public CloudClusterResult(final String name, Comment c) { + this.clusterName = name; + this.comment = c; + } + + @Override + public String toString() { + return "CloudClusterResult{" + + "clusterName='" + clusterName + '\'' + + ", comment=" + comment + + '}'; + } + } + + + // can't get cluster from context, use the following strategy to obtain the cluster name + // 当用户有多个集群的权限时,会按照如下策略进行拉取: + // 如果当前mysql用户没有指定cluster(没有default 或者 use), 选择有权限的cluster。 + // 如果有多个cluster满足权限条件,优先选活的,按字母序选 + // 如果没有活的,则拉起一个,按字母序选 + public CloudClusterResult getCloudClusterByPolicy() { + List cloudClusterNames = ((CloudSystemInfoService) Env.getCurrentSystemInfo()).getCloudClusterNames(); + // try set default cluster + String defaultCloudCluster = Env.getCurrentEnv().getAuth().getDefaultCloudCluster(getQualifiedUser()); + if (!Strings.isNullOrEmpty(defaultCloudCluster)) { + // check cluster validity + CloudClusterResult r; + if (cloudClusterNames.contains(defaultCloudCluster)) { + // valid + r = new CloudClusterResult(defaultCloudCluster, + CloudClusterResult.Comment.FOUND_BY_DEFAULT_CLUSTER); + LOG.info("use default cluster {}", defaultCloudCluster); + } else { + // invalid + r = new CloudClusterResult(defaultCloudCluster, + CloudClusterResult.Comment.DEFAULT_CLUSTER_SET_BUT_NOT_EXIST); + LOG.warn("default cluster {} current invalid, please change it", r); + } + return r; + } + + List hasAuthCluster = new ArrayList<>(); + // get all available cluster of the user + for (String cloudClusterName : cloudClusterNames) { + if (Env.getCurrentEnv().getAuth().checkCloudPriv(getCurrentUserIdentity(), + cloudClusterName, PrivPredicate.USAGE, ResourceTypeEnum.CLUSTER)) { + hasAuthCluster.add(cloudClusterName); + // find a cluster has more than one alive be + List bes = ((CloudSystemInfoService) Env.getCurrentSystemInfo()) + .getBackendsByClusterName(cloudClusterName); + AtomicBoolean hasAliveBe = new AtomicBoolean(false); + bes.stream().filter(Backend::isAlive).findAny().ifPresent(backend -> { + LOG.debug("get a clusterName {}, it's has more than one alive be {}", cloudCluster, backend); + hasAliveBe.set(true); + }); + if (hasAliveBe.get()) { + // set a cluster to context cloudCluster + CloudClusterResult r = new CloudClusterResult(cloudClusterName, + CloudClusterResult.Comment.FOUND_BY_FIRST_CLUSTER_WITH_ALIVE_BE); + LOG.debug("set context {}", r); + return r; + } + } + } + return hasAuthCluster.isEmpty() ? null + : new CloudClusterResult(hasAuthCluster.get(0), CloudClusterResult.Comment.FOUND_BY_FRIST_CLUSTER_HAS_AUTH); + } + /** * @param updateErr whether set this connect state to error when the returned cluster is null or empty. * diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java index 0f8f887ee901f0..b6278d0b55f7fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -29,6 +29,8 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; import org.apache.doris.cloud.catalog.CloudEnv; +import org.apache.doris.cloud.proto.Cloud; +import org.apache.doris.cloud.system.CloudSystemInfoService; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; @@ -193,6 +195,15 @@ protected void auditAfterExec(String origStmt, StatementBase parsedStmt, // only throw an exception when there is a problem interacting with the requesting client protected void handleQuery(MysqlCommand mysqlCommand, String originStmt) { + if (Config.isCloudMode()) { + if (!ctx.getCurrentUserIdentity().isRootUser() + && ((CloudSystemInfoService) Env.getCurrentSystemInfo()).getInstanceStatus() + == Cloud.InstanceInfoPB.Status.OVERDUE) { + Exception exception = new Exception("warehouse is overdue!"); + handleQueryException(exception, originStmt, null, null); + return; + } + } try { executeQuery(mysqlCommand, originStmt); } catch (Exception ignored) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index 53ff62edb684fc..61beeb57ce698f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -171,8 +171,6 @@ import org.apache.doris.common.proc.TabletsProcDir; import org.apache.doris.common.proc.TrashProcDir; import org.apache.doris.common.proc.TrashProcNode; -import org.apache.doris.common.profile.ProfileTreeNode; -import org.apache.doris.common.profile.ProfileTreePrinter; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.ListComparator; import org.apache.doris.common.util.LogBuilder; @@ -180,8 +178,6 @@ import org.apache.doris.common.util.NetUtils; import org.apache.doris.common.util.OrderByPair; import org.apache.doris.common.util.PrintableMap; -import org.apache.doris.common.util.ProfileManager; -import org.apache.doris.common.util.RuntimeProfile; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.CatalogIf; @@ -224,7 +220,6 @@ import org.apache.doris.thrift.TShowProcessListRequest; import org.apache.doris.thrift.TShowProcessListResult; import org.apache.doris.thrift.TTaskType; -import org.apache.doris.thrift.TUnit; import org.apache.doris.transaction.GlobalTransactionMgrIface; import org.apache.doris.transaction.TransactionStatus; @@ -235,7 +230,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.tuple.Triple; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -2390,116 +2384,21 @@ private void handleShowPlugins() throws AnalysisException { } private void handleShowQueryProfile() throws AnalysisException { - ShowQueryProfileStmt showStmt = (ShowQueryProfileStmt) stmt; - ShowQueryProfileStmt.PathType pathType = showStmt.getPathType(); - List> rows = Lists.newArrayList(); - switch (pathType) { - case QUERY_IDS: - rows = ProfileManager.getInstance().getQueryWithType(ProfileManager.ProfileType.QUERY); - break; - case FRAGMENTS: { - ProfileTreeNode treeRoot = ProfileManager.getInstance().getFragmentProfileTree(showStmt.getQueryId(), - showStmt.getQueryId()); - if (treeRoot == null) { - throw new AnalysisException("Failed to get fragment tree for query: " + showStmt.getQueryId()); - } - List row = Lists.newArrayList(ProfileTreePrinter.printFragmentTree(treeRoot)); - rows.add(row); - break; - } - case INSTANCES: { - // For query profile, there should be only one execution profile, - // And the execution id is same as query id - List> instanceList - = ProfileManager.getInstance().getFragmentInstanceList( - showStmt.getQueryId(), showStmt.getQueryId(), showStmt.getFragmentId()); - if (instanceList == null) { - throw new AnalysisException("Failed to get instance list for fragment: " - + showStmt.getFragmentId()); - } - for (Triple triple : instanceList) { - List row = Lists.newArrayList(triple.getLeft(), triple.getMiddle(), - RuntimeProfile.printCounter(triple.getRight(), TUnit.TIME_NS)); - rows.add(row); - } - break; - } - case SINGLE_INSTANCE: { - // For query profile, there should be only one execution profile, - // And the execution id is same as query id - ProfileTreeNode treeRoot = ProfileManager.getInstance().getInstanceProfileTree(showStmt.getQueryId(), - showStmt.getQueryId(), showStmt.getFragmentId(), showStmt.getInstanceId()); - if (treeRoot == null) { - throw new AnalysisException("Failed to get instance tree for instance: " - + showStmt.getInstanceId()); - } - List row = Lists.newArrayList(ProfileTreePrinter.printInstanceTree(treeRoot)); - rows.add(row); - break; - } - default: - break; - } - - resultSet = new ShowResultSet(showStmt.getMetaData(), rows); + String selfHost = Env.getCurrentEnv().getSelfNode().getHost(); + int httpPort = Config.http_port; + String terminalMsg = String.format( + "try visit http://%s:%d/QueryProfile, show query/load profile syntax is a deprecated feature", + selfHost, httpPort); + throw new AnalysisException(terminalMsg); } private void handleShowLoadProfile() throws AnalysisException { - ShowLoadProfileStmt showStmt = (ShowLoadProfileStmt) stmt; - ShowLoadProfileStmt.PathType pathType = showStmt.getPathType(); - List> rows = Lists.newArrayList(); - switch (pathType) { - case QUERY_IDS: - rows = ProfileManager.getInstance().getQueryWithType(ProfileManager.ProfileType.LOAD); - break; - case TASK_IDS: { - rows = ProfileManager.getInstance().getLoadJobTaskList(showStmt.getJobId()); - break; - } - case FRAGMENTS: { - ProfileTreeNode treeRoot = ProfileManager.getInstance().getFragmentProfileTree(showStmt.getJobId(), - showStmt.getTaskId()); - if (treeRoot == null) { - throw new AnalysisException("Failed to get fragment tree for load: " + showStmt.getJobId()); - } - List row = Lists.newArrayList(ProfileTreePrinter.printFragmentTree(treeRoot)); - rows.add(row); - break; - } - case INSTANCES: { - // For load profile, there should be only one fragment in each execution profile - // And the fragment id is 0. - List> instanceList - = ProfileManager.getInstance().getFragmentInstanceList(showStmt.getJobId(), - showStmt.getTaskId(), ((ShowLoadProfileStmt) stmt).getFragmentId()); - if (instanceList == null) { - throw new AnalysisException("Failed to get instance list for task: " + showStmt.getTaskId()); - } - for (Triple triple : instanceList) { - List row = Lists.newArrayList(triple.getLeft(), triple.getMiddle(), - RuntimeProfile.printCounter(triple.getRight(), TUnit.TIME_NS)); - rows.add(row); - } - break; - } - case SINGLE_INSTANCE: { - // For load profile, there should be only one fragment in each execution profile. - // And the fragment id is 0. - ProfileTreeNode treeRoot = ProfileManager.getInstance().getInstanceProfileTree(showStmt.getJobId(), - showStmt.getTaskId(), showStmt.getFragmentId(), showStmt.getInstanceId()); - if (treeRoot == null) { - throw new AnalysisException("Failed to get instance tree for instance: " - + showStmt.getInstanceId()); - } - List row = Lists.newArrayList(ProfileTreePrinter.printInstanceTree(treeRoot)); - rows.add(row); - break; - } - default: - break; - } - - resultSet = new ShowResultSet(showStmt.getMetaData(), rows); + String selfHost = Env.getCurrentEnv().getSelfNode().getHost(); + int httpPort = Config.http_port; + String terminalMsg = String.format( + "try visit http://%s:%d/QueryProfile, show query/load profile syntax is a deprecated feature", + selfHost, httpPort); + throw new AnalysisException(terminalMsg); } private void handleShowCreateRepository() throws AnalysisException { diff --git a/fe/fe-core/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/fe/fe-core/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java index b1e47c494841f2..1064a3e70e6807 100644 --- a/fe/fe-core/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java +++ b/fe/fe-core/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -288,6 +288,9 @@ * 17. renamePartition() * 18. truncateTable() * 19. drop_table_with_environment_context() + * + * ATTN: There is a copy of this file in be-java-extensions. + * If you want to modify this file, please modify the file in be-java-extensions. */ @InterfaceAudience.Public @InterfaceStability.Evolving @@ -352,6 +355,7 @@ public HiveMetaStoreClient(Configuration conf, HiveMetaHookLoader hookLoader, Bo } hiveVersion = HiveVersionUtil.getVersion(conf.get(HMSProperties.HIVE_VERSION)); + LOG.info("Loading Doris HiveMetaStoreClient. Hive version: " + conf.get(HMSProperties.HIVE_VERSION)); // For hive 2.3.7, there is no ClientCapability.INSERT_ONLY_TABLES if (hiveVersion == HiveVersion.V1_0 || hiveVersion == HiveVersion.V2_0 || hiveVersion == HiveVersion.V2_3) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java index b7e90218e10825..8e2d436e2195e5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java @@ -22,10 +22,12 @@ import org.apache.doris.analysis.CreateTableStmt; import org.apache.doris.analysis.HashDistributionDesc; import org.apache.doris.analysis.SwitchStmt; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.common.Config; import org.apache.doris.common.ExceptionChecker; import org.apache.doris.common.FeConstants; +import org.apache.doris.datasource.TableMetadata; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; @@ -36,6 +38,8 @@ import mockit.Mock; import mockit.MockUp; import mockit.Mocked; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Table; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -52,6 +56,8 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService { @Mocked private ThriftHMSCachedClient mockedHiveClient; + private List checkedHiveCols; + @Override protected void runBeforeAll() throws Exception { connectContext.getSessionVariable().enableFallbackToOriginalPlanner = false; @@ -101,6 +107,25 @@ public List getAllDatabases() { } }; } + + @Mock + public void createTable(TableMetadata tbl, boolean ignoreIfExists) { + if (tbl instanceof HiveTableMetadata) { + Table table = HiveUtil.toHiveTable((HiveTableMetadata) tbl); + if (checkedHiveCols == null) { + // if checkedHiveCols is null, skip column check + return; + } + List fieldSchemas = table.getSd().getCols(); + Assertions.assertEquals(checkedHiveCols.size(), fieldSchemas.size()); + for (int i = 0; i < checkedHiveCols.size(); i++) { + FieldSchema checkedCol = checkedHiveCols.get(i); + FieldSchema actualCol = fieldSchemas.get(i); + Assertions.assertEquals(checkedCol.getName(), actualCol.getName().toLowerCase()); + Assertions.assertEquals(checkedCol.getType(), actualCol.getType().toLowerCase()); + } + } + } }; CreateDbStmt createDbStmt = new CreateDbStmt(true, mockedDbName, dbProps); Env.getCurrentEnv().createDb(createDbStmt); @@ -369,4 +394,110 @@ public void testInsertOverwritePlanSql() throws Exception { LogicalPlan plan2 = nereidsParser.parseSingle(insertSql2); Assertions.assertTrue(plan2 instanceof InsertOverwriteTableCommand); } + + @Test + public void testComplexTypeCreateTable() throws Exception { + checkedHiveCols = new ArrayList<>(); // init it to enable check + switchHive(); + useDatabase(mockedDbName); + String createArrayTypeTable = "CREATE TABLE complex_type_array(\n" + + " `col1` ARRAY COMMENT 'col1',\n" + + " `col2` ARRAY COMMENT 'col2',\n" + + " `col3` ARRAY COMMENT 'col3',\n" + + " `col4` ARRAY COMMENT 'col4',\n" + + " `col5` ARRAY COMMENT 'col5'\n" + + ") ENGINE=hive\n" + + "PROPERTIES ('file_format'='orc')"; + List checkArrayCols = new ArrayList<>(); + checkArrayCols.add(new FieldSchema("col1", "array", "")); + checkArrayCols.add(new FieldSchema("col2", "array", "")); + checkArrayCols.add(new FieldSchema("col3", "array", "")); + checkArrayCols.add(new FieldSchema("col4", "array", "")); + checkArrayCols.add(new FieldSchema("col5", "array", "")); + resetCheckedColumns(checkArrayCols); + + LogicalPlan plan = createTablesAndReturnPlans(true, createArrayTypeTable).get(0); + List columns = ((CreateTableCommand) plan).getCreateTableInfo().translateToLegacyStmt().getColumns(); + Assertions.assertEquals(5, columns.size()); + dropTable("complex_type_array", true); + + String createMapTypeTable = "CREATE TABLE complex_type_map(\n" + + " `col1` MAP COMMENT 'col1',\n" + + " `col2` MAP COMMENT 'col2',\n" + + " `col3` MAP COMMENT 'col3',\n" + + " `col4` MAP COMMENT 'col4'\n" + + ") ENGINE=hive\n" + + "PROPERTIES ('file_format'='orc')"; + checkArrayCols = new ArrayList<>(); + checkArrayCols.add(new FieldSchema("col1", "map", "")); + checkArrayCols.add(new FieldSchema("col2", "map", "")); + checkArrayCols.add(new FieldSchema("col3", "map", "")); + checkArrayCols.add(new FieldSchema("col4", "map", "")); + resetCheckedColumns(checkArrayCols); + + plan = createTablesAndReturnPlans(true, createMapTypeTable).get(0); + columns = ((CreateTableCommand) plan).getCreateTableInfo().translateToLegacyStmt().getColumns(); + Assertions.assertEquals(4, columns.size()); + dropTable("complex_type_map", true); + + String createStructTypeTable = "CREATE TABLE complex_type_struct(\n" + + " `col1` STRUCT,name:string> COMMENT 'col1',\n" + + " `col2` STRUCT COMMENT 'col2',\n" + + " `col3` STRUCT COMMENT 'col3',\n" + + " `col4` STRUCT> COMMENT 'col4'\n" + + ") ENGINE=hive\n" + + "PROPERTIES ('file_format'='orc')"; + checkArrayCols = new ArrayList<>(); + checkArrayCols.add(new FieldSchema("col1", "struct,name:string>", "")); + checkArrayCols.add(new FieldSchema("col2", "struct", "")); + checkArrayCols.add(new FieldSchema("col3", "struct", "")); + checkArrayCols.add(new FieldSchema("col4", "struct>", "")); + resetCheckedColumns(checkArrayCols); + + plan = createTablesAndReturnPlans(true, createStructTypeTable).get(0); + columns = ((CreateTableCommand) plan).getCreateTableInfo().translateToLegacyStmt().getColumns(); + Assertions.assertEquals(4, columns.size()); + dropTable("complex_type_struct", true); + + String compoundTypeTable1 = "CREATE TABLE complex_type_compound1(\n" + + " `col1` ARRAY> COMMENT 'col1',\n" + + " `col2` ARRAY> COMMENT 'col2'\n" + + ") ENGINE=hive\n" + + "PROPERTIES ('file_format'='orc')"; + checkArrayCols = new ArrayList<>(); + checkArrayCols.add(new FieldSchema("col1", "array>", "")); + checkArrayCols.add(new FieldSchema("col2", + "array>", "")); + resetCheckedColumns(checkArrayCols); + + plan = createTablesAndReturnPlans(true, compoundTypeTable1).get(0); + columns = ((CreateTableCommand) plan).getCreateTableInfo().translateToLegacyStmt().getColumns(); + Assertions.assertEquals(2, columns.size()); + dropTable("complex_type_compound1", true); + + String compoundTypeTable2 = "CREATE TABLE complex_type_compound2(\n" + + " `col1` MAP> COMMENT 'col1',\n" + + " `col2` MAP>> COMMENT 'col2',\n" + + " `col3` MAP> COMMENT 'col3',\n" + + " `col4` MAP> COMMENT 'col4'\n" + + ") ENGINE=hive\n" + + "PROPERTIES ('file_format'='orc')"; + checkArrayCols = new ArrayList<>(); + checkArrayCols.add(new FieldSchema("col1", "map>", "")); + checkArrayCols.add(new FieldSchema("col2", "map>>", "")); + checkArrayCols.add(new FieldSchema("col3", "map>", "")); + checkArrayCols.add(new FieldSchema("col4", + "map>", "")); + resetCheckedColumns(checkArrayCols); + + plan = createTablesAndReturnPlans(true, compoundTypeTable2).get(0); + columns = ((CreateTableCommand) plan).getCreateTableInfo().translateToLegacyStmt().getColumns(); + Assertions.assertEquals(4, columns.size()); + dropTable("complex_type_compound2", true); + } + + private void resetCheckedColumns(List checkArrayCols) { + checkedHiveCols.clear(); + checkedHiveCols.addAll(checkArrayCols); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java index 4ea50bf1f8817c..174592270dd973 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java @@ -87,7 +87,7 @@ void testSimplifyArithmeticRuleOnly() { assertRewriteAfterTypeCoercion("(-IA / 2) / ((-IB - 1) / (3 + (IC * 4)))", "(((cast((0 - IA) as DOUBLE) / cast(((0 - IB) - 1) as DOUBLE)) * cast((3 + (IC * 4)) as DOUBLE)) / cast(2 as DOUBLE))"); // unsupported decimal - assertRewriteAfterTypeCoercion("-2 - MA - ((1 - IB) - (3 + IC))", "((cast(-2 as DECIMALV3(38, 0)) - MA) - cast(((1 - IB) - (3 + IC)) as DECIMALV3(38, 0)))"); + assertRewriteAfterTypeCoercion("-2 - MA - ((1 - IB) - (3 + IC))", "((cast(-2 as DECIMALV3(38, 9)) - MA) - cast(((1 - IB) - (3 + IC)) as DECIMALV3(38, 9)))"); assertRewriteAfterTypeCoercion("-IA / 2.0 * ((-IB - 1) - (3 + (IC + 4)))", "((cast((0 - IA) as DECIMALV3(25, 5)) / 2.0) * cast((((0 - IB) - 1) - (3 + (IC + 4))) as DECIMALV3(20, 0)))"); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java index a30d67d4d314e4..d9a2946ba883c5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java @@ -284,7 +284,7 @@ public void testFindCommonPrimitiveTypeForCaseWhen() { testFindCommonPrimitiveTypeForCaseWhen(LargeIntType.INSTANCE, LargeIntType.INSTANCE, BigIntType.INSTANCE); testFindCommonPrimitiveTypeForCaseWhen(LargeIntType.INSTANCE, LargeIntType.INSTANCE, LargeIntType.INSTANCE); testFindCommonPrimitiveTypeForCaseWhen(DecimalV2Type.SYSTEM_DEFAULT, LargeIntType.INSTANCE, DecimalV2Type.SYSTEM_DEFAULT); - testFindCommonPrimitiveTypeForCaseWhen(DecimalV3Type.SYSTEM_DEFAULT, LargeIntType.INSTANCE, DecimalV3Type.SYSTEM_DEFAULT); + testFindCommonPrimitiveTypeForCaseWhen(DecimalV3Type.createDecimalV3Type(38), LargeIntType.INSTANCE, DecimalV3Type.createDecimalV3Type(38)); testFindCommonPrimitiveTypeForCaseWhen(DoubleType.INSTANCE, LargeIntType.INSTANCE, FloatType.INSTANCE); testFindCommonPrimitiveTypeForCaseWhen(DoubleType.INSTANCE, LargeIntType.INSTANCE, DoubleType.INSTANCE); testFindCommonPrimitiveTypeForCaseWhen(StringType.INSTANCE, LargeIntType.INSTANCE, CharType.SYSTEM_DEFAULT); @@ -308,7 +308,7 @@ public void testFindCommonPrimitiveTypeForCaseWhen() { testFindCommonPrimitiveTypeForCaseWhen(DecimalV2Type.SYSTEM_DEFAULT, DecimalV2Type.SYSTEM_DEFAULT, LargeIntType.INSTANCE); testFindCommonPrimitiveTypeForCaseWhen(DecimalV2Type.SYSTEM_DEFAULT, DecimalV2Type.SYSTEM_DEFAULT, DecimalV2Type.SYSTEM_DEFAULT); - testFindCommonPrimitiveTypeForCaseWhen(DoubleType.INSTANCE, DecimalV2Type.SYSTEM_DEFAULT, + testFindCommonPrimitiveTypeForCaseWhen(DecimalV3Type.SYSTEM_DEFAULT, DecimalV2Type.SYSTEM_DEFAULT, DecimalV3Type.SYSTEM_DEFAULT); testFindCommonPrimitiveTypeForCaseWhen(DoubleType.INSTANCE, DecimalV2Type.SYSTEM_DEFAULT, FloatType.INSTANCE); testFindCommonPrimitiveTypeForCaseWhen(DoubleType.INSTANCE, DecimalV2Type.SYSTEM_DEFAULT, DoubleType.INSTANCE); @@ -331,10 +331,10 @@ public void testFindCommonPrimitiveTypeForCaseWhen() { testFindCommonPrimitiveTypeForCaseWhen(DecimalV3Type.SYSTEM_DEFAULT, DecimalV3Type.SYSTEM_DEFAULT, SmallIntType.INSTANCE); testFindCommonPrimitiveTypeForCaseWhen(DecimalV3Type.SYSTEM_DEFAULT, DecimalV3Type.SYSTEM_DEFAULT, IntegerType.INSTANCE); testFindCommonPrimitiveTypeForCaseWhen(DecimalV3Type.SYSTEM_DEFAULT, DecimalV3Type.SYSTEM_DEFAULT, BigIntType.INSTANCE); - testFindCommonPrimitiveTypeForCaseWhen(DecimalV3Type.SYSTEM_DEFAULT, DecimalV3Type.SYSTEM_DEFAULT, LargeIntType.INSTANCE); + testFindCommonPrimitiveTypeForCaseWhen(DecimalV3Type.createDecimalV3Type(38), DecimalV3Type.createDecimalV3Type(38), LargeIntType.INSTANCE); testFindCommonPrimitiveTypeForCaseWhen(DecimalV3Type.SYSTEM_DEFAULT, DecimalV3Type.SYSTEM_DEFAULT, DecimalV2Type.createDecimalV2Type(27, 0)); - testFindCommonPrimitiveTypeForCaseWhen(DoubleType.INSTANCE, DecimalV3Type.SYSTEM_DEFAULT, + testFindCommonPrimitiveTypeForCaseWhen(DecimalV3Type.SYSTEM_DEFAULT, DecimalV3Type.SYSTEM_DEFAULT, DecimalV2Type.SYSTEM_DEFAULT); testFindCommonPrimitiveTypeForCaseWhen(DecimalV3Type.SYSTEM_DEFAULT, DecimalV3Type.SYSTEM_DEFAULT, DecimalV3Type.SYSTEM_DEFAULT); diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/ListPartitionPrunerV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/planner/ListPartitionPrunerV2Test.java new file mode 100644 index 00000000000000..0a00a94d597d9a --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/ListPartitionPrunerV2Test.java @@ -0,0 +1,136 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.planner; + +import org.apache.doris.analysis.PartitionValue; +import org.apache.doris.catalog.ListPartitionItem; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionKey; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.catalog.Type; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ThreadPoolManager; +import org.apache.doris.datasource.hive.HMSCachedClient; +import org.apache.doris.datasource.hive.HMSExternalCatalog; +import org.apache.doris.datasource.hive.HiveMetaStoreCache; +import org.apache.doris.datasource.hive.ThriftHMSCachedClient; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import mockit.Mock; +import mockit.MockUp; +import org.apache.hadoop.hive.conf.HiveConf; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadPoolExecutor; + +public class ListPartitionPrunerV2Test { + @Test + public void testPartitionValuesMap() throws AnalysisException { + List partitionValues = new ArrayList<>(); + partitionValues.add(new PartitionValue("1.123000")); + ArrayList types = new ArrayList<>(); + types.add(ScalarType.DOUBLE); + + // for hive table + PartitionKey key = PartitionKey.createListPartitionKeyWithTypes(partitionValues, types, true); + ListPartitionItem listPartitionItem = new ListPartitionItem(Lists.newArrayList(key)); + Map idToPartitionItem = Maps.newHashMapWithExpectedSize(partitionValues.size()); + idToPartitionItem.put(1L, listPartitionItem); + + // for olap table + PartitionKey key2 = PartitionKey.createListPartitionKeyWithTypes(partitionValues, types, false); + ListPartitionItem listPartitionItem2 = new ListPartitionItem(Lists.newArrayList(key2)); + idToPartitionItem.put(2L, listPartitionItem2); + + Map> partitionValuesMap = ListPartitionPrunerV2.getPartitionValuesMap(idToPartitionItem); + Assert.assertEquals("1.123000", partitionValuesMap.get(1L).get(0)); + Assert.assertEquals("1.123", partitionValuesMap.get(2L).get(0)); + } + + @Test + public void testInvalidateTable() { + + new MockUp(HMSExternalCatalog.class) { + @Mock + public HMSCachedClient getClient() { + return new ThriftHMSCachedClient(new HiveConf(), 2); + } + }; + + new MockUp(ThriftHMSCachedClient.class) { + @Mock + public List listPartitionNames(String dbName, String tblName) { + // Mock is used here to represent the existence of a partition in the original table + return new ArrayList() {{ + add("c1=1.234000"); + }}; + } + }; + + ThreadPoolExecutor executor = ThreadPoolManager.newDaemonFixedThreadPool( + 20, 20, "mgr", 120, false); + HiveMetaStoreCache cache = new HiveMetaStoreCache( + new HMSExternalCatalog(1L, "catalog", null, new HashMap<>(), null), executor); + ArrayList types = new ArrayList<>(); + types.add(ScalarType.DOUBLE); + + // test cache + // the original partition of the table (in mock) will be loaded here + String dbName = "db"; + String tblName = "tb"; + HiveMetaStoreCache.HivePartitionValues partitionValues = cache.getPartitionValues(dbName, tblName, types); + Assert.assertEquals(1, partitionValues.getIdToPartitionItem().size()); + Assert.assertTrue(partitionValues.getIdToPartitionItem().containsKey(0L)); + List items = partitionValues.getIdToPartitionItem().get(0L).getItems(); + Assert.assertEquals(1, items.size()); + PartitionKey partitionKey = items.get(0); + Assert.assertEquals("1.234", partitionKey.getKeys().get(0).toString()); + Assert.assertEquals("1.234000", partitionKey.getOriginHiveKeys().get(0)); + + // test add cache + ArrayList values = new ArrayList<>(); + values.add("c1=5.678000"); + cache.addPartitionsCache(dbName, tblName, values, types); + HiveMetaStoreCache.HivePartitionValues partitionValues2 = cache.getPartitionValues(dbName, tblName, types); + Assert.assertEquals(2, partitionValues2.getIdToPartitionItem().size()); + Assert.assertTrue(partitionValues2.getIdToPartitionItem().containsKey(1L)); + List items2 = partitionValues2.getIdToPartitionItem().get(1L).getItems(); + Assert.assertEquals(1, items2.size()); + PartitionKey partitionKey2 = items2.get(0); + Assert.assertEquals("5.678", partitionKey2.getKeys().get(0).toString()); + Assert.assertEquals("5.678000", partitionKey2.getOriginHiveKeys().get(0)); + + // test refresh table + // simulates the manually added partition table being deleted, leaving only one original partition in mock + cache.invalidateTableCache(dbName, tblName); + HiveMetaStoreCache.HivePartitionValues partitionValues3 = cache.getPartitionValues(dbName, tblName, types); + Assert.assertEquals(1, partitionValues3.getIdToPartitionItem().size()); + Assert.assertTrue(partitionValues3.getIdToPartitionItem().containsKey(0L)); + List items3 = partitionValues3.getIdToPartitionItem().get(0L).getItems(); + Assert.assertEquals(1, items3.size()); + PartitionKey partitionKey3 = items3.get(0); + Assert.assertEquals("1.234", partitionKey3.getKeys().get(0).toString()); + Assert.assertEquals("1.234000", partitionKey3.getOriginHiveKeys().get(0)); + } +} diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index e91bc5ec4a5d5f..2db7dd1a5ef557 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -258,6 +258,7 @@ message TxnCoordinatorPB { message RoutineLoadProgressPB { map partition_to_offset = 1; + optional RoutineLoadJobStatisticPB stat = 2; } message RLTaskTxnCommitAttachmentPB { @@ -272,6 +273,14 @@ message RLTaskTxnCommitAttachmentPB { optional string error_log_url = 9; } +message RoutineLoadJobStatisticPB { + optional int64 filtered_rows = 1; + optional int64 loaded_rows = 2; + optional int64 unselected_rows = 3; + optional int64 received_bytes = 4; + optional int64 task_execution_time_ms = 5; +} + message TxnCommitAttachmentPB { enum Type { LODD_JOB_FINAL_OPERATION = 0; @@ -1205,6 +1214,7 @@ enum MetaServiceCode { JOB_TABLET_BUSY = 5001; JOB_ALREADY_SUCCESS = 5002; ROUTINE_LOAD_DATA_INCONSISTENT = 5003; + ROUTINE_LOAD_PROGRESS_NOT_FOUND = 5004; // Rate limit MAX_QPS_LIMIT = 6001; @@ -1220,6 +1230,9 @@ enum MetaServiceCode { // partial update ROWSET_META_NOT_FOUND = 9001; + // The meta service retries KV_TXN_CONFLICT error but is exceeded the max times. + KV_TXN_CONFLICT_RETRY_EXCEEDED_MAX_TIMES = 10001; + UNDEFINED_ERR = 1000000; } @@ -1285,6 +1298,17 @@ message GetDeleteBitmapUpdateLockResponse { optional MetaServiceResponseStatus status = 1; } +message GetRLTaskCommitAttachRequest { + optional string cloud_unique_id = 1; // For auth + optional int64 db_id = 2; + optional int64 job_id = 3; +} + +message GetRLTaskCommitAttachResponse { + optional MetaServiceResponseStatus status = 1; + optional RLTaskTxnCommitAttachmentPB commit_attach = 2; +} + service MetaService { rpc begin_txn(BeginTxnRequest) returns (BeginTxnResponse); rpc precommit_txn(PrecommitTxnRequest) returns (PrecommitTxnResponse); @@ -1347,6 +1371,9 @@ service MetaService { rpc update_delete_bitmap(UpdateDeleteBitmapRequest) returns(UpdateDeleteBitmapResponse); rpc get_delete_bitmap(GetDeleteBitmapRequest) returns(GetDeleteBitmapResponse); rpc get_delete_bitmap_update_lock(GetDeleteBitmapUpdateLockRequest) returns(GetDeleteBitmapUpdateLockResponse); + + // routine load progress + rpc get_rl_task_commit_attach(GetRLTaskCommitAttachRequest) returns (GetRLTaskCommitAttachResponse); }; service RecyclerService { diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index 148db2b9a1709b..2fadcdae538795 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -154,6 +154,8 @@ struct THdfsParams { 3: optional string hdfs_kerberos_principal 4: optional string hdfs_kerberos_keytab 5: optional list hdfs_conf + // Used for Cold Heat Separation to specify the root path + 6: optional string root_path } // One broker range information. diff --git a/regression-test/conf/regression-conf.groovy b/regression-test/conf/regression-conf.groovy index c6764a2ce86373..041744c9b02880 100644 --- a/regression-test/conf/regression-conf.groovy +++ b/regression-test/conf/regression-conf.groovy @@ -86,7 +86,10 @@ excludeGroups = "" // this suites will not be executed excludeSuites = "test_broker_load" // this directories will not be executed -excludeDirectories = "segcompaction_p2,workload_manager_p1" +excludeDirectories = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line + "segcompaction_p2," + + "workload_manager_p1," + + "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line customConf1 = "test_custom_conf_value" diff --git a/regression-test/data/external_table_p0/hive/write/test_hive_write_insert.out b/regression-test/data/external_table_p0/hive/write/test_hive_write_insert.out new file mode 100644 index 00000000000000..7b5c0e7d7bcd9b --- /dev/null +++ b/regression-test/data/external_table_p0/hive/write/test_hive_write_insert.out @@ -0,0 +1,167 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q01 -- +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q04 -- +false -7 -15 16 -9223372036854775808 -123.45 -123456.789 123456789 -1234.5678 -123456.789012 -123456789.012345678901 str binary_value 2024-03-25 2024-03-25T12:00 2024-03-25T12:00:00.123457 2024-03-25T12:00:00.123457 char_value11111 char_value22222 char_value33333 varchar_value11111 varchar_value22222 varchar_value33333 {"key7":"value1"} {"key7":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {5.3456:2.3456} {5.34567890:2.34567890} {2.34567890:2.34567890} {7.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [9.4567, 4.5678] [6.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240325 + +-- !q05 -- +\N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240321 + +-- !q06 -- + +-- !q01 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q04 -- +\N \N \N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N + +-- !q05 -- + +-- !q01 -- +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q04 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +false -7 -15 16 -9223372036854775808 -123.45 -123456.789 123456789 -1234.5678 -123456.789012 -123456789.012345678901 str binary_value 2024-03-25 2024-03-25T12:00 2024-03-25T12:00:00.123457 2024-03-25T12:00:00.123457 char_value11111 char_value22222 char_value33333 varchar_value11111 varchar_value22222 varchar_value33333 {"key7":"value1"} {"key7":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {5.3456:2.3456} {5.34567890:2.34567890} {2.34567890:2.34567890} {7.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [9.4567, 4.5678] [6.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240321 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q01 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q01 -- +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q04 -- +false -7 -15 16 -9223372036854775808 -123.45 -123456.789 123456789 -1234.5678 -123456.789012 -123456789.012345678901 str binary_value 2024-03-25 2024-03-25T12:00 2024-03-25T12:00:00.123457 2024-03-25T12:00:00.123457 char_value11111 char_value22222 char_value33333 varchar_value11111 varchar_value22222 varchar_value33333 {"key7":"value1"} {"key7":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {5.3456:2.3456} {5.34567890:2.34567890} {2.34567890:2.34567890} {7.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [9.4567, 4.5678] [6.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240325 + +-- !q05 -- +\N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240321 + +-- !q06 -- + +-- !q01 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q04 -- +\N \N \N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N + +-- !q05 -- + +-- !q01 -- +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q04 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +false -7 -15 16 -9223372036854775808 -123.45 -123456.789 123456789 -1234.5678 -123456.789012 -123456789.012345678901 str binary_value 2024-03-25 2024-03-25T12:00 2024-03-25T12:00:00.123457 2024-03-25T12:00:00.123457 char_value11111 char_value22222 char_value33333 varchar_value11111 varchar_value22222 varchar_value33333 {"key7":"value1"} {"key7":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {5.3456:2.3456} {5.34567890:2.34567890} {2.34567890:2.34567890} {7.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [9.4567, 4.5678] [6.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240321 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q01 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + diff --git a/regression-test/data/external_table_p0/hive/write/test_hive_write_partitions.out b/regression-test/data/external_table_p0/hive/write/test_hive_write_partitions.out new file mode 100644 index 00000000000000..2c0202874ceac3 --- /dev/null +++ b/regression-test/data/external_table_p0/hive/write/test_hive_write_partitions.out @@ -0,0 +1,129 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q01 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q01 -- +1 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 + +-- !q02 -- +1 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +2 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 + +-- !q03 -- +1 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +2 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +3 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 + +-- !q04 -- +1 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +2 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +3 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 +3 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 + +-- !q05 -- +1 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +2 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +7 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 + +-- !q01 -- +1 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-22 2024-03-22T12:00 char_value1 varchar_value1 + +-- !q02 -- +1 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-22 2024-03-22T12:00 char_value1 varchar_value1 +2 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-21 2024-03-21T12:00 char_value1 varchar_value1 + +-- !q03 -- +1 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-22 2024-03-22T12:00 char_value1 varchar_value1 +2 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-21 2024-03-21T12:00 char_value1 varchar_value1 +3 123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-20 2024-03-20T12:00 char_value1 varchar_value1 + +-- !q04 -- +1 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-22 2024-03-22T12:00 char_value1 varchar_value1 +2 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-21 2024-03-21T12:00 char_value1 varchar_value1 +3 123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-20 2024-03-20T12:00 char_value1 varchar_value1 +3 123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-20 2024-03-20T12:00 char_value1 varchar_value1 + +-- !q05 -- +1 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-22 2024-03-22T12:00 char_value1 varchar_value1 +2 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-21 2024-03-21T12:00 char_value1 varchar_value1 +7 123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-20 2024-03-20T12:00 char_value1 varchar_value1 + +-- !q01 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value binary_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint": -1234567890} {"key":[{"s_int": -123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": -123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": -123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": -123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint": 1234567890} {"key":[{"s_int": 123}]} {"struct_field": ["value1", "value2"]} {"struct_field_null": null, "struct_field_null2": null} {"struct_non_nulls_after_nulls1": 123, "struct_non_nulls_after_nulls2": "value"} {"struct_field1": 123, "struct_field2": "value", "strict_field3": {"nested_struct_field1": 123, "nested_struct_field2": "nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q01 -- +1 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 + +-- !q02 -- +1 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +2 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 + +-- !q03 -- +1 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +2 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +3 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 + +-- !q04 -- +1 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +2 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +3 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 +3 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 + +-- !q05 -- +1 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +2 false -128 -32768 -2147483648 -9223372036854775808 -123.45 -123456.789 +7 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 + +-- !q01 -- +1 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-22 2024-03-22T12:00 char_value1 varchar_value1 + +-- !q02 -- +1 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-22 2024-03-22T12:00 char_value1 varchar_value1 +2 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-21 2024-03-21T12:00 char_value1 varchar_value1 + +-- !q03 -- +1 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-22 2024-03-22T12:00 char_value1 varchar_value1 +2 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-21 2024-03-21T12:00 char_value1 varchar_value1 +3 123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-20 2024-03-20T12:00 char_value1 varchar_value1 + +-- !q04 -- +1 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-22 2024-03-22T12:00 char_value1 varchar_value1 +2 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-21 2024-03-21T12:00 char_value1 varchar_value1 +3 123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-20 2024-03-20T12:00 char_value1 varchar_value1 +3 123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-20 2024-03-20T12:00 char_value1 varchar_value1 + +-- !q05 -- +1 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-22 2024-03-22T12:00 char_value1 varchar_value1 +2 -123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-21 2024-03-21T12:00 char_value1 varchar_value1 +7 123456.789012 string_value 62 69 6e 61 72 79 5f 76 61 6c 75 65 2024-03-20 2024-03-20T12:00 char_value1 varchar_value1 + diff --git a/regression-test/data/external_table_p0/jdbc/test_switch_catalog_and_delete_internal.out b/regression-test/data/external_table_p0/jdbc/test_switch_catalog_and_delete_internal.out new file mode 100644 index 00000000000000..cda9c9293b3716 --- /dev/null +++ b/regression-test/data/external_table_p0/jdbc/test_switch_catalog_and_delete_internal.out @@ -0,0 +1,2 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !test -- diff --git a/regression-test/data/external_table_p2/hive/test_hive_hudi.out b/regression-test/data/external_table_p2/hive/test_hive_hudi.out index d217a289d8d74b..3edb91d0b06fde 100644 --- a/regression-test/data/external_table_p2/hive/test_hive_hudi.out +++ b/regression-test/data/external_table_p2/hive/test_hive_hudi.out @@ -136,16 +136,16 @@ row_4 2021-02-01 4 v_4 20240221111000868 20240221111000868_0_4 e96c4396-3fad-413a-a942-4cb36106d721 san_francisco 3efcaa94-3e58-436a-b489-1232731ed088 1695091554788 e96c4396-3fad-413a-a942-4cb36106d721 rider-C driver-M 27.7 san_francisco -- !incr_mor_table -- +20240312163613712 20240312163613712_0_1 20240312163541346_0_0 pid=p1 ead436a2-f99b-4d35-8b3e-9d67e4828dd5-0_0-99-636_20240312163738946.parquet 1 k1 1.134 p1 20240312163737521 20240312163737521_0_1 20240312163737521_1_0 pid=p2 92e677af-6487-4213-b42f-ee56c5a2acdc-0_0-86-568_20240312163737521.parquet 4 k4 4.992 p2 20240312163737521 20240312163737521_1_0 20240312163737521_0_0 pid=p3 22af1878-d8e0-4829-b4af-c7c9693d33f3-0_1-86-569_20240312163737521.parquet 5 k5 5.5 p3 -20240312163613712 20240312163613712_0_1 20240312163541346_0_0 pid=p1 ead436a2-f99b-4d35-8b3e-9d67e4828dd5-0_0-99-636_20240312163738946.parquet 1 k1 1.134 p1 -- !inc_cow_table -- +20240312164613448 20240312164613448_0_0 20240312164538551_0_0 pid=p1 372f896b-10cb-4803-b109-d467189326b8-0_0-166-1042_20240312164650751.parquet 1 k1 1.37 p1 +20240312164650751 20240312164650751_0_1 20240312164538551_1_0 pid=p1 372f896b-10cb-4803-b109-d467189326b8-0_0-166-1042_20240312164650751.parquet 2 k2 2.2 p1 20240312164834145 20240312164834145_0_0 20240312164538551_2_0 pid=p2 b8c2db38-e6b8-41f4-8132-620106e8c7e9-0_0-222-1414_20240312164938557.parquet 3 k3-3 3.77 p2 20240312164938557 20240312164938557_0_1 20240312164938557_1_0 pid=p2 b8c2db38-e6b8-41f4-8132-620106e8c7e9-0_0-222-1414_20240312164938557.parquet 4 k4 4.992 p2 20240312164938557 20240312164938557_1_0 20240312164938557_0_0 pid=p3 1335e747-d611-4575-8612-15e491224a0e-0_1-222-1415_20240312164938557.parquet 5 k5 5.5 p3 -20240312164613448 20240312164613448_0_0 20240312164538551_0_0 pid=p1 372f896b-10cb-4803-b109-d467189326b8-0_0-166-1042_20240312164650751.parquet 1 k1 1.37 p1 -20240312164650751 20240312164650751_0_1 20240312164538551_1_0 pid=p1 372f896b-10cb-4803-b109-d467189326b8-0_0-166-1042_20240312164650751.parquet 2 k2 2.2 p1 -- !skip_merge -- 20230605145009209 20230605145009209_0_0 rowId:row_1 partitionId=2021-01-01/versionId=v_0 65ffc5d9-397a-456e-a735-30f3ad37466f-0_0-33-96_20230605145009209.parquet row_1 2021-01-01 0 bob v_0 toBeDel0 0 1000000 diff --git a/regression-test/data/nereids_p0/join/test_runtimefilter_on_decimal.out b/regression-test/data/nereids_p0/join/test_runtimefilter_on_decimal.out new file mode 100644 index 00000000000000..bdcd7847029973 --- /dev/null +++ b/regression-test/data/nereids_p0/join/test_runtimefilter_on_decimal.out @@ -0,0 +1,8 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !dec_rftest_1 -- + +-- !dec_rftest_2 -- +-99999999999999999999.999999 -99999999999999999999.999999 +12345678901234567890.123456 12345678901234567890.123456 +99999999999999999999.999999 99999999999999999999.999999 + diff --git a/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out b/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out index fbbbf32b065ac0..5a9c194c37342d 100644 --- a/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out +++ b/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out @@ -285,3 +285,31 @@ PhysicalResultSink -- !res -- +-- !name -- +multi_table_join_with_pk_predicate + +-- !shape -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((fkt_not_null.fk = fkt_not_null2.fk)) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() +------filter((cast(p as DOUBLE) = 1.0)) +--------PhysicalOlapScan[pkt] +------PhysicalOlapScan[fkt_not_null] +----PhysicalOlapScan[fkt_not_null] + +-- !res -- + +-- !name -- +multi_table_join_with_pk_fk_predicate + +-- !shape -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((fkt_not_null.fk = fkt_not_null2.fk)) otherCondition=() +----filter((fkt_not_null.fk = 1)) +------PhysicalOlapScan[fkt_not_null] +----filter((fkt_not_null2.fk = 1)) +------PhysicalOlapScan[fkt_not_null] + +-- !res -- +1 John + diff --git a/regression-test/data/schema_change_p0/test_unique_model_schema_value_change.out b/regression-test/data/schema_change_p0/test_unique_model_schema_value_change.out index 593a45475daf4a..7babab624e0de4 100644 --- a/regression-test/data/schema_change_p0/test_unique_model_schema_value_change.out +++ b/regression-test/data/schema_change_p0/test_unique_model_schema_value_change.out @@ -419,3 +419,43 @@ 789012345 Grace 21234683141 Xian 29 0 13333333333 No. 222 Street, Xian 2022-07-07T22:00 123456689 Alice asd Yaan 25 0 13812345678 No. 123 Street, Beijing 2022-01-01T10:00 +-- ! -- +123456789 Alice 2.14748324E10 Beijing 25 0 13812345678 No. 123 Street, Beijing 2022-01-01T10:00 +234567890 Bob 2.14743491E10 Shanghai 30 1 13998765432 No. 456 Street, Shanghai 2022-02-02T12:00 +345678901 Carol 2.14742385E11 Guangzhou 28 0 13724681357 No. 789 Street, Guangzhou 2022-03-03T14:00 +456789012 Dave 2.14742835E10 Shenzhen 35 1 13680864279 No. 987 Street, Shenzhen 2022-04-04T16:00 +567890123 Eve 2.12748636E10 Chengdu 27 0 13572468091 No. 654 Street, Chengdu 2022-05-05T18:00 +678901234 Frank 2.12448829E10 Hangzhou 32 1 13467985213 No. 321 Street, Hangzhou 2022-06-06T20:00 +789012345 Grace 2.12346839E10 Xian 29 0 13333333333 No. 222 Street, Xian 2022-07-07T22:00 +123456689 Alice 1.2 Yaan 25 0 13812345678 No. 123 Street, Beijing 2022-01-01T10:00 + +-- ! -- +123456789 Alice 2.1474832641E10 Beijing 25 0 13812345678 No. 123 Street, Beijing 2022-01-01T10:00 +234567890 Bob 2.1474348364E10 Shanghai 30 1 13998765432 No. 456 Street, Shanghai 2022-02-02T12:00 +345678901 Carol 2.14742383441E11 Guangzhou 28 0 13724681357 No. 789 Street, Guangzhou 2022-03-03T14:00 +456789012 Dave 2.1474283141E10 Shenzhen 35 1 13680864279 No. 987 Street, Shenzhen 2022-04-04T16:00 +567890123 Eve 2.1274863141E10 Chengdu 27 0 13572468091 No. 654 Street, Chengdu 2022-05-05T18:00 +678901234 Frank 2.1244883141E10 Hangzhou 32 1 13467985213 No. 321 Street, Hangzhou 2022-06-06T20:00 +789012345 Grace 2.1234683141E10 Xian 29 0 13333333333 No. 222 Street, Xian 2022-07-07T22:00 +123456689 Alice 1.23 Yaan 25 0 13812345678 No. 123 Street, Beijing 2022-01-01T10:00 + +-- ! -- +123456789 Alice 21474832641 Beijing 25 0 13812345678 No. 123 Street, Beijing 2022-01-01T10:00 +234567890 Bob 21474348364 Shanghai 30 1 13998765432 No. 456 Street, Shanghai 2022-02-02T12:00 +345678901 Carol 214742383441 Guangzhou 28 0 13724681357 No. 789 Street, Guangzhou 2022-03-03T14:00 +456789012 Dave 21474283141 Shenzhen 35 1 13680864279 No. 987 Street, Shenzhen 2022-04-04T16:00 +567890123 Eve 21274863141 Chengdu 27 0 13572468091 No. 654 Street, Chengdu 2022-05-05T18:00 +678901234 Frank 21244883141 Hangzhou 32 1 13467985213 No. 321 Street, Hangzhou 2022-06-06T20:00 +789012345 Grace 21234683141 Xian 29 0 13333333333 No. 222 Street, Xian 2022-07-07T22:00 +123456689 Alice asd Yaan 25 0 13812345678 No. 123 Street, Beijing 2022-01-01T10:00 + +-- ! -- +123456789 Alice 21474832641 Beijing 25 0 13812345678 No. 123 Street, Beijing 2022-01-01T10:00 +234567890 Bob 21474348364 Shanghai 30 1 13998765432 No. 456 Street, Shanghai 2022-02-02T12:00 +345678901 Carol 214742383441 Guangzhou 28 0 13724681357 No. 789 Street, Guangzhou 2022-03-03T14:00 +456789012 Dave 21474283141 Shenzhen 35 1 13680864279 No. 987 Street, Shenzhen 2022-04-04T16:00 +567890123 Eve 21274863141 Chengdu 27 0 13572468091 No. 654 Street, Chengdu 2022-05-05T18:00 +678901234 Frank 21244883141 Hangzhou 32 1 13467985213 No. 321 Street, Hangzhou 2022-06-06T20:00 +789012345 Grace 21234683141 Xian 29 0 13333333333 No. 222 Street, Xian 2022-07-07T22:00 +123456689 Alice asd Yaan 25 0 13812345678 No. 123 Street, Beijing 2022-01-01T10:00 + diff --git a/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy b/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy index 92198e6339c13c..5c7162e1eef197 100644 --- a/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy +++ b/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy @@ -16,20 +16,50 @@ // under the License. testGroups = "p0" -//exclude groups and exclude suites is more prior than include groups and include suites. -excludeSuites = "test_index_failure_injection,test_dump_image,test_profile,test_spark_load,test_refresh_mtmv,test_bitmap_filter,test_information_schema_external,test_stream_load_new_move_memtable,test_stream_load_move_memtable,test_materialized_view_move_memtable,test_disable_move_memtable,test_insert_move_memtable,set_and_unset_variable,test_pk_uk_case_cluster,test_point_query_cluster_key,test_compaction_uniq_cluster_keys_with_delete,test_compaction_uniq_keys_cluster_key,test_set_partition_version,test_show_transaction,test_be_inject_publish_txn_fail,test_report_version_missing,test_publish_timeout,set_replica_status" +// exclude groups and exclude suites is more prior than include groups and include suites. +// keep them in lexico order(add/remove cases between the sentinals and sort): +// * sort lines in vim: select lines and then type :sort +// * sort lines in vscode: https://ulfschneider.io/2023-09-01-sort-in-vscode/ +excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line + "set_and_unset_variable," + + "set_replica_status," + // not a case for cloud mode, no need to run + "test_be_inject_publish_txn_fail," + // not a case for cloud mode, no need to run + "test_bitmap_filter," + + "test_compaction_uniq_cluster_keys_with_delete," + + "test_compaction_uniq_keys_cluster_key," + + "test_disable_move_memtable," + + "test_dump_image," + + "test_index_failure_injection," + + "test_information_schema_external," + + "test_insert_move_memtable," + + "test_materialized_view_move_memtable," + + "test_pk_uk_case_cluster," + + "test_point_query_cluster_key," + + "test_profile," + + "test_publish_timeout," + + "test_refresh_mtmv," + // not supported yet + "test_report_version_missing," + + "test_set_partition_version," + + "test_show_transaction," + // not supported yet + "test_spark_load," + + "test_stream_load_move_memtable," + + "test_stream_load_new_move_memtable," + + "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line -excludeDirectories = """ - cloud/multi_cluster, - workload_manager_p1, - nereids_rules_p0/subquery, - unique_with_mow_p0/cluster_key, - unique_with_mow_p0/ssb_unique_sql_zstd_cluster, - unique_with_mow_p0/ssb_unique_load_zstd_c, - nereids_rules_p0/mv, - backup_restore, - cold_heat_separation, - storage_medium_p0 -""" + +excludeDirectories = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line + "cloud/multi_cluster," + // run in specific regression pipeline + "workload_manager_p1," + + "nereids_rules_p0/subquery," + + "unique_with_mow_p0/cluster_key," + + "unique_with_mow_p0/ssb_unique_sql_zstd_cluster," + + "unique_with_mow_p0/ssb_unique_load_zstd_c," + + "nereids_rules_p0/mv," + + "backup_restore," + // not a case for cloud mode, no need to run + "cold_heat_separation," + + "storage_medium_p0," + + "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line max_failure_num = 100 + +// vim: tw=10086 et ts=4 sw=4: diff --git a/regression-test/pipeline/cloud_p1/conf/regression-conf-custom.groovy b/regression-test/pipeline/cloud_p1/conf/regression-conf-custom.groovy index b693cf0fb01d6f..2cd61d23a1a973 100644 --- a/regression-test/pipeline/cloud_p1/conf/regression-conf-custom.groovy +++ b/regression-test/pipeline/cloud_p1/conf/regression-conf-custom.groovy @@ -1,5 +1,19 @@ testGroups = "p1" //exclude groups and exclude suites is more prior than include groups and include suites. -excludeSuites = "test_big_pad,test_profile,test_broker_load,test_spark_load,test_analyze_stats_p1,test_refresh_mtmv,test_bitmap_filter" -excludeDirectories = "workload_manager_p1,fault_injection_p0" +excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line + "test_analyze_stats_p1," + + "test_big_pad," + + "test_bitmap_filter," + + "test_broker_load," + + "test_profile," + + "test_refresh_mtmv," + + "test_spark_load," + + "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line + +excludeDirectories = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line + "fault_injection_p0," + + "workload_manager_p1," + + "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line + max_failure_num = 50 + diff --git a/regression-test/pipeline/external/conf/regression-conf.groovy b/regression-test/pipeline/external/conf/regression-conf.groovy index 02b5c68fd084aa..52bf2ff8908fb2 100644 --- a/regression-test/pipeline/external/conf/regression-conf.groovy +++ b/regression-test/pipeline/external/conf/regression-conf.groovy @@ -57,7 +57,14 @@ testDirectories = "" // this groups will not be executed excludeGroups = "" // this suites will not be executed -excludeSuites = "test_cast_string_to_array,test_broker_load,test_spark_load,test_analyze_stats_p1,test_refresh_mtmv" +excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line + "test_analyze_stats_p1," + + "test_broker_load," + + "test_cast_string_to_array," + + "test_refresh_mtmv," + + "test_spark_load," + + "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line + // this directories will not be executed excludeDirectories = "" diff --git a/regression-test/pipeline/p0/conf/regression-conf.groovy b/regression-test/pipeline/p0/conf/regression-conf.groovy index bd909f903fafac..3cbadc1d481ea6 100644 --- a/regression-test/pipeline/p0/conf/regression-conf.groovy +++ b/regression-test/pipeline/p0/conf/regression-conf.groovy @@ -62,14 +62,23 @@ excludeGroups = "" // this suites will not be executed // load_stream_fault_injection may cause bad disk -excludeSuites = "test_stream_stub_fault_injection,test_index_failure_injection,test_dump_image,test_profile,test_spark_load,test_refresh_mtmv,test_bitmap_filter,test_information_schema_external" +excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line + "test_bitmap_filter," + + "test_dump_image," + + "test_index_failure_injection," + + "test_information_schema_external," + + "test_profile," + + "test_refresh_mtmv," + + "test_spark_load," + + "test_stream_stub_fault_injection," + + "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line // this directories will not be executed -excludeDirectories = """ - cloud, - nereids_rules_p0/subquery, - workload_manager_p1 -""" +excludeDirectories = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line + "cloud," + + "nereids_rules_p0/subquery," + + "workload_manager_p1," + + "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line customConf1 = "test_custom_conf_value" diff --git a/regression-test/pipeline/p1/conf/regression-conf.groovy b/regression-test/pipeline/p1/conf/regression-conf.groovy index 362d33b100284e..cfe34f96bc96ab 100644 --- a/regression-test/pipeline/p1/conf/regression-conf.groovy +++ b/regression-test/pipeline/p1/conf/regression-conf.groovy @@ -52,9 +52,22 @@ testGroups = "" // empty suite will test all suite testSuites = "" // this suites will not be executed -excludeSuites = "test_big_pad,test_profile,test_broker_load,test_spark_load,test_analyze_stats_p1,test_refresh_mtmv,test_bitmap_filter" +excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line + "test_analyze_stats_p1," + + "test_big_pad," + + "test_bitmap_filter," + + "test_broker_load," + + "test_profile," + + "test_refresh_mtmv," + + "test_spark_load," + + "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line + // this dir will not be executed -excludeDirectories = "workload_manager_p1,fault_injection_p0" +excludeDirectories = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line + "fault_injection_p0," + + "workload_manager_p1," + + "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line + cacheDataPath="/data/regression/" s3Endpoint = "cos.ap-hongkong.myqcloud.com" diff --git a/regression-test/suites/delete_p0/test_delete_handler.groovy b/regression-test/suites/delete_p0/test_delete_handler.groovy new file mode 100644 index 00000000000000..8c9ac33edc95a8 --- /dev/null +++ b/regression-test/suites/delete_p0/test_delete_handler.groovy @@ -0,0 +1,84 @@ +// 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. + +/** + * Test delete with strange name + */ +suite("test_delete_handler") { + // test condition operator + sql """drop table if exists td1;""" + sql """ + CREATE TABLE `td1` ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `score` int(11) NULL + ) ENGINE=OLAP + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( "replication_num" = "1" ); + """ + + sql """insert into td1(id,name,`score`) values(1,"a",1),(2,"a",2),(3,"a",3),(4,"b",4);""" + sql """insert into td1(id,name,`score`) values(1,"a",1),(2,"a",2),(3,"a",3),(4,"b",4);""" + sql """insert into td1(id,name,`score`) values(1,"a",1),(2,"a",2),(3,"a",3),(4,"b",4);""" + + sql """delete from td1 where `score` = 4 and `score` = 3 and `score` = 1;""" + sql """delete from td1 where `score` is null;""" + sql """delete from td1 where `score` is not null;""" + sql """delete from td1 where `score` in (1,2);""" + sql """delete from td1 where `score` not in (3,4);""" + sql """select * from td1;""" + sql """insert into td1(id,name,`score`) values(1,"a",1),(2,"a",2),(3,"a",3),(4,"b",4);""" + sql """select * from td1;""" + + + // test column name + sql """drop table if exists td2;""" + sql """ + CREATE TABLE `td2` ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `@score` int(11) NULL, + `scoreIS` int(11) NULL, + `sc ore` int(11) NULL, + `score IS score` int(11) NULL, + `_a-zA-Z0-9_.+-/?@#\$%^&*" ,:` int(11) NULL + ) ENGINE=OLAP + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( "replication_num" = "1" ); + """ + + sql """insert into td2(id,name,`@score`) values(1,"a",1),(2,"a",2),(3,"a",3),(4,"b",4);""" + sql """insert into td2(id,name,`@score`) values(1,"a",1),(2,"a",2),(3,"a",3),(4,"b",4);""" + sql """insert into td2(id,name,`@score`) values(1,"a",1),(2,"a",2),(3,"a",3),(4,"b",4);""" + + sql """select * from td2;""" + sql """delete from td2 where `@score` = 88;""" + sql """delete from td2 where `scoreIS` is null;""" + sql """delete from td2 where `score IS score` is null;""" + sql """delete from td2 where `sc ore` is null;""" + sql """delete from td2 where `_a-zA-Z0-9_.+-/?@#\$%^&*" ,:` is null;""" + sql """delete from td2 where `_a-zA-Z0-9_.+-/?@#\$%^&*" ,:` is not null;""" + sql """delete from td2 where `_a-zA-Z0-9_.+-/?@#\$%^&*" ,:` in (1,2,3);""" + sql """delete from td2 where `_a-zA-Z0-9_.+-/?@#\$%^&*" ,:` not in (1,2,3);""" + sql """select * from td2;""" + sql """insert into td2(id,name,`@score`) values(1,"a",1),(2,"a",2),(3,"a",3),(4,"b",4);""" + sql """select * from td2;""" + +} + diff --git a/regression-test/suites/external_table_p0/hive/write/test_hive_write_insert.groovy b/regression-test/suites/external_table_p0/hive/write/test_hive_write_insert.groovy new file mode 100644 index 00000000000000..e5f57d7f78d8ef --- /dev/null +++ b/regression-test/suites/external_table_p0/hive/write/test_hive_write_insert.groovy @@ -0,0 +1,925 @@ +// 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_hive_write_insert", "p0,external,hive,external_docker,external_docker_hive") { + def format_compressions = ["parquet_snappy", "orc_zlib"] + + def q01 = { String format_compression, String catalog_name -> + logger.info("hive sql: " + """ truncate table all_types_${format_compression}; """) + hive_docker """ truncate table all_types_${format_compression}; """ + sql """refresh catalog ${catalog_name};""" + + sql """ + INSERT INTO all_types_${format_compression} + VALUES ( + CAST(1 AS BOOLEAN), -- boolean_col + CAST(127 AS TINYINT), -- tinyint_col + CAST(32767 AS SMALLINT), -- smallint_col + CAST(2147483647 AS INT), -- int_col + CAST(9223372036854775807 AS BIGINT), -- bigint_col + CAST(123.45 AS FLOAT), -- float_col + CAST(123456.789 AS DOUBLE), -- double_col + CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + CAST('string_value' AS STRING), -- string_col + 'binary_value', -- binary_col + '2024-03-20', -- date_col + '2024-03-20 12:00:00', -- timestamp_col1 + '2024-03-20 12:00:00.123456789', -- timestamp_col2 + '2024-03-20 12:00:00.123456789', -- timestamp_col3 + CAST('char_value1' AS CHAR(50)), -- char_col1 + CAST('char_value2' AS CHAR(100)), -- char_col2 + CAST('char_value3' AS CHAR(255)), -- char_col3 + CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1 + CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2 + CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3 + MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string + MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar + MAP(CAST('a' AS CHAR(10)), CAST('b' AS CHAR(10))), -- t_map_char + MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int + MAP(CAST(1 AS BIGINT), CAST(100000000000 AS BIGINT)), -- t_map_bigint + MAP(CAST(1.1 AS FLOAT), CAST(10.1 AS FLOAT)), -- t_map_float + MAP(CAST(1.1 AS DOUBLE), CAST(10.1 AS DOUBLE)), -- t_map_double + MAP(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_map_boolean + MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string + ARRAY(CAST(1 AS INT), CAST(2 AS INT), CAST(3 AS INT)), -- t_array_int + ARRAY(CAST(100000000000 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_array_bigint + ARRAY(CAST(1.1 AS FLOAT), CAST(2.2 AS FLOAT)), -- t_array_float + ARRAY(CAST(1.123456789 AS DOUBLE), CAST(2.123456789 AS DOUBLE)), -- t_array_double + ARRAY(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_array_boolean + ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar + ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char + ARRAY(CAST(1.1 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(1.23 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(1.23456789 AS DECIMAL(17,8)), CAST(2.34567891 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(1.23456789 AS DECIMAL(18,8)), CAST(2.34567891 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', CAST(1234567890 AS BIGINT)), -- t_struct_bigint + MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(123 AS INT)))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested + NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null + NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', CAST(123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls + MAP('null_key', null), -- t_map_null_value + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls + ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between + ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls + ARRAY(null, null, null), -- t_array_string_all_nulls + 20240320 -- dt + ); + """ + order_qt_q01 """ select * from all_types_${format_compression}; + """ + + sql """ + INSERT INTO all_types_${format_compression} + VALUES ( + CAST(1 AS BOOLEAN), -- boolean_col + CAST(127 AS TINYINT), -- tinyint_col + CAST(32767 AS SMALLINT), -- smallint_col + CAST(2147483647 AS INT), -- int_col + CAST(9223372036854775807 AS BIGINT), -- bigint_col + CAST(123.45 AS FLOAT), -- float_col + CAST(123456.789 AS DOUBLE), -- double_col + CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + CAST('string_value' AS STRING), -- string_col + 'binary_value', -- binary_col + '2024-03-20', -- date_col + '2024-03-20 12:00:00', -- timestamp_col1 + '2024-03-20 12:00:00.123456789', -- timestamp_col2 + '2024-03-20 12:00:00.123456789', -- timestamp_col3 + CAST('char_value1' AS CHAR(50)), -- char_col1 + CAST('char_value2' AS CHAR(100)), -- char_col2 + CAST('char_value3' AS CHAR(255)), -- char_col3 + CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1 + CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2 + CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3 + MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string + MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar + MAP(CAST('a' AS CHAR(10)), CAST('b' AS CHAR(10))), -- t_map_char + MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int + MAP(CAST(1 AS BIGINT), CAST(100000000000 AS BIGINT)), -- t_map_bigint + MAP(CAST(1.1 AS FLOAT), CAST(10.1 AS FLOAT)), -- t_map_float + MAP(CAST(1.1 AS DOUBLE), CAST(10.1 AS DOUBLE)), -- t_map_double + MAP(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_map_boolean + MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string + ARRAY(CAST(1 AS INT), CAST(2 AS INT), CAST(3 AS INT)), -- t_array_int + ARRAY(CAST(100000000000 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_array_bigint + ARRAY(CAST(1.1 AS FLOAT), CAST(2.2 AS FLOAT)), -- t_array_float + ARRAY(CAST(1.123456789 AS DOUBLE), CAST(2.123456789 AS DOUBLE)), -- t_array_double + ARRAY(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_array_boolean + ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar + ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char + ARRAY(CAST(1.1 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(1.23 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(1.23456789 AS DECIMAL(17,8)), CAST(2.34567891 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(1.23456789 AS DECIMAL(18,8)), CAST(2.34567891 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', CAST(1234567890 AS BIGINT)), -- t_struct_bigint + MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(123 AS INT)))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested + NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null + NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', CAST(123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls + MAP('null_key', null), -- t_map_null_value + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls + ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between + ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls + ARRAY(null, null, null), -- t_array_string_all_nulls + 20240320 -- dt + ), + ( + CAST(0 AS BOOLEAN), -- boolean_col + CAST(-128 AS TINYINT), -- tinyint_col + CAST(-32768 AS SMALLINT), -- smallint_col + CAST(-2147483648 AS INT), -- int_col + CAST(-9223372036854775808 AS BIGINT), -- bigint_col + CAST(-123.45 AS FLOAT), -- float_col + CAST(-123456.789 AS DOUBLE), -- double_col + CAST(-123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + CAST('string_value' AS STRING), -- string_col + 'binary_value', -- binary_col + '2024-03-21', -- date_col + '2024-03-21 12:00:00', -- timestamp_col1 + '2024-03-21 12:00:00.123456789', -- timestamp_col2 + '2024-03-21 12:00:00.123456789', -- timestamp_col3 + CAST('char_value1' AS CHAR(50)), -- char_col1 + CAST('char_value2' AS CHAR(100)), -- char_col2 + CAST('char_value3' AS CHAR(255)), -- char_col3 + CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1 + CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2 + CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3 + MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string + MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar + MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char + MAP(CAST(2 AS INT), CAST(20 AS INT)), -- t_map_int + MAP(CAST(2 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint + MAP(CAST(2.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float + MAP(CAST(2.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double + MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean + MAP(CAST(2.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(2.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(2.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string + ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int + ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint + ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float + ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double + ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean + ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar + ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char + ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(3.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(3.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint + MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested + NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null + NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls + MAP('null_key', null), -- t_map_null_value + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls + ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between + ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls + ARRAY(null, null, null), -- t_array_string_all_nulls + 20240321 -- dt + ), + ( + CAST(0 AS BOOLEAN), -- boolean_col + CAST(-128 AS TINYINT), -- tinyint_col + CAST(-32768 AS SMALLINT), -- smallint_col + CAST(-2147483648 AS INT), -- int_col + CAST(-9223372036854775808 AS BIGINT), -- bigint_col + CAST(-123.45 AS FLOAT), -- float_col + CAST(-123456.789 AS DOUBLE), -- double_col + CAST(-123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + CAST('string_value' AS STRING), -- string_col + 'binary_value', -- binary_col + '2024-03-22', -- date_col + '2024-03-22 12:00:00', -- timestamp_col1 + '2024-03-22 12:00:00.123456789', -- timestamp_col2 + '2024-03-22 12:00:00.123456789', -- timestamp_col3 + CAST('char_value1' AS CHAR(50)), -- char_col1 + CAST('char_value2' AS CHAR(100)), -- char_col2 + CAST('char_value3' AS CHAR(255)), -- char_col3 + CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1 + CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2 + CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3 + MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string + MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar + MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char + MAP(CAST(3 AS INT), CAST(20 AS INT)), -- t_map_int + MAP(CAST(3 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint + MAP(CAST(3.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float + MAP(CAST(3.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double + MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean + MAP(CAST(3.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(3.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(3.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string + ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int + ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint + ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float + ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double + ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean + ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar + ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char + ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(8.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(3.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint + MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested + NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null + NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls + MAP('null_key', null), -- t_map_null_value + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls + ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between + ARRAY(CAST('value11' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls + ARRAY(null, null, null), -- t_array_string_all_nulls + 20240322 -- dt + ); + """ + order_qt_q02 """ select * from all_types_${format_compression}; + """ + + sql """ + INSERT INTO all_types_${format_compression}(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls) + VALUES ( + CAST(123.45 AS FLOAT), -- float_col + MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int + ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)) -- t_array_string_starting_with_nulls + ); + """ + order_qt_q03 """ select * from all_types_${format_compression}; + """ + + sql """ + insert overwrite table all_types_${format_compression} + VALUES ( + CAST(0 AS BOOLEAN), -- boolean_col + CAST(-7 AS TINYINT), -- tinyint_col + CAST(-15 AS SMALLINT), -- smallint_col + CAST(16 AS INT), -- int_col + CAST(-9223372036854775808 AS BIGINT), -- bigint_col + CAST(-123.45 AS FLOAT), -- float_col + CAST(-123456.789 AS DOUBLE), -- double_col + CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + CAST('str' AS STRING), -- string_col + 'binary_value', -- binary_col + '2024-03-25', -- date_col + '2024-03-25 12:00:00', -- timestamp_col1 + '2024-03-25 12:00:00.123456789', -- timestamp_col2 + '2024-03-25 12:00:00.123456789', -- timestamp_col3 + CAST('char_value11111' AS CHAR(50)), -- char_col1 + CAST('char_value22222' AS CHAR(100)), -- char_col2 + CAST('char_value33333' AS CHAR(255)), -- char_col3 + CAST('varchar_value11111' AS VARCHAR(50)), -- varchar_col1 + CAST('varchar_value22222' AS VARCHAR(100)), -- varchar_col2 + CAST('varchar_value33333' AS VARCHAR(255)), -- varchar_col3 + MAP(CAST('key7' AS STRING), CAST('value1' AS STRING)), -- t_map_string + MAP(CAST('key7' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar + MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char + MAP(CAST(3 AS INT), CAST(20 AS INT)), -- t_map_int + MAP(CAST(3 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint + MAP(CAST(3.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float + MAP(CAST(3.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double + MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean + MAP(CAST(3.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(3.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(5.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(5.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(7.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string + ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int + ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint + ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float + ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double + ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean + ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar + ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char + ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(9.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(6.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint + MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested + NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null + NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls + MAP('null_key', null), -- t_map_null_value + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls + ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between + ARRAY(CAST('value11' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls + ARRAY(null, null, null), -- t_array_string_all_nulls + 20240325 -- dt + ); + """ + order_qt_q04 """ select * from all_types_${format_compression}; + """ + + sql """ + INSERT overwrite table all_types_${format_compression}(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt) + VALUES ( + CAST(123.45 AS FLOAT), -- float_col + MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int + ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls + 20240321 -- dt + ); + """ + order_qt_q05 """ select * from all_types_${format_compression}; + """ + + logger.info("hive sql: " + """ truncate table all_types_${format_compression}; """) + hive_docker """ truncate table all_types_${format_compression}; """ + sql """refresh catalog ${catalog_name};""" + order_qt_q06 """ select * from all_types_${format_compression}; + """ + } + + def q02 = { String format_compression, String catalog_name -> + logger.info("hive sql: " + """ truncate table all_types_${format_compression}; """) + hive_docker """ truncate table all_types_${format_compression}; """ + sql """refresh catalog ${catalog_name};""" + + sql """ + INSERT INTO all_types_${format_compression} + SELECT * FROM all_types_parquet_snappy_src; + """ + order_qt_q01 """ select * from all_types_${format_compression}; + """ + + sql """ + INSERT INTO all_types_${format_compression} + SELECT boolean_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, decimal_col1, decimal_col2, + decimal_col3, decimal_col4, string_col, binary_col, date_col, timestamp_col1, timestamp_col2, timestamp_col3, char_col1, + char_col2, char_col3, varchar_col1, varchar_col2, varchar_col3, t_map_string, t_map_varchar, t_map_char, t_map_int, + t_map_bigint, t_map_float, t_map_double, t_map_boolean, t_map_decimal_precision_2, t_map_decimal_precision_4, + t_map_decimal_precision_8, t_map_decimal_precision_17, t_map_decimal_precision_18, t_map_decimal_precision_38, + t_array_string, t_array_int, t_array_bigint, t_array_float, t_array_double, t_array_boolean, t_array_varchar, + t_array_char, t_array_decimal_precision_2, t_array_decimal_precision_4, t_array_decimal_precision_8, + t_array_decimal_precision_17, t_array_decimal_precision_18, t_array_decimal_precision_38, t_struct_bigint, t_complex, + t_struct_nested, t_struct_null, t_struct_non_nulls_after_nulls, t_nested_struct_non_nulls_after_nulls, + t_map_null_value, t_array_string_starting_with_nulls, t_array_string_with_nulls_in_between, + t_array_string_ending_with_nulls, t_array_string_all_nulls, dt FROM all_types_parquet_snappy_src; + """ + order_qt_q02 """ select * from all_types_${format_compression}; + """ + + sql """ + INSERT INTO all_types_${format_compression}(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls) + SELECT float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls FROM all_types_parquet_snappy_src; + """ + order_qt_q03 """ + select * from all_types_${format_compression}; + """ + + sql """ + INSERT OVERWRITE TABLE all_types_${format_compression}(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls) + SELECT float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls FROM all_types_parquet_snappy_src; + """ + order_qt_q04 """ + select * from all_types_${format_compression}; + """ + + logger.info("hive sql: " + """ truncate table all_types_${format_compression}; """) + hive_docker """ truncate table all_types_${format_compression}; """ + sql """refresh catalog ${catalog_name};""" + order_qt_q05 """ + select * from all_types_${format_compression}; + """ + } + def q03 = { String format_compression, String catalog_name -> + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q03; """) + hive_docker """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q03; """ + logger.info("hive sql: " + """CREATE TABLE IF NOT EXISTS all_types_par_${format_compression}_${catalog_name}_q03 like all_types_par_${format_compression};""") + hive_docker """ CREATE TABLE IF NOT EXISTS all_types_par_${format_compression}_${catalog_name}_q03 like all_types_par_${format_compression}; """ + sql """refresh catalog ${catalog_name};""" + + sql """ + INSERT INTO all_types_par_${format_compression}_${catalog_name}_q03 + VALUES ( + CAST(1 AS BOOLEAN), -- boolean_col + CAST(127 AS TINYINT), -- tinyint_col + CAST(32767 AS SMALLINT), -- smallint_col + CAST(2147483647 AS INT), -- int_col + CAST(9223372036854775807 AS BIGINT), -- bigint_col + CAST(123.45 AS FLOAT), -- float_col + CAST(123456.789 AS DOUBLE), -- double_col + CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + CAST('string_value' AS STRING), -- string_col + 'binary_value', -- binary_col + '2024-03-20', -- date_col + '2024-03-20 12:00:00', -- timestamp_col1 + '2024-03-20 12:00:00.123456789', -- timestamp_col2 + '2024-03-20 12:00:00.123456789', -- timestamp_col3 + CAST('char_value1' AS CHAR(50)), -- char_col1 + CAST('char_value2' AS CHAR(100)), -- char_col2 + CAST('char_value3' AS CHAR(255)), -- char_col3 + CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1 + CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2 + CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3 + MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string + MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar + MAP(CAST('a' AS CHAR(10)), CAST('b' AS CHAR(10))), -- t_map_char + MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int + MAP(CAST(1 AS BIGINT), CAST(100000000000 AS BIGINT)), -- t_map_bigint + MAP(CAST(1.1 AS FLOAT), CAST(10.1 AS FLOAT)), -- t_map_float + MAP(CAST(1.1 AS DOUBLE), CAST(10.1 AS DOUBLE)), -- t_map_double + MAP(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_map_boolean + MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string + ARRAY(CAST(1 AS INT), CAST(2 AS INT), CAST(3 AS INT)), -- t_array_int + ARRAY(CAST(100000000000 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_array_bigint + ARRAY(CAST(1.1 AS FLOAT), CAST(2.2 AS FLOAT)), -- t_array_float + ARRAY(CAST(1.123456789 AS DOUBLE), CAST(2.123456789 AS DOUBLE)), -- t_array_double + ARRAY(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_array_boolean + ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar + ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char + ARRAY(CAST(1.1 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(1.23 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(1.23456789 AS DECIMAL(17,8)), CAST(2.34567891 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(1.23456789 AS DECIMAL(18,8)), CAST(2.34567891 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', CAST(1234567890 AS BIGINT)), -- t_struct_bigint + MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(123 AS INT)))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested + NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null + NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', CAST(123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls + MAP('null_key', null), -- t_map_null_value + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls + ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between + ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls + ARRAY(null, null, null), -- t_array_string_all_nulls + 20240320 -- dt + ); + """ + order_qt_q01 """ select * from all_types_par_${format_compression}_${catalog_name}_q03; + """ + //sql """refresh catalog ${catalog_name};""" + + sql """ + INSERT INTO all_types_par_${format_compression}_${catalog_name}_q03 + VALUES ( + CAST(1 AS BOOLEAN), -- boolean_col + CAST(127 AS TINYINT), -- tinyint_col + CAST(32767 AS SMALLINT), -- smallint_col + CAST(2147483647 AS INT), -- int_col + CAST(9223372036854775807 AS BIGINT), -- bigint_col + CAST(123.45 AS FLOAT), -- float_col + CAST(123456.789 AS DOUBLE), -- double_col + CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + CAST('string_value' AS STRING), -- string_col + 'binary_value', -- binary_col + '2024-03-20', -- date_col + '2024-03-20 12:00:00', -- timestamp_col1 + '2024-03-20 12:00:00.123456789', -- timestamp_col2 + '2024-03-20 12:00:00.123456789', -- timestamp_col3 + CAST('char_value1' AS CHAR(50)), -- char_col1 + CAST('char_value2' AS CHAR(100)), -- char_col2 + CAST('char_value3' AS CHAR(255)), -- char_col3 + CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1 + CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2 + CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3 + MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string + MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar + MAP(CAST('a' AS CHAR(10)), CAST('b' AS CHAR(10))), -- t_map_char + MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int + MAP(CAST(1 AS BIGINT), CAST(100000000000 AS BIGINT)), -- t_map_bigint + MAP(CAST(1.1 AS FLOAT), CAST(10.1 AS FLOAT)), -- t_map_float + MAP(CAST(1.1 AS DOUBLE), CAST(10.1 AS DOUBLE)), -- t_map_double + MAP(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_map_boolean + MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string + ARRAY(CAST(1 AS INT), CAST(2 AS INT), CAST(3 AS INT)), -- t_array_int + ARRAY(CAST(100000000000 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_array_bigint + ARRAY(CAST(1.1 AS FLOAT), CAST(2.2 AS FLOAT)), -- t_array_float + ARRAY(CAST(1.123456789 AS DOUBLE), CAST(2.123456789 AS DOUBLE)), -- t_array_double + ARRAY(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_array_boolean + ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar + ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char + ARRAY(CAST(1.1 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(1.23 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(1.23456789 AS DECIMAL(17,8)), CAST(2.34567891 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(1.23456789 AS DECIMAL(18,8)), CAST(2.34567891 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', CAST(1234567890 AS BIGINT)), -- t_struct_bigint + MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(123 AS INT)))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested + NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null + NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', CAST(123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls + MAP('null_key', null), -- t_map_null_value + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls + ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between + ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls + ARRAY(null, null, null), -- t_array_string_all_nulls + 20240320 -- dt + ), + ( + CAST(0 AS BOOLEAN), -- boolean_col + CAST(-128 AS TINYINT), -- tinyint_col + CAST(-32768 AS SMALLINT), -- smallint_col + CAST(-2147483648 AS INT), -- int_col + CAST(-9223372036854775808 AS BIGINT), -- bigint_col + CAST(-123.45 AS FLOAT), -- float_col + CAST(-123456.789 AS DOUBLE), -- double_col + CAST(-123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + CAST('string_value' AS STRING), -- string_col + 'binary_value', -- binary_col + '2024-03-21', -- date_col + '2024-03-21 12:00:00', -- timestamp_col1 + '2024-03-21 12:00:00.123456789', -- timestamp_col2 + '2024-03-21 12:00:00.123456789', -- timestamp_col3 + CAST('char_value1' AS CHAR(50)), -- char_col1 + CAST('char_value2' AS CHAR(100)), -- char_col2 + CAST('char_value3' AS CHAR(255)), -- char_col3 + CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1 + CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2 + CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3 + MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string + MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar + MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char + MAP(CAST(2 AS INT), CAST(20 AS INT)), -- t_map_int + MAP(CAST(2 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint + MAP(CAST(2.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float + MAP(CAST(2.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double + MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean + MAP(CAST(2.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(2.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(2.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string + ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int + ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint + ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float + ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double + ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean + ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar + ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char + ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(3.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(3.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint + MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested + NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null + NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls + MAP('null_key', null), -- t_map_null_value + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls + ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between + ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls + ARRAY(null, null, null), -- t_array_string_all_nulls + 20240321 -- dt + ), + ( + CAST(0 AS BOOLEAN), -- boolean_col + CAST(-128 AS TINYINT), -- tinyint_col + CAST(-32768 AS SMALLINT), -- smallint_col + CAST(-2147483648 AS INT), -- int_col + CAST(-9223372036854775808 AS BIGINT), -- bigint_col + CAST(-123.45 AS FLOAT), -- float_col + CAST(-123456.789 AS DOUBLE), -- double_col + CAST(-123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + CAST('string_value' AS STRING), -- string_col + 'binary_value', -- binary_col + '2024-03-22', -- date_col + '2024-03-22 12:00:00', -- timestamp_col1 + '2024-03-22 12:00:00.123456789', -- timestamp_col2 + '2024-03-22 12:00:00.123456789', -- timestamp_col3 + CAST('char_value1' AS CHAR(50)), -- char_col1 + CAST('char_value2' AS CHAR(100)), -- char_col2 + CAST('char_value3' AS CHAR(255)), -- char_col3 + CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1 + CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2 + CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3 + MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string + MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar + MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char + MAP(CAST(3 AS INT), CAST(20 AS INT)), -- t_map_int + MAP(CAST(3 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint + MAP(CAST(3.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float + MAP(CAST(3.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double + MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean + MAP(CAST(3.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(3.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(3.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string + ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int + ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint + ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float + ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double + ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean + ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar + ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char + ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(8.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(3.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint + MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested + NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null + NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls + MAP('null_key', null), -- t_map_null_value + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls + ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between + ARRAY(CAST('value11' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls + ARRAY(null, null, null), -- t_array_string_all_nulls + 20240322 -- dt + ); + """ + order_qt_q02 """ select * from all_types_par_${format_compression}_${catalog_name}_q03; + """ +// disable it temporarily +// sql """refresh catalog ${catalog_name};""" + +// sql """ +// INSERT INTO all_types_par_${format_compression}_${catalog_name}_q03(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt) +// VALUES ( +// CAST(123.45 AS FLOAT), -- float_col +// MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int +// ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 +// ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls +// 20240321 -- dt +// ); +// """ +// order_qt_q03 """ select * from all_types_par_${format_compression}_${catalog_name}_q03; +// """ + + sql """ + insert overwrite table all_types_par_${format_compression}_${catalog_name}_q03 + VALUES ( + CAST(0 AS BOOLEAN), -- boolean_col + CAST(-7 AS TINYINT), -- tinyint_col + CAST(-15 AS SMALLINT), -- smallint_col + CAST(16 AS INT), -- int_col + CAST(-9223372036854775808 AS BIGINT), -- bigint_col + CAST(-123.45 AS FLOAT), -- float_col + CAST(-123456.789 AS DOUBLE), -- double_col + CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + CAST('str' AS STRING), -- string_col + 'binary_value', -- binary_col + '2024-03-25', -- date_col + '2024-03-25 12:00:00', -- timestamp_col1 + '2024-03-25 12:00:00.123456789', -- timestamp_col2 + '2024-03-25 12:00:00.123456789', -- timestamp_col3 + CAST('char_value11111' AS CHAR(50)), -- char_col1 + CAST('char_value22222' AS CHAR(100)), -- char_col2 + CAST('char_value33333' AS CHAR(255)), -- char_col3 + CAST('varchar_value11111' AS VARCHAR(50)), -- varchar_col1 + CAST('varchar_value22222' AS VARCHAR(100)), -- varchar_col2 + CAST('varchar_value33333' AS VARCHAR(255)), -- varchar_col3 + MAP(CAST('key7' AS STRING), CAST('value1' AS STRING)), -- t_map_string + MAP(CAST('key7' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar + MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char + MAP(CAST(3 AS INT), CAST(20 AS INT)), -- t_map_int + MAP(CAST(3 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint + MAP(CAST(3.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float + MAP(CAST(3.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double + MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean + MAP(CAST(3.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(3.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(5.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(5.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(7.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string + ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int + ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint + ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float + ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double + ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean + ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar + ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char + ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(9.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(6.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint + MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested + NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null + NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls + MAP('null_key', null), -- t_map_null_value + ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls + ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between + ARRAY(CAST('value11' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls + ARRAY(null, null, null), -- t_array_string_all_nulls + 20240321 -- dt + ); + """ + order_qt_q04 """ select * from all_types_par_${format_compression}_${catalog_name}_q03; + """ +// disable it temporarily +// sql """refresh catalog ${catalog_name};""" +// +// sql """ +// INSERT overwrite table all_types_par_${format_compression}_${catalog_name}_q03(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt) +// VALUES ( +// CAST(123.45 AS FLOAT), -- float_col +// MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int +// ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 +// ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls +// 20240321 -- dt +// ); +// """ +// order_qt_q05 """ select * from all_types_par_${format_compression}_${catalog_name}_q03; +// """ + + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q03; """) + hive_docker """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q03; """ + } + + def q04 = { String format_compression, String catalog_name -> + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q04; """) + hive_docker """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q04; """ + logger.info("hive sql: " + """ CREATE TABLE IF NOT EXISTS all_types_par_${format_compression}_${catalog_name}_q04 like all_types_par_${format_compression}; """) + hive_docker """ CREATE TABLE IF NOT EXISTS all_types_par_${format_compression}_${catalog_name}_q04 like all_types_par_${format_compression}; """ + sql """refresh catalog ${catalog_name};""" + + sql """ + INSERT INTO all_types_par_${format_compression}_${catalog_name}_q04 + SELECT * FROM all_types_par_parquet_snappy_src; + """ + order_qt_q01 """ select * from all_types_par_${format_compression}_${catalog_name}_q04; + """ + //sql """refresh catalog ${catalog_name};""" + + sql """ + INSERT INTO all_types_par_${format_compression}_${catalog_name}_q04 + SELECT boolean_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, decimal_col1, decimal_col2, + decimal_col3, decimal_col4, string_col, binary_col, date_col, timestamp_col1, timestamp_col2, timestamp_col3, char_col1, + char_col2, char_col3, varchar_col1, varchar_col2, varchar_col3, t_map_string, t_map_varchar, t_map_char, t_map_int, + t_map_bigint, t_map_float, t_map_double, t_map_boolean, t_map_decimal_precision_2, t_map_decimal_precision_4, + t_map_decimal_precision_8, t_map_decimal_precision_17, t_map_decimal_precision_18, t_map_decimal_precision_38, + t_array_string, t_array_int, t_array_bigint, t_array_float, t_array_double, t_array_boolean, t_array_varchar, + t_array_char, t_array_decimal_precision_2, t_array_decimal_precision_4, t_array_decimal_precision_8, + t_array_decimal_precision_17, t_array_decimal_precision_18, t_array_decimal_precision_38, t_struct_bigint, t_complex, + t_struct_nested, t_struct_null, t_struct_non_nulls_after_nulls, t_nested_struct_non_nulls_after_nulls, + t_map_null_value, t_array_string_starting_with_nulls, t_array_string_with_nulls_in_between, + t_array_string_ending_with_nulls, t_array_string_all_nulls, dt FROM all_types_parquet_snappy_src; + """ + order_qt_q02 """ select * from all_types_par_${format_compression}_${catalog_name}_q04; + """ + //sql """refresh catalog ${catalog_name};""" + + //sql """ + //INSERT INTO all_types_par_${format_compression}_${catalog_name}_q04(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt) + //SELECT float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt FROM all_types_parquet_snappy_src; + //""" + //order_qt_q03 """ select * from all_types_par_${format_compression}_${catalog_name}_q04; + //""" + //sql """refresh catalog ${catalog_name};""" + + //sql """ + //INSERT OVERWRITE TABLE all_types_par_${format_compression}_${catalog_name}_q04(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt) + //SELECT float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt FROM all_types_parquet_snappy_src; + //""" + //order_qt_q04 """ + //select * from all_types_par_${format_compression}_${catalog_name}_q04; + //""" + //sql """refresh catalog ${catalog_name};""" + + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q04; """) + hive_docker """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q04; """ + } + + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + try { + String hms_port = context.config.otherConfigs.get("hms_port") + String hdfs_port = context.config.otherConfigs.get("hdfs_port") + String catalog_name = "test_hive_write_insert" + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + sql """drop catalog if exists ${catalog_name}""" + sql """create catalog if not exists ${catalog_name} properties ( + 'type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}' + );""" + sql """use `${catalog_name}`.`write_test`""" + logger.info("hive sql: " + """ use `write_test` """) + hive_docker """use `write_test`""" + + sql """set enable_fallback_to_original_planner=false;""" + + for (String format_compression in format_compressions) { + logger.info("Process format_compression" + format_compression) + q01(format_compression, catalog_name) + q02(format_compression, catalog_name) + q03(format_compression, catalog_name) + q04(format_compression, catalog_name) + } + + sql """drop catalog if exists ${catalog_name}""" + } finally { + } + } +} + diff --git a/regression-test/suites/external_table_p0/hive/write/test_hive_write_partitions.groovy b/regression-test/suites/external_table_p0/hive/write/test_hive_write_partitions.groovy new file mode 100644 index 00000000000000..d2baf6ddc61380 --- /dev/null +++ b/regression-test/suites/external_table_p0/hive/write/test_hive_write_partitions.groovy @@ -0,0 +1,226 @@ +// 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_hive_write_partitions", "p0,external,hive,external_docker,external_docker_hive") { + def format_compressions = ["parquet_snappy", "orc_zlib"] + + def q01 = { String format_compression, String catalog_name -> + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q01; """) + hive_docker """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q01; """ + logger.info("hive sql: " + """ CREATE TABLE IF NOT EXISTS all_types_par_${format_compression}_${catalog_name}_q01 like all_types_par_${format_compression}; """) + hive_docker """ CREATE TABLE IF NOT EXISTS all_types_par_${format_compression}_${catalog_name}_q01 like all_types_par_${format_compression}; """ + sql """refresh catalog ${catalog_name};""" + + sql """ + INSERT INTO all_types_par_${format_compression}_${catalog_name}_q01 + SELECT * FROM all_types_par_parquet_snappy_src; + """ + order_qt_q01 """ select * from all_types_par_${format_compression}_${catalog_name}_q01; + """ + + sql """ + INSERT INTO all_types_par_${format_compression}_${catalog_name}_q01 + SELECT * FROM all_types_par_parquet_snappy_src; + """ + order_qt_q02 """ select * from all_types_par_${format_compression}_${catalog_name}_q01; + """ + + sql """ + INSERT OVERWRITE TABLE all_types_par_${format_compression}_${catalog_name}_q01 + SELECT * FROM all_types_par_parquet_snappy_src; + """ + order_qt_q03 """ select * from all_types_par_${format_compression}_${catalog_name}_q01; + """ + + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q01; """) + hive_docker """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q01; """ + } + + + def q02 = { String format_compression, String catalog_name -> + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_partition_types1_${format_compression}_${catalog_name}_q02; """) + hive_docker """ DROP TABLE IF EXISTS all_partition_types1_${format_compression}_${catalog_name}_q02; """ + logger.info("hive sql: " + """ CREATE TABLE IF NOT EXISTS all_partition_types1_${format_compression}_${catalog_name}_q02 like all_partition_types1_${format_compression}; """) + hive_docker """ CREATE TABLE IF NOT EXISTS all_partition_types1_${format_compression}_${catalog_name}_q02 like all_partition_types1_${format_compression}; """ + sql """refresh catalog ${catalog_name};""" + + sql """ + INSERT INTO all_partition_types1_${format_compression}_${catalog_name}_q02 + SELECT * FROM all_partition_types1_parquet_snappy_src where id = 1; + """ + order_qt_q01 """ select * from all_partition_types1_${format_compression}_${catalog_name}_q02; + """ + + sql """ + INSERT INTO all_partition_types1_${format_compression}_${catalog_name}_q02 + SELECT * FROM all_partition_types1_parquet_snappy_src where id = 2; + """ + order_qt_q02 """ select * from all_partition_types1_${format_compression}_${catalog_name}_q02; + """ + + sql """ + INSERT OVERWRITE TABLE all_partition_types1_${format_compression}_${catalog_name}_q02 + SELECT * FROM all_partition_types1_parquet_snappy_src; + """ + order_qt_q03 """ select * from all_partition_types1_${format_compression}_${catalog_name}_q02; + """ + + sql """ + INSERT INTO all_partition_types1_${format_compression}_${catalog_name}_q02 + SELECT * FROM all_partition_types1_parquet_snappy_src where id = 3; + """ + order_qt_q04 """ select * from all_partition_types1_${format_compression}_${catalog_name}_q02; + """ + + sql """ + INSERT OVERWRITE TABLE all_partition_types1_${format_compression}_${catalog_name}_q02 + SELECT CAST(7 as INT) as id, boolean_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col FROM all_partition_types1_parquet_snappy_src where id = 3; + """ + order_qt_q05 """ select * from all_partition_types1_${format_compression}_${catalog_name}_q02; + """ + + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_partition_types1_${format_compression}_${catalog_name}_q02; """) + hive_docker """ DROP TABLE IF EXISTS all_partition_types1_${format_compression}_${catalog_name}_q02; """ + } + + def q03 = { String format_compression, String catalog_name -> + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_partition_types2_${format_compression}_${catalog_name}_q03; """) + hive_docker """ DROP TABLE IF EXISTS all_partition_types2_${format_compression}_${catalog_name}_q03; """ + logger.info("hive sql: " + """ CREATE TABLE IF NOT EXISTS all_partition_types2_${format_compression}_${catalog_name}_q03 like all_partition_types2_${format_compression}; """) + hive_docker """ CREATE TABLE IF NOT EXISTS all_partition_types2_${format_compression}_${catalog_name}_q03 like all_partition_types2_${format_compression}; """ + sql """refresh catalog ${catalog_name};""" + + sql """ + INSERT INTO all_partition_types2_${format_compression}_${catalog_name}_q03 + SELECT * FROM all_partition_types2_parquet_snappy_src where id = 1; + """ + order_qt_q01 """ select * from all_partition_types2_${format_compression}_${catalog_name}_q03; + """ + + sql """ + INSERT INTO all_partition_types2_${format_compression}_${catalog_name}_q03 + SELECT * FROM all_partition_types2_parquet_snappy_src where id = 2; + """ + order_qt_q02 """ select * from all_partition_types2_${format_compression}_${catalog_name}_q03; + """ + + sql """ + INSERT OVERWRITE TABLE all_partition_types2_${format_compression}_${catalog_name}_q03 + SELECT * FROM all_partition_types2_parquet_snappy_src; + """ + order_qt_q03 """ select * from all_partition_types2_${format_compression}_${catalog_name}_q03; + """ + + sql """ + INSERT INTO all_partition_types2_${format_compression}_${catalog_name}_q03 + SELECT * FROM all_partition_types2_parquet_snappy_src where id = 3; + """ + order_qt_q04 """ select * from all_partition_types2_${format_compression}_${catalog_name}_q03; + """ + + sql """ + INSERT OVERWRITE TABLE all_partition_types2_${format_compression}_${catalog_name}_q03 + SELECT CAST(7 as INT) as id, decimal_col, string_col, binary_col, date_col, timestamp_col, char_col, varchar_col FROM all_partition_types2_parquet_snappy_src where id = 3; + """ + order_qt_q05 """ select * from all_partition_types2_${format_compression}_${catalog_name}_q03; + """ + + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_partition_types2_${format_compression}_${catalog_name}_q03; """) + hive_docker """ DROP TABLE IF EXISTS all_partition_types2_${format_compression}_${catalog_name}_q03; """ + } + + def q04 = { String format_compression, String catalog_name -> + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_partition_types1_${format_compression}_${catalog_name}_q04; """) + hive_docker """ DROP TABLE IF EXISTS all_partition_types1_${format_compression}_${catalog_name}_q04; """ + logger.info("hive sql: " + """ CREATE TABLE IF NOT EXISTS all_partition_types1_${format_compression}_${catalog_name}_q04 like all_partition_types1_${format_compression}; """) + hive_docker """ CREATE TABLE IF NOT EXISTS all_partition_types1_${format_compression}_${catalog_name}_q04 like all_partition_types1_${format_compression}; """ + sql """refresh catalog ${catalog_name};""" + + sql """ + INSERT INTO all_partition_types1_${format_compression}_${catalog_name}_q04 + SELECT id, null, tinyint_col, null, int_col, null, float_col, null FROM all_partition_types1_parquet_snappy_src where id = 1; + """ + order_qt_q01 """ select * from all_partition_types1_${format_compression}_${catalog_name}_q04; + """ + + sql """ + INSERT INTO all_partition_types1_${format_compression}_${catalog_name}_q04 + SELECT id, null, tinyint_col, null, int_col, null, float_col, null FROM all_partition_types1_parquet_snappy_src where id = 2; + """ + order_qt_q02 """ select * from all_partition_types1_${format_compression}_${catalog_name}_q04; + """ + + sql """ + INSERT OVERWRITE TABLE all_partition_types1_${format_compression}_${catalog_name}_q04 + SELECT id, null, tinyint_col, null, int_col, null, float_col, null FROM all_partition_types1_parquet_snappy_src; + """ + order_qt_q03 """ select * from all_partition_types1_${format_compression}_${catalog_name}_q04; + """ + + sql """ + INSERT INTO all_partition_types1_${format_compression}_${catalog_name}_q04 + SELECT id, null, tinyint_col, null, int_col, null, float_col, null FROM all_partition_types1_parquet_snappy_src where id = 3; + """ + order_qt_q04 """ select * from all_partition_types1_${format_compression}_${catalog_name}_q04; + """ + + sql """ + INSERT OVERWRITE TABLE all_partition_types1_${format_compression}_${catalog_name}_q04 + SELECT CAST(7 as INT) as id, boolean_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col FROM all_partition_types1_parquet_snappy_src where id = 3; + """ + order_qt_q05 """ select * from all_partition_types1_${format_compression}_${catalog_name}_q04; + """ + + logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_partition_types1_${format_compression}_${catalog_name}_q04; """) + hive_docker """ DROP TABLE IF EXISTS all_partition_types1_${format_compression}_${catalog_name}_q04; """ + } + + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + try { + String hms_port = context.config.otherConfigs.get("hms_port") + String hdfs_port = context.config.otherConfigs.get("hdfs_port") + String catalog_name = "test_hive_write_partitions" + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + sql """drop catalog if exists ${catalog_name}""" + sql """create catalog if not exists ${catalog_name} properties ( + 'type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}' + );""" + sql """use `${catalog_name}`.`write_test`""" + logger.info("hive sql: " + """ use `write_test` """) + hive_docker """use `write_test`""" + + sql """set enable_fallback_to_original_planner=false;""" + + for (String format_compression in format_compressions) { + logger.info("Process format_compression " + format_compression) + q01(format_compression, catalog_name) + q02(format_compression, catalog_name) + q03(format_compression, catalog_name) +// disable it temporarily +// q04(format_compression, catalog_name) + } + sql """drop catalog if exists ${catalog_name}""" + } finally { + } + } +} + + diff --git a/regression-test/suites/external_table_p0/jdbc/test_switch_catalog_and_delete_internal.groovy b/regression-test/suites/external_table_p0/jdbc/test_switch_catalog_and_delete_internal.groovy new file mode 100644 index 00000000000000..1e9e2ffdf48cbb --- /dev/null +++ b/regression-test/suites/external_table_p0/jdbc/test_switch_catalog_and_delete_internal.groovy @@ -0,0 +1,58 @@ +// 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_switch_catalog_and_delete_internal") { + String enabled = context.config.otherConfigs.get("enableJdbcTest") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-java-8.0.25.jar" + if (enabled != null && enabled.equalsIgnoreCase("true")) { + // 0.create internal db and table + String db = context.config.getDbNameByFile(new File(context.file)) + sql "drop table if exists test_switch_catalog_and_delete_internal" + sql """ + create table test_switch_catalog_and_delete_internal(pk int, a int, b int) distributed by hash(pk) buckets 10 + properties('replication_num' = '1'); + """ + + sql """ + insert into test_switch_catalog_and_delete_internal values(2,1,3),(1,1,2),(3,5,6),(6,null,6),(4,5,6); + """ + // 1.create catalog + String catalog_name = "test_switch_catalog_and_delete_internal_catalog" + sql """drop catalog if exists ${catalog_name} """ + + sql """create catalog if not exists ${catalog_name} properties( + "type"="jdbc", + "user"="root", + "password"="123456", + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}/doris_test?useSSL=false&zeroDateTimeBehavior=convertToNull", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver" + );""" + // 2.switch catalog/ refresh + sql "switch test_switch_catalog_and_delete_internal_catalog" + sql "refresh catalog test_switch_catalog_and_delete_internal_catalog" + // 3.delete table + sql "delete from internal.${db}.test_switch_catalog_and_delete_internal;" + // 4.select table + qt_test "select * from internal.maldb.test_switch_catalog_and_delete_internal;" + } + +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hive/test_hive_hudi.groovy b/regression-test/suites/external_table_p2/hive/test_hive_hudi.groovy index 75c22b26cd7773..1c69c5a738d826 100644 --- a/regression-test/suites/external_table_p2/hive/test_hive_hudi.groovy +++ b/regression-test/suites/external_table_p2/hive/test_hive_hudi.groovy @@ -49,10 +49,10 @@ suite("test_hive_hudi", "p2,external,hive,hudi") { qt_flink_hudi_catalog """select * from hudi_ctl_table order by uuid""" // incremental read for MOR table - qt_incr_mor_table """select * from incr_mor_partition@incr('beginTime'='20240312163541346')""" + order_qt_incr_mor_table """select * from incr_mor_partition@incr('beginTime'='20240312163541346')""" // incremental read for COW table - qt_inc_cow_table """select * from incr_cow_partition@incr('beginTime'='20240312164538551')""" + order_qt_inc_cow_table """select * from incr_cow_partition@incr('beginTime'='20240312164538551')""" // skip logs sql """drop catalog if exists ${catalog_name};""" diff --git a/regression-test/suites/nereids_p0/join/test_runtimefilter_on_decimal.groovy b/regression-test/suites/nereids_p0/join/test_runtimefilter_on_decimal.groovy new file mode 100644 index 00000000000000..f1bef86bbb2c4f --- /dev/null +++ b/regression-test/suites/nereids_p0/join/test_runtimefilter_on_decimal.groovy @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_runtimefilter_on_decimal", "nereids_p0") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + + // bug fix + sql "set disable_join_reorder=true;" + sql "set enable_runtime_filter_prune=false;" + sql "set runtime_filter_type='MIN_MAX';" + sql "set runtime_filter_wait_infinitely=true;" + + sql "drop table if exists decimal_rftest_l"; + sql "drop table if exists decimal_rftest_r"; + sql """ + CREATE TABLE `decimal_rftest_l` ( + `k1_dec_l` decimalv3(26, 6) + ) + DISTRIBUTED BY HASH(`k1_dec_l`) buckets 16 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql """ + CREATE TABLE `decimal_rftest_r` ( + `k1_dec_r` decimalv3(27, 6) + ) + DISTRIBUTED BY HASH(`k1_dec_r`) buckets 16 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql """ + insert into decimal_rftest_l values ("12345678901234567890.123456"); + """ + sql """ + insert into decimal_rftest_r values (null); + """ + qt_dec_rftest_1 """ + select /*+SET_VAR(parallel_pipeline_task_num=2)*/ * from decimal_rftest_l join decimal_rftest_r on k1_dec_l = k1_dec_r order by 1, 2; + """ + + sql """ + insert into decimal_rftest_l values ("-99999999999999999999.999999"), ("99999999999999999999.999999"); + """ + sql """ + insert into decimal_rftest_r values ("-99999999999999999999.999999"), ("12345678901234567890.123456"), ("99999999999999999999.999999"); + """ + qt_dec_rftest_2 """ + select /*+SET_VAR(parallel_pipeline_task_num=8)*/ * from decimal_rftest_l join decimal_rftest_r on k1_dec_l = k1_dec_r order by 1, 2; + """ +} diff --git a/regression-test/suites/nereids_rules_p0/pkfk/eliminate_inner.groovy b/regression-test/suites/nereids_rules_p0/pkfk/eliminate_inner.groovy index a2e750f3dd2980..652dc1b1250214 100644 --- a/regression-test/suites/nereids_rules_p0/pkfk/eliminate_inner.groovy +++ b/regression-test/suites/nereids_rules_p0/pkfk/eliminate_inner.groovy @@ -88,6 +88,7 @@ suite("eliminate_inner") { qt_shape "explain shape plan ${sql}" order_qt_res "${sql}" } + // not nullable check_shape_res("select fkt_not_null.* from pkt inner join fkt_not_null on pkt.pk = fkt_not_null.fk;", "simple_case") check_shape_res("select fkt_not_null.*, pkt.pk from pkt inner join fkt_not_null on pkt.pk = fkt_not_null.fk;", "with_pk_col") @@ -112,6 +113,11 @@ suite("eliminate_inner") { check_shape_res("select fkt.*, pkt.pk from pkt inner join (select fk from fkt order by fk limit 1 ) fkt on pkt.pk = fkt.fk;", "fk with limit") check_shape_res("select fkt.*, pkt.pk from pkt inner join fkt on pkt.pk = fkt.fk where pkt.pk = 1 and fkt.fk = 1;", "pk with filter that same as fk") check_shape_res("select fkt.*, pkt.pk from pkt inner join fkt on pkt.pk = fkt.fk where pkt.pk = 1 and fkt.fk = 1 and fkt.f = 1;", "pk with filter that included same as fk") - check_shape_res("select fkt.*, pkt.pk from pkt inner join fkt on pkt.pk = fkt.fk where pkt.p = 1 and fkt.fk = 1 and fkt.f = 1;;", "pk with filter that not same as fk") + check_shape_res("select fkt.*, pkt.pk from pkt inner join fkt on pkt.pk = fkt.fk where pkt.p = 1 and fkt.fk = 1 and fkt.f = 1;", "pk with filter that not same as fk") + + // Test multiple table joins, where fkt_not_null and pkt have a primary key-foreign key relationship, and fkt_not_null2 is an foreign table. + check_shape_res("select fkt_not_null.* from pkt inner join fkt_not_null on pkt.pk = fkt_not_null.fk inner join fkt_not_null as fkt_not_null2 on fkt_not_null.fk = fkt_not_null2.fk where pkt.p = 1;", "multi_table_join_with_pk_predicate") + // Test multiple table joins, where fkt_not_null and pkt have a primary key-foreign key relationship, fkt_not_null2 is an foreign table, and predicates exist in both tables. + check_shape_res("select fkt_not_null.* from pkt inner join fkt_not_null on pkt.pk = fkt_not_null.fk inner join fkt_not_null as fkt_not_null2 on fkt_not_null.fk = fkt_not_null2.fk where pkt.pk = 1;", "multi_table_join_with_pk_fk_predicate") } diff --git a/regression-test/suites/schema_change_p0/test_unique_model_schema_value_change.groovy b/regression-test/suites/schema_change_p0/test_unique_model_schema_value_change.groovy index 0f92f45e1c3810..9137e44d6a82d1 100644 --- a/regression-test/suites/schema_change_p0/test_unique_model_schema_value_change.groovy +++ b/regression-test/suites/schema_change_p0/test_unique_model_schema_value_change.groovy @@ -894,11 +894,11 @@ suite("test_unique_model_schema_value_change","p0") { //TODO Test the unique model by modify a value type from INT to DECIMAL - errorMessage = "errCode = 2, detailMessage = Can not change INT to DECIMAL32" + errorMessage = "errCode = 2, detailMessage = Can not change INT to DECIMAL128" expectException({ sql initTable sql initTableData - sql """ alter table ${tbName} MODIFY column sn_number DECIMAL """ + sql """ alter table ${tbName} MODIFY column sn_number DECIMAL(38,0) """ insertSql = "insert into ${tbName} values(123456689, 'Alice', 1.23, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " waitForSchemaChangeDone({ sql getTableStatusSql @@ -931,6 +931,18 @@ suite("test_unique_model_schema_value_change","p0") { time 60 }, insertSql, true, "${tbName}") + //Test the unique model by modify a value type from INT to VARCHAR + errorMessage="errCode = 2, detailMessage = Can not change from wider type INT to narrower type VARCHAR(2)" + expectException({ + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column sn_number VARCHAR(2) """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 'asd', 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + },errorMessage) //Test the unique model by modify a value type from INT to STRING sql initTable @@ -1060,11 +1072,11 @@ suite("test_unique_model_schema_value_change","p0") { //TODO Test the unique model by modify a value type from BIGINT to DECIMAL - errorMessage = "errCode = 2, detailMessage = Can not change BIGINT to DECIMAL32" + errorMessage = "errCode = 2, detailMessage = Can not change BIGINT to DECIMAL128" expectException({ sql initTable sql initTableData - sql """ alter table ${tbName} MODIFY column fan_number DECIMAL """ + sql """ alter table ${tbName} MODIFY column fan_number DECIMAL(38,0) """ insertSql = "insert into ${tbName} values(123456689, 'Alice', 1.23, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " waitForSchemaChangeDone({ sql getTableStatusSql @@ -1098,6 +1110,20 @@ suite("test_unique_model_schema_value_change","p0") { }, insertSql, true, "${tbName}") + //Test the unique model by modify a value type from BIGINT to VARCHAR(2) + errorMessage="errCode = 2, detailMessage = Can not change from wider type BIGINT to narrower type VARCHAR(2)" + expectException({ + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column fan_number VARCHAR(2) """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 'asd', 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + },errorMessage) + + //Test the unique model by modify a value type from BIGINT to STRING sql initTable sql initTableData @@ -1108,4 +1134,188 @@ suite("test_unique_model_schema_value_change","p0") { time 60 }, insertSql, true, "${tbName}") + /** + * Test the unique model by modify a value type from LARGEINT to other type + */ + sql """ DROP TABLE IF EXISTS ${tbName} """ + initTable = " CREATE TABLE IF NOT EXISTS ${tbName}\n" + + " (\n" + + " `user_id` LARGEINT NOT NULL COMMENT \"用户id\",\n" + + " `username` VARCHAR(50) NOT NULL COMMENT \"用户昵称\",\n" + + " `item_number` LARGEINT COMMENT \"item序列号\",\n" + + " `city` VARCHAR(20) COMMENT \"用户所在城市\",\n" + + " `age` SMALLINT COMMENT \"用户年龄\",\n" + + " `sex` TINYINT COMMENT \"用户性别\",\n" + + " `phone` LARGEINT COMMENT \"用户电话\",\n" + + " `address` VARCHAR(500) COMMENT \"用户地址\",\n" + + " `register_time` DATETIME COMMENT \"用户注册时间\"\n" + + " )\n" + + " UNIQUE KEY(`user_id`, `username`)\n" + + " DISTRIBUTED BY HASH(`user_id`) BUCKETS 1\n" + + " PROPERTIES (\n" + + " \"replication_allocation\" = \"tag.location.default: 1\",\n" + + " \"enable_unique_key_merge_on_write\" = \"true\"\n" + + " );" + + initTableData = "insert into ${tbName} values(123456789, 'Alice', 21474832641, 'Beijing', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00')," + + " (234567890, 'Bob', 21474348364, 'Shanghai', 30, 1, 13998765432, 'No. 456 Street, Shanghai', '2022-02-02 12:00:00')," + + " (345678901, 'Carol', 214742383441, 'Guangzhou', 28, 0, 13724681357, 'No. 789 Street, Guangzhou', '2022-03-03 14:00:00')," + + " (456789012, 'Dave', 21474283141, 'Shenzhen', 35, 1, 13680864279, 'No. 987 Street, Shenzhen', '2022-04-04 16:00:00')," + + " (567890123, 'Eve', 21274863141, 'Chengdu', 27, 0, 13572468091, 'No. 654 Street, Chengdu', '2022-05-05 18:00:00')," + + " (678901234, 'Frank', 21244883141, 'Hangzhou', 32, 1, 13467985213, 'No. 321 Street, Hangzhou', '2022-06-06 20:00:00')," + + " (789012345, 'Grace', 21234683141, 'Xian', 29, 0, 13333333333, 'No. 222 Street, Xian', '2022-07-07 22:00:00');" + + //TODO Test the unique model by modify a value type from LARGEINT to BOOLEAN + errorMessage = "errCode = 2, detailMessage = Can not change LARGEINT to BOOLEAN" + expectException({ + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number BOOLEAN """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', false, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + }, errorMessage) + + + // TODO Test the unique model by modify a value type from LARGEINT to TINYINT + errorMessage = "errCode = 2, detailMessage = Can not change LARGEINT to TINYINT" + expectException({ + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number TINYINT """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 2, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + }, errorMessage) + + + //Test the unique model by modify a value type from LARGEINT to SMALLINT + errorMessage = "errCode = 2, detailMessage = Can not change LARGEINT to SMALLINT" + expectException({ + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number SMALLINT """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 3, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + }, errorMessage) + + //Test the unique model by modify a value type from LARGEINT to INT + errorMessage = "errCode = 2, detailMessage = Can not change LARGEINT to INT" + expectException({ + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number INT """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 4, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + }, errorMessage) + + //Test the unique model by modify a value type from LARGEINT to BIGINT + errorMessage="errCode = 2, detailMessage = Can not change LARGEINT to BIGINT" + expectException({ + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number BIGINT """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 5, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + },errorMessage) + + + //Test the unique model by modify a value type from LARGEINT to FLOAT + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number FLOAT """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 1.2, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + + //Test the unique model by modify a value type from LARGEINT to DOUBLE + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number DOUBLE """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 1.23, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + + + //TODO Test the unique model by modify a value type from LARGEINT to DECIMAL + errorMessage = "errCode = 2, detailMessage = Can not change LARGEINT to DECIMAL128" + expectException({ + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number DECIMAL(38,0) """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 1.23, 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + + }, errorMessage) + + //TODO Test the unique model by modify a value type from LARGEINT to CHAR + errorMessage = "errCode = 2, detailMessage = Can not change LARGEINT to CHAR" + expectException({ + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number CHAR(15) """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 'asd', 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + }, errorMessage) + + + //Test the unique model by modify a value type from LARGEINT to VARCHAR + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number VARCHAR(100) """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 'asd', 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + + + //Test the unique model by modify a value type from LARGEINT to VARCHAR(2) + errorMessage="errCode = 2, detailMessage = Can not change from wider type LARGEINT to narrower type VARCHAR(2)" + expectException({ + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number VARCHAR(2) """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 'asd', 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + },errorMessage) + + + //Test the unique model by modify a value type from LARGEINT to STRING + sql initTable + sql initTableData + sql """ alter table ${tbName} MODIFY column item_number STRING """ + insertSql = "insert into ${tbName} values(123456689, 'Alice', 'asd', 'Yaan', 25, 0, 13812345678, 'No. 123 Street, Beijing', '2022-01-01 10:00:00'); " + waitForSchemaChangeDone({ + sql getTableStatusSql + time 60 + }, insertSql, true, "${tbName}") + + } diff --git a/resource/zoneinfo.tar.gz b/resource/zoneinfo.tar.gz index db010ccce99775..840cd0a50e5e5f 100644 Binary files a/resource/zoneinfo.tar.gz and b/resource/zoneinfo.tar.gz differ diff --git a/thirdparty/patches/brpc-1.4.0-fix-stream-rpc-set-connected.patch b/thirdparty/patches/brpc-1.4.0-fix-stream-rpc-set-connected.patch index 8e122a57b8f10b..4328165cfdbb91 100644 --- a/thirdparty/patches/brpc-1.4.0-fix-stream-rpc-set-connected.patch +++ b/thirdparty/patches/brpc-1.4.0-fix-stream-rpc-set-connected.patch @@ -1,20 +1,48 @@ -From 9acc6ef89d8770d5516953e2eadf0c27a7d424fc Mon Sep 17 00:00:00 2001 +From 031194784d540235dfa6ba56bd196a7aad92e30c Mon Sep 17 00:00:00 2001 From: Kaijie Chen Date: Sun, 28 Jan 2024 15:58:31 +0800 Subject: [PATCH] fix set connected for stream rpc --- - src/brpc/policy/baidu_rpc_protocol.cpp | 11 +++++------ - 1 file changed, 5 insertions(+), 6 deletions(-) + src/brpc/policy/baidu_rpc_protocol.cpp | 25 ++++++++++++++++++------- + 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/src/brpc/policy/baidu_rpc_protocol.cpp b/src/brpc/policy/baidu_rpc_protocol.cpp -index 0239960e..c47903a6 100644 +index 0239960e..e8a90e34 100644 --- a/src/brpc/policy/baidu_rpc_protocol.cpp +++ b/src/brpc/policy/baidu_rpc_protocol.cpp -@@ -234,6 +234,11 @@ void SendRpcResponse(int64_t correlation_id, +@@ -25,6 +25,7 @@ + #include "butil/iobuf.h" // butil::IOBuf + #include "butil/raw_pack.h" // RawPacker RawUnpacker + #include "brpc/controller.h" // Controller ++#include "brpc/errno.pb.h" + #include "brpc/socket.h" // Socket + #include "brpc/server.h" // Server + #include "brpc/span.h" +@@ -212,7 +213,9 @@ void SendRpcResponse(int64_t correlation_id, + if (Socket::Address(response_stream_id, &stream_ptr) == 0) { + Stream* s = (Stream*)stream_ptr->conn(); + s->FillSettings(meta.mutable_stream_settings()); +- s->SetHostSocket(sock); ++ if (s->SetHostSocket(sock) != 0) { ++ LOG(WARNING) << "SetHostSocket failed"; ++ } + } else { + LOG(WARNING) << "Stream=" << response_stream_id + << " was closed before sending response"; +@@ -234,6 +237,20 @@ void SendRpcResponse(int64_t correlation_id, // Send rpc response over stream even if server side failed to create // stream for some reasons. if(cntl->has_remote_stream()){ ++ if (sock->Failed()) { ++ LOG(WARNING) << "Fail to write into " << *sock; ++ cntl->SetFailed(EFAILEDSOCKET, "Fail to write into %s", ++ sock->description().c_str()); ++ if (stream_ptr) { ++ ((Stream *)stream_ptr->conn())->Close(); ++ } ++ return; ++ } + if(stream_ptr) { + // Now it's ok the mark this server-side stream as connectted as all the + // written user data would follower the RPC response. @@ -23,7 +51,7 @@ index 0239960e..c47903a6 100644 // Send the response over stream to notify that this stream connection // is successfully built. // Response_stream can be INVALID_STREAM_ID when error occurs. -@@ -249,12 +254,6 @@ void SendRpcResponse(int64_t correlation_id, +@@ -249,12 +266,6 @@ void SendRpcResponse(int64_t correlation_id, } return; }