diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp index 535b4884b8fa93..1734a00ad6a7e3 100644 --- a/be/src/exec/tablet_info.cpp +++ b/be/src/exec/tablet_info.cpp @@ -66,6 +66,9 @@ Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) { _is_partial_update = pschema.partial_update(); _is_strict_mode = pschema.is_strict_mode(); _timestamp_ms = pschema.timestamp_ms(); + if (pschema.has_nano_seconds()) { + _nano_seconds = pschema.nano_seconds(); + } _timezone = pschema.timezone(); for (auto& col : pschema.partial_update_input_columns()) { @@ -211,6 +214,7 @@ void OlapTableSchemaParam::to_protobuf(POlapTableSchemaParam* pschema) const { pschema->set_is_strict_mode(_is_strict_mode); pschema->set_timestamp_ms(_timestamp_ms); pschema->set_timezone(_timezone); + pschema->set_nano_seconds(_nano_seconds); for (auto col : _partial_update_input_columns) { *pschema->add_partial_update_input_columns() = col; } diff --git a/be/src/exec/tablet_info.h b/be/src/exec/tablet_info.h index d7e9aec7e51e80..8d960d626d7a89 100644 --- a/be/src/exec/tablet_info.h +++ b/be/src/exec/tablet_info.h @@ -90,6 +90,8 @@ class OlapTableSchemaParam { } void set_timestamp_ms(int64_t timestamp_ms) { _timestamp_ms = timestamp_ms; } int64_t timestamp_ms() const { return _timestamp_ms; } + void set_nano_seconds(int32_t nano_seconds) { _nano_seconds = nano_seconds; } + int32_t nano_seconds() const { return _nano_seconds; } void set_timezone(std::string timezone) { _timezone = timezone; } std::string timezone() const { return _timezone; } bool is_strict_mode() const { return _is_strict_mode; } @@ -109,6 +111,7 @@ class OlapTableSchemaParam { std::set _partial_update_input_columns; bool _is_strict_mode = false; int64_t _timestamp_ms = 0; + int32_t _nano_seconds {0}; std::string _timezone; }; diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp index 1b0eadd47c3d02..3557d750ed4a20 100644 --- a/be/src/http/action/stream_load.cpp +++ b/be/src/http/action/stream_load.cpp @@ -75,6 +75,8 @@ DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(streaming_load_requests_total, MetricUnit:: DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(streaming_load_duration_ms, MetricUnit::MILLISECONDS); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(streaming_load_current_processing, MetricUnit::REQUESTS); +bvar::LatencyRecorder g_stream_load_receive_data_latency_ms("stream_load_receive_data_latency_ms"); + static constexpr size_t MIN_CHUNK_SIZE = 64 * 1024; static const string CHUNK = "chunked"; @@ -188,8 +190,10 @@ int StreamLoadAction::on_header(HttpRequest* req) { LOG(INFO) << "new income streaming load request." << ctx->brief() << ", db=" << ctx->db << ", tbl=" << ctx->table; + ctx->begin_receive_and_read_data_cost_nanos = MonotonicNanos(); auto st = _on_header(req, ctx); + if (!st.ok()) { ctx->status = std::move(st); if (ctx->need_rollback) { @@ -340,7 +344,15 @@ void StreamLoadAction::on_chunk_data(HttpRequest* req) { } ctx->receive_bytes += remove_bytes; } - ctx->read_data_cost_nanos += (MonotonicNanos() - start_read_data_time); + int64_t read_data_time = MonotonicNanos() - start_read_data_time; + int64_t last_receive_and_read_data_cost_nanos = ctx->receive_and_read_data_cost_nanos; + ctx->read_data_cost_nanos += read_data_time; + ctx->receive_and_read_data_cost_nanos = + MonotonicNanos() - ctx->begin_receive_and_read_data_cost_nanos; + g_stream_load_receive_data_latency_ms + << (ctx->receive_and_read_data_cost_nanos - last_receive_and_read_data_cost_nanos - + read_data_time) / + 1000000; } void StreamLoadAction::free_handler_ctx(std::shared_ptr param) { diff --git a/be/src/olap/cumulative_compaction_time_series_policy.cpp b/be/src/olap/cumulative_compaction_time_series_policy.cpp index b1a8cddf20f91a..ad9d91c1d1c6c6 100644 --- a/be/src/olap/cumulative_compaction_time_series_policy.cpp +++ b/be/src/olap/cumulative_compaction_time_series_policy.cpp @@ -28,14 +28,11 @@ namespace doris { uint32_t TimeSeriesCumulativeCompactionPolicy::calc_cumulative_compaction_score(Tablet* tablet) { uint32_t score = 0; - uint32_t level0_score = 0; bool base_rowset_exist = false; const int64_t point = tablet->cumulative_layer_point(); - int64_t level0_total_size = 0; RowsetMetaSharedPtr first_meta; int64_t first_version = INT64_MAX; - std::list checked_rs_metas; // NOTE: tablet._meta_lock is hold auto& rs_metas = tablet->tablet_meta()->all_rs_metas(); // check the base rowset and collect the rowsets of cumulative part @@ -54,12 +51,6 @@ uint32_t TimeSeriesCumulativeCompactionPolicy::calc_cumulative_compaction_score( } else { // collect the rowsets of cumulative part score += rs_meta->get_compaction_score(); - if (rs_meta->compaction_level() == 0) { - level0_total_size += rs_meta->total_disk_size(); - level0_score += rs_meta->get_compaction_score(); - } else { - checked_rs_metas.push_back(rs_meta); - } } } @@ -74,60 +65,7 @@ uint32_t TimeSeriesCumulativeCompactionPolicy::calc_cumulative_compaction_score( return 0; } - // Condition 1: the size of input files for compaction meets the requirement of parameter compaction_goal_size - int64_t compaction_goal_size_mbytes = - tablet->tablet_meta()->time_series_compaction_goal_size_mbytes(); - if (level0_total_size >= compaction_goal_size_mbytes * 1024 * 1024) { - return score; - } - - // Condition 2: the number of input files reaches the threshold specified by parameter compaction_file_count_threshold - if (level0_score >= tablet->tablet_meta()->time_series_compaction_file_count_threshold()) { - return score; - } - - // Condition 3: level1 achieve compaction_goal_size - if (tablet->tablet_meta()->time_series_compaction_level_threshold() >= 2) { - checked_rs_metas.sort([](const RowsetMetaSharedPtr& a, const RowsetMetaSharedPtr& b) { - return a->version().first < b->version().first; - }); - int32_t rs_meta_count = 0; - int64_t continuous_size = 0; - for (const auto& rs_meta : checked_rs_metas) { - rs_meta_count++; - continuous_size += rs_meta->total_disk_size(); - if (rs_meta_count >= 2) { - if (continuous_size >= compaction_goal_size_mbytes * 1024 * 1024) { - return score; - } - } - } - } - - int64_t now = UnixMillis(); - int64_t last_cumu = tablet->last_cumu_compaction_success_time(); - if (last_cumu != 0) { - int64_t cumu_interval = now - last_cumu; - - // Condition 4: the time interval between compactions exceeds the value specified by parameter _compaction_time_threshold_second - if (cumu_interval > - (tablet->tablet_meta()->time_series_compaction_time_threshold_seconds() * 1000)) { - return score; - } - } else if (score > 0) { - // If the compaction process has not been successfully executed, - // the condition for triggering compaction based on the last successful compaction time (condition 3) will never be met - tablet->set_last_cumu_compaction_success_time(now); - } - - // Condition 5: If there is a continuous set of empty rowsets, prioritize merging. - auto consecutive_empty_rowsets = tablet->pick_first_consecutive_empty_rowsets( - tablet->tablet_meta()->time_series_compaction_empty_rowsets_threshold()); - if (!consecutive_empty_rowsets.empty()) { - return score; - } - - return 0; + return score; } void TimeSeriesCumulativeCompactionPolicy::calculate_cumulative_point( diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp index d19d1a8d2b6729..894c577c301105 100644 --- a/be/src/olap/delta_writer.cpp +++ b/be/src/olap/delta_writer.cpp @@ -648,11 +648,11 @@ void DeltaWriter::_build_current_tablet_schema(int64_t index_id, _tablet_schema->set_table_id(table_schema_param->table_id()); // set partial update columns info _partial_update_info = std::make_shared(); - _partial_update_info->init(*_tablet_schema, table_schema_param->is_partial_update(), - table_schema_param->partial_update_input_columns(), - table_schema_param->is_strict_mode(), - table_schema_param->timestamp_ms(), table_schema_param->timezone(), - _cur_max_version); + _partial_update_info->init( + *_tablet_schema, table_schema_param->is_partial_update(), + table_schema_param->partial_update_input_columns(), + table_schema_param->is_strict_mode(), table_schema_param->timestamp_ms(), + table_schema_param->nano_seconds(), table_schema_param->timezone(), _cur_max_version); } void DeltaWriter::_request_slave_tablet_pull_rowset(PNodeInfo node_info) { diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp index 01f3e45fd69c5b..3455fb267fd13b 100644 --- a/be/src/olap/olap_server.cpp +++ b/be/src/olap/olap_server.cpp @@ -356,21 +356,50 @@ void StorageEngine::_unused_rowset_monitor_thread_callback() { !k_doris_exit); } +int32_t StorageEngine::_auto_get_interval_by_disk_capacity(DataDir* data_dir) { + double disk_used = data_dir->get_usage(0); + double remain_used = 1 - disk_used; + DCHECK(remain_used >= 0 && remain_used <= 1); + DCHECK(config::path_gc_check_interval_second >= 0); + int32_t ret = 0; + if (remain_used > 0.9) { + // if config::path_gc_check_interval_second == 24h + ret = config::path_gc_check_interval_second; + } else if (remain_used > 0.7) { + // 12h + ret = config::path_gc_check_interval_second / 2; + } else if (remain_used > 0.5) { + // 6h + ret = config::path_gc_check_interval_second / 4; + } else if (remain_used > 0.3) { + // 4h + ret = config::path_gc_check_interval_second / 6; + } else { + // 3h + ret = config::path_gc_check_interval_second / 8; + } + return ret; +} + void StorageEngine::_path_gc_thread_callback(DataDir* data_dir) { LOG(INFO) << "try to start path gc thread!"; - int32_t interval = config::path_gc_check_interval_second; + int32_t last_exec_time = 0; do { - LOG(INFO) << "try to perform path gc!"; - data_dir->perform_path_gc(); + int32_t current_time = time(nullptr); - interval = config::path_gc_check_interval_second; + int32_t interval = _auto_get_interval_by_disk_capacity(data_dir); if (interval <= 0) { LOG(WARNING) << "path gc thread check interval config is illegal:" << interval << "will be forced set to half hour"; interval = 1800; // 0.5 hour } - } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)) && - !k_doris_exit); + if (current_time - last_exec_time >= interval) { + LOG(INFO) << "try to perform path gc! disk remain [" << 1 - data_dir->get_usage(0) + << "] internal [" << interval << "]"; + data_dir->perform_path_gc(); + last_exec_time = time(nullptr); + } + } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(5)) && !k_doris_exit); LOG(INFO) << "stop path gc thread!"; } diff --git a/be/src/olap/partial_update_info.cpp b/be/src/olap/partial_update_info.cpp index 8bb5bb97ca4706..8f5478317953be 100644 --- a/be/src/olap/partial_update_info.cpp +++ b/be/src/olap/partial_update_info.cpp @@ -26,12 +26,13 @@ namespace doris { void PartialUpdateInfo::init(const TabletSchema& tablet_schema, bool partial_update, const std::set& partial_update_cols, bool is_strict_mode, - int64_t timestamp_ms, const std::string& timezone, - int64_t cur_max_version) { + int64_t timestamp_ms, int32_t nano_seconds, + const std::string& timezone, int64_t cur_max_version) { is_partial_update = partial_update; partial_update_input_columns = partial_update_cols; max_version_in_flush_phase = cur_max_version; this->timestamp_ms = timestamp_ms; + this->nano_seconds = nano_seconds; this->timezone = timezone; missing_cids.clear(); update_cids.clear(); @@ -66,6 +67,7 @@ void PartialUpdateInfo::to_pb(PartialUpdateInfoPB* partial_update_info_pb) const can_insert_new_rows_in_partial_update); partial_update_info_pb->set_is_strict_mode(is_strict_mode); partial_update_info_pb->set_timestamp_ms(timestamp_ms); + partial_update_info_pb->set_nano_seconds(nano_seconds); partial_update_info_pb->set_timezone(timezone); for (const auto& value : default_values) { partial_update_info_pb->add_default_values(value); @@ -94,6 +96,9 @@ void PartialUpdateInfo::from_pb(PartialUpdateInfoPB* partial_update_info_pb) { is_strict_mode = partial_update_info_pb->is_strict_mode(); timestamp_ms = partial_update_info_pb->timestamp_ms(); timezone = partial_update_info_pb->timezone(); + if (partial_update_info_pb->has_nano_seconds()) { + nano_seconds = partial_update_info_pb->nano_seconds(); + } default_values.clear(); for (const auto& value : partial_update_info_pb->default_values()) { default_values.push_back(value); @@ -117,9 +122,18 @@ void PartialUpdateInfo::_generate_default_values_for_missing_cids( to_lower(tablet_schema.column(cur_cid).default_value()) .find(to_lower("CURRENT_TIMESTAMP")) != std::string::npos)) { - vectorized::DateV2Value dtv; - dtv.from_unixtime(timestamp_ms / 1000, timezone); - default_value = dtv.debug_string(); + auto pos = to_lower(tablet_schema.column(cur_cid).default_value()).find('('); + if (pos == std::string::npos) { + vectorized::DateV2Value dtv; + dtv.from_unixtime(timestamp_ms / 1000, timezone); + default_value = dtv.debug_string(); + } else { + int precision = std::stoi( + tablet_schema.column(cur_cid).default_value().substr(pos + 1)); + vectorized::DateV2Value dtv; + dtv.from_unixtime(timestamp_ms / 1000, nano_seconds, timezone, precision); + default_value = dtv.debug_string(); + } } else if (UNLIKELY(tablet_schema.column(cur_cid).type() == FieldType::OLAP_FIELD_TYPE_DATEV2 && to_lower(tablet_schema.column(cur_cid).default_value()) diff --git a/be/src/olap/partial_update_info.h b/be/src/olap/partial_update_info.h index fe85e8d19f5d79..45950d36399b13 100644 --- a/be/src/olap/partial_update_info.h +++ b/be/src/olap/partial_update_info.h @@ -28,7 +28,8 @@ class PartialUpdateInfoPB; struct PartialUpdateInfo { void init(const TabletSchema& tablet_schema, bool partial_update, const std::set& partial_update_cols, bool is_strict_mode, - int64_t timestamp_ms, const std::string& timezone, int64_t cur_max_version = -1); + int64_t timestamp_ms, int32_t nano_seconds, const std::string& timezone, + int64_t cur_max_version = -1); void to_pb(PartialUpdateInfoPB* partial_update_info) const; void from_pb(PartialUpdateInfoPB* partial_update_info); std::string summary() const; @@ -47,6 +48,7 @@ struct PartialUpdateInfo { bool can_insert_new_rows_in_partial_update {true}; bool is_strict_mode {false}; int64_t timestamp_ms {0}; + int32_t nano_seconds {0}; std::string timezone; // default values for missing cids diff --git a/be/src/olap/rowset/rowset_meta_manager.cpp b/be/src/olap/rowset/rowset_meta_manager.cpp index db0ad8f3d450c5..be1731461c261d 100644 --- a/be/src/olap/rowset/rowset_meta_manager.cpp +++ b/be/src/olap/rowset/rowset_meta_manager.cpp @@ -361,6 +361,69 @@ Status RowsetMetaManager::_get_rowset_binlog_metas(OlapMeta* meta, const TabletU return status; } +Status RowsetMetaManager::get_rowset_binlog_metas(OlapMeta* meta, TabletUid tablet_uid, + Version version, RowsetBinlogMetasPB* metas_pb) { + Status status; + auto tablet_uid_str = tablet_uid.to_string(); + auto prefix_key = make_binlog_meta_key_prefix(tablet_uid); + auto begin_key = make_binlog_meta_key_prefix(tablet_uid, version.first); + auto end_key = make_binlog_meta_key_prefix(tablet_uid, version.second + 1); + auto traverse_func = [meta, metas_pb, &status, &tablet_uid_str, &end_key]( + std::string_view key, std::string_view value) -> bool { + VLOG_DEBUG << fmt::format("get rowset binlog metas, key={}, value={}", key, value); + if (key.compare(end_key) > 0) { // the binlog meta key is binary comparable. + // All binlog meta has been scanned + return false; + } + + if (!starts_with_binlog_meta(key)) { + auto err_msg = fmt::format("invalid binlog meta key:{}", key); + status = Status::InternalError(err_msg); + LOG(WARNING) << err_msg; + return false; + } + + BinlogMetaEntryPB binlog_meta_entry_pb; + if (!binlog_meta_entry_pb.ParseFromArray(value.data(), value.size())) { + auto err_msg = fmt::format("fail to parse binlog meta value:{}", value); + status = Status::InternalError(err_msg); + LOG(WARNING) << err_msg; + return false; + } + + const auto& rowset_id = binlog_meta_entry_pb.rowset_id_v2(); + auto* binlog_meta_pb = metas_pb->add_rowset_binlog_metas(); + binlog_meta_pb->set_rowset_id(rowset_id); + binlog_meta_pb->set_version(binlog_meta_entry_pb.version()); + binlog_meta_pb->set_num_segments(binlog_meta_entry_pb.num_segments()); + binlog_meta_pb->set_meta_key(std::string {key}); + binlog_meta_pb->set_meta(std::string {value}); + + auto binlog_data_key = + make_binlog_data_key(tablet_uid_str, binlog_meta_entry_pb.version(), rowset_id); + std::string binlog_data; + status = meta->get(META_COLUMN_FAMILY_INDEX, binlog_data_key, &binlog_data); + if (!status.ok()) { + LOG(WARNING) << status.to_string(); + return false; + } + binlog_meta_pb->set_data_key(binlog_data_key); + binlog_meta_pb->set_data(binlog_data); + + return true; + }; + + Status iterStatus = + meta->iterate(META_COLUMN_FAMILY_INDEX, begin_key, prefix_key, traverse_func); + if (!iterStatus.ok()) { + LOG(WARNING) << fmt::format( + "fail to iterate binlog meta. prefix_key:{}, version:{}, status:{}", prefix_key, + version.to_string(), iterStatus.to_string()); + return iterStatus; + } + return status; +} + Status RowsetMetaManager::_get_all_rowset_binlog_metas(OlapMeta* meta, const TabletUid tablet_uid, RowsetBinlogMetasPB* metas_pb) { Status status; diff --git a/be/src/olap/rowset/rowset_meta_manager.h b/be/src/olap/rowset/rowset_meta_manager.h index d8cf9c37152762..f5b6f07767643c 100644 --- a/be/src/olap/rowset/rowset_meta_manager.h +++ b/be/src/olap/rowset/rowset_meta_manager.h @@ -66,6 +66,9 @@ class RowsetMetaManager { static Status get_rowset_binlog_metas(OlapMeta* meta, const TabletUid tablet_uid, const std::vector& binlog_versions, RowsetBinlogMetasPB* metas_pb); + // get all binlog metas of a tablet in version. + static Status get_rowset_binlog_metas(OlapMeta* meta, const TabletUid tablet_uid, + Version version, RowsetBinlogMetasPB* metas_pb); static Status remove_binlog(OlapMeta* meta, const std::string& suffix); static Status ingest_binlog_metas(OlapMeta* meta, TabletUid tablet_uid, RowsetBinlogMetasPB* metas_pb); diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp index 0fcc09cec495cb..6b97591014a25a 100644 --- a/be/src/olap/snapshot_manager.cpp +++ b/be/src/olap/snapshot_manager.cpp @@ -86,16 +86,33 @@ Status SnapshotManager::make_snapshot(const TSnapshotRequest& request, string* s return Status::Error("output parameter cannot be null"); } - TabletSharedPtr ref_tablet = + TabletSharedPtr target_tablet = StorageEngine::instance()->tablet_manager()->get_tablet(request.tablet_id); - DBUG_EXECUTE_IF("SnapshotManager::make_snapshot.inject_failure", { ref_tablet = nullptr; }) + DBUG_EXECUTE_IF("SnapshotManager::make_snapshot.inject_failure", { target_tablet = nullptr; }) - if (ref_tablet == nullptr) { + if (target_tablet == nullptr) { return Status::Error("failed to get tablet. tablet={}", request.tablet_id); } - res = _create_snapshot_files(ref_tablet, request, snapshot_path, allow_incremental_clone); + TabletSharedPtr ref_tablet = target_tablet; + if (request.__isset.ref_tablet_id) { + int64_t ref_tablet_id = request.ref_tablet_id; + TabletSharedPtr base_tablet = + StorageEngine::instance()->tablet_manager()->get_tablet(ref_tablet_id); + + // Some tasks, like medium migration, cause the target tablet and base tablet to stay on + // different disks. In this case, we fall through to the normal restore path. + // + // Otherwise, we can directly link the rowset files from the base tablet to the target tablet. + if (base_tablet != nullptr && + base_tablet->data_dir()->path() == target_tablet->data_dir()->path()) { + ref_tablet = std::move(base_tablet); + } + } + + res = _create_snapshot_files(ref_tablet, target_tablet, request, snapshot_path, + allow_incremental_clone); if (!res.ok()) { LOG(WARNING) << "failed to make snapshot. res=" << res << " tablet=" << request.tablet_id; @@ -378,6 +395,7 @@ Status check_version_continuity(const std::vector& rowsets) { } Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet, + const TabletSharedPtr& target_tablet, const TSnapshotRequest& request, string* snapshot_path, bool* allow_incremental_clone) { @@ -397,10 +415,10 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet timeout_s = request.timeout; } std::string snapshot_id_path; - res = _calc_snapshot_id_path(ref_tablet, timeout_s, &snapshot_id_path); + res = _calc_snapshot_id_path(target_tablet, timeout_s, &snapshot_id_path); if (!res.ok()) { - LOG(WARNING) << "failed to calc snapshot_id_path, ref tablet=" - << ref_tablet->data_dir()->path(); + LOG(WARNING) << "failed to calc snapshot_id_path, tablet=" + << target_tablet->data_dir()->path(); return res; } @@ -408,12 +426,12 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet // schema_full_path_desc.filepath: // /snapshot_id_path/tablet_id/schema_hash/ - auto schema_full_path = get_schema_hash_full_path(ref_tablet, snapshot_id_path); + auto schema_full_path = get_schema_hash_full_path(target_tablet, snapshot_id_path); // header_path: // /schema_full_path/tablet_id.hdr - auto header_path = _get_header_full_path(ref_tablet, schema_full_path); + auto header_path = _get_header_full_path(target_tablet, schema_full_path); // /schema_full_path/tablet_id.hdr.json - auto json_header_path = _get_json_header_full_path(ref_tablet, schema_full_path); + auto json_header_path = _get_json_header_full_path(target_tablet, schema_full_path); bool exists = true; RETURN_IF_ERROR(io::global_local_filesystem()->exists(schema_full_path, &exists)); if (exists) { @@ -425,13 +443,13 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet string snapshot_id; RETURN_IF_ERROR(io::global_local_filesystem()->canonicalize(snapshot_id_path, &snapshot_id)); + std::vector consistent_rowsets; do { TabletMetaSharedPtr new_tablet_meta(new (nothrow) TabletMeta()); if (new_tablet_meta == nullptr) { res = Status::Error("fail to malloc TabletMeta."); break; } - std::vector consistent_rowsets; DeleteBitmap delete_bitmap_snapshot(new_tablet_meta->tablet_id()); /// If set missing_version, try to get all missing version. @@ -595,7 +613,9 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet << rs->rowset_meta()->empty(); } if (!res.ok()) { - LOG(WARNING) << "fail to create hard link. [path=" << snapshot_id_path << "]"; + LOG(WARNING) << "fail to create hard link. path=" << snapshot_id_path + << " tablet=" << target_tablet->tablet_id() + << " ref tablet=" << ref_tablet->tablet_id(); break; } @@ -640,17 +660,16 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet } RowsetBinlogMetasPB rowset_binlog_metas_pb; - if (request.__isset.missing_version) { - res = ref_tablet->get_rowset_binlog_metas(request.missing_version, - &rowset_binlog_metas_pb); - } else { - std::vector missing_versions; - res = ref_tablet->get_rowset_binlog_metas(missing_versions, &rowset_binlog_metas_pb); - } - if (!res.ok()) { - break; + for (auto& rs : consistent_rowsets) { + if (!rs->is_local()) { + continue; + } + res = ref_tablet->get_rowset_binlog_metas(rs->version(), &rowset_binlog_metas_pb); + if (!res.ok()) { + break; + } } - if (rowset_binlog_metas_pb.rowset_binlog_metas_size() == 0) { + if (!res.ok() || rowset_binlog_metas_pb.rowset_binlog_metas_size() == 0) { break; } diff --git a/be/src/olap/snapshot_manager.h b/be/src/olap/snapshot_manager.h index 23b38dc302c41c..78b9db8659b3e9 100644 --- a/be/src/olap/snapshot_manager.h +++ b/be/src/olap/snapshot_manager.h @@ -76,6 +76,7 @@ class SnapshotManager { const std::vector& consistent_rowsets); Status _create_snapshot_files(const TabletSharedPtr& ref_tablet, + const TabletSharedPtr& target_tablet, const TSnapshotRequest& request, std::string* snapshot_path, bool* allow_incremental_clone); diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h index 3dc59c19bed92e..86993c7aad09aa 100644 --- a/be/src/olap/storage_engine.h +++ b/be/src/olap/storage_engine.h @@ -350,6 +350,8 @@ class StorageEngine { int _get_and_set_next_disk_index(int64 partition_id, TStorageMedium::type storage_medium); + int32_t _auto_get_interval_by_disk_capacity(DataDir* data_dir); + private: EngineOptions _options; std::mutex _store_lock; diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index b2d5f9c114d9e6..7c85e8238f8f36 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -3999,6 +3999,11 @@ Status Tablet::get_rowset_binlog_metas(const std::vector& binlog_versio binlog_versions, metas_pb); } +Status Tablet::get_rowset_binlog_metas(Version binlog_versions, RowsetBinlogMetasPB* metas_pb) { + return RowsetMetaManager::get_rowset_binlog_metas(_data_dir->get_meta(), tablet_uid(), + binlog_versions, metas_pb); +} + std::string Tablet::get_segment_filepath(std::string_view rowset_id, std::string_view segment_index) const { return fmt::format("{}/_binlog/{}_{}.dat", _tablet_path, rowset_id, segment_index); diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index 54d841b6bddb17..5ca2248c5b5ac2 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -548,6 +548,7 @@ class Tablet : public BaseTablet { std::string_view rowset_id) const; Status get_rowset_binlog_metas(const std::vector& binlog_versions, RowsetBinlogMetasPB* metas_pb); + Status get_rowset_binlog_metas(Version binlog_versions, RowsetBinlogMetasPB* metas_pb); std::string get_segment_filepath(std::string_view rowset_id, std::string_view segment_index) const; std::string get_segment_filepath(std::string_view rowset_id, int64_t segment_index) const; diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index 573ec920160120..348ab792b6f7c2 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -276,9 +276,12 @@ Status TabletManager::create_tablet(const TCreateTabletReq& request, std::vector // we need use write lock on shard-1 and then use read lock on shard-2 // if there have create rollup tablet C(assume on shard-2) from tablet D(assume on shard-1) at the same time, we will meet deadlock std::unique_lock two_tablet_lock(_two_tablet_mtx, std::defer_lock); - bool is_schema_change = request.__isset.base_tablet_id && request.base_tablet_id > 0; - bool need_two_lock = is_schema_change && ((_tablets_shards_mask & request.base_tablet_id) != - (_tablets_shards_mask & tablet_id)); + bool in_restore_mode = request.__isset.in_restore_mode && request.in_restore_mode; + bool is_schema_change_or_atomic_restore = + request.__isset.base_tablet_id && request.base_tablet_id > 0; + bool need_two_lock = + is_schema_change_or_atomic_restore && + ((_tablets_shards_mask & request.base_tablet_id) != (_tablets_shards_mask & tablet_id)); if (need_two_lock) { SCOPED_TIMER(ADD_TIMER(profile, "GetTwoTableLock")); two_tablet_lock.lock(); @@ -307,7 +310,7 @@ Status TabletManager::create_tablet(const TCreateTabletReq& request, std::vector TabletSharedPtr base_tablet = nullptr; // If the CreateTabletReq has base_tablet_id then it is a alter-tablet request - if (is_schema_change) { + if (is_schema_change_or_atomic_restore) { // if base_tablet_id's lock diffrent with new_tablet_id, we need lock it. if (need_two_lock) { SCOPED_TIMER(ADD_TIMER(profile, "GetBaseTablet")); @@ -320,22 +323,28 @@ Status TabletManager::create_tablet(const TCreateTabletReq& request, std::vector if (base_tablet == nullptr) { DorisMetrics::instance()->create_tablet_requests_failed->increment(1); return Status::Error( - "fail to create tablet(change schema), base tablet does not exist. " - "new_tablet_id={}, base_tablet_id={}", + "fail to create tablet(change schema/atomic restore), base tablet does not " + "exist. new_tablet_id={}, base_tablet_id={}", tablet_id, request.base_tablet_id); } - // If we are doing schema-change, we should use the same data dir + // If we are doing schema-change or atomic-restore, we should use the same data dir // TODO(lingbin): A litter trick here, the directory should be determined before // entering this method - if (request.storage_medium == base_tablet->data_dir()->storage_medium()) { + // + // ATTN: Since all restored replicas will be saved to HDD, so no storage_medium check here. + if (in_restore_mode || + request.storage_medium == base_tablet->data_dir()->storage_medium()) { + LOG(INFO) << "create tablet use the base tablet data dir. tablet_id=" << tablet_id + << ", base tablet_id=" << request.base_tablet_id + << ", data dir=" << base_tablet->data_dir()->path(); stores.clear(); stores.push_back(base_tablet->data_dir()); } } // set alter type to schema-change. it is useless - TabletSharedPtr tablet = _internal_create_tablet_unlocked(request, is_schema_change, - base_tablet.get(), stores, profile); + TabletSharedPtr tablet = _internal_create_tablet_unlocked( + request, is_schema_change_or_atomic_restore, base_tablet.get(), stores, profile); if (tablet == nullptr) { DorisMetrics::instance()->create_tablet_requests_failed->increment(1); return Status::Error("fail to create tablet. tablet_id={}", @@ -947,6 +956,7 @@ Status TabletManager::load_tablet_from_dir(DataDir* store, TTabletId tablet_id, if (binlog_meta_filesize > 0) { contain_binlog = true; RETURN_IF_ERROR(read_pb(binlog_metas_file, &rowset_binlog_metas_pb)); + VLOG_DEBUG << "load rowset binlog metas from file. file_path=" << binlog_metas_file; } RETURN_IF_ERROR(io::global_local_filesystem()->delete_file(binlog_metas_file)); } @@ -1049,6 +1059,7 @@ Status TabletManager::build_all_report_tablets_info(std::map t_tablet_stat.__set_remote_data_size(tablet_info.remote_data_size); t_tablet_stat.__set_row_num(tablet_info.row_count); t_tablet_stat.__set_version_count(tablet_info.version_count); + t_tablet_stat.__set_visible_version(tablet_info.version); }; for_each_tablet(handler, filter_all_tablets); diff --git a/be/src/olap/task/engine_storage_migration_task.cpp b/be/src/olap/task/engine_storage_migration_task.cpp index 239f99bb40b43a..218922069c7bd6 100644 --- a/be/src/olap/task/engine_storage_migration_task.cpp +++ b/be/src/olap/task/engine_storage_migration_task.cpp @@ -37,6 +37,7 @@ #include "olap/data_dir.h" #include "olap/olap_common.h" #include "olap/olap_define.h" +#include "olap/pb_helper.h" #include "olap/rowset/rowset_meta.h" #include "olap/snapshot_manager.h" #include "olap/storage_engine.h" @@ -258,9 +259,11 @@ Status EngineStorageMigrationTask::_migrate() { } std::vector temp_consistent_rowsets(consistent_rowsets); + RowsetBinlogMetasPB rowset_binlog_metas_pb; do { // migrate all index and data files but header file - res = _copy_index_and_data_files(full_path, temp_consistent_rowsets); + res = _copy_index_and_data_files(full_path, temp_consistent_rowsets, + &rowset_binlog_metas_pb); if (!res.ok()) { break; } @@ -288,7 +291,8 @@ Status EngineStorageMigrationTask::_migrate() { // we take the lock to complete it to avoid long-term competition with other tasks if (_is_rowsets_size_less_than_threshold(temp_consistent_rowsets)) { // force to copy the remaining data and index - res = _copy_index_and_data_files(full_path, temp_consistent_rowsets); + res = _copy_index_and_data_files(full_path, temp_consistent_rowsets, + &rowset_binlog_metas_pb); if (!res.ok()) { break; } @@ -303,6 +307,16 @@ Status EngineStorageMigrationTask::_migrate() { } } + // save rowset binlog metas + if (rowset_binlog_metas_pb.rowset_binlog_metas_size() > 0) { + auto rowset_binlog_metas_pb_filename = + fmt::format("{}/rowset_binlog_metas.pb", full_path); + res = write_pb(rowset_binlog_metas_pb_filename, rowset_binlog_metas_pb); + if (!res.ok()) { + break; + } + } + // generate new tablet meta and write to hdr file res = _gen_and_write_header_to_hdr_file(shard, full_path, consistent_rowsets, end_version); if (!res.ok()) { @@ -346,10 +360,73 @@ void EngineStorageMigrationTask::_generate_new_header( } Status EngineStorageMigrationTask::_copy_index_and_data_files( - const string& full_path, const std::vector& consistent_rowsets) const { + const string& full_path, const std::vector& consistent_rowsets, + RowsetBinlogMetasPB* all_binlog_metas_pb) const { + RowsetBinlogMetasPB rowset_binlog_metas_pb; for (const auto& rs : consistent_rowsets) { RETURN_IF_ERROR(rs->copy_files_to(full_path, rs->rowset_id())); + + Version binlog_versions = rs->version(); + RETURN_IF_ERROR(_tablet->get_rowset_binlog_metas(binlog_versions, &rowset_binlog_metas_pb)); } + + // copy index binlog files. + for (const auto& rowset_binlog_meta : rowset_binlog_metas_pb.rowset_binlog_metas()) { + auto num_segments = rowset_binlog_meta.num_segments(); + std::string_view rowset_id = rowset_binlog_meta.rowset_id(); + + RowsetMetaPB rowset_meta_pb; + if (!rowset_meta_pb.ParseFromString(rowset_binlog_meta.data())) { + auto err_msg = fmt::format("fail to parse binlog meta data value:{}", + rowset_binlog_meta.data()); + LOG(WARNING) << err_msg; + return Status::InternalError(err_msg); + } + const auto& tablet_schema_pb = rowset_meta_pb.tablet_schema(); + TabletSchema tablet_schema; + tablet_schema.init_from_pb(tablet_schema_pb); + + // copy segment files and index files + for (int64_t segment_index = 0; segment_index < num_segments; ++segment_index) { + std::string segment_file_path = _tablet->get_segment_filepath(rowset_id, segment_index); + auto snapshot_segment_file_path = + fmt::format("{}/{}_{}.binlog", full_path, rowset_id, segment_index); + + Status status = io::global_local_filesystem()->copy_path(segment_file_path, + snapshot_segment_file_path); + if (!status.ok()) { + LOG(WARNING) << "fail to copy binlog segment file. [src=" << segment_file_path + << ", dest=" << snapshot_segment_file_path << "]" << status; + return status; + } + VLOG_DEBUG << "copy " << segment_file_path << " to " << snapshot_segment_file_path; + + for (const auto& index : tablet_schema.indexes()) { + if (index.index_type() != IndexType::INVERTED) { + continue; + } + auto index_id = index.index_id(); + auto index_file = + _tablet->get_segment_index_filepath(rowset_id, segment_index, index_id); + auto snapshot_segment_index_file_path = fmt::format( + "{}/{}_{}_{}.binlog-index", full_path, rowset_id, segment_index, index_id); + VLOG_DEBUG << "copy " << index_file << " to " << snapshot_segment_index_file_path; + status = io::global_local_filesystem()->copy_path(index_file, + snapshot_segment_index_file_path); + if (!status.ok()) { + LOG(WARNING) << "fail to copy binlog index file. [src=" << index_file + << ", dest=" << snapshot_segment_index_file_path << "]" << status; + return status; + } + } + } + } + + std::move(rowset_binlog_metas_pb.mutable_rowset_binlog_metas()->begin(), + rowset_binlog_metas_pb.mutable_rowset_binlog_metas()->end(), + google::protobuf::RepeatedFieldBackInserter( + all_binlog_metas_pb->mutable_rowset_binlog_metas())); + return Status::OK(); } diff --git a/be/src/olap/task/engine_storage_migration_task.h b/be/src/olap/task/engine_storage_migration_task.h index e4fc89107c166c..f25fecbd17861c 100644 --- a/be/src/olap/task/engine_storage_migration_task.h +++ b/be/src/olap/task/engine_storage_migration_task.h @@ -20,6 +20,8 @@ #include +#include + #include #include #include @@ -72,7 +74,8 @@ class EngineStorageMigrationTask : public EngineTask { // TODO: hkp // rewrite this function Status _copy_index_and_data_files(const std::string& full_path, - const std::vector& consistent_rowsets) const; + const std::vector& consistent_rowsets, + RowsetBinlogMetasPB* all_binlog_metas_pb) const; private: // tablet to do migrated diff --git a/be/src/runtime/stream_load/stream_load_context.cpp b/be/src/runtime/stream_load/stream_load_context.cpp index f381ba097db1a5..2e9aa4fad9f2d1 100644 --- a/be/src/runtime/stream_load/stream_load_context.cpp +++ b/be/src/runtime/stream_load/stream_load_context.cpp @@ -100,6 +100,8 @@ std::string StreamLoadContext::to_json() const { writer.Int64(read_data_cost_nanos / 1000000); writer.Key("WriteDataTimeMs"); writer.Int(write_data_cost_nanos / 1000000); + writer.Key("ReceiveDataTimeMs"); + writer.Int((receive_and_read_data_cost_nanos - read_data_cost_nanos) / 1000000); writer.Key("CommitAndPublishTimeMs"); writer.Int64(commit_and_publish_txn_cost_nanos / 1000000); diff --git a/be/src/runtime/stream_load/stream_load_context.h b/be/src/runtime/stream_load/stream_load_context.h index 0e821956470901..334e9ad6af2caf 100644 --- a/be/src/runtime/stream_load/stream_load_context.h +++ b/be/src/runtime/stream_load/stream_load_context.h @@ -194,6 +194,8 @@ class StreamLoadContext { int64_t pre_commit_txn_cost_nanos = 0; int64_t read_data_cost_nanos = 0; int64_t write_data_cost_nanos = 0; + int64_t receive_and_read_data_cost_nanos = 0; + int64_t begin_receive_and_read_data_cost_nanos = 0; std::string error_url = ""; // if label already be used, set existing job's status here diff --git a/be/src/runtime/stream_load/stream_load_executor.cpp b/be/src/runtime/stream_load/stream_load_executor.cpp index 19d25e9ffa1e4a..75386d7aa6ba08 100644 --- a/be/src/runtime/stream_load/stream_load_executor.cpp +++ b/be/src/runtime/stream_load/stream_load_executor.cpp @@ -69,7 +69,7 @@ Status StreamLoadExecutor::execute_plan_fragment(std::shared_ptrstart_write_data_nanos = MonotonicNanos(); - LOG(INFO) << "begin to execute job. label=" << ctx->label << ", txn_id=" << ctx->txn_id + LOG(INFO) << "begin to execute stream load. label=" << ctx->label << ", txn_id=" << ctx->txn_id << ", query_id=" << print_id(ctx->put_result.params.params.query_id); Status st; if (ctx->put_result.__isset.params) { @@ -144,6 +144,16 @@ Status StreamLoadExecutor::execute_plan_fragment(std::shared_ptrcommit_txn(ctx.get()); } } + + LOG(INFO) << "finished to execute stream load. label=" << ctx->label + << ", txn_id=" << ctx->txn_id + << ", query_id=" << print_id(ctx->put_result.params.params.query_id) + << ", receive_data_cost_ms=" + << (ctx->receive_and_read_data_cost_nanos - + ctx->read_data_cost_nanos) / + 1000000 + << ", read_data_cost_ms=" << ctx->read_data_cost_nanos / 1000000 + << ", write_data_cost_ms=" << ctx->write_data_cost_nanos / 1000000; }); } else { st = _exec_env->fragment_mgr()->exec_plan_fragment( @@ -217,6 +227,16 @@ Status StreamLoadExecutor::execute_plan_fragment(std::shared_ptrcommit_txn(ctx.get()); } } + + LOG(INFO) << "finished to execute stream load. label=" << ctx->label + << ", txn_id=" << ctx->txn_id + << ", query_id=" << print_id(ctx->put_result.params.params.query_id) + << ", receive_data_cost_ms=" + << (ctx->receive_and_read_data_cost_nanos - + ctx->read_data_cost_nanos) / + 1000000 + << ", read_data_cost_ms=" << ctx->read_data_cost_nanos / 1000000 + << ", write_data_cost_ms=" << ctx->write_data_cost_nanos / 1000000; }); } if (!st.ok()) { diff --git a/be/src/util/block_compression.cpp b/be/src/util/block_compression.cpp index 7afbafa9790a68..1c2a65b5337749 100644 --- a/be/src/util/block_compression.cpp +++ b/be/src/util/block_compression.cpp @@ -108,11 +108,13 @@ class Lz4BlockCompression : public BlockCompressionCodec { } Status compress(const Slice& input, faststring* output) override { - if (input.size > INT_MAX) { + if (input.size > LZ4_MAX_INPUT_SIZE) { return Status::InvalidArgument( - "LZ4 not support those case(input.size>INT_MAX), maybe you should change " - "fragment_transmission_compression_codec to snappy, size={}", - input.size); + "LZ4 not support those case(input.size>LZ4_MAX_INPUT_SIZE), maybe you should " + "change " + "fragment_transmission_compression_codec to snappy, input.size={}, " + "LZ4_MAX_INPUT_SIZE={}", + input.size, LZ4_MAX_INPUT_SIZE); } Context* context; diff --git a/be/test/olap/cumulative_compaction_time_series_policy_test.cpp b/be/test/olap/cumulative_compaction_time_series_policy_test.cpp index 7c93d4d64b9927..f96e801e1d31fc 100644 --- a/be/test/olap/cumulative_compaction_time_series_policy_test.cpp +++ b/be/test/olap/cumulative_compaction_time_series_policy_test.cpp @@ -393,7 +393,7 @@ TEST_F(TestTimeSeriesCumulativeCompactionPolicy, calc_cumulative_compaction_scor const uint32_t score = _tablet->calc_compaction_score(CompactionType::CUMULATIVE_COMPACTION, cumulative_compaction_policy); - EXPECT_EQ(0, score); + EXPECT_EQ(9, score); } TEST_F(TestTimeSeriesCumulativeCompactionPolicy, calc_cumulative_compaction_score_big_rowset) { 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 ffc23900a0fa8b..8df13b1df6dfcb 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 @@ -1145,6 +1145,12 @@ public class Config extends ConfigBase { @ConfField(mutable = true, masterOnly = true) public static int max_routine_load_task_num_per_be = 1024; + /** + * routine load timeout is equal to maxBatchIntervalS * routine_load_task_timeout_multiplier. + */ + @ConfField(mutable = true, masterOnly = true) + public static int routine_load_task_timeout_multiplier = 10; + /** * the max timeout of get kafka meta. */ @@ -2461,4 +2467,8 @@ public class Config extends ConfigBase { @ConfField(mutable = true) public static boolean enable_cooldown_replica_affinity = true; + + @ConfField(mutable = true, description = { + "设置为 true,root 和 admin 将跳过 sql block rule", "Set to true, root and admin will skip SQL block rule"}) + public static boolean sql_block_rule_ignore_admin = false; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index a40f0891b0bf04..9d1dd2de3f9dc1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -720,13 +720,12 @@ public void replayModifyViewDef(AlterViewInfo alterViewInfo) throws MetaNotFound try { String viewName = view.getName(); view.setInlineViewDefWithSqlMode(inlineViewDef, alterViewInfo.getSqlMode()); - try { - view.init(); - } catch (UserException e) { - throw new DdlException("failed to init view stmt, reason=" + e.getMessage()); - } view.setNewFullSchema(newFullSchema); + // We do not need to init view here. + // During the `init` phase, some `Alter-View` statements will access the remote file system, + // but they should not access it during the metadata replay phase. + db.dropTable(viewName); db.createTable(view); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index f60352f6e6bcf7..1fd46769fbfdfe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -2693,11 +2693,7 @@ public void modifyTableLightSchemaChange(String rawSql, Database db, OlapTable o LOG.debug("logModifyTableAddOrDropInvertedIndices info:{}", info); Env.getCurrentEnv().getEditLog().logModifyTableAddOrDropInvertedIndices(info); // Drop table column stats after light schema change finished. - try { - Env.getCurrentEnv().getAnalysisManager().dropStats(olapTable); - } catch (Exception e) { - LOG.info("Failed to drop stats after light schema change. Reason: {}", e.getMessage()); - } + Env.getCurrentEnv().getAnalysisManager().dropStats(olapTable); if (isDropIndex) { // send drop rpc to be @@ -2723,11 +2719,7 @@ public void modifyTableLightSchemaChange(String rawSql, Database db, OlapTable o LOG.debug("logModifyTableAddOrDropColumns info:{}", info); Env.getCurrentEnv().getEditLog().logModifyTableAddOrDropColumns(info); // Drop table column stats after light schema change finished. - try { - Env.getCurrentEnv().getAnalysisManager().dropStats(olapTable); - } catch (Exception e) { - LOG.info("Failed to drop stats after light schema change. Reason: {}", e.getMessage()); - } + Env.getCurrentEnv().getAnalysisManager().dropStats(olapTable); } LOG.info("finished modify table's add or drop or modify columns. table: {}, job: {}, is replay: {}", olapTable.getName(), jobId, isReplay); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index 2591d3106e03f6..70799f0b4b6159 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -605,11 +605,7 @@ protected void runRunningJob() throws AlterCancelException { changeTableState(dbId, tableId, OlapTableState.NORMAL); LOG.info("set table's state to NORMAL, table id: {}, job id: {}", tableId, jobId); // Drop table column stats after schema change finished. - try { - Env.getCurrentEnv().getAnalysisManager().dropStats(tbl); - } catch (Exception e) { - LOG.info("Failed to drop stats after schema change finished. Reason: {}", e.getMessage()); - } + Env.getCurrentEnv().getAnalysisManager().dropStats(tbl); } private void onFinished(OlapTable tbl) { @@ -916,6 +912,10 @@ protected void getInfo(List> infos) { } } + public Map getIndexIdMap() { + return indexIdMap; + } + public List> getUnfinishedTasks(int limit) { List> taskInfos = Lists.newArrayList(); if (jobState == JobState.RUNNING) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java index da911c989af9e4..d74e1788fb62b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java @@ -167,6 +167,17 @@ public String getValue() { } return value; } + + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("DEFAULT "); + if (value != null) { + sb.append('"').append(value).append('"'); + } else { + sb.append("NULL"); + } + return sb.toString(); + } } // parameter initialized in constructor @@ -582,7 +593,7 @@ public String toSql() { } if (defaultValue.isSet) { - sb.append("DEFAULT \"").append(defaultValue.value).append("\" "); + sb.append(defaultValue.toSql()).append(" "); } sb.append("COMMENT \"").append(comment).append("\""); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java index fe66f0ee4cbe9c..9585a2e5069237 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java @@ -43,6 +43,7 @@ public class RestoreStmt extends AbstractBackupStmt { public static final String PROP_RESERVE_DYNAMIC_PARTITION_ENABLE = "reserve_dynamic_partition_enable"; public static final String PROP_CLEAN_TABLES = "clean_tables"; public static final String PROP_CLEAN_PARTITIONS = "clean_partitions"; + public static final String PROP_ATOMIC_RESTORE = "atomic_restore"; private boolean allowLoad = false; private ReplicaAllocation replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION; @@ -54,6 +55,7 @@ public class RestoreStmt extends AbstractBackupStmt { private boolean isBeingSynced = false; private boolean isCleanTables = false; private boolean isCleanPartitions = false; + private boolean isAtomicRestore = false; private byte[] meta = null; private byte[] jobInfo = null; @@ -121,6 +123,10 @@ public boolean isCleanPartitions() { return isCleanPartitions; } + public boolean isAtomicRestore() { + return isAtomicRestore; + } + @Override public void analyze(Analyzer analyzer) throws UserException { if (repoName.equals(Repository.KEEP_ON_LOCAL_REPO_NAME)) { @@ -199,6 +205,9 @@ public void analyzeProperties() throws AnalysisException { // is clean partitions isCleanPartitions = eatBooleanProperty(copiedProperties, PROP_CLEAN_PARTITIONS, isCleanPartitions); + // is atomic restore + isAtomicRestore = eatBooleanProperty(copiedProperties, PROP_ATOMIC_RESTORE, isAtomicRestore); + if (!copiedProperties.isEmpty()) { ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Unknown restore job properties: " + copiedProperties.keySet()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java index 8e9800fc410880..12f097729d29cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java @@ -63,7 +63,9 @@ public class ShowTableStatsStmt extends ShowStmt { new ImmutableList.Builder() .add("table_name") .add("index_name") - .add("row_count") + .add("analyze_row_count") + .add("report_row_count") + .add("report_row_count_for_nereids") .build(); private final TableName tableName; @@ -166,37 +168,33 @@ public long getTableId() { return tableId; } - public ShowResultSet constructResultSet(TableStatsMeta tableStatistic) { + public ShowResultSet constructResultSet(TableStatsMeta tableStatistic, TableIf table) { if (indexName != null) { - return constructIndexResultSet(tableStatistic); + return constructIndexResultSet(tableStatistic, table); } - return constructTableResultSet(tableStatistic); + return constructTableResultSet(tableStatistic, table); } public ShowResultSet constructEmptyResultSet() { return new ShowResultSet(getMetaData(), new ArrayList<>()); } - public ShowResultSet constructResultSet(long rowCount) { - List> result = Lists.newArrayList(); - List row = Lists.newArrayList(); - row.add(""); - row.add(""); - row.add(String.valueOf(rowCount)); - row.add(""); - row.add(""); - row.add(""); - row.add(""); - row.add(""); - result.add(row); - return new ShowResultSet(getMetaData(), result); - } - - public ShowResultSet constructTableResultSet(TableStatsMeta tableStatistic) { - DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + public ShowResultSet constructTableResultSet(TableStatsMeta tableStatistic, TableIf table) { if (tableStatistic == null) { - return new ShowResultSet(getMetaData(), new ArrayList<>()); + List> result = Lists.newArrayList(); + List row = Lists.newArrayList(); + row.add(""); + row.add(""); + row.add(String.valueOf(table.getRowCount())); + row.add(""); + row.add(""); + row.add(""); + row.add(""); + row.add(""); + result.add(row); + return new ShowResultSet(getMetaData(), result); } + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); List> result = Lists.newArrayList(); List row = Lists.newArrayList(); row.add(String.valueOf(tableStatistic.updatedRows)); @@ -215,7 +213,7 @@ public ShowResultSet constructTableResultSet(TableStatsMeta tableStatistic) { return new ShowResultSet(getMetaData(), result); } - public ShowResultSet constructIndexResultSet(TableStatsMeta tableStatistic) { + public ShowResultSet constructIndexResultSet(TableStatsMeta tableStatistic, TableIf table) { List> result = Lists.newArrayList(); if (!(table instanceof OlapTable)) { return new ShowResultSet(getMetaData(), result); @@ -225,14 +223,13 @@ public ShowResultSet constructIndexResultSet(TableStatsMeta tableStatistic) { if (indexId == null) { throw new RuntimeException(String.format("Index %s not exist.", indexName)); } - long rowCount = tableStatistic.getRowCount(olapTable.getIndexIdByName(indexName)); - if (rowCount == -1) { - return new ShowResultSet(getMetaData(), result); - } + long rowCount = tableStatistic == null ? -1 : tableStatistic.getRowCount(olapTable.getIndexIdByName(indexName)); List row = Lists.newArrayList(); row.add(table.getName()); row.add(indexName); row.add(String.valueOf(rowCount)); + row.add(String.valueOf(olapTable.getRowCountForIndex(indexId, false))); + row.add(String.valueOf(olapTable.getRowCountForIndex(indexId, true))); result.add(row); return new ShowResultSet(getMetaData(), result); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java index 144b4e49360f5f..136a13e00059c8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java @@ -457,12 +457,14 @@ private void restore(Repository repository, Database db, RestoreStmt stmt) throw db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicaAlloc(), stmt.getTimeoutMs(), metaVersion, stmt.reserveReplica(), stmt.reserveDynamicPartitionEnable(), stmt.isBeingSynced(), - stmt.isCleanTables(), stmt.isCleanPartitions(), env, Repository.KEEP_ON_LOCAL_REPO_ID, backupMeta); + stmt.isCleanTables(), stmt.isCleanPartitions(), stmt.isAtomicRestore(), + env, Repository.KEEP_ON_LOCAL_REPO_ID, backupMeta); } else { restoreJob = new RestoreJob(stmt.getLabel(), stmt.getBackupTimestamp(), db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicaAlloc(), stmt.getTimeoutMs(), stmt.getMetaVersion(), stmt.reserveReplica(), stmt.reserveDynamicPartitionEnable(), - stmt.isBeingSynced(), stmt.isCleanTables(), stmt.isCleanPartitions(), env, repository.getId()); + stmt.isBeingSynced(), stmt.isCleanTables(), stmt.isCleanPartitions(), stmt.isAtomicRestore(), + env, repository.getId()); } env.getEditLog().logRestoreJob(restoreJob); diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreFileMapping.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreFileMapping.java index 07ddf6844dc31d..f712afbb27171e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreFileMapping.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreFileMapping.java @@ -39,7 +39,7 @@ private IdChain() { } public IdChain(Long... ids) { - Preconditions.checkState(ids.length == 5); + Preconditions.checkState(ids.length == 6); chain = ids; } @@ -63,6 +63,14 @@ public long getReplicaId() { return chain[4]; } + public boolean hasRefTabletId() { + return chain.length >= 6 && chain[5] != -1L; + } + + public long getRefTabletId() { + return chain[5]; + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); @@ -78,8 +86,12 @@ public boolean equals(Object obj) { return false; } + if (((IdChain) obj).chain.length != chain.length) { + return false; + } + IdChain other = (IdChain) obj; - for (int i = 0; i < 5; i++) { + for (int i = 0; i < chain.length; i++) { // DO NOT use ==, Long_1 != Long_2 if (!chain[i].equals(other.chain[i])) { return false; @@ -92,7 +104,7 @@ public boolean equals(Object obj) { @Override public int hashCode() { int code = chain[0].hashCode(); - for (int i = 1; i < 5; i++) { + for (int i = 1; i < chain.length; i++) { code ^= chain[i].hashCode(); } return code; diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index cca381cc90d787..6e2fb3bcb6aa5c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -61,6 +61,7 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.io.Text; import org.apache.doris.common.util.DbUtil; +import org.apache.doris.common.util.DebugPointUtil; import org.apache.doris.common.util.DynamicPartitionUtil; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.TimeUtils; @@ -114,6 +115,8 @@ public class RestoreJob extends AbstractJob { private static final String PROP_IS_BEING_SYNCED = PropertyAnalyzer.PROPERTIES_IS_BEING_SYNCED; private static final String PROP_CLEAN_TABLES = RestoreStmt.PROP_CLEAN_TABLES; private static final String PROP_CLEAN_PARTITIONS = RestoreStmt.PROP_CLEAN_PARTITIONS; + private static final String PROP_ATOMIC_RESTORE = RestoreStmt.PROP_ATOMIC_RESTORE; + private static final String ATOMIC_RESTORE_TABLE_PREFIX = "__doris_atomic_restore_prefix__"; private static final Logger LOG = LogManager.getLogger(RestoreJob.class); @@ -182,6 +185,8 @@ public enum RestoreJobState { private boolean isCleanTables = false; // Whether to delete existing partitions that are not involved in the restore. private boolean isCleanPartitions = false; + // Whether to restore the data into a temp table, and then replace the origin one. + private boolean isAtomicRestore = false; // restore properties private Map properties = Maps.newHashMap(); @@ -193,7 +198,7 @@ public RestoreJob() { public RestoreJob(String label, String backupTs, long dbId, String dbName, BackupJobInfo jobInfo, boolean allowLoad, ReplicaAllocation replicaAlloc, long timeoutMs, int metaVersion, boolean reserveReplica, boolean reserveDynamicPartitionEnable, boolean isBeingSynced, boolean isCleanTables, - boolean isCleanPartitions, Env env, long repoId) { + boolean isCleanPartitions, boolean isAtomicRestore, Env env, long repoId) { super(JobType.RESTORE, label, dbId, dbName, timeoutMs, env, repoId); this.backupTimestamp = backupTs; this.jobInfo = jobInfo; @@ -210,19 +215,22 @@ public RestoreJob(String label, String backupTs, long dbId, String dbName, Backu this.isBeingSynced = isBeingSynced; this.isCleanTables = isCleanTables; this.isCleanPartitions = isCleanPartitions; + this.isAtomicRestore = isAtomicRestore; properties.put(PROP_RESERVE_REPLICA, String.valueOf(reserveReplica)); properties.put(PROP_RESERVE_DYNAMIC_PARTITION_ENABLE, String.valueOf(reserveDynamicPartitionEnable)); properties.put(PROP_IS_BEING_SYNCED, String.valueOf(isBeingSynced)); properties.put(PROP_CLEAN_TABLES, String.valueOf(isCleanTables)); properties.put(PROP_CLEAN_PARTITIONS, String.valueOf(isCleanPartitions)); + properties.put(PROP_ATOMIC_RESTORE, String.valueOf(isAtomicRestore)); } public RestoreJob(String label, String backupTs, long dbId, String dbName, BackupJobInfo jobInfo, boolean allowLoad, ReplicaAllocation replicaAlloc, long timeoutMs, int metaVersion, boolean reserveReplica, boolean reserveDynamicPartitionEnable, boolean isBeingSynced, boolean isCleanTables, - boolean isCleanPartitions, Env env, long repoId, BackupMeta backupMeta) { + boolean isCleanPartitions, boolean isAtomicRestore, Env env, long repoId, BackupMeta backupMeta) { this(label, backupTs, dbId, dbName, jobInfo, allowLoad, replicaAlloc, timeoutMs, metaVersion, reserveReplica, - reserveDynamicPartitionEnable, isBeingSynced, isCleanTables, isCleanPartitions, env, repoId); + reserveDynamicPartitionEnable, isBeingSynced, isCleanTables, isCleanPartitions, isAtomicRestore, env, + repoId); this.backupMeta = backupMeta; } @@ -387,6 +395,12 @@ public void run() { checkIfNeedCancel(); if (status.ok()) { + if (state != RestoreJobState.PENDING && label.equals( + DebugPointUtil.getDebugParamOrDefault("FE.PAUSE_NON_PENDING_RESTORE_JOB", ""))) { + LOG.info("pause restore job by debug point: {}", this); + return; + } + switch (state) { case PENDING: checkAndPrepareMeta(); @@ -494,8 +508,10 @@ private void checkAndPrepareMeta() { } Preconditions.checkNotNull(backupMeta); - // Set all restored tbls' state to RESTORE - // Table's origin state must be NORMAL and does not have unfinished load job. + // Check the olap table state. + // + // If isAtomicRestore is not set, set all restored tbls' state to RESTORE, + // the table's origin state must be NORMAL and does not have unfinished load job. for (String tableName : jobInfo.backupOlapTableObjects.keySet()) { Table tbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(tableName)); if (tbl == null) { @@ -523,6 +539,13 @@ private void checkAndPrepareMeta() { return; } + if (isAtomicRestore) { + // We will create new OlapTable in atomic restore, so does not set the RESTORE state. + // Instead, set table in atomic restore state, to forbid the alter table operation. + olapTbl.setInAtomicRestore(); + continue; + } + for (Partition partition : olapTbl.getPartitions()) { if (!env.getLoadInstance().checkPartitionLoadFinished(partition.getId(), null)) { status = new Status(ErrCode.COMMON_ERROR, @@ -584,6 +607,9 @@ private void checkAndPrepareMeta() { } } + // the new tablets -> { local tablet, schema hash, storage medium }, used in atomic restore. + Map tabletBases = new HashMap<>(); + // Check and prepare meta objects. AgentBatchTask batchTask = new AgentBatchTask(); db.readLock(); @@ -594,14 +620,15 @@ private void checkAndPrepareMeta() { Table remoteTbl = backupMeta.getTable(tableName); Preconditions.checkNotNull(remoteTbl); Table localTbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(tableName)); + if (localTbl != null && localTbl.getType() != TableType.OLAP) { + // table already exist, but is not OLAP + status = new Status(ErrCode.COMMON_ERROR, + "The type of local table should be same as type of remote table: " + + remoteTbl.getName()); + return; + } + if (localTbl != null) { - // table already exist, check schema - if (localTbl.getType() != TableType.OLAP) { - status = new Status(ErrCode.COMMON_ERROR, - "The type of local table should be same as type of remote table: " - + remoteTbl.getName()); - return; - } OlapTable localOlapTbl = (OlapTable) localTbl; OlapTable remoteOlapTbl = (OlapTable) remoteTbl; @@ -647,28 +674,26 @@ private void checkAndPrepareMeta() { PartitionItem localItem = localPartInfo.getItem(localPartition.getId()); PartitionItem remoteItem = remoteOlapTbl .getPartitionInfo().getItem(backupPartInfo.id); - if (localItem.equals(remoteItem)) { - // Same partition, same range - if (genFileMappingWhenBackupReplicasEqual(localPartInfo, localPartition, - localTbl, backupPartInfo, partitionName, tblInfo, remoteReplicaAlloc)) { - return; - } - } else { + if (!localItem.equals(remoteItem)) { // Same partition name, different range status = new Status(ErrCode.COMMON_ERROR, "Partition " + partitionName + " in table " + localTbl.getName() + " has different partition item with partition in repository"); return; } - } else { - // If this is a single partitioned table. - if (genFileMappingWhenBackupReplicasEqual(localPartInfo, localPartition, localTbl, - backupPartInfo, partitionName, tblInfo, remoteReplicaAlloc)) { - return; - } } - } else { + if (isAtomicRestore) { + // skip gen file mapping for atomic restore. + continue; + } + + // Same partition, same range or a single partitioned table. + if (genFileMappingWhenBackupReplicasEqual(localPartInfo, localPartition, + localTbl, backupPartInfo, partitionName, tblInfo, remoteReplicaAlloc)) { + return; + } + } else if (!isAtomicRestore) { // partitions does not exist PartitionInfo localPartitionInfo = localOlapTbl.getPartitionInfo(); if (localPartitionInfo.getType() == PartitionType.RANGE @@ -709,8 +734,10 @@ private void checkAndPrepareMeta() { } finally { localOlapTbl.readUnlock(); } - } else { - // Table does not exist + } + + // Table does not exist or atomic restore + if (localTbl == null || isAtomicRestore) { OlapTable remoteOlapTbl = (OlapTable) remoteTbl; // Retain only expected restore partitions in this table; Set allPartNames = remoteOlapTbl.getPartitionNames(); @@ -738,7 +765,18 @@ private void checkAndPrepareMeta() { // DO NOT set remote table's new name here, cause we will still need the origin name later // remoteOlapTbl.setName(jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); remoteOlapTbl.setState(allowLoad ? OlapTableState.RESTORE_WITH_LOAD : OlapTableState.RESTORE); - LOG.debug("put remote table {} to restoredTbls", remoteOlapTbl.getName()); + + if (isAtomicRestore && localTbl != null) { + // bind the backends and base tablets from local tbl. + status = bindLocalAndRemoteOlapTableReplicas((OlapTable) localTbl, remoteOlapTbl, tabletBases); + if (!status.ok()) { + return; + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug("put remote table {} to restoredTbls", remoteOlapTbl.getName()); + } restoredTbls.add(remoteOlapTbl); } } // end of all restore olap tables @@ -751,16 +789,23 @@ private void checkAndPrepareMeta() { if (localTbl != null) { Preconditions.checkState(localTbl.getType() == TableType.VIEW); View localView = (View) localTbl; - if (!localView.getSignature(BackupHandler.SIGNATURE_VERSION) - .equals(remoteView.getSignature(BackupHandler.SIGNATURE_VERSION))) { - status = new Status(ErrCode.COMMON_ERROR, "View " - + jobInfo.getAliasByOriginNameIfSet(backupViewName) - + " already exist but with different schema"); - return; + String localViewSignature = localView.getSignature(BackupHandler.SIGNATURE_VERSION); + // keep compatible with old version, compare the signature without reset view def + if (!localViewSignature.equals(remoteView.getSignature(BackupHandler.SIGNATURE_VERSION))) { + // reset view def to dest db name and compare signature again + String srcDbName = jobInfo.dbName; + remoteView.resetViewDefForRestore(srcDbName, db.getName()); + if (!localViewSignature.equals(remoteView.getSignature(BackupHandler.SIGNATURE_VERSION))) { + status = new Status(ErrCode.COMMON_ERROR, "View " + + jobInfo.getAliasByOriginNameIfSet(backupViewName) + + " already exist but with different schema"); + return; + } } } else { String srcDbName = jobInfo.dbName; - remoteView.resetIdsForRestore(env, srcDbName, db.getFullName()); + remoteView.resetViewDefForRestore(srcDbName, db.getName()); + remoteView.resetIdsForRestore(env); restoredTbls.add(remoteView); } } @@ -790,6 +835,9 @@ private void checkAndPrepareMeta() { // for now, nothing is modified in catalog // generate create replica tasks for all restored partitions + if (isAtomicRestore && !restoredPartitions.isEmpty()) { + throw new RuntimeException("atomic restore is set, but the restored partitions is not empty"); + } for (Pair entry : restoredPartitions) { OlapTable localTbl = (OlapTable) db.getTableNullable(entry.first); Preconditions.checkNotNull(localTbl, localTbl.getName()); @@ -809,11 +857,12 @@ private void checkAndPrepareMeta() { if (restoreTbl.getType() == TableType.OLAP) { OlapTable restoreOlapTable = (OlapTable) restoreTbl; for (Partition restorePart : restoreOlapTable.getPartitions()) { - createReplicas(db, batchTask, restoreOlapTable, restorePart); + createReplicas(db, batchTask, restoreOlapTable, restorePart, tabletBases); BackupOlapTableInfo backupOlapTableInfo = jobInfo.getOlapTableInfo(restoreOlapTable.getName()); genFileMapping(restoreOlapTable, restorePart, backupOlapTableInfo.id, backupOlapTableInfo.getPartInfo(restorePart.getName()), - !allowLoad /* if allow load, do not overwrite when commit */); + !allowLoad /* if allow load, do not overwrite when commit */, + tabletBases); } } // set restored table's new name after all 'genFileMapping' @@ -821,6 +870,9 @@ private void checkAndPrepareMeta() { if (Env.isStoredTableNamesLowerCase()) { tableName = tableName.toLowerCase(); } + if (restoreTbl.getType() == TableType.OLAP && isAtomicRestore) { + tableName = tableAliasWithAtomicRestore(tableName); + } restoreTbl.setName(tableName); } @@ -940,6 +992,90 @@ private void checkAndPrepareMeta() { // No log here, PENDING state restore job will redo this method } + private Status bindLocalAndRemoteOlapTableReplicas( + OlapTable localOlapTbl, OlapTable remoteOlapTbl, + Map tabletBases) { + localOlapTbl.readLock(); + try { + // The storage medium of the remote olap table's storage is HDD, because we want to + // restore the tables in another cluster might without SSD. + // + // Keep the storage medium of the new olap table the same as the old one, so that + // the replicas in the new olap table will not be migrated to other storage mediums. + remoteOlapTbl.setStorageMedium(localOlapTbl.getStorageMedium()); + for (Partition partition : remoteOlapTbl.getPartitions()) { + Partition localPartition = localOlapTbl.getPartition(partition.getName()); + if (localPartition == null) { + continue; + } + // Since the replicas are bound to the same disk, the storage medium must be the same + // to avoid media migration. + TStorageMedium storageMedium = localOlapTbl.getPartitionInfo() + .getDataProperty(localPartition.getId()).getStorageMedium(); + remoteOlapTbl.getPartitionInfo().getDataProperty(partition.getId()) + .setStorageMedium(storageMedium); + if (LOG.isDebugEnabled()) { + LOG.debug("bind local partition {} and remote partition {} with same storage medium {}, name: {}", + localPartition.getId(), partition.getId(), storageMedium, partition.getName()); + } + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { + String indexName = remoteOlapTbl.getIndexNameById(index.getId()); + Long localIndexId = localOlapTbl.getIndexIdByName(indexName); + MaterializedIndex localIndex = localIndexId == null ? null : localPartition.getIndex(localIndexId); + if (localIndex == null) { + continue; + } + int schemaHash = localOlapTbl.getSchemaHashByIndexId(localIndexId); + if (schemaHash == -1) { + return new Status(ErrCode.COMMON_ERROR, String.format( + "schema hash of local index %d is not found, remote table=%d, remote index=%d, " + + "local table=%d, local index=%d", localIndexId, remoteOlapTbl.getId(), index.getId(), + localOlapTbl.getId(), localIndexId)); + } + + List localTablets = localIndex.getTablets(); + List remoteTablets = index.getTablets(); + if (localTablets.size() != remoteTablets.size()) { + return new Status(ErrCode.COMMON_ERROR, String.format( + "the size of local tablet %s is not equals to the remote %s, " + + "is_atomic_restore=true, remote table=%d, remote index=%d, " + + "local table=%d, local index=%d", localTablets.size(), remoteTablets.size(), + remoteOlapTbl.getId(), index.getId(), localOlapTbl.getId(), localIndexId)); + } + for (int i = 0; i < remoteTablets.size(); i++) { + Tablet localTablet = localTablets.get(i); + Tablet remoteTablet = remoteTablets.get(i); + List localReplicas = localTablet.getReplicas(); + List remoteReplicas = remoteTablet.getReplicas(); + if (localReplicas.size() != remoteReplicas.size()) { + return new Status(ErrCode.COMMON_ERROR, String.format( + "the size of local replicas %s is not equals to the remote %s, " + + "is_atomic_restore=true, remote table=%d, remote index=%d, " + + "local table=%d, local index=%d, local replicas=%d, remote replicas=%d", + localTablets.size(), remoteTablets.size(), remoteOlapTbl.getId(), + index.getId(), localOlapTbl.getId(), localIndexId, localReplicas.size(), + remoteReplicas.size())); + } + for (int j = 0; j < remoteReplicas.size(); j++) { + long backendId = localReplicas.get(j).getBackendId(); + remoteReplicas.get(j).setBackendId(backendId); + if (LOG.isDebugEnabled()) { + LOG.debug("bind local replica {} and remote replica {} with same backend {}, table={}", + localReplicas.get(j).getId(), remoteReplicas.get(j).getId(), backendId, + localOlapTbl.getName()); + } + } + tabletBases.put(remoteTablet.getId(), + new TabletRef(localTablet.getId(), schemaHash, storageMedium)); + } + } + } + } finally { + localOlapTbl.readUnlock(); + } + return Status.OK; + } + private void prepareAndSendSnapshotTaskForOlapTable(Database db) { LOG.info("begin to make snapshot. {} when restore content is ALL", this); // begin to make snapshots for all replicas @@ -951,7 +1087,8 @@ private void prepareAndSendSnapshotTaskForOlapTable(Database db) { AgentBatchTask batchTask = new AgentBatchTask(); db.readLock(); try { - for (IdChain idChain : fileMapping.getMapping().keySet()) { + for (Map.Entry entry : fileMapping.getMapping().entrySet()) { + IdChain idChain = entry.getKey(); OlapTable tbl = (OlapTable) db.getTableNullable(idChain.getTblId()); tbl.readLock(); try { @@ -960,9 +1097,15 @@ private void prepareAndSendSnapshotTaskForOlapTable(Database db) { Tablet tablet = index.getTablet(idChain.getTabletId()); Replica replica = tablet.getReplicaById(idChain.getReplicaId()); long signature = env.getNextId(); + boolean isRestoreTask = true; + // We don't care the visible version in restore job, the end version is used. + long visibleVersion = -1L; SnapshotTask task = new SnapshotTask(null, replica.getBackendId(), signature, jobId, db.getId(), - tbl.getId(), part.getId(), index.getId(), tablet.getId(), part.getVisibleVersion(), - tbl.getSchemaHashByIndexId(index.getId()), timeoutMs, true /* is restore task*/); + tbl.getId(), part.getId(), index.getId(), tablet.getId(), visibleVersion, + tbl.getSchemaHashByIndexId(index.getId()), timeoutMs, isRestoreTask); + if (entry.getValue().hasRefTabletId()) { + task.setRefTabletId(entry.getValue().getRefTabletId()); + } batchTask.addTask(task); unfinishedSignatureToId.put(signature, tablet.getId()); bePathsMap.put(replica.getBackendId(), replica.getPathHash()); @@ -1050,6 +1193,11 @@ private boolean genFileMappingWhenBackupReplicasEqual(PartitionInfo localPartInf } private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable localTbl, Partition restorePart) { + createReplicas(db, batchTask, localTbl, restorePart, null); + } + + private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable localTbl, Partition restorePart, + Map tabletBases) { Set bfColumns = localTbl.getCopiedBfColumns(); double bfFpp = localTbl.getBfFpp(); @@ -1064,8 +1212,12 @@ private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable loc for (MaterializedIndex restoredIdx : restorePart.getMaterializedIndices(IndexExtState.VISIBLE)) { MaterializedIndexMeta indexMeta = localTbl.getIndexMetaByIndexId(restoredIdx.getId()); for (Tablet restoreTablet : restoredIdx.getTablets()) { + TabletRef baseTabletRef = tabletBases == null ? null : tabletBases.get(restoreTablet.getId()); + // All restored replicas will be saved to HDD by default. + TStorageMedium storageMedium = baseTabletRef == null + ? TStorageMedium.HDD : baseTabletRef.storageMedium; TabletMeta tabletMeta = new TabletMeta(db.getId(), localTbl.getId(), restorePart.getId(), - restoredIdx.getId(), indexMeta.getSchemaHash(), TStorageMedium.HDD); + restoredIdx.getId(), indexMeta.getSchemaHash(), storageMedium); Env.getCurrentInvertedIndex().addTablet(restoreTablet.getId(), tabletMeta); for (Replica restoreReplica : restoreTablet.getReplicas()) { Env.getCurrentInvertedIndex().addReplica(restoreTablet.getId(), restoreReplica); @@ -1074,7 +1226,7 @@ private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable loc restoreTablet.getId(), restoreReplica.getId(), indexMeta.getShortKeyColumnCount(), indexMeta.getSchemaHash(), restoreReplica.getVersion(), indexMeta.getKeysType(), TStorageType.COLUMN, - TStorageMedium.HDD /* all restored replicas will be saved to HDD */, + storageMedium, indexMeta.getSchema(), bfColumns, bfFpp, null, localTbl.getCopiedIndexes(), localTbl.isInMemory(), @@ -1096,6 +1248,12 @@ private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable loc binlogConfig, objectPool); task.setInRestoreMode(true); + if (baseTabletRef != null) { + // ensure this replica is bound to the same backend disk as the origin table's replica. + task.setBaseTablet(baseTabletRef.tabletId, baseTabletRef.schemaHash); + LOG.info("set base tablet {} for replica {} in restore job {}, tablet id={}", + baseTabletRef.tabletId, restoreReplica.getId(), jobId, restoreTablet.getId()); + } batchTask.addTask(task); } } @@ -1178,6 +1336,11 @@ protected Partition resetPartitionForRestore(OlapTable localTbl, OlapTable remot // files in repo to files in local private void genFileMapping(OlapTable localTbl, Partition localPartition, Long remoteTblId, BackupPartitionInfo backupPartInfo, boolean overwrite) { + genFileMapping(localTbl, localPartition, remoteTblId, backupPartInfo, overwrite, null); + } + + private void genFileMapping(OlapTable localTbl, Partition localPartition, Long remoteTblId, + BackupPartitionInfo backupPartInfo, boolean overwrite, Map tabletBases) { for (MaterializedIndex localIdx : localPartition.getMaterializedIndices(IndexExtState.VISIBLE)) { LOG.debug("get index id: {}, index name: {}", localIdx.getId(), localTbl.getIndexNameById(localIdx.getId())); @@ -1188,10 +1351,21 @@ private void genFileMapping(OlapTable localTbl, Partition localPartition, Long r BackupTabletInfo backupTabletInfo = backupIdxInfo.sortedTabletInfoList.get(i); LOG.debug("get tablet mapping: {} to {}, index {}", backupTabletInfo.id, localTablet.getId(), i); for (Replica localReplica : localTablet.getReplicas()) { - IdChain src = new IdChain(remoteTblId, backupPartInfo.id, backupIdxInfo.id, backupTabletInfo.id, - -1L /* no replica id */); - IdChain dest = new IdChain(localTbl.getId(), localPartition.getId(), - localIdx.getId(), localTablet.getId(), localReplica.getId()); + long refTabletId = -1L; + if (tabletBases != null && tabletBases.containsKey(localTablet.getId())) { + refTabletId = tabletBases.get(localTablet.getId()).tabletId; + if (LOG.isDebugEnabled()) { + LOG.debug("restored tablet {} is based on exists tablet {}", + localTablet.getId(), refTabletId); + } + } + + long noReplicaId = -1L; + long noRefTabletId = -1L; + IdChain src = new IdChain(remoteTblId, backupPartInfo.id, backupIdxInfo.id, + backupTabletInfo.id, noReplicaId, refTabletId); + IdChain dest = new IdChain(localTbl.getId(), localPartition.getId(), localIdx.getId(), + localTablet.getId(), localReplica.getId(), noRefTabletId); fileMapping.putMapping(dest, src, overwrite); } } @@ -1238,6 +1412,12 @@ private void replayCheckAndPrepareMeta() { OlapTable olapTbl = (OlapTable) tbl; tbl.writeLock(); try { + if (isAtomicRestore) { + // Atomic restore will creates new replica of the OlapTable. + olapTbl.setInAtomicRestore(); + continue; + } + olapTbl.setState(OlapTableState.RESTORE); // set restore status for partitions BackupOlapTableInfo tblInfo = jobInfo.backupOlapTableObjects.get(tableName); @@ -1358,7 +1538,7 @@ private void downloadSnapshots() { } private void downloadRemoteSnapshots() { - // Categorize snapshot onfos by db id. + // Categorize snapshot infos by db id. ArrayListMultimap dbToSnapshotInfos = ArrayListMultimap.create(); for (SnapshotInfo info : snapshotInfos.values()) { dbToSnapshotInfos.put(info.getDbId(), info); @@ -1453,8 +1633,9 @@ private void downloadRemoteSnapshots() { return; } + long refTabletId = -1L; // no ref tablet id IdChain catalogIds = new IdChain(tbl.getId(), part.getId(), idx.getId(), - info.getTabletId(), replica.getId()); + info.getTabletId(), replica.getId(), refTabletId); IdChain repoIds = fileMapping.get(catalogIds); if (repoIds == null) { status = new Status(ErrCode.NOT_FOUND, @@ -1596,8 +1777,9 @@ private void downloadLocalSnapshots() { return; } + long refTabletId = -1L; // no ref tablet id IdChain catalogIds = new IdChain(tbl.getId(), part.getId(), idx.getId(), - info.getTabletId(), replica.getId()); + info.getTabletId(), replica.getId(), refTabletId); IdChain repoIds = fileMapping.get(catalogIds); if (repoIds == null) { status = new Status(ErrCode.NOT_FOUND, @@ -1739,6 +1921,14 @@ private Status allTabletCommitted(boolean isReplay) { return new Status(ErrCode.NOT_FOUND, "database " + dbId + " does not exist"); } + // replace the origin tables in atomic. + if (isAtomicRestore) { + Status st = atomicReplaceOlapTables(db, isReplay); + if (!st.ok()) { + return st; + } + } + // set all restored partition version and version hash // set all tables' state to NORMAL setTableStateToNormalAndUpdateProperties(db, true, isReplay); @@ -1813,22 +2003,28 @@ private Status allTabletCommitted(boolean isReplay) { } private Status dropAllNonRestoredTableAndPartitions(Database db) { + Set restoredViews = jobInfo.newBackupObjects.views.stream() + .map(view -> view.name).collect(Collectors.toSet()); + try { for (Table table : db.getTables()) { long tableId = table.getId(); String tableName = table.getName(); TableType tableType = table.getType(); - BackupOlapTableInfo backupTableInfo = jobInfo.backupOlapTableObjects.get(tableName); - if (tableType != TableType.OLAP && tableType != TableType.ODBC && tableType != TableType.VIEW) { - continue; - } - if (tableType == TableType.OLAP && backupTableInfo != null) { - // drop the non restored partitions. - dropNonRestoredPartitions(db, (OlapTable) table, backupTableInfo); - } else if (isCleanTables) { - // otherwise drop the entire table. - LOG.info("drop non restored table {}({}). {}", tableName, tableId, this); - boolean isForceDrop = false; // move this table into recyclebin. + if (tableType == TableType.OLAP) { + BackupOlapTableInfo backupTableInfo = jobInfo.backupOlapTableObjects.get(tableName); + if (tableType == TableType.OLAP && backupTableInfo != null) { + // drop the non restored partitions. + dropNonRestoredPartitions(db, (OlapTable) table, backupTableInfo); + } else if (isCleanTables) { + // otherwise drop the entire table. + LOG.info("drop non restored table {}, table id: {}. {}", tableName, tableId, this); + boolean isForceDrop = false; // move this table into recyclebin. + env.getInternalCatalog().dropTableWithoutCheck(db, table, isForceDrop); + } + } else if (tableType == TableType.VIEW && isCleanTables && !restoredViews.contains(tableName)) { + LOG.info("drop non restored view {}, table id: {}. {}", tableName, tableId, this); + boolean isForceDrop = false; // move this view into recyclebin. env.getInternalCatalog().dropTableWithoutCheck(db, table, isForceDrop); } } @@ -1987,6 +2183,12 @@ public void cancelInternal(boolean isReplay) { // remove restored tbls for (Table restoreTbl : restoredTbls) { + if (isAtomicRestore && restoreTbl.getType() == TableType.OLAP + && !restoreTbl.getName().startsWith(ATOMIC_RESTORE_TABLE_PREFIX)) { + // In atomic restore, a table registered to db must have a name with the prefix, + // otherwise, it has not been registered and can be ignored here. + continue; + } LOG.info("remove restored table when cancelled: {}", restoreTbl.getName()); if (db.writeLockIfExist()) { try { @@ -2063,6 +2265,86 @@ public void cancelInternal(boolean isReplay) { LOG.info("finished to cancel restore job. is replay: {}. {}", isReplay, this); } + private Status atomicReplaceOlapTables(Database db, boolean isReplay) { + assert isAtomicRestore; + for (String tableName : jobInfo.backupOlapTableObjects.keySet()) { + String originName = jobInfo.getAliasByOriginNameIfSet(tableName); + if (Env.isStoredTableNamesLowerCase()) { + originName = originName.toLowerCase(); + } + String aliasName = tableAliasWithAtomicRestore(originName); + + if (!db.writeLockIfExist()) { + return Status.OK; + } + try { + Table newTbl = db.getTableNullable(aliasName); + if (newTbl == null) { + LOG.warn("replace table from {} to {}, but the temp table is not found", aliasName, originName); + return new Status(ErrCode.COMMON_ERROR, "replace table failed, the temp table " + + aliasName + " is not found"); + } + if (newTbl.getType() != TableType.OLAP) { + LOG.warn("replace table from {} to {}, but the temp table is not OLAP, it type is {}", + aliasName, originName, newTbl.getType()); + return new Status(ErrCode.COMMON_ERROR, "replace table failed, the temp table " + aliasName + + " is not OLAP table, it is " + newTbl.getType()); + } + + OlapTable originOlapTbl = null; + Table originTbl = db.getTableNullable(originName); + if (originTbl != null) { + if (originTbl.getType() != TableType.OLAP) { + LOG.warn("replace table from {} to {}, but the origin table is not OLAP, it type is {}", + aliasName, originName, originTbl.getType()); + return new Status(ErrCode.COMMON_ERROR, "replace table failed, the origin table " + + originName + " is not OLAP table, it is " + originTbl.getType()); + } + originOlapTbl = (OlapTable) originTbl; // save the origin olap table, then drop it. + } + + // replace the table. + OlapTable newOlapTbl = (OlapTable) newTbl; + newOlapTbl.writeLock(); + try { + // rename new table name to origin table name and add the new table to database. + db.dropTable(aliasName); + db.dropTable(originName); + newOlapTbl.checkAndSetName(originName, false); + db.createTable(newOlapTbl); + + // set the olap table state to normal immediately for querying + newOlapTbl.setState(OlapTableState.NORMAL); + LOG.info("atomic restore replace table {} name to {}, and set state to normal, origin table={}", + newOlapTbl.getId(), originName, originOlapTbl == null ? -1L : originOlapTbl.getId()); + } catch (DdlException e) { + LOG.warn("atomic restore replace table {} name from {} to {}", + newOlapTbl.getId(), aliasName, originName, e); + return new Status(ErrCode.COMMON_ERROR, "replace table from " + aliasName + " to " + originName + + " failed, reason=" + e.getMessage()); + } finally { + newOlapTbl.writeUnlock(); + } + + if (originOlapTbl != null) { + // The origin table is not used anymore, need to drop all its tablets. + originOlapTbl.writeLock(); + try { + LOG.info("drop the origin olap table {} by atomic restore. table={}", + originOlapTbl.getName(), originOlapTbl.getId()); + Env.getCurrentEnv().onEraseOlapTable(originOlapTbl, isReplay); + } finally { + originOlapTbl.writeUnlock(); + } + } + } finally { + db.writeUnlock(); + } + } + + return Status.OK; + } + private void setTableStateToNormalAndUpdateProperties(Database db, boolean committed, boolean isReplay) { for (String tableName : jobInfo.backupOlapTableObjects.keySet()) { Table tbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(tableName)); @@ -2083,6 +2365,10 @@ private void setTableStateToNormalAndUpdateProperties(Database db, boolean commi || olapTbl.getState() == OlapTableState.RESTORE_WITH_LOAD) { olapTbl.setState(OlapTableState.NORMAL); } + if (olapTbl.isInAtomicRestore()) { + olapTbl.clearInAtomicRestore(); + LOG.info("table {} set state from atomic restore to normal", tableName); + } BackupOlapTableInfo tblInfo = jobInfo.backupOlapTableObjects.get(tableName); for (Map.Entry partitionEntry : tblInfo.partitions.entrySet()) { @@ -2271,6 +2557,7 @@ public void readFields(DataInput in) throws IOException { isBeingSynced = Boolean.parseBoolean(properties.get(PROP_IS_BEING_SYNCED)); isCleanTables = Boolean.parseBoolean(properties.get(PROP_CLEAN_TABLES)); isCleanPartitions = Boolean.parseBoolean(properties.get(PROP_CLEAN_PARTITIONS)); + isAtomicRestore = Boolean.parseBoolean(properties.get(PROP_ATOMIC_RESTORE)); } @Override @@ -2280,5 +2567,21 @@ public String toString() { sb.append(", state: ").append(state.name()); return sb.toString(); } + + public static String tableAliasWithAtomicRestore(String tableName) { + return ATOMIC_RESTORE_TABLE_PREFIX + tableName; + } + + private static class TabletRef { + public long tabletId; + public int schemaHash; + public TStorageMedium storageMedium; + + TabletRef(long tabletId, int schemaHash, TStorageMedium storageMedium) { + this.tabletId = tabletId; + this.schemaHash = schemaHash; + this.storageMedium = storageMedium; + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/AlterJobRecord.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/AlterJobRecord.java index 36c772d5246371..51d1103530021a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/AlterJobRecord.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/AlterJobRecord.java @@ -18,10 +18,15 @@ package org.apache.doris.binlog; import org.apache.doris.alter.AlterJobV2; +import org.apache.doris.alter.SchemaChangeJobV2; import org.apache.doris.persist.gson.GsonUtils; import com.google.gson.annotations.SerializedName; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + public class AlterJobRecord { @SerializedName(value = "type") private AlterJobV2.JobType type; @@ -37,6 +42,8 @@ public class AlterJobRecord { private AlterJobV2.JobState jobState; @SerializedName(value = "rawSql") private String rawSql; + @SerializedName(value = "iim") + private Map indexIdMap; public AlterJobRecord(AlterJobV2 job) { this.type = job.getType(); @@ -46,9 +53,31 @@ public AlterJobRecord(AlterJobV2 job) { this.jobId = job.getJobId(); this.jobState = job.getJobState(); this.rawSql = job.getRawSql(); + if (type == AlterJobV2.JobType.SCHEMA_CHANGE && job instanceof SchemaChangeJobV2) { + this.indexIdMap = ((SchemaChangeJobV2) job).getIndexIdMap(); + } + } + + public boolean isJobFinished() { + return jobState == AlterJobV2.JobState.FINISHED; + } + + public boolean isSchemaChangeJob() { + return type == AlterJobV2.JobType.SCHEMA_CHANGE; + } + + public List getOriginIndexIdList() { + if (indexIdMap == null) { + return new ArrayList<>(); + } + return new ArrayList<>(indexIdMap.values()); } public String toJson() { return GsonUtils.GSON.toJson(this); } + + public static AlterJobRecord fromJson(String json) { + return GsonUtils.GSON.fromJson(json, AlterJobRecord.class); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java index fc3115e2b92da3..db49b5c2b26d95 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java @@ -225,7 +225,7 @@ public void addAlterJobV2(AlterJobV2 alterJob, long commitSeq) { AlterJobRecord alterJobRecord = new AlterJobRecord(alterJob); String data = alterJobRecord.toJson(); - addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, alterJob); + addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, alterJobRecord); } public void addModifyTableAddOrDropColumns(TableAddOrDropColumnsInfo info, long commitSeq) { @@ -383,6 +383,20 @@ public List getDroppedTables(long dbId) { } } + // get the dropped indexes of the db. + public List getDroppedIndexes(long dbId) { + lock.readLock().lock(); + try { + DBBinlog dbBinlog = dbBinlogMap.get(dbId); + if (dbBinlog == null) { + return Lists.newArrayList(); + } + return dbBinlog.getDroppedIndexes(); + } finally { + lock.readLock().unlock(); + } + } + public List gc() { LOG.info("begin gc binlog"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java index 502491004e5dab..8469bdcc7d1200 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java @@ -64,6 +64,8 @@ public class DBBinlog { private List> droppedPartitions; // The commit seq of the dropped tables private List> droppedTables; + // The commit seq of the dropped indexes + private List> droppedIndexes; private List tableDummyBinlogs; @@ -82,6 +84,7 @@ public DBBinlog(BinlogConfigCache binlogConfigCache, TBinlog binlog) { timestamps = Lists.newArrayList(); droppedPartitions = Lists.newArrayList(); droppedTables = Lists.newArrayList(); + droppedIndexes = Lists.newArrayList(); TBinlog dummy; if (binlog.getType() == TBinlogType.DUMMY) { @@ -129,6 +132,15 @@ public void recoverBinlog(TBinlog binlog, boolean dbBinlogEnable) { if (record != null && record.getTableId() > 0) { droppedTables.add(Pair.of(record.getTableId(), binlog.getCommitSeq())); } + } else if (binlog.getType() == TBinlogType.ALTER_JOB) { + AlterJobRecord record = AlterJobRecord.fromJson(binlog.data); + if (record != null && record.isSchemaChangeJob() && record.isJobFinished()) { + for (Long indexId : record.getOriginIndexIdList()) { + if (indexId != null && indexId > 0) { + droppedIndexes.add(Pair.of(indexId, binlog.getCommitSeq())); + } + } + } } if (tableIds == null) { @@ -193,6 +205,15 @@ public void addBinlog(TBinlog binlog, Object raw) { if (tableId > 0) { droppedTables.add(Pair.of(tableId, binlog.getCommitSeq())); } + } else if (binlog.getType() == TBinlogType.ALTER_JOB && raw instanceof AlterJobRecord) { + AlterJobRecord alterJobRecord = (AlterJobRecord) raw; + if (alterJobRecord.isJobFinished() && alterJobRecord.isSchemaChangeJob()) { + for (Long indexId : alterJobRecord.getOriginIndexIdList()) { + if (indexId != null && indexId > 0) { + droppedIndexes.add(Pair.of(indexId, binlog.getCommitSeq())); + } + } + } } switch (binlog.getType()) { @@ -263,6 +284,18 @@ public List getDroppedTables() { } } + // Get the dropped indexes of the db. + public List getDroppedIndexes() { + lock.readLock().lock(); + try { + return droppedIndexes.stream() + .map(v -> v.first) + .collect(Collectors.toList()); + } finally { + lock.readLock().unlock(); + } + } + public Pair getBinlogLag(long tableId, long prevCommitSeq) { TStatus status = new TStatus(TStatusCode.OK); lock.readLock().lock(); @@ -380,7 +413,7 @@ private void removeExpiredMetaData(long largestExpiredCommitSeq) { } } - gcDroppedPartitionAndTables(largestExpiredCommitSeq); + gcDroppedResources(largestExpiredCommitSeq); if (lastCommitSeq != -1) { dummy.setCommitSeq(lastCommitSeq); } @@ -418,7 +451,7 @@ private TBinlog getLastExpiredBinlog(BinlogComparator checker) { timeIter.remove(); } - gcDroppedPartitionAndTables(lastExpiredBinlog.getCommitSeq()); + gcDroppedResources(lastExpiredBinlog.getCommitSeq()); } return lastExpiredBinlog; @@ -528,7 +561,7 @@ public void dbBinlogDisableReplayGc(BinlogTombstone tombstone) { } } - private void gcDroppedPartitionAndTables(long commitSeq) { + private void gcDroppedResources(long commitSeq) { Iterator> iter = droppedPartitions.iterator(); while (iter.hasNext() && iter.next().second < commitSeq) { iter.remove(); @@ -537,6 +570,10 @@ private void gcDroppedPartitionAndTables(long commitSeq) { while (iter.hasNext() && iter.next().second < commitSeq) { iter.remove(); } + iter = droppedIndexes.iterator(); + while (iter.hasNext() && iter.next().second < commitSeq) { + iter.remove(); + } } // not thread safety, do this without lock diff --git a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRule.java b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRule.java index 87d1830a170c78..41eecf9492533a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRule.java @@ -22,6 +22,7 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.SqlBlockUtil; +import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; import com.google.common.collect.Lists; @@ -37,7 +38,7 @@ /** * Use for block some sql by rule. **/ -public class SqlBlockRule implements Writable { +public class SqlBlockRule implements Writable, GsonPostProcessable { public static final String NAME_TYPE = "SQL BLOCK RULE NAME"; @@ -191,11 +192,14 @@ public void write(DataOutput out) throws IOException { **/ public static SqlBlockRule read(DataInput in) throws IOException { String json = Text.readString(in); - SqlBlockRule sqlBlockRule = GsonUtils.GSON.fromJson(json, SqlBlockRule.class); - if (StringUtils.isNotEmpty(sqlBlockRule.getSql()) && !SqlBlockUtil.STRING_DEFAULT.equals( - sqlBlockRule.getSql())) { - sqlBlockRule.setSqlPattern(Pattern.compile(sqlBlockRule.getSql())); + return GsonUtils.GSON.fromJson(json, SqlBlockRule.class); + } + + @Override + public void gsonPostProcess() { + if (StringUtils.isNotEmpty(this.getSql()) && !SqlBlockUtil.STRING_DEFAULT.equals( + this.getSql())) { + this.setSqlPattern(Pattern.compile(this.getSql())); } - return sqlBlockRule; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java index 1bc5505edfd291..02e0db4ff5cbfe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java @@ -23,12 +23,14 @@ import org.apache.doris.analysis.ShowSqlBlockRuleStmt; import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.UserException; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.SqlBlockUtil; import org.apache.doris.metric.MetricRepo; +import org.apache.doris.mysql.privilege.Auth; import org.apache.doris.persist.gson.GsonUtils; import com.google.common.collect.Lists; @@ -224,6 +226,9 @@ public void unprotectedDrop(List ruleNames) { * Match SQL according to rules. **/ public void matchSql(String originSql, String sqlHash, String user) throws AnalysisException { + if (Config.sql_block_rule_ignore_admin && (Auth.ROOT_USER.equals(user) || Auth.ADMIN_USER.equals(user))) { + return; + } // match global rule List globalRules = nameToSqlBlockRuleMap.values().stream().filter(SqlBlockRule::getGlobal).collect(Collectors.toList()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index 3dc7da3e0a0c8d..527e22cbecdbce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -976,4 +976,10 @@ public boolean isMaterializedViewColumn() { return getName().startsWith(CreateMaterializedViewStmt.MATERIALIZED_VIEW_NAME_PREFIX) || getName().startsWith(CreateMaterializedViewStmt.MATERIALIZED_VIEW_AGGREGATE_NAME_PREFIX); } + + public void setDefaultValueInfo(Column refColumn) { + this.defaultValue = refColumn.defaultValue; + this.defaultValueExprDef = refColumn.defaultValueExprDef; + this.realDefaultValue = refColumn.realDefaultValue; + } } 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 91154b8d76c18e..f7148d45390a72 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 @@ -78,6 +78,7 @@ import org.apache.doris.analysis.TruncateTableStmt; import org.apache.doris.analysis.UninstallPluginStmt; import org.apache.doris.backup.BackupHandler; +import org.apache.doris.backup.RestoreJob; import org.apache.doris.binlog.BinlogGcer; import org.apache.doris.binlog.BinlogManager; import org.apache.doris.blockrule.SqlBlockRuleMgr; @@ -924,8 +925,7 @@ public void initialize(String[] args) throws Exception { createTxnCleaner(); // 6. start state listener thread - createStateListener(); - listener.start(); + startStateListener(); if (!Config.edit_log_type.equalsIgnoreCase("bdb")) { // If not using bdb, we need to notify the FE type transfer manually. @@ -1478,11 +1478,22 @@ void advanceNextId() { * 2. register some hook. * If there is, add them here. */ - public void postProcessAfterMetadataReplayed(boolean waitCatalogReady) { + public boolean postProcessAfterMetadataReplayed(boolean waitCatalogReady) { if (waitCatalogReady) { while (!isReady()) { + // Avoid endless waiting if the state has changed. + // + // Consider the following situation: + // 1. The follower replay journals and is not set to ready because the synchronization internval + // exceeds meta delay toleration seconds. + // 2. The underlying BEBJE node of this follower is selected as the master, but the state listener + // thread is waiting for catalog ready. + if (typeTransferQueue.peek() != null) { + return false; + } + try { - Thread.sleep(10 * 1000); + Thread.sleep(100); } catch (InterruptedException e) { LOG.warn("", e); } @@ -1491,6 +1502,7 @@ public void postProcessAfterMetadataReplayed(boolean waitCatalogReady) { auth.rectifyPrivs(); catalogMgr.registerCatalogRefreshListener(this); + return true; } // start all daemon threads only running on Master @@ -1608,7 +1620,10 @@ private void transferToNonMaster(FrontendNodeType newType) { } // 'isReady' will be set to true in 'setCanRead()' method - postProcessAfterMetadataReplayed(true); + if (!postProcessAfterMetadataReplayed(true)) { + // the state has changed, exit early. + return; + } checkLowerCaseTableNames(); @@ -2465,7 +2480,7 @@ public void notifyNewFETypeTransfer(FrontendNodeType newType) { } } - public void createStateListener() { + public void startStateListener() { listener = new Daemon("stateListener", STATE_CHANGE_CHECK_INTERVAL_MS) { @Override protected synchronized void runOneCycle() { @@ -2573,6 +2588,7 @@ protected synchronized void runOneCycle() { }; listener.setMetaContext(metaContext); + listener.start(); } public synchronized boolean replayJournal(long toJournalId) { @@ -3308,6 +3324,10 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_ENABLE_MOW_LIGHT_DELETE).append("\" = \""); sb.append(olapTable.getEnableMowLightDelete()).append("\""); + if (olapTable.isInAtomicRestore()) { + sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_IN_ATOMIC_RESTORE).append("\" = \"true\""); + } + sb.append("\n)"); } else if (table.getType() == TableType.MYSQL) { MysqlTable mysqlTable = (MysqlTable) table; @@ -4132,6 +4152,9 @@ public void renameTable(Database db, Table table, TableRenameClause tableRenameC if (db.getTable(newTableName).isPresent()) { throw new DdlException("Table name[" + newTableName + "] is already used"); } + if (db.getTable(RestoreJob.tableAliasWithAtomicRestore(newTableName)).isPresent()) { + throw new DdlException("Table name[" + newTableName + "] is already used (in restoring)"); + } if (table.getType() == TableType.OLAP) { // olap table should also check if any rollup has same name as "newTableName" @@ -4528,11 +4551,7 @@ private void renameColumn(Database db, OlapTable table, String colName, indexIdToSchemaVersion); editLog.logColumnRename(info); LOG.info("rename coloumn[{}] to {}", colName, newColName); - try { - Env.getCurrentEnv().getAnalysisManager().dropStats(table); - } catch (Exception e) { - LOG.info("Failed to drop stats after rename column. Reason: {}", e.getMessage()); - } + Env.getCurrentEnv().getAnalysisManager().dropStats(table); } } @@ -5620,6 +5639,7 @@ public static TGetMetaResult getMeta(Database db, List tables) throws Met BinlogManager binlogManager = Env.getCurrentEnv().getBinlogManager(); dbMeta.setDroppedPartitions(binlogManager.getDroppedPartitions(db.getId())); dbMeta.setDroppedTables(binlogManager.getDroppedTables(db.getId())); + dbMeta.setDroppedIndexes(binlogManager.getDroppedIndexes(db.getId())); } result.setDbMeta(dbMeta); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java index 094aebc6fc3008..585d18759254e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndex.java @@ -73,6 +73,8 @@ public enum IndexExtState { @SerializedName(value = "rollupFinishedVersion") private long rollupFinishedVersion; + private boolean rowCountReported = false; + public MaterializedIndex() { this.state = IndexState.NORMAL; this.idToTablets = new HashMap<>(); @@ -206,6 +208,14 @@ public int getTabletOrderIdx(long tabletId) { return -1; } + public void setRowCountReported(boolean reported) { + this.rowCountReported = reported; + } + + public boolean getRowCountReported() { + return this.rowCountReported; + } + @Override public void write(DataOutput out) throws IOException { super.write(out); @@ -225,6 +235,7 @@ public void write(DataOutput out) throws IOException { out.writeLong(rollupFinishedVersion); } + @Deprecated public void readFields(DataInput in) throws IOException { super.readFields(in); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java index 23b0a353366aaa..4ab0c536aecc37 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java @@ -18,6 +18,7 @@ package org.apache.doris.catalog; import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.CastExpr; import org.apache.doris.analysis.CreateMaterializedViewStmt; import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.SlotRef; @@ -202,6 +203,24 @@ private void setColumnsDefineExpr(Map columnNameToDefineExpr) thro } } + boolean isCastSlot = + entry.getValue() instanceof CastExpr && entry.getValue().getChild(0) instanceof SlotRef; + + // Compatibility code for older versions of mv + // old version: + // goods_number -> mva_SUM__CAST(`goods_number` AS BIGINT) + // new version: + // goods_number -> mva_SUM__CAST(`goods_number` AS bigint) + if (isCastSlot && !match) { + for (Column column : schema) { + if (column.getName().equalsIgnoreCase(entry.getKey())) { + column.setDefineExpr(entry.getValue()); + match = true; + break; + } + } + } + if (!match) { // Compatibility code for older versions of mv // store_id -> mv_store_id diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 8595c549595e99..a84e8a09368192 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -591,7 +591,7 @@ public Status resetIdsForRestore(Env env, Database db, ReplicaAllocation restore baseIndexId = newIdxId; } MaterializedIndexMeta indexMeta = origIdxIdToMeta.get(entry.getKey()); - indexMeta.resetIndexIdForRestore(newIdxId, srcDbName, db.getFullName()); + indexMeta.resetIndexIdForRestore(newIdxId, srcDbName, db.getName()); indexIdToMeta.put(newIdxId, indexMeta); indexNameToId.put(entry.getValue(), newIdxId); } @@ -1100,7 +1100,7 @@ public void setSequenceMapCol(String colName) { getOrCreatTableProperty().setSequenceMapCol(colName); } - public void setSequenceInfo(Type type) { + public void setSequenceInfo(Type type, Column refColumn) { this.hasSequenceCol = true; this.sequenceType = type; @@ -1114,6 +1114,9 @@ public void setSequenceInfo(Type type) { // unique key table sequenceCol = ColumnDef.newSequenceColumnDef(type, AggregateType.REPLACE).toColumn(); } + if (refColumn != null) { + sequenceCol.setDefaultValueInfo(refColumn); + } // add sequence column at last fullSchema.add(sequenceCol); nameToColumn.put(Column.SEQUENCE_COL, sequenceCol); @@ -1271,22 +1274,51 @@ public Map> findReAnalyzeNeededPartitions() { @Override public long fetchRowCount() { - long rowCount = 0; - for (Map.Entry entry : idToPartition.entrySet()) { - rowCount += entry.getValue().getBaseIndex().getRowCount(); - } - return rowCount; + return getRowCountForIndex(baseIndexId, false); } - public long getRowCountForIndex(long indexId) { + /** + * @return If strict is true, -1 if there are some tablets whose row count is not reported to FE + * If strict is false, return the sum of all partition's index current reported row count. + */ + public long getRowCountForIndex(long indexId, boolean strict) { long rowCount = 0; for (Map.Entry entry : idToPartition.entrySet()) { MaterializedIndex index = entry.getValue().getIndex(indexId); - rowCount += index == null ? 0 : index.getRowCount(); + if (index == null) { + LOG.warn("Index {} not exist in partition {}, table {}, {}", + indexId, entry.getValue().getName(), id, name); + return -1; + } + if (strict && !index.getRowCountReported()) { + return -1; + } + rowCount += index.getRowCount() == -1 ? 0 : index.getRowCount(); } return rowCount; } + /** + * @return If strict is true, -1 if there are some tablets whose row count is not reported to FE. + * If strict is false, return the sum of partition's all indexes current reported row count. + */ + public long getRowCountForPartitionIndex(long partitionId, long indexId, boolean strict) { + Partition partition = idToPartition.get(partitionId); + if (partition == null) { + LOG.warn("Partition {} not exist in table {}, {}", partitionId, id, name); + return -1; + } + MaterializedIndex index = partition.getIndex(indexId); + if (index == null) { + LOG.warn("Index {} not exist in partition {}, table {}, {}", indexId, partitionId, id, name); + return -1; + } + if (strict && !index.getRowCountReported()) { + return -1; + } + return index.getRowCount() == -1 ? 0 : index.getRowCount(); + } + @Override public long getAvgRowLength() { long rowCount = 0; @@ -1581,6 +1613,18 @@ public void readFields(DataInput in) throws IOException { defaultDistributionInfo.markAutoBucket(); } + if (isUniqKeyMergeOnWrite() && getSequenceMapCol() != null) { + // set the hidden sequence column's default value the same with + // the sequence map column's for partial update + String seqMapColName = getSequenceMapCol(); + Column seqMapCol = getBaseSchema().stream().filter(col -> col.getName().equalsIgnoreCase(seqMapColName)) + .findFirst().orElse(null); + Column hiddenSeqCol = getSequenceCol(); + if (seqMapCol != null && hiddenSeqCol != null) { + hiddenSeqCol.setDefaultValueInfo(seqMapCol); + } + } + // temp partitions tempPartitions = TempPartitions.read(in); RangePartitionInfo tempRangeInfo = tempPartitions.getPartitionInfo(); @@ -1722,6 +1766,10 @@ public void checkNormalStateForAlter() throws DdlException { if (state != OlapTableState.NORMAL) { throw new DdlException("Table[" + name + "]'s state is not NORMAL. Do not allow doing ALTER ops"); } + if (tableProperty != null && tableProperty.isInAtomicRestore()) { + throw new DdlException("Table[" + name + "] is in atomic restore state. " + + "Do not allow doing ALTER ops"); + } } public boolean isStable(SystemInfoService infoService, TabletScheduler tabletScheduler, String clusterName) { @@ -1959,6 +2007,21 @@ public String getEstimatePartitionSize() { return ""; } + public void setInAtomicRestore() { + getOrCreatTableProperty().setInAtomicRestore().buildInAtomicRestore(); + } + + public void clearInAtomicRestore() { + getOrCreatTableProperty().clearInAtomicRestore().buildInAtomicRestore(); + } + + public boolean isInAtomicRestore() { + if (tableProperty != null) { + return tableProperty.isInAtomicRestore(); + } + return false; + } + public boolean getEnableLightSchemaChange() { if (tableProperty != null) { return tableProperty.getUseSchemaLightChange(); @@ -2350,6 +2413,10 @@ public boolean getEnableUniqueKeyMergeOnWrite() { return tableProperty.getEnableUniqueKeyMergeOnWrite(); } + public boolean isUniqKeyMergeOnWrite() { + return getKeysType() == KeysType.UNIQUE_KEYS && getEnableUniqueKeyMergeOnWrite(); + } + public boolean isDuplicateWithoutKey() { return getKeysType() == KeysType.DUP_KEYS && getKeysNum() == 0; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java index 70dffaa16ecc14..2c03f7c805df2e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java @@ -152,6 +152,8 @@ public enum ReplicaStatus { private long userDropTime = -1; + private long lastReportVersion = 0; + public Replica() { } @@ -731,4 +733,12 @@ public boolean isScheduleAvailable() { return Env.getCurrentSystemInfo().checkBackendScheduleAvailable(backendId) && !isUserDrop(); } + + public void setLastReportVersion(long version) { + this.lastReportVersion = version; + } + + public long getLastReportVersion() { + return lastReportVersion; + } } 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 996b4a8a0c533f..2904dda1efe870 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 @@ -59,6 +59,7 @@ public class TableProperty implements Writable { private DynamicPartitionProperty dynamicPartitionProperty = new DynamicPartitionProperty(Maps.newHashMap()); private ReplicaAllocation replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION; private boolean isInMemory = false; + private boolean isInAtomicRestore = false; private String storagePolicy = ""; private Boolean isBeingSynced = null; @@ -190,6 +191,26 @@ public TableProperty buildInMemory() { return this; } + public TableProperty buildInAtomicRestore() { + isInAtomicRestore = Boolean.parseBoolean(properties.getOrDefault( + PropertyAnalyzer.PROPERTIES_IN_ATOMIC_RESTORE, "false")); + return this; + } + + public boolean isInAtomicRestore() { + return isInAtomicRestore; + } + + public TableProperty setInAtomicRestore() { + properties.put(PropertyAnalyzer.PROPERTIES_IN_ATOMIC_RESTORE, "true"); + return this; + } + + public TableProperty clearInAtomicRestore() { + properties.remove(PropertyAnalyzer.PROPERTIES_IN_ATOMIC_RESTORE); + return this; + } + public TableProperty buildEnableLightSchemaChange() { enableLightSchemaChange = Boolean.parseBoolean( properties.getOrDefault(PropertyAnalyzer.PROPERTIES_ENABLE_LIGHT_SCHEMA_CHANGE, "false")); @@ -574,7 +595,8 @@ public static TableProperty read(DataInput in) throws IOException { .buildDisableAutoCompaction() .buildEnableSingleReplicaCompaction() .buildTimeSeriesCompactionEmptyRowsetsThreshold() - .buildTimeSeriesCompactionLevelThreshold(); + .buildTimeSeriesCompactionLevelThreshold() + .buildInAtomicRestore(); if (Env.getCurrentEnvJournalVersion() < FeMetaVersion.VERSION_105) { // get replica num from property map and create replica allocation String repNum = tableProperty.properties.remove(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java index 092bf84a6617db..00b3bcfca87178 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java @@ -113,17 +113,41 @@ protected void runAfterCatalogReady() { long version = partition.getVisibleVersion(); for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { long indexRowCount = 0L; + boolean indexReported = true; for (Tablet tablet : index.getTablets()) { long tabletRowCount = 0L; + boolean tabletReported = false; for (Replica replica : tablet.getReplicas()) { + LOG.debug("Table {} replica {} current version {}, report version {}", + olapTable.getName(), replica.getId(), + replica.getVersion(), replica.getLastReportVersion()); if (replica.checkVersionCatchUp(version, false) - && replica.getRowCount() > tabletRowCount) { + && replica.getRowCount() >= tabletRowCount) { + // 1. If replica version and reported replica version are all equal to + // PARTITION_INIT_VERSION, set tabletReported to true, which indicates this + // tablet is empty for sure when previous report. + // 2. If last report version is larger than PARTITION_INIT_VERSION, set + // tabletReported to true as well. That is, we only guarantee all replicas of + // the tablet are reported for the init version. + // e.g. When replica version is 2, but last reported version is 1, + // tabletReported would be false. + if (replica.getVersion() == Partition.PARTITION_INIT_VERSION + && replica.getLastReportVersion() == Partition.PARTITION_INIT_VERSION + || replica.getLastReportVersion() > Partition.PARTITION_INIT_VERSION) { + tabletReported = true; + } tabletRowCount = replica.getRowCount(); } } indexRowCount += tabletRowCount; + // Only when all tablets of this index are reported, we set indexReported to true. + indexReported = indexReported && tabletReported; } // end for tablets + index.setRowCountReported(indexReported); index.setRowCount(indexRowCount); + LOG.debug("Table {} index {} all tablets reported[{}], row count {}", + olapTable.getName(), olapTable.getIndexNameById(index.getId()), + indexReported, indexRowCount); } // end for indices } // end for partitions LOG.debug("finished to set row num for table: {} in database: {}", @@ -148,6 +172,9 @@ private void updateTabletStat(Long beId, TTabletStatResult result) { replica.setRemoteDataSize(stat.getRemoteDataSize()); replica.setRowCount(stat.getRowNum()); replica.setVersionCount(stat.getVersionCount()); + // Older version BE doesn't set visible version. Set it to max for compatibility. + replica.setLastReportVersion(stat.isSetVisibleVersion() ? stat.getVisibleVersion() + : Long.MAX_VALUE); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/View.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/View.java index 0919ce6c80dfeb..870912906cd85c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/View.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/View.java @@ -223,7 +223,14 @@ public String getSignature(int signatureVersion) { sb.append(type); sb.append(Util.getSchemaSignatureString(fullSchema)); sb.append(inlineViewDef); - sb.append(sqlMode); + + // ATTN: sqlMode is missing when persist view, so we should not append it here. + // + // To keep compatible with the old version, without sqlMode, if the signature of views + // are the same, we think the should has the same sqlMode. (since the sqlMode doesn't + // effect the parsing of inlineViewDef, otherwise the parsing will fail), + // + // sb.append(sqlMode); String md5 = DigestUtils.md5Hex(sb.toString()); LOG.debug("get signature of view {}: {}. signature string: {}", name, md5, sb.toString()); return md5; @@ -240,9 +247,11 @@ public View clone() { return copied; } - public void resetIdsForRestore(Env env, String srcDbName, String dbName) { + public void resetIdsForRestore(Env env) { id = env.getNextId(); + } + public void resetViewDefForRestore(String srcDbName, String dbName) { // the source db name is not setted in old BackupMeta, keep compatible with the old one. if (srcDbName != null) { inlineViewDef = inlineViewDef.replaceAll(srcDbName, dbName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java index 201001672b4a02..1bccf464c2f5cb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java @@ -70,6 +70,7 @@ public class PropertyAnalyzer { public static final String PROPERTIES_VERSION_INFO = "version_info"; // for restore public static final String PROPERTIES_SCHEMA_VERSION = "schema_version"; + public static final String PROPERTIES_IN_ATOMIC_RESTORE = "in_atomic_restore"; public static final String PROPERTIES_BF_COLUMNS = "bloom_filter_columns"; public static final String PROPERTIES_BF_FPP = "bloom_filter_fpp"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index fd73fa51cd366e..d41d11252eb6ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -51,6 +51,7 @@ import org.apache.doris.analysis.TableRef; import org.apache.doris.analysis.TruncateTableStmt; import org.apache.doris.analysis.TypeDef; +import org.apache.doris.backup.RestoreJob; import org.apache.doris.catalog.BinlogConfig; import org.apache.doris.catalog.BrokerTable; import org.apache.doris.catalog.ColocateGroupSchema; @@ -903,10 +904,16 @@ public void dropTable(DropTableStmt stmt) throws DdlException { OlapTable olapTable = (OlapTable) table; if ((olapTable.getState() != OlapTableState.NORMAL)) { throw new DdlException("The table [" + tableName + "]'s state is " + olapTable.getState() - + ", cannot be dropped." + " please cancel the operation on olap table firstly." + + ", cannot be dropped. please cancel the operation on olap table firstly." + " If you want to forcibly drop(cannot be recovered)," + " please use \"DROP table FORCE\"."); } + if (olapTable.isInAtomicRestore()) { + throw new DdlException("The table [" + tableName + "]'s state is in atomic restore" + + ", cannot be dropped. please cancel the restore operation on olap table" + + " firstly. If you want to forcibly drop(cannot be recovered)," + + " please use \"DROP table FORCE\"."); + } } dropTableInternal(db, table, stmt.isForceDrop()); @@ -1143,6 +1150,11 @@ public boolean createTable(CreateTableStmt stmt) throws UserException { ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); } } + if (db.getTable(RestoreJob.tableAliasWithAtomicRestore(tableName)).isPresent()) { + ErrorReport.reportDdlException( + "table[{}] is in atomic restore, please cancel the restore operation firstly", + ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); + } if (engineName.equals("olap")) { return createOlapTable(db, stmt); @@ -2499,7 +2511,7 @@ private boolean createOlapTable(Database db, CreateTableStmt stmt) throws UserEx throw new DdlException("Sequence type only support integer types and date types"); } olapTable.setSequenceMapCol(col.getName()); - olapTable.setSequenceInfo(col.getType()); + olapTable.setSequenceInfo(col.getType(), col); } } catch (Exception e) { throw new DdlException(e.getMessage()); @@ -2513,7 +2525,7 @@ private boolean createOlapTable(Database db, CreateTableStmt stmt) throws UserEx throw new DdlException("The sequence_col and sequence_type cannot be set at the same time"); } if (sequenceColType != null) { - olapTable.setSequenceInfo(sequenceColType); + olapTable.setSequenceInfo(sequenceColType, null); } } catch (Exception e) { throw new DdlException(e.getMessage()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClientException.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClientException.java index f9c1e53c2ab409..7fcea7aa61aa2f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClientException.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClientException.java @@ -19,10 +19,33 @@ public class JdbcClientException extends RuntimeException { public JdbcClientException(String format, Throwable cause, Object... msg) { - super(String.format(format, msg), cause); + super(formatMessage(format, msg), cause); } public JdbcClientException(String format, Object... msg) { - super(String.format(format, msg)); + super(formatMessage(format, msg)); + } + + private static String formatMessage(String format, Object... msg) { + if (msg == null || msg.length == 0) { + return format; + } else { + return String.format(format, escapePercentInArgs(msg)); + } + } + + private static Object[] escapePercentInArgs(Object... args) { + if (args == null) { + return null; + } + Object[] escapedArgs = new Object[args.length]; + for (int i = 0; i < args.length; i++) { + if (args[i] instanceof String) { + escapedArgs[i] = ((String) args[i]).replace("%", "%%"); + } else { + escapedArgs[i] = args[i]; + } + } + return escapedArgs; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java index 8fedd791b1ed63..488ae5b429ae31 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java @@ -227,7 +227,8 @@ public void divideRoutineLoadJob(int currentConcurrentTaskNum) throws UserExcept ((KafkaProgress) progress).getOffsetByPartition(kafkaPartition)); } KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), id, clusterName, - maxBatchIntervalS * 2 * 1000, 0, taskKafkaProgress, isMultiTable()); + maxBatchIntervalS * Config.routine_load_task_timeout_multiplier * 1000, + taskKafkaProgress, isMultiTable()); routineLoadTaskInfoList.add(kafkaTaskInfo); result.add(kafkaTaskInfo); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java index dbddc695a68904..476a74836f4625 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java @@ -47,16 +47,14 @@ public class KafkaTaskInfo extends RoutineLoadTaskInfo { private Map partitionIdToOffset; public KafkaTaskInfo(UUID id, long jobId, String clusterName, - long timeoutMs, int timeoutBackOffCount, - Map partitionIdToOffset, boolean isMultiTable) { - super(id, jobId, clusterName, timeoutMs, timeoutBackOffCount, isMultiTable); + long timeoutMs, Map partitionIdToOffset, boolean isMultiTable) { + super(id, jobId, clusterName, timeoutMs, isMultiTable); this.partitionIdToOffset = partitionIdToOffset; } public KafkaTaskInfo(KafkaTaskInfo kafkaTaskInfo, Map partitionIdToOffset, boolean isMultiTable) { super(UUID.randomUUID(), kafkaTaskInfo.getJobId(), kafkaTaskInfo.getClusterName(), - kafkaTaskInfo.getTimeoutMs(), kafkaTaskInfo.getTimeoutBackOffCount(), - kafkaTaskInfo.getBeId(), isMultiTable); + kafkaTaskInfo.getTimeoutMs(), kafkaTaskInfo.getBeId(), isMultiTable); this.partitionIdToOffset = partitionIdToOffset; this.isEof = kafkaTaskInfo.getIsEof(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index 8892ac79b0240a..b4ef59206abe2a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -715,18 +715,6 @@ public void processTimeoutTasks() { // and after renew, the previous task is removed from routineLoadTaskInfoList, // so task can no longer be committed successfully. // the already committed task will not be handled here. - int timeoutBackOffCount = routineLoadTaskInfo.getTimeoutBackOffCount(); - if (timeoutBackOffCount > RoutineLoadTaskInfo.MAX_TIMEOUT_BACK_OFF_COUNT) { - try { - updateState(JobState.PAUSED, new ErrorReason(InternalErrorCode.TIMEOUT_TOO_MUCH, - "task " + routineLoadTaskInfo.getId() + " timeout too much"), false); - } catch (UserException e) { - LOG.warn("update job state to pause failed", e); - } - return; - } - routineLoadTaskInfo.setTimeoutBackOffCount(timeoutBackOffCount + 1); - routineLoadTaskInfo.setTimeoutMs((routineLoadTaskInfo.getTimeoutMs() << 1)); RoutineLoadTaskInfo newTask = unprotectRenewTask(routineLoadTaskInfo); Env.getCurrentEnv().getRoutineLoadTaskScheduler().addTaskInQueue(newTask); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java index 69c07507487983..d8ecfe1abe0be2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java @@ -74,30 +74,24 @@ public abstract class RoutineLoadTaskInfo { protected boolean isMultiTable = false; - protected static final int MAX_TIMEOUT_BACK_OFF_COUNT = 3; - protected int timeoutBackOffCount = 0; - protected boolean isEof = false; // this status will be set when corresponding transaction's status is changed. // so that user or other logic can know the status of the corresponding txn. protected TransactionStatus txnStatus = TransactionStatus.UNKNOWN; - - public RoutineLoadTaskInfo(UUID id, long jobId, String clusterName, long timeoutMs, - int timeoutBackOffCount, boolean isMultiTable) { + public RoutineLoadTaskInfo(UUID id, long jobId, String clusterName, long timeoutMs, boolean isMultiTable) { this.id = id; this.jobId = jobId; this.clusterName = clusterName; this.createTimeMs = System.currentTimeMillis(); this.timeoutMs = timeoutMs; - this.timeoutBackOffCount = timeoutBackOffCount; this.isMultiTable = isMultiTable; } - public RoutineLoadTaskInfo(UUID id, long jobId, String clusterName, long timeoutMs, - int timeoutBackOffCount, long previousBeId, boolean isMultiTable) { - this(id, jobId, clusterName, timeoutMs, timeoutBackOffCount, isMultiTable); + public RoutineLoadTaskInfo(UUID id, long jobId, String clusterName, long timeoutMs, long previousBeId, + boolean isMultiTable) { + this(id, jobId, clusterName, timeoutMs, isMultiTable); this.previousBeId = previousBeId; } @@ -145,10 +139,6 @@ public void setLastScheduledTime(long lastScheduledTime) { this.lastScheduledTime = lastScheduledTime; } - public void setTimeoutMs(long timeoutMs) { - this.timeoutMs = timeoutMs; - } - public long getTimeoutMs() { return timeoutMs; } @@ -161,14 +151,6 @@ public TransactionStatus getTxnStatus() { return txnStatus; } - public void setTimeoutBackOffCount(int timeoutBackOffCount) { - this.timeoutBackOffCount = timeoutBackOffCount; - } - - public int getTimeoutBackOffCount() { - return timeoutBackOffCount; - } - public boolean getIsEof() { return isEof; } @@ -188,25 +170,9 @@ public boolean isTimeout() { } public void handleTaskByTxnCommitAttachment(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment) { - selfAdaptTimeout(rlTaskTxnCommitAttachment); judgeEof(rlTaskTxnCommitAttachment); } - private void selfAdaptTimeout(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment) { - long taskExecutionTime = rlTaskTxnCommitAttachment.getTaskExecutionTimeMs(); - long timeoutMs = this.timeoutMs; - - while (this.timeoutBackOffCount > 0) { - timeoutMs = timeoutMs >> 1; - if (timeoutMs <= taskExecutionTime) { - this.timeoutMs = timeoutMs << 1; - return; - } - this.timeoutBackOffCount--; - } - this.timeoutMs = timeoutMs; - } - private void judgeEof(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment) { RoutineLoadJob routineLoadJob = routineLoadManager.getJob(jobId); if (rlTaskTxnCommitAttachment.getTotalRows() < routineLoadJob.getMaxBatchRows() diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlCommand.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlCommand.java index f8a03029d5a383..b83be01405d87a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlCommand.java @@ -47,6 +47,8 @@ public enum MysqlCommand { COM_STMT_SEND_LONG_DATA("COM_STMT_SEND_LONG_DATA", 24), COM_STMT_CLOSE("COM_STMT_CLOSE", 25), COM_STMT_RESET("COM_STMT_RESET", 26), + COM_SET_OPTION("COM_RESET_CONNECTION", 27), + COM_STMT_FETCH("COM_RESET_CONNECTION", 28), COM_DAEMON("COM_DAEMON", 29), COM_RESET_CONNECTION("COM_RESET_CONNECTION", 31); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlHandshakePacket.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlHandshakePacket.java index c2ba21a23ee321..566f4ac3f7e0ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlHandshakePacket.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlHandshakePacket.java @@ -33,7 +33,7 @@ public class MysqlHandshakePacket extends MysqlPacket { private static final MysqlCapability SSL_CAPABILITY = MysqlCapability.SSL_CAPABILITY; // status flags not supported in palo private static final int STATUS_FLAGS = 0; - private static final String AUTH_PLUGIN_NAME = "mysql_native_password"; + public static final String AUTH_PLUGIN_NAME = "mysql_native_password"; // connection id used in KILL statement. private int connectionId; diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlProto.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlProto.java index 3d198adb6e5821..b442aaa3241024 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlProto.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlProto.java @@ -137,6 +137,7 @@ public static boolean negotiate(ConnectContext context) throws IOException { serializer.reset(); MysqlHandshakePacket handshakePacket = new MysqlHandshakePacket(context.getConnectionId()); handshakePacket.writeTo(serializer); + context.setMysqlHandshakePacket(handshakePacket); try { channel.sendAndFlush(serializer.toByteBuffer()); } catch (IOException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java index 7b130c1700a002..2d2a84c3be2dae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java @@ -34,8 +34,10 @@ import org.apache.doris.analysis.TablePattern; import org.apache.doris.analysis.UserIdentity; import org.apache.doris.analysis.WorkloadGroupPattern; +import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.InfoSchemaDb; +import org.apache.doris.catalog.TableIf; import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.AuthenticationException; @@ -51,6 +53,7 @@ import org.apache.doris.common.PatternMatcherException; import org.apache.doris.common.UserException; import org.apache.doris.common.io.Writable; +import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.ldap.LdapManager; import org.apache.doris.ldap.LdapUserInfo; @@ -81,6 +84,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import java.util.Set; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; @@ -578,6 +582,7 @@ private void grantInternal(UserIdentity userIdent, String role, TablePattern tbl throws DdlException { writeLock(); try { + checkTablePatternExist(tblPattern); if (role == null) { if (!doesUserExist(userIdent)) { throw new DdlException("user " + userIdent + " does not exist"); @@ -596,6 +601,32 @@ private void grantInternal(UserIdentity userIdent, String role, TablePattern tbl } } + private void checkTablePatternExist(TablePattern tablePattern) throws DdlException { + Objects.requireNonNull(tablePattern, "tablePattern can not be null"); + PrivLevel privLevel = tablePattern.getPrivLevel(); + if (privLevel == PrivLevel.GLOBAL) { + return; + } + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(tablePattern.getQualifiedCtl()); + if (catalog == null) { + throw new DdlException("catalog:" + tablePattern.getQualifiedCtl() + " does not exist"); + } + if (privLevel == PrivLevel.CATALOG) { + return; + } + DatabaseIf db = catalog.getDbNullable(tablePattern.getQualifiedDb()); + if (db == null) { + throw new DdlException("database:" + tablePattern.getQualifiedDb() + " does not exist"); + } + if (privLevel == PrivLevel.DATABASE) { + return; + } + TableIf table = db.getTableNullable(tablePattern.getTbl()); + if (table == null) { + throw new DdlException("table:" + tablePattern.getTbl() + " does not exist"); + } + } + // grant for ResourcePattern private void grantInternal(UserIdentity userIdent, String role, ResourcePattern resourcePattern, PrivBitSet privs, boolean errOnNonExist, boolean isReplay) throws DdlException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index c17d3fa545ba30..150780d95e1ad8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -23,6 +23,7 @@ import org.apache.doris.analysis.StatementBase; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.NereidsException; import org.apache.doris.common.Pair; import org.apache.doris.common.util.DebugUtil; @@ -43,12 +44,14 @@ import org.apache.doris.nereids.processor.post.PlanPostProcessors; import org.apache.doris.nereids.processor.pre.PlanPreprocessors; import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.stats.StatsCalculator; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.plans.AbstractPlan; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.physical.PhysicalOneRowRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; @@ -258,6 +261,16 @@ private Plan planWithoutLock( } } + // if we cannot get table row count, skip join reorder + // except: + // 1. user set leading hint + // 2. ut test. In ut test, FeConstants.enableInternalSchemaDb is false or FeConstants.runningUnitTest is true + if (FeConstants.enableInternalSchemaDb && !FeConstants.runningUnitTest && cascadesContext.isLeadingJoin()) { + List scans = cascadesContext.getRewritePlan() + .collectToList(LogicalOlapScan.class::isInstance); + StatsCalculator.disableJoinReorderIfTableRowCountNotAvailable(scans, cascadesContext); + } + optimize(); if (statementContext.getConnectContext().getExecutor() != null) { statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsOptimizeTime(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 7d9f36750d5184..07d6f77f6d0f8c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -1225,7 +1225,15 @@ public Expression visitFunctionCall(DorisParser.FunctionCallContext ctx) { List orderKeys = visit(ctx.sortItem(), OrderKey.class); if (!orderKeys.isEmpty()) { - return parseFunctionWithOrderKeys(functionName, isDistinct, params, orderKeys, ctx); + Expression expression = parseFunctionWithOrderKeys(functionName, isDistinct, params, orderKeys, ctx); + if (ctx.windowSpec() != null) { + if (isDistinct) { + throw new ParseException("DISTINCT not allowed in analytic function: " + functionName, ctx); + } + return withWindowSpec(ctx.windowSpec(), expression); + } else { + return expression; + } } List unboundStars = ExpressionUtils.collectAll(params, UnboundStar.class::isInstance); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java index ad1ad4f52676a2..7bbbc7841e8235 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java @@ -114,7 +114,10 @@ public List buildRules() { .when(agg -> { Set funcs = agg.getAggregateFunctions(); return !funcs.isEmpty() && funcs.stream() - .allMatch(f -> f instanceof Count && !f.isDistinct()); + .allMatch(f -> f instanceof Count && !f.isDistinct() && (((Count) f).isStar() + || f.children().isEmpty() + || (f.children().size() == 1 && f.child(0) instanceof Literal) + || f.child(0) instanceof Slot)); }) .thenApply(ctx -> { LogicalAggregate> agg = ctx.root; @@ -133,7 +136,11 @@ public List buildRules() { .when(agg -> agg.getGroupByExpressions().isEmpty()) .when(agg -> { Set funcs = agg.getAggregateFunctions(); - return !funcs.isEmpty() && funcs.stream().allMatch(f -> f instanceof Count && !f.isDistinct()); + return !funcs.isEmpty() && funcs.stream() + .allMatch(f -> f instanceof Count && !f.isDistinct() && (((Count) f).isStar() + || f.children().isEmpty() + || (f.children().size() == 1 && f.child(0) instanceof Literal) + || f.child(0) instanceof Slot)); }) .thenApply(ctx -> { LogicalAggregate>> agg = ctx.root; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java index c4a27b9748d7c9..73ff74066887ec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java @@ -31,9 +31,12 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; +import org.apache.doris.nereids.trees.plans.logical.LogicalEsScan; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalGenerate; +import org.apache.doris.nereids.trees.plans.logical.LogicalJdbcScan; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOdbcScan; import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; @@ -253,6 +256,39 @@ public Plan visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, Map replaceMap) { + if (!scan.getConjuncts().isEmpty()) { + scan.getOutputSet().forEach(s -> replaceMap.put(s.getExprId(), s)); + Set conjuncts = updateExpressions(scan.getConjuncts(), replaceMap); + return scan.withConjuncts(conjuncts).recomputeLogicalProperties(); + } else { + return scan; + } + } + + @Override + public Plan visitLogicalJdbcScan(LogicalJdbcScan scan, Map replaceMap) { + if (!scan.getConjuncts().isEmpty()) { + scan.getOutputSet().forEach(s -> replaceMap.put(s.getExprId(), s)); + Set conjuncts = updateExpressions(scan.getConjuncts(), replaceMap); + return scan.withConjuncts(conjuncts).recomputeLogicalProperties(); + } else { + return scan; + } + } + + @Override + public Plan visitLogicalOdbcScan(LogicalOdbcScan scan, Map replaceMap) { + if (!scan.getConjuncts().isEmpty()) { + scan.getOutputSet().forEach(s -> replaceMap.put(s.getExprId(), s)); + Set conjuncts = updateExpressions(scan.getConjuncts(), replaceMap); + return scan.withConjuncts(conjuncts).recomputeLogicalProperties(); + } else { + return scan; + } + } + private T updateExpression(T input, Map replaceMap) { return (T) input.rewriteDownShortCircuit(e -> e.accept(SlotReferenceReplacer.INSTANCE, replaceMap)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java index 383ad266511f32..fe69536840096b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java @@ -17,12 +17,14 @@ package org.apache.doris.nereids.rules.rewrite; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.rewrite.NormalizeToSlot.NormalizeToSlotContext; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.OrderExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.plans.Plan; @@ -52,6 +54,10 @@ public Rule build() { if (output instanceof WindowExpression) { // remove literal partition by and order by keys WindowExpression windowExpression = (WindowExpression) output; + Expression function = windowExpression.getFunction(); + if (function.containsType(OrderExpression.class)) { + throw new AnalysisException("order by is not supported in " + function); + } return windowExpression.withPartitionKeysOrderKeys( windowExpression.getPartitionKeys().stream() .filter(expression -> !expression.isConstant()) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java index faa9fd323d68f9..71a4b64c841812 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java @@ -602,11 +602,13 @@ private Statistics estimateBinaryComparisonFilter(Expression leftExpr, DataType .setMaxExpr(intersectRange.getHighExpr()) .setNdv(intersectRange.getDistinctValues()) .setNumNulls(0); - double sel = leftRange.overlapPercentWith(rightRange); + double sel = leftRange.getDistinctValues() == 0 + ? 1.0 + : intersectRange.getDistinctValues() / leftRange.getDistinctValues(); if (!(dataType instanceof RangeScalable) && (sel != 0.0 && sel != 1.0)) { sel = DEFAULT_INEQUALITY_COEFFICIENT; - } else if (sel < RANGE_SELECTIVITY_THRESHOLD) { - sel = RANGE_SELECTIVITY_THRESHOLD; + } else { + sel = Math.max(sel, RANGE_SELECTIVITY_THRESHOLD); } sel = getNotNullSelectivity(leftStats, sel); updatedStatistics = context.statistics.withSel(sel); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java index 1b67b80a62d5d5..9458407295d903 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.stats; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.FeConstants; import org.apache.doris.common.Pair; @@ -179,6 +180,11 @@ private StatsCalculator(GroupExpression groupExpression, boolean forbidUnknownCo this.cascadesContext = context; } + private StatsCalculator(CascadesContext context) { + this.groupExpression = null; + this.cascadesContext = context; + } + public Map getTotalHistogramMap() { return totalHistogramMap; } @@ -1132,4 +1138,46 @@ public Statistics visitPhysicalCTEAnchor( return groupExpression.childStatistics(1); } + /** + * if the table is not analyzed and BE does not report row count, return -1 + */ + private double getOlapTableRowCount(OlapScan olapScan) { + OlapTable olapTable = olapScan.getTable(); + AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); + TableStatsMeta tableMeta = analysisManager.findTableStatsStatus(olapScan.getTable().getId()); + double rowCount = -1; + if (tableMeta != null && tableMeta.userInjected) { + rowCount = tableMeta.getRowCount(olapScan.getSelectedIndexId()); + } else { + rowCount = olapTable.getRowCountForIndex(olapScan.getSelectedIndexId(), true); + if (rowCount == -1) { + if (tableMeta != null) { + rowCount = tableMeta.getRowCount(olapScan.getSelectedIndexId()); + } + } + } + return rowCount; + } + + /** + * disable join reorder if any table row count is not available. + */ + public static void disableJoinReorderIfTableRowCountNotAvailable( + List scans, + CascadesContext context) { + StatsCalculator calculator = new StatsCalculator(context); + for (LogicalOlapScan scan : scans) { + double rowCount = calculator.getOlapTableRowCount(scan); + if (rowCount == -1 && ConnectContext.get() != null) { + try { + ConnectContext.get().getSessionVariable().disableNereidsJoinReorderOnce(); + LOG.info("disable join reorder since row count not available: " + + scan.getTable().getNameWithFullQualifiers()); + } catch (Exception e) { + LOG.info("disableNereidsJoinReorderOnce failed"); + } + return; + } + } + } } 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 a10f8a49a39baa..07cb110c0ee302 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 @@ -44,6 +44,7 @@ import org.apache.doris.mysql.MysqlCapability; import org.apache.doris.mysql.MysqlChannel; import org.apache.doris.mysql.MysqlCommand; +import org.apache.doris.mysql.MysqlHandshakePacket; import org.apache.doris.mysql.MysqlSslContext; import org.apache.doris.mysql.ProxyMysqlChannel; import org.apache.doris.nereids.StatementContext; @@ -200,6 +201,8 @@ public class ConnectContext { // In this case, `skipAuth` needs to be set to `true` to skip the permission check of `AlterTable` private boolean skipAuth = false; + private MysqlHandshakePacket mysqlHandshakePacket; + public void setUserQueryTimeout(int queryTimeout) { if (queryTimeout > 0) { sessionVariable.setQueryTimeoutS(queryTimeout); @@ -952,5 +955,13 @@ public boolean isSkipAuth() { public void setSkipAuth(boolean skipAuth) { this.skipAuth = skipAuth; } + + public void setMysqlHandshakePacket(MysqlHandshakePacket mysqlHandshakePacket) { + this.mysqlHandshakePacket = mysqlHandshakePacket; + } + + public byte[] getAuthPluginData() { + return mysqlHandshakePacket == null ? null : mysqlHandshakePacket.getAuthPluginData(); + } } 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 1ee08fb298e67f..c994d524c3e37a 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 @@ -33,6 +33,7 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.AuthenticationException; import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; @@ -44,25 +45,31 @@ import org.apache.doris.common.util.SqlUtils; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.metric.MetricRepo; import org.apache.doris.mysql.MysqlChannel; import org.apache.doris.mysql.MysqlCommand; +import org.apache.doris.mysql.MysqlHandshakePacket; import org.apache.doris.mysql.MysqlPacket; import org.apache.doris.mysql.MysqlProto; import org.apache.doris.mysql.MysqlSerializer; import org.apache.doris.mysql.MysqlServerStatusFlag; +import org.apache.doris.mysql.privilege.Auth; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.minidump.MinidumpUtils; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.stats.StatsErrorEstimator; import org.apache.doris.proto.Data; import org.apache.doris.qe.QueryState.MysqlStateType; +import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TExprNode; import org.apache.doris.thrift.TMasterOpRequest; import org.apache.doris.thrift.TMasterOpResult; import org.apache.doris.thrift.TUniqueId; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.SpanKind; @@ -174,10 +181,19 @@ private void handlePing() { ctx.getState().setOk(); } - private void handleStmtReset() { + // Do nothing for now. + protected void handleStatistics() { ctx.getState().setOk(); } + // Do nothing for now. + protected void handleDebug() { + ctx.getState().setOk(); + } + + protected void handleStmtReset() { + } + private void handleStmtClose() { packetBuf = packetBuf.order(ByteOrder.LITTLE_ENDIAN); int stmtId = packetBuf.getInt(); @@ -517,6 +533,15 @@ private void dispatch() throws IOException { case COM_PING: handlePing(); break; + case COM_STATISTICS: + handleStatistics(); + break; + case COM_DEBUG: + handleDebug(); + break; + case COM_CHANGE_USER: + handleChangeUser(); + break; case COM_STMT_RESET: handleStmtReset(); break; @@ -746,6 +771,96 @@ public TMasterOpResult proxyExecute(TMasterOpRequest request) throws TException return result; } + private void handleChangeUser() throws IOException { + // Random bytes generated when creating connection. + byte[] authPluginData = ctx.getAuthPluginData(); + Preconditions.checkNotNull(authPluginData, "Auth plugin data is null."); + String userName = new String(MysqlProto.readNulTerminateString(packetBuf)); + userName = ClusterNamespace.getFullName(SystemInfoService.DEFAULT_CLUSTER, userName); + int passwordLen = MysqlProto.readInt1(packetBuf); + byte[] password = MysqlProto.readFixedString(packetBuf, passwordLen); + String db = new String(MysqlProto.readNulTerminateString(packetBuf)); + // Read the character set. + MysqlProto.readInt2(packetBuf); + String authPluginName = new String(MysqlProto.readNulTerminateString(packetBuf)); + + // Send Protocol::AuthSwitchRequest to client if auth plugin name is not mysql_native_password + if (!MysqlHandshakePacket.AUTH_PLUGIN_NAME.equals(authPluginName)) { + MysqlChannel channel = ctx.mysqlChannel; + MysqlSerializer serializer = MysqlSerializer.newInstance(); + serializer.writeInt1((byte) 0xfe); + serializer.writeNulTerminateString(MysqlHandshakePacket.AUTH_PLUGIN_NAME); + serializer.writeBytes(authPluginData); + serializer.writeInt1(0); + channel.sendAndFlush(serializer.toByteBuffer()); + // Server receive auth switch response packet from client. + ByteBuffer authSwitchResponse = channel.fetchOnePacket(); + int length = authSwitchResponse.limit(); + password = new byte[length]; + System.arraycopy(authSwitchResponse.array(), 0, password, 0, length); + } + + // For safety, not allowed to change to root or admin. + if (Auth.ROOT_USER.equals(userName) || Auth.ADMIN_USER.equals(userName)) { + ctx.getState().setError(ErrorCode.ERR_ACCESS_DENIED_ERROR, "Change to root or admin is forbidden"); + return; + } + + // Check password. + List currentUserIdentity = Lists.newArrayList(); + try { + Env.getCurrentEnv().getAuth() + .checkPassword(userName, ctx.remoteIP, password, authPluginData, currentUserIdentity); + } catch (AuthenticationException e) { + ctx.getState().setError(ErrorCode.ERR_ACCESS_DENIED_ERROR, "Authentication failed."); + return; + } + ctx.setCurrentUserIdentity(currentUserIdentity.get(0)); + ctx.setQualifiedUser(userName); + + // Change default db if set. + if (Strings.isNullOrEmpty(db)) { + ctx.changeDefaultCatalog(InternalCatalog.INTERNAL_CATALOG_NAME); + } else { + String catalogName = null; + String dbName = null; + String[] dbNames = db.split("\\."); + if (dbNames.length == 1) { + dbName = db; + } else if (dbNames.length == 2) { + catalogName = dbNames[0]; + dbName = dbNames[1]; + } else if (dbNames.length > 2) { + ctx.getState().setError(ErrorCode.ERR_BAD_DB_ERROR, "Only one dot can be in the name: " + db); + return; + } + dbName = ClusterNamespace.getFullName(ctx.getClusterName(), dbName); + + // check catalog and db exists + if (catalogName != null) { + CatalogIf catalogIf = ctx.getEnv().getCatalogMgr().getCatalog(catalogName); + if (catalogIf == null) { + ctx.getState().setError(ErrorCode.ERR_BAD_DB_ERROR, "No match catalog in doris: " + db); + return; + } + if (catalogIf.getDbNullable(dbName) == null) { + ctx.getState().setError(ErrorCode.ERR_BAD_DB_ERROR, "No match database in doris: " + db); + return; + } + } + try { + if (catalogName != null) { + ctx.getEnv().changeCatalog(ctx, catalogName); + } + Env.getCurrentEnv().changeDb(ctx, dbName); + } catch (DdlException e) { + ctx.getState().setError(e.getMysqlErrorCode(), e.getMessage()); + return; + } + } + ctx.getState().setOk(); + } + // Process a MySQL request public void processOnce() throws IOException { // set status of query to OK. 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 6de1f0883bc68e..69c92d0f1668e6 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 @@ -2489,21 +2489,12 @@ private void handleShowTableStats() { if (tableStats == null) { resultSet = showTableStatsStmt.constructEmptyResultSet(); } else { - resultSet = showTableStatsStmt.constructResultSet(tableStats); + resultSet = showTableStatsStmt.constructResultSet(tableStats, tableIf); } return; } TableStatsMeta tableStats = Env.getCurrentEnv().getAnalysisManager().findTableStatsStatus(tableIf.getId()); - /* - HMSExternalTable table will fetch row count from HMS - or estimate with file size and schema if it's not analyzed. - tableStats == null means it's not analyzed, in this case show the estimated row count. - */ - if (tableStats == null) { - resultSet = showTableStatsStmt.constructResultSet(tableIf.getRowCount()); - } else { - resultSet = showTableStatsStmt.constructResultSet(tableStats); - } + resultSet = showTableStatsStmt.constructResultSet(tableStats, tableIf); } private void handleShowColumnStats() throws AnalysisException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index 7f81104d94368a..d5aef9a78e3969 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -1730,13 +1730,8 @@ private boolean commitTxnImpl(TCommitTxnRequest request) throws UserException { throw new UserException("transaction [" + request.getTxnId() + "] not found"); } List tableIdList = transactionState.getTableIdList(); - List
tableList = new ArrayList<>(); - List tables = new ArrayList<>(); // if table was dropped, transaction must be aborted - tableList = db.getTablesOnIdOrderOrThrowException(tableIdList); - for (Table table : tableList) { - tables.add(table.getName()); - } + List
tableList = db.getTablesOnIdOrderOrThrowException(tableIdList); // Step 3: check auth if (request.isSetAuthCode()) { @@ -1744,6 +1739,7 @@ private boolean commitTxnImpl(TCommitTxnRequest request) throws UserException { } else if (request.isSetToken()) { checkToken(request.getToken()); } else { + List tables = tableList.stream().map(Table::getName).collect(Collectors.toList()); checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), tables, request.getUserIp(), PrivPredicate.LOAD); } @@ -1912,12 +1908,7 @@ private void rollbackTxnImpl(TRollbackTxnRequest request) throws UserException { throw new UserException("transaction [" + request.getTxnId() + "] not found"); } List tableIdList = transactionState.getTableIdList(); - List
tableList = new ArrayList<>(); - List tables = new ArrayList<>(); - tableList = db.getTablesOnIdOrderOrThrowException(tableIdList); - for (Table table : tableList) { - tables.add(table.getName()); - } + List
tableList = db.getTablesOnIdOrderOrThrowException(tableIdList); // Step 3: check auth if (request.isSetAuthCode()) { @@ -1925,6 +1916,7 @@ private void rollbackTxnImpl(TRollbackTxnRequest request) throws UserException { } else if (request.isSetToken()) { checkToken(request.getToken()); } else { + List tables = tableList.stream().map(Table::getName).collect(Collectors.toList()); checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), tables, request.getUserIp(), PrivPredicate.LOAD); } @@ -2917,6 +2909,9 @@ private TRestoreSnapshotResult restoreSnapshotImpl(TRestoreSnapshotRequest reque if (request.isCleanTables()) { properties.put(RestoreStmt.PROP_CLEAN_TABLES, "true"); } + if (request.isAtomicRestore()) { + properties.put(RestoreStmt.PROP_ATOMIC_RESTORE, "true"); + } AbstractBackupTableRefClause restoreTableRefClause = null; if (request.isSetTableRefs()) { @@ -3118,9 +3113,6 @@ public TStatus invalidateStatsCache(TInvalidateFollowerStatsCacheRequest request InvalidateStatsTarget target = GsonUtils.GSON.fromJson(request.key, InvalidateStatsTarget.class); AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); TableStatsMeta tableStats = analysisManager.findTableStatsStatus(target.tableId); - if (tableStats == null) { - return new TStatus(TStatusCode.OK); - } analysisManager.invalidateLocalStats(target.catalogId, target.dbId, target.tableId, target.columns, tableStats); return new TStatus(TStatusCode.OK); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java index ddef30ee4de28b..17a32900be3f10 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java @@ -63,7 +63,7 @@ public class AnalysisInfoBuilder { private boolean usingSqlForPartitionColumn; private long tblUpdateTime; private boolean emptyJob; - private boolean userInject; + private boolean userInject = false; private long rowCount; public AnalysisInfoBuilder() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java index 5d3debb8ddd427..af17a63da20816 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java @@ -90,6 +90,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.NavigableMap; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.StringJoiner; @@ -690,33 +691,45 @@ public void dropStats(DropStatsStmt dropStatsStmt) throws DdlException { return; } + TableStatsMeta tableStats = findTableStatsStatus(dropStatsStmt.getTblId()); + if (tableStats == null) { + return; + } Set cols = dropStatsStmt.getColumnNames(); long catalogId = dropStatsStmt.getCatalogIdId(); long dbId = dropStatsStmt.getDbId(); long tblId = dropStatsStmt.getTblId(); - TableStatsMeta tableStats = findTableStatsStatus(dropStatsStmt.getTblId()); - if (tableStats == null) { - return; + // Remove tableMetaStats if drop whole table stats. + if (dropStatsStmt.isAllColumns()) { + removeTableStats(tblId); + Env.getCurrentEnv().getEditLog().logDeleteTableStats(new TableStatsDeletionLog(tblId)); } - invalidateLocalStats(catalogId, dbId, tblId, cols, tableStats); + invalidateLocalStats(catalogId, dbId, tblId, dropStatsStmt.isAllColumns() ? null : cols, tableStats); // Drop stats ddl is master only operation. invalidateRemoteStats(catalogId, dbId, tblId, cols, dropStatsStmt.isAllColumns()); StatisticsRepository.dropStatistics(tblId, cols); } - public void dropStats(TableIf table) throws DdlException { - TableStatsMeta tableStats = findTableStatsStatus(table.getId()); - if (tableStats == null) { - return; + public void dropStats(TableIf table) { + try { + TableStatsMeta tableStats = findTableStatsStatus(table.getId()); + if (tableStats == null) { + return; + } + long catalogId = table.getDatabase().getCatalog().getId(); + long dbId = table.getDatabase().getId(); + long tableId = table.getId(); + removeTableStats(tableId); + Env.getCurrentEnv().getEditLog().logDeleteTableStats(new TableStatsDeletionLog(tableId)); + Set cols = table.getSchemaAllIndexes(false).stream().map(Column::getName) + .collect(Collectors.toSet()); + invalidateLocalStats(catalogId, dbId, tableId, null, tableStats); + // Drop stats ddl is master only operation. + invalidateRemoteStats(catalogId, dbId, tableId, cols, true); + StatisticsRepository.dropStatistics(table.getId(), cols); + } catch (Throwable e) { + LOG.warn("Failed to drop stats for table {}", table.getName(), e); } - long catalogId = table.getDatabase().getCatalog().getId(); - long dbId = table.getDatabase().getId(); - long tableId = table.getId(); - Set cols = table.getSchemaAllIndexes(false).stream().map(Column::getName).collect(Collectors.toSet()); - invalidateLocalStats(catalogId, dbId, tableId, cols, tableStats); - // Drop stats ddl is master only operation. - invalidateRemoteStats(catalogId, dbId, tableId, cols, true); - StatisticsRepository.dropStatistics(table.getId(), cols); } public void dropCachedStats(long catalogId, long dbId, long tableId) { @@ -739,14 +752,9 @@ public void dropCachedStats(long catalogId, long dbId, long tableId) { public void invalidateLocalStats(long catalogId, long dbId, long tableId, Set columns, TableStatsMeta tableStats) { - if (tableStats == null) { - return; - } TableIf table = StatisticsUtil.findTable(catalogId, dbId, tableId); StatisticsCache statsCache = Env.getCurrentEnv().getStatisticsCache(); - boolean allColumn = false; if (columns == null) { - allColumn = true; columns = table.getSchemaAllIndexes(false) .stream().map(Column::getName).collect(Collectors.toSet()); } @@ -759,16 +767,16 @@ public void invalidateLocalStats(long catalogId, long dbId, long tableId, indexIds.add(-1L); } for (long indexId : indexIds) { - tableStats.removeColumn(column); + if (tableStats != null) { + tableStats.removeColumn(column); + } statsCache.invalidate(tableId, indexId, column); } } - // To remove stale column name that is changed before. - if (allColumn) { - tableStats.removeAllColumn(); + if (tableStats != null) { + tableStats.updatedTime = 0; + tableStats.userInjected = false; } - tableStats.updatedTime = 0; - tableStats.userInjected = false; } public void invalidateRemoteStats(long catalogId, long dbId, long tableId, @@ -778,18 +786,15 @@ public void invalidateRemoteStats(long catalogId, long dbId, long tableId, request.key = GsonUtils.GSON.toJson(target); StatisticsCache statisticsCache = Env.getCurrentEnv().getStatisticsCache(); SystemInfoService.HostInfo selfNode = Env.getCurrentEnv().getSelfNode(); - boolean success = true; for (Frontend frontend : Env.getCurrentEnv().getFrontends(null)) { // Skip master if (selfNode.getHost().equals(frontend.getHost())) { continue; } - success = success && statisticsCache.invalidateStats(frontend, request); + statisticsCache.invalidateStats(frontend, request); } - if (!success) { - // If any rpc failed, use edit log to sync table stats to non-master FEs. - LOG.warn("Failed to invalidate all remote stats by rpc for table {}, use edit log.", tableId); - TableStatsMeta tableStats = findTableStatsStatus(tableId); + TableStatsMeta tableStats = findTableStatsStatus(tableId); + if (tableStats != null) { logCreateTableStats(tableStats); } } @@ -949,7 +954,7 @@ public List findTasks(long jobId) { public List findTasksByTaskIds(long jobId) { AnalysisInfo jobInfo = analysisJobInfoMap.get(jobId); if (jobInfo != null && jobInfo.taskIds != null) { - return jobInfo.taskIds.stream().map(analysisTaskInfoMap::get).filter(i -> i != null) + return jobInfo.taskIds.stream().map(analysisTaskInfoMap::get).filter(Objects::nonNull) .collect(Collectors.toList()); } return null; @@ -966,7 +971,7 @@ public void removeAll(List analysisInfos) { public void dropAnalyzeJob(DropAnalyzeJobStmt analyzeJobStmt) throws DdlException { AnalysisInfo jobInfo = analysisJobInfoMap.get(analyzeJobStmt.getJobId()); if (jobInfo == null) { - throw new DdlException(String.format("Analyze job [%d] not exists", jobInfo.jobId)); + throw new DdlException(String.format("Analyze job [%d] not exists", analyzeJobStmt.getJobId())); } checkPriv(jobInfo); long jobId = analyzeJobStmt.getJobId(); @@ -1003,12 +1008,12 @@ public static boolean needAbandon(AnalysisInfo analysisInfo) { if (analysisInfo == null) { return true; } - if ((AnalysisState.PENDING.equals(analysisInfo.state) || AnalysisState.RUNNING.equals(analysisInfo.state)) - && ScheduleType.ONCE.equals(analysisInfo.scheduleType) - && JobType.MANUAL.equals(analysisInfo.jobType)) { + if (analysisInfo.scheduleType == null || analysisInfo.jobType == null) { return true; } - return false; + return (AnalysisState.PENDING.equals(analysisInfo.state) || AnalysisState.RUNNING.equals(analysisInfo.state)) + && ScheduleType.ONCE.equals(analysisInfo.scheduleType) + && JobType.MANUAL.equals(analysisInfo.jobType); } private static void readIdToTblStats(DataInput in, Map map) throws IOException { @@ -1177,17 +1182,14 @@ public void removeJob(long id) { /** * Only OlapTable and Hive HMSExternalTable can sample for now. - * @param table + * @param table Table to check * @return Return true if the given table can do sample analyze. False otherwise. */ public boolean canSample(TableIf table) { if (table instanceof OlapTable) { return true; } - if (table instanceof HMSExternalTable - && ((HMSExternalTable) table).getDlaType().equals(HMSExternalTable.DLAType.HIVE)) { - return true; - } - return false; + return table instanceof HMSExternalTable + && ((HMSExternalTable) table).getDlaType().equals(HMSExternalTable.DLAType.HIVE); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java index 85a2fd0de3f762..6ab65677e7ace1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java @@ -40,7 +40,6 @@ import java.text.MessageFormat; import java.util.Collections; -import java.util.concurrent.TimeUnit; public abstract class BaseAnalysisTask { @@ -83,30 +82,6 @@ public abstract class BaseAnalysisTask { + "NOW() " + "FROM `${catalogName}`.`${dbName}`.`${tblName}` ${index} ${sampleHints} ${limit}"; - protected static final String DUJ1_ANALYZE_STRING_TEMPLATE = "SELECT " - + "CONCAT('${tblId}', '-', '${idxId}', '-', '${colId}') AS `id`, " - + "${catalogId} AS `catalog_id`, " - + "${dbId} AS `db_id`, " - + "${tblId} AS `tbl_id`, " - + "${idxId} AS `idx_id`, " - + "'${colId}' AS `col_id`, " - + "NULL AS `part_id`, " - + "${rowCount} AS `row_count`, " - + "${ndvFunction} as `ndv`, " - + "IFNULL(SUM(IF(`t1`.`column_key` IS NULL, `t1`.`count`, 0)), 0) * ${scaleFactor} as `null_count`, " - + "SUBSTRING(CAST(${min} AS STRING), 1, 1024) AS `min`, " - + "SUBSTRING(CAST(${max} AS STRING), 1, 1024) AS `max`, " - + "${dataSizeFunction} * ${scaleFactor} AS `data_size`, " - + "NOW() " - + "FROM ( " - + " SELECT t0.`colValue` as `column_key`, COUNT(1) as `count` " - + " FROM " - + " (SELECT SUBSTRING(CAST(`${colName}` AS STRING), 1, 1024) AS `colValue` " - + " FROM `${catalogName}`.`${dbName}`.`${tblName}` ${index} " - + " ${sampleHints} ${limit}) as `t0` " - + " GROUP BY `t0`.`colValue` " - + ") as `t1` "; - protected static final String DUJ1_ANALYZE_TEMPLATE = "SELECT " + "CONCAT('${tblId}', '-', '${idxId}', '-', '${colId}') AS `id`, " + "${catalogId} AS `catalog_id`, " @@ -123,11 +98,11 @@ public abstract class BaseAnalysisTask { + "${dataSizeFunction} * ${scaleFactor} AS `data_size`, " + "NOW() " + "FROM ( " - + " SELECT t0.`${colName}` as `column_key`, COUNT(1) as `count` " + + " SELECT t0.`colValue` as `column_key`, COUNT(1) as `count`, SUM(`len`) as `column_length` " + " FROM " - + " (SELECT `${colName}` FROM `${catalogName}`.`${dbName}`.`${tblName}` ${index} " - + " ${sampleHints} ${limit}) as `t0` " - + " GROUP BY `t0`.`${colName}` " + + " (SELECT ${subStringColName} AS `colValue`, LENGTH(`${colName}`) as `len` " + + " FROM `${catalogName}`.`${dbName}`.`${tblName}` ${index} ${sampleHints} ${limit}) as `t0` " + + " GROUP BY `t0`.`colValue` " + ") as `t1` "; protected static final String ANALYZE_PARTITION_COLUMN_TEMPLATE = " SELECT " @@ -195,9 +170,9 @@ protected void init(AnalysisInfo info) { } } - public void execute() { + public void execute() throws Exception { prepareExecution(); - executeWithRetry(); + doExecute(); afterExecution(); } @@ -205,29 +180,6 @@ protected void prepareExecution() { setTaskStateToRunning(); } - protected void executeWithRetry() { - int retriedTimes = 0; - while (retriedTimes < StatisticConstants.ANALYZE_TASK_RETRY_TIMES) { - if (killed) { - throw new RuntimeException("Task is Killed or Timeout"); - } - try { - doExecute(); - break; - } catch (Throwable t) { - if (killed) { - throw new RuntimeException(t); - } - LOG.warn("Failed to execute analysis task, retried times: {}", retriedTimes++, t); - if (retriedTimes >= StatisticConstants.ANALYZE_TASK_RETRY_TIMES) { - job.taskFailed(this, t.getMessage()); - throw new RuntimeException(t); - } - StatisticsUtil.sleep(TimeUnit.SECONDS.toMillis(2 ^ retriedTimes) * 10); - } - } - } - public abstract void doExecute() throws Exception; protected void afterExecution() {} @@ -254,7 +206,7 @@ public long getJobId() { protected String getDataSizeFunction(Column column, boolean useDuj1) { if (useDuj1) { if (column.getType().isStringType()) { - return "SUM(LENGTH(`column_key`) * count)"; + return "SUM(`column_length`)"; } else { return "SUM(t1.count) * " + column.getType().getSlotSize(); } @@ -267,6 +219,14 @@ protected String getDataSizeFunction(Column column, boolean useDuj1) { } } + protected String getStringTypeColName(Column column) { + if (column.getType().isStringType()) { + return "murmur_hash3_64(SUBSTRING(CAST(`${colName}` AS STRING), 1, 1024))"; + } else { + return "`${colName}`"; + } + } + protected String getMinFunction() { if (tableSample == null) { return "CAST(MIN(`${colName}`) as ${type}) "; @@ -285,9 +245,8 @@ protected String getNdvFunction(String totalRows) { // (https://github.com/postgres/postgres/blob/master/src/backend/commands/analyze.c) // (http://citeseerx.ist.psu.edu/viewdoc/download?doi=10.1.1.93.8637&rep=rep1&type=pdf) // sample_row * count_distinct / ( sample_row - once_count + once_count * sample_row / total_row) - String fn = MessageFormat.format("{0} * {1} / ({0} - {2} + {2} * {0} / {3})", sampleRows, + return MessageFormat.format("{0} * {1} / ({0} - {2} + {2} * {0} / {3})", sampleRows, countDistinct, onceCount, totalRows); - return fn; } // Max value is not accurate while sample, so set it to NULL to avoid optimizer generate bad plan. @@ -345,6 +304,9 @@ protected void runQuery(String sql) { Env.getCurrentEnv().getStatisticsCache().syncColStats(colStatsData); queryId = DebugUtil.printId(stmtExecutor.getContext().queryId()); job.appendBuf(this, Collections.singletonList(colStatsData)); + } catch (Exception e) { + LOG.warn("Failed to execute sql {}", sql); + throw e; } finally { if (LOG.isDebugEnabled()) { LOG.debug("End cost time in millisec: " + (System.currentTimeMillis() - startTime) diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java index 42fb10fc449d10..015796ff8f3aef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java @@ -151,11 +151,8 @@ private void getOrdinaryColumnStats() throws Exception { params.put("ndvFunction", "ROUND(NDV(`${colName}`) * ${scaleFactor})"); params.put("rowCount", "ROUND(count(1) * ${scaleFactor})"); } else { - if (col.getType().isStringType()) { - sb.append(DUJ1_ANALYZE_STRING_TEMPLATE); - } else { - sb.append(DUJ1_ANALYZE_TEMPLATE); - } + sb.append(DUJ1_ANALYZE_TEMPLATE); + params.put("subStringColName", getStringTypeColName(col)); params.put("dataSizeFunction", getDataSizeFunction(col, true)); params.put("ndvFunction", getNdvFunction("ROUND(SUM(t1.count) * ${scaleFactor})")); params.put("rowCount", "ROUND(SUM(t1.count) * ${scaleFactor})"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java index 3e02c47497b097..59c239e5878e6c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java @@ -78,11 +78,6 @@ public void doExecute() throws Exception { Env.getCurrentEnv().getStatisticsCache().refreshHistogramSync(tbl.getId(), -1, col.getName()); } - @Override - protected void afterExecution() { - // DO NOTHING - } - private String getSampleRateFunction() { if (info.analysisMethod == AnalysisMethod.FULL) { return "0"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java index d93b0b74e540ee..34fb339564abed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java @@ -101,7 +101,7 @@ protected void doSample() throws Exception { List tabletIds = pair.first; long totalRowCount = info.indexId == -1 ? tbl.getRowCount() - : ((OlapTable) tbl).getRowCountForIndex(info.indexId); + : ((OlapTable) tbl).getRowCountForIndex(info.indexId, false); double scaleFactor = (double) totalRowCount / (double) pair.second; // might happen if row count in fe metadata hasn't been updated yet if (Double.isInfinite(scaleFactor) || Double.isNaN(scaleFactor)) { @@ -133,8 +133,8 @@ protected void doSample() throws Exception { params.put("colId", StatisticsUtil.escapeSQL(String.valueOf(info.colName))); params.put("dataSizeFunction", getDataSizeFunction(col, false)); params.put("dbName", db.getFullName()); - params.put("colName", StatisticsUtil.escapeColumnName(info.colName)); - params.put("tblName", tbl.getName()); + params.put("colName", StatisticsUtil.escapeColumnName(String.valueOf(info.colName))); + params.put("tblName", String.valueOf(tbl.getName())); params.put("scaleFactor", String.valueOf(scaleFactor)); params.put("sampleHints", tabletStr.isEmpty() ? "" : String.format("TABLET(%s)", tabletStr)); params.put("ndvFunction", getNdvFunction(String.valueOf(totalRowCount))); @@ -167,11 +167,8 @@ protected void doSample() throws Exception { sql = stringSubstitutor.replace(LINEAR_ANALYZE_TEMPLATE); } else { params.put("dataSizeFunction", getDataSizeFunction(col, true)); - if (col.getType().isStringType()) { - sql = stringSubstitutor.replace(DUJ1_ANALYZE_STRING_TEMPLATE); - } else { - sql = stringSubstitutor.replace(DUJ1_ANALYZE_TEMPLATE); - } + params.put("subStringColName", getStringTypeColName(col)); + sql = stringSubstitutor.replace(DUJ1_ANALYZE_TEMPLATE); } LOG.info("Sample for column [{}]. Total rows [{}], rows to sample [{}], scale factor [{}], " + "limited [{}], distribute column [{}], partition column [{}], key column [{}], " @@ -195,7 +192,7 @@ protected ResultRow collectBasicStat(AutoCloseConnectContext context) { Map params = new HashMap<>(); params.put("dbName", db.getFullName()); params.put("colName", StatisticsUtil.escapeColumnName(info.colName)); - params.put("tblName", tbl.getName()); + params.put("tblName", String.valueOf(tbl.getName())); params.put("index", getIndex()); StringSubstitutor stringSubstitutor = new StringSubstitutor(params); String sql = stringSubstitutor.replace(BASIC_STATS_TEMPLATE); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java index 593ae475810649..8db3dd396b42db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java @@ -64,8 +64,6 @@ public class StatisticConstants { public static List SYSTEM_DBS = new ArrayList<>(); - public static int ANALYZE_TASK_RETRY_TIMES = 5; - public static final String DB_NAME = SystemInfoService.DEFAULT_CLUSTER + ":" + FeConstants.INTERNAL_DB_NAME; public static final String FULL_QUALIFIED_STATS_TBL_NAME = InternalCatalog.INTERNAL_CATALOG_NAME diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticRange.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticRange.java index 7b7b08ab24669d..2ca9b50eaeefeb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticRange.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticRange.java @@ -124,6 +124,10 @@ public boolean isInfinite() { return Double.isInfinite(low) || Double.isInfinite(high); } + public boolean isOneSideInfinite() { + return isInfinite() && !isBothInfinite(); + } + public boolean isFinite() { return Double.isFinite(low) && Double.isFinite(high); } @@ -175,22 +179,29 @@ public Pair maxPair(double r1, LiteralExpr e1, double r2, L } public StatisticRange cover(StatisticRange other) { - // double newLow = Math.max(low, other.low); - // double newHigh = Math.min(high, other.high); + StatisticRange resultRange; Pair biggerLow = maxPair(low, lowExpr, other.low, other.lowExpr); double newLow = biggerLow.first; LiteralExpr newLowExpr = biggerLow.second; Pair smallerHigh = minPair(high, highExpr, other.high, other.highExpr); double newHigh = smallerHigh.first; LiteralExpr newHighExpr = smallerHigh.second; - if (newLow <= newHigh) { double overlapPercentOfLeft = overlapPercentWith(other); double overlapDistinctValuesLeft = overlapPercentOfLeft * distinctValues; double coveredDistinctValues = minExcludeNaN(distinctValues, overlapDistinctValuesLeft); - return new StatisticRange(newLow, newLowExpr, newHigh, newHighExpr, coveredDistinctValues, dataType); + if (this.isBothInfinite() && other.isOneSideInfinite()) { + resultRange = new StatisticRange(newLow, newLowExpr, newHigh, newHighExpr, + distinctValues * INFINITE_TO_INFINITE_RANGE_INTERSECT_OVERLAP_HEURISTIC_FACTOR, + dataType); + } else { + resultRange = new StatisticRange(newLow, newLowExpr, newHigh, newHighExpr, coveredDistinctValues, + dataType); + } + } else { + resultRange = empty(dataType); } - return empty(dataType); + return resultRange; } public StatisticRange union(StatisticRange other) { @@ -241,6 +252,6 @@ public double getDistinctValues() { @Override public String toString() { - return "(" + lowExpr + "," + highExpr + ")"; + return "range=(" + lowExpr + "," + highExpr + "), ndv=" + distinctValues; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java index 5cad3dbaec1cb0..d0673998b5476d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java @@ -24,7 +24,6 @@ import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; -import org.apache.doris.statistics.AnalysisInfo.AnalysisMethod; import org.apache.doris.statistics.AnalysisInfo.JobType; import org.apache.doris.statistics.util.StatisticsUtil; @@ -93,7 +92,7 @@ public class TableStatsMeta implements Writable, GsonPostProcessable { public boolean userInjected; @SerializedName("irc") - public ConcurrentMap indexesRowCount = new ConcurrentHashMap<>(); + private ConcurrentMap indexesRowCount = new ConcurrentHashMap<>(); @VisibleForTesting public TableStatsMeta() { @@ -167,7 +166,9 @@ public void reset() { public void update(AnalysisInfo analyzedJob, TableIf tableIf) { updatedTime = analyzedJob.tblUpdateTime; - userInjected = analyzedJob.userInject; + if (analyzedJob.userInject) { + userInjected = true; + } String colNameStr = analyzedJob.colName; // colName field AnalyzeJob's format likes: "[col1, col2]", we need to remove brackets here // TODO: Refactor this later @@ -195,9 +196,6 @@ public void update(AnalysisInfo analyzedJob, TableIf tableIf) { indexesRowCount.putAll(analyzedJob.indexesRowCount); clearStaleIndexRowCount((OlapTable) tableIf); } - if (analyzedJob.emptyJob && AnalysisMethod.SAMPLE.equals(analyzedJob.analysisMethod)) { - return; - } if (analyzedJob.colToPartitions.keySet() .containsAll(tableIf.getBaseSchema().stream() .filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) @@ -205,6 +203,10 @@ public void update(AnalysisInfo analyzedJob, TableIf tableIf) { updatedRows.set(0); newPartitionLoaded.set(false); } + // Set userInject back to false after manual analyze. + if (JobType.MANUAL.equals(jobType) && !analyzedJob.userInject) { + userInjected = false; + } } } @@ -225,6 +227,10 @@ public long getRowCount(long indexId) { return indexesRowCount.getOrDefault(indexId, -1L); } + public void clearIndexesRowCount() { + indexesRowCount.clear(); + } + private void clearStaleIndexRowCount(OlapTable table) { Iterator iterator = indexesRowCount.keySet().iterator(); List indexIds = table.getIndexIds(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java index e0eef39a217643..fe32755be455a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java @@ -976,37 +976,26 @@ public static boolean isMvColumn(TableIf table, String columnName) { } public static boolean isEmptyTable(TableIf table, AnalysisInfo.AnalysisMethod method) { - int waitRowCountReportedTime = 90; + int waitRowCountReportedTime = 120; if (!(table instanceof OlapTable) || method.equals(AnalysisInfo.AnalysisMethod.FULL)) { return false; } OlapTable olapTable = (OlapTable) table; + long rowCount = 0; for (int i = 0; i < waitRowCountReportedTime; i++) { - if (olapTable.getRowCount() > 0) { - return false; - } - boolean allInitVersion = true; - // If all partitions' visible version are PARTITION_INIT_VERSION, return true. - // If any partition's visible version is greater than 2, return true. - // Otherwise, wait row count to be reported. - for (Partition p : olapTable.getPartitions()) { - if (p.getVisibleVersion() != Partition.PARTITION_INIT_VERSION) { - allInitVersion = false; + rowCount = olapTable.getRowCountForIndex(olapTable.getBaseIndexId(), true); + // rowCount == -1 means new table or first load row count not fully reported, need to wait. + if (rowCount == -1) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + LOG.info("Sleep interrupted."); } - if (p.getVisibleVersion() > Partition.PARTITION_INIT_VERSION + 1) { - return true; - } - } - if (allInitVersion) { - return true; - } - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - LOG.info("Sleep interrupted.", e); + continue; } + break; } - return true; + return rowCount == 0; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/SnapshotTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/SnapshotTask.java index 71b3570f2882f8..81177305683dee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/SnapshotTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/SnapshotTask.java @@ -29,6 +29,7 @@ public class SnapshotTask extends AgentTask { private int schemaHash; private long timeoutMs; private boolean isRestoreTask; + private Long refTabletId; // Set to true if this task for AdminCopyTablet. // Otherwise, it is for Backup/Restore operation. @@ -98,13 +99,23 @@ public String getResultSnapshotPath() { return resultSnapshotPath; } + public void setRefTabletId(long refTabletId) { + assert refTabletId > 0; + this.refTabletId = refTabletId; + } + public TSnapshotRequest toThrift() { TSnapshotRequest request = new TSnapshotRequest(tabletId, schemaHash); - request.setVersion(version); request.setListFiles(true); request.setPreferredSnapshotVersion(TypesConstants.TPREFER_SNAPSHOT_REQ_VERSION); request.setTimeout(timeoutMs / 1000); request.setIsCopyTabletTask(isCopyTabletTask); + if (refTabletId != null) { + request.setRefTabletId(refTabletId); + } + if (version > 0L) { + request.setVersion(version); + } return request; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java index 3bc676a1b422d8..869c82817607a5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java @@ -84,7 +84,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; @@ -119,14 +118,6 @@ private enum PublishResult { // transactionId -> running TransactionState private final Map idToRunningTransactionState = Maps.newHashMap(); - /** - * the multi table ids that are in transaction, used to check whether a table is in transaction - * multi table transaction state - * txnId -> tableId list - */ - private final ConcurrentHashMap> multiTableRunningTransactionTableIdMaps = - new ConcurrentHashMap<>(); - // transactionId -> final status TransactionState private final Map idToFinalStatusTransactionState = Maps.newHashMap(); @@ -436,8 +427,13 @@ public void preCommitTransaction2PC(List
tableList, long transactionId, checkCommitStatus(tableList, transactionState, tabletCommitInfos, txnCommitAttachment, errorReplicaIds, tableToPartition, totalInvolvedBackends); - unprotectedPreCommitTransaction2PC(transactionState, errorReplicaIds, tableToPartition, - totalInvolvedBackends, db); + writeLock(); + try { + unprotectedPreCommitTransaction2PC(transactionState, errorReplicaIds, tableToPartition, + totalInvolvedBackends, db); + } finally { + writeUnlock(); + } LOG.info("transaction:[{}] successfully pre-committed", transactionState); } @@ -1956,6 +1952,9 @@ private boolean updateCatalogAfterVisible(TransactionState transactionState, Dat } } replica.updateVersionWithFailed(newVersion, lastFailedVersion, lastSuccessVersion); + if (newVersion == Partition.PARTITION_INIT_VERSION + 1) { + index.setRowCountReported(false); + } } } } // end for indices diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java index 35c2195eb33055..614843dcbbd85c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java @@ -207,7 +207,7 @@ public void preCommitTransaction2PC(Database db, List
tableList, long tra } } - public void preCommitTransaction2PC(long dbId, List
tableList, long transactionId, + private void preCommitTransaction2PC(long dbId, List
tableList, long transactionId, List tabletCommitInfos, TxnCommitAttachment txnCommitAttachment) throws UserException { if (Config.disable_load_job) { @@ -219,6 +219,7 @@ public void preCommitTransaction2PC(long dbId, List
tableList, long trans dbTransactionMgr.preCommitTransaction2PC(tableList, transactionId, tabletCommitInfos, txnCommitAttachment); } + @Deprecated public void commitTransaction(long dbId, List
tableList, long transactionId, List tabletCommitInfos) throws UserException { @@ -675,6 +676,7 @@ public void abortTxnWhenCoordinateBeRestart(long coordinateBeId, String coordina TransactionState transactionState = dbTransactionMgr.getTransactionState(txnInfo.second); long coordStartTime = transactionState.getCoordinator().startTime; if (coordStartTime < beStartTime) { + // does not hold table write lock dbTransactionMgr.abortTransaction(txnInfo.second, "coordinate BE restart", null); } } catch (UserException e) { @@ -692,6 +694,7 @@ public void abortTxnWhenCoordinateBeDown(long coordinateBeId, String coordinateH = getPrepareTransactionIdByCoordinateBe(coordinateBeId, coordinateHost, limit); for (Pair txnInfo : transactionIdByCoordinateBe) { try { + // does not hold table write lock DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(txnInfo.first); dbTransactionMgr.abortTransaction(txnInfo.second, "coordinate BE is down", null); } catch (UserException e) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ColumnDefTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ColumnDefTest.java index fc9bd4a7ac6c14..9649b8523d69ca 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ColumnDefTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ColumnDefTest.java @@ -89,7 +89,7 @@ public void testReplaceIfNotNull() throws AnalysisException { ColumnDef column = new ColumnDef("col", intCol, false, AggregateType.REPLACE_IF_NOT_NULL, false, DefaultValue.NOT_SET, ""); column.analyze(true); Assert.assertEquals(AggregateType.REPLACE_IF_NOT_NULL, column.getAggregateType()); - Assert.assertEquals("`col` int REPLACE_IF_NOT_NULL NULL DEFAULT \"null\" COMMENT \"\"", column.toSql()); + Assert.assertEquals("`col` int REPLACE_IF_NOT_NULL NULL DEFAULT NULL COMMENT \"\"", column.toSql()); } // CHECKSTYLE IGNORE THIS LINE { // CHECKSTYLE IGNORE THIS LINE // not allow null @@ -98,6 +98,12 @@ public void testReplaceIfNotNull() throws AnalysisException { Assert.assertEquals(AggregateType.REPLACE_IF_NOT_NULL, column.getAggregateType()); Assert.assertEquals("`col` int REPLACE_IF_NOT_NULL NULL DEFAULT \"10\" COMMENT \"\"", column.toSql()); } // CHECKSTYLE IGNORE THIS LINE + { // CHECKSTYLE IGNORE THIS LINE + ColumnDef column = new ColumnDef("col", intCol, false, AggregateType.REPLACE_IF_NOT_NULL, true, DefaultValue.NULL_DEFAULT_VALUE, ""); + column.analyze(true); + Assert.assertEquals(AggregateType.REPLACE_IF_NOT_NULL, column.getAggregateType()); + Assert.assertEquals("`col` int REPLACE_IF_NOT_NULL NULL DEFAULT NULL COMMENT \"\"", column.toSql()); + } // CHECKSTYLE IGNORE THIS LINE } @Test(expected = AnalysisException.class) diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreFileMappingTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreFileMappingTest.java index d37a63f6d14bf2..85de627fa447b4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreFileMappingTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreFileMappingTest.java @@ -31,14 +31,14 @@ public class RestoreFileMappingTest { @Before public void setUp() { - src = new IdChain(10005L, 10006L, 10005L, 10007L, 10008L); - dest = new IdChain(10004L, 10003L, 10004L, 10007L, -1L); + src = new IdChain(10005L, 10006L, 10005L, 10007L, 10008L, -1L); + dest = new IdChain(10004L, 10003L, 10004L, 10007L, -1L, -1L); fileMapping.putMapping(src, dest, true); } @Test public void test() { - IdChain key = new IdChain(10005L, 10006L, 10005L, 10007L, 10008L); + IdChain key = new IdChain(10005L, 10006L, 10005L, 10007L, 10008L, -1L); Assert.assertEquals(key, src); Assert.assertEquals(src, key); IdChain val = fileMapping.get(key); diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java index 8026d47741c226..7e8e55eea327c4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java @@ -251,7 +251,8 @@ boolean await(long timeout, TimeUnit unit) { db.dropTable(expectedRestoreTbl.getName()); job = new RestoreJob(label, "2018-01-01 01:01:01", db.getId(), db.getFullName(), jobInfo, false, - new ReplicaAllocation((short) 3), 100000, -1, false, false, false, false, false, env, repo.getId()); + new ReplicaAllocation((short) 3), 100000, -1, false, false, false, false, false, false, + env, repo.getId()); List
tbls = Lists.newArrayList(); List resources = Lists.newArrayList(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java index 2f93ee5beaa152..b1684ef74a0420 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java @@ -22,6 +22,7 @@ import org.apache.doris.analysis.ShowSqlBlockRuleStmt; import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.ExceptionChecker; import org.apache.doris.metric.MetricRepo; @@ -303,4 +304,28 @@ public void testIfNotExists() throws Exception { () -> dropSqlBlockRule("DROP SQL_BLOCK_RULE test_rule")); dropSqlBlockRule("DROP SQL_BLOCK_RULE if exists test_rule"); } + + @Test + public void testIgnoreAdmin() throws Exception { + String sql = "select * from test_table1 limit 10"; + String sqlHash = DigestUtils.md5Hex(sql); + String sqlRule = "CREATE SQL_BLOCK_RULE test_rule PROPERTIES(\"sql\"=\"select \\\\* from test_table1\"," + + " \"global\"=\"true\", \"enable\"=\"true\");"; + createSqlBlockRule(sqlRule); + Config.sql_block_rule_ignore_admin = false; + ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, + "sql match regex sql block rule: test_rule", + () -> mgr.matchSql(sql, sqlHash, "root")); + ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, + "sql match regex sql block rule: test_rule", + () -> mgr.matchSql(sql, sqlHash, "admin")); + Config.sql_block_rule_ignore_admin = true; + ExceptionChecker.expectThrowsNoException(() -> mgr.matchSql(sql, sqlHash, "root")); + ExceptionChecker.expectThrowsNoException(() -> mgr.matchSql(sql, sqlHash, "admin")); + ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, + "sql match regex sql block rule: test_rule", + () -> mgr.matchSql(sql, sqlHash, "other_user")); + Config.sql_block_rule_ignore_admin = false; + dropSqlBlockRule(dropSqlRule); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java index aa85178c08e405..fd394353cd21d5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java @@ -121,4 +121,51 @@ public void testResetPropertiesForRestore() { Assert.assertFalse(olapTable.getTableProperty().getDynamicPartitionProperty().getEnable()); Assert.assertEquals((short) 3, olapTable.getDefaultReplicaAllocation().getTotalReplicaNum()); } + + @Test + public void testGetPartitionRowCount() { + OlapTable olapTable = new OlapTable(); + // Partition is null. + long row = olapTable.getRowCountForPartitionIndex(0, 0, true); + Assert.assertEquals(-1, row); + + // Index is null. + MaterializedIndex index = new MaterializedIndex(10, MaterializedIndex.IndexState.NORMAL); + Partition partition = new Partition(11, "p1", index, null); + olapTable.addPartition(partition); + row = olapTable.getRowCountForPartitionIndex(11, 0, true); + Assert.assertEquals(-1, row); + + // Strict is true and index is not reported. + index.setRowCountReported(false); + index.setRowCount(100); + row = olapTable.getRowCountForPartitionIndex(11, 10, true); + Assert.assertEquals(-1, row); + + // Strict is true and index is reported. + index.setRowCountReported(true); + index.setRowCount(101); + row = olapTable.getRowCountForPartitionIndex(11, 10, true); + Assert.assertEquals(101, row); + + // Strict is false and index is not reported. + index.setRowCountReported(false); + index.setRowCount(102); + row = olapTable.getRowCountForPartitionIndex(11, 10, false); + Assert.assertEquals(102, row); + + // Reported row is -1, we should return 0 + index.setRowCountReported(true); + index.setRowCount(-1); + row = olapTable.getRowCountForPartitionIndex(11, 10, false); + Assert.assertEquals(0, row); + + // Return reported row. + index.setRowCountReported(true); + index.setRowCount(103); + row = olapTable.getRowCountForPartitionIndex(11, 10, false); + Assert.assertEquals(103, row); + + olapTable.getRowCountForPartitionIndex(11, 10, true); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/CatalogMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/CatalogMgrTest.java index 95d1826017d35b..4a3dd818790a95 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/CatalogMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/CatalogMgrTest.java @@ -101,13 +101,6 @@ protected void runBeforeAll() throws Exception { // grant with no catalog is switched, internal catalog works. CreateRoleStmt createRole1 = (CreateRoleStmt) parseAndAnalyzeStmt("create role role1;", rootCtx); auth.createRole(createRole1); - GrantStmt grantRole1 = (GrantStmt) parseAndAnalyzeStmt("grant grant_priv on tpch.* to role 'role1';", rootCtx); - auth.grant(grantRole1); - // grant with ctl.db.tbl. grant can succeed even if the catalog does not exist - GrantStmt grantRole1WithCtl = (GrantStmt) parseAndAnalyzeStmt( - "grant select_priv on testc.testdb.* to role 'role1';", rootCtx); - auth.grant(grantRole1WithCtl); - // user1 can't switch to hive auth.createUser((CreateUserStmt) parseAndAnalyzeStmt( "create user 'user1'@'%' identified by 'pwd1' default role 'role1';", rootCtx)); user1 = new UserIdentity("user1", "%"); @@ -152,7 +145,8 @@ protected void runBeforeAll() throws Exception { env.changeCatalog(rootCtx, switchHive.getCatalogName()); CreateRoleStmt createRole2 = (CreateRoleStmt) parseAndAnalyzeStmt("create role role2;", rootCtx); auth.createRole(createRole2); - GrantStmt grantRole2 = (GrantStmt) parseAndAnalyzeStmt("grant grant_priv on tpch.customer to role 'role2';", + GrantStmt grantRole2 = (GrantStmt) parseAndAnalyzeStmt( + "grant grant_priv, select_priv on hive.*.* to role 'role2';", rootCtx); auth.grant(grantRole2); auth.createUser((CreateUserStmt) parseAndAnalyzeStmt( @@ -366,7 +360,7 @@ public void testSwitchCommand() throws Exception { Assert.assertEquals(user2Ctx.getDefaultCatalog(), "hive"); // user2 can grant select_priv to tpch.customer GrantStmt user2GrantHiveTable = (GrantStmt) parseAndAnalyzeStmt( - "grant select_priv on tpch.customer to 'user2'@'%';", user2Ctx); + "grant select_priv on hive.*.* to 'user2'@'%';", user2Ctx); auth.grant(user2GrantHiveTable); showCatalogSql = "SHOW CATALOGS"; diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/jdbc/JdbcClientExceptionTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/jdbc/JdbcClientExceptionTest.java new file mode 100644 index 00000000000000..1bbf54e9438512 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/jdbc/JdbcClientExceptionTest.java @@ -0,0 +1,132 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.jdbc; + +import org.apache.doris.datasource.jdbc.client.JdbcClientException; + +import org.junit.Assert; +import org.junit.Test; + +public class JdbcClientExceptionTest { + + @Test + public void testExceptionWithoutArgs() { + String message = "An error occurred."; + JdbcClientException exception = new JdbcClientException(message); + + Assert.assertEquals(message, exception.getMessage()); + Assert.assertNull(exception.getCause()); + } + + @Test + public void testExceptionWithFormattingArgs() { + String format = "Error code: %d, message: %s"; + int errorCode = 404; + String errorMsg = "Not Found"; + JdbcClientException exception = new JdbcClientException(format, errorCode, errorMsg); + + String expectedMessage = String.format(format, errorCode, errorMsg); + Assert.assertEquals(expectedMessage, exception.getMessage()); + Assert.assertNull(exception.getCause()); + } + + @Test + public void testExceptionWithPercentInFormatString() { + String format = "Usage is at 80%%, threshold is %d%%"; + int threshold = 75; + JdbcClientException exception = new JdbcClientException(format, threshold); + + String expectedMessage = String.format(format, threshold); + Assert.assertEquals(expectedMessage, exception.getMessage()); + Assert.assertNull(exception.getCause()); + } + + @Test + public void testExceptionWithPercentInArgs() { + String format = "Invalid input: %s"; + String input = "50% discount"; + JdbcClientException exception = new JdbcClientException(format, input); + + String expectedMessage = String.format(format, input.replace("%", "%%")); + Assert.assertEquals(expectedMessage, exception.getMessage()); + Assert.assertNull(exception.getCause()); + } + + @Test + public void testExceptionWithCause() { + String message = "Database connection failed."; + Exception cause = new Exception("Timeout occurred"); + JdbcClientException exception = new JdbcClientException(message, cause); + + Assert.assertEquals(message, exception.getMessage()); + Assert.assertEquals(cause, exception.getCause()); + } + + @Test + public void testExceptionWithFormattingArgsAndCause() { + String format = "Failed to execute query: %s"; + String query = "SELECT * FROM users"; + Exception cause = new Exception("Syntax error"); + JdbcClientException exception = new JdbcClientException(format, cause, query); + + String expectedMessage = String.format(format, query); + Assert.assertEquals(expectedMessage, exception.getMessage()); + Assert.assertEquals(cause, exception.getCause()); + } + + @Test + public void testExceptionWithPercentInArgsAndCause() { + String format = "File path: %s"; + String filePath = "C:\\Program Files\\App%20Data"; + Exception cause = new Exception("File not found"); + JdbcClientException exception = new JdbcClientException(format, cause, filePath); + + String expectedMessage = String.format(format, filePath.replace("%", "%%")); + Assert.assertEquals(expectedMessage, exception.getMessage()); + Assert.assertEquals(cause, exception.getCause()); + } + + @Test + public void testExceptionWithNoFormattingNeeded() { + String message = "Simple error message."; + JdbcClientException exception = new JdbcClientException(message, (Object[]) null); + + Assert.assertEquals(message, exception.getMessage()); + Assert.assertNull(exception.getCause()); + } + + @Test + public void testExceptionWithNullArgs() { + String format = "Error occurred: %s"; + JdbcClientException exception = new JdbcClientException(format, (Object[]) null); + + // Since args are null, message should remain unformatted + Assert.assertEquals(format, exception.getMessage()); + Assert.assertNull(exception.getCause()); + } + + @Test + public void testExceptionWithEmptyArgs() { + String format = "Error occurred: %s"; + JdbcClientException exception = new JdbcClientException(format, new Object[]{}); + + // Since args are empty, message should remain unformatted + Assert.assertEquals(format, exception.getMessage()); + Assert.assertNull(exception.getCause()); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java index 842d4d4003f50f..2df0afd0d36991 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java @@ -227,7 +227,7 @@ public void testProcessTimeOutTasks(@Injectable GlobalTransactionMgr globalTrans Map partitionIdsToOffset = Maps.newHashMap(); partitionIdsToOffset.put(100, 0L); KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo(new UUID(1, 1), 1L, "default_cluster", - maxBatchIntervalS * 2 * 1000, 0, partitionIdsToOffset, false); + maxBatchIntervalS * 2 * 1000, partitionIdsToOffset, false); kafkaTaskInfo.setExecuteStartTimeMs(System.currentTimeMillis() - maxBatchIntervalS * 2 * 1000 - 1); routineLoadTaskInfoList.add(kafkaTaskInfo); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java index 0ce694bfb11b20..5a0ee0e5dfa687 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java @@ -68,7 +68,7 @@ public void testRunOneCycle(@Injectable KafkaRoutineLoadJob kafkaRoutineLoadJob1 Deencapsulation.setField(kafkaProgress, "partitionIdToOffset", partitionIdToOffset); LinkedBlockingDeque routineLoadTaskInfoQueue = new LinkedBlockingDeque<>(); - KafkaTaskInfo routineLoadTaskInfo1 = new KafkaTaskInfo(new UUID(1, 1), 1L, "default_cluster", 20000, 0, + KafkaTaskInfo routineLoadTaskInfo1 = new KafkaTaskInfo(new UUID(1, 1), 1L, "default_cluster", 20000, partitionIdToOffset, false); routineLoadTaskInfoQueue.addFirst(routineLoadTaskInfo1); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AdjustNullableTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AdjustNullableTest.java new file mode 100644 index 00000000000000..5a6b7ac1b46497 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AdjustNullableTest.java @@ -0,0 +1,75 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.GreaterThan; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.logical.LogicalJdbcScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.util.MemoPatternMatchSupported; +import org.apache.doris.nereids.util.PlanConstructor; + +import mockit.Mock; +import mockit.MockUp; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; + +/** + * Tests for {@link AdjustNullableTest}. + */ +class AdjustNullableTest implements MemoPatternMatchSupported { + private final LogicalOlapScan scan = PlanConstructor.newLogicalOlapScan(0, "t1", 0); + + @Test + void testLogicalExternalRelation() { + new MockUp() { + @Mock + public Set getOutputSet() { + Set output = new HashSet<>(); + output.add(new SlotReference(new ExprId(1), "id", IntegerType.INSTANCE, false, + new ArrayList<>())); + return output; + } + + }; + + GreaterThan gt = new GreaterThan(new SlotReference(new ExprId(1), "id", + IntegerType.INSTANCE, true, new ArrayList<>()), Literal.of("1")); + Set conjuncts = new HashSet<>(); + conjuncts.add(gt); + Assertions.assertTrue(conjuncts.iterator().next().nullable()); + LogicalJdbcScan jdbcScan = + new LogicalJdbcScan(new RelationId(1), PlanConstructor.newOlapTable(0, "t1", 0), + new ArrayList<>(), Optional.empty(), Optional.empty(), conjuncts); + AdjustNullable adjustNullable = new AdjustNullable(); + LogicalJdbcScan newJdbcScan = (LogicalJdbcScan) adjustNullable.rewriteRoot(jdbcScan, null); + conjuncts = newJdbcScan.getConjuncts(); + Assertions.assertFalse(conjuncts.iterator().next().nullable()); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java index d0f6b3da0772cb..a5410965b69914 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java @@ -1292,4 +1292,40 @@ public void testStringRangeColToCol() { Statistics agrtc = new FilterEstimation().estimate(new GreaterThan(a, c), baseStats); Assertions.assertEquals(50, agrtc.getRowCount()); } + + @Test + void testAndWithInfinity() { + Double row = 1000.0; + SlotReference a = new SlotReference("a", new VarcharType(25)); + ColumnStatisticBuilder columnStatisticBuilderA = new ColumnStatisticBuilder() + .setNdv(10) + .setAvgSizeByte(4) + .setNumNulls(0) + .setCount(row); + + SlotReference b = new SlotReference("b", IntegerType.INSTANCE); + ColumnStatisticBuilder columnStatisticBuilderB = new ColumnStatisticBuilder() + .setNdv(488) + .setAvgSizeByte(25) + .setNumNulls(0) + .setCount(row); + StatisticsBuilder statsBuilder = new StatisticsBuilder(); + statsBuilder.setRowCount(row); + statsBuilder.putColumnStatistics(a, columnStatisticBuilderA.build()); + statsBuilder.putColumnStatistics(b, columnStatisticBuilderB.build()); + Expression strGE = new GreaterThanEqual(a, + new org.apache.doris.nereids.trees.expressions.literal.StringLiteral("2024-05-14")); + Statistics strStats = new FilterEstimation().estimate(strGE, statsBuilder.build()); + Assertions.assertEquals(500, strStats.getRowCount()); + + Expression intGE = new GreaterThan(b, new IntegerLiteral(0)); + Statistics intStats = new FilterEstimation().estimate(intGE, statsBuilder.build()); + Assertions.assertEquals(500, intStats.getRowCount()); + + Expression predicate = new And(strGE, intGE); + + Statistics stats = new FilterEstimation().estimate(predicate, statsBuilder.build()); + Assertions.assertEquals(250, stats.getRowCount()); + } + } diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/BaseAnalysisTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/BaseAnalysisTaskTest.java index 187c4d207dfcad..86ccfff26e0375 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/BaseAnalysisTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/BaseAnalysisTaskTest.java @@ -31,7 +31,7 @@ public void testGetFunctions() { OlapAnalysisTask olapAnalysisTask = new OlapAnalysisTask(); Column column = new Column("string_column", PrimitiveType.STRING); String dataSizeFunction = olapAnalysisTask.getDataSizeFunction(column, true); - Assertions.assertEquals("SUM(LENGTH(`column_key`) * count)", dataSizeFunction); + Assertions.assertEquals("SUM(`column_length`)", dataSizeFunction); dataSizeFunction = olapAnalysisTask.getDataSizeFunction(column, false); Assertions.assertEquals("SUM(LENGTH(`${colName}`))", dataSizeFunction); diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java index 75506b1c85a014..a78bc81cf6b5a1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/OlapAnalysisTaskTest.java @@ -160,9 +160,11 @@ public void runQuery(String sql) { + "SUBSTRING(CAST('1' AS STRING), 1, 1024) AS `min`," + " SUBSTRING(CAST('2' AS STRING), 1, 1024) AS `max`, " + "SUM(t1.count) * 4 * 5.0 AS `data_size`, NOW() " - + "FROM ( SELECT t0.`${colName}` as `column_key`, COUNT(1) " - + "as `count` FROM (SELECT `${colName}` FROM `catalogName`.`${dbName}`.`${tblName}`" - + " limit 100) as `t0` GROUP BY `t0`.`${colName}` ) as `t1` ", sql); + + "FROM ( SELECT t0.`colValue` as `column_key`, COUNT(1) " + + "as `count`, SUM(`len`) as `column_length` FROM " + + "(SELECT `null` AS `colValue`, LENGTH(`null`) as `len` " + + "FROM `catalogName`.`${dbName}`.`null`" + + " limit 100) as `t0` GROUP BY `t0`.`colValue` ) as `t1` ", sql); return; } }; @@ -232,12 +234,12 @@ public void runQuery(String sql) { + "SELECT CONCAT(30001, '-', -1, '-', 'null') AS `id`, " + "10001 AS `catalog_id`, 20001 AS `db_id`, 30001 AS `tbl_id`, " + "-1 AS `idx_id`, 'null' AS `col_id`, NULL AS `part_id`, " - + "500 AS `row_count`, ROUND(NDV(`${colName}`) * 5.0) as `ndv`, " - + "ROUND(SUM(CASE WHEN `${colName}` IS NULL THEN 1 ELSE 0 END) * 5.0) " + + "500 AS `row_count`, ROUND(NDV(`null`) * 5.0) as `ndv`, " + + "ROUND(SUM(CASE WHEN `null` IS NULL THEN 1 ELSE 0 END) * 5.0) " + "AS `null_count`, SUBSTRING(CAST('1' AS STRING), 1, 1024) AS `min`, " + "SUBSTRING(CAST('2' AS STRING), 1, 1024) AS `max`, " - + "SUM(LENGTH(`${colName}`)) * 5.0 AS `data_size`, NOW() " - + "FROM `catalogName`.`${dbName}`.`${tblName}` limit 100", sql); + + "SUM(LENGTH(`null`)) * 5.0 AS `data_size`, NOW() " + + "FROM `catalogName`.`${dbName}`.`null` limit 100", sql); return; } }; @@ -320,9 +322,12 @@ public void runQuery(String sql) { + "IS NULL, `t1`.`count`, 0)), 0) * 5.0 as `null_count`, " + "SUBSTRING(CAST('1' AS STRING), 1, 1024) AS `min`, " + "SUBSTRING(CAST('2' AS STRING), 1, 1024) AS `max`, " - + "SUM(LENGTH(`column_key`) * count) * 5.0 AS `data_size`, NOW() " - + "FROM ( SELECT t0.`colValue` as `column_key`, COUNT(1) as `count` FROM " - + "(SELECT SUBSTRING(CAST(`${colName}` AS STRING), 1, 1024) AS `colValue` FROM `catalogName`.`${dbName}`.`${tblName}` limit 100) as `t0` GROUP BY `t0`.`colValue` ) as `t1` ", sql); + + "SUM(`column_length`) * 5.0 AS `data_size`, NOW() " + + "FROM ( SELECT t0.`colValue` as `column_key`, COUNT(1) as `count`, SUM(`len`) as " + + "`column_length` FROM (SELECT murmur_hash3_64(SUBSTRING(CAST(`null` AS STRING), 1, 1024)) " + + "AS `colValue`, LENGTH(`null`) as `len`" + + " FROM `catalogName`.`${dbName}`.`null` limit 100) as `t0` " + + "GROUP BY `t0`.`colValue` ) as `t1` ", sql); return; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java index ea63a5e18b10d0..9a1a934e0d52d7 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java @@ -110,7 +110,7 @@ public Map addTransactionToTransactionMgr() throws UserException { transTablets.add(tabletCommitInfo3); Table testTable1 = masterEnv.getInternalCatalog().getDbOrMetaException(CatalogTestUtil.testDbId1) .getTableOrMetaException(CatalogTestUtil.testTableId1); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId1, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId1, transTablets, null); TransactionState transactionState1 = fakeEditLog.getTransaction(transactionId1); setTransactionFinishPublish(transactionState1, Lists.newArrayList(CatalogTestUtil.testBackendId1, diff --git a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java index cc00237a4c48ed..7337ef64532829 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java @@ -170,7 +170,7 @@ public void testCommitTransaction1() throws UserException { Table testTable1 = masterEnv.getInternalCatalog().getDbOrMetaException(CatalogTestUtil.testDbId1) .getTableOrMetaException(CatalogTestUtil.testTableId1); masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, - transTablets); + transTablets, null); TransactionState transactionState = fakeEditLog.getTransaction(transactionId); // check status is committed Assert.assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); @@ -210,7 +210,8 @@ public void testCommitTransactionWithOneFailed() throws UserException { transTablets.add(tabletCommitInfo2); Table testTable1 = masterEnv.getInternalCatalog().getDbOrMetaException(CatalogTestUtil.testDbId1) .getTableOrMetaException(CatalogTestUtil.testTableId1); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, + transTablets, null); // follower catalog replay the transaction transactionState = fakeEditLog.getTransaction(transactionId); @@ -231,7 +232,8 @@ public void testCommitTransactionWithOneFailed() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo3); try { - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, + transTablets, null); Assert.fail(); } catch (TabletQuorumFailedException e) { transactionState = masterTransMgr.getTransactionState(CatalogTestUtil.testDbId1, transactionId2); @@ -260,7 +262,8 @@ public void testCommitTransactionWithOneFailed() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); transTablets.add(tabletCommitInfo3); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, + transTablets, null); transactionState = fakeEditLog.getTransaction(transactionId2); // check status is commit Assert.assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); @@ -319,7 +322,7 @@ public void testCommitRoutineLoadTransaction(@Injectable TabletCommitInfo tablet List routineLoadTaskInfoList = Deencapsulation.getField(routineLoadJob, "routineLoadTaskInfoList"); Map partitionIdToOffset = Maps.newHashMap(); partitionIdToOffset.put(1, 0L); - KafkaTaskInfo routineLoadTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), 1L, "default_cluster", 20000, 0, + KafkaTaskInfo routineLoadTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), 1L, "default_cluster", 20000, partitionIdToOffset, false); Deencapsulation.setField(routineLoadTaskInfo, "txnId", 1L); routineLoadTaskInfoList.add(routineLoadTaskInfo); @@ -393,7 +396,7 @@ public void testCommitRoutineLoadTransactionWithErrorMax(@Injectable TabletCommi List routineLoadTaskInfoList = Deencapsulation.getField(routineLoadJob, "routineLoadTaskInfoList"); Map partitionIdToOffset = Maps.newHashMap(); partitionIdToOffset.put(1, 0L); - KafkaTaskInfo routineLoadTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), 1L, "defualt_cluster", 20000, 0, + KafkaTaskInfo routineLoadTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), 1L, "defualt_cluster", 20000, partitionIdToOffset, false); Deencapsulation.setField(routineLoadTaskInfo, "txnId", 1L); routineLoadTaskInfoList.add(routineLoadTaskInfo); @@ -465,7 +468,8 @@ public void testFinishTransaction() throws UserException { transTablets.add(tabletCommitInfo3); Table testTable1 = masterEnv.getInternalCatalog().getDbOrMetaException(CatalogTestUtil.testDbId1) .getTableOrMetaException(CatalogTestUtil.testTableId1); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, + transTablets, null); TransactionState transactionState = fakeEditLog.getTransaction(transactionId); Assert.assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); slaveTransMgr.replayUpsertTransactionState(transactionState); @@ -519,7 +523,8 @@ public void testFinishTransactionWithOneFailed() throws UserException { transTablets.add(tabletCommitInfo2); Table testTable1 = masterEnv.getInternalCatalog().getDbOrMetaException(CatalogTestUtil.testDbId1) .getTableOrMetaException(CatalogTestUtil.testTableId1); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, + transTablets, null); // follower catalog replay the transaction transactionState = fakeEditLog.getTransaction(transactionId); @@ -582,7 +587,8 @@ public void testFinishTransactionWithOneFailed() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo3); try { - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, + transTablets, null); Assert.fail(); } catch (TabletQuorumFailedException e) { transactionState = masterTransMgr.getTransactionState(CatalogTestUtil.testDbId1, transactionId2); @@ -598,7 +604,8 @@ public void testFinishTransactionWithOneFailed() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); transTablets.add(tabletCommitInfo3); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, + transTablets, null); transactionState = fakeEditLog.getTransaction(transactionId2); // check status is commit Assert.assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); diff --git a/gensrc/proto/descriptors.proto b/gensrc/proto/descriptors.proto index 8c52780ba6b689..723e757c4145bc 100644 --- a/gensrc/proto/descriptors.proto +++ b/gensrc/proto/descriptors.proto @@ -70,5 +70,8 @@ message POlapTableSchemaParam { optional bool is_strict_mode = 9 [default = false]; optional int64 timestamp_ms = 11 [default = 0]; optional string timezone = 12; + // not used, for upgrade compatibility + optional int32 auto_increment_column_unique_id = 13 [default = -1]; + optional int32 nano_seconds = 14 [default = 0]; }; diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index ba897ea21b2670..0c44dec1bd47e1 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -373,4 +373,5 @@ message PartialUpdateInfoPB { optional bool is_schema_contains_auto_inc_column = 10 [default = false]; repeated string default_values = 11; optional int64 max_version_in_flush_phase = 12 [default = -1]; + optional int32 nano_seconds = 13 [default = 0]; } diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index 79d3eb158d178b..9d9693c5d94bf7 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -344,6 +344,7 @@ struct TSnapshotRequest { 11: optional Types.TVersion start_version 12: optional Types.TVersion end_version 13: optional bool is_copy_binlog + 14: optional Types.TTabletId ref_tablet_id } struct TReleaseSnapshotRequest { diff --git a/gensrc/thrift/BackendService.thrift b/gensrc/thrift/BackendService.thrift index eb9ad6a64d8baf..84868bbdf87fef 100644 --- a/gensrc/thrift/BackendService.thrift +++ b/gensrc/thrift/BackendService.thrift @@ -36,6 +36,7 @@ struct TTabletStat { 3: optional i64 row_num 4: optional i64 version_count 5: optional i64 remote_data_size + 6: optional i64 visible_version } struct TTabletStatResult { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 69aa37964dcc77..a3c25c17616b82 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1104,6 +1104,7 @@ struct TRestoreSnapshotRequest { 12: optional binary job_info 13: optional bool clean_tables 14: optional bool clean_partitions + 15: optional bool atomic_restore } struct TRestoreSnapshotResult { @@ -1229,6 +1230,7 @@ struct TGetMetaDBMeta { 3: optional list tables 4: optional list dropped_partitions 5: optional list dropped_tables + 6: optional list dropped_indexes } struct TGetMetaResult { diff --git a/regression-test/data/backup_restore/test_backup_restore_atomic.out b/regression-test/data/backup_restore/test_backup_restore_atomic.out new file mode 100644 index 00000000000000..bee7a4da44f1fc --- /dev/null +++ b/regression-test/data/backup_restore/test_backup_restore_atomic.out @@ -0,0 +1,78 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +10 10 +20 20 +30 30 +40 40 +50 50 +60 60 +70 70 +80 80 +90 90 +100 100 + +-- !sql -- +10 10 +20 20 + +-- !sql -- +10 10 +20 20 +30 30 +40 40 +50 50 +60 60 +70 70 +80 80 +90 90 +100 100 + +-- !sql -- +10 10 +20 20 +30 30 +40 40 +50 50 +60 60 +70 70 +80 80 +90 90 +100 100 + +-- !sql -- +10 10 +20 20 +30 30 +40 40 +50 50 +60 60 +70 70 +80 80 +90 90 +100 100 + +-- !sql -- +10 10 +20 20 +30 30 +40 40 +50 50 +60 60 +70 70 +80 80 +90 90 +100 100 + +-- !sql -- +10 20 +20 40 +30 60 +40 80 +50 100 +60 120 +70 140 +80 160 +90 180 +100 200 +200 200 + diff --git a/regression-test/data/backup_restore/test_backup_restore_atomic_with_view.out b/regression-test/data/backup_restore/test_backup_restore_atomic_with_view.out new file mode 100644 index 00000000000000..cad6dbe8fd8b5c --- /dev/null +++ b/regression-test/data/backup_restore/test_backup_restore_atomic_with_view.out @@ -0,0 +1,60 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 + +-- !sql -- +6 6 +7 7 +8 8 +9 9 +10 10 + +-- !sql -- +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 + +-- !sql -- +6 6 +7 7 +8 8 +9 9 +10 10 + +-- !sql -- +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 + +-- !sql -- +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 + diff --git a/regression-test/data/backup_restore/test_backup_restore_with_view.out b/regression-test/data/backup_restore/test_backup_restore_with_view.out new file mode 100644 index 00000000000000..465e5d40ac9d35 --- /dev/null +++ b/regression-test/data/backup_restore/test_backup_restore_with_view.out @@ -0,0 +1,39 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 + +-- !sql -- +6 6 +7 7 +8 8 +9 9 +10 10 + +-- !sql -- +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 + +-- !sql -- +6 6 +7 7 +8 8 +9 9 +10 10 + diff --git a/regression-test/data/external_table_p0/tvf/test_hdfs_tvf.out b/regression-test/data/external_table_p0/tvf/test_hdfs_tvf.out index de4f8bd579c248..bc14cd5a9e180b 100644 --- a/regression-test/data/external_table_p0/tvf/test_hdfs_tvf.out +++ b/regression-test/data/external_table_p0/tvf/test_hdfs_tvf.out @@ -616,3 +616,47 @@ string_col text Yes false \N NONE string_col text Yes false \N NONE tinyint_col tinyint Yes false \N NONE +-- !create_view -- +0 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +1 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +10 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +100 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +101 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +102 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +103 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +104 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +105 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +106 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +107 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +108 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +109 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +11 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +110 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +111 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +112 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +113 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +114 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 +115 2 3 4 5 6.6 7.7 8.8 abc def ghiaaaaaa 2020-10-10 2020-10-10 11:12:59 + +-- !alter_view -- +0 +1 +10 +100 +101 +102 +103 +104 +105 +106 +107 +108 +109 +11 +110 +111 +112 +113 +114 +115 + diff --git a/regression-test/data/inverted_index_p0/count-on-index.json b/regression-test/data/inverted_index_p0/count-on-index.json new file mode 100644 index 00000000000000..6ef96d081c657a --- /dev/null +++ b/regression-test/data/inverted_index_p0/count-on-index.json @@ -0,0 +1,100 @@ +{"a": "2024-03-25", "b": "ISZHfv2OQ4", "c": "c1", "d": "d1", "e": "e1"} +{"a": "2023-02-20", "b": "0MgsB3vcIf", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2023-11-13", "b": "tczLqYSRhY", "c": "c1", "d": "d2", "e": "e1"} +{"a": "2023-08-13", "b": "f86oV0P4s8", "c": "c2", "d": "d2", "e": "e2"} +{"a": "2023-07-13", "b": "YHD5LihHpK", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2024-07-26", "b": "iSJxD3yKvH", "c": "c1", "d": "d2", "e": "e1"} +{"a": "2023-12-25", "b": "aNiLbwF1vg", "c": "c1", "d": "d2", "e": "e2"} +{"a": "2024-04-04", "b": "DGih88jW0H", "c": "c3", "d": "d3", "e": "e2"} +{"a": "2024-03-30", "b": "kxqqsrD1RH", "c": "c2", "d": "d3", "e": "e1"} +{"a": "2023-10-01", "b": "eySjxEnJvW", "c": "c2", "d": "d3", "e": "e1"} +{"a": "2023-09-19", "b": "lHAJZpcky5", "c": "c2", "d": "d2", "e": "e1"} +{"a": "2023-11-25", "b": "PB4l4NsNAn", "c": "c1", "d": "d3", "e": "e2"} +{"a": "2023-05-10", "b": "TP5M3xQDCj", "c": "c1", "d": "d2", "e": "e1"} +{"a": "2024-08-28", "b": "b1D8xLX4DN", "c": "c3", "d": "d2", "e": "e1"} +{"a": "2023-05-31", "b": "9gTsOoFITb", "c": "c2", "d": "d3", "e": "e2"} +{"a": "2023-01-14", "b": "rsv96RalgR", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2024-08-31", "b": "Uz67DnurlH", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2023-04-05", "b": "kPlM5F56kj", "c": "c2", "d": "d3", "e": "e2"} +{"a": "2023-09-18", "b": "wSGVCB6s3I", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2023-01-28", "b": "2ZpPp1y5G5", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2023-03-27", "b": "VwdMxBUnrc", "c": "c3", "d": "d1", "e": "e1"} +{"a": "2024-03-17", "b": "QPV71OyuZ2", "c": "c1", "d": "d2", "e": "e1"} +{"a": "2023-08-10", "b": "pyhphs1Mj4", "c": "c1", "d": "d1", "e": "e1"} +{"a": "2024-01-26", "b": "wxRO18q0EY", "c": "c2", "d": "d3", "e": "e1"} +{"a": "2023-11-04", "b": "vfVK2TsjTl", "c": "c2", "d": "d1", "e": "e2"} +{"a": "2023-04-27", "b": "eS6vCuQAXP", "c": "c3", "d": "d2", "e": "e1"} +{"a": "2024-02-25", "b": "6dx8DMPK9f", "c": "c2", "d": "d3", "e": "e2"} +{"a": "2023-04-16", "b": "3aJhtwXa7E", "c": "c2", "d": "d3", "e": "e1"} +{"a": "2024-07-08", "b": "Ue9xroXnHI", "c": "c3", "d": "d3", "e": "e2"} +{"a": "2023-08-19", "b": "gVPYCdN2eY", "c": "c2", "d": "d3", "e": "e2"} +{"a": "2024-02-24", "b": "qAaaKQpvd3", "c": "c2", "d": "d2", "e": "e1"} +{"a": "2024-01-10", "b": "XsNcGPnvvC", "c": "c2", "d": "d1", "e": "e2"} +{"a": "2023-03-04", "b": "BD7CEdzIhP", "c": "c2", "d": "d1", "e": "e1"} +{"a": "2023-03-18", "b": "sXESaouuHE", "c": "c2", "d": "d3", "e": "e1"} +{"a": "2023-06-28", "b": "DD3RE2pufi", "c": "c2", "d": "d3", "e": "e2"} +{"a": "2024-04-07", "b": "RdEFKIz8QW", "c": "c2", "d": "d1", "e": "e2"} +{"a": "2024-05-10", "b": "u3OhzAL2LH", "c": "c3", "d": "d1", "e": "e1"} +{"a": "2024-07-20", "b": "U0n5EVKjPm", "c": "c1", "d": "d1", "e": "e1"} +{"a": "2024-08-07", "b": "TXypE2ItVh", "c": "c3", "d": "d2", "e": "e1"} +{"a": "2024-08-08", "b": "8g3hPyCB2B", "c": "c3", "d": "d2", "e": "e1"} +{"a": "2023-01-05", "b": "aT6WlbObnZ", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2023-06-05", "b": "mVqMi8Rzfi", "c": "c1", "d": "d1", "e": "e2"} +{"a": "2024-02-12", "b": "HxpmQ0draG", "c": "c1", "d": "d1", "e": "e1"} +{"a": "2024-05-20", "b": "R5a7gA61KY", "c": "c1", "d": "d2", "e": "e1"} +{"a": "2023-04-19", "b": "QacPa5V0Fj", "c": "c3", "d": "d3", "e": "e1"} +{"a": "2023-07-29", "b": "dwT8GxkWDA", "c": "c1", "d": "d1", "e": "e1"} +{"a": "2024-06-29", "b": "UCRkZWVEhK", "c": "c3", "d": "d1", "e": "e2"} +{"a": "2023-02-22", "b": "yMSAdFkaq9", "c": "c3", "d": "d1", "e": "e1"} +{"a": "2024-04-18", "b": "6Aa4VUyj7b", "c": "c2", "d": "d1", "e": "e2"} +{"a": "2023-09-20", "b": "xet5tOBGLy", "c": "c1", "d": "d1", "e": "e2"} +{"a": "2024-07-09", "b": "kyE5wM71uC", "c": "c1", "d": "d1", "e": "e2"} +{"a": "2024-03-05", "b": "J9UtyRClVj", "c": "c2", "d": "d2", "e": "e2"} +{"a": "2024-08-02", "b": "BnNajVStTq", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2024-03-20", "b": "YFyjBh6JeE", "c": "c1", "d": "d3", "e": "e1"} +{"a": "2024-01-25", "b": "kF462Dpave", "c": "c1", "d": "d1", "e": "e2"} +{"a": "2023-11-20", "b": "uUsipxur13", "c": "c1", "d": "d3", "e": "e2"} +{"a": "2024-08-06", "b": "g4i1sEGImS", "c": "c2", "d": "d3", "e": "e1"} +{"a": "2024-02-09", "b": "NUz4tjvKt5", "c": "c3", "d": "d1", "e": "e2"} +{"a": "2024-04-20", "b": "p72Gn18tWd", "c": "c3", "d": "d3", "e": "e2"} +{"a": "2024-01-22", "b": "3DriXIqvSg", "c": "c2", "d": "d3", "e": "e2"} +{"a": "2024-07-26", "b": "rorCsbghiO", "c": "c1", "d": "d1", "e": "e2"} +{"a": "2023-10-26", "b": "XAWPiEQVmE", "c": "c3", "d": "d1", "e": "e1"} +{"a": "2023-10-30", "b": "L3FWcbrzen", "c": "c1", "d": "d2", "e": "e2"} +{"a": "2024-06-25", "b": "Lj6SZ26GJN", "c": "c3", "d": "d3", "e": "e1"} +{"a": "2023-07-20", "b": "U6nYzFhfwM", "c": "c2", "d": "d2", "e": "e2"} +{"a": "2023-06-26", "b": "J7jWtTmtZT", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2024-05-27", "b": "hDWYIRDHV4", "c": "c2", "d": "d2", "e": "e2"} +{"a": "2023-05-23", "b": "kvjedf4zF8", "c": "c3", "d": "d3", "e": "e1"} +{"a": "2023-02-06", "b": "RsPN2cGb2L", "c": "c2", "d": "d3", "e": "e1"} +{"a": "2024-06-04", "b": "belvas0y6p", "c": "c3", "d": "d1", "e": "e2"} +{"a": "2024-06-17", "b": "J6vYAcFuGZ", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2024-04-18", "b": "qHuHh0Y29i", "c": "c1", "d": "d2", "e": "e1"} +{"a": "2023-08-20", "b": "1GS5UtXMdz", "c": "c2", "d": "d3", "e": "e1"} +{"a": "2024-01-23", "b": "gnjJ4TZ6A6", "c": "c1", "d": "d1", "e": "e1"} +{"a": "2023-02-10", "b": "LX6ddQvIX2", "c": "c2", "d": "d3", "e": "e1"} +{"a": "2024-03-02", "b": "MkImkgiAfm", "c": "c3", "d": "d1", "e": "e2"} +{"a": "2023-11-30", "b": "tGmu0DD8W0", "c": "c3", "d": "d3", "e": "e1"} +{"a": "2023-02-01", "b": "NgzjCOPAku", "c": "c1", "d": "d3", "e": "e2"} +{"a": "2023-01-03", "b": "5mWMiuYwbi", "c": "c3", "d": "d1", "e": "e2"} +{"a": "2023-02-12", "b": "9324TZqLjh", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2023-06-24", "b": "7fAYU4XSma", "c": "c1", "d": "d3", "e": "e1"} +{"a": "2024-08-28", "b": "iNvBMy8AB8", "c": "c1", "d": "d1", "e": "e1"} +{"a": "2024-06-08", "b": "nwJjFx21my", "c": "c3", "d": "d1", "e": "e1"} +{"a": "2023-03-15", "b": "Oonv8eGNIF", "c": "c3", "d": "d2", "e": "e1"} +{"a": "2024-04-22", "b": "6UYDEqQxxf", "c": "c2", "d": "d2", "e": "e2"} +{"a": "2024-05-08", "b": "aVPqpTufJf", "c": "c1", "d": "d2", "e": "e1"} +{"a": "2024-08-02", "b": "z1baLbjnTj", "c": "c3", "d": "d3", "e": "e2"} +{"a": "2023-07-05", "b": "vMLMALySMJ", "c": "c3", "d": "d3", "e": "e1"} +{"a": "2023-08-30", "b": "s7VZKlOG27", "c": "c3", "d": "d3", "e": "e2"} +{"a": "2023-03-06", "b": "gWAuqLvHpJ", "c": "c2", "d": "d1", "e": "e2"} +{"a": "2024-02-24", "b": "2IQL6pazn3", "c": "c1", "d": "d2", "e": "e1"} +{"a": "2024-08-15", "b": "4YbxyhwaRF", "c": "c3", "d": "d2", "e": "e1"} +{"a": "2023-02-28", "b": "wAOOLl8Kqj", "c": "c1", "d": "d2", "e": "e2"} +{"a": "2024-05-31", "b": "FTQKB8rURb", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2023-02-20", "b": "knNNhnMXLN", "c": "c1", "d": "d3", "e": "e2"} +{"a": "2024-07-01", "b": "cxA5xpl6NM", "c": "c2", "d": "d1", "e": "e2"} +{"a": "2023-05-10", "b": "5FnmdQtOA0", "c": "c1", "d": "d2", "e": "e1"} +{"a": "2024-08-21", "b": "ldggIaWfYF", "c": "c3", "d": "d2", "e": "e2"} +{"a": "2024-01-05", "b": "7lwOfCQs5o", "c": "c2", "d": "d3", "e": "e1"} +{"a": "2024-08-21", "b": "2zvnCMAkZG", "c": "c1", "d": "d1", "e": "e2"} diff --git a/regression-test/data/inverted_index_p0/test_count_on_index.out b/regression-test/data/inverted_index_p0/test_count_on_index.out index 59910b7fb5d6a8..3c0f47e7f8baf9 100644 --- a/regression-test/data/inverted_index_p0/test_count_on_index.out +++ b/regression-test/data/inverted_index_p0/test_count_on_index.out @@ -74,3 +74,6 @@ -- !sql -- 0 +-- !sql_bad -- +0 1 + diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out index 700186ae78d70a..9b201bb9804eda 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out @@ -24,20 +24,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer] --------------PhysicalDistribute -----------------hashJoin[INNER_JOIN](ctr1.ctr_store_sk = ctr2.ctr_store_sk)(cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE)) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store.s_store_sk = ctr1.ctr_store_sk) -----------------------PhysicalDistribute -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((store.s_state = 'SD')) -----------------------------PhysicalOlapScan[store] -------------------PhysicalDistribute ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](ctr1.ctr_store_sk = ctr2.ctr_store_sk)(cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE)) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](store.s_store_sk = ctr1.ctr_store_sk) +------------------------PhysicalDistribute +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((store.s_state = 'SD')) +------------------------------PhysicalOlapScan[store] +--------------------PhysicalDistribute +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out index 18a3538a32c11d..0496973cea7973 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out @@ -95,78 +95,75 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------PhysicalUnion ------------------------PhysicalProject --------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[RIGHT_SEMI_JOIN](store_sales.ss_item_sk = cross_items.ss_item_sk) +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[RIGHT_SEMI_JOIN](store_sales.ss_item_sk = cross_items.ss_item_sk) +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ----------------------------------------PhysicalDistribute -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------------------------PhysicalDistribute ---------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalDistribute +------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] +----------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) +--------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalDistribute ------------------------------PhysicalAssertNumRows --------------------------------PhysicalDistribute ----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ------------------------PhysicalProject --------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = cross_items.ss_item_sk) +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = cross_items.ss_item_sk) +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ----------------------------------------PhysicalDistribute -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -------------------------------------------PhysicalDistribute ---------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalDistribute +------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] +----------------------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 2002)) +--------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalDistribute ------------------------------PhysicalAssertNumRows --------------------------------PhysicalDistribute ----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ------------------------PhysicalProject --------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = cross_items.ss_item_sk) +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = cross_items.ss_item_sk) +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ----------------------------------------PhysicalDistribute -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -------------------------------------------PhysicalDistribute ---------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalDistribute +------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] +----------------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) +--------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalDistribute ------------------------------PhysicalAssertNumRows --------------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out index abb2fe6156f5ae..3746b5b247c972 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out @@ -10,13 +10,14 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)((substring(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR ca_state IN ('CA', 'WA', 'GA')) OR (catalog_sales.cs_sales_price > 500.00)) ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] -----------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 1)(date_dim.d_year = 2001)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalOlapScan[catalog_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 1)(date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out index 60cc009a7b3a8e..f01f7b5a8e57fa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out @@ -10,31 +10,30 @@ PhysicalResultSink --------------hashAgg[LOCAL] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk)( not (substring(ca_zip, 1, 5) = substring(s_zip, 1, 5))) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter((item.i_manager_id = 2)) -----------------------------------------------PhysicalOlapScan[item] +----------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------filter((item.i_manager_id = 2)) +--------------------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +------------------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out index 7e04b2502b1a7a..78149630a0edd1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out @@ -12,12 +12,13 @@ PhysicalResultSink ------------------hashJoin[INNER_JOIN](inventory.inv_warehouse_sk = warehouse.w_warehouse_sk) --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) -------------------------hashJoin[INNER_JOIN](item.i_item_sk = inventory.inv_item_sk) ---------------------------PhysicalOlapScan[inventory] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49)(item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](item.i_item_sk = inventory.inv_item_sk) +----------------------------PhysicalOlapScan[inventory] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((item.i_current_price <= 1.49)(item.i_current_price >= 0.99)) +----------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject ----------------------------filter((date_dim.d_date >= '2002-01-28')(date_dim.d_date <= '2002-03-29')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out index c827c6f8235f17..9fe9cb16a09610 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out @@ -10,8 +10,8 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) -----------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject @@ -26,10 +26,10 @@ PhysicalResultSink ------------------------------PhysicalProject --------------------------------filter((date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) -----------------------------PhysicalOlapScan[promotion] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------------PhysicalOlapScan[promotion] ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out index fc17a5bab55f63..6516eca4168b97 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out @@ -10,14 +10,13 @@ PhysicalResultSink --------------hashAgg[LOCAL] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) ---------------------PhysicalDistribute -----------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((item.i_manufact_id = 816)) -------------------------------PhysicalOlapScan[item] +--------------------------filter((item.i_manufact_id = 816)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((dt.d_moy = 11)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out index b3d184b6480096..7ea8c8644fe191 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out @@ -32,9 +32,8 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------filter((customer_address.ca_state = 'IN')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out index d479e8c20c6fb1..cea4ec9ccc22be 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out @@ -32,9 +32,8 @@ PhysicalResultSink ----------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute ------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------filter((item.i_category = 'Home')) @@ -63,39 +62,39 @@ PhysicalResultSink ----------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute ------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------filter((item.i_category = 'Home')) --------------------------------------PhysicalOlapScan[item] ----------------PhysicalProject -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------------PhysicalDistribute +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] -----------------------------------------PhysicalDistribute +--------------------------------PhysicalOlapScan[item] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) -----------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) -----------------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((item.i_category = 'Home')) -------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalOlapScan[web_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) +------------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((item.i_category = 'Home')) +----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out index 928ef78cd268e4..34f698aa52da49 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out @@ -15,7 +15,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter(d_year IN (2000, 1999)) +--------------------------filter(d_year IN (1999, 2000)) ----------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject @@ -32,7 +32,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------PhysicalOlapScan[catalog_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter(d_year IN (2000, 1999)) +--------------------------filter(d_year IN (1999, 2000)) ----------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject @@ -49,7 +49,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------PhysicalOlapScan[web_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter(d_year IN (2000, 1999)) +--------------------------filter(d_year IN (1999, 2000)) ----------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject @@ -61,31 +61,30 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------PhysicalProject ------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL)) --------------PhysicalProject -----------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_secyear.customer_id)(if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL)) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) ---------------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_firstyear.customer_id) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((t_s_firstyear.year_total > 0.000000)(t_s_firstyear.dyear = 1999)(t_s_firstyear.sale_type = 's')) -----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((t_c_firstyear.year_total > 0.000000)(t_c_firstyear.dyear = 1999)(t_c_firstyear.sale_type = 'c')) -----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_secyear.customer_id)(if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL)) +------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) +------------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_firstyear.customer_id) --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2000)) +------------------------------filter((t_s_firstyear.year_total > 0.000000)(t_s_firstyear.dyear = 1999)(t_s_firstyear.sale_type = 's')) --------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((t_c_secyear.dyear = 2000)(t_c_secyear.sale_type = 'c')) -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((t_c_firstyear.year_total > 0.000000)(t_c_firstyear.dyear = 1999)(t_c_firstyear.sale_type = 'c')) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((t_w_firstyear.dyear = 1999)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.year_total > 0.000000)) +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2000)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((t_w_firstyear.dyear = 1999)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.year_total > 0.000000)) +----------------------filter((t_c_secyear.dyear = 2000)(t_c_secyear.sale_type = 'c')) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out index 665050611ca71b..e50ac4dd8f2406 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out @@ -13,18 +13,19 @@ PhysicalResultSink --------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((item.i_current_price >= 0.99)(item.i_current_price <= 1.49)) +----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((item.i_current_price >= 0.99)(item.i_current_price <= 1.49)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date >= '2001-03-03')(date_dim.d_date <= '2001-05-02')) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_date >= '2001-03-03')(date_dim.d_date <= '2001-05-02')) +--------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out index d60ffc848136fb..dd434fc73399c9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out @@ -19,6 +19,6 @@ PhysicalResultSink --------------------------PhysicalDistribute ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter((((((((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('violet', 'chiffon')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('small', 'extra large'))) OR ((((item.i_category = 'Women') AND i_color IN ('tomato', 'blanched')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium'))) OR ((((item.i_category = 'Women') AND i_color IN ('lime', 'almond')) AND i_units IN ('Dram', 'Box')) AND i_size IN ('small', 'extra large'))) OR ((((item.i_category = 'Men') AND i_color IN ('chartreuse', 'blue')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('large', 'N/A'))) OR ((((item.i_category = 'Men') AND i_color IN ('tan', 'dodger')) AND i_units IN ('Tsp', 'Bunch')) AND i_size IN ('economy', 'medium'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Pallet', 'Gram')) AND i_size IN ('large', 'N/A'))) OR ((((item.i_category = 'Men') AND i_color IN ('spring', 'indian')) AND i_units IN ('Unknown', 'Carton')) AND i_size IN ('economy', 'medium')))) +--------------------------------filter((((((((((((item.i_category = 'Women') AND i_color IN ('gainsboro', 'aquamarine')) AND i_units IN ('Ounce', 'Dozen')) AND i_size IN ('medium', 'economy')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Ton', 'Pound')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Tbl', 'Case')) AND i_size IN ('medium', 'economy'))) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('chartreuse', 'blue')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('tan', 'dodger')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('medium', 'economy'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Pallet', 'Gram')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Unknown', 'Carton')) AND i_size IN ('medium', 'economy')))) ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out index 298efbd86325e4..795319e23da889 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out @@ -9,13 +9,14 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] ---------------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 1)) ---------------------------PhysicalOlapScan[item] +------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((item.i_manager_id = 1)) +----------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute --------------------PhysicalProject ----------------------filter((dt.d_year = 2002)(dt.d_moy = 11)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out index 48f68b04e2b32f..53a11c2dabd077 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out @@ -11,37 +11,36 @@ PhysicalResultSink ------------PhysicalDistribute --------------PhysicalProject ----------------hashJoin[INNER_JOIN](asceding.rnk = descending.rnk) -------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](i1.i_item_sk = asceding.item_sk) +------------------hashJoin[INNER_JOIN](i1.i_item_sk = asceding.item_sk) +--------------------PhysicalProject +----------------------PhysicalOlapScan[item] +--------------------PhysicalDistribute ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((rnk < 11)) -----------------------------PhysicalWindow -------------------------------PhysicalQuickSort ---------------------------------PhysicalDistribute -----------------------------------PhysicalQuickSort -------------------------------------PhysicalPartitionTopN ---------------------------------------PhysicalProject -----------------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) -------------------------------------------PhysicalProject ---------------------------------------------hashAgg[GLOBAL] -----------------------------------------------PhysicalDistribute -------------------------------------------------hashAgg[LOCAL] ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((ss1.ss_store_sk = 146)) -------------------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalAssertNumRows -----------------------------------------------PhysicalDistribute +------------------------filter((rnk < 11)) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort +------------------------------PhysicalDistribute +--------------------------------PhysicalQuickSort +----------------------------------PhysicalPartitionTopN +------------------------------------PhysicalProject +--------------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) +----------------------------------------PhysicalProject +------------------------------------------hashAgg[GLOBAL] +--------------------------------------------PhysicalDistribute +----------------------------------------------hashAgg[LOCAL] ------------------------------------------------PhysicalProject ---------------------------------------------------hashAgg[GLOBAL] -----------------------------------------------------PhysicalDistribute -------------------------------------------------------hashAgg[LOCAL] ---------------------------------------------------------PhysicalProject -----------------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146)) -------------------------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------------------filter((ss1.ss_store_sk = 146)) +----------------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalAssertNumRows +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------hashAgg[GLOBAL] +--------------------------------------------------PhysicalDistribute +----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146)) +----------------------------------------------------------PhysicalOlapScan[store_sales] ------------------PhysicalDistribute --------------------PhysicalProject ----------------------filter((rnk < 11)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out index 8a6e45f913977c..3565440b382ee6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out @@ -35,11 +35,12 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] ------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[customer_address] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out index 96569a9c77b08e..bb00dd5c668f22 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out @@ -36,17 +36,18 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute ----------PhysicalTopN ------------PhysicalProject ---------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.s_store_name = v1_lag.s_store_name)(v1.s_company_name = v1_lag.s_company_name)(v1.rn = expr_(rn + 1)) +--------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.s_store_name = v1_lead.s_store_name)(v1.s_company_name = v1_lead.s_company_name)(v1.rn = expr_(rn - 1)) ----------------PhysicalDistribute ------------------PhysicalProject --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.s_store_name = v1_lead.s_store_name)(v1.s_company_name = v1_lead.s_company_name)(v1.rn = expr_(rn - 1)) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((if((avg_monthly_sales > 0.0000), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)(v2.d_year = 2001)(v2.avg_monthly_sales > 0.0000)) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.s_store_name = v1_lag.s_store_name)(v1.s_company_name = v1_lag.s_company_name)(v1.rn = expr_(rn + 1)) +----------------------PhysicalDistribute +------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((if((avg_monthly_sales > 0.0000), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)(v2.d_year = 2001)(v2.avg_monthly_sales > 0.0000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out index a2e8c26203b57f..e5764dbbd01b32 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out @@ -38,20 +38,19 @@ PhysicalResultSink --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN](salesreturns.page_sk = catalog_page.cp_catalog_page_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalUnion -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) +------------------------------------PhysicalUnion +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[catalog_sales] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_date >= '2000-08-19')(date_dim.d_date <= '2000-09-02')) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalOlapScan[catalog_returns] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_date >= '2000-08-19')(date_dim.d_date <= '2000-09-02')) +------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[catalog_page] @@ -76,7 +75,7 @@ PhysicalResultSink ----------------------------------------------PhysicalOlapScan[web_returns] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_date >= '2000-08-19')(date_dim.d_date <= '2000-09-02')) +----------------------------------------filter((date_dim.d_date <= '2000-09-02')(date_dim.d_date >= '2000-08-19')) ------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out index b9cef62510c295..e8564c2c922cdc 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out @@ -4,45 +4,44 @@ PhysicalResultSink --PhysicalTopN ----PhysicalDistribute ------PhysicalTopN ---------PhysicalProject -----------filter((web_cumulative > store_cumulative)) -------------PhysicalWindow ---------------PhysicalQuickSort -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[FULL_OUTER_JOIN](web.item_sk = store.item_sk)(web.d_date = store.d_date) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalWindow -----------------------------PhysicalQuickSort -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------filter((web_cumulative > store_cumulative)) +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[FULL_OUTER_JOIN](web.item_sk = store.item_sk)(web.d_date = store.d_date) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalWindow +--------------------------PhysicalQuickSort +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq <= 1227)(date_dim.d_month_seq >= 1216)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalWindow -----------------------------PhysicalQuickSort -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------------------filter((date_dim.d_month_seq <= 1227)(date_dim.d_month_seq >= 1216)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalWindow +--------------------------PhysicalQuickSort +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[web_sales] +------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[web_sales] ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq >= 1216)(date_dim.d_month_seq <= 1227)) ---------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------filter((date_dim.d_month_seq >= 1216)(date_dim.d_month_seq <= 1227)) +------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out index 8722968733277d..e7edc630a2f6a7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out @@ -15,22 +15,20 @@ PhysicalResultSink ------------------------hashAgg[LOCAL] --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalProject -----------------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) -------------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) --------------------------------------PhysicalProject -----------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) -------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) +--------------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) +----------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out index 04d6c5a8f7c209..5180c2191e01b1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out @@ -26,39 +26,38 @@ PhysicalResultSink ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN](customer_address.ca_county = store.s_county)(customer_address.ca_state = store.s_state) -------------------------------------------------PhysicalDistribute ---------------------------------------------------hashJoin[INNER_JOIN](my_customers.c_current_addr_sk = customer_address.ca_address_sk) +------------------------------------------------hashJoin[INNER_JOIN](my_customers.c_current_addr_sk = customer_address.ca_address_sk) +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[customer_address] +--------------------------------------------------PhysicalDistribute ----------------------------------------------------PhysicalProject -------------------------------------------------------PhysicalOlapScan[customer_address] -----------------------------------------------------PhysicalDistribute -------------------------------------------------------PhysicalProject ---------------------------------------------------------hashAgg[GLOBAL] -----------------------------------------------------------PhysicalDistribute -------------------------------------------------------------hashAgg[LOCAL] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------hashJoin[INNER_JOIN](customer.c_customer_sk = cs_or_ws_sales.customer_sk) +------------------------------------------------------hashAgg[GLOBAL] +--------------------------------------------------------PhysicalDistribute +----------------------------------------------------------hashAgg[LOCAL] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN](customer.c_customer_sk = cs_or_ws_sales.customer_sk) +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[customer] +----------------------------------------------------------------PhysicalDistribute ------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------PhysicalOlapScan[customer] -------------------------------------------------------------------PhysicalDistribute ---------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk) -------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.item_sk = item.i_item_sk) -----------------------------------------------------------------------------PhysicalUnion -------------------------------------------------------------------------------PhysicalDistribute ---------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] -------------------------------------------------------------------------------PhysicalDistribute ---------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk) +----------------------------------------------------------------------PhysicalProject +------------------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.item_sk = item.i_item_sk) +--------------------------------------------------------------------------PhysicalUnion ----------------------------------------------------------------------------PhysicalDistribute ------------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------------filter((item.i_class = 'maternity')(item.i_category = 'Women')) -----------------------------------------------------------------------------------PhysicalOlapScan[item] -------------------------------------------------------------------------PhysicalDistribute ---------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) -------------------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------------------------------------------------PhysicalDistribute +------------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------------------------------------------PhysicalDistribute +----------------------------------------------------------------------------PhysicalProject +------------------------------------------------------------------------------filter((item.i_class = 'maternity')(item.i_category = 'Women')) +--------------------------------------------------------------------------------PhysicalOlapScan[item] +----------------------------------------------------------------------PhysicalDistribute +------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) +----------------------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------------PhysicalDistribute --------------------------------------------------PhysicalProject ----------------------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out index 8903c6716160b3..330145cdc5028d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out @@ -16,20 +16,18 @@ PhysicalResultSink --------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) ----------------------------PhysicalDistribute ------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) @@ -49,20 +47,18 @@ PhysicalResultSink --------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute ------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) @@ -78,20 +74,18 @@ PhysicalResultSink --------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute ------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[web_sales] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out index 4456679c6fc405..40019178d3fa92 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out @@ -34,61 +34,62 @@ PhysicalResultSink ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_date = '2001-03-24')) --------------------------------------------------PhysicalOlapScan[date_dim] -------------hashJoin[INNER_JOIN](ss_items.item_id = ws_items.item_id)(cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] -------------------------------PhysicalDistribute +------------PhysicalProject +--------------hashJoin[INNER_JOIN](ss_items.item_id = ws_items.item_id)(cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalDistribute +----------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalAssertNumRows -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_date = '2001-03-24')) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] -------------------------------PhysicalDistribute +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalAssertNumRows +------------------------------------------------PhysicalDistribute +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_date = '2001-03-24')) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalDistribute +----------------------------------PhysicalOlapScan[web_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalAssertNumRows -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_date = '2001-03-24')) -----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalAssertNumRows +------------------------------------------------PhysicalDistribute +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_date = '2001-03-24')) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out index bd043d536a71dc..225b470a916b48 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out @@ -14,27 +14,25 @@ PhysicalResultSink ----------------------hashAgg[LOCAL] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) -----------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((customer_address.ca_gmt_offset = -7.00)) -----------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalDistribute -------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +------------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) +--------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] +------------------------------------filter((customer_address.ca_gmt_offset = -7.00)) +--------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------filter((item.i_category = 'Children')) @@ -45,27 +43,25 @@ PhysicalResultSink ----------------------hashAgg[LOCAL] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_moy = 8)(date_dim.d_year = 2000)) -----------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((customer_address.ca_gmt_offset = -7.00)) -----------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalDistribute -------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +------------------------------------------filter((date_dim.d_moy = 8)(date_dim.d_year = 2000)) +--------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] +------------------------------------filter((customer_address.ca_gmt_offset = -7.00)) +--------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------filter((item.i_category = 'Children')) @@ -76,27 +72,25 @@ PhysicalResultSink ----------------------hashAgg[LOCAL] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) -----------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((customer_address.ca_gmt_offset = -7.00)) -----------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalDistribute -------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +------------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) +--------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] +------------------------------------filter((customer_address.ca_gmt_offset = -7.00)) +--------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------filter((item.i_category = 'Children')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out index ec78b77b89e5f3..3cd4ec1cbac1f3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) ------------------PhysicalProject --------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) -------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_sales] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) +--------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq >= 1194)(date_dim.d_month_seq <= 1205)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq >= 1194)(date_dim.d_month_seq <= 1205)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_site] +------------------------------PhysicalOlapScan[web_site] ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out index 1b979039047736..0c75b1ba2cfe7c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out @@ -15,22 +15,20 @@ PhysicalResultSink ------------------------hashAgg[LOCAL] --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalProject -----------------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) -------------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) --------------------------------------PhysicalProject -----------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) -------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) +--------------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) +----------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out index 5d8a503ccae58c..978389da3b3f99 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out @@ -16,20 +16,19 @@ PhysicalResultSink --------------------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_year = 1998)) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +--------------------------------------------PhysicalOlapScan[ship_mode] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 1998)) +------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------filter((cast(t_time as BIGINT) <= 77621)(time_dim.t_time >= 48821)) @@ -45,20 +44,19 @@ PhysicalResultSink --------------------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_time_sk = time_dim.t_time_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_year = 1998)) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +--------------------------------------------PhysicalOlapScan[ship_mode] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 1998)) +------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------filter((time_dim.t_time >= 48821)(cast(t_time as BIGINT) <= 77621)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out index 96ae616ceb1f84..27f925621778f8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out @@ -48,9 +48,8 @@ PhysicalResultSink --------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------filter(ca_state IN ('TX', 'VA', 'MI')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out index 58908af1bc5f2c..7f889521bb8995 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out @@ -10,25 +10,26 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((customer_demographics.cd_gender = 'F')(customer_demographics.cd_marital_status = 'W')(customer_demographics.cd_education_status = 'College')) -------------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((date_dim.d_year = 2001)) ---------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) ------------------------PhysicalProject ---------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) -----------------------------PhysicalOlapScan[promotion] +--------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((customer_demographics.cd_gender = 'F')(customer_demographics.cd_marital_status = 'W')(customer_demographics.cd_education_status = 'College')) +--------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter(((promotion.p_channel_email = 'N') OR (promotion.p_channel_event = 'N'))) +------------------------------PhysicalOlapScan[promotion] ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out index 2771503c464747..318e3f111b2f10 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out @@ -26,30 +26,28 @@ PhysicalResultSink ------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------hashJoin[LEFT_SEMI_JOIN](store.s_state = tmp1.s_state) -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------filter((ranking <= 5)) ------------------------------------------PhysicalWindow --------------------------------------------PhysicalQuickSort -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalPartitionTopN ---------------------------------------------------PhysicalProject -----------------------------------------------------hashAgg[GLOBAL] -------------------------------------------------------PhysicalDistribute ---------------------------------------------------------hashAgg[LOCAL] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) ---------------------------------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------------------------------------PhysicalDistribute -------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------filter((date_dim.d_month_seq >= 1213)(date_dim.d_month_seq <= 1224)) -----------------------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +------------------------------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +--------------------------------------------------------------PhysicalProject +----------------------------------------------------------------PhysicalOlapScan[store_sales] --------------------------------------------------------------PhysicalDistribute ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------------filter((date_dim.d_month_seq >= 1213)(date_dim.d_month_seq <= 1224)) +--------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------PhysicalProject +----------------------------------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out index 01fd2572b9e8b8..b7b91d09a74996 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out @@ -15,52 +15,49 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------PhysicalProject ----------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](warehouse.w_warehouse_sk = inventory.inv_warehouse_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](warehouse.w_warehouse_sk = inventory.inv_warehouse_sk) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = inventory.inv_item_sk)(inventory.inv_date_sk = d2.d_date_sk)(inventory.inv_quantity_on_hand < catalog_sales.cs_quantity) -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalOlapScan[inventory] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN](d1.d_week_seq = d2.d_week_seq) -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = d3.d_date_sk)(d3.d_date > cast((cast(d_date as BIGINT) + 5) as DATEV2)) -----------------------------------------------------PhysicalDistribute -------------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) ---------------------------------------------------------PhysicalDistribute -----------------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d1.d_date_sk) -------------------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------------------------------------PhysicalDistribute -----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------filter((household_demographics.hd_buy_potential = '501-1000')) ---------------------------------------------------------------------PhysicalOlapScan[household_demographics] -------------------------------------------------------------PhysicalDistribute ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((d1.d_year = 2002)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = inventory.inv_item_sk)(inventory.inv_date_sk = d2.d_date_sk)(inventory.inv_quantity_on_hand < catalog_sales.cs_quantity) +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalOlapScan[inventory] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN](d1.d_week_seq = d2.d_week_seq) +--------------------------------------------PhysicalProject +----------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = d3.d_date_sk)(d3.d_date > cast((cast(d_date as BIGINT) + 5) as DATEV2)) +------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) +--------------------------------------------------PhysicalDistribute +----------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d1.d_date_sk) +------------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_hdemo_sk = household_demographics.hd_demo_sk) +--------------------------------------------------------PhysicalProject +----------------------------------------------------------PhysicalOlapScan[catalog_sales] --------------------------------------------------------PhysicalDistribute ----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((customer_demographics.cd_marital_status = 'W')) ---------------------------------------------------------------PhysicalOlapScan[customer_demographics] -----------------------------------------------------PhysicalDistribute -------------------------------------------------------PhysicalProject ---------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[warehouse] +------------------------------------------------------------filter((household_demographics.hd_buy_potential = '501-1000')) +--------------------------------------------------------------PhysicalOlapScan[household_demographics] +------------------------------------------------------PhysicalDistribute +--------------------------------------------------------PhysicalProject +----------------------------------------------------------filter((d1.d_year = 2002)) +------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------PhysicalDistribute +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((customer_demographics.cd_marital_status = 'W')) +--------------------------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------------------PhysicalDistribute +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[warehouse] ------------------------PhysicalDistribute --------------------------PhysicalProject ----------------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out index d6477a3695cb39..73305f15762de9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out @@ -15,7 +15,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter(d_year IN (2000, 1999)) +--------------------------filter(d_year IN (1999, 2000)) ----------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject @@ -32,7 +32,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------PhysicalOlapScan[web_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter(d_year IN (2000, 1999)) +--------------------------filter(d_year IN (1999, 2000)) ----------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject @@ -42,23 +42,24 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(if((year_total > 0), (year_total / year_total), NULL) > if((year_total > 0), (year_total / year_total), NULL)) ---------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id) +------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(if((year_total > 0), (year_total / year_total), NULL) > if((year_total > 0), (year_total / year_total), NULL)) +--------------PhysicalProject ----------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0)(t_w_firstyear.sale_type = 'w')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject ----------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.year = 2000)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((t_w_secyear.year = 2000)(t_w_secyear.sale_type = 'w')) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject -------------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0)(t_w_firstyear.sale_type = 'w')) +------------------filter((t_w_secyear.year = 2000)(t_w_secyear.sale_type = 'w')) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out index e66c5338ae855a..f3499519a81926 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out @@ -14,37 +14,40 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalDistribute ------------------PhysicalProject --------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] ----------------PhysicalDistribute ------------------PhysicalProject --------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter(cs_warehouse_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out index f1df37846ba9c1..170e60a5b895e7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out @@ -58,7 +58,7 @@ PhysicalResultSink --------------------------------------PhysicalOlapScan[catalog_sales] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_date >= '1998-08-05')(date_dim.d_date <= '1998-09-04')) +----------------------------------------filter((date_dim.d_date <= '1998-09-04')(date_dim.d_date >= '1998-08-05')) ------------------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject @@ -76,27 +76,28 @@ PhysicalResultSink --------------------PhysicalProject ----------------------hashJoin[LEFT_OUTER_JOIN](ws.wp_web_page_sk = wr.wp_web_page_sk) ------------------------PhysicalProject ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[web_sales] ---------------------------------------PhysicalDistribute +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_date >= '1998-08-05')(date_dim.d_date <= '1998-09-04')) ---------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_page] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_returns.wr_web_page_sk = web_page.wp_web_page_sk) -----------------------------------PhysicalDistribute +------------------------------------------PhysicalOlapScan[web_sales] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((date_dim.d_date >= '1998-08-05')(date_dim.d_date <= '1998-09-04')) +----------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_page] +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_returns.wr_web_page_sk = web_page.wp_web_page_sk) ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) ----------------------------------------PhysicalProject @@ -105,7 +106,7 @@ PhysicalResultSink ------------------------------------------PhysicalProject --------------------------------------------filter((date_dim.d_date >= '1998-08-05')(date_dim.d_date <= '1998-09-04')) ----------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_page] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_page] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out index 3cd8d0185ab450..1eeff0d0be86e5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out @@ -14,50 +14,44 @@ PhysicalResultSink ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute +----------------------------hashJoin[LEFT_ANTI_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) +------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) --------------------------------PhysicalProject -----------------------------------filter(sr_ticket_number IS NULL) -------------------------------------hashJoin[LEFT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_returns] +----------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_returns] ------------------PhysicalProject --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute +----------------------------hashJoin[LEFT_ANTI_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) +------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) --------------------------------PhysicalProject -----------------------------------filter(wr_order_number IS NULL) -------------------------------------hashJoin[LEFT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[web_sales] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[web_returns] +----------------------------------PhysicalOlapScan[web_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_returns] --------------PhysicalProject ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2000)) -------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute +------------------------hashJoin[LEFT_ANTI_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------filter(cr_order_number IS NULL) ---------------------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_returns] +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_returns] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out index a7d48e8011a85f..7b4e8ce31978e6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out @@ -16,66 +16,59 @@ PhysicalResultSink --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject ------------------------------hashJoin[RIGHT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_returns] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------------------------PhysicalDistribute -----------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------------------PhysicalDistribute ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_date >= '1998-08-28')(date_dim.d_date <= '1998-09-27')) -------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[store_sales] --------------------------------------------PhysicalDistribute ----------------------------------------------PhysicalProject -------------------------------------------------filter((item.i_current_price > 50.00)) ---------------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter((promotion.p_channel_tv = 'N')) -----------------------------------------------PhysicalOlapScan[promotion] +------------------------------------------------filter((date_dim.d_date >= '1998-08-28')(date_dim.d_date <= '1998-09-27')) +--------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((item.i_current_price > 50.00)) +------------------------------------------------PhysicalOlapScan[item] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store] +------------------------------------------filter((promotion.p_channel_tv = 'N')) +--------------------------------------------PhysicalOlapScan[promotion] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store] --------------------PhysicalProject ----------------------hashAgg[GLOBAL] ------------------------PhysicalDistribute --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_catalog_page_sk = catalog_page.cp_catalog_page_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_returns] ---------------------------------------PhysicalDistribute -----------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------------------------------------PhysicalDistribute -------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------------------------PhysicalDistribute -----------------------------------------------------PhysicalProject -------------------------------------------------------filter((date_dim.d_date >= '1998-08-28')(date_dim.d_date <= '1998-09-27')) ---------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------filter((item.i_current_price > 50.00)) -----------------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[catalog_returns] +------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_date <= '1998-09-27')(date_dim.d_date >= '1998-08-28')) +--------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------filter((promotion.p_channel_tv = 'N')) -------------------------------------------------PhysicalOlapScan[promotion] +----------------------------------------------filter((item.i_current_price > 50.00)) +------------------------------------------------PhysicalOlapScan[item] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((promotion.p_channel_tv = 'N')) +--------------------------------------------PhysicalOlapScan[promotion] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[catalog_page] @@ -85,32 +78,29 @@ PhysicalResultSink --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject ------------------------------hashJoin[RIGHT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_returns] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) ---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_promo_sk = promotion.p_promo_sk) -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ---------------------------------------------PhysicalDistribute -----------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] -------------------------------------------------PhysicalDistribute ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_date >= '1998-08-28')(date_dim.d_date <= '1998-09-27')) -------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_returns] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) +------------------------------------hashJoin[INNER_JOIN](web_sales.ws_promo_sk = promotion.p_promo_sk) +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] --------------------------------------------PhysicalDistribute ----------------------------------------------PhysicalProject -------------------------------------------------filter((item.i_current_price > 50.00)) ---------------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter((promotion.p_channel_tv = 'N')) -----------------------------------------------PhysicalOlapScan[promotion] +------------------------------------------------filter((date_dim.d_date >= '1998-08-28')(date_dim.d_date <= '1998-09-27')) +--------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((item.i_current_price > 50.00)) +------------------------------------------------PhysicalOlapScan[item] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_site] +------------------------------------------filter((promotion.p_channel_tv = 'N')) +--------------------------------------------PhysicalOlapScan[promotion] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out index 6a6c1a787b5d9e..5ba79b761081ed 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out @@ -32,57 +32,58 @@ PhysicalResultSink ------------------------------------------PhysicalProject --------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) ----------------------------------------------PhysicalOlapScan[date_dim] -------------hashJoin[INNER_JOIN](sr_items.item_id = wr_items.item_id) ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = item.i_item_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] -------------------------------PhysicalDistribute +------------PhysicalProject +--------------hashJoin[INNER_JOIN](sr_items.item_id = wr_items.item_id) +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = item.i_item_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalDistribute +----------------------------------PhysicalOlapScan[store_returns] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) --------------------------------------PhysicalProject -----------------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalDistribute +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) --------------------------------------------PhysicalProject -----------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) -------------------------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = item.i_item_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_returns] -------------------------------PhysicalDistribute +----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) +--------------------------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = item.i_item_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) --------------------------------PhysicalProject -----------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalDistribute +----------------------------------PhysicalOlapScan[web_returns] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) --------------------------------------PhysicalProject -----------------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalDistribute +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) --------------------------------------------PhysicalProject -----------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) -------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter(d_date IN ('2001-06-06', '2001-09-02', '2001-11-11')) +--------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out index 3d5febe35de013..710652c2f1b65e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out @@ -16,15 +16,14 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] +----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_city = 'Oakwood')) -------------------------------------PhysicalOlapScan[customer_address] +--------------------------------filter((customer_address.ca_city = 'Oakwood')) +----------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalDistribute --------------------------hashJoin[INNER_JOIN](income_band.ib_income_band_sk = household_demographics.hd_income_band_sk) ----------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out index 327ff7b7af068f..5a8742f29e6873 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out @@ -2,56 +2,51 @@ -- !ds_shape_90 -- PhysicalResultSink --PhysicalTopN -----PhysicalTopN -------PhysicalProject ---------NestedLoopJoin[CROSS_JOIN] +----PhysicalProject +------NestedLoopJoin[CROSS_JOIN] +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](web_sales.ws_ship_hdemo_sk = household_demographics.hd_demo_sk) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) +----------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[web_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((web_page.wp_char_count >= 5000)(web_page.wp_char_count <= 5200)) +------------------------------PhysicalOlapScan[web_page] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((time_dim.t_hour >= 10)(time_dim.t_hour <= 11)) +----------------------------PhysicalOlapScan[time_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((household_demographics.hd_dep_count = 2)) +------------------------PhysicalOlapScan[household_demographics] +--------PhysicalDistribute ----------hashAgg[GLOBAL] ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN](web_sales.ws_ship_hdemo_sk = household_demographics.hd_demo_sk) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((web_page.wp_char_count >= 5000)(web_page.wp_char_count <= 5200)) -------------------------------------PhysicalOlapScan[web_page] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) +------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[web_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((time_dim.t_hour >= 10)(time_dim.t_hour <= 11)) ---------------------------------PhysicalOlapScan[time_dim] +------------------------------filter((web_page.wp_char_count >= 5000)(web_page.wp_char_count <= 5200)) +--------------------------------PhysicalOlapScan[web_page] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((time_dim.t_hour <= 17)(time_dim.t_hour >= 16)) +------------------------------PhysicalOlapScan[time_dim] --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((household_demographics.hd_dep_count = 2)) --------------------------PhysicalOlapScan[household_demographics] -----------PhysicalDistribute -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](web_sales.ws_ship_hdemo_sk = household_demographics.hd_demo_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) -----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((web_page.wp_char_count >= 5000)(web_page.wp_char_count <= 5200)) ---------------------------------------PhysicalOlapScan[web_page] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((time_dim.t_hour <= 17)(time_dim.t_hour >= 16)) -----------------------------------PhysicalOlapScan[time_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_dep_count = 2)) -----------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out index ba4a23e3f0ba58..06812568dc9027 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out @@ -10,37 +10,36 @@ PhysicalResultSink --------------hashAgg[LOCAL] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN](catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returning_customer_sk = customer.c_customer_sk) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returning_customer_sk = customer.c_customer_sk) +----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[catalog_returns] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) -----------------------------------PhysicalProject -------------------------------------filter((customer_address.ca_gmt_offset = -6.00)) ---------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalDistribute -------------------------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) +--------------------------------PhysicalProject +----------------------------------filter((customer_address.ca_gmt_offset = -6.00)) +------------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) +------------------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) --------------------------------------PhysicalDistribute -----------------------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[customer] -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalProject -----------------------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree')))) -------------------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[customer] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((hd_buy_potential like '1001-5000%')) ---------------------------------------------PhysicalOlapScan[household_demographics] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 2001)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree')))) +--------------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((hd_buy_potential like '1001-5000%')) +------------------------------------------PhysicalOlapScan[household_demographics] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out index 295f54863da7c6..c5bb64a7623580 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out @@ -2,27 +2,26 @@ -- !ds_shape_92 -- PhysicalResultSink --PhysicalTopN -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((cast(ws_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(ws_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) -----------------PhysicalWindow -------------------PhysicalQuickSort ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((item.i_manufact_id = 320)) -------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute +----hashAgg[GLOBAL] +------PhysicalDistribute +--------hashAgg[LOCAL] +----------PhysicalProject +------------filter((cast(ws_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(ws_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +--------------PhysicalWindow +----------------PhysicalQuickSort +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) ----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '2002-05-27')(date_dim.d_date >= '2002-02-26')) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalOlapScan[web_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 320)) +----------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((date_dim.d_date <= '2002-05-27')(date_dim.d_date >= '2002-02-26')) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out index cc6eba640d6693..98559a81403c12 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out @@ -11,11 +11,12 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = store_sales.ss_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) ------------------PhysicalProject --------------------PhysicalOlapScan[store_sales] -------------------hashJoin[INNER_JOIN](store_returns.sr_reason_sk = reason.r_reason_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_returns] ---------------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](store_returns.sr_reason_sk = reason.r_reason_sk) ----------------------PhysicalProject -------------------------filter((reason.r_reason_desc = 'duplicate purchase')) ---------------------------PhysicalOlapScan[reason] +------------------------PhysicalOlapScan[store_returns] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((reason.r_reason_desc = 'duplicate purchase')) +----------------------------PhysicalOlapScan[reason] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out index 8d508bacd9b967..b805eb188adf09 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out @@ -2,17 +2,17 @@ -- !ds_shape_94 -- PhysicalResultSink --PhysicalTopN -----PhysicalTopN -------hashAgg[DISTINCT_GLOBAL] ---------PhysicalDistribute -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws2.ws_order_number)( not (ws_warehouse_sk = ws_warehouse_sk)) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] +----hashAgg[DISTINCT_GLOBAL] +------PhysicalDistribute +--------hashAgg[DISTINCT_LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws2.ws_order_number)( not (ws_warehouse_sk = ws_warehouse_sk)) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[web_sales] +------------------PhysicalProject --------------------hashJoin[RIGHT_ANTI_JOIN](ws1.ws_order_number = wr1.wr_order_number) ----------------------PhysicalDistribute ------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out index b3a19d18c18cdd..605d94861d2d1a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out @@ -12,22 +12,22 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------PhysicalOlapScan[web_sales] --PhysicalResultSink ----PhysicalTopN -------PhysicalTopN ---------hashAgg[DISTINCT_GLOBAL] -----------PhysicalDistribute -------------hashAgg[DISTINCT_LOCAL] ---------------hashAgg[GLOBAL] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = web_returns.wr_order_number) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_returns.wr_order_number = ws_wh.ws_order_number) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_returns] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = web_returns.wr_order_number) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](web_returns.wr_order_number = ws_wh.ws_order_number) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[web_returns] +--------------------PhysicalProject ----------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws_wh.ws_order_number) ------------------------PhysicalDistribute --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out index 6d7df70b717438..34e7fc5538e577 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out @@ -2,27 +2,26 @@ -- !ds_shape_96 -- PhysicalResultSink --PhysicalTopN -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) -----------------------------PhysicalOlapScan[time_dim] +----hashAgg[GLOBAL] +------PhysicalDistribute +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +--------------------PhysicalProject +----------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((household_demographics.hd_dep_count = 3)) ---------------------------PhysicalOlapScan[household_demographics] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((store.s_store_name = 'ese')) -----------------------PhysicalOlapScan[store] +------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) +--------------------------PhysicalOlapScan[time_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((household_demographics.hd_dep_count = 3)) +------------------------PhysicalOlapScan[household_demographics] +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((store.s_store_name = 'ese')) +--------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out index 4f20f7bf82aa1b..2874c226f788ec 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out @@ -11,17 +11,18 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) ------------------PhysicalProject --------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk) -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq >= 1224)(date_dim.d_month_seq <= 1235)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq >= 1224)(date_dim.d_month_seq <= 1235)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[call_center] +------------------------------PhysicalOlapScan[call_center] ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out index ca1de573ae694f..878b6c708fb611 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out @@ -9,23 +9,24 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalOlapScan[supplier] ------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((lineitem.l_shipdate >= '1996-01-01')(lineitem.l_shipdate < '1996-04-01')) -----------------------------PhysicalOlapScan[lineitem] -----------------PhysicalDistribute -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((lineitem.l_shipdate >= '1996-01-01')(lineitem.l_shipdate < '1996-04-01')) -------------------------------------PhysicalOlapScan[lineitem] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((lineitem.l_shipdate >= '1996-01-01')(lineitem.l_shipdate < '1996-04-01')) +------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalDistribute +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((lineitem.l_shipdate >= '1996-01-01')(lineitem.l_shipdate < '1996-04-01')) +--------------------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out index e2b58de0eb720c..728ebb37a8e5c1 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out @@ -10,12 +10,13 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[partsupp] -----------------------PhysicalProject -------------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) ---------------------------PhysicalOlapScan[part] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[partsupp] +------------------------PhysicalProject +--------------------------filter(( not (p_brand = 'Brand#45'))( not (p_type like 'MEDIUM POLISHED%'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) +----------------------------PhysicalOlapScan[part] ------------------PhysicalDistribute --------------------PhysicalProject ----------------------filter((s_comment like '%Customer%Complaints%')) diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out index 4c8fc44b7c203f..cb45679eac73f7 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out @@ -25,9 +25,8 @@ PhysicalResultSink ----------------------------PhysicalOlapScan[supplier] ----------------------------PhysicalDistribute ------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[nation] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------filter((region.r_name = 'EUROPE')) diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out index 5a7a1020814fb6..528a9f78ed93b0 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out @@ -24,11 +24,12 @@ PhysicalResultSink ------------------------filter((p_name like 'forest%')) --------------------------PhysicalOlapScan[part] ------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -----------------PhysicalProject -------------------PhysicalOlapScan[supplier] -----------------PhysicalDistribute +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ------------------PhysicalProject ---------------------filter((nation.n_name = 'CANADA')) -----------------------PhysicalOlapScan[nation] +--------------------PhysicalOlapScan[supplier] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((nation.n_name = 'CANADA')) +------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out index 8cb5b171bc706f..6b1e9f37b4b32a 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out @@ -23,11 +23,12 @@ PhysicalResultSink ------------------------filter((p_name like 'forest%')) --------------------------PhysicalOlapScan[part] ------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -----------------PhysicalProject -------------------PhysicalOlapScan[supplier] -----------------PhysicalDistribute +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ------------------PhysicalProject ---------------------filter((nation.n_name = 'CANADA')) -----------------------PhysicalOlapScan[nation] +--------------------PhysicalOlapScan[supplier] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((nation.n_name = 'CANADA')) +------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out index 2c0a3b2a637c31..312a23d2bef92d 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out @@ -11,24 +11,25 @@ PhysicalResultSink ----------------hashJoin[RIGHT_SEMI_JOIN](l2.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) ------------------PhysicalProject --------------------PhysicalOlapScan[lineitem] -------------------hashJoin[INNER_JOIN](orders.o_orderkey = l1.l_orderkey) ---------------------PhysicalProject -----------------------filter((orders.o_orderstatus = 'F')) -------------------------PhysicalOlapScan[orders] ---------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](orders.o_orderkey = l1.l_orderkey) ----------------------PhysicalProject -------------------------filter((l3.l_receiptdate > l3.l_commitdate)) ---------------------------PhysicalOlapScan[lineitem] -----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) +------------------------filter((orders.o_orderstatus = 'F')) +--------------------------PhysicalOlapScan[orders] +----------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) ------------------------PhysicalProject ---------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +--------------------------filter((l3.l_receiptdate > l3.l_commitdate)) ----------------------------PhysicalOlapScan[lineitem] -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[supplier] -----------------------------PhysicalDistribute +------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) +--------------------------PhysicalProject +----------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +------------------------------PhysicalOlapScan[lineitem] +--------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ------------------------------PhysicalProject ---------------------------------filter((nation.n_name = 'SAUDI ARABIA')) -----------------------------------PhysicalOlapScan[nation] +--------------------------------PhysicalOlapScan[supplier] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((nation.n_name = 'SAUDI ARABIA')) +------------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out index 7845eba2baf8c4..dc8f7970dcd585 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out @@ -13,15 +13,16 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalOlapScan[orders] ------------------PhysicalDistribute ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) -----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) ---------------------------PhysicalOlapScan[customer] -----------------------PhysicalDistribute -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) -----------------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +------------------------PhysicalProject +--------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +----------------------------PhysicalOlapScan[customer] +------------------------PhysicalDistribute +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out index 57e1aad20cd1d1..02ade98d230100 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out @@ -22,9 +22,8 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[supplier] --------------------------------PhysicalDistribute ----------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[nation] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[nation] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------filter((region.r_name = 'ASIA')) diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q15.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q15.out index 6d60f8b8052413..48482b276f3d29 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q15.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q15.out @@ -6,25 +6,26 @@ PhysicalResultSink ------PhysicalQuickSort --------PhysicalProject ----------hashJoin[INNER_JOIN](supplier.s_suppkey = revenue0.supplier_no) -------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------filter((lineitem.l_shipdate >= '1996-01-01')(lineitem.l_shipdate < '1996-04-01')) ---------------------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter((lineitem.l_shipdate >= '1996-01-01')(lineitem.l_shipdate < '1996-04-01')) -----------------------------------PhysicalOlapScan[lineitem] +------------PhysicalProject +--------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------filter((lineitem.l_shipdate >= '1996-01-01')(lineitem.l_shipdate < '1996-04-01')) +----------------------------PhysicalOlapScan[lineitem] +----------------PhysicalDistribute +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((lineitem.l_shipdate >= '1996-01-01')(lineitem.l_shipdate < '1996-04-01')) +------------------------------------PhysicalOlapScan[lineitem] ------------PhysicalDistribute --------------PhysicalProject ----------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out index 6eabacb321d308..1eb948e315b211 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out @@ -9,17 +9,14 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) -------------------PhysicalDistribute ---------------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[partsupp] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((s_comment like '%Customer%Complaints%')) -----------------------------PhysicalOlapScan[supplier] -------------------PhysicalDistribute +------------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) --------------------PhysicalProject -----------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) -------------------------PhysicalOlapScan[part] +----------------------PhysicalOlapScan[partsupp] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((s_comment like '%Customer%Complaints%')) +--------------------------PhysicalOlapScan[supplier] +------------------PhysicalProject +--------------------filter(( not (p_brand = 'Brand#45'))( not (p_type like 'MEDIUM POLISHED%'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) +----------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out index 7845eba2baf8c4..dc8f7970dcd585 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out @@ -13,15 +13,16 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalOlapScan[orders] ------------------PhysicalDistribute ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) -----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) ---------------------------PhysicalOlapScan[customer] -----------------------PhysicalDistribute -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) -----------------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +------------------------PhysicalProject +--------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +----------------------------PhysicalOlapScan[customer] +------------------------PhysicalDistribute +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q5.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q5.out index 9fc242ea918989..a2071db0cd1ed9 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q5.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q5.out @@ -13,21 +13,20 @@ PhysicalResultSink --------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN](customer.c_nationkey = supplier.s_nationkey)(lineitem.l_suppkey = supplier.s_suppkey) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineitem] +------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[lineitem] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((orders.o_orderdate < '1995-01-01')(orders.o_orderdate >= '1994-01-01')) ---------------------------------------------PhysicalOlapScan[orders] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[customer] +----------------------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((orders.o_orderdate < '1995-01-01')(orders.o_orderdate >= '1994-01-01')) +------------------------------------------PhysicalOlapScan[orders] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[customer] --------------------------PhysicalDistribute ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q8.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q8.out index 4ce12cd85a37cc..ae4326989d5810 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q8.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q8.out @@ -16,25 +16,24 @@ PhysicalResultSink --------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ---------------------------------------PhysicalDistribute -----------------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) -------------------------------------------PhysicalDistribute ---------------------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[lineitem] -----------------------------------------------PhysicalProject -------------------------------------------------filter((orders.o_orderdate <= '1996-12-31')(orders.o_orderdate >= '1995-01-01')) ---------------------------------------------------PhysicalOlapScan[orders] -------------------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +------------------------------------PhysicalDistribute +--------------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) +----------------------------------------PhysicalDistribute +------------------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[lineitem] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[customer] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) ---------------------------------------------PhysicalOlapScan[part] +----------------------------------------------filter((orders.o_orderdate <= '1996-12-31')(orders.o_orderdate >= '1995-01-01')) +------------------------------------------------PhysicalOlapScan[orders] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[customer] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) +------------------------------------------PhysicalOlapScan[part] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q9.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q9.out index e173453102b71c..98ca2fe323df1e 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q9.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q9.out @@ -11,24 +11,23 @@ PhysicalResultSink ----------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ------------------PhysicalProject --------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ---------------------------------PhysicalDistribute -----------------------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[lineitem] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[orders] ---------------------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +------------------------------PhysicalDistribute +--------------------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) ----------------------------------PhysicalProject -------------------------------------filter((p_name like '%green%')) ---------------------------------------PhysicalOlapScan[part] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[partsupp] +------------------------------------PhysicalOlapScan[lineitem] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[orders] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((p_name like '%green%')) +------------------------------------PhysicalOlapScan[part] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[partsupp] ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_seq_map_col.out b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_seq_map_col.out new file mode 100644 index 00000000000000..5bdfef021827ac --- /dev/null +++ b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_seq_map_col.out @@ -0,0 +1,189 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +1 1 \N +2 2 \N +3 3 \N +4 4 \N + +-- !sql1 -- +1 1 20 +2 2 20 +3 3 \N +4 4 \N + +-- !sql1 -- +1 1 20 +2 2 20 +3 3 \N +4 4 \N + +-- !sql1 -- +3 3 2099-09-10T12:00:00.977174 \N 2099-09-10T12:00:00.977174 +4 4 2099-09-10T12:00:00.977174 \N 2099-09-10T12:00:00.977174 + +-- !sql2 -- +1 1 +2 2 +3 3 +4 4 + +-- !sql3 -- +1 1 999 999 +2 2 999 999 +3 3 999 999 +4 4 999 999 + +-- !sql3 -- +1 99 8888 8888 +2 99 8888 8888 +3 3 999 999 +4 4 999 999 +5 99 8888 8888 + +-- !sql4 -- +1 1 \N \N +2 2 \N \N +3 3 \N \N +4 4 \N \N + +-- !sql1 -- +1 1 \N +2 2 \N +3 3 \N +4 4 \N + +-- !sql1 -- +1 1 20 +2 2 20 +3 3 \N +4 4 \N + +-- !sql1 -- +1 1 20 +2 2 20 +3 3 \N +4 4 \N + +-- !sql1 -- +3 3 2099-09-10T12:00:00.977174 \N 2099-09-10T12:00:00.977174 +4 4 2099-09-10T12:00:00.977174 \N 2099-09-10T12:00:00.977174 + +-- !sql2 -- +1 1 +2 2 +3 3 +4 4 + +-- !sql3 -- +1 1 999 999 +2 2 999 999 +3 3 999 999 +4 4 999 999 + +-- !sql3 -- +1 99 8888 8888 +2 99 8888 8888 +3 3 999 999 +4 4 999 999 +5 99 8888 8888 + +-- !sql4 -- +1 1 \N \N +2 2 \N \N +3 3 \N \N +4 4 \N \N + +-- !sql1 -- +1 1 \N +2 2 \N +3 3 \N +4 4 \N + +-- !sql1 -- +1 1 20 +2 2 20 +3 3 \N +4 4 \N + +-- !sql1 -- +1 1 20 +2 2 20 +3 3 \N +4 4 \N + +-- !sql1 -- +3 3 2099-09-10T12:00:00.977174 \N 2099-09-10T12:00:00.977174 +4 4 2099-09-10T12:00:00.977174 \N 2099-09-10T12:00:00.977174 + +-- !sql2 -- +1 1 +2 2 +3 3 +4 4 + +-- !sql3 -- +1 1 999 999 +2 2 999 999 +3 3 999 999 +4 4 999 999 + +-- !sql3 -- +1 99 8888 8888 +2 99 8888 8888 +3 3 999 999 +4 4 999 999 +5 99 8888 8888 + +-- !sql4 -- +1 1 \N \N +2 2 \N \N +3 3 \N \N +4 4 \N \N + +-- !sql1 -- +1 1 \N +2 2 \N +3 3 \N +4 4 \N + +-- !sql1 -- +1 1 20 +2 2 20 +3 3 \N +4 4 \N + +-- !sql1 -- +1 1 20 +2 2 20 +3 3 \N +4 4 \N + +-- !sql1 -- +3 3 2099-09-10T12:00:00.977174 \N 2099-09-10T12:00:00.977174 +4 4 2099-09-10T12:00:00.977174 \N 2099-09-10T12:00:00.977174 + +-- !sql2 -- +1 1 +2 2 +3 3 +4 4 + +-- !sql3 -- +1 1 999 999 +2 2 999 999 +3 3 999 999 +4 4 999 999 + +-- !sql3 -- +1 99 8888 8888 +2 99 8888 8888 +3 3 999 999 +4 4 999 999 +5 99 8888 8888 + +-- !sql4 -- +1 1 \N \N +2 2 \N \N +3 3 \N \N +4 4 \N \N + diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index 9a14346af10bd8..9a688a1b4d1edd 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -532,6 +532,16 @@ class Suite implements GroovyInterceptable { runAction(new WaitForAction(context), actionSupplier) } + void expectExceptionLike(Closure userFunction, String errorMessage = null) { + try { + userFunction() + } catch (Exception e) { + if (!e.getMessage().contains(errorMessage)) { + throw e + } + } + } + String getBrokerName() { String brokerName = context.config.otherConfigs.get("brokerName") return brokerName diff --git a/regression-test/suites/auth_p0/test_grant_nonexist_table.groovy b/regression-test/suites/auth_p0/test_grant_nonexist_table.groovy new file mode 100644 index 00000000000000..36e75707be7252 --- /dev/null +++ b/regression-test/suites/auth_p0/test_grant_nonexist_table.groovy @@ -0,0 +1,45 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.junit.Assert; + +suite("test_grant_nonexist_table","p0,auth") { + String suiteName = "test_grant_nonexist_table" + String dbName = context.config.getDbNameByFile(context.file) + String user = "${suiteName}_user" + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + + test { + sql """grant select_priv on non_exist_catalog.*.* to ${user}""" + exception "catalog" + } + + test { + sql """grant select_priv on internal.non_exist_db.* to ${user}""" + exception "database" + } + + test { + sql """grant select_priv on internal.${dbName}.non_exist_table to ${user}""" + exception "table" + } + + + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/backup_restore/test_backup_restore_atomic.groovy b/regression-test/suites/backup_restore/test_backup_restore_atomic.groovy new file mode 100644 index 00000000000000..4b87340fb35aa1 --- /dev/null +++ b/regression-test/suites/backup_restore/test_backup_restore_atomic.groovy @@ -0,0 +1,209 @@ +// 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_backup_restore_atomic", "backup_restore") { + String suiteName = "test_backup_restore_atomic" + String dbName = "${suiteName}_db_1" + String dbName1 = "${suiteName}_db_2" + String repoName = "repo_" + UUID.randomUUID().toString().replace("-", "") + String snapshotName = "${suiteName}_snapshot" + String tableNamePrefix = "${suiteName}_tables" + + def syncer = getSyncer() + syncer.createS3Repository(repoName) + sql "CREATE DATABASE IF NOT EXISTS ${dbName}" + sql "CREATE DATABASE IF NOT EXISTS ${dbName1}" + + // 1. restore to not exists table_0 + // 2. restore partial data to table_1 + // 3. restore less data to table_2 + // 4. restore incremental data to table_3 + int numTables = 4; + List tables = [] + for (int i = 0; i < numTables; ++i) { + String tableName = "${tableNamePrefix}_${i}" + tables.add(tableName) + sql "DROP TABLE IF EXISTS ${dbName}.${tableName}" + sql """ + CREATE TABLE ${dbName}.${tableName} ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0" + ) + AGGREGATE KEY(`id`) + PARTITION BY RANGE(`id`) + ( + PARTITION p1 VALUES LESS THAN ("10"), + PARTITION p2 VALUES LESS THAN ("20"), + PARTITION p3 VALUES LESS THAN ("30"), + PARTITION p4 VALUES LESS THAN ("40"), + PARTITION p5 VALUES LESS THAN ("50"), + PARTITION p6 VALUES LESS THAN ("60"), + PARTITION p7 VALUES LESS THAN ("120") + ) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + } + + // 5. the len of table name equals to the config table_name_length_limit + def maxLabelLen = getFeConfig("table_name_length_limit").toInteger() + def maxTableName = "".padRight(maxLabelLen, "x") + logger.info("config table_name_length_limit = ${maxLabelLen}, table name = ${maxTableName}") + sql "DROP TABLE IF EXISTS ${dbName}.${maxTableName}" + sql """ + CREATE TABLE ${dbName}.${maxTableName} ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0" + ) + AGGREGATE KEY(`id`) + PARTITION BY RANGE(`id`) + ( + PARTITION p1 VALUES LESS THAN ("10"), + PARTITION p2 VALUES LESS THAN ("20"), + PARTITION p3 VALUES LESS THAN ("30"), + PARTITION p4 VALUES LESS THAN ("40"), + PARTITION p5 VALUES LESS THAN ("50"), + PARTITION p6 VALUES LESS THAN ("60"), + PARTITION p7 VALUES LESS THAN ("120") + ) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + tables.add(maxTableName) + + int numRows = 10; + List values = [] + for (int j = 1; j <= numRows; ++j) { + values.add("(${j}0, ${j}0)") + } + + sql "INSERT INTO ${dbName}.${tableNamePrefix}_0 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_1 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_2 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_3 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${maxTableName} VALUES ${values.join(",")}" + + // the other partitions of table_1 will be drop + sql """ + BACKUP SNAPSHOT ${dbName}.${snapshotName} + TO `${repoName}` + ON ( + ${tableNamePrefix}_0, + ${tableNamePrefix}_1 PARTITION (p1, p2, p3), + ${tableNamePrefix}_2, + ${tableNamePrefix}_3, + ${maxTableName} + ) + """ + + syncer.waitSnapshotFinish(dbName) + + def snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName) + assertTrue(snapshot != null) + + // drop table_0 + sql "DROP TABLE ${dbName}.${tableNamePrefix}_0 FORCE" + + // insert external data to table_2 + sql "INSERT INTO ${dbName}.${tableNamePrefix}_2 VALUES ${values.join(",")}" + + sql "TRUNCATE TABLE ${dbName}.${tableNamePrefix}_3" + + sql """ + RESTORE SNAPSHOT ${dbName}.${snapshotName} + FROM `${repoName}` + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true", + "atomic_restore" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName) + + for (def tableName in tables) { + qt_sql "SELECT * FROM ${dbName}.${tableName} ORDER BY id" + } + + // restore table_3 to new db + sql """ + RESTORE SNAPSHOT ${dbName1}.${snapshotName} + FROM `${repoName}` + ON (${tableNamePrefix}_3) + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true", + "atomic_restore" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName1) + + qt_sql "SELECT * FROM ${dbName1}.${tableNamePrefix}_3 ORDER BY id" + + // add partition and insert some data. + sql "ALTER TABLE ${dbName}.${tableNamePrefix}_3 ADD PARTITION p8 VALUES LESS THAN MAXVALUE" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_3 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_3 VALUES (200, 200)" + + // backup again + snapshotName = "${snapshotName}_1" + sql """ + BACKUP SNAPSHOT ${dbName}.${snapshotName} + TO `${repoName}` + ON (${tableNamePrefix}_3) + """ + + syncer.waitSnapshotFinish(dbName) + + snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName) + assertTrue(snapshot != null) + + // restore with incremental data + sql """ + RESTORE SNAPSHOT ${dbName1}.${snapshotName} + FROM `${repoName}` + ON (${tableNamePrefix}_3) + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true", + "atomic_restore" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName1) + + qt_sql "SELECT * FROM ${dbName1}.${tableNamePrefix}_3 ORDER BY id" + + for (def tableName in tables) { + sql "DROP TABLE ${dbName}.${tableName} FORCE" + } + sql "DROP DATABASE ${dbName} FORCE" + sql "DROP DATABASE ${dbName1} FORCE" + sql "DROP REPOSITORY `${repoName}`" +} + + diff --git a/regression-test/suites/backup_restore/test_backup_restore_atomic_cancel.groovy b/regression-test/suites/backup_restore/test_backup_restore_atomic_cancel.groovy new file mode 100644 index 00000000000000..3487c93b0d6572 --- /dev/null +++ b/regression-test/suites/backup_restore/test_backup_restore_atomic_cancel.groovy @@ -0,0 +1,128 @@ +// 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_backup_restore_atomic_cancel") { + String suiteName = "test_backup_restore_atomic_cancelled" + String repoName = "repo_" + UUID.randomUUID().toString().replace("-", "") + String dbName = "${suiteName}_db" + String tableName = "${suiteName}_table" + String tableName1 = "${suiteName}_table_1" + String viewName = "${suiteName}_view" + String snapshotName = "${suiteName}_snapshot" + + def syncer = getSyncer() + syncer.createS3Repository(repoName) + + sql "CREATE DATABASE IF NOT EXISTS ${dbName}" + sql "DROP TABLE IF EXISTS ${dbName}.${tableName}" + sql """ + CREATE TABLE ${dbName}.${tableName} ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0") + AGGREGATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + sql "DROP TABLE IF EXISTS ${dbName}.${tableName1}" + sql """ + CREATE TABLE ${dbName}.${tableName1} ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0") + AGGREGATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + sql "DROP VIEW IF EXISTS ${dbName}.${viewName}" + sql """ + CREATE VIEW ${dbName}.${viewName} + AS + SELECT id, count FROM ${dbName}.${tableName} + WHERE id > 5 + """ + + List values = [] + for (int i = 1; i <= 10; ++i) { + values.add("(${i}, ${i})") + } + sql "INSERT INTO ${dbName}.${tableName} VALUES ${values.join(",")}" + def result = sql "SELECT * FROM ${dbName}.${tableName}" + assertEquals(result.size(), values.size()); + + sql "INSERT INTO ${dbName}.${tableName1} VALUES ${values.join(",")}" + result = sql "SELECT * FROM ${dbName}.${tableName1}" + assertEquals(result.size(), values.size()); + + + sql """ + BACKUP SNAPSHOT ${dbName}.${snapshotName} + TO `${repoName}` + """ + + syncer.waitSnapshotFinish(dbName) + + // alter view and restore, it must failed because the signatures are not matched + + sql """ + ALTER VIEW ${dbName}.${viewName} + AS + SELECT id,count FROM ${dbName}.${tableName} + WHERE id < 100 + + """ + + sql "INSERT INTO ${dbName}.${tableName} VALUES (11, 11)" + + def snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName) + sql """ + RESTORE SNAPSHOT ${dbName}.${snapshotName} + FROM `${repoName}` + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true", + "atomic_restore" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName) + + def restore_result = sql_return_maparray """ SHOW RESTORE FROM ${dbName} WHERE Label ="${snapshotName}" """ + restore_result.last() + logger.info("show restore result: ${restore_result}") + assertTrue(restore_result.last().State == "CANCELLED") + + + // Do not affect any tables. + result = sql "SELECT * FROM ${dbName}.${tableName}" + assertEquals(result.size(), values.size() + 1); + + result = sql "SELECT * FROM ${dbName}.${tableName1}" + assertEquals(result.size(), values.size()); + + sql "DROP TABLE ${dbName}.${tableName} FORCE" + sql "DROP TABLE ${dbName}.${tableName1} FORCE" + sql "DROP DATABASE ${dbName} FORCE" + sql "DROP REPOSITORY `${repoName}`" +} + + diff --git a/regression-test/suites/backup_restore/test_backup_restore_atomic_with_alter.groovy b/regression-test/suites/backup_restore/test_backup_restore_atomic_with_alter.groovy new file mode 100644 index 00000000000000..46a3ca5b29dbf2 --- /dev/null +++ b/regression-test/suites/backup_restore/test_backup_restore_atomic_with_alter.groovy @@ -0,0 +1,241 @@ +// 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_backup_restore_atomic_with_alter", "backup_restore") { + if (!getFeConfig("enable_debug_points").equals("true")) { + logger.info("Config.enable_debug_points=true is required") + return + } + + String suiteName = "test_backup_restore_atomic_with_alter" + String dbName = "${suiteName}_db" + String repoName = "repo_" + UUID.randomUUID().toString().replace("-", "") + String snapshotName = "snapshot_" + UUID.randomUUID().toString().replace("-", "") + String tableNamePrefix = "${suiteName}_tables" + + def syncer = getSyncer() + syncer.createS3Repository(repoName) + sql "DROP DATABASE IF EXISTS ${dbName} FORCE" + sql "CREATE DATABASE ${dbName}" + + // during restoring, if: + // 1. table_0 not exists, create table_0 is not allowed + // 2. table_1 exists, alter operation is not allowed + // 3. table_1 exists, drop table is not allowed + // 4. table_0 not exists, rename table_2 to table_0 is not allowed + int numTables = 3; + List tables = [] + for (int i = 0; i < numTables; ++i) { + String tableName = "${tableNamePrefix}_${i}" + tables.add(tableName) + sql "DROP TABLE IF EXISTS ${dbName}.${tableName}" + sql """ + CREATE TABLE ${dbName}.${tableName} ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0" + ) + AGGREGATE KEY(`id`) + PARTITION BY RANGE(`id`) + ( + PARTITION p1 VALUES LESS THAN ("10"), + PARTITION p2 VALUES LESS THAN ("20"), + PARTITION p3 VALUES LESS THAN ("30"), + PARTITION p4 VALUES LESS THAN ("40"), + PARTITION p5 VALUES LESS THAN ("50"), + PARTITION p6 VALUES LESS THAN ("60"), + PARTITION p7 VALUES LESS THAN ("120") + ) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + } + + int numRows = 10; + List values = [] + for (int j = 1; j <= numRows; ++j) { + values.add("(${j}0, ${j}0)") + } + + sql "INSERT INTO ${dbName}.${tableNamePrefix}_0 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_1 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_2 VALUES ${values.join(",")}" + + // only backup table 0,1 + sql """ + BACKUP SNAPSHOT ${dbName}.${snapshotName} + TO `${repoName}` + ON ( + ${tableNamePrefix}_0, + ${tableNamePrefix}_1 + ) + """ + + syncer.waitSnapshotFinish(dbName) + + def snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName) + assertTrue(snapshot != null) + + // drop table_0 + sql "DROP TABLE ${dbName}.${tableNamePrefix}_0 FORCE" + + // disable restore + GetDebugPoint().enableDebugPointForAllFEs("FE.PAUSE_NON_PENDING_RESTORE_JOB", [value:snapshotName]) + + sql """ + RESTORE SNAPSHOT ${dbName}.${snapshotName} + FROM `${repoName}` + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true", + "atomic_restore" = "true" + ) + """ + + boolean restore_paused = false + for (int k = 0; k < 60; k++) { + def records = sql_return_maparray """ SHOW RESTORE FROM ${dbName} WHERE Label = "${snapshotName}" """ + if (records.size() == 1 && records[0].State != 'PENDING') { + restore_paused = true + break + } + logger.info("SHOW RESTORE result: ${records}") + sleep(3000) + } + assertTrue(restore_paused) + + // 0. table_1 has in_atomic_restore property + def show_result = sql """ SHOW CREATE TABLE ${dbName}.${tableNamePrefix}_1 """ + logger.info("SHOW CREATE TABLE ${tableNamePrefix}_1: ${show_result}") + assertTrue(show_result[0][1].contains("in_atomic_restore")) + + // 1. create a restoring table (not exists before) + expectExceptionLike({ -> + sql """ + CREATE TABLE ${dbName}.${tableNamePrefix}_0 + ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0" + ) + AGGREGATE KEY(`id`) + PARTITION BY RANGE(`id`) + ( + PARTITION p1 VALUES LESS THAN ("10"), + PARTITION p2 VALUES LESS THAN ("20"), + PARTITION p3 VALUES LESS THAN ("30"), + PARTITION p4 VALUES LESS THAN ("40"), + PARTITION p5 VALUES LESS THAN ("50"), + PARTITION p6 VALUES LESS THAN ("60"), + PARTITION p7 VALUES LESS THAN ("120") + ) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + }, "is in atomic restore, please cancel the restore operation firstly") + + // 2. alter is not allowed + expectExceptionLike({ + sql """ + ALTER TABLE ${dbName}.${tableNamePrefix}_1 + ADD PARTITION p8 VALUES LESS THAN("200") + """ + }, "Do not allow doing ALTER ops") + expectExceptionLike({ + sql """ + ALTER TABLE ${dbName}.${tableNamePrefix}_1 + DROP PARTITION p1 + """ + }, "Do not allow doing ALTER ops") + expectExceptionLike({ + sql """ + ALTER TABLE ${dbName}.${tableNamePrefix}_1 + MODIFY PARTITION p1 SET ("key"="value") + """ + }, "Do not allow doing ALTER ops") + expectExceptionLike({ + sql """ + ALTER TABLE ${dbName}.${tableNamePrefix}_1 + ADD COLUMN new_col INT DEFAULT "0" AFTER count + """ + }, "Do not allow doing ALTER ops") + expectExceptionLike({ + sql """ + ALTER TABLE ${dbName}.${tableNamePrefix}_1 + DROP COLUMN count + """ + }, "Do not allow doing ALTER ops") + expectExceptionLike({ + sql """ + ALTER TABLE ${dbName}.${tableNamePrefix}_1 + SET ("is_being_synced"="false") + """ + }, "Do not allow doing ALTER ops") + expectExceptionLike({ + sql """ + ALTER TABLE ${dbName}.${tableNamePrefix}_1 + RENAME newTableName + """ + }, "Do not allow doing ALTER ops") + // BTW, the tmp table also don't allow rename + expectExceptionLike({ + sql """ + ALTER TABLE ${dbName}.__doris_atomic_restore_prefix__${tableNamePrefix}_1 + RENAME newTableName + """ + }, "Do not allow doing ALTER ops") + // 3. drop table is not allowed + expectExceptionLike({ + sql """ + DROP TABLE ${dbName}.${tableNamePrefix}_1 + """ + }, "state is in atomic restore") + expectExceptionLike({ + sql """ + DROP TABLE ${dbName}.__doris_atomic_restore_prefix__${tableNamePrefix}_1 + """ + }, "state is RESTORE") + // 4. the table name is occupied + expectExceptionLike({ + sql """ + ALTER TABLE ${dbName}.${tableNamePrefix}_2 + RENAME ${tableNamePrefix}_0 + """ + }, "is already used (in restoring)") + + + sql "CANCEL RESTORE FROM ${dbName}" + + // 5. The restore job is cancelled, the in_atomic_restore property has been removed. + show_result = sql """ SHOW CREATE TABLE ${dbName}.${tableNamePrefix}_1 """ + logger.info("SHOW CREATE TABLE ${tableNamePrefix}_1: ${show_result}") + assertFalse(show_result[0][1].contains("in_atomic_restore")) + + for (def tableName in tables) { + sql "DROP TABLE IF EXISTS ${dbName}.${tableName} FORCE" + } + sql "DROP DATABASE ${dbName} FORCE" + sql "DROP REPOSITORY `${repoName}`" +} + + + diff --git a/regression-test/suites/backup_restore/test_backup_restore_atomic_with_view.groovy b/regression-test/suites/backup_restore/test_backup_restore_atomic_with_view.groovy new file mode 100644 index 00000000000000..9d090281364245 --- /dev/null +++ b/regression-test/suites/backup_restore/test_backup_restore_atomic_with_view.groovy @@ -0,0 +1,124 @@ +// 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_backup_restore_atomic_with_view", "backup_restore") { + String suiteName = "backup_restore_atomic_with_view" + String dbName = "${suiteName}_db" + String dbName1 = "${suiteName}_db_1" + String repoName = "repo_" + UUID.randomUUID().toString().replace("-", "") + String snapshotName = "${suiteName}_snapshot" + String tableName = "${suiteName}_table" + String viewName = "${suiteName}_view" + + def syncer = getSyncer() + syncer.createS3Repository(repoName) + sql "CREATE DATABASE IF NOT EXISTS ${dbName}" + sql "CREATE DATABASE IF NOT EXISTS ${dbName1}" + + int numRows = 10; + sql "DROP TABLE IF EXISTS ${dbName}.${tableName} FORCE" + sql "DROP VIEW IF EXISTS ${dbName}.${viewName}" + sql """ + CREATE TABLE ${dbName}.${tableName} ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0" + ) + AGGREGATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + List values = [] + for (int j = 1; j <= numRows; ++j) { + values.add("(${j}, ${j})") + } + sql "INSERT INTO ${dbName}.${tableName} VALUES ${values.join(",")}" + + sql """CREATE VIEW ${dbName}.${viewName} (id, count) + AS + SELECT * FROM ${dbName}.${tableName} WHERE count > 5 + """ + + qt_sql "SELECT * FROM ${dbName}.${tableName} ORDER BY id ASC" + qt_sql "SELECT * FROM ${dbName}.${viewName} ORDER BY id ASC" + + sql """ + BACKUP SNAPSHOT ${dbName}.${snapshotName} + TO `${repoName}` + """ + + syncer.waitSnapshotFinish(dbName) + + def snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName) + assertTrue(snapshot != null) + + // restore new view + sql "DROP TABLE IF EXISTS ${dbName1}.${tableName} FORCE" + sql "DROP VIEW IF EXISTS ${dbName1}.${viewName}" + + sql """ + RESTORE SNAPSHOT ${dbName1}.${snapshotName} + FROM `${repoName}` + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "atomic_restore" = "true", + "reserve_replica" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName1) + + qt_sql "SELECT * FROM ${dbName1}.${tableName} ORDER BY id ASC" + qt_sql "SELECT * FROM ${dbName1}.${viewName} ORDER BY id ASC" + def show_view_result = sql_return_maparray "SHOW VIEW FROM ${tableName} FROM ${dbName1}" + logger.info("show view result: ${show_view_result}") + assertTrue(show_view_result.size() == 1); + def show_view = show_view_result[0]['Create View'] + assertTrue(show_view.contains("${dbName1}")) + assertTrue(show_view.contains("${tableName}")) + + // restore an exists view + sql """ + RESTORE SNAPSHOT ${dbName}.${snapshotName} + FROM `${repoName}` + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "atomic_restore" = "true", + "reserve_replica" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName) + def restore_result = sql_return_maparray """ SHOW RESTORE FROM ${dbName} WHERE Label ="${snapshotName}" """ + restore_result.last() + logger.info("show restore result: ${restore_result}") + assertTrue(restore_result.last().State == "FINISHED") + + // View could read the incremental data. + sql "INSERT INTO ${dbName}.${tableName} VALUES (11, 11)" + + qt_sql "SELECT * FROM ${dbName}.${tableName} ORDER BY id ASC" + qt_sql "SELECT * FROM ${dbName}.${viewName} ORDER BY id ASC" + + sql "DROP REPOSITORY `${repoName}`" +} + + diff --git a/regression-test/suites/backup_restore/test_backup_restore_clean_restore.groovy b/regression-test/suites/backup_restore/test_backup_restore_clean_restore.groovy index c80bc0d0060232..b667e0cfcc3164 100644 --- a/regression-test/suites/backup_restore/test_backup_restore_clean_restore.groovy +++ b/regression-test/suites/backup_restore/test_backup_restore_clean_restore.groovy @@ -77,6 +77,7 @@ suite("test_backup_restore_clean_restore", "backup_restore") { ) """ + sql "INSERT INTO ${dbName}.${tableName2} VALUES ${values.join(",")}" result = sql "SELECT * FROM ${dbName}.${tableName2}" assertEquals(result.size(), numRows); @@ -106,6 +107,25 @@ suite("test_backup_restore_clean_restore", "backup_restore") { result = sql "SELECT * FROM ${dbName}.${tableName3}" assertEquals(result.size(), numRows); + // view 1 must exists + String viewName1 = "${tableNamePrefix}_4" + sql "DROP VIEW IF EXISTS ${dbName}.${viewName1}" + sql """ + CREATE VIEW ${dbName}.${viewName1} (k1, k2) + AS + SELECT id as k1, count as k2 FROM ${dbName}.${tableName1} + WHERE id in (1,3,5,7,9) + """ + + // view 2 will be deleted + String viewName2 = "${tableNamePrefix}_5" + sql "DROP VIEW IF EXISTS ${dbName}.${viewName2}" + sql """ + CREATE VIEW ${dbName}.${viewName2} (k1, k2) + AS + SELECT id as k1, count as k2 FROM ${dbName}.${tableName3} + WHERE id in (1,3,5,7,9) + """ sql """ BACKUP SNAPSHOT ${dbName}.${snapshotName} @@ -119,13 +139,14 @@ suite("test_backup_restore_clean_restore", "backup_restore") { def snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName) assertTrue(snapshot != null) - // restore table1, partition 3 of table2 + // restore table1, partition 3 of table2, view1 sql """ RESTORE SNAPSHOT ${dbName}.${snapshotName} FROM `${repoName}` ON ( `${tableName1}`, - `${tableName2}` PARTITION (`p3`) + `${tableName2}` PARTITION (`p3`), + `${viewName1}` ) PROPERTIES ( @@ -148,12 +169,23 @@ suite("test_backup_restore_clean_restore", "backup_restore") { result = sql "SELECT * FROM ${dbName}.${tableName2}" assertEquals(result.size(), numRows-10) + // view1 are exists + result = sql """ SHOW VIEW FROM ${tableName1} FROM ${dbName} """ + assertEquals(result.size(), 1) + + // view2 are dropped + result = sql """ + SHOW TABLE STATUS FROM ${dbName} LIKE "${viewName2}" + """ + assertEquals(result.size(), 0) + // table3 are dropped result = sql """ SHOW TABLE STATUS FROM ${dbName} LIKE "${tableName3}" """ assertEquals(result.size(), 0) + sql "DROP VIEW ${dbName}.${viewName1}" sql "DROP TABLE ${dbName}.${tableName1} FORCE" sql "DROP TABLE ${dbName}.${tableName2} FORCE" sql "DROP DATABASE ${dbName} FORCE" diff --git a/regression-test/suites/backup_restore/test_backup_restore_with_view.groovy b/regression-test/suites/backup_restore/test_backup_restore_with_view.groovy new file mode 100644 index 00000000000000..10b21bb3442082 --- /dev/null +++ b/regression-test/suites/backup_restore/test_backup_restore_with_view.groovy @@ -0,0 +1,137 @@ +// 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_backup_restore_with_view", "backup_restore") { + String suiteName = "backup_restore_with_view" + String dbName = "${suiteName}_db" + String dbName1 = "${suiteName}_db_1" + String repoName = "repo_" + UUID.randomUUID().toString().replace("-", "") + String snapshotName = "${suiteName}_snapshot" + String tableName = "${suiteName}_table" + String viewName = "${suiteName}_view" + + def syncer = getSyncer() + syncer.createS3Repository(repoName) + sql "CREATE DATABASE IF NOT EXISTS ${dbName}" + sql "CREATE DATABASE IF NOT EXISTS ${dbName1}" + + int numRows = 10; + sql "DROP TABLE IF EXISTS ${dbName}.${tableName} FORCE" + sql "DROP VIEW IF EXISTS ${dbName}.${viewName}" + sql """ + CREATE TABLE ${dbName}.${tableName} ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0" + ) + AGGREGATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + List values = [] + for (int j = 1; j <= numRows; ++j) { + values.add("(${j}, ${j})") + } + sql "INSERT INTO ${dbName}.${tableName} VALUES ${values.join(",")}" + + sql """CREATE VIEW ${dbName}.${viewName} (id, count) + AS + SELECT * FROM ${dbName}.${tableName} WHERE count > 5 + """ + + qt_sql "SELECT * FROM ${dbName}.${tableName} ORDER BY id ASC" + qt_sql "SELECT * FROM ${dbName}.${viewName} ORDER BY id ASC" + + sql """ + BACKUP SNAPSHOT ${dbName}.${snapshotName} + TO `${repoName}` + """ + + syncer.waitSnapshotFinish(dbName) + + def snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName) + assertTrue(snapshot != null) + + sql "DROP TABLE IF EXISTS ${dbName1}.${tableName} FORCE" + sql "DROP VIEW IF EXISTS ${dbName1}.${viewName}" + + sql """ + RESTORE SNAPSHOT ${dbName1}.${snapshotName} + FROM `${repoName}` + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName1) + + qt_sql "SELECT * FROM ${dbName1}.${tableName} ORDER BY id ASC" + qt_sql "SELECT * FROM ${dbName1}.${viewName} ORDER BY id ASC" + def show_view_result = sql_return_maparray "SHOW VIEW FROM ${tableName} FROM ${dbName1}" + logger.info("show view result: ${show_view_result}") + assertTrue(show_view_result.size() == 1); + def show_view = show_view_result[0]['Create View'] + assertTrue(show_view.contains("${dbName1}")) + assertTrue(show_view.contains("${tableName}")) + + // restore to db, test the view signature. + sql """ + RESTORE SNAPSHOT ${dbName}.${snapshotName} + FROM `${repoName}` + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName) + def restore_result = sql_return_maparray """ SHOW RESTORE FROM ${dbName} WHERE Label ="${snapshotName}" """ + restore_result.last() + logger.info("show restore result: ${restore_result}") + assertTrue(restore_result.last().State == "FINISHED") + + // restore to db1, test the view signature. + sql """ + RESTORE SNAPSHOT ${dbName1}.${snapshotName} + FROM `${repoName}` + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName1) + restore_result = sql_return_maparray """ SHOW RESTORE FROM ${dbName1} WHERE Label ="${snapshotName}" """ + restore_result.last() + logger.info("show restore result: ${restore_result}") + assertTrue(restore_result.last().State == "FINISHED") + + sql "DROP TABLE ${dbName}.${tableName} FORCE" + sql "DROP VIEW ${dbName}.${viewName}" + sql "DROP DATABASE ${dbName} FORCE" + sql "DROP TABLE ${dbName1}.${tableName} FORCE" + sql "DROP VIEW ${dbName1}.${viewName}" + sql "DROP DATABASE ${dbName1} FORCE" + sql "DROP REPOSITORY `${repoName}`" +} + diff --git a/regression-test/suites/ccr_mow_syncer_p0/test_get_binlog.groovy b/regression-test/suites/ccr_mow_syncer_p0/test_get_binlog.groovy index 62ba97551f6e69..13fe4eeec33d2d 100644 --- a/regression-test/suites/ccr_mow_syncer_p0/test_get_binlog.groovy +++ b/regression-test/suites/ccr_mow_syncer_p0/test_get_binlog.groovy @@ -133,7 +133,6 @@ suite("test_mow_get_binlog_case") { sql """DROP USER IF EXISTS ${noPrivUser}""" sql """CREATE USER ${noPrivUser} IDENTIFIED BY '123456'""" sql """GRANT ALL ON ${context.config.defaultDb}.* TO ${noPrivUser}""" - sql """GRANT ALL ON TEST_${context.dbName}.${emptyTable} TO ${noPrivUser}""" syncer.context.user = "${noPrivUser}" syncer.context.passwd = "123456" assertTrue((syncer.getBinlog("${seqTableName}")) == false) diff --git a/regression-test/suites/external_table_p0/es/test_es_catalog_http_open_api.groovy b/regression-test/suites/external_table_p0/es/test_es_catalog_http_open_api.groovy index 485cdd061869d7..9eafacacbb11a2 100644 --- a/regression-test/suites/external_table_p0/es/test_es_catalog_http_open_api.groovy +++ b/regression-test/suites/external_table_p0/es/test_es_catalog_http_open_api.groovy @@ -24,9 +24,13 @@ suite("test_es_catalog_http_open_api", "p0,external,es,external_docker,external_ String es_7_port = context.config.otherConfigs.get("es_7_port") String es_8_port = context.config.otherConfigs.get("es_8_port") - // test old create-catalog syntax for compatibility + sql """drop catalog if exists test_es_catalog_http_open_api_es5;""" + sql """drop catalog if exists test_es_catalog_http_open_api_es6;""" + sql """drop catalog if exists test_es_catalog_http_open_api_es7;""" + sql """drop catalog if exists test_es_catalog_http_open_api_es8;""" + sql """ - create catalog if not exists test_es_query_es5 + create catalog if not exists test_es_catalog_http_open_api_es5 properties ( "type"="es", "elasticsearch.hosts"="http://${externalEnvIp}:$es_5_port", @@ -35,7 +39,7 @@ suite("test_es_catalog_http_open_api", "p0,external,es,external_docker,external_ ); """ sql """ - create catalog if not exists test_es_query_es6 + create catalog if not exists test_es_catalog_http_open_api_es6 properties ( "type"="es", "elasticsearch.hosts"="http://${externalEnvIp}:$es_6_port", @@ -45,7 +49,7 @@ suite("test_es_catalog_http_open_api", "p0,external,es,external_docker,external_ """ // test new create catalog syntax - sql """create catalog if not exists test_es_query_es7 properties( + sql """create catalog if not exists test_es_catalog_http_open_api_es7 properties( "type"="es", "hosts"="http://${externalEnvIp}:$es_7_port", "nodes_discovery"="false", @@ -53,7 +57,7 @@ suite("test_es_catalog_http_open_api", "p0,external,es,external_docker,external_ ); """ - sql """create catalog if not exists test_es_query_es8 properties( + sql """create catalog if not exists test_es_catalog_http_open_api_es8 properties( "type"="es", "hosts"="http://${externalEnvIp}:$es_8_port", "nodes_discovery"="false", @@ -64,7 +68,7 @@ suite("test_es_catalog_http_open_api", "p0,external,es,external_docker,external_ List feHosts = getFrontendIpHttpPort() // for each catalog 5..8, send a request for (int i = 5; i <= 8; i++) { - String catalog = String.format("test_es_query_es%s", i) + String catalog = String.format("test_es_catalog_http_open_api_es%s", i) def (code, out, err) = curl("GET", String.format("http://%s/rest/v2/api/es_catalog/get_mapping?catalog=%s&table=test1", feHosts[0], catalog)) logger.info("Get mapping response: code=" + code + ", out=" + out + ", err=" + err) assertTrue(code == 0) diff --git a/regression-test/suites/external_table_p0/hive/test_hive_statistics_all_type_p0.groovy b/regression-test/suites/external_table_p0/hive/test_hive_statistics_all_type_p0.groovy index a37a55351163e2..6ce76af588f3d1 100644 --- a/regression-test/suites/external_table_p0/hive/test_hive_statistics_all_type_p0.groovy +++ b/regression-test/suites/external_table_p0/hive/test_hive_statistics_all_type_p0.groovy @@ -29,10 +29,35 @@ suite("test_hive_statistics_all_type_p0", "all_types,p0,external,hive,external_d 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}' );""" sql """use `${catalog_name}`.`default`""" - sql """analyze table orc_all_types with sync""" + sql """analyze table orc_all_types with sync with sample rows 4000000""" def result = sql """show column stats orc_all_types;""" assertEquals(16, result.size()) + result = sql """show column stats orc_all_types (int_col);""" + assertEquals("int_col", result[0][0]) + assertEquals("3600.0", result[0][2]) + assertEquals("3240.0", result[0][3]) + assertEquals("361.0", result[0][4]) + assertEquals("14400.0", result[0][5]) + + result = sql """show column stats orc_all_types (string_col);""" + assertEquals("string_col", result[0][0]) + assertEquals("3600.0", result[0][2]) + assertEquals("3254.0", result[0][3]) + assertEquals("347.0", result[0][4]) + assertEquals("453634.0", result[0][5]) + + result = sql """show column stats orc_all_types (varchar_col);""" + assertEquals("varchar_col", result[0][0]) + assertEquals("3600.0", result[0][2]) + assertEquals("6.0", result[0][3]) + assertEquals("0.0", result[0][4]) + assertEquals("35950.0", result[0][5]) + + sql """drop stats orc_all_types""" + sql """analyze table orc_all_types with sync""" + result = sql """show column stats orc_all_types;""" + assertEquals(16, result.size()) result = sql """show column stats orc_all_types (int_col);""" assertEquals("int_col", result[0][0]) assertEquals("3600.0", result[0][2]) diff --git a/regression-test/suites/external_table_p0/tvf/test_hdfs_tvf.groovy b/regression-test/suites/external_table_p0/tvf/test_hdfs_tvf.groovy index 2c2754739a528d..209f870b09747a 100644 --- a/regression-test/suites/external_table_p0/tvf/test_hdfs_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/test_hdfs_tvf.groovy @@ -277,6 +277,32 @@ suite("test_hdfs_tvf","external,hive,tvf,external_docker") { "column_separator" = ",", "format" = "${format}"); """ + + // test create view from tvf and alter view from tvf + uri = "${defaultFS}" + "/user/doris/preinstalled_data/csv_format_test/all_types.csv" + format = "csv" + sql """ DROP VIEW IF EXISTS test_hdfs_tvf_create_view;""" + sql """ + create view test_hdfs_tvf_create_view as + select * from HDFS( + "uri" = "${uri}", + "hadoop.username" = "${hdfsUserName}", + "column_separator" = ",", + "format" = "${format}") order by c1; + """ + + order_qt_create_view """ select * from test_hdfs_tvf_create_view order by c1 limit 20; """ + + sql """ + alter view test_hdfs_tvf_create_view as + select c1 from HDFS( + "uri" = "${uri}", + "hadoop.username" = "${hdfsUserName}", + "column_separator" = ",", + "format" = "${format}") order by c1; + """ + + order_qt_alter_view """ select * from test_hdfs_tvf_create_view order by c1 limit 20; """ } finally { } } diff --git a/regression-test/suites/inverted_index_p0/test_count_on_index.groovy b/regression-test/suites/inverted_index_p0/test_count_on_index.groovy index 8fb98221ef542d..490e998ed58cef 100644 --- a/regression-test/suites/inverted_index_p0/test_count_on_index.groovy +++ b/regression-test/suites/inverted_index_p0/test_count_on_index.groovy @@ -93,7 +93,7 @@ suite("test_count_on_index_httplogs", "p0") { """ } - def load_httplogs_data = {table_name, label, read_flag, format_flag, file_name, ignore_failure=false, + def stream_load_data = {table_name, label, read_flag, format_flag, file_name, ignore_failure=false, expected_succ_rows = -1, load_to_single_tablet = 'true' -> // load the json data @@ -137,8 +137,8 @@ suite("test_count_on_index_httplogs", "p0") { create_httplogs_dup_table.call(testTable_dup) create_httplogs_unique_table.call(testTable_unique) - load_httplogs_data.call(testTable_dup, 'test_httplogs_load_count_on_index', 'true', 'json', 'documents-1000.json') - load_httplogs_data.call(testTable_unique, 'test_httplogs_load_count_on_index', 'true', 'json', 'documents-1000.json') + stream_load_data.call(testTable_dup, 'test_httplogs_load_count_on_index', 'true', 'json', 'documents-1000.json') + stream_load_data.call(testTable_unique, 'test_httplogs_load_count_on_index', 'true', 'json', 'documents-1000.json') sql "sync" sql """set experimental_enable_nereids_planner=true;""" @@ -272,6 +272,41 @@ suite("test_count_on_index_httplogs", "p0") { // case4: test compound query when inverted_index_query disable qt_sql "SELECT COUNT() from ${testTable_dup} where request = 'images' or (size = 0 and status > 400)" qt_sql "SELECT /*+SET_VAR(enable_inverted_index_query=false) */ COUNT() from ${testTable_dup} where request = 'images' or (size = 0 and status > 400)" + + // case5: test complex count to testify bad case + def tableName5 = 'test_count_on_index_bad_case' + sql "DROP TABLE IF EXISTS ${tableName5}" + sql """ + CREATE TABLE `${tableName5}` ( + `a` DATE NOT NULL COMMENT '', + `b` VARCHAR(4096) NULL COMMENT '', + `c` VARCHAR(4096) NULL COMMENT '', + `d` VARCHAR(4096) NULL COMMENT '', + `e` VARCHAR(4096) NULL COMMENT '', + INDEX idx_a(`a`) USING INVERTED COMMENT '', + INDEX idx_e(`e`) USING INVERTED COMMENT '' + ) ENGINE=OLAP + UNIQUE KEY(`a`, `b`) + COMMENT '' + DISTRIBUTED BY HASH(`a`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + stream_load_data.call(tableName5, 'test_count_on_index_bad_case', 'true', 'json', 'count-on-index.json') + def bad_sql = """ + SELECT + COUNT(CASE WHEN c IN ('c1', 'c2', 'c3') AND d = 'd1' THEN b END) AS num1, + COUNT(CASE WHEN e = 'e1' AND c IN ('c1', 'c2', 'c3') THEN b END) AS num2 + FROM ${tableName5} + WHERE a = '2024-07-26' + AND e = 'e1'; + """ + explain { + sql("${bad_sql}") + contains "pushAggOp=NONE" + } + qt_sql_bad "${bad_sql}" } finally { //try_sql("DROP TABLE IF EXISTS ${testTable}") } diff --git a/regression-test/suites/nereids_syntax_p0/window_function.groovy b/regression-test/suites/nereids_syntax_p0/window_function.groovy index 004f47d6f1ce2c..e4c4b2a2aeb891 100644 --- a/regression-test/suites/nereids_syntax_p0/window_function.groovy +++ b/regression-test/suites/nereids_syntax_p0/window_function.groovy @@ -195,4 +195,26 @@ suite("window_function") { qt_select_lead "SELECT c3,c2,c1,lead(c1, 0, 111) over(partition by c3 order by c2,c1) FROM window_test order by c3;" qt_select_lag "SELECT c3,c2,c1,lag(c1, 0, 222) over(partition by c3 order by c2,c1) FROM window_test order by c3;" + + sql "drop table if exists test_normalize_window" + sql """CREATE TABLE test_normalize_window ( + `xwho` varchar(50) NULL COMMENT 'xwho', + `xwhen` datetime COMMENT 'xwhen', + `xwhat` int NULL COMMENT 'xwhat' + ) + DUPLICATE KEY(xwho) + DISTRIBUTED BY HASH(xwho) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + );""" + + sql """INSERT into test_normalize_window (xwho, xwhen, xwhat) values ('1', '2022-03-12 10:41:00', 1), + ('1', '2022-03-12 13:28:02', 2), + ('1', '2022-03-12 16:15:01', 3), + ('1', '2022-03-12 19:05:04', 4);""" + + test { + sql "select group_concat(xwho order by xwhat) over(partition by xwhen) from test_normalize_window;" + exception "order by is not supported" + } } diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf78.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf78.groovy index 4d0dcc1fd02897..ff672d6ce35be8 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf78.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf78.groovy @@ -108,5 +108,5 @@ limit 100; //File file = new File(outFile) //file.write(getRuntimeFilters(plan)) - assertEquals("RF2[ss_sold_date_sk->[d_date_sk],RF1[ws_sold_date_sk->[d_date_sk],RF0[cs_sold_date_sk->[d_date_sk]", getRuntimeFilters(plan)) + assertEquals("RF2[d_date_sk->[ss_sold_date_sk],RF1[d_date_sk->[ws_sold_date_sk],RF0[d_date_sk->[cs_sold_date_sk]", getRuntimeFilters(plan)) } diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf85.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf85.groovy index 91ab923b212bf2..d95634b54d1fde 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf85.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf85.groovy @@ -134,5 +134,5 @@ limit 100; //File file = new File(outFile) //file.write(getRuntimeFilters(plan)) - assertEquals("RF9[wr_reason_sk->[r_reason_sk],RF8[ws_web_page_sk->[wp_web_page_sk],RF5[cd_marital_status->[cd_marital_status],RF6[cd_education_status->[cd_education_status],RF7[wr_returning_cdemo_sk->[cd_demo_sk],RF4[wr_refunded_cdemo_sk->[cd_demo_sk],RF3[wr_refunded_addr_sk->[ca_address_sk],RF1[ws_item_sk->[wr_item_sk],RF2[ws_order_number->[wr_order_number],RF0[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) + assertEquals("RF9[wr_reason_sk->[r_reason_sk],RF8[ws_web_page_sk->[wp_web_page_sk],RF5[cd_marital_status->[cd_marital_status],RF6[cd_education_status->[cd_education_status],RF7[wr_returning_cdemo_sk->[cd_demo_sk],RF4[wr_refunded_addr_sk->[ca_address_sk],RF3[wr_refunded_cdemo_sk->[cd_demo_sk],RF1[ws_item_sk->[wr_item_sk],RF2[ws_order_number->[wr_order_number],RF0[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) } diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf95.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf95.groovy index 490dc720f8c070..d69a6c3eace471 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf95.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf95.groovy @@ -82,5 +82,5 @@ limit 100; //File file = new File(outFile) //file.write(getRuntimeFilters(plan)) - assertEquals("RF3[ws_order_number->[ws_order_number],RF5[wr_order_number->[ws_order_number, ws_order_number],RF4[ws_order_number->[ws_order_number, ws_order_number],RF2[web_site_sk->[ws_web_site_sk],RF1[d_date_sk->[ws_ship_date_sk],RF0[ca_address_sk->[ws_ship_addr_sk]", getRuntimeFilters(plan)) + assertEquals("RF3[ws_order_number->[ws_order_number],RF4[wr_order_number->[ws_order_number, ws_order_number],RF5[ws_order_number->[ws_order_number, ws_order_number],RF2[web_site_sk->[ws_web_site_sk],RF1[d_date_sk->[ws_ship_date_sk],RF0[ca_address_sk->[ws_ship_addr_sk]", getRuntimeFilters(plan)) } diff --git a/regression-test/suites/statistics/analyze_stats.groovy b/regression-test/suites/statistics/analyze_stats.groovy index f0e31e4d7c2b47..993e6f531da8eb 100644 --- a/regression-test/suites/statistics/analyze_stats.groovy +++ b/regression-test/suites/statistics/analyze_stats.groovy @@ -2701,7 +2701,7 @@ PARTITION `p599` VALUES IN (599) assertEquals("true", alter_result[0][7]) sql """drop stats alter_test""" alter_result = sql """show table stats alter_test""" - assertEquals("false", alter_result[0][7]) + assertEquals("", alter_result[0][7]) sql """alter table alter_test modify column id set stats ('row_count'='100', 'ndv'='0', 'num_nulls'='0.0', 'data_size'='2.69975443E8', 'min_value'='1', 'max_value'='2');""" alter_result = sql """show column stats alter_test(id)""" assertEquals(1, alter_result.size()) diff --git a/regression-test/suites/statistics/test_analyze_mv.groovy b/regression-test/suites/statistics/test_analyze_mv.groovy index 60284ef8bc40bb..d7cef1b38a862d 100644 --- a/regression-test/suites/statistics/test_analyze_mv.groovy +++ b/regression-test/suites/statistics/test_analyze_mv.groovy @@ -111,6 +111,7 @@ suite("test_analyze_mv") { sql """drop database if exists test_analyze_mv""" sql """create database test_analyze_mv""" sql """use test_analyze_mv""" + sql """set global enable_auto_analyze=false""" sql """CREATE TABLE mvTestDup ( key1 bigint NOT NULL, @@ -126,18 +127,26 @@ suite("test_analyze_mv") { "replication_num" = "1" ) """ - sql """create materialized view mv1 as select key1 from mvTestDup;""" - wait_mv_finish("test_analyze_mv", "mvTestDup") - sql """create materialized view mv2 as select key2 from mvTestDup;""" - wait_mv_finish("test_analyze_mv", "mvTestDup") - sql """create materialized view mv3 as select key1, key2, sum(value1), max(value2), min(value3) from mvTestDup group by key1, key2;""" - wait_mv_finish("test_analyze_mv", "mvTestDup") + def result_row + if (!isCloudMode()) { + // Test row count report and report for nereids + result_row = sql """show index stats mvTestDup mvTestDup""" + assertEquals(1, result_row.size()) + assertEquals("mvTestDup", result_row[0][0]) + assertEquals("mvTestDup", result_row[0][1]) + assertEquals("0", result_row[0][3]) + assertEquals("-1", result_row[0][4]) + } + + createMV("create materialized view mv1 as select key1 from mvTestDup;") + createMV("create materialized view mv2 as select key2 from mvTestDup;") + createMV("create materialized view mv3 as select key1, key2, sum(value1), max(value2), min(value3) from mvTestDup group by key1, key2;") sql """insert into mvTestDup values (1, 2, 3, 4, 5), (1, 2, 3, 4, 5), (10, 20, 30, 40, 50), (10, 20, 30, 40, 50), (100, 200, 300, 400, 500), (1001, 2001, 3001, 4001, 5001);""" sql """analyze table mvTestDup with sync;""" // Test show index row count - def result_row = sql """show index stats mvTestDup mvTestDup""" + result_row = sql """show index stats mvTestDup mvTestDup""" assertEquals(1, result_row.size()) assertEquals("mvTestDup", result_row[0][0]) assertEquals("mvTestDup", result_row[0][1]) @@ -470,6 +479,35 @@ suite("test_analyze_mv") { logger.info(e.getMessage()); return; } + + if (!isCloudMode()) { + // Test row count report and report for nereids + result_row = sql """show index stats mvTestDup mvTestDup""" + assertEquals(1, result_row.size()) + assertEquals("mvTestDup", result_row[0][0]) + assertEquals("mvTestDup", result_row[0][1]) + assertEquals("6", result_row[0][3]) + assertEquals("6", result_row[0][4]) + result_row = sql """show index stats mvTestDup mv1""" + assertEquals(1, result_row.size()) + assertEquals("mvTestDup", result_row[0][0]) + assertEquals("mv1", result_row[0][1]) + assertEquals("6", result_row[0][3]) + assertEquals("6", result_row[0][4]) + result_row = sql """show index stats mvTestDup mv2""" + assertEquals(1, result_row.size()) + assertEquals("mvTestDup", result_row[0][0]) + assertEquals("mv2", result_row[0][1]) + assertEquals("6", result_row[0][3]) + assertEquals("6", result_row[0][4]) + result_row = sql """show index stats mvTestDup mv3""" + assertEquals(1, result_row.size()) + assertEquals("mvTestDup", result_row[0][0]) + assertEquals("mv3", result_row[0][1]) + assertEquals("4", result_row[0][3]) + assertEquals("4", result_row[0][4]) + } + sql """analyze table mvTestDup with sample rows 4000000""" wait_analyze_finish("mvTestDup") result_sample = sql """SHOW ANALYZE mvTestDup;""" @@ -617,6 +655,38 @@ suite("test_analyze_mv") { verifyTaskStatus(result_sample, "mva_MIN__`value3`", "mv3") verifyTaskStatus(result_sample, "mva_SUM__CAST(`value1` AS bigint)", "mv3") + if (!isCloudMode()) { + // Test row count report and report for nereids + sql """truncate table mvTestDup""" + result_row = sql """show index stats mvTestDup mv3""" + assertEquals(1, result_row.size()) + assertEquals("mvTestDup", result_row[0][0]) + assertEquals("mv3", result_row[0][1]) + assertEquals("0", result_row[0][3]) + assertEquals("-1", result_row[0][4]) + + for (int i = 0; i < 120; i++) { + result_row = sql """show index stats mvTestDup mv3""" + logger.info("mv3 stats: " + result_row) + if (result_row[0][4] == "0") { + break; + } + Thread.sleep(5000) + } + result_row = sql """show index stats mvTestDup mv3""" + assertEquals(1, result_row.size()) + assertEquals("mvTestDup", result_row[0][0]) + assertEquals("mv3", result_row[0][1]) + assertEquals("0", result_row[0][3]) + assertEquals("0", result_row[0][4]) + sql """insert into mvTestDup values (1, 2, 3, 4, 5), (1, 2, 3, 4, 5), (10, 20, 30, 40, 50), (10, 20, 30, 40, 50), (100, 200, 300, 400, 500), (1001, 2001, 3001, 4001, 5001);""" + result_row = sql """show index stats mvTestDup mv3""" + assertEquals(1, result_row.size()) + assertEquals("mvTestDup", result_row[0][0]) + assertEquals("mv3", result_row[0][1]) + assertEquals("-1", result_row[0][4]) + } + // Test alter column stats sql """drop stats mvTestDup""" sql """alter table mvTestDup modify column key1 set stats ('ndv'='1', 'num_nulls'='1', 'min_value'='10', 'max_value'='40', 'row_count'='50');""" diff --git a/regression-test/suites/statistics/test_drop_stats_and_truncate.groovy b/regression-test/suites/statistics/test_drop_stats_and_truncate.groovy new file mode 100644 index 00000000000000..abeb754724b945 --- /dev/null +++ b/regression-test/suites/statistics/test_drop_stats_and_truncate.groovy @@ -0,0 +1,227 @@ +// 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_drop_stats_and_truncate") { + + sql """drop database if exists test_drop_stats_and_truncate""" + sql """create database test_drop_stats_and_truncate""" + sql """use test_drop_stats_and_truncate""" + sql """set global enable_auto_analyze=false""" + + sql """CREATE TABLE non_part ( + r_regionkey int NOT NULL, + r_name VARCHAR(25) NOT NULL, + r_comment VARCHAR(152) + )ENGINE=OLAP + DUPLICATE KEY(`r_regionkey`) + COMMENT "OLAP" + DISTRIBUTED BY HASH(`r_regionkey`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """CREATE TABLE `part` ( + `id` INT NULL, + `colint` INT NULL, + `coltinyint` int NULL, + `colsmallint` smallINT NULL, + `colbigint` bigINT NULL, + `collargeint` largeINT NULL, + `colfloat` float NULL, + `coldouble` double NULL, + `coldecimal` decimal(27, 9) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + PARTITION BY RANGE(`id`) + ( + PARTITION p1 VALUES [("-2147483648"), ("10000")), + PARTITION p2 VALUES [("10000"), ("20000")), + PARTITION p3 VALUES [("20000"), ("30000")) + ) + DISTRIBUTED BY HASH(`id`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ) + """ + sql """insert into non_part values (1, "1", "1");""" + sql """analyze table non_part with sync""" + + def result = sql """show column cached stats non_part""" + assertEquals(3, result.size()) + result = sql """show column stats non_part""" + assertEquals(3, result.size()) + result = sql """show table stats non_part""" + def all_columns = result[0][4] + String[] columns = all_columns.split(","); + assertEquals(3, columns.size()) + + sql """drop stats non_part(r_comment)""" + result = sql """show column cached stats non_part""" + assertEquals(2, result.size()) + result = sql """show column stats non_part""" + assertEquals(2, result.size()) + result = sql """show table stats non_part""" + all_columns = result[0][4] + columns = all_columns.split(","); + assertEquals(2, columns.size()) + + sql """drop stats non_part""" + result = sql """show column cached stats non_part""" + assertEquals(0, result.size()) + result = sql """show column stats non_part""" + assertEquals(0, result.size()) + result = sql """show table stats non_part""" + all_columns = result[0][4] + assertEquals("", all_columns) + + sql """analyze table non_part with sync""" + result = sql """show column cached stats non_part""" + assertEquals(3, result.size()) + result = sql """show column stats non_part""" + assertEquals(3, result.size()) + result = sql """show table stats non_part""" + all_columns = result[0][4] + columns = all_columns.split(","); + assertEquals(3, columns.size()) + + sql """truncate table non_part""" + result = sql """show column stats non_part""" + assertEquals(0, result.size()) + result = sql """show table stats non_part""" + all_columns = result[0][4] + assertEquals("", all_columns) + + sql """Insert into part values (1, 1, 1, 1, 1, 1, 1.1, 1.1, 1.1), (2, 2, 2, 2, 2, 2, 2.2, 2.2, 2.2), (3, 3, 3, 3, 3, 3, 3.3, 3.3, 3.3),(4, 4, 4, 4, 4, 4, 4.4, 4.4, 4.4),(5, 5, 5, 5, 5, 5, 5.5, 5.5, 5.5),(6, 6, 6, 6, 6, 6, 6.6, 6.6, 6.6),(10001, 10001, 10001, 10001, 10001, 10001, 10001.10001, 10001.10001, 10001.10001),(10002, 10002, 10002, 10002, 10002, 10002, 10002.10002, 10002.10002, 10002.10002),(10003, 10003, 10003, 10003, 10003, 10003, 10003.10003, 10003.10003, 10003.10003),(10004, 10004, 10004, 10004, 10004, 10004, 10004.10004, 10004.10004, 10004.10004),(10005, 10005, 10005, 10005, 10005, 10005, 10005.10005, 10005.10005, 10005.10005),(10006, 10006, 10006, 10006, 10006, 10006, 10006.10006, 10006.10006, 10006.10006),(20001, 20001, 20001, 20001, 20001, 20001, 20001.20001, 20001.20001, 20001.20001),(20002, 20002, 20002, 20002, 20002, 20002, 20002.20002, 20002.20002, 20002.20002),(20003, 20003, 20003, 20003, 20003, 20003, 20003.20003, 20003.20003, 20003.20003),(20004, 20004, 20004, 20004, 20004, 20004, 20004.20004, 20004.20004, 20004.20004),(20005, 20005, 20005, 20005, 20005, 20005, 20005.20005, 20005.20005, 20005.20005),(20006, 20006, 20006, 20006, 20006, 20006, 20006.20006, 20006.20006, 20006.20006)""" + sql """analyze table part with sync""" + result = sql """show column cached stats part""" + assertEquals(9, result.size()) + result = sql """show column stats part""" + assertEquals(9, result.size()) + result = sql """show table stats part""" + all_columns = result[0][4] + columns = all_columns.split(","); + assertEquals(9, columns.size()) + + sql """drop stats part(colint)""" + result = sql """show column cached stats part""" + assertEquals(8, result.size()) + result = sql """show column stats part""" + assertEquals(8, result.size()) + result = sql """show table stats part""" + all_columns = result[0][4] + columns = all_columns.split(","); + assertEquals(8, columns.size()) + + sql """drop stats part""" + result = sql """show column cached stats part""" + assertEquals(0, result.size()) + result = sql """show column stats part""" + assertEquals(0, result.size()) + result = sql """show table stats part""" + all_columns = result[0][4] + assertEquals("", all_columns) + + sql """analyze table part with sync""" + result = sql """show column cached stats part""" + assertEquals(9, result.size()) + result = sql """show column stats part""" + assertEquals(9, result.size()) + result = sql """show table stats part""" + all_columns = result[0][4] + columns = all_columns.split(","); + assertEquals(9, columns.size()) + + sql """truncate table part""" + result = sql """show column stats part""" + assertEquals(0, result.size()) + result = sql """show table stats part""" + all_columns = result[0][4] + assertEquals("", all_columns) + + sql """Insert into part values (1, 1, 1, 1, 1, 1, 1.1, 1.1, 1.1), (2, 2, 2, 2, 2, 2, 2.2, 2.2, 2.2), (3, 3, 3, 3, 3, 3, 3.3, 3.3, 3.3),(4, 4, 4, 4, 4, 4, 4.4, 4.4, 4.4),(5, 5, 5, 5, 5, 5, 5.5, 5.5, 5.5),(6, 6, 6, 6, 6, 6, 6.6, 6.6, 6.6),(10001, 10001, 10001, 10001, 10001, 10001, 10001.10001, 10001.10001, 10001.10001),(10002, 10002, 10002, 10002, 10002, 10002, 10002.10002, 10002.10002, 10002.10002),(10003, 10003, 10003, 10003, 10003, 10003, 10003.10003, 10003.10003, 10003.10003),(10004, 10004, 10004, 10004, 10004, 10004, 10004.10004, 10004.10004, 10004.10004),(10005, 10005, 10005, 10005, 10005, 10005, 10005.10005, 10005.10005, 10005.10005),(10006, 10006, 10006, 10006, 10006, 10006, 10006.10006, 10006.10006, 10006.10006),(20001, 20001, 20001, 20001, 20001, 20001, 20001.20001, 20001.20001, 20001.20001),(20002, 20002, 20002, 20002, 20002, 20002, 20002.20002, 20002.20002, 20002.20002),(20003, 20003, 20003, 20003, 20003, 20003, 20003.20003, 20003.20003, 20003.20003),(20004, 20004, 20004, 20004, 20004, 20004, 20004.20004, 20004.20004, 20004.20004),(20005, 20005, 20005, 20005, 20005, 20005, 20005.20005, 20005.20005, 20005.20005),(20006, 20006, 20006, 20006, 20006, 20006, 20006.20006, 20006.20006, 20006.20006)""" + sql """analyze table part with sync""" + result = sql """show column cached stats part""" + assertEquals(9, result.size()) + result = sql """show column stats part""" + assertEquals(9, result.size()) + result = sql """show table stats part""" + all_columns = result[0][4] + columns = all_columns.split(","); + assertEquals(9, columns.size()) + + sql """truncate table part partition(p1)""" + result = sql """show column cached stats part""" + assertEquals(9, result.size()) + result = sql """show column stats part""" + assertEquals(9, result.size()) + result = sql """show table stats part""" + all_columns = result[0][4] + columns = all_columns.split(","); + assertEquals(9, columns.size()) + + sql """drop table part""" + sql """CREATE TABLE `part` ( + `id` INT NULL, + `colint` INT NULL, + `coltinyint` tinyint NULL, + `colsmallint` smallINT NULL, + `colbigint` bigINT NULL, + `collargeint` largeINT NULL, + `colfloat` float NULL, + `coldouble` double NULL, + `coldecimal` decimal(27, 9) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + PARTITION BY RANGE(`id`) + ( + PARTITION p1 VALUES [("-2147483648"), ("10000")), + PARTITION p2 VALUES [("10000"), ("20000")), + PARTITION p3 VALUES [("20000"), ("30000")) + ) + DISTRIBUTED BY HASH(`id`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ) + """ + sql """analyze table part with sync""" + sql """Insert into part values (1, 1, 1, 1, 1, 1, 1.1, 1.1, 1.1)""" + result = sql """show table stats part""" + assertEquals("true", result[0][6]) + sql """truncate table part partition(p1)""" + result = sql """show table stats part""" + assertEquals("true", result[0][6]) + sql """analyze table part with sample rows 100 with sync""" + result = sql """show table stats part""" + if (result[0][6].equals("true")) { + result = """show index stats part part""" + logger.info("Report not ready. index stats: " + result) + sql """analyze table part with sample rows 100 with sync""" + result = sql """show table stats part""" + } + if (result[0][6].equals("true")) { + result = """show index stats part part""" + logger.info("Report not ready. index stats: " + result) + sql """analyze table part with sample rows 100 with sync""" + result = sql """show table stats part""" + } + assertEquals("false", result[0][6]) + + sql """drop database if exists test_drop_stats_and_truncate""" +} + diff --git a/regression-test/suites/statistics/test_select_mv.groovy b/regression-test/suites/statistics/test_select_mv.groovy index 8c3417cb4b9ca5..b6a8697fa7f61c 100644 --- a/regression-test/suites/statistics/test_select_mv.groovy +++ b/regression-test/suites/statistics/test_select_mv.groovy @@ -67,9 +67,7 @@ suite("test_select_mv") { ); """ - sql """ - create materialized view dup1 as select key2, sum(value) from test_dup group by key2; - """ + createMV("create materialized view dup1 as select key2, sum(value) from test_dup group by key2;") sql """CREATE TABLE test_agg ( key1 int NOT NULL, @@ -83,10 +81,7 @@ suite("test_select_mv") { ); """ - sql """ - create materialized view agg1 as select key2, sum(value) from test_agg group by key2; - """ - wait_mv_finish("test_select_mv") + createMV("create materialized view agg1 as select key2, sum(value) from test_agg group by key2;") sql """insert into test_dup values (1, 1, 1), (2, 2, 2)""" sql """insert into test_dup values (1, 1, 1), (2, 2, 2)""" diff --git a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_only_keys.groovy b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_only_keys.groovy index 1f56643b6e8a93..29f1257f9cb4ad 100644 --- a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_only_keys.groovy +++ b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_only_keys.groovy @@ -26,7 +26,7 @@ suite("test_partial_update_only_keys", "p0") { connect(user = context.config.jdbcUser, password = context.config.jdbcPassword, url = context.config.jdbcUrl) { sql "use ${db};" - def tableName = "test_primary_key_partial_update" + def tableName = "test_partial_update_only_keys" sql """ DROP TABLE IF EXISTS ${tableName} force""" sql """ CREATE TABLE ${tableName} ( `k` BIGINT NOT NULL, diff --git a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_parallel.groovy b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_parallel.groovy index 9a96c3358e17f7..99300f2133fb9b 100644 --- a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_parallel.groovy +++ b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_parallel.groovy @@ -18,7 +18,7 @@ suite("test_primary_key_partial_update_parallel", "p0") { // case 1: concurrent partial update - def tableName = "test_primary_key_partial_update" + def tableName = "test_primary_key_partial_update_parallel" sql """ DROP TABLE IF EXISTS ${tableName} """ sql """ CREATE TABLE ${tableName} ( diff --git a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_seq_map_col.groovy b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_seq_map_col.groovy new file mode 100644 index 00000000000000..550817d6a3a893 --- /dev/null +++ b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_seq_map_col.groovy @@ -0,0 +1,146 @@ +// 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_partial_update_seq_map_col", "p0") { + for (def use_nereids : [true, false]) { + for (def use_row_store : [false, true]) { + logger.info("current params: use_nereids: ${use_nereids}, use_row_store: ${use_row_store}") + if (use_nereids) { + sql """ set enable_nereids_planner=true; """ + sql """ set enable_fallback_to_original_planner=false; """ + } else { + sql """ set enable_nereids_planner = false; """ + } + sql "set enable_insert_strict=false;" + sql "set enable_unique_key_partial_update=true;" + sql "sync;" + + def tableName = "test_partial_update_seq_map_col1" + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( + `k` BIGINT NOT NULL, + `c1` int, + `c2` datetime(6) null default current_timestamp(6), + c3 int, + c4 int, + c5 int, + c6 int + ) UNIQUE KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "enable_unique_key_merge_on_write" = "true", + "function_column.sequence_col" = "c2", + "store_row_column" = "${use_row_store}"); """ + sql "insert into ${tableName}(k,c1) values(1,1);" + sql "insert into ${tableName}(k,c1) values(2,2);" + sql "insert into ${tableName}(k,c1) values(3,3);" + sql "insert into ${tableName}(k,c1) values(4,4);" + order_qt_sql1 "select k,c1,c3 from ${tableName} where c2=__DORIS_SEQUENCE_COL__;" + // update column which is not sequence map col + sql "update ${tableName} set c3=20 where c1<=2;" + order_qt_sql1 "select k,c1,c3 from ${tableName} where c2=__DORIS_SEQUENCE_COL__;" + // update sequence map col + sql "update ${tableName} set c2='2099-09-10 12:00:00.977174' where k>2;" + order_qt_sql1 "select k,c1,c3 from ${tableName} where c2=__DORIS_SEQUENCE_COL__;" + order_qt_sql1 "select k,c1,c2,c3,__DORIS_SEQUENCE_COL__ from ${tableName} where c1>2;" + + tableName = "test_partial_update_seq_map_col2" + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( + `k` BIGINT NOT NULL, + `c1` int, + `c2` datetime not null default current_timestamp, + ) UNIQUE KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "enable_unique_key_merge_on_write" = "true", + "function_column.sequence_col" = "c2", + "store_row_column" = "${use_row_store}"); """ + sql "insert into ${tableName}(k,c1) values(1,1);" + sql "insert into ${tableName}(k,c1) values(2,2);" + sql "insert into ${tableName}(k,c1) values(3,3);" + sql "insert into ${tableName}(k,c1) values(4,4);" + order_qt_sql2 "select k,c1 from ${tableName} where c2=__DORIS_SEQUENCE_COL__;" + + + tableName = "test_partial_update_seq_map_col3" + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( + `k` BIGINT NOT NULL, + `c1` int, + `c2` int not null default "999", + ) UNIQUE KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "enable_unique_key_merge_on_write" = "true", + "function_column.sequence_col" = "c2", + "store_row_column" = "${use_row_store}"); """ + sql "insert into ${tableName}(k,c1) values(1,1);" + sql "insert into ${tableName}(k,c1) values(2,2);" + sql "insert into ${tableName}(k,c1) values(3,3);" + sql "insert into ${tableName}(k,c1) values(4,4);" + order_qt_sql3 "select k,c1,c2,__DORIS_SEQUENCE_COL__ from ${tableName};" + sql "insert into ${tableName}(k,c1,c2) values(1,99,8888);" + sql "insert into ${tableName}(k,c1,c2) values(2,99,8888);" + sql "insert into ${tableName}(k,c1,c2) values(4,99,77);" + sql "insert into ${tableName}(k,c1,c2) values(5,99,8888);" + order_qt_sql3 "select k,c1,c2,__DORIS_SEQUENCE_COL__ from ${tableName}" + + + tableName = "test_partial_update_seq_map_col4" + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( + `k` BIGINT NOT NULL, + `c1` int, + `c2` int null, + ) UNIQUE KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "enable_unique_key_merge_on_write" = "true", + "function_column.sequence_col" = "c2", + "store_row_column" = "${use_row_store}"); """ + sql "insert into ${tableName}(k,c1) values(1,1);" + sql "insert into ${tableName}(k,c1) values(2,2);" + sql "insert into ${tableName}(k,c1) values(3,3);" + sql "insert into ${tableName}(k,c1) values(4,4);" + order_qt_sql4 "select k,c1,c2,__DORIS_SEQUENCE_COL__ from ${tableName};" + + + tableName = "test_partial_update_seq_map_col5" + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( + `k` BIGINT NOT NULL, + `c1` int, + `c2` int not null + ) UNIQUE KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "enable_unique_key_merge_on_write" = "true", + "function_column.sequence_col" = "c2", + "store_row_column" = "${use_row_store}"); """ + test { + sql "insert into ${tableName}(k,c1) values(1,1);" + exception "the unmentioned column `c2` should have default value or be nullable for newly inserted rows in non-strict mode partial update" + } + } + } +}