diff --git a/.github/workflows/code-checks.yml b/.github/workflows/code-checks.yml index 2762480a0cb566..4fe4090b516a1e 100644 --- a/.github/workflows/code-checks.yml +++ b/.github/workflows/code-checks.yml @@ -50,7 +50,7 @@ jobs: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} with: sh_checker_comment: true - sh_checker_exclude: .git .github ^docker ^thirdparty/src ^thirdparty/installed ^ui ^docs/node_modules ^tools/clickbench-tools ^extension ^output ^fs_brokers/apache_hdfs_broker/output (^|.*/)Dockerfile$ ^be/src/apache-orc ^be/src/clucene ^pytest + sh_checker_exclude: .git .github ^docker ^thirdparty/src ^thirdparty/installed ^ui ^docs/node_modules ^tools/clickbench-tools ^extension ^output ^fs_brokers/apache_hdfs_broker/output (^|.*/)Dockerfile$ ^be/src/apache-orc ^be/src/clucene ^pytest ^samples preparation: name: "Clang Tidy Preparation" diff --git a/.github/workflows/scope-label.yml b/.github/workflows/scope-label.yml index 77f0e50a44f705..ba88dbbb02f7c0 100644 --- a/.github/workflows/scope-label.yml +++ b/.github/workflows/scope-label.yml @@ -35,7 +35,7 @@ jobs: github.event_name == 'pull_request_target' && (github.event.action == 'opened' || github.event.action == 'synchronize') - uses: actions/labeler@v5.5.0 + uses: actions/labeler@2.2.0 with: repo-token: ${{ secrets.GITHUB_TOKEN }} configuration-path: .github/workflows/labeler/scope-label-conf.yml diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index bc6c3a0f098e7e..8138ea52421102 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -715,7 +715,10 @@ Status CloudMetaMgr::prepare_rowset(const RowsetMeta& rs_meta, RowsetMetaSharedPtr* existed_rs_meta) { VLOG_DEBUG << "prepare rowset, tablet_id: " << rs_meta.tablet_id() << ", rowset_id: " << rs_meta.rowset_id() << " txn_id: " << rs_meta.txn_id(); - + { + Status ret_st; + TEST_INJECTION_POINT_RETURN_WITH_VALUE("CloudMetaMgr::prepare_rowset", ret_st); + } CreateRowsetRequest req; CreateRowsetResponse resp; req.set_cloud_unique_id(config::cloud_unique_id); @@ -741,6 +744,10 @@ Status CloudMetaMgr::commit_rowset(const RowsetMeta& rs_meta, RowsetMetaSharedPtr* existed_rs_meta) { VLOG_DEBUG << "commit rowset, tablet_id: " << rs_meta.tablet_id() << ", rowset_id: " << rs_meta.rowset_id() << " txn_id: " << rs_meta.txn_id(); + { + Status ret_st; + TEST_INJECTION_POINT_RETURN_WITH_VALUE("CloudMetaMgr::commit_rowset", ret_st); + } CreateRowsetRequest req; CreateRowsetResponse resp; req.set_cloud_unique_id(config::cloud_unique_id); @@ -841,6 +848,10 @@ static void send_stats_to_fe_async(const int64_t db_id, const int64_t txn_id, Status CloudMetaMgr::commit_txn(const StreamLoadContext& ctx, bool is_2pc) { VLOG_DEBUG << "commit txn, db_id: " << ctx.db_id << ", txn_id: " << ctx.txn_id << ", label: " << ctx.label << ", is_2pc: " << is_2pc; + { + Status ret_st; + TEST_INJECTION_POINT_RETURN_WITH_VALUE("CloudMetaMgr::commit_txn", ret_st); + } CommitTxnRequest req; CommitTxnResponse res; req.set_cloud_unique_id(config::cloud_unique_id); @@ -860,6 +871,10 @@ Status CloudMetaMgr::commit_txn(const StreamLoadContext& ctx, bool is_2pc) { Status CloudMetaMgr::abort_txn(const StreamLoadContext& ctx) { VLOG_DEBUG << "abort txn, db_id: " << ctx.db_id << ", txn_id: " << ctx.txn_id << ", label: " << ctx.label; + { + Status ret_st; + TEST_INJECTION_POINT_RETURN_WITH_VALUE("CloudMetaMgr::abort_txn", ret_st); + } AbortTxnRequest req; AbortTxnResponse res; req.set_cloud_unique_id(config::cloud_unique_id); @@ -879,6 +894,10 @@ Status CloudMetaMgr::abort_txn(const StreamLoadContext& ctx) { Status CloudMetaMgr::precommit_txn(const StreamLoadContext& ctx) { VLOG_DEBUG << "precommit txn, db_id: " << ctx.db_id << ", txn_id: " << ctx.txn_id << ", label: " << ctx.label; + { + Status ret_st; + TEST_INJECTION_POINT_RETURN_WITH_VALUE("CloudMetaMgr::precommit_txn", ret_st); + } PrecommitTxnRequest req; PrecommitTxnResponse res; req.set_cloud_unique_id(config::cloud_unique_id); diff --git a/be/src/cloud/cloud_stream_load_executor.cpp b/be/src/cloud/cloud_stream_load_executor.cpp index 1b8167c96ebd48..1352b4aac81a5f 100644 --- a/be/src/cloud/cloud_stream_load_executor.cpp +++ b/be/src/cloud/cloud_stream_load_executor.cpp @@ -129,7 +129,7 @@ void CloudStreamLoadExecutor::rollback_txn(StreamLoadContext* ctx) { std::stringstream ss; ss << "db_id=" << ctx->db_id << " txn_id=" << ctx->txn_id << " label=" << ctx->label; std::string op_info = ss.str(); - LOG(INFO) << "rollback stream laod txn " << op_info; + LOG(INFO) << "rollback stream load txn " << op_info; TxnOpParamType topt = ctx->txn_id > 0 ? TxnOpParamType::WITH_TXN_ID : !ctx->label.empty() ? TxnOpParamType::WITH_LABEL : TxnOpParamType::ILLEGAL; diff --git a/be/src/cloud/injection_point_action.cpp b/be/src/cloud/injection_point_action.cpp index be90ee23afddae..1880f14a3b77d1 100644 --- a/be/src/cloud/injection_point_action.cpp +++ b/be/src/cloud/injection_point_action.cpp @@ -124,6 +124,17 @@ void register_suites() { *arg0 = Status::Corruption("test_file_segment_cache_corruption injection error"); }); }); + // curl be_ip:http_port/api/injection_point/apply_suite?name=test_cloud_meta_mgr_commit_txn' + suite_map.emplace("test_cloud_meta_mgr_commit_txn", [] { + auto* sp = SyncPoint::get_instance(); + sp->set_call_back("CloudMetaMgr::commit_txn", [](auto&& args) { + LOG(INFO) << "injection CloudMetaMgr::commit_txn"; + auto* arg0 = try_any_cast_ret(args); + arg0->first = Status::InternalError( + "test_file_segment_cache_corruption injection error"); + arg0->second = true; + }); + }); } void set_sleep(const std::string& point, HttpRequest* req) { @@ -139,7 +150,8 @@ void set_sleep(const std::string& point, HttpRequest* req) { } } auto sp = SyncPoint::get_instance(); - sp->set_call_back(point, [duration](auto&& args) { + sp->set_call_back(point, [point, duration](auto&& args) { + LOG(INFO) << "injection point hit, point=" << point << " sleep milliseconds=" << duration; std::this_thread::sleep_for(std::chrono::milliseconds(duration)); }); HttpChannel::send_reply(req, HttpStatus::OK, "OK"); @@ -147,8 +159,9 @@ void set_sleep(const std::string& point, HttpRequest* req) { void set_return(const std::string& point, HttpRequest* req) { auto sp = SyncPoint::get_instance(); - sp->set_call_back(point, [](auto&& args) { + sp->set_call_back(point, [point](auto&& args) { try { + LOG(INFO) << "injection point hit, point=" << point << " return void"; auto pred = try_any_cast(args.back()); *pred = true; } catch (const std::bad_any_cast&) { @@ -160,8 +173,9 @@ void set_return(const std::string& point, HttpRequest* req) { void set_return_ok(const std::string& point, HttpRequest* req) { auto sp = SyncPoint::get_instance(); - sp->set_call_back(point, [](auto&& args) { + sp->set_call_back(point, [point](auto&& args) { try { + LOG(INFO) << "injection point hit, point=" << point << " return ok"; auto* pair = try_any_cast_ret(args); pair->first = Status::OK(); pair->second = true; @@ -188,8 +202,9 @@ void set_return_error(const std::string& point, HttpRequest* req) { } auto sp = SyncPoint::get_instance(); - sp->set_call_back(point, [code](auto&& args) { + sp->set_call_back(point, [code, point](auto&& args) { try { + LOG(INFO) << "injection point hit, point=" << point << " return error code=" << code; auto* pair = try_any_cast_ret(args); pair->first = Status::Error(code, "injected error"); pair->second = true; @@ -243,7 +258,7 @@ void handle_clear(HttpRequest* req) { HttpChannel::send_reply(req, HttpStatus::OK, "OK"); } -void handle_suite(HttpRequest* req) { +void handle_apply_suite(HttpRequest* req) { auto& suite = req->param("name"); if (suite.empty()) { HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, "empty suite name"); @@ -253,10 +268,11 @@ void handle_suite(HttpRequest* req) { std::call_once(register_suites_once, register_suites); if (auto it = suite_map.find(suite); it != suite_map.end()) { it->second(); // set injection callbacks - HttpChannel::send_reply(req, HttpStatus::OK, "OK"); + HttpChannel::send_reply(req, HttpStatus::OK, "OK apply suite " + suite + "\n"); return; } - HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, "unknown suite: " + suite); + HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, + "unknown suite: " + suite + "\n"); } void handle_enable(HttpRequest* req) { @@ -273,6 +289,37 @@ void handle_disable(HttpRequest* req) { InjectionPointAction::InjectionPointAction() = default; +// +// enable/disable injection point +// ``` +// curl "be_ip:http_port/api/injection_point/enable" +// curl "be_ip:http_port/api/injection_point/disable" +// ``` +// +// clear all injection points +// ``` +// curl "be_ip:http_port/api/injection_point/clear" +// ``` +// +// apply/activate specific suite with registered action, see `register_suites()` for more details +// ``` +// curl "be_ip:http_port/api/injection_point/apply_suite?name=${suite_name}" +// ``` +// +// set predifined action for specific injection point, supported actions are: +// * sleep: for injection point with callback, accepted param is `duration` in milliseconds +// * return: for injection point without return value (return void) +// * return_ok: for injection point with return value, always return Status::OK +// * return_error: for injection point with return value, accepted param is `code`, +// which is an int, valid values can be found in status.h, e.g. -235 or -230, +// if `code` is not present return Status::InternalError +// ``` +// curl "be_ip:http_port/api/injection_point/set?name=${injection_point_name}&behavior=sleep&duration=${x_millsec}" # sleep x millisecs +// curl "be_ip:http_port/api/injection_point/set?name=${injection_point_name}&behavior=return" # return void +// curl "be_ip:http_port/api/injection_point/set?name=${injection_point_name}&behavior=return_ok" # return ok +// curl "be_ip:http_port/api/injection_point/set?name=${injection_point_name}&behavior=return_error" # internal error +// curl "be_ip:http_port/api/injection_point/set?name=${injection_point_name}&behavior=return_error&code=${code}" # -235 +// ``` void InjectionPointAction::handle(HttpRequest* req) { LOG(INFO) << "handle InjectionPointAction " << req->debug_string(); auto& op = req->param("op"); @@ -283,7 +330,7 @@ void InjectionPointAction::handle(HttpRequest* req) { handle_clear(req); return; } else if (op == "apply_suite") { - handle_suite(req); + handle_apply_suite(req); return; } else if (op == "enable") { handle_enable(req); diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 2b3b2ffb09e3ec..67a12fc02f1d70 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -296,7 +296,7 @@ DEFINE_mInt32(default_num_rows_per_column_file_block, "1024"); // pending data policy DEFINE_mInt32(pending_data_expire_time_sec, "1800"); // inc_rowset snapshot rs sweep time interval -DEFINE_mInt32(tablet_rowset_stale_sweep_time_sec, "300"); +DEFINE_mInt32(tablet_rowset_stale_sweep_time_sec, "600"); // tablet stale rowset sweep by threshold size DEFINE_Bool(tablet_rowset_stale_sweep_by_size, "false"); DEFINE_mInt32(tablet_rowset_stale_sweep_threshold_size, "100"); diff --git a/be/src/olap/rowset/segment_v2/segment.cpp b/be/src/olap/rowset/segment_v2/segment.cpp index 64f58e546c22ba..2556597419ef87 100644 --- a/be/src/olap/rowset/segment_v2/segment.cpp +++ b/be/src/olap/rowset/segment_v2/segment.cpp @@ -553,7 +553,7 @@ Status Segment::_create_column_readers(const SegmentFooterPB& footer) { vectorized::PathInData path; path.from_protobuf(spase_column_pb.column_path_info()); // Read from root column, so reader is nullptr - _sparse_column_tree[column.unique_id()].add( + _sparse_column_tree[unique_id].add( path.copy_pop_front(), SubcolumnReader {nullptr, vectorized::DataTypeFactory::instance().create_data_type( @@ -617,9 +617,10 @@ Status Segment::new_column_iterator_with_path(const TabletColumn& tablet_column, const auto* node = tablet_column.has_path_info() ? _sub_column_tree[unique_id].find_exact(relative_path) : nullptr; - const auto* sparse_node = tablet_column.has_path_info() - ? _sparse_column_tree[unique_id].find_exact(relative_path) - : nullptr; + const auto* sparse_node = + tablet_column.has_path_info() && _sparse_column_tree.contains(unique_id) + ? _sparse_column_tree[unique_id].find_exact(relative_path) + : nullptr; // Currently only compaction and checksum need to read flat leaves // They both use tablet_schema_with_merged_max_schema_version as read schema auto type_to_read_flat_leaves = [](ReaderType type) { diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index dc2ec52d12bb1b..f0d9cc6605d767 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -503,35 +503,41 @@ Status SegmentIterator::_get_row_ranges_by_column_conditions() { } RETURN_IF_ERROR(_apply_bitmap_index()); - RETURN_IF_ERROR(_apply_inverted_index()); - RETURN_IF_ERROR(_apply_index_expr()); - size_t input_rows = _row_bitmap.cardinality(); - for (auto it = _common_expr_ctxs_push_down.begin(); it != _common_expr_ctxs_push_down.end();) { - if ((*it)->all_expr_inverted_index_evaluated()) { - const auto* result = - (*it)->get_inverted_index_context()->get_inverted_index_result_for_expr( - (*it)->root().get()); - if (result != nullptr) { - _row_bitmap &= *result->get_data_bitmap(); - auto root = (*it)->root(); - auto iter_find = std::find(_remaining_conjunct_roots.begin(), - _remaining_conjunct_roots.end(), root); - if (iter_find != _remaining_conjunct_roots.end()) { - _remaining_conjunct_roots.erase(iter_find); + { + if (_opts.runtime_state && + _opts.runtime_state->query_options().enable_inverted_index_query) { + SCOPED_RAW_TIMER(&_opts.stats->inverted_index_filter_timer); + size_t input_rows = _row_bitmap.cardinality(); + RETURN_IF_ERROR(_apply_inverted_index()); + RETURN_IF_ERROR(_apply_index_expr()); + for (auto it = _common_expr_ctxs_push_down.begin(); + it != _common_expr_ctxs_push_down.end();) { + if ((*it)->all_expr_inverted_index_evaluated()) { + const auto* result = + (*it)->get_inverted_index_context()->get_inverted_index_result_for_expr( + (*it)->root().get()); + if (result != nullptr) { + _row_bitmap &= *result->get_data_bitmap(); + auto root = (*it)->root(); + auto iter_find = std::find(_remaining_conjunct_roots.begin(), + _remaining_conjunct_roots.end(), root); + if (iter_find != _remaining_conjunct_roots.end()) { + _remaining_conjunct_roots.erase(iter_find); + } + it = _common_expr_ctxs_push_down.erase(it); + } + } else { + ++it; } - it = _common_expr_ctxs_push_down.erase(it); } - } else { - ++it; - } - } - - _opts.stats->rows_inverted_index_filtered += (input_rows - _row_bitmap.cardinality()); - for (auto cid : _schema->column_ids()) { - bool result_true = _check_all_conditions_passed_inverted_index_for_column(cid); + _opts.stats->rows_inverted_index_filtered += (input_rows - _row_bitmap.cardinality()); + for (auto cid : _schema->column_ids()) { + bool result_true = _check_all_conditions_passed_inverted_index_for_column(cid); - if (result_true) { - _need_read_data_indices[cid] = false; + if (result_true) { + _need_read_data_indices[cid] = false; + } + } } } if (!_row_bitmap.isEmpty() && @@ -734,18 +740,7 @@ Status SegmentIterator::_extract_common_expr_columns(const vectorized::VExprSPtr return Status::OK(); } -bool SegmentIterator::_check_apply_by_inverted_index(ColumnId col_id) { - if (_opts.runtime_state && !_opts.runtime_state->query_options().enable_inverted_index_query) { - return false; - } - if (_inverted_index_iterators[col_id] == nullptr) { - //this column without inverted index - return false; - } - return true; -} - -bool SegmentIterator::_check_apply_by_inverted_index(ColumnPredicate* pred, bool pred_in_compound) { +bool SegmentIterator::_check_apply_by_inverted_index(ColumnPredicate* pred) { if (_opts.runtime_state && !_opts.runtime_state->query_options().enable_inverted_index_query) { return false; } @@ -765,11 +760,6 @@ bool SegmentIterator::_check_apply_by_inverted_index(ColumnPredicate* pred, bool if (predicate_param->marked_by_runtime_filter) { return false; } - // the in_list or not_in_list value count cannot be greater than threshold - int32_t threshold = _opts.runtime_state->query_options().in_list_value_count_threshold; - if (pred_in_compound && predicate_param->values.size() > threshold) { - return false; - } } // UNTOKENIZED strings exceed ignore_above, they are written as null, causing range query errors @@ -788,15 +778,11 @@ bool SegmentIterator::_check_apply_by_inverted_index(ColumnPredicate* pred, bool bool handle_by_fulltext = _column_has_fulltext_index(pred_column_id); if (handle_by_fulltext) { - // when predicate in compound condition which except leafNode of andNode, - // only can apply match query for fulltext index, // when predicate is leafNode of andNode, - // can apply 'match qeury' and 'equal query' and 'list query' for fulltext index. - return (pred_in_compound ? pred->type() == PredicateType::MATCH - : (pred->type() == PredicateType::MATCH || - pred->type() == PredicateType::IS_NULL || - pred->type() == PredicateType::IS_NOT_NULL || - PredicateTypeTraits::is_equal_or_list(pred->type()))); + // can apply 'match query' and 'equal query' and 'list query' for fulltext index. + return pred->type() == PredicateType::MATCH || pred->type() == PredicateType::IS_NULL || + pred->type() == PredicateType::IS_NOT_NULL || + PredicateTypeTraits::is_equal_or_list(pred->type()); } return true; @@ -968,11 +954,6 @@ bool SegmentIterator::_is_target_expr_match_predicate(const vectorized::VExprSPt } Status SegmentIterator::_apply_inverted_index() { - SCOPED_RAW_TIMER(&_opts.stats->inverted_index_filter_timer); - if (_opts.runtime_state && !_opts.runtime_state->query_options().enable_inverted_index_query) { - return Status::OK(); - } - size_t input_rows = _row_bitmap.cardinality(); std::vector remaining_predicates; std::set no_need_to_pass_column_predicate_set; @@ -990,7 +971,6 @@ Status SegmentIterator::_apply_inverted_index() { } _col_predicates = std::move(remaining_predicates); - _opts.stats->rows_inverted_index_filtered += (input_rows - _row_bitmap.cardinality()); return Status::OK(); } diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h index 6ffd1666f602e2..f5c133485aa012 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/segment_iterator.h @@ -298,8 +298,7 @@ class SegmentIterator : public RowwiseIterator { void _convert_dict_code_for_predicate_if_necessary_impl(ColumnPredicate* predicate); - bool _check_apply_by_inverted_index(ColumnId col_id); - bool _check_apply_by_inverted_index(ColumnPredicate* pred, bool pred_in_compound = false); + bool _check_apply_by_inverted_index(ColumnPredicate* pred); void _output_index_result_column_for_expr(uint16_t* sel_rowid_idx, uint16_t select_size, vectorized::Block* block); diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index d7af005394479e..73cd02b5a5dd18 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -45,6 +45,8 @@ namespace doris::pipeline { +const static int32_t ADAPTIVE_PIPELINE_TASK_SERIAL_READ_ON_LIMIT_DEFAULT = 10000; + #define RETURN_IF_PUSH_DOWN(stmt, status) \ if (pdt == PushDownType::UNACCEPTABLE) { \ status = stmt; \ @@ -1147,12 +1149,6 @@ ScanOperatorX::ScanOperatorX(ObjectPool* pool, const TPlanNode& : OperatorX(pool, tnode, operator_id, descs), _runtime_filter_descs(tnode.runtime_filters), _parallel_tasks(parallel_tasks) { - if (!tnode.__isset.conjuncts || tnode.conjuncts.empty()) { - // Which means the request could be fullfilled in a single segment iterator request. - if (tnode.limit > 0 && tnode.limit < 1024) { - _should_run_serial = true; - } - } if (tnode.__isset.push_down_count) { _push_down_count = tnode.push_down_count; } @@ -1185,6 +1181,34 @@ Status ScanOperatorX::init(const TPlanNode& tnode, RuntimeState* if (tnode.__isset.topn_filter_source_node_ids) { topn_filter_source_node_ids = tnode.topn_filter_source_node_ids; } + + // The first branch is kept for compatibility with the old version of the FE + if (!query_options.__isset.enable_adaptive_pipeline_task_serial_read_on_limit) { + if (!tnode.__isset.conjuncts || tnode.conjuncts.empty()) { + // Which means the request could be fullfilled in a single segment iterator request. + if (tnode.limit > 0 && + tnode.limit <= ADAPTIVE_PIPELINE_TASK_SERIAL_READ_ON_LIMIT_DEFAULT) { + _should_run_serial = true; + } + } + } else { + DCHECK(query_options.__isset.adaptive_pipeline_task_serial_read_on_limit); + // The set of enable_adaptive_pipeline_task_serial_read_on_limit + // is checked in previous branch. + if (query_options.enable_adaptive_pipeline_task_serial_read_on_limit) { + int32_t adaptive_pipeline_task_serial_read_on_limit = + ADAPTIVE_PIPELINE_TASK_SERIAL_READ_ON_LIMIT_DEFAULT; + if (query_options.__isset.adaptive_pipeline_task_serial_read_on_limit) { + adaptive_pipeline_task_serial_read_on_limit = + query_options.adaptive_pipeline_task_serial_read_on_limit; + } + + if (tnode.limit > 0 && tnode.limit <= adaptive_pipeline_task_serial_read_on_limit) { + _should_run_serial = true; + } + } + } + return Status::OK(); } diff --git a/be/src/runtime/memory/mem_tracker.h b/be/src/runtime/memory/mem_tracker.h index 8a574398e0eaed..8a977e49388d52 100644 --- a/be/src/runtime/memory/mem_tracker.h +++ b/be/src/runtime/memory/mem_tracker.h @@ -146,8 +146,6 @@ class MemTracker { // Creates and adds the tracker to the mem_tracker_pool. MemTracker(const std::string& label, MemTrackerLimiter* parent = nullptr); - // For MemTrackerLimiter - MemTracker() { _parent_group_num = -1; } virtual ~MemTracker(); @@ -203,6 +201,9 @@ class MemTracker { } protected: + // Only used by MemTrackerLimiter + MemTracker() { _parent_group_num = -1; } + void bind_parent(MemTrackerLimiter* parent); Type _type; diff --git a/be/src/runtime/memory/mem_tracker_limiter.cpp b/be/src/runtime/memory/mem_tracker_limiter.cpp index 6df577f8a50347..774e06eb7f5ac4 100644 --- a/be/src/runtime/memory/mem_tracker_limiter.cpp +++ b/be/src/runtime/memory/mem_tracker_limiter.cpp @@ -336,7 +336,6 @@ void MemTrackerLimiter::make_process_snapshots(std::vector snapshot.cur_consumption = GlobalMemoryArbitrator::process_reserved_memory(); snapshot.peak_consumption = -1; (*snapshots).emplace_back(snapshot); - all_trackers_mem_sum += GlobalMemoryArbitrator::process_reserved_memory(); snapshot.type = "overview"; snapshot.label = "sum_of_all_trackers"; // is virtual memory diff --git a/be/src/vec/data_types/get_least_supertype.cpp b/be/src/vec/data_types/get_least_supertype.cpp index 7d8b3ddbe507c9..82bea452923ed5 100644 --- a/be/src/vec/data_types/get_least_supertype.cpp +++ b/be/src/vec/data_types/get_least_supertype.cpp @@ -41,6 +41,7 @@ #include "vec/data_types/data_type_number.h" #include "vec/data_types/data_type_object.h" #include "vec/data_types/data_type_string.h" +#include "vec/data_types/data_type_time_v2.h" namespace doris::vectorized { @@ -284,6 +285,14 @@ void get_least_supertype_jsonb(const TypeIndexSet& types, DataTypePtr* type) { *type = std::make_shared(); return; } + if (which.is_date_v2()) { + *type = std::make_shared(); + return; + } + if (which.is_date_time_v2()) { + *type = std::make_shared(); + return; + } *type = std::make_shared(); return; } diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h index 90048511c90856..382713b2afc018 100644 --- a/be/src/vec/exprs/vexpr.h +++ b/be/src/vec/exprs/vexpr.h @@ -321,7 +321,6 @@ class VExpr { uint32_t _index_unique_id = 0; bool _can_fast_execute = false; bool _enable_inverted_index_query = true; - uint32_t _in_list_value_count_threshold = 10; }; } // namespace vectorized diff --git a/be/src/vec/exprs/vin_predicate.cpp b/be/src/vec/exprs/vin_predicate.cpp index 361f55f038ca4c..9e00a3afbaef43 100644 --- a/be/src/vec/exprs/vin_predicate.cpp +++ b/be/src/vec/exprs/vin_predicate.cpp @@ -81,6 +81,14 @@ Status VInPredicate::prepare(RuntimeState* state, const RowDescriptor& desc, VExpr::register_function_context(state, context); _prepare_finished = true; + + if (state->query_options().__isset.in_list_value_count_threshold) { + _in_list_value_count_threshold = state->query_options().in_list_value_count_threshold; + } + + const auto in_list_value_count = _children.size() - 1; + // When the number of values in the IN condition exceeds this threshold, fast_execute will not be used + _can_fast_execute = in_list_value_count <= _in_list_value_count_threshold; return Status::OK(); } @@ -94,10 +102,24 @@ Status VInPredicate::open(RuntimeState* state, VExprContext* context, if (scope == FunctionContext::FRAGMENT_LOCAL) { RETURN_IF_ERROR(VExpr::get_const_col(context, nullptr)); } + + _is_args_all_constant = std::all_of(_children.begin() + 1, _children.end(), + [](const VExprSPtr& expr) { return expr->is_constant(); }); _open_finished = true; return Status::OK(); } +size_t VInPredicate::skip_constant_args_size() const { + if (_is_args_all_constant && !_can_fast_execute) { + // This is an optimization. For expressions like colA IN (1, 2, 3, 4), + // where all values inside the IN clause are constants, + // a hash set is created during open, and it will not be accessed again during execute + // Here, _children[0] is colA + return 1; + } + return _children.size(); +} + void VInPredicate::close(VExprContext* context, FunctionContext::FunctionStateScope scope) { VExpr::close_function_context(context, scope, _function); VExpr::close(context, scope); @@ -116,9 +138,8 @@ Status VInPredicate::execute(VExprContext* context, Block* block, int* result_co return Status::OK(); } DCHECK(_open_finished || _getting_const_col); - // TODO: not execute const expr again, but use the const column in function context - doris::vectorized::ColumnNumbers arguments(_children.size()); - for (int i = 0; i < _children.size(); ++i) { + doris::vectorized::ColumnNumbers arguments(skip_constant_args_size()); + for (int i = 0; i < skip_constant_args_size(); ++i) { int column_id = -1; RETURN_IF_ERROR(_children[i]->execute(context, block, &column_id)); arguments[i] = column_id; diff --git a/be/src/vec/exprs/vin_predicate.h b/be/src/vec/exprs/vin_predicate.h index 4d227510b910ce..024ad68f2ba576 100644 --- a/be/src/vec/exprs/vin_predicate.h +++ b/be/src/vec/exprs/vin_predicate.h @@ -51,6 +51,8 @@ class VInPredicate final : public VExpr { std::string debug_string() const override; + size_t skip_constant_args_size() const; + const FunctionBasePtr function() { return _function; } bool is_not_in() const { return _is_not_in; }; @@ -62,5 +64,7 @@ class VInPredicate final : public VExpr { const bool _is_not_in; static const constexpr char* function_name = "in"; + uint32_t _in_list_value_count_threshold = 10; + bool _is_args_all_constant = false; }; } // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/functions/function_collection_in.h b/be/src/vec/functions/function_collection_in.h index 1e86ce25b3405f..33a4a2570800a9 100644 --- a/be/src/vec/functions/function_collection_in.h +++ b/be/src/vec/functions/function_collection_in.h @@ -112,9 +112,8 @@ class FunctionCollectionIn : public IFunction { // FE should make element type consistent and // equalize the length of the elements in struct const auto& const_column_ptr = context->get_constant_col(i); - if (const_column_ptr == nullptr) { - break; - } + // Types like struct, array, and map only support constant expressions. + DCHECK(const_column_ptr != nullptr); const auto& [col, _] = unpack_if_const(const_column_ptr->column_ptr); if (col->is_nullable()) { auto* null_col = vectorized::check_and_get_column(col); diff --git a/be/src/vec/functions/function_math_unary_alway_nullable.h b/be/src/vec/functions/function_math_unary_alway_nullable.h new file mode 100644 index 00000000000000..8d2cea1bc0db87 --- /dev/null +++ b/be/src/vec/functions/function_math_unary_alway_nullable.h @@ -0,0 +1,94 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include "vec/columns/column.h" +#include "vec/columns/column_decimal.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/columns_number.h" +#include "vec/core/call_on_type_index.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type_decimal.h" +#include "vec/data_types/data_type_nullable.h" +#include "vec/data_types/data_type_number.h" +#include "vec/functions/function.h" +#include "vec/functions/function_helpers.h" +#include "vec/utils/util.hpp" + +namespace doris::vectorized { + +template +class FunctionMathUnaryAlwayNullable : public IFunction { +public: + using IFunction::execute; + + static constexpr auto name = Impl::name; + static FunctionPtr create() { return std::make_shared(); } + +private: + String get_name() const override { return name; } + size_t get_number_of_arguments() const override { return 1; } + + DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { + return make_nullable(std::make_shared()); + } + + static void execute_in_iterations(const double* src_data, double* dst_data, size_t size) { + for (size_t i = 0; i < size; i++) { + Impl::execute(&src_data[i], &dst_data[i]); + } + } + + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count) const override { + const ColumnFloat64* col = + assert_cast(block.get_by_position(arguments[0]).column.get()); + auto dst = ColumnFloat64::create(); + auto& dst_data = dst->get_data(); + dst_data.resize(input_rows_count); + + execute_in_iterations(col->get_data().data(), dst_data.data(), input_rows_count); + + auto result_null_map = ColumnUInt8::create(input_rows_count, 0); + + for (size_t i = 0; i < input_rows_count; i++) { + if (Impl::is_invalid_input(col->get_data()[i])) [[unlikely]] { + result_null_map->get_data().data()[i] = 1; + } + } + + block.replace_by_position( + result, ColumnNullable::create(std::move(dst), std::move(result_null_map))); + return Status::OK(); + } +}; + +template +struct UnaryFunctionPlainAlwayNullable { + using Type = DataTypeFloat64; + static constexpr auto name = Name::name; + + static constexpr bool is_invalid_input(Float64 x) { return Name::is_invalid_input(x); } + + template + static void execute(const T* src, U* dst) { + *dst = static_cast(Function(*src)); + } +}; + +} // namespace doris::vectorized diff --git a/be/src/vec/functions/functions_comparison.h b/be/src/vec/functions/functions_comparison.h index f6aa46fe1c42c9..bb1666ab864070 100644 --- a/be/src/vec/functions/functions_comparison.h +++ b/be/src/vec/functions/functions_comparison.h @@ -546,14 +546,6 @@ class FunctionComparison : public IFunction { //NOT support comparison predicate when parser is FULLTEXT for expr inverted index evaluate. return Status::OK(); } - std::string column_name = data_type_with_name.first; - Field param_value; - arguments[0].column->get(0, param_value); - auto param_type = arguments[0].type->get_type_as_type_descriptor().type; - - std::unique_ptr query_param = nullptr; - RETURN_IF_ERROR(segment_v2::InvertedIndexQueryParamFactory::create_query_value( - param_type, ¶m_value, query_param)); segment_v2::InvertedIndexQueryType query_type; std::string_view name_view(name); if (name_view == NameEquals::name || name_view == NameNotEquals::name) { @@ -570,6 +562,19 @@ class FunctionComparison : public IFunction { return Status::InvalidArgument("invalid comparison op type {}", Name::name); } + if (segment_v2::is_range_query(query_type) && + iter->get_inverted_index_reader_type() == + segment_v2::InvertedIndexReaderType::STRING_TYPE) { + // untokenized strings exceed ignore_above, they are written as null, causing range query errors + return Status::OK(); + } + std::string column_name = data_type_with_name.first; + Field param_value; + arguments[0].column->get(0, param_value); + auto param_type = arguments[0].type->get_type_as_type_descriptor().type; + std::unique_ptr query_param = nullptr; + RETURN_IF_ERROR(segment_v2::InvertedIndexQueryParamFactory::create_query_value( + param_type, ¶m_value, query_param)); std::shared_ptr roaring = std::make_shared(); RETURN_IF_ERROR(segment_v2::InvertedIndexQueryParamFactory::create_query_value( param_type, ¶m_value, query_param)); @@ -585,7 +590,7 @@ class FunctionComparison : public IFunction { bitmap_result = result; bitmap_result.mask_out_null(); - if (name == "ne") { + if (name_view == NameNotEquals::name) { roaring::Roaring full_result; full_result.addRange(0, num_rows); bitmap_result.op_not(&full_result); diff --git a/be/src/vec/functions/functions_geo.cpp b/be/src/vec/functions/functions_geo.cpp index b389bc1636e45f..7b833f91a8ddfd 100644 --- a/be/src/vec/functions/functions_geo.cpp +++ b/be/src/vec/functions/functions_geo.cpp @@ -28,6 +28,7 @@ #include "vec/columns/column.h" #include "vec/columns/column_nullable.h" #include "vec/columns/columns_number.h" +#include "vec/common/assert_cast.h" #include "vec/common/string_ref.h" #include "vec/core/block.h" #include "vec/core/column_with_type_and_name.h" @@ -58,14 +59,16 @@ struct StPoint { auto res = ColumnString::create(); auto null_map = ColumnUInt8::create(size, 0); auto& null_map_data = null_map->get_data(); + const auto* left_column_f64 = assert_cast(left_column.get()); + const auto* right_column_f64 = assert_cast(right_column.get()); GeoPoint point; std::string buf; if (left_const) { - const_vector(left_column, right_column, res, null_map_data, size, point, buf); + const_vector(left_column_f64, right_column_f64, res, null_map_data, size, point, buf); } else if (right_const) { - vector_const(left_column, right_column, res, null_map_data, size, point, buf); + vector_const(left_column_f64, right_column_f64, res, null_map_data, size, point, buf); } else { - vector_vector(left_column, right_column, res, null_map_data, size, point, buf); + vector_vector(left_column_f64, right_column_f64, res, null_map_data, size, point, buf); } block.replace_by_position(result, @@ -86,32 +89,32 @@ struct StPoint { res->insert_data(buf.data(), buf.size()); } - static void const_vector(const ColumnPtr& left_column, const ColumnPtr& right_column, + static void const_vector(const ColumnFloat64* left_column, const ColumnFloat64* right_column, ColumnString::MutablePtr& res, NullMap& null_map, const size_t size, GeoPoint& point, std::string& buf) { - double x = left_column->operator[](0).get(); + double x = left_column->get_element(0); for (int row = 0; row < size; ++row) { - auto cur_res = point.from_coord(x, right_column->operator[](row).get()); + auto cur_res = point.from_coord(x, right_column->get_element(row)); loop_do(cur_res, res, null_map, row, point, buf); } } - static void vector_const(const ColumnPtr& left_column, const ColumnPtr& right_column, + static void vector_const(const ColumnFloat64* left_column, const ColumnFloat64* right_column, ColumnString::MutablePtr& res, NullMap& null_map, const size_t size, GeoPoint& point, std::string& buf) { - double y = right_column->operator[](0).get(); + double y = right_column->get_element(0); for (int row = 0; row < size; ++row) { - auto cur_res = point.from_coord(right_column->operator[](row).get(), y); + auto cur_res = point.from_coord(right_column->get_element(row), y); loop_do(cur_res, res, null_map, row, point, buf); } } - static void vector_vector(const ColumnPtr& left_column, const ColumnPtr& right_column, + static void vector_vector(const ColumnFloat64* left_column, const ColumnFloat64* right_column, ColumnString::MutablePtr& res, NullMap& null_map, const size_t size, GeoPoint& point, std::string& buf) { for (int row = 0; row < size; ++row) { - auto cur_res = point.from_coord(left_column->operator[](row).get(), - right_column->operator[](row).get()); + auto cur_res = + point.from_coord(left_column->get_element(row), right_column->get_element(row)); loop_do(cur_res, res, null_map, row, point, buf); } } @@ -246,11 +249,15 @@ struct StDistanceSphere { DCHECK_EQ(arguments.size(), 4); auto return_type = block.get_data_type(result); - auto x_lng = block.get_by_position(arguments[0]).column->convert_to_full_column_if_const(); - auto x_lat = block.get_by_position(arguments[1]).column->convert_to_full_column_if_const(); - auto y_lng = block.get_by_position(arguments[2]).column->convert_to_full_column_if_const(); - auto y_lat = block.get_by_position(arguments[3]).column->convert_to_full_column_if_const(); - + const auto* x_lng = check_and_get_column( + block.get_by_position(arguments[0]).column->convert_to_full_column_if_const()); + const auto* x_lat = check_and_get_column( + block.get_by_position(arguments[1]).column->convert_to_full_column_if_const()); + const auto* y_lng = check_and_get_column( + block.get_by_position(arguments[2]).column->convert_to_full_column_if_const()); + const auto* y_lat = check_and_get_column( + block.get_by_position(arguments[3]).column->convert_to_full_column_if_const()); + CHECK(x_lng && x_lat && y_lng && y_lat); const auto size = x_lng->size(); auto res = ColumnFloat64::create(); res->reserve(size); @@ -258,10 +265,9 @@ struct StDistanceSphere { auto& null_map_data = null_map->get_data(); for (int row = 0; row < size; ++row) { double distance = 0; - if (!GeoPoint::ComputeDistance(x_lng->operator[](row).get(), - x_lat->operator[](row).get(), - y_lng->operator[](row).get(), - y_lat->operator[](row).get(), &distance)) { + if (!GeoPoint::ComputeDistance(x_lng->get_element(row), x_lat->get_element(row), + y_lng->get_element(row), y_lat->get_element(row), + &distance)) { null_map_data[row] = 1; res->insert_default(); continue; @@ -284,10 +290,15 @@ struct StAngleSphere { DCHECK_EQ(arguments.size(), 4); auto return_type = block.get_data_type(result); - auto x_lng = block.get_by_position(arguments[0]).column->convert_to_full_column_if_const(); - auto x_lat = block.get_by_position(arguments[1]).column->convert_to_full_column_if_const(); - auto y_lng = block.get_by_position(arguments[2]).column->convert_to_full_column_if_const(); - auto y_lat = block.get_by_position(arguments[3]).column->convert_to_full_column_if_const(); + const auto* x_lng = check_and_get_column( + block.get_by_position(arguments[0]).column->convert_to_full_column_if_const()); + const auto* x_lat = check_and_get_column( + block.get_by_position(arguments[1]).column->convert_to_full_column_if_const()); + const auto* y_lng = check_and_get_column( + block.get_by_position(arguments[2]).column->convert_to_full_column_if_const()); + const auto* y_lat = check_and_get_column( + block.get_by_position(arguments[3]).column->convert_to_full_column_if_const()); + CHECK(x_lng && x_lat && y_lng && y_lat); const auto size = x_lng->size(); @@ -298,10 +309,9 @@ struct StAngleSphere { for (int row = 0; row < size; ++row) { double angle = 0; - if (!GeoPoint::ComputeAngleSphere(x_lng->operator[](row).get(), - x_lat->operator[](row).get(), - y_lng->operator[](row).get(), - y_lat->operator[](row).get(), &angle)) { + if (!GeoPoint::ComputeAngleSphere(x_lng->get_element(row), x_lat->get_element(row), + y_lng->get_element(row), y_lat->get_element(row), + &angle)) { null_map_data[row] = 1; res->insert_default(); continue; diff --git a/be/src/vec/functions/math.cpp b/be/src/vec/functions/math.cpp index a3b54c8026db75..af2e68ec9822c8 100644 --- a/be/src/vec/functions/math.cpp +++ b/be/src/vec/functions/math.cpp @@ -37,6 +37,7 @@ #include "vec/functions/function_const.h" #include "vec/functions/function_math_log.h" #include "vec/functions/function_math_unary.h" +#include "vec/functions/function_math_unary_alway_nullable.h" #include "vec/functions/function_string.h" #include "vec/functions/function_totype.h" #include "vec/functions/function_unary_arithmetic.h" @@ -53,13 +54,19 @@ struct Log2Impl; namespace doris::vectorized { struct AcosName { static constexpr auto name = "acos"; + // https://dev.mysql.com/doc/refman/8.4/en/mathematical-functions.html#function_acos + static constexpr bool is_invalid_input(Float64 x) { return x < -1 || x > 1; } }; -using FunctionAcos = FunctionMathUnary>; +using FunctionAcos = + FunctionMathUnaryAlwayNullable>; struct AsinName { static constexpr auto name = "asin"; + // https://dev.mysql.com/doc/refman/8.4/en/mathematical-functions.html#function_asin + static constexpr bool is_invalid_input(Float64 x) { return x < -1 || x > 1; } }; -using FunctionAsin = FunctionMathUnary>; +using FunctionAsin = + FunctionMathUnaryAlwayNullable>; struct AtanName { static constexpr auto name = "atan"; @@ -242,8 +249,11 @@ using FunctionSin = FunctionMathUnary; struct SqrtName { static constexpr auto name = "sqrt"; + // https://dev.mysql.com/doc/refman/8.4/en/mathematical-functions.html#function_sqrt + static constexpr bool is_invalid_input(Float64 x) { return x < 0; } }; -using FunctionSqrt = FunctionMathUnary>; +using FunctionSqrt = + FunctionMathUnaryAlwayNullable>; struct CbrtName { static constexpr auto name = "cbrt"; diff --git a/be/test/vec/core/get_common_type_test.cpp b/be/test/vec/core/get_common_type_test.cpp index c8c60a3776cd2d..86b7299176dba0 100644 --- a/be/test/vec/core/get_common_type_test.cpp +++ b/be/test/vec/core/get_common_type_test.cpp @@ -22,6 +22,7 @@ #include "gtest/gtest_pred_impl.h" #include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_jsonb.h" #include "vec/data_types/data_type_nothing.h" #include "vec/data_types/data_type_number.h" #include "vec/data_types/data_type_string.h" @@ -67,6 +68,8 @@ static DataTypePtr typeFromString(const std::string& str) { return std::make_shared(); } else if (str == "String") { return std::make_shared(); + } else if (str == "Jsonb") { + return std::make_shared(); } return nullptr; } @@ -141,12 +144,7 @@ INSTANTIATE_TEST_SUITE_P(data_type, LeastSuperTypeTest, {"Int8 Int32 Int64", "Int64"}, {"UInt8 UInt32 Int64", "Int64"}, {"Float32 Float64", "Float64"}, + {"Date Date", "Date"}, {"Float32 UInt16 Int32", "Float64"}, {"Float32 Int16 UInt32", "Float64"}, - {"Date Date", "Date"}, - {"DateTime DateTime", "DateTime"}, - {"String String String", "String"}, - {"Int8 String", nullptr}, - {"Int64 UInt64", nullptr}, - {"Float32 UInt64", nullptr}, - {"Float64 Int64", nullptr}})); + {"String String String", "String"}})); diff --git a/cloud/script/start.sh b/cloud/script/start.sh index 2e444ac1778e6d..28e986166ae832 100644 --- a/cloud/script/start.sh +++ b/cloud/script/start.sh @@ -82,7 +82,7 @@ if ldd "${bin}" | grep -Ei 'libfdb_c.*not found' &>/dev/null; then exit 1 fi patchelf --set-rpath "${lib_path}" "${bin}" - ldd "${bin}" + # ldd "${bin}" fi chmod 550 "${DORIS_HOME}/lib/doris_cloud" @@ -91,7 +91,7 @@ if [[ -z "${JAVA_HOME}" ]]; then echo "The JAVA_HOME environment variable is not defined correctly" echo "This environment variable is needed to run this program" echo "NB: JAVA_HOME should point to a JDK not a JRE" - echo "You can set it in be.conf" + echo "You can set it in doris_cloud.conf" exit 1 fi diff --git a/cloud/script/stop.sh b/cloud/script/stop.sh index b40b4061c398dc..48f01c545ae840 100644 --- a/cloud/script/stop.sh +++ b/cloud/script/stop.sh @@ -32,4 +32,17 @@ fi pid=$(cat "${DORIS_HOME}/bin/${process}.pid") kill -2 "${pid}" +cnt=0 +while true; do + cnt=$((cnt + 1)) + echo "waiting ${pid} to quit, ${cnt} seconds elapsed" + msg=$(ps "${pid}") + ret=$? + if [[ ${ret} -ne 0 ]]; then + echo "${pid} has quit" + break + fi + echo "${msg}" + sleep 1 +done rm -f "${DORIS_HOME}/bin/${process}.pid" diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h index d401caa4ad975f..e31a60a0d69f80 100644 --- a/cloud/src/common/config.h +++ b/cloud/src/common/config.h @@ -55,7 +55,7 @@ CONF_Bool(enable_file_logger, "true"); // recycler config CONF_mInt64(recycle_interval_seconds, "3600"); -CONF_mInt64(retention_seconds, "259200"); // 72h +CONF_mInt64(retention_seconds, "259200"); // 72h, global retention time CONF_Int32(recycle_concurrency, "16"); CONF_Int32(recycle_job_lease_expired_ms, "60000"); CONF_mInt64(compacted_rowset_retention_seconds, "10800"); // 3h @@ -77,7 +77,8 @@ CONF_mInt32(scan_instances_interval_seconds, "60"); // 1min CONF_mInt32(check_object_interval_seconds, "43200"); // 12hours CONF_mInt64(check_recycle_task_interval_seconds, "600"); // 10min -CONF_mInt64(recycle_task_threshold_seconds, "10800"); // 3h +// log a warning if a recycle task takes longer than this duration +CONF_mInt64(recycle_task_threshold_seconds, "10800"); // 3h CONF_String(test_s3_ak, ""); CONF_String(test_s3_sk, ""); diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index 3cd1bd798bbc84..5f2638c8d1fa28 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -479,8 +479,8 @@ void MetaServiceImpl::precommit_txn(::google::protobuf::RpcController* controlle return; } - LOG(INFO) << "xxx put running_key=" << hex(running_key) << " txn_id=" << txn_id; txn->put(running_key, running_val); + LOG(INFO) << "xxx put running_key=" << hex(running_key) << " txn_id=" << txn_id; err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { @@ -569,8 +569,6 @@ void put_routine_load_progress(MetaServiceCode& code, std::string& msg, new_statistic_info->set_task_execution_time_ms(commit_attachment.task_execution_time_ms()); } - LOG(INFO) << "routine load new progress: " << new_progress_info.ShortDebugString(); - if (!new_progress_info.SerializeToString(&new_progress_val)) { code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; ss << "failed to serialize new progress val, txn_id=" << txn_id; @@ -579,6 +577,8 @@ void put_routine_load_progress(MetaServiceCode& code, std::string& msg, } txn->put(rl_progress_key, new_progress_val); + LOG(INFO) << "put rl_progress_key key=" << hex(rl_progress_key) + << " routine load new progress: " << new_progress_info.ShortDebugString(); } void MetaServiceImpl::get_rl_task_commit_attach(::google::protobuf::RpcController* controller, @@ -689,6 +689,7 @@ void MetaServiceImpl::reset_rl_progress(::google::protobuf::RpcController* contr if (request->partition_to_offset().size() == 0) { txn->remove(rl_progress_key); + LOG(INFO) << "remove rl_progress_key key=" << hex(rl_progress_key); } if (request->partition_to_offset().size() > 0) { @@ -738,6 +739,7 @@ void MetaServiceImpl::reset_rl_progress(::google::protobuf::RpcController* contr return; } txn->put(rl_progress_key, new_progress_val); + LOG(INFO) << "put rl_progress_key key=" << hex(rl_progress_key); } err = txn->commit(); @@ -892,6 +894,7 @@ void update_tablet_stats(const StatsTabletKeyInfo& info, const TabletStats& stat stats_pb.set_num_segments(stats_pb.num_segments() + stats.num_segs); stats_pb.SerializeToString(&val); txn->put(key, val); + LOG(INFO) << "put stats_tablet_key key=" << hex(key); } } @@ -2370,6 +2373,7 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* stats_pb.set_num_segments(stats_pb.num_segments() + stats.num_segs); stats_pb.SerializeToString(&val); txn->put(key, val); + LOG(INFO) << "put stats_tablet_key, key=" << hex(key); }; } for (auto& [tablet_id, stats] : tablet_stats) { @@ -2405,8 +2409,9 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* return; } txn->put(recycle_key, recycle_val); + LOG(INFO) << "xxx commit_txn put recycle_txn_key key=" << hex(recycle_key) + << " txn_id=" << txn_id; - LOG(INFO) << "xxx commit_txn put recycle_key key=" << hex(recycle_key) << " txn_id=" << txn_id; LOG(INFO) << "commit_txn put_size=" << txn->put_bytes() << " del_size=" << txn->delete_bytes() << " num_put_keys=" << txn->num_put_keys() << " num_del_keys=" << txn->num_del_keys() << " txn_size=" << txn->approximate_bytes() << " txn_id=" << txn_id; @@ -2541,7 +2546,7 @@ static void _abort_txn(const std::string& instance_id, const AbortTxnRequest* re //1. abort txn by txn id //2. abort txn by label and db_id if (txn_id > 0) { - VLOG_DEBUG << "abort_txn by txn_id"; + VLOG_DEBUG << "abort_txn by txn_id, txn_id=" << txn_id; //abort txn by txn id // Get db id with txn id @@ -2610,7 +2615,7 @@ static void _abort_txn(const std::string& instance_id, const AbortTxnRequest* re return; } } else { - VLOG_DEBUG << "abort_txn by db_id and txn label"; + VLOG_DEBUG << "abort_txn db_id and label, db_id=" << db_id << " label=" << label; //abort txn by label. std::string label_key = txn_label_key({instance_id, db_id, label}); std::string label_val; @@ -2726,7 +2731,7 @@ static void _abort_txn(const std::string& instance_id, const AbortTxnRequest* re return; } txn->put(recycle_key, recycle_val); - LOG(INFO) << "xxx put recycle_key=" << hex(recycle_key) + LOG(INFO) << "put recycle_txn_key=" << hex(recycle_key) << " txn_id=" << return_txn_info.txn_id(); } diff --git a/cloud/src/recycler/hdfs_accessor.cpp b/cloud/src/recycler/hdfs_accessor.cpp index e5038735f5735f..97a4670d2bfc6b 100644 --- a/cloud/src/recycler/hdfs_accessor.cpp +++ b/cloud/src/recycler/hdfs_accessor.cpp @@ -354,8 +354,27 @@ int HdfsAccessor::init() { } int HdfsAccessor::delete_prefix(const std::string& path_prefix, int64_t expiration_time) { - LOG_INFO("delete prefix").tag("uri", to_uri(path_prefix)); // Audit log - return 0; + auto uri = to_uri(path_prefix); + LOG(INFO) << "delete prefix, uri=" << uri; + std::unique_ptr list_iter; + int ret = list_all(&list_iter); + if (ret != 0) { + LOG(WARNING) << "delete prefix, failed to list" << uri; + return ret; + } + size_t num_listed = 0, num_deleted = 0; + for (auto file = list_iter->next(); file; file = list_iter->next()) { + ++num_listed; + if (file->path.find(path_prefix) != 0) continue; + if (int del_ret = delete_file(file->path); del_ret != 0) { + ret = del_ret; + break; + } + ++num_deleted; + } + LOG(INFO) << "delete prefix " << (ret != 0 ? "failed" : "succ") << " ret=" << ret + << " uri=" << uri << " num_listed=" << num_listed << " num_deleted=" << num_deleted; + return ret; } int HdfsAccessor::delete_directory_impl(const std::string& dir_path) { diff --git a/cloud/src/recycler/recycler.cpp b/cloud/src/recycler/recycler.cpp index 919b50358c77f5..9db16a18c13d4c 100644 --- a/cloud/src/recycler/recycler.cpp +++ b/cloud/src/recycler/recycler.cpp @@ -1219,7 +1219,7 @@ int InstanceRecycler::recycle_tablets(int64_t table_id, int64_t index_id, int64_ return {std::string_view(), range_move}; } ++num_recycled; - LOG_INFO("k is {}, is empty {}", k, k.empty()); + LOG(INFO) << "recycle_tablets scan, key=" << (k.empty() ? "(empty)" : hex(k)); return {k, range_move}; }); } else { @@ -1694,6 +1694,10 @@ int InstanceRecycler::recycle_rowsets() { LOG_WARNING("malformed recycle rowset").tag("key", hex(k)); return -1; } + + VLOG_DEBUG << "recycle rowset scan, key=" << hex(k) << " num_scanned=" << num_scanned + << " num_expired=" << num_expired << " expiration=" << calc_expiration(rowset) + << " RecycleRowsetPB=" << rowset.ShortDebugString(); int64_t current_time = ::time(nullptr); if (current_time < calc_expiration(rowset)) { // not expired return 0; @@ -1745,8 +1749,8 @@ int InstanceRecycler::recycle_rowsets() { << rowset_meta->start_version() << '-' << rowset_meta->end_version() << "] txn_id=" << rowset_meta->txn_id() << " type=" << RecycleRowsetPB_Type_Name(rowset.type()) - << " rowset_meta_size=" << v.size() << " creation_time" - << rowset_meta->creation_time(); + << " rowset_meta_size=" << v.size() + << " creation_time=" << rowset_meta->creation_time(); if (rowset.type() == RecycleRowsetPB::PREPARE) { // unable to calculate file path, can only be deleted by rowset id prefix num_prepare += 1; @@ -1910,6 +1914,10 @@ int InstanceRecycler::recycle_tmp_rowsets() { // duration or timeout always < `retention_time` in practice. int64_t expiration = rowset.txn_expiration() > 0 ? rowset.txn_expiration() : rowset.creation_time(); + VLOG_DEBUG << "recycle tmp rowset scan, key=" << hex(k) << " num_scanned=" << num_scanned + << " num_expired=" << num_expired << " expiration=" << expiration + << " txn_expiration=" << rowset.txn_expiration() + << " rowset_creation_time=" << rowset.creation_time(); if (current_time < expiration + config::retention_seconds) { // not expired return 0; @@ -2150,7 +2158,7 @@ int InstanceRecycler::recycle_expired_txn_label() { recycle_txn_key(recycle_txn_key_info0, &begin_recycle_txn_key); recycle_txn_key(recycle_txn_key_info1, &end_recycle_txn_key); - LOG_INFO("begin to recycle expire txn").tag("instance_id", instance_id_); + LOG_INFO("begin to recycle expired txn").tag("instance_id", instance_id_); int64_t start_time = duration_cast(steady_clock::now().time_since_epoch()).count(); register_recycle_task(task_name, start_time); diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 index 42092b896822c2..2343f208642d6e 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 @@ -554,6 +554,7 @@ VAULT: 'VAULT'; VERBOSE: 'VERBOSE'; VERSION: 'VERSION'; VIEW: 'VIEW'; +VIEWS: 'VIEWS'; WARM: 'WARM'; WARNINGS: 'WARNINGS'; WEEK: 'WEEK'; diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index b46b55a96e0da8..087af9d717de13 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -38,7 +38,7 @@ statement | CALL name=multipartIdentifier LEFT_PAREN (expression (COMMA expression)*)? RIGHT_PAREN #callProcedure | (ALTER | CREATE (OR REPLACE)? | REPLACE) (PROCEDURE | PROC) name=multipartIdentifier LEFT_PAREN .*? RIGHT_PAREN .*? #createProcedure | DROP (PROCEDURE | PROC) (IF EXISTS)? name=multipartIdentifier #dropProcedure - | SHOW PROCEDURE STATUS (LIKE pattern=valueExpression | whereClause)? #showProcedureStatus + | SHOW (PROCEDURE | FUNCTION) STATUS (LIKE pattern=valueExpression | whereClause)? #showProcedureStatus | SHOW CREATE PROCEDURE name=multipartIdentifier #showCreateProcedure // FIXME: like should be wildWhere? FRONTEND should not contain FROM backendid | ADMIN? SHOW type=(FRONTEND | BACKEND) CONFIG (LIKE pattern=valueExpression)? (FROM backendId=INTEGER_VALUE)? #showConfig @@ -49,7 +49,7 @@ statementBase | supportedDmlStatement #supportedDmlStatementAlias | supportedCreateStatement #supportedCreateStatementAlias | supportedAlterStatement #supportedAlterStatementAlias - | materailizedViewStatement #materailizedViewStatementAlias + | materializedViewStatement #materializedViewStatementAlias | constraintStatement #constraintStatementAlias | supportedDropStatement #supportedDropStatementAlias | unsupportedStatement #unsupported @@ -75,10 +75,11 @@ unsupportedStatement | unsupportedCleanStatement | unsupportedRefreshStatement | unsupportedLoadStatement + | unsupportedShowStatement | unsupportedOtherStatement ; -materailizedViewStatement +materializedViewStatement : CREATE MATERIALIZED VIEW (IF NOT EXISTS)? mvName=multipartIdentifier (LEFT_PAREN cols=simpleColumnDefs RIGHT_PAREN)? buildMode? (REFRESH refreshMethod? refreshTrigger?)? @@ -202,6 +203,121 @@ lockTable (READ (LOCAL)? | (LOW_PRIORITY)? WRITE) ; +unsupportedShowStatement + : SHOW SQL_BLOCK_RULE (FOR ruleName=identifier)? #showSqlBlockRule + | SHOW ROW POLICY (FOR (userIdentify | (ROLE role=identifier)))? #showRowPolicy + | SHOW STORAGE POLICY (USING (FOR policy=identifierOrText)?)? #showStoragePolicy + | SHOW STAGES #showStages + | SHOW STORAGE VAULT #showStorageVault + | SHOW CREATE REPOSITORY FOR identifier #showCreateRepository + | SHOW WHITELIST #showWhitelist + | SHOW (GLOBAL | SESSION | LOCAL)? VARIABLES wildWhere? #showVariables + | SHOW OPEN TABLES ((FROM | IN) database=multipartIdentifier)? wildWhere? #showOpenTables + | SHOW TABLE STATUS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTableStatus + | SHOW FULL? TABLES ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTables + | SHOW FULL? VIEWS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showViews + | SHOW TABLE tableId=INTEGER_VALUE #showTableId + | SHOW FULL? PROCESSLIST #showProcessList + | SHOW (GLOBAL | SESSION | LOCAL)? STATUS wildWhere? #showStatus + | SHOW FULL? TRIGGERS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTriggers + | SHOW EVENTS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showEvents + | SHOW PLUGINS #showPlugins + | SHOW STORAGE? ENGINES #showStorageEngines + | SHOW AUTHORS #showAuthors + | SHOW BRIEF? CREATE TABLE name=multipartIdentifier #showCreateTable + | SHOW CREATE VIEW name=multipartIdentifier #showCreateView + | SHOW CREATE MATERIALIZED VIEW name=multipartIdentifier #showMaterializedView + | SHOW CREATE (DATABASE | SCHEMA) name=multipartIdentifier #showCreateDatabase + | SHOW CREATE CATALOG name=identifier #showCreateCatalog + | SHOW CREATE (GLOBAL | SESSION | LOCAL)? FUNCTION functionIdentifier + LEFT_PAREN functionArguments? RIGHT_PAREN + ((FROM | IN) database=multipartIdentifier)? #showCreateFunction + | SHOW (DATABASES | SCHEMAS) (FROM catalog=identifier)? wildWhere? #showDatabases + | SHOW DATABASE databaseId=INTEGER_VALUE #showDatabaseId + | SHOW DATA TYPES #showDataTypes + | SHOW CATALOGS wildWhere? #showCatalogs + | SHOW CATALOG name=identifier #showCatalog + | SHOW DYNAMIC PARTITION TABLES ((FROM | IN) database=multipartIdentifier)? #showDynamicPartition + | SHOW FULL? (COLUMNS | FIELDS) (FROM | IN) tableName=multipartIdentifier + ((FROM | IN) database=multipartIdentifier)? wildWhere? #showColumns + | SHOW COLLATION wildWhere? #showCollation + | SHOW ((CHAR SET) | CHARSET) wildWhere? #showCharset + | SHOW PROC path=STRING_LITERAL #showProc + | SHOW COUNT LEFT_PAREN ASTERISK RIGHT_PAREN (WARNINGS | ERRORS) #showWaringErrorCount + | SHOW (WARNINGS | ERRORS) limitClause? #showWaringErrors + | SHOW LOAD WARNINGS ((((FROM | IN) database=multipartIdentifier)? + wildWhere? limitClause?) | (ON url=STRING_LITERAL)) #showLoadWarings + | SHOW STREAM? LOAD ((FROM | IN) database=multipartIdentifier)? wildWhere? + sortClause? limitClause? #showLoad + | SHOW EXPORT ((FROM | IN) database=multipartIdentifier)? wildWhere? + sortClause? limitClause? #showExport + | SHOW DELETE ((FROM | IN) database=multipartIdentifier)? #showDelete + | SHOW ALTER TABLE (ROLLUP | (MATERIALIZED VIEW) | COLUMN) + ((FROM | IN) database=multipartIdentifier)? wildWhere? + sortClause? limitClause? #showAlterTable + | SHOW DATA SKEW FROM baseTableRef #showDataSkew + | SHOW DATA (FROM tableName=multipartIdentifier)? sortClause? propertyClause? #showData + | SHOW TEMPORARY? PARTITIONS FROM tableName=multipartIdentifier + wildWhere? sortClause? limitClause? #showPartitions + | SHOW PARTITION partitionId=INTEGER_VALUE #showPartitionId + | SHOW TABLET tabletId=INTEGER_VALUE #showTabletId + | SHOW TABLETS BELONG + tabletIds+=INTEGER_VALUE (COMMA tabletIds+=INTEGER_VALUE)* #showTabletBelong + | SHOW TABLETS FROM tableName=multipartIdentifier partitionSpec? + wildWhere? sortClause? limitClause? #showTabletsFromTable + | SHOW PROPERTY (FOR user=identifierOrText)? wildWhere? #showUserProperties + | SHOW ALL PROPERTIES wildWhere? #showAllProperties + | SHOW BACKUP ((FROM | IN) database=multipartIdentifier)? wildWhere? #showBackup + | SHOW BRIEF? RESTORE ((FROM | IN) database=multipartIdentifier)? wildWhere? #showRestore + | SHOW BROKER #showBroker + | SHOW RESOURCES wildWhere? sortClause? limitClause? #showResources + | SHOW WORKLOAD GROUPS wildWhere? #showWorkloadGroups + | SHOW BACKENDS #showBackends + | SHOW TRASH (ON backend=STRING_LITERAL)? #showTrash + | SHOW FRONTENDS name=identifier? #showFrontends + | SHOW REPOSITORIES #showRepositories + | SHOW SNAPSHOT ON repo=identifier wildWhere? #showSnapshot + | SHOW ALL? GRANTS #showGrants + | SHOW GRANTS FOR userIdentify #showGrantsForUser + | SHOW ROLES #showRoles + | SHOW PRIVILEGES #showPrivileges + | SHOW FULL? BUILTIN? FUNCTIONS + ((FROM | IN) database=multipartIdentifier)? wildWhere? #showFunctions + | SHOW GLOBAL FULL? FUNCTIONS wildWhere? #showGlobalFunctions + | SHOW TYPECAST ((FROM | IN) database=multipartIdentifier)? #showTypeCast + | SHOW FILE ((FROM | IN) database=multipartIdentifier)? #showSmallFiles + | SHOW (KEY | KEYS | INDEX | INDEXES) + (FROM |IN) tableName=multipartIdentifier + ((FROM | IN) database=multipartIdentifier)? #showIndex + | SHOW VIEW + (FROM |IN) tableName=multipartIdentifier + ((FROM | IN) database=multipartIdentifier)? #showView + | SHOW TRANSACTION ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTransaction + | SHOW QUERY PROFILE queryIdPath=STRING_LITERAL #showQueryProfile + | SHOW LOAD PROFILE loadIdPath=STRING_LITERAL #showLoadProfile + | SHOW CACHE HOTSPOT tablePath=STRING_LITERAL #showCacheHotSpot + | SHOW ENCRYPTKEYS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showEncryptKeys + | SHOW SYNC JOB ((FROM | IN) database=multipartIdentifier)? #showSyncJob + | SHOW TABLE CREATION ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTableCreation + | SHOW LAST INSERT #showLastInsert + | SHOW CREATE MATERIALIZED VIEW mvName=identifier + ON tableName=multipartIdentifier #showCreateMaterializedView + | SHOW CATALOG RECYCLE BIN wildWhere? #showCatalogRecycleBin + | SHOW QUERY STATS ((FOR database=identifier) + | (FROM tableName=multipartIdentifier (ALL VERBOSE?)?))? #showQueryStats + | SHOW BUILD INDEX ((FROM | IN) database=multipartIdentifier)? + wildWhere? sortClause? limitClause? #showBuildIndex + | SHOW CLUSTERS #showClusters + | SHOW CONVERT_LSC ((FROM | IN) database=multipartIdentifier)? #showConvertLsc + | SHOW REPLICA STATUS FROM baseTableRef wildWhere? #showReplicaStatus + | SHOW REPLICA DISTRIBUTION FROM baseTableRef #showREplicaDistribution + | SHOW TABLET STORAGE FORMAT VERBOSE? #showTabletStorageFormat + | SHOW TABLET DIAGNOSIS tabletId=INTEGER_VALUE #showDiagnoseTablet + | SHOW COPY ((FROM | IN) database=multipartIdentifier)? + whereClause? sortClause? limitClause? #showCopy + | SHOW WARM UP JOB wildWhere? #showWarmUpJob + ; + unsupportedLoadStatement : LOAD mysqlDataDesc (PROPERTIES LEFT_PAREN properties=propertyItemList RIGHT_PAREN)? @@ -432,6 +548,7 @@ unsupportedAlterStatement properties=propertyClause #alterStoragePlicy | ALTER USER (IF EXISTS)? grantUserIdentify passwordOption (COMMENT STRING_LITERAL)? #alterUser + | ALTER ROLE role=identifier commentSpec #alterRole | ALTER REPOSITORY name=identifier properties=propertyClause? #alterRepository ; @@ -562,6 +679,18 @@ unsupportedStatsStatement | DROP EXPIRED STATS #dropExpiredStats | DROP ANALYZE JOB INTEGER_VALUE #dropAanalyzeJob | KILL ANALYZE jobId=INTEGER_VALUE #killAnalyzeJob + | SHOW TABLE STATS tableName=multipartIdentifier + partitionSpec? columnList=identifierList? #showTableStats + | SHOW TABLE STATS tableId=INTEGER_VALUE #showTableStats + | SHOW INDEX STATS tableName=multipartIdentifier indexId=identifier #showIndexStats + | SHOW COLUMN CACHED? STATS tableName=multipartIdentifier + columnList=identifierList? partitionSpec? #showColumnStats + | SHOW COLUMN HISTOGRAM tableName=multipartIdentifier + columnList=identifierList #showColumnHistogramStats + | SHOW AUTO? ANALYZE tableName=multipartIdentifier? wildWhere? #showAnalyze + | SHOW ANALYZE jobId=INTEGER_VALUE wildWhere? #showAnalyzeFromJobId + | SHOW AUTO JOBS tableName=multipartIdentifier? wildWhere? #showAutoAnalyzeJobs + | SHOW ANALYZE TASK STATUS jobId=INTEGER_VALUE #showAnalyzeTask ; analyzeProperties @@ -1952,6 +2081,7 @@ nonReserved | VERBOSE | VERSION | VIEW + | VIEWS | WARM | WARNINGS | WEEK diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 921fdd590ed607..29a05856ff3b8e 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -718,7 +718,7 @@ nonterminal StatementBase stmt, show_stmt, show_param, help_stmt, load_stmt, use_stmt, use_cloud_cluster_stmt, kill_stmt, drop_stmt, recover_stmt, grant_stmt, revoke_stmt, create_stmt, set_stmt, sync_stmt, cancel_stmt, cancel_param, delete_stmt, switch_stmt, transaction_stmt, unsupported_stmt, export_stmt, admin_stmt, truncate_stmt, import_columns_stmt, import_delete_on_stmt, import_sequence_stmt, import_where_stmt, install_plugin_stmt, uninstall_plugin_stmt, - import_preceding_filter_stmt, unlock_tables_stmt, lock_tables_stmt, refresh_stmt, clean_stmt, analyze_stmt, show_mtmv_stmt, kill_analysis_job_stmt, insert_overwrite_stmt, copy_stmt, + import_preceding_filter_stmt, unlock_tables_stmt, lock_tables_stmt, refresh_stmt, clean_stmt, analyze_stmt, kill_analysis_job_stmt, insert_overwrite_stmt, copy_stmt, warm_up_stmt; nonterminal FromClause opt_using_clause; @@ -1272,8 +1272,6 @@ stmt ::= {: RESULT = stmt; :} | analyze_stmt:stmt {: RESULT = stmt; :} - | show_mtmv_stmt : stmt - {: RESULT = stmt; :} | warm_up_stmt:stmt {: RESULT = stmt; :} | /* empty: query only has comments */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterRoleStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterRoleStmt.java index ec428cb82f4272..f993b672220d11 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterRoleStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterRoleStmt.java @@ -26,7 +26,7 @@ import com.google.common.base.Strings; -public class AlterRoleStmt extends DdlStmt { +public class AlterRoleStmt extends DdlStmt implements NotFallbackInParser { private String role; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeStmt.java index 9b5145a7d0e9c3..495539842b49f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeStmt.java @@ -30,7 +30,7 @@ import java.util.Map; -public class AnalyzeStmt extends StatementBase { +public class AnalyzeStmt extends StatementBase implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(AnalyzeStmt.class); protected AnalyzeProperties analyzeProperties; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoleStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoleStmt.java index 1f64ceddb85559..336bcc50406cde 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoleStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoleStmt.java @@ -30,7 +30,7 @@ import com.google.common.base.Strings; import org.apache.commons.lang3.StringUtils; -public class CreateRoleStmt extends DdlStmt { +public class CreateRoleStmt extends DdlStmt implements NotFallbackInParser { private boolean ifNotExists; private String role; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DdlStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DdlStmt.java index 596341bb37479c..38533bd49d4940 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DdlStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DdlStmt.java @@ -17,7 +17,7 @@ package org.apache.doris.analysis; -public abstract class DdlStmt extends StatementBase { +public abstract class DdlStmt extends StatementBase implements NotFallbackInParser { @Override public RedirectStatus getRedirectStatus() { return RedirectStatus.FORWARD_WITH_SYNC; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DiagnoseTabletStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DiagnoseTabletStmt.java index b86cb30199e51e..4b5e5c6979c7c5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DiagnoseTabletStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DiagnoseTabletStmt.java @@ -30,7 +30,7 @@ import com.google.common.collect.ImmutableList; // SHOW TABLET DIAGNOSIS tablet_id -public class DiagnoseTabletStmt extends ShowStmt { +public class DiagnoseTabletStmt extends ShowStmt implements NotFallbackInParser { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("Item").add("Info").add("Suggestion") .build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java index e9b3e6fe794a2c..f9bb218744fc85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java @@ -30,7 +30,7 @@ // drop user cmy@['domain']; // drop user cmy <==> drop user cmy@'%' // drop user cmy@'192.168.1.%' -public class DropUserStmt extends DdlStmt { +public class DropUserStmt extends DdlStmt implements NotFallbackInParser { private boolean ifExists; private UserIdentity userIdent; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java index 232a61f1edc019..a9ce85b2d3e078 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java @@ -60,7 +60,7 @@ // [PROPERTIES("key"="value")] // WITH BROKER 'broker_name' [( $broker_attrs)] @Getter -public class ExportStmt extends StatementBase { +public class ExportStmt extends StatementBase implements NotFallbackInParser { public static final String PARALLELISM = "parallelism"; public static final String LABEL = "label"; public static final String DATA_CONSISTENCY = "data_consistency"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java index 4041fa4873b5be..a1d89e12ec43b0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java @@ -645,11 +645,7 @@ private String paramsToSql() { && (fnName.getFunction().equalsIgnoreCase("aes_decrypt") || fnName.getFunction().equalsIgnoreCase("aes_encrypt") || fnName.getFunction().equalsIgnoreCase("sm4_decrypt") - || fnName.getFunction().equalsIgnoreCase("sm4_encrypt") - || fnName.getFunction().equalsIgnoreCase("aes_decrypt_v2") - || fnName.getFunction().equalsIgnoreCase("aes_encrypt_v2") - || fnName.getFunction().equalsIgnoreCase("sm4_decrypt_v2") - || fnName.getFunction().equalsIgnoreCase("sm4_encrypt_v2"))) { + || fnName.getFunction().equalsIgnoreCase("sm4_encrypt"))) { sb.append("\'***\'"); } else if (orderByElements.size() > 0 && i == len - orderByElements.size()) { sb.append("ORDER BY "); @@ -734,22 +730,13 @@ private String paramsToDigest() { if (fnName.getFunction().equalsIgnoreCase("aes_decrypt") || fnName.getFunction().equalsIgnoreCase("aes_encrypt") || fnName.getFunction().equalsIgnoreCase("sm4_decrypt") - || fnName.getFunction().equalsIgnoreCase("sm4_encrypt") - || fnName.getFunction().equalsIgnoreCase("aes_decrypt_v2") - || fnName.getFunction().equalsIgnoreCase("aes_encrypt_v2") - || fnName.getFunction().equalsIgnoreCase("sm4_decrypt_v2") - || fnName.getFunction().equalsIgnoreCase("sm4_encrypt_v2")) { + || fnName.getFunction().equalsIgnoreCase("sm4_encrypt")) { len = len - 1; } for (int i = 0; i < len; ++i) { if (i == 1 && (fnName.getFunction().equalsIgnoreCase("aes_decrypt") || fnName.getFunction().equalsIgnoreCase("aes_encrypt") - || fnName.getFunction().equalsIgnoreCase("sm4_decrypt") - || fnName.getFunction().equalsIgnoreCase("sm4_encrypt") - || fnName.getFunction().equalsIgnoreCase("aes_decrypt_v2") - || fnName.getFunction().equalsIgnoreCase("aes_encrypt_v2") - || fnName.getFunction().equalsIgnoreCase("sm4_decrypt_v2") - || fnName.getFunction().equalsIgnoreCase("sm4_encrypt_v2"))) { + || fnName.getFunction().equalsIgnoreCase("sm4_decrypt"))) { result.add("\'***\'"); } else { result.add(children.get(i).toDigest()); @@ -1152,13 +1139,8 @@ private void analyzeBuiltinAggFunction(Analyzer analyzer) throws AnalysisExcepti if ((fnName.getFunction().equalsIgnoreCase("aes_decrypt") || fnName.getFunction().equalsIgnoreCase("aes_encrypt") || fnName.getFunction().equalsIgnoreCase("sm4_decrypt") - || fnName.getFunction().equalsIgnoreCase("sm4_encrypt") - || fnName.getFunction().equalsIgnoreCase("aes_decrypt_v2") - || fnName.getFunction().equalsIgnoreCase("aes_encrypt_v2") - || fnName.getFunction().equalsIgnoreCase("sm4_decrypt_v2") - || fnName.getFunction().equalsIgnoreCase("sm4_encrypt_v2")) + || fnName.getFunction().equalsIgnoreCase("sm4_encrypt")) && (children.size() == 2 || children.size() == 3)) { - String blockEncryptionMode = ""; Set aesModes = new HashSet<>(Arrays.asList( "AES_128_ECB", "AES_192_ECB", @@ -1192,43 +1174,20 @@ private void analyzeBuiltinAggFunction(Analyzer analyzer) throws AnalysisExcepti "SM4_128_OFB", "SM4_128_CTR")); + String blockEncryptionMode = ""; if (ConnectContext.get() != null) { blockEncryptionMode = ConnectContext.get().getSessionVariable().getBlockEncryptionMode(); if (fnName.getFunction().equalsIgnoreCase("aes_decrypt") - || fnName.getFunction().equalsIgnoreCase("aes_encrypt") - || fnName.getFunction().equalsIgnoreCase("aes_decrypt_v2") - || fnName.getFunction().equalsIgnoreCase("aes_encrypt_v2")) { + || fnName.getFunction().equalsIgnoreCase("aes_encrypt")) { if (StringUtils.isAllBlank(blockEncryptionMode)) { blockEncryptionMode = "AES_128_ECB"; } if (!aesModes.contains(blockEncryptionMode.toUpperCase())) { throw new AnalysisException("session variable block_encryption_mode is invalid with aes"); } - if (children.size() == 2) { - boolean isECB = blockEncryptionMode.equalsIgnoreCase("AES_128_ECB") - || blockEncryptionMode.equalsIgnoreCase("AES_192_ECB") - || blockEncryptionMode.equalsIgnoreCase("AES_256_ECB"); - if (fnName.getFunction().equalsIgnoreCase("aes_decrypt_v2")) { - if (!isECB) { - throw new AnalysisException( - "Incorrect parameter count in the call to native function 'aes_decrypt'"); - } - } else if (fnName.getFunction().equalsIgnoreCase("aes_encrypt_v2")) { - if (!isECB) { - throw new AnalysisException( - "Incorrect parameter count in the call to native function 'aes_encrypt'"); - } - } else { - // if there are only 2 params, we need set encryption mode to AES_128_ECB - // this keeps the behavior consistent with old doris ver. - blockEncryptionMode = "AES_128_ECB"; - } - } } if (fnName.getFunction().equalsIgnoreCase("sm4_decrypt") - || fnName.getFunction().equalsIgnoreCase("sm4_encrypt") - || fnName.getFunction().equalsIgnoreCase("sm4_decrypt_v2") - || fnName.getFunction().equalsIgnoreCase("sm4_encrypt_v2")) { + || fnName.getFunction().equalsIgnoreCase("sm4_encrypt")) { if (StringUtils.isAllBlank(blockEncryptionMode)) { blockEncryptionMode = "SM4_128_ECB"; } @@ -1236,36 +1195,12 @@ private void analyzeBuiltinAggFunction(Analyzer analyzer) throws AnalysisExcepti throw new AnalysisException( "session variable block_encryption_mode is invalid with sm4"); } - if (children.size() == 2) { - if (fnName.getFunction().equalsIgnoreCase("sm4_decrypt_v2")) { - throw new AnalysisException( - "Incorrect parameter count in the call to native function 'sm4_decrypt'"); - } else if (fnName.getFunction().equalsIgnoreCase("sm4_encrypt_v2")) { - throw new AnalysisException( - "Incorrect parameter count in the call to native function 'sm4_encrypt'"); - } else { - // if there are only 2 params, we need add an empty string as the third param - // and set encryption mode to SM4_128_ECB - // this keeps the behavior consistent with old doris ver. - children.add(new StringLiteral("")); - blockEncryptionMode = "SM4_128_ECB"; - } - } } + } else { + throw new AnalysisException("cannot get session variable `block_encryption_mode`, " + + "please explicitly specify by using 4-args function"); } - if (!blockEncryptionMode.equals(children.get(children.size() - 1).toString())) { - children.add(new StringLiteral(blockEncryptionMode)); - } - - if (fnName.getFunction().equalsIgnoreCase("aes_decrypt_v2")) { - fnName = FunctionName.createBuiltinName("aes_decrypt"); - } else if (fnName.getFunction().equalsIgnoreCase("aes_encrypt_v2")) { - fnName = FunctionName.createBuiltinName("aes_encrypt"); - } else if (fnName.getFunction().equalsIgnoreCase("sm4_decrypt_v2")) { - fnName = FunctionName.createBuiltinName("sm4_decrypt"); - } else if (fnName.getFunction().equalsIgnoreCase("sm4_encrypt_v2")) { - fnName = FunctionName.createBuiltinName("sm4_encrypt"); - } + children.add(new StringLiteral(blockEncryptionMode)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertOverwriteTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertOverwriteTableStmt.java index bef4a38bc3eb19..a0fca62e2848be 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertOverwriteTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertOverwriteTableStmt.java @@ -34,7 +34,7 @@ import java.util.ArrayList; import java.util.List; -public class InsertOverwriteTableStmt extends DdlStmt { +public class InsertOverwriteTableStmt extends DdlStmt implements NotFallbackInParser { private final InsertTarget target; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java index b7b778ae9012ea..d9f8e00c292bf0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java @@ -80,7 +80,7 @@ * clauses. */ @Deprecated -public class SelectStmt extends QueryStmt { +public class SelectStmt extends QueryStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(SelectStmt.class); public static final String DEFAULT_VALUE = "__DEFAULT_VALUE__"; private UUID id = UUID.randomUUID(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetDefaultStorageVaultStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetDefaultStorageVaultStmt.java index 26687942d0298b..3c3fc6f1863645 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetDefaultStorageVaultStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetDefaultStorageVaultStmt.java @@ -29,7 +29,7 @@ import org.apache.doris.qe.ConnectContext; // SET vault_name DEFAULT STORAGE VAULT -public class SetDefaultStorageVaultStmt extends DdlStmt { +public class SetDefaultStorageVaultStmt extends DdlStmt implements NotFallbackInParser { public final String vaultName; public SetDefaultStorageVaultStmt(String vaultName) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java index 3c4f8a76fd549d..2746da4d8fd595 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java @@ -53,7 +53,7 @@ * and we need to mark the slots of resolved exprs as materialized. */ @Deprecated -public class SetOperationStmt extends QueryStmt { +public class SetOperationStmt extends QueryStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(SetOperationStmt.class); public enum Operation { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetUserPropertyStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetUserPropertyStmt.java index 7342fe7a79c2a2..04247fb227f430 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetUserPropertyStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetUserPropertyStmt.java @@ -27,7 +27,7 @@ import java.util.List; -public class SetUserPropertyStmt extends DdlStmt { +public class SetUserPropertyStmt extends DdlStmt implements NotFallbackInParser { private String user; private final List propertyList; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java index 466d958ab19e35..8040271005ad83 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java @@ -48,7 +48,7 @@ * SHOW ALTER TABLE [COLUMN | ROLLUP] [FROM dbName] [WHERE TableName="xxx"] * [ORDER BY CreateTime DESC] [LIMIT [offset,]rows] */ -public class ShowAlterStmt extends ShowStmt { +public class ShowAlterStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowAlterStmt.class); public static enum AlterType { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeStmt.java index f015cdb0555215..56e16bba76a0d8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeStmt.java @@ -45,7 +45,7 @@ * [ORDER BY ...] * [LIMIT limit]; */ -public class ShowAnalyzeStmt extends ShowStmt { +public class ShowAnalyzeStmt extends ShowStmt implements NotFallbackInParser { private static final String STATE_NAME = "state"; private static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("job_id") diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeTaskStatus.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeTaskStatus.java index 364e3d0a4f3918..ff52cf0ccbaa40 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeTaskStatus.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAnalyzeTaskStatus.java @@ -26,7 +26,7 @@ /** * SHOW ANALYZE TASK STATUS [JOB_ID] */ -public class ShowAnalyzeTaskStatus extends ShowStmt { +public class ShowAnalyzeTaskStatus extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData ROW_META_DATA = ShowResultSetMetaData.builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAuthorStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAuthorStmt.java index 477efd694d274f..f82f6022d4166f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAuthorStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAuthorStmt.java @@ -22,7 +22,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // Used to test show executor. -public class ShowAuthorStmt extends ShowStmt { +public class ShowAuthorStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Name", ScalarType.createVarchar(30))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAutoAnalyzeJobsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAutoAnalyzeJobsStmt.java index 560387fa5bc11c..9b07796df784d4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAutoAnalyzeJobsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAutoAnalyzeJobsStmt.java @@ -43,7 +43,7 @@ * [PRIORITY = ["HIGH"|"MID"|"LOW"]] * ] */ -public class ShowAutoAnalyzeJobsStmt extends ShowStmt { +public class ShowAutoAnalyzeJobsStmt extends ShowStmt implements NotFallbackInParser { private static final String PRIORITY = "priority"; private static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("catalog_name") diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackendsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackendsStmt.java index 46009bd1c09798..8e1e589477d2dc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackendsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackendsStmt.java @@ -28,7 +28,7 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; -public class ShowBackendsStmt extends ShowStmt { +public class ShowBackendsStmt extends ShowStmt implements NotFallbackInParser { public ShowBackendsStmt() { } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackupStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackupStmt.java index 3c06a1da29eecd..a76857a8f8bc58 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackupStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackupStmt.java @@ -37,7 +37,7 @@ import java.util.function.Predicate; -public class ShowBackupStmt extends ShowStmt { +public class ShowBackupStmt extends ShowStmt implements NotFallbackInParser { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("JobId").add("SnapshotName").add("DbName").add("State").add("BackupObjs").add("CreateTime") .add("SnapshotFinishedTime").add("UploadFinishedTime").add("FinishedTime").add("UnfinishedTasks") diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBrokerStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBrokerStmt.java index 2ef68cd149e7d2..9f2b4d4ad5f2b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBrokerStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBrokerStmt.java @@ -28,7 +28,7 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; -public class ShowBrokerStmt extends ShowStmt { +public class ShowBrokerStmt extends ShowStmt implements NotFallbackInParser { public ShowBrokerStmt() { } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java index 5b02e306aa9deb..944365fb0d48df 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java @@ -45,7 +45,7 @@ // syntax: // SHOW BUILD INDEX [FROM db] [WHERE ] // [ORDER BY [DESC|ASC] [NULLS LAST | NULLS FIRST]]] [ LIMIT 1, 100] -public class ShowBuildIndexStmt extends ShowStmt { +public class ShowBuildIndexStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowBuildIndexStmt.class); private String dbName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCacheHotSpotStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCacheHotSpotStmt.java index 23b72938e96674..847b015825dc1c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCacheHotSpotStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCacheHotSpotStmt.java @@ -40,7 +40,7 @@ import java.util.Arrays; import java.util.List; -public class ShowCacheHotSpotStmt extends ShowStmt { +public class ShowCacheHotSpotStmt extends ShowStmt implements NotFallbackInParser { public static final ShowResultSetMetaData[] RESULT_SET_META_DATAS = { ShowResultSetMetaData.builder() .addColumn(new Column("cluster_id", ScalarType.createType(PrimitiveType.BIGINT))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogRecycleBinStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogRecycleBinStmt.java index f15c3657240d87..3a4a33e4d628fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogRecycleBinStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogRecycleBinStmt.java @@ -36,7 +36,7 @@ import java.util.function.Predicate; -public class ShowCatalogRecycleBinStmt extends ShowStmt { +public class ShowCatalogRecycleBinStmt extends ShowStmt implements NotFallbackInParser { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("Type").add("Name").add("DbId").add("TableId").add("PartitionId").add("DropTime") .add("DataSize").add("RemoteDataSize").build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogStmt.java index 191c5ba9a40ed5..f1770859002c7d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogStmt.java @@ -25,7 +25,7 @@ /** * Statement for show all catalog or desc the specific catalog. */ -public class ShowCatalogStmt extends ShowStmt { +public class ShowCatalogStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA_ALL = ShowResultSetMetaData.builder().addColumn(new Column("CatalogId", ScalarType.BIGINT)) .addColumn(new Column("CatalogName", ScalarType.createVarchar(64))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCharsetStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCharsetStmt.java index 07e78fdf9cec8b..50725f690445a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCharsetStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCharsetStmt.java @@ -21,7 +21,7 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.qe.ShowResultSetMetaData; -public class ShowCharsetStmt extends ShowStmt { +public class ShowCharsetStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Charset", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCloudWarmUpStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCloudWarmUpStmt.java index d34c46d54a5d31..e91e9b7d6fe251 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCloudWarmUpStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCloudWarmUpStmt.java @@ -28,7 +28,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -public class ShowCloudWarmUpStmt extends ShowStmt { +public class ShowCloudWarmUpStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowCloudWarmUpStmt.class); private Expr whereClause; private boolean showAllJobs = false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowClusterStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowClusterStmt.java index 50bd36b1882767..acb6d789f45e59 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowClusterStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowClusterStmt.java @@ -33,7 +33,7 @@ import com.google.common.collect.ImmutableList; -public class ShowClusterStmt extends ShowStmt { +public class ShowClusterStmt extends ShowStmt implements NotFallbackInParser { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("cluster").add("is_current").add("users").build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCollationStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCollationStmt.java index df7e1f6a1637f3..6fda6ab9220c64 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCollationStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCollationStmt.java @@ -22,7 +22,7 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.qe.ShowResultSetMetaData; -public class ShowCollationStmt extends ShowStmt { +public class ShowCollationStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Collation", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnHistStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnHistStmt.java index 88eb244589621b..ae5010ffe0d72f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnHistStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnHistStmt.java @@ -45,7 +45,7 @@ import java.util.Set; import java.util.stream.Collectors; -public class ShowColumnHistStmt extends ShowStmt { +public class ShowColumnHistStmt extends ShowStmt implements NotFallbackInParser { private static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java index 833ca5e27a9784..354b57bc55c7d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java @@ -50,7 +50,7 @@ import java.util.Set; import java.util.stream.Collectors; -public class ShowColumnStatsStmt extends ShowStmt { +public class ShowColumnStatsStmt extends ShowStmt implements NotFallbackInParser { private static final ImmutableList TABLE_COLUMN_TITLE_NAMES = new ImmutableList.Builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStmt.java index 9af269104cc993..cf9c4a27660ec7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStmt.java @@ -32,7 +32,7 @@ import com.google.common.collect.Lists; // SHOW COLUMNS -public class ShowColumnStmt extends ShowStmt { +public class ShowColumnStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Field", ScalarType.createVarchar(20))) .addColumn(new Column("Type", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowConvertLSCStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowConvertLSCStmt.java index bab2e3b34b8847..cedf3e2d2466e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowConvertLSCStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowConvertLSCStmt.java @@ -27,7 +27,7 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; -public class ShowConvertLSCStmt extends ShowStmt { +public class ShowConvertLSCStmt extends ShowStmt implements NotFallbackInParser { private final String dbName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCopyStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCopyStmt.java index 0d4ba6a909e42c..50988fd6a47493 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCopyStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCopyStmt.java @@ -38,7 +38,7 @@ // // syntax: // SHOW COPY [FROM db] [LIKE mask] -public class ShowCopyStmt extends ShowLoadStmt { +public class ShowCopyStmt extends ShowLoadStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowCopyStmt.class); public ShowCopyStmt(String db, Expr labelExpr, List orderByElements, LimitElement limitElement) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateCatalogStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateCatalogStmt.java index 45208394eaf17b..f324b0701a8029 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateCatalogStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateCatalogStmt.java @@ -32,7 +32,7 @@ // Show create catalog statement // Syntax: // SHOW CREATE CATALOG catalogName -public class ShowCreateCatalogStmt extends ShowStmt { +public class ShowCreateCatalogStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Catalog", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java index b709be8d7e95c2..99cd1b3343823f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java @@ -33,7 +33,7 @@ // Show create database statement // Syntax: // SHOW CREATE DATABASE db -public class ShowCreateDbStmt extends ShowStmt { +public class ShowCreateDbStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Database", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateFunctionStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateFunctionStmt.java index e1ff16ba8ddc8f..b65f9318e58d4e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateFunctionStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateFunctionStmt.java @@ -33,7 +33,7 @@ import com.google.common.base.Strings; -public class ShowCreateFunctionStmt extends ShowStmt { +public class ShowCreateFunctionStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Function Signature", ScalarType.createVarchar(256))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateMTMVStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateMTMVStmt.java index fb8e69e779a70e..215217f63bd292 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateMTMVStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateMTMVStmt.java @@ -30,7 +30,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // SHOW CREATE Materialized View statement. -public class ShowCreateMTMVStmt extends ShowStmt { +public class ShowCreateMTMVStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Materialized View", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateMaterializedViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateMaterializedViewStmt.java index f0c48520a5c64c..ad5469809cf41b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateMaterializedViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateMaterializedViewStmt.java @@ -36,7 +36,7 @@ **/ @AllArgsConstructor @Getter -public class ShowCreateMaterializedViewStmt extends ShowStmt { +public class ShowCreateMaterializedViewStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateRepositoryStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateRepositoryStmt.java index 9de7dd0e9eed72..efd436ea240639 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateRepositoryStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateRepositoryStmt.java @@ -28,7 +28,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // SHOW CREATE REPOSITORY statement -public class ShowCreateRepositoryStmt extends ShowStmt { +public class ShowCreateRepositoryStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateTableStmt.java index 1a8ec38080ff84..9b60417c721b0f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateTableStmt.java @@ -31,7 +31,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // SHOW CREATE TABLE statement. -public class ShowCreateTableStmt extends ShowStmt { +public class ShowCreateTableStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Table", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataSkewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataSkewStmt.java index c1ccc521c3bf94..6db5aeaa9da264 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataSkewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataSkewStmt.java @@ -31,7 +31,7 @@ import com.google.common.collect.ImmutableList; // show data skew from tbl [partition(p1, p2, ...)] -public class ShowDataSkewStmt extends ShowStmt { +public class ShowDataSkewStmt extends ShowStmt implements NotFallbackInParser { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("PartitionName").add("BucketIdx").add("AvgRowCount").add("AvgDataSize") .add("Graph").add("Percent") diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java index 84ce67283ac487..26b77e0b3d5d26 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java @@ -56,7 +56,7 @@ import java.util.TreeMap; import java.util.TreeSet; -public class ShowDataStmt extends ShowStmt { +public class ShowDataStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData SHOW_DATABASE_DATA_META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("DbId", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataTypesStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataTypesStmt.java index 0ee95dd817e9f7..31bc8f3f598b96 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataTypesStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataTypesStmt.java @@ -31,7 +31,7 @@ import java.util.Comparator; import java.util.List; -public class ShowDataTypesStmt extends ShowStmt { +public class ShowDataTypesStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDbIdStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDbIdStmt.java index 967714e0fa2546..6e800274817e8a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDbIdStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDbIdStmt.java @@ -28,7 +28,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // SHOW DATABASE ID -public class ShowDbIdStmt extends ShowStmt { +public class ShowDbIdStmt extends ShowStmt implements NotFallbackInParser { private long dbId; public ShowDbIdStmt(long dbId) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDbStmt.java index c9606bc5bd224a..aba12fbd0c6c6d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDbStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDbStmt.java @@ -29,7 +29,7 @@ import com.google.common.collect.Lists; // Show database statement. -public class ShowDbStmt extends ShowStmt { +public class ShowDbStmt extends ShowStmt implements NotFallbackInParser { private static final TableName TABLE_NAME = new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, InfoSchemaDb.DATABASE_NAME, "schemata"); private static final String DB_COL = "Database"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDeleteStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDeleteStmt.java index 85df05718998b8..dcf36d116094c8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDeleteStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDeleteStmt.java @@ -28,7 +28,7 @@ import com.google.common.base.Strings; -public class ShowDeleteStmt extends ShowStmt { +public class ShowDeleteStmt extends ShowStmt implements NotFallbackInParser { private String dbName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDynamicPartitionStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDynamicPartitionStmt.java index d8b0c1779214d7..ef1d731d77b078 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDynamicPartitionStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDynamicPartitionStmt.java @@ -26,7 +26,7 @@ import com.google.common.base.Strings; -public class ShowDynamicPartitionStmt extends ShowStmt { +public class ShowDynamicPartitionStmt extends ShowStmt implements NotFallbackInParser { private String db; private static final ShowResultSetMetaData SHOW_DYNAMIC_PARTITION_META_DATA = ShowResultSetMetaData.builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEncryptKeysStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEncryptKeysStmt.java index 83358209c23539..44ebc7b2b70dea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEncryptKeysStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEncryptKeysStmt.java @@ -30,7 +30,7 @@ import com.google.common.base.Strings; -public class ShowEncryptKeysStmt extends ShowStmt { +public class ShowEncryptKeysStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("EncryptKey Name", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEnginesStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEnginesStmt.java index f6ff28a671c93e..86ac4af67fb001 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEnginesStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEnginesStmt.java @@ -21,7 +21,7 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.qe.ShowResultSetMetaData; -public class ShowEnginesStmt extends ShowStmt { +public class ShowEnginesStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Engine", ScalarType.createVarchar(64))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEventsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEventsStmt.java index 4e47ac79dec5f0..97ac779cd28729 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEventsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEventsStmt.java @@ -22,7 +22,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // Show Events statement -public class ShowEventsStmt extends ShowStmt { +public class ShowEventsStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Db", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowExportStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowExportStmt.java index 14b5d05537c298..757eaf83bacf36 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowExportStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowExportStmt.java @@ -40,7 +40,7 @@ // // syntax: // SHOW EXPORT [FROM db] [where ...] -public class ShowExportStmt extends ShowStmt { +public class ShowExportStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowExportStmt.class); private String dbName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowFrontendsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowFrontendsStmt.java index b3b5808d945d92..d8b5c454ff9d08 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowFrontendsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowFrontendsStmt.java @@ -30,7 +30,7 @@ import com.google.common.collect.ImmutableList; -public class ShowFrontendsStmt extends ShowStmt { +public class ShowFrontendsStmt extends ShowStmt implements NotFallbackInParser { private String detail; public ShowFrontendsStmt() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowFunctionsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowFunctionsStmt.java index 5abd18f74631f5..a994d0f6463c4f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowFunctionsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowFunctionsStmt.java @@ -32,7 +32,7 @@ import com.google.common.base.Strings; -public class ShowFunctionsStmt extends ShowStmt { +public class ShowFunctionsStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Signature", ScalarType.createVarchar(256))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowGrantsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowGrantsStmt.java index f78d4ca59fa682..0439544ae0e3a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowGrantsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowGrantsStmt.java @@ -43,7 +43,7 @@ // // SHOW GRANTS; // SHOW GRANTS FOR user@'xxx' -public class ShowGrantsStmt extends ShowStmt { +public class ShowGrantsStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowIndexStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowIndexStmt.java index b35f2e77287b5e..43f02f4426d579 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowIndexStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowIndexStmt.java @@ -30,7 +30,7 @@ import com.google.common.base.Strings; -public class ShowIndexStmt extends ShowStmt { +public class ShowIndexStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Table", ScalarType.createVarchar(64))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLastInsertStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLastInsertStmt.java index 9ed0ebc6cd7f77..4ba3521d51cbd3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLastInsertStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLastInsertStmt.java @@ -22,7 +22,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // SHOW LAST INSERT -public class ShowLastInsertStmt extends ShowStmt { +public class ShowLastInsertStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("TransactionId", ScalarType.createVarchar(128))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java index 490a521132119a..78437cec796898 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java @@ -24,7 +24,7 @@ // deprecated stmt, use will be guided to a specific url to get profile from // web browser -public class ShowLoadProfileStmt extends ShowStmt { +public class ShowLoadProfileStmt extends ShowStmt implements NotFallbackInParser { private String loadIdPath; public ShowLoadProfileStmt(String path) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadStmt.java index b08ef917ab3aa2..760f70481fcf9c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadStmt.java @@ -42,7 +42,7 @@ // // syntax: // SHOW LOAD [FROM db] [LIKE mask] -public class ShowLoadStmt extends ShowStmt { +public class ShowLoadStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowLoadStmt.class); private String dbName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadWarningsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadWarningsStmt.java index 624fc0a402b02e..f034c32a496626 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadWarningsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadWarningsStmt.java @@ -33,7 +33,7 @@ import java.net.URL; // SHOW LOAD WARNINGS statement used to get error detail of src data. -public class ShowLoadWarningsStmt extends ShowStmt { +public class ShowLoadWarningsStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowLoadWarningsStmt.class); private static final ShowResultSetMetaData META_DATA = diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowOpenTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowOpenTableStmt.java index 7c7dd8a9253a7f..3c2ae9f3795045 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowOpenTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowOpenTableStmt.java @@ -22,7 +22,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // SHOW OPEN TABLES -public class ShowOpenTableStmt extends ShowStmt { +public class ShowOpenTableStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Database", ScalarType.createVarchar(64))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionIdStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionIdStmt.java index 5aca99da337539..793a886ff31e9b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionIdStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionIdStmt.java @@ -28,7 +28,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // SHOW PARTITION ID -public class ShowPartitionIdStmt extends ShowStmt { +public class ShowPartitionIdStmt extends ShowStmt implements NotFallbackInParser { private long partitionId; public ShowPartitionIdStmt(long partitionId) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java index 1c61018bc53ef4..0be41ef60fa7ba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java @@ -54,7 +54,7 @@ import java.util.Map; import java.util.Objects; -public class ShowPartitionsStmt extends ShowStmt { +public class ShowPartitionsStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowPartitionsStmt.class); private static final String FILTER_PARTITION_ID = "PartitionId"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPluginsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPluginsStmt.java index 249e49da28dbaf..dc04e77d2ac7f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPluginsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPluginsStmt.java @@ -29,7 +29,7 @@ // Show plugins statement. // TODO(zhaochun): only for support MySQL -public class ShowPluginsStmt extends ShowStmt { +public class ShowPluginsStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Name", ScalarType.createVarchar(64))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPolicyStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPolicyStmt.java index df413ee61e465c..4c471fbcc42c3b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPolicyStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPolicyStmt.java @@ -36,7 +36,7 @@ * syntax: * SHOW ROW POLICY [FOR user|ROLE role] **/ -public class ShowPolicyStmt extends ShowStmt { +public class ShowPolicyStmt extends ShowStmt implements NotFallbackInParser { @Getter private final PolicyTypeEnum type; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPrivilegesStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPrivilegesStmt.java index 31f2d0edeab7f3..a053c9e327f130 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPrivilegesStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPrivilegesStmt.java @@ -21,7 +21,7 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.qe.ShowResultSetMetaData; -public class ShowPrivilegesStmt extends ShowStmt { +public class ShowPrivilegesStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA; static { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcStmt.java index bc7c70cdc79257..0d154db62326f7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcStmt.java @@ -31,7 +31,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // SHOW PROC statement. Used to show proc information, only admin can use. -public class ShowProcStmt extends ShowStmt { +public class ShowProcStmt extends ShowStmt implements NotFallbackInParser { private String path; private ProcNodeInterface node; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcedureStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcedureStmt.java index db7684591fd752..960f54a2a8c2d3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcedureStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcedureStmt.java @@ -22,7 +22,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // SHOW PROCEDURE STATUS -public class ShowProcedureStmt extends ShowStmt { +public class ShowProcedureStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Db", ScalarType.createVarchar(64))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcesslistStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcesslistStmt.java index e602e85f499105..a4fd6a183c0678 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcesslistStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcesslistStmt.java @@ -25,7 +25,7 @@ // SHOW PROCESSLIST statement. // Used to show connection belong to this user. -public class ShowProcesslistStmt extends ShowStmt { +public class ShowProcesslistStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("CurrentConnected", ScalarType.createVarchar(16))) .addColumn(new Column("Id", ScalarType.createType(PrimitiveType.BIGINT))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java index 39d077438264bb..ad664652d53343 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java @@ -24,7 +24,7 @@ // deprecated stmt, use will be guided to a specific url to get profile from // web browser -public class ShowQueryProfileStmt extends ShowStmt { +public class ShowQueryProfileStmt extends ShowStmt implements NotFallbackInParser { private String queryIdPath; public ShowQueryProfileStmt(String queryIdPath) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryStatsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryStatsStmt.java index fee9809f96e36f..723a0ef8629a6c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryStatsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryStatsStmt.java @@ -39,7 +39,7 @@ import java.util.List; import java.util.Map; -public class ShowQueryStatsStmt extends ShowStmt { +public class ShowQueryStatsStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData SHOW_QUERY_STATS_CATALOG_META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Database", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowReplicaDistributionStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowReplicaDistributionStmt.java index 58d2ac1052f653..24ef8eddfc9a31 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowReplicaDistributionStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowReplicaDistributionStmt.java @@ -32,7 +32,7 @@ import com.google.common.collect.ImmutableList; // show replica distribution from tbl [partition(p1, p2, ...)] -public class ShowReplicaDistributionStmt extends ShowStmt { +public class ShowReplicaDistributionStmt extends ShowStmt implements NotFallbackInParser { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("BackendId").add("ReplicaNum").add("ReplicaSize") .add("NumGraph").add("NumPercent") diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowReplicaStatusStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowReplicaStatusStmt.java index 2a1f96fed96224..44215ed115008f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowReplicaStatusStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowReplicaStatusStmt.java @@ -36,7 +36,7 @@ import java.util.List; -public class ShowReplicaStatusStmt extends ShowStmt { +public class ShowReplicaStatusStmt extends ShowStmt implements NotFallbackInParser { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("TabletId").add("ReplicaId").add("BackendId").add("Version").add("LastFailedVersion") .add("LastSuccessVersion").add("CommittedVersion").add("SchemaHash").add("VersionNum") diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRepositoriesStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRepositoriesStmt.java index 6e5166a5c870b8..6dc6e3bf0bbed8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRepositoriesStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRepositoriesStmt.java @@ -29,7 +29,7 @@ import com.google.common.collect.ImmutableList; -public class ShowRepositoriesStmt extends ShowStmt { +public class ShowRepositoriesStmt extends ShowStmt implements NotFallbackInParser { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("RepoId").add("RepoName").add("CreateTime").add("IsReadOnly").add("Location") .add("Broker").add("Type").add("ErrMsg") diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowResourcesStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowResourcesStmt.java index 97393334679c21..9289d903627782 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowResourcesStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowResourcesStmt.java @@ -40,7 +40,7 @@ // // syntax: // SHOW RESOURCES [LIKE mask] -public class ShowResourcesStmt extends ShowStmt { +public class ShowResourcesStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowResourcesStmt.class); private String pattern; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRestoreStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRestoreStmt.java index 26016be7814dea..2ffa8607fbd86e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRestoreStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRestoreStmt.java @@ -38,7 +38,7 @@ import java.util.List; import java.util.function.Predicate; -public class ShowRestoreStmt extends ShowStmt { +public class ShowRestoreStmt extends ShowStmt implements NotFallbackInParser { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("JobId").add("Label").add("Timestamp").add("DbName").add("State") .add("AllowLoad").add("ReplicationNum").add("ReplicaAllocation").add("ReserveReplica") diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRolesStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRolesStmt.java index bbb66b274438ee..1f3f19a6d29202 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRolesStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRolesStmt.java @@ -27,7 +27,7 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; -public class ShowRolesStmt extends ShowStmt { +public class ShowRolesStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA; static { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRollupStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRollupStmt.java index 2684f3bcea8d4e..f7f65800b58326 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRollupStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRollupStmt.java @@ -31,7 +31,7 @@ // // Syntax: // SHOW ROLLUP { FROM | IN } table [ FROM db ] -public class ShowRollupStmt extends ShowStmt { +public class ShowRollupStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Table", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSmallFilesStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSmallFilesStmt.java index ad8c57b7a47734..5a227f96eb4ca3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSmallFilesStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSmallFilesStmt.java @@ -30,7 +30,7 @@ import com.google.common.base.Strings; -public class ShowSmallFilesStmt extends ShowStmt { +public class ShowSmallFilesStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Id", ScalarType.createVarchar(32))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSnapshotStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSnapshotStmt.java index 83465a34600587..6a736499d8725d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSnapshotStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSnapshotStmt.java @@ -32,7 +32,7 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; -public class ShowSnapshotStmt extends ShowStmt { +public class ShowSnapshotStmt extends ShowStmt implements NotFallbackInParser { public enum SnapshotType { REMOTE, LOCAL diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSqlBlockRuleStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSqlBlockRuleStmt.java index c487e1c7978b30..ccf2b1ee6bf3df 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSqlBlockRuleStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSqlBlockRuleStmt.java @@ -36,7 +36,7 @@ show sql_block_rule show sql_block_rule for rule_name */ -public class ShowSqlBlockRuleStmt extends ShowStmt { +public class ShowSqlBlockRuleStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStageStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStageStmt.java index 09626911c1e328..eac5b4aec83f3e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStageStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStageStmt.java @@ -30,7 +30,7 @@ // // syntax: // SHOW STAGES -public class ShowStageStmt extends ShowStmt { +public class ShowStageStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowStageStmt.class); private static final String NAME_COL = "StageName"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStatusStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStatusStmt.java index a6276a88d4ee30..abce39aec6f0ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStatusStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStatusStmt.java @@ -23,7 +23,7 @@ // Show Status statement // TODO(zhaochun): Add status information. -public class ShowStatusStmt extends ShowStmt { +public class ShowStatusStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Variable_name", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStmt.java index 9be50ee9dbd1f2..92ed01acdfc81c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStmt.java @@ -20,7 +20,7 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.qe.ShowResultSetMetaData; -public abstract class ShowStmt extends StatementBase { +public abstract class ShowStmt extends StatementBase implements NotFallbackInParser { public abstract ShowResultSetMetaData getMetaData(); public SelectStmt toSelectStmt(Analyzer analyzer) throws AnalysisException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStoragePolicyUsingStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStoragePolicyUsingStmt.java index 56598a194eec12..b6a8e367124833 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStoragePolicyUsingStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStoragePolicyUsingStmt.java @@ -34,7 +34,7 @@ * syntax: * SHOW STORAGE POLICY USING [for policy_name] **/ -public class ShowStoragePolicyUsingStmt extends ShowStmt { +public class ShowStoragePolicyUsingStmt extends ShowStmt implements NotFallbackInParser { public static final ShowResultSetMetaData RESULT_META_DATA = ShowResultSetMetaData.builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStorageVaultStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStorageVaultStmt.java index 1cd1d06a1499d1..f6124c4d20184d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStorageVaultStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStorageVaultStmt.java @@ -31,7 +31,7 @@ * syntax: * SHOW STORAGE VAULT **/ -public class ShowStorageVaultStmt extends ShowStmt { +public class ShowStorageVaultStmt extends ShowStmt implements NotFallbackInParser { private final String stmt = "SHOW STORAGE VAULT"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStreamLoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStreamLoadStmt.java index ae133aa2a23e9e..92b67e6cc3683c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStreamLoadStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowStreamLoadStmt.java @@ -41,7 +41,7 @@ // // syntax: // SHOW STREAM LOAD [FROM db] [LIKE mask] -public class ShowStreamLoadStmt extends ShowStmt { +public class ShowStreamLoadStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowStreamLoadStmt.class); public enum StreamLoadState { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSyncJobStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSyncJobStmt.java index f0671f8afe0619..17a31fa1e16a6e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSyncJobStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSyncJobStmt.java @@ -37,7 +37,7 @@ // // syntax: // SHOW SYNC JOB [FROM db] -public class ShowSyncJobStmt extends ShowStmt { +public class ShowSyncJobStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowSyncJobStmt.class); public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableCreationStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableCreationStmt.java index d6bd9d0ea647ca..500265d45c6526 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableCreationStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableCreationStmt.java @@ -32,7 +32,7 @@ * Syntax: * SHOW TABLE CREATION [FROM db] [LIKE mask] */ -public class ShowTableCreationStmt extends ShowStmt { +public class ShowTableCreationStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableIdStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableIdStmt.java index fc07f1c672219c..d5830489645750 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableIdStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableIdStmt.java @@ -29,7 +29,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // SHOW TABLE ID -public class ShowTableIdStmt extends ShowStmt { +public class ShowTableIdStmt extends ShowStmt implements NotFallbackInParser { private long tableId; public ShowTableIdStmt(long tableId) { 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 8ecf8d86b0985c..54c8f652e125fa 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 @@ -49,7 +49,7 @@ import java.util.Map; import java.util.Set; -public class ShowTableStatsStmt extends ShowStmt { +public class ShowTableStatsStmt extends ShowStmt implements NotFallbackInParser { private static final ImmutableList TABLE_TITLE_NAMES = new ImmutableList.Builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatusStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatusStmt.java index 20e8790826832b..296eb628da2df5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatusStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatusStmt.java @@ -34,7 +34,7 @@ import com.google.common.collect.Lists; // SHOW TABLE STATUS -public class ShowTableStatusStmt extends ShowStmt { +public class ShowTableStatusStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Name", ScalarType.createVarchar(64))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStmt.java index f3e12f1aa92e4d..92c5f34435f846 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStmt.java @@ -33,7 +33,7 @@ import org.apache.logging.log4j.Logger; // SHOW TABLES -public class ShowTableStmt extends ShowStmt { +public class ShowTableStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowTableStmt.class); private static final String NAME_COL_PREFIX = "Tables_in_"; private static final String TYPE_COL = "Table_type"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java index 71ba4e2be13a84..646d6a806393af 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java @@ -37,7 +37,7 @@ import java.util.ArrayList; import java.util.List; -public class ShowTabletStmt extends ShowStmt { +public class ShowTabletStmt extends ShowStmt implements NotFallbackInParser { private TableName dbTableName; private String dbName; private String tableName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletStorageFormatStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletStorageFormatStmt.java index 9d0f3b88e6c3b2..cf76ee504ee6f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletStorageFormatStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletStorageFormatStmt.java @@ -27,7 +27,7 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; -public class ShowTabletStorageFormatStmt extends ShowStmt { +public class ShowTabletStorageFormatStmt extends ShowStmt implements NotFallbackInParser { private boolean verbose; public ShowTabletStorageFormatStmt(boolean verbose) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletsBelongStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletsBelongStmt.java index 3819541fea9a72..81d9d6d50171f5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletsBelongStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletsBelongStmt.java @@ -36,7 +36,7 @@ * syntax: * SHOW TABLETS BELONG tablet_ids */ -public class ShowTabletsBelongStmt extends ShowStmt { +public class ShowTabletsBelongStmt extends ShowStmt implements NotFallbackInParser { private List tabletIds; private static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTransactionStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTransactionStmt.java index 3d5d03bdf8f768..cd7d2ca5b016fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTransactionStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTransactionStmt.java @@ -37,7 +37,7 @@ // syntax: // SHOW TRANSACTION WHERE id=123 -public class ShowTransactionStmt extends ShowStmt { +public class ShowTransactionStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowTransactionStmt.class); private String dbName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTrashDiskStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTrashDiskStmt.java index f5fad57d0f544e..7fb15eae6638c3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTrashDiskStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTrashDiskStmt.java @@ -32,7 +32,7 @@ import com.google.common.collect.ImmutableMap; -public class ShowTrashDiskStmt extends ShowStmt { +public class ShowTrashDiskStmt extends ShowStmt implements NotFallbackInParser { private Backend backend; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTrashStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTrashStmt.java index 3071a657c53a4c..e8e301193bcd8c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTrashStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTrashStmt.java @@ -34,7 +34,7 @@ import java.util.List; -public class ShowTrashStmt extends ShowStmt { +public class ShowTrashStmt extends ShowStmt implements NotFallbackInParser { private List backends = Lists.newArrayList(); public ShowTrashStmt() throws AnalysisException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTriggersStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTriggersStmt.java index e903d2387ee075..de9cdf76816a22 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTriggersStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTriggersStmt.java @@ -21,7 +21,7 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.qe.ShowResultSetMetaData; -public class ShowTriggersStmt extends ShowStmt { +public class ShowTriggersStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Trigger", ScalarType.createVarchar(64))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTypeCastStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTypeCastStmt.java index c88c39a03db25a..8ddf1a3d07f49c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTypeCastStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTypeCastStmt.java @@ -27,7 +27,7 @@ import com.google.common.base.Strings; -public class ShowTypeCastStmt extends ShowStmt { +public class ShowTypeCastStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Origin Type", ScalarType.createVarchar(32))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowUserPropertyStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowUserPropertyStmt.java index 7521e7497bcb15..76a9b9a9b4fa51 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowUserPropertyStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowUserPropertyStmt.java @@ -46,7 +46,7 @@ // Show Property Stmt // syntax: // SHOW [ALL] PROPERTY [FOR user] [LIKE key pattern] -public class ShowUserPropertyStmt extends ShowStmt { +public class ShowUserPropertyStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowUserPropertyStmt.class); private String user; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowVariablesStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowVariablesStmt.java index efe36311ca1a3e..f398f021dc2c69 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowVariablesStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowVariablesStmt.java @@ -28,7 +28,7 @@ import org.apache.logging.log4j.Logger; // Show variables statement. -public class ShowVariablesStmt extends ShowStmt { +public class ShowVariablesStmt extends ShowStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(ShowVariablesStmt.class); private static final String NAME_COL = "Variable_name"; private static final String VALUE_COL = "Value"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowViewStmt.java index 67c77664cccdab..f8cf26ed1bd14b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowViewStmt.java @@ -43,7 +43,7 @@ // // Syntax: // SHOW VIEW { FROM | IN } table [ FROM db ] -public class ShowViewStmt extends ShowStmt { +public class ShowViewStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("View", ScalarType.createVarchar(30))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWarningStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWarningStmt.java index 7de4525ac2d007..3018e9e56fd8cf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWarningStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWarningStmt.java @@ -22,7 +22,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; // Show Warning stmt -public class ShowWarningStmt extends ShowStmt { +public class ShowWarningStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("Level", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWhiteListStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWhiteListStmt.java index 43609d235cf8a8..42682a5780856b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWhiteListStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWhiteListStmt.java @@ -21,7 +21,7 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.qe.ShowResultSetMetaData; -public class ShowWhiteListStmt extends ShowStmt { +public class ShowWhiteListStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("user_name", ScalarType.createVarchar(20))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWorkloadGroupsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWorkloadGroupsStmt.java index c802411be81913..bb097e25a7fa77 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWorkloadGroupsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowWorkloadGroupsStmt.java @@ -25,7 +25,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; import org.apache.doris.resource.workloadgroup.WorkloadGroupMgr; -public class ShowWorkloadGroupsStmt extends ShowStmt { +public class ShowWorkloadGroupsStmt extends ShowStmt implements NotFallbackInParser { private String pattern; // TODO: not supported yet diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionStmt.java index 60c8d040f7281d..a1ad0ed2ed7a74 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionStmt.java @@ -20,7 +20,7 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; -public class TransactionStmt extends StatementBase { +public class TransactionStmt extends StatementBase implements NotFallbackInParser { @Override public RedirectStatus getRedirectStatus() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index 383a8bbc4fee3c..bcb0864b64f2e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -22,9 +22,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Abs; import org.apache.doris.nereids.trees.expressions.functions.scalar.Acos; import org.apache.doris.nereids.trees.expressions.functions.scalar.AesDecrypt; -import org.apache.doris.nereids.trees.expressions.functions.scalar.AesDecryptV2; import org.apache.doris.nereids.trees.expressions.functions.scalar.AesEncrypt; -import org.apache.doris.nereids.trees.expressions.functions.scalar.AesEncryptV2; import org.apache.doris.nereids.trees.expressions.functions.scalar.AppendTrailingCharIfAbsent; import org.apache.doris.nereids.trees.expressions.functions.scalar.Array; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayApply; @@ -370,9 +368,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm3; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm3sum; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4Decrypt; -import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4DecryptV2; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4Encrypt; -import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4EncryptV2; import org.apache.doris.nereids.trees.expressions.functions.scalar.Space; import org.apache.doris.nereids.trees.expressions.functions.scalar.SplitByChar; import org.apache.doris.nereids.trees.expressions.functions.scalar.SplitByRegexp; @@ -479,9 +475,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(Abs.class, "abs"), scalar(Acos.class, "acos"), scalar(AesDecrypt.class, "aes_decrypt"), - scalar(AesDecryptV2.class, "aes_decrypt_v2"), scalar(AesEncrypt.class, "aes_encrypt"), - scalar(AesEncryptV2.class, "aes_encrypt_v2"), scalar(AppendTrailingCharIfAbsent.class, "append_trailing_char_if_absent"), scalar(Array.class, "array"), scalar(ArrayApply.class, "array_apply"), @@ -849,9 +843,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(Sm3.class, "sm3"), scalar(Sm3sum.class, "sm3sum"), scalar(Sm4Decrypt.class, "sm4_decrypt"), - scalar(Sm4DecryptV2.class, "sm4_decrypt_v2"), scalar(Sm4Encrypt.class, "sm4_encrypt"), - scalar(Sm4EncryptV2.class, "sm4_encrypt_v2"), scalar(Space.class, "space"), scalar(SplitByChar.class, "split_by_char"), scalar(SplitByRegexp.class, "split_by_regexp"), 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 597fee4dd8a1a1..23ba86e53cd8fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -1680,10 +1680,15 @@ private void transferToMaster() { */ void advanceNextId() { long currentId = idGenerator.getBatchEndId(); - long currentNanos = System.nanoTime(); + long currentMill = System.currentTimeMillis(); long nextId = currentId + 1; - if (nextId < currentNanos) { - nextId = currentNanos; + // Reserve ~1 trillion for use in case of bugs or frequent reboots (~2 billion reboots) + if ((1L << 63) - nextId < (1L << 40)) { + LOG.warn("nextId is too large: {}, it may be a bug and consider backup and migration", nextId); + } else { + // Keep compatible with previous impl, the previous impl may result in extreme large nextId, + // and guess there are no more than 1L<<32 (~4e9) ids used since last reboot + nextId = (currentId + 1) < currentMill ? currentMill : currentId + (1L << 32); } // ATTN: Because MetaIdGenerator has guaranteed that each id it returns must have 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 01f0bb900eac24..533c24daa0e7bc 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 @@ -54,6 +54,8 @@ import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVSnapshotIf; import org.apache.doris.mtmv.MTMVVersionSnapshot; +import org.apache.doris.nereids.hint.Hint; +import org.apache.doris.nereids.hint.UseMvHint; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ConnectContext; @@ -565,6 +567,99 @@ public Map getVisibleIndexIdToMeta() { return visibleMVs; } + public Long getBestMvIdWithHint(List orderedMvs) { + Optional useMvHint = getUseMvHint("USE_MV"); + Optional noUseMvHint = getUseMvHint("NO_USE_MV"); + if (useMvHint.isPresent() && noUseMvHint.isPresent()) { + if (noUseMvHint.get().getNoUseMVName(this.name).contains(useMvHint.get().getUseMvName(this.name))) { + String errorMsg = "conflict mv exist in use_mv and no_use_mv in the same time" + + useMvHint.get().getUseMvName(this.name); + useMvHint.get().setStatus(Hint.HintStatus.SYNTAX_ERROR); + useMvHint.get().setErrorMessage(errorMsg); + noUseMvHint.get().setStatus(Hint.HintStatus.SYNTAX_ERROR); + noUseMvHint.get().setErrorMessage(errorMsg); + } + return getMvIdWithUseMvHint(useMvHint.get(), orderedMvs); + } else if (useMvHint.isPresent()) { + return getMvIdWithUseMvHint(useMvHint.get(), orderedMvs); + } else if (noUseMvHint.isPresent()) { + return getMvIdWithNoUseMvHint(noUseMvHint.get(), orderedMvs); + } + return orderedMvs.get(0); + } + + private Long getMvIdWithUseMvHint(UseMvHint useMvHint, List orderedMvs) { + if (useMvHint.isAllMv()) { + useMvHint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + useMvHint.setErrorMessage("use_mv hint should only have one mv in one table: " + + this.name); + return orderedMvs.get(0); + } else { + String mvName = useMvHint.getUseMvName(this.name); + if (mvName != null) { + if (mvName.equals("`*`")) { + useMvHint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + useMvHint.setErrorMessage("use_mv hint should only have one mv in one table: " + + this.name); + return orderedMvs.get(0); + } + Long choosedIndexId = indexNameToId.get(mvName); + if (orderedMvs.contains(choosedIndexId)) { + useMvHint.setStatus(Hint.HintStatus.SUCCESS); + return choosedIndexId; + } else { + useMvHint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + useMvHint.setErrorMessage("do not have mv: " + mvName + " in table: " + this.name); + } + } + } + return orderedMvs.get(0); + } + + private Long getMvIdWithNoUseMvHint(UseMvHint noUseMvHint, List orderedMvs) { + if (noUseMvHint.isAllMv()) { + noUseMvHint.setStatus(Hint.HintStatus.SUCCESS); + return getBaseIndex().getId(); + } else { + List mvNames = noUseMvHint.getNoUseMVName(this.name); + Set forbiddenIndexIds = Sets.newHashSet(); + for (int i = 0; i < mvNames.size(); i++) { + if (mvNames.get(i).equals("`*`")) { + noUseMvHint.setStatus(Hint.HintStatus.SUCCESS); + return getBaseIndex().getId(); + } + if (hasMaterializedIndex(mvNames.get(i))) { + Long forbiddenIndexId = indexNameToId.get(mvNames.get(i)); + forbiddenIndexIds.add(forbiddenIndexId); + } else { + noUseMvHint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + noUseMvHint.setErrorMessage("do not have mv: " + mvNames.get(i) + " in table: " + this.name); + break; + } + } + for (int i = 0; i < orderedMvs.size(); i++) { + if (forbiddenIndexIds.contains(orderedMvs.get(i))) { + noUseMvHint.setStatus(Hint.HintStatus.SUCCESS); + } else { + return orderedMvs.get(i); + } + } + } + return orderedMvs.get(0); + } + + private Optional getUseMvHint(String useMvName) { + for (Hint hint : ConnectContext.get().getStatementContext().getHints()) { + if (hint.isSyntaxError()) { + continue; + } + if (hint.getHintName().equalsIgnoreCase(useMvName)) { + return Optional.of((UseMvHint) hint); + } + } + return Optional.empty(); + } + public List getVisibleIndex() { Optional partition = idToPartition.values().stream().findFirst(); if (!partition.isPresent()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/analysis/UseCloudClusterStmt.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/analysis/UseCloudClusterStmt.java index de7fdf213a565f..40bd2dbebc6be4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/analysis/UseCloudClusterStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/analysis/UseCloudClusterStmt.java @@ -18,6 +18,7 @@ package org.apache.doris.cloud.analysis; import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.NotFallbackInParser; import org.apache.doris.analysis.RedirectStatus; import org.apache.doris.analysis.ResourceTypeEnum; import org.apache.doris.analysis.StatementBase; @@ -39,7 +40,7 @@ /** * Representation of a use cluster statement. */ -public class UseCloudClusterStmt extends StatementBase { +public class UseCloudClusterStmt extends StatementBase implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(UseCloudClusterStmt.class); private String cluster; private String database; diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/Daemon.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/Daemon.java index 472285b476497f..16ac3259318f60 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/Daemon.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/Daemon.java @@ -121,7 +121,7 @@ public void run() { try { Thread.sleep(intervalMs); } catch (InterruptedException e) { - LOG.error("InterruptedException: ", e); + LOG.info("InterruptedException: ", e); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/UseMvHint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/UseMvHint.java new file mode 100644 index 00000000000000..5e37bdc27603bc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/UseMvHint.java @@ -0,0 +1,144 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.hint; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * rule hint. + */ +public class UseMvHint extends Hint { + + private final boolean isUseMv; + + private final boolean isAllMv; + + private final List parameters; + + private final Map useMvTableColumnMap; + + private final Map> noUseMvTableColumnMap; + + /** + * constructor of use mv hint + * @param hintName use mv + * @param parameters original parameters + * @param isUseMv use_mv hint or no_use_mv hint + * @param isAllMv should all mv be controlled + */ + public UseMvHint(String hintName, List parameters, boolean isUseMv, boolean isAllMv) { + super(hintName); + this.isUseMv = isUseMv; + this.isAllMv = isAllMv; + this.parameters = parameters; + this.useMvTableColumnMap = initUseMvTableColumnMap(parameters); + this.noUseMvTableColumnMap = initNoUseMvTableColumnMap(parameters); + } + + private Map initUseMvTableColumnMap(List parameters) { + Map tempUseMvTableColumnMap = new HashMap<>(); + if (!isUseMv) { + return tempUseMvTableColumnMap; + } + if (parameters.size() % 2 == 1) { + this.setStatus(HintStatus.SYNTAX_ERROR); + this.setErrorMessage("parameter of use_mv hint must be in pairs"); + return tempUseMvTableColumnMap; + } + for (int i = 0; i < parameters.size(); i += 2) { + String tableName = parameters.get(i); + String columnName = parameters.get(i + 1); + if (tempUseMvTableColumnMap.containsKey(tableName)) { + this.setStatus(HintStatus.SYNTAX_ERROR); + this.setErrorMessage("use_mv hint should only have one mv in one table: " + + tableName + "." + columnName); + break; + } + tempUseMvTableColumnMap.put(tableName, columnName); + } + return tempUseMvTableColumnMap; + } + + private Map> initNoUseMvTableColumnMap(List parameters) { + Map> tempNoUseMvTableColumnMap = new HashMap<>(); + if (isUseMv) { + return tempNoUseMvTableColumnMap; + } + if (parameters.size() % 2 == 1) { + this.setStatus(HintStatus.SYNTAX_ERROR); + this.setErrorMessage("parameter of no_use_mv hint must be in pairs"); + return tempNoUseMvTableColumnMap; + } + for (int i = 0; i < parameters.size(); i += 2) { + String tableName = parameters.get(i); + String columnName = parameters.get(i + 1); + if (tempNoUseMvTableColumnMap.containsKey(tableName)) { + tempNoUseMvTableColumnMap.get(tableName).add(columnName); + } else { + List list = new ArrayList<>(); + list.add(columnName); + tempNoUseMvTableColumnMap.put(tableName, list); + } + } + return tempNoUseMvTableColumnMap; + } + + public boolean isUseMv() { + return isUseMv; + } + + public boolean isAllMv() { + return isAllMv; + } + + public String getUseMvName(String tableName) { + return useMvTableColumnMap.get(tableName); + } + + public List getNoUseMVName(String tableName) { + return noUseMvTableColumnMap.get(tableName); + } + + @Override + public String getExplainString() { + StringBuilder out = new StringBuilder(); + if (isUseMv) { + out.append("use_mv"); + } else { + out.append("no_use_mv"); + } + if (!parameters.isEmpty()) { + out.append("("); + for (int i = 0; i < parameters.size(); i++) { + if (i % 2 == 0) { + out.append(parameters.get(i)); + } else { + out.append("."); + out.append(parameters.get(i)); + out.append(" "); + } + } + out.append(")"); + } + + return out.toString(); + } +} 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 7369b714f7439c..324ab808226930 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 @@ -232,6 +232,7 @@ import org.apache.doris.nereids.properties.SelectHintOrdered; import org.apache.doris.nereids.properties.SelectHintSetVar; import org.apache.doris.nereids.properties.SelectHintUseCboRule; +import org.apache.doris.nereids.properties.SelectHintUseMv; import org.apache.doris.nereids.trees.TableSample; import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.And; @@ -3165,7 +3166,7 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, List hints = Maps.newLinkedHashMap(); + ImmutableList.Builder hints = ImmutableList.builder(); for (ParserRuleContext hintContext : hintContexts) { SelectHintContext selectHintContext = (SelectHintContext) hintContext; for (HintStatementContext hintStatement : selectHintContext.hintStatements) { @@ -3187,7 +3188,7 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, List leadingParameters = new ArrayList<>(); @@ -3197,10 +3198,10 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, List useRuleParameters = new ArrayList<>(); @@ -3210,7 +3211,7 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, List noUseRuleParameters = new ArrayList<>(); @@ -3220,14 +3221,34 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, List useIndexParameters = new ArrayList(); + for (HintAssignmentContext kv : hintStatement.parameters) { + String parameterName = visitIdentifierOrText(kv.key); + if (kv.key != null) { + useIndexParameters.add(parameterName); + } + } + hints.add(new SelectHintUseMv(hintName, useIndexParameters, true)); + break; + case "no_use_mv": + List noUseIndexParameters = new ArrayList(); + for (HintAssignmentContext kv : hintStatement.parameters) { + String parameterName = visitIdentifierOrText(kv.key); + if (kv.key != null) { + noUseIndexParameters.add(parameterName); + } + } + hints.add(new SelectHintUseMv(hintName, noUseIndexParameters, false)); break; default: break; } } } - return new LogicalSelectHint<>(hints, logicalPlan); + return new LogicalSelectHint<>(hints.build(), logicalPlan); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java index 8e8889f5e62df2..31a205d5ed5054 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java @@ -59,7 +59,7 @@ public Plan visitUnboundResultSink(UnboundResultSink unboundResu public Plan visitLogicalSubQueryAlias(LogicalSubQueryAlias alias, StatementContext context) { if (alias.child() instanceof LogicalSelectHint - && ((LogicalSelectHint) alias.child()).isIncludeLeading()) { + && ((LogicalSelectHint) alias.child()).isIncludeHint("Leading")) { aliasQueries.add((LogicalSubQueryAlias) alias); List tableName = new ArrayList<>(); tableName.add(alias.getAlias()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintUseMv.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintUseMv.java new file mode 100644 index 00000000000000..35ce25fb4f47c6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintUseMv.java @@ -0,0 +1,53 @@ +// 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.properties; + +import java.util.List; + +/** + * select hint UseMv. + */ +public class SelectHintUseMv extends SelectHint { + private final List parameters; + + private final boolean isUseMv; + + public SelectHintUseMv(String hintName, List parameters, boolean isUseMv) { + super(hintName); + this.parameters = parameters; + this.isUseMv = isUseMv; + } + + public List getParameters() { + return parameters; + } + + public boolean isUseMv() { + return isUseMv; + } + + @Override + public String getHintName() { + return super.getHintName(); + } + + @Override + public String toString() { + return super.getHintName(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index be4d8b390c9f1f..e63d4d77bca26e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -88,6 +88,7 @@ import org.apache.doris.nereids.rules.rewrite.ConvertOuterJoinToAntiJoin; import org.apache.doris.nereids.rules.rewrite.CreatePartitionTopNFromWindow; import org.apache.doris.nereids.rules.rewrite.EliminateOuterJoin; +import org.apache.doris.nereids.rules.rewrite.MaxMinFilterPushDown; import org.apache.doris.nereids.rules.rewrite.MergeFilters; import org.apache.doris.nereids.rules.rewrite.MergeGenerates; import org.apache.doris.nereids.rules.rewrite.MergeLimits; @@ -132,6 +133,7 @@ public class RuleSet { .build(); public static final List PUSH_DOWN_FILTERS = ImmutableList.of( + new MaxMinFilterPushDown(), new CreatePartitionTopNFromWindow(), new PushDownFilterThroughProject(), new PushDownFilterThroughSort(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index f2c572f7779e91..ca26ab1d9f843c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -176,6 +176,7 @@ public enum RuleType { PUSH_DOWN_FILTER_THROUGH_CTE(RuleTypeClass.REWRITE), PUSH_DOWN_FILTER_THROUGH_CTE_ANCHOR(RuleTypeClass.REWRITE), + MAX_MIN_FILTER_PUSH_DOWN(RuleTypeClass.REWRITE), PUSH_DOWN_DISTINCT_THROUGH_JOIN(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateLogicalSelectHint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateLogicalSelectHint.java index ea2c9994606502..ebff9f838a447e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateLogicalSelectHint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateLogicalSelectHint.java @@ -27,10 +27,12 @@ import org.apache.doris.nereids.hint.LeadingHint; import org.apache.doris.nereids.hint.OrderedHint; import org.apache.doris.nereids.hint.UseCboRuleHint; +import org.apache.doris.nereids.hint.UseMvHint; import org.apache.doris.nereids.properties.SelectHint; import org.apache.doris.nereids.properties.SelectHintLeading; import org.apache.doris.nereids.properties.SelectHintSetVar; import org.apache.doris.nereids.properties.SelectHintUseCboRule; +import org.apache.doris.nereids.properties.SelectHintUseMv; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory; @@ -43,7 +45,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Map; import java.util.Map.Entry; import java.util.Optional; @@ -57,10 +58,10 @@ public class EliminateLogicalSelectHint extends OneRewriteRuleFactory { public Rule build() { return logicalSelectHint().thenApply(ctx -> { LogicalSelectHint selectHintPlan = ctx.root; - for (Entry hint : selectHintPlan.getHints().entrySet()) { - String hintName = hint.getKey(); + for (SelectHint hint : selectHintPlan.getHints()) { + String hintName = hint.getHintName(); if (hintName.equalsIgnoreCase("SET_VAR")) { - setVar((SelectHintSetVar) hint.getValue(), ctx.statementContext); + setVar((SelectHintSetVar) hint, ctx.statementContext); } else if (hintName.equalsIgnoreCase("ORDERED")) { try { ctx.cascadesContext.getConnectContext().getSessionVariable() @@ -73,12 +74,16 @@ public Rule build() { ctx.cascadesContext.getHintMap().put("Ordered", ordered); ctx.statementContext.addHint(ordered); } else if (hintName.equalsIgnoreCase("LEADING")) { - extractLeading((SelectHintLeading) hint.getValue(), ctx.cascadesContext, - ctx.statementContext, selectHintPlan.getHints()); + extractLeading((SelectHintLeading) hint, ctx.cascadesContext, + ctx.statementContext, selectHintPlan); } else if (hintName.equalsIgnoreCase("USE_CBO_RULE")) { - extractRule((SelectHintUseCboRule) hint.getValue(), ctx.statementContext); + extractRule((SelectHintUseCboRule) hint, ctx.statementContext); + } else if (hintName.equalsIgnoreCase("USE_MV")) { + extractMv((SelectHintUseMv) hint, ConnectContext.get().getStatementContext()); + } else if (hintName.equalsIgnoreCase("NO_USE_MV")) { + extractMv((SelectHintUseMv) hint, ConnectContext.get().getStatementContext()); } else { - logger.warn("Can not process select hint '{}' and skip it", hint.getKey()); + logger.warn("Can not process select hint '{}' and skip it", hint.getHintName()); } } return selectHintPlan.child(); @@ -116,7 +121,7 @@ private void setVar(SelectHintSetVar selectHint, StatementContext context) { } private void extractLeading(SelectHintLeading selectHint, CascadesContext context, - StatementContext statementContext, Map hints) { + StatementContext statementContext, LogicalSelectHint selectHintPlan) { LeadingHint hint = new LeadingHint("Leading", selectHint.getParameters(), selectHint.toString()); if (context.getHintMap().get("Leading") != null) { hint.setStatus(Hint.HintStatus.SYNTAX_ERROR); @@ -139,7 +144,8 @@ private void extractLeading(SelectHintLeading selectHint, CascadesContext contex if (!hint.isSyntaxError()) { hint.setStatus(Hint.HintStatus.SUCCESS); } - if (hints.get("ordered") != null || ConnectContext.get().getSessionVariable().isDisableJoinReorder() + if (selectHintPlan.isIncludeHint("Ordered") + || ConnectContext.get().getSessionVariable().isDisableJoinReorder() || context.isLeadingDisableJoinReorder()) { context.setLeadingJoin(false); hint.setStatus(Hint.HintStatus.UNUSED); @@ -158,4 +164,22 @@ private void extractRule(SelectHintUseCboRule selectHint, StatementContext state } } + private void extractMv(SelectHintUseMv selectHint, StatementContext statementContext) { + boolean isAllMv = selectHint.getParameters().isEmpty(); + UseMvHint useMvHint = new UseMvHint(selectHint.getHintName(), selectHint.getParameters(), + selectHint.isUseMv(), isAllMv); + for (Hint hint : statementContext.getHints()) { + if (hint.getHintName().equals(selectHint.getHintName())) { + hint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + hint.setErrorMessage("only one " + selectHint.getHintName() + " hint is allowed"); + useMvHint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + useMvHint.setErrorMessage("only one " + selectHint.getHintName() + " hint is allowed"); + } + } + if (!useMvHint.isSyntaxError()) { + ConnectContext.get().getSessionVariable().setEnableSyncMvCostBasedRewrite(false); + } + statementContext.addHint(useMvHint); + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MaxMinFilterPushDown.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MaxMinFilterPushDown.java new file mode 100644 index 00000000000000..a54c3785b35a72 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MaxMinFilterPushDown.java @@ -0,0 +1,133 @@ +// 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.annotation.DependsRules; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.expression.ExpressionRewrite; +import org.apache.doris.nereids.trees.expressions.Alias; +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.GreaterThanEqual; +import org.apache.doris.nereids.trees.expressions.LessThan; +import org.apache.doris.nereids.trees.expressions.LessThanEqual; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.Max; +import org.apache.doris.nereids.trees.expressions.functions.agg.Min; +import org.apache.doris.nereids.trees.expressions.literal.Literal; +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.LogicalFilter; +import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.nereids.util.PlanUtils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +/** + * select id, max(a) from t group by id having max(a)>10; + * -> + * select id, max(a) from t where a>10 group by id; + * select id, min(a) from t group by id having min(a)<10; + * -> + * select id, min(a) from t where a<10 group by id; + */ +@DependsRules({ + ExpressionRewrite.class +}) +public class MaxMinFilterPushDown extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalFilter(logicalAggregate().whenNot(agg -> agg.getGroupByExpressions().isEmpty())) + .then(this::pushDownMaxMinFilter) + .toRule(RuleType.MAX_MIN_FILTER_PUSH_DOWN); + } + + private Plan pushDownMaxMinFilter(LogicalFilter> filter) { + Set conjuncts = filter.getConjuncts(); + LogicalAggregate agg = filter.child(); + Plan aggChild = agg.child(); + List aggOutputExpressions = agg.getOutputExpressions(); + Set aggFuncs = ExpressionUtils.collect(aggOutputExpressions, + expr -> expr instanceof AggregateFunction); + Set maxMinFunc = ExpressionUtils.collect(aggFuncs, + expr -> expr instanceof Max || expr instanceof Min); + // LogicalAggregate only outputs one aggregate function, which is max or min + if (aggFuncs.size() != 1 || maxMinFunc.size() != 1) { + return null; + } + ExprId exprId = null; + Expression func = maxMinFunc.iterator().next(); + for (NamedExpression expr : aggOutputExpressions) { + if (expr instanceof Alias && ((Alias) expr).child().equals(func)) { + Alias alias = (Alias) expr; + exprId = alias.getExprId(); + } + } + // try to find min(a)<10 or max(a)>10 + Expression originConjunct = findMatchingConjunct(conjuncts, func instanceof Max, exprId).orElse(null); + if (null == originConjunct) { + return null; + } + Set newUpperConjuncts = new HashSet<>(conjuncts); + newUpperConjuncts.remove(originConjunct); + Expression newPredicate = null; + if (func instanceof Max) { + if (originConjunct instanceof GreaterThan) { + newPredicate = new GreaterThan(func.child(0), originConjunct.child(1)); + } else if (originConjunct instanceof GreaterThanEqual) { + newPredicate = new GreaterThanEqual(func.child(0), originConjunct.child(1)); + } + } else { + if (originConjunct instanceof LessThan) { + newPredicate = new LessThan(func.child(0), originConjunct.child(1)); + } else if (originConjunct instanceof LessThanEqual) { + newPredicate = new LessThanEqual(func.child(0), originConjunct.child(1)); + } + } + Preconditions.checkState(newPredicate != null, "newPredicate is null"); + LogicalFilter newPushDownFilter = new LogicalFilter<>(ImmutableSet.of(newPredicate), aggChild); + LogicalAggregate newAgg = agg.withChildren(ImmutableList.of(newPushDownFilter)); + return PlanUtils.filterOrSelf(newUpperConjuncts, newAgg); + } + + private Optional findMatchingConjunct(Set conjuncts, boolean isMax, ExprId exprId) { + for (Expression conjunct : conjuncts) { + if ((isMax && (conjunct instanceof GreaterThan || conjunct instanceof GreaterThanEqual)) + || (!isMax && (conjunct instanceof LessThan || conjunct instanceof LessThanEqual))) { + if (conjunct.child(0) instanceof SlotReference && conjunct.child(1) instanceof Literal) { + SlotReference slot = (SlotReference) conjunct.child(0); + if (slot.getExprId().equals(exprId)) { + return Optional.of(conjunct); + } + } + } + } + return Optional.empty(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java index 1124c141416f3f..f17ab1c96bd9cf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java @@ -276,7 +276,7 @@ protected static long selectBestIndex( .thenComparing(rid -> (Long) rid)) .collect(Collectors.toList()); - return sortedIndexIds.get(0); + return table.getBestMvIdWithHint(sortedIndexIds); } protected static List matchPrefixMost( 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 3bc2a880da7c6a..a65a07fea30bdf 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 @@ -607,11 +607,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/trees/expressions/functions/scalar/Acos.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Acos.java index c99af81123fc63..2193221c326363 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Acos.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Acos.java @@ -19,8 +19,8 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; -import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DoubleType; @@ -34,7 +34,7 @@ * ScalarFunction 'acos'. This class is generated by GenerateFunction. */ public class Acos extends ScalarFunction - implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNullable { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(DoubleType.INSTANCE).args(DoubleType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesDecrypt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesDecrypt.java index 5e7d760b0810a0..7608cf4e40ea35 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesDecrypt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesDecrypt.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.trees.expressions.functions.scalar; import org.apache.doris.catalog.FunctionSignature; -import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -58,16 +57,7 @@ public class AesDecrypt extends AesCryptoFunction { * AesDecrypt */ public AesDecrypt(Expression arg0, Expression arg1) { - // if there are only 2 params, we need set encryption mode to AES_128_ECB - // this keeps the behavior consistent with old doris ver. - super("aes_decrypt", arg0, arg1, new StringLiteral("AES_128_ECB")); - - // check if encryptionMode from session variables is valid - StringLiteral encryptionMode = CryptoFunction.getDefaultBlockEncryptionMode("AES_128_ECB"); - if (!AES_MODES.contains(encryptionMode.getValue())) { - throw new AnalysisException( - "session variable block_encryption_mode is invalid with aes"); - } + super("aes_decrypt", arg0, arg1, new StringLiteral(""), getDefaultBlockEncryptionMode()); } public AesDecrypt(Expression arg0, Expression arg1, Expression arg2) { @@ -89,7 +79,7 @@ public AesDecrypt withChildren(List children) { } else if (children().size() == 3) { return new AesDecrypt(children.get(0), children.get(1), children.get(2)); } else { - return new AesDecrypt(children.get(0), children.get(1), children.get(2), (StringLiteral) children.get(3)); + return new AesDecrypt(children.get(0), children.get(1), children.get(2), children.get(3)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesDecryptV2.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesDecryptV2.java deleted file mode 100644 index 3a5a8121cff119..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesDecryptV2.java +++ /dev/null @@ -1,74 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.trees.expressions.functions.scalar; - -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; -import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; - -import com.google.common.base.Preconditions; - -import java.util.List; - -/** - * ScalarFunction 'aes_decrypt'. This class is generated by GenerateFunction. - */ -public class AesDecryptV2 extends AesDecrypt { - - /** - * AesDecryptV2 - */ - public AesDecryptV2(Expression arg0, Expression arg1) { - super(arg0, arg1, getDefaultBlockEncryptionMode()); - String blockEncryptionMode = String.valueOf(getDefaultBlockEncryptionMode()); - if (!blockEncryptionMode.toUpperCase().equals("'AES_128_ECB'") - && !blockEncryptionMode.toUpperCase().equals("'AES_192_ECB'") - && !blockEncryptionMode.toUpperCase().equals("'AES_256_ECB'")) { - throw new AnalysisException("Incorrect parameter count in the call to native function 'aes_decrypt'"); - } - } - - public AesDecryptV2(Expression arg0, Expression arg1, Expression arg2) { - super(arg0, arg1, arg2); - } - - public AesDecryptV2(Expression arg0, Expression arg1, Expression arg2, Expression arg3) { - super(arg0, arg1, arg2, arg3); - } - - /** - * withChildren. - */ - @Override - public AesDecryptV2 withChildren(List children) { - Preconditions.checkArgument(children.size() >= 2 && children.size() <= 4); - if (children.size() == 2) { - return new AesDecryptV2(children.get(0), children.get(1)); - } else if (children().size() == 3) { - return new AesDecryptV2(children.get(0), children.get(1), children.get(2)); - } else { - return new AesDecryptV2(children.get(0), children.get(1), children.get(2), (StringLiteral) children.get(3)); - } - } - - @Override - public R accept(ExpressionVisitor visitor, C context) { - return visitor.visitAesDecryptV2(this, context); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesEncrypt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesEncrypt.java index ef99bdbe21db60..455d6b0dbd5645 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesEncrypt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesEncrypt.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.trees.expressions.functions.scalar; import org.apache.doris.catalog.FunctionSignature; -import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -58,16 +57,7 @@ public class AesEncrypt extends AesCryptoFunction { * Some javadoc for checkstyle... */ public AesEncrypt(Expression arg0, Expression arg1) { - // if there are only 2 params, we need set encryption mode to AES_128_ECB - // this keeps the behavior consistent with old doris ver. - super("aes_encrypt", arg0, arg1, new StringLiteral("AES_128_ECB")); - - // check if encryptionMode from session variables is valid - StringLiteral encryptionMode = CryptoFunction.getDefaultBlockEncryptionMode("AES_128_ECB"); - if (!AES_MODES.contains(encryptionMode.getValue())) { - throw new AnalysisException( - "session variable block_encryption_mode is invalid with aes"); - } + super("aes_encrypt", arg0, arg1, new StringLiteral(""), getDefaultBlockEncryptionMode()); } public AesEncrypt(Expression arg0, Expression arg1, Expression arg2) { @@ -89,7 +79,7 @@ public AesEncrypt withChildren(List children) { } else if (children().size() == 3) { return new AesEncrypt(children.get(0), children.get(1), children.get(2)); } else { - return new AesEncrypt(children.get(0), children.get(1), children.get(2), (StringLiteral) children.get(3)); + return new AesEncrypt(children.get(0), children.get(1), children.get(2), children.get(3)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesEncryptV2.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesEncryptV2.java deleted file mode 100644 index 08e1d1e915e52e..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/AesEncryptV2.java +++ /dev/null @@ -1,74 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.trees.expressions.functions.scalar; - -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; -import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; - -import com.google.common.base.Preconditions; - -import java.util.List; - -/** - * ScalarFunction 'aes_encrypt'. This class is generated by GenerateFunction. - */ -public class AesEncryptV2 extends AesEncrypt { - - /** - * AesEncryptV2 - */ - public AesEncryptV2(Expression arg0, Expression arg1) { - super(arg0, arg1, getDefaultBlockEncryptionMode()); - String blockEncryptionMode = String.valueOf(getDefaultBlockEncryptionMode()); - if (!blockEncryptionMode.toUpperCase().equals("'AES_128_ECB'") - && !blockEncryptionMode.toUpperCase().equals("'AES_192_ECB'") - && !blockEncryptionMode.toUpperCase().equals("'AES_256_ECB'")) { - throw new AnalysisException("Incorrect parameter count in the call to native function 'aes_encrypt'"); - } - } - - public AesEncryptV2(Expression arg0, Expression arg1, Expression arg2) { - super(arg0, arg1, arg2); - } - - public AesEncryptV2(Expression arg0, Expression arg1, Expression arg2, Expression arg3) { - super(arg0, arg1, arg2, arg3); - } - - /** - * withChildren. - */ - @Override - public AesEncryptV2 withChildren(List children) { - Preconditions.checkArgument(children.size() >= 2 && children.size() <= 4); - if (children.size() == 2) { - return new AesEncryptV2(children.get(0), children.get(1)); - } else if (children().size() == 3) { - return new AesEncryptV2(children.get(0), children.get(1), children.get(2)); - } else { - return new AesEncryptV2(children.get(0), children.get(1), children.get(2), (StringLiteral) children.get(3)); - } - } - - @Override - public R accept(ExpressionVisitor visitor, C context) { - return visitor.visitAesEncryptV2(this, context); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Asin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Asin.java index 0e06d8d77edb10..22e1ff59b7df28 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Asin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Asin.java @@ -19,8 +19,8 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; -import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DoubleType; @@ -34,7 +34,7 @@ * ScalarFunction 'asin'. This class is generated by GenerateFunction. */ public class Asin extends ScalarFunction - implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNullable { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(DoubleType.INSTANCE).args(DoubleType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Dsqrt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Dsqrt.java index 874befd09dba4d..3caef79776b3bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Dsqrt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Dsqrt.java @@ -19,8 +19,8 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; -import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DoubleType; @@ -34,7 +34,7 @@ * ScalarFunction 'dsqrt'. This class is generated by GenerateFunction. */ public class Dsqrt extends ScalarFunction - implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNullable { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(DoubleType.INSTANCE).args(DoubleType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4Decrypt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4Decrypt.java index 6e0f2bb48de1f0..c87c2a42c16e5a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4Decrypt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4Decrypt.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.trees.expressions.functions.scalar; import org.apache.doris.catalog.FunctionSignature; -import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -63,17 +62,7 @@ public class Sm4Decrypt extends Sm4CryptoFunction { * constructor with 2 arguments. */ public Sm4Decrypt(Expression arg0, Expression arg1) { - // if there are only 2 params, we need add an empty string as the third param - // and set encryption mode to SM4_128_ECB - // this keeps the behavior consistent with old doris ver. - super("sm4_decrypt", arg0, arg1, new StringLiteral(""), new StringLiteral("SM4_128_ECB")); - - // check if encryptionMode from session variables is valid - StringLiteral encryptionMode = CryptoFunction.getDefaultBlockEncryptionMode("SM4_128_ECB"); - if (!SM4_MODES.contains(encryptionMode.getValue())) { - throw new AnalysisException( - "session variable block_encryption_mode is invalid with sm4"); - } + super("sm4_decrypt", arg0, arg1, new StringLiteral(""), getDefaultBlockEncryptionMode()); } /** @@ -98,7 +87,7 @@ public Sm4Decrypt withChildren(List children) { } else if (children().size() == 3) { return new Sm4Decrypt(children.get(0), children.get(1), children.get(2)); } else { - return new Sm4Decrypt(children.get(0), children.get(1), children.get(2), (StringLiteral) children.get(3)); + return new Sm4Decrypt(children.get(0), children.get(1), children.get(2), children.get(3)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4DecryptV2.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4DecryptV2.java deleted file mode 100644 index 4c743a82945068..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4DecryptV2.java +++ /dev/null @@ -1,69 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.trees.expressions.functions.scalar; - -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; -import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; - -import com.google.common.base.Preconditions; - -import java.util.List; - -/** - * ScalarFunction 'sm4_decrypt'. This class is generated by GenerateFunction. - */ -public class Sm4DecryptV2 extends Sm4Decrypt { - - /** - * Sm4DecryptV2 - */ - public Sm4DecryptV2(Expression arg0, Expression arg1) { - super(arg0, arg1); - throw new AnalysisException("Incorrect parameter count in the call to native function 'sm4_decrypt'"); - } - - public Sm4DecryptV2(Expression arg0, Expression arg1, Expression arg2) { - super(arg0, arg1, arg2); - } - - public Sm4DecryptV2(Expression arg0, Expression arg1, Expression arg2, Expression arg3) { - super(arg0, arg1, arg2, arg3); - } - - /** - * withChildren. - */ - @Override - public Sm4DecryptV2 withChildren(List children) { - Preconditions.checkArgument(children.size() >= 2 && children.size() <= 4); - if (children.size() == 2) { - return new Sm4DecryptV2(children.get(0), children.get(1)); - } else if (children().size() == 3) { - return new Sm4DecryptV2(children.get(0), children.get(1), children.get(2)); - } else { - return new Sm4DecryptV2(children.get(0), children.get(1), children.get(2), (StringLiteral) children.get(3)); - } - } - - @Override - public R accept(ExpressionVisitor visitor, C context) { - return visitor.visitSm4DecryptV2(this, context); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4Encrypt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4Encrypt.java index 00525558d41982..dbf5c61db07de5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4Encrypt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4Encrypt.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.trees.expressions.functions.scalar; import org.apache.doris.catalog.FunctionSignature; -import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -58,17 +57,7 @@ public class Sm4Encrypt extends Sm4CryptoFunction { * constructor with 2 arguments. */ public Sm4Encrypt(Expression arg0, Expression arg1) { - // if there are only 2 params, we need add an empty string as the third param - // and set encryption mode to SM4_128_ECB - // this keeps the behavior consistent with old doris ver. - super("sm4_encrypt", arg0, arg1, new StringLiteral(""), new StringLiteral("SM4_128_ECB")); - - // check if encryptionMode from session variables is valid - StringLiteral encryptionMode = CryptoFunction.getDefaultBlockEncryptionMode("SM4_128_ECB"); - if (!SM4_MODES.contains(encryptionMode.getValue())) { - throw new AnalysisException( - "session variable block_encryption_mode is invalid with sm4"); - } + super("sm4_encrypt", arg0, arg1, new StringLiteral(""), getDefaultBlockEncryptionMode()); } /** @@ -93,7 +82,7 @@ public Sm4Encrypt withChildren(List children) { } else if (children().size() == 3) { return new Sm4Encrypt(children.get(0), children.get(1), children.get(2)); } else { - return new Sm4Encrypt(children.get(0), children.get(1), children.get(2), (StringLiteral) children.get(3)); + return new Sm4Encrypt(children.get(0), children.get(1), children.get(2), children.get(3)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4EncryptV2.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4EncryptV2.java deleted file mode 100644 index 7c895fc3090596..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sm4EncryptV2.java +++ /dev/null @@ -1,72 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.trees.expressions.functions.scalar; - -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; -import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; - -import com.google.common.base.Preconditions; - -import java.util.List; - -/** - * ScalarFunction 'sm4_encrypt'. This class is generated by GenerateFunction. - */ -public class Sm4EncryptV2 extends Sm4Encrypt { - - /** - * constructor with 2 arguments. - */ - public Sm4EncryptV2(Expression arg0, Expression arg1) { - super(arg0, arg1); - throw new AnalysisException("Incorrect parameter count in the call to native function 'sm4_encrypt'"); - } - - /** - * constructor with 3 arguments. - */ - public Sm4EncryptV2(Expression arg0, Expression arg1, Expression arg2) { - super(arg0, arg1, arg2); - } - - public Sm4EncryptV2(Expression arg0, Expression arg1, Expression arg2, Expression arg3) { - super(arg0, arg1, arg2, arg3); - } - - /** - * withChildren. - */ - @Override - public Sm4EncryptV2 withChildren(List children) { - Preconditions.checkArgument(children.size() >= 2 && children.size() <= 4); - if (children.size() == 2) { - return new Sm4EncryptV2(children.get(0), children.get(1)); - } else if (children().size() == 3) { - return new Sm4EncryptV2(children.get(0), children.get(1), children.get(2)); - } else { - return new Sm4EncryptV2(children.get(0), children.get(1), children.get(2), (StringLiteral) children.get(3)); - } - } - - @Override - public R accept(ExpressionVisitor visitor, C context) { - return visitor.visitSm4EncryptV2(this, context); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sqrt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sqrt.java index 495321c6dfa8c5..f954eb07a54083 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sqrt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sqrt.java @@ -19,8 +19,8 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; -import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DoubleType; @@ -34,7 +34,7 @@ * ScalarFunction 'sqrt'. This class is generated by GenerateFunction. */ public class Sqrt extends ScalarFunction - implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNullable { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(DoubleType.INSTANCE).args(DoubleType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 5d47094f247e8c..20772ae716be2f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -24,9 +24,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Abs; import org.apache.doris.nereids.trees.expressions.functions.scalar.Acos; import org.apache.doris.nereids.trees.expressions.functions.scalar.AesDecrypt; -import org.apache.doris.nereids.trees.expressions.functions.scalar.AesDecryptV2; import org.apache.doris.nereids.trees.expressions.functions.scalar.AesEncrypt; -import org.apache.doris.nereids.trees.expressions.functions.scalar.AesEncryptV2; import org.apache.doris.nereids.trees.expressions.functions.scalar.AppendTrailingCharIfAbsent; import org.apache.doris.nereids.trees.expressions.functions.scalar.Array; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayApply; @@ -368,9 +366,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm3; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm3sum; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4Decrypt; -import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4DecryptV2; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4Encrypt; -import org.apache.doris.nereids.trees.expressions.functions.scalar.Sm4EncryptV2; import org.apache.doris.nereids.trees.expressions.functions.scalar.Space; import org.apache.doris.nereids.trees.expressions.functions.scalar.SplitByChar; import org.apache.doris.nereids.trees.expressions.functions.scalar.SplitByRegexp; @@ -480,18 +476,10 @@ default R visitAesDecrypt(AesDecrypt aesDecrypt, C context) { return visitScalarFunction(aesDecrypt, context); } - default R visitAesDecryptV2(AesDecryptV2 aesDecryptV2, C context) { - return visitScalarFunction(aesDecryptV2, context); - } - default R visitAesEncrypt(AesEncrypt aesEncrypt, C context) { return visitScalarFunction(aesEncrypt, context); } - default R visitAesEncryptV2(AesEncryptV2 aesEncryptV2, C context) { - return visitScalarFunction(aesEncryptV2, context); - } - default R visitAppendTrailingCharIfAbsent(AppendTrailingCharIfAbsent function, C context) { return visitScalarFunction(function, context); } @@ -1836,18 +1824,10 @@ default R visitSm4Decrypt(Sm4Decrypt sm4Decrypt, C context) { return visitScalarFunction(sm4Decrypt, context); } - default R visitSm4DecryptV2(Sm4DecryptV2 sm4DecryptV2, C context) { - return visitScalarFunction(sm4DecryptV2, context); - } - default R visitSm4Encrypt(Sm4Encrypt sm4Encrypt, C context) { return visitScalarFunction(sm4Encrypt, context); } - default R visitSm4EncryptV2(Sm4EncryptV2 sm4EncryptV2, C context) { - return visitScalarFunction(sm4EncryptV2, context); - } - default R visitSpace(Space space, C context) { return visitScalarFunction(space, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSelectHint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSelectHint.java index 127889ea7ed471..a33e2194131c8d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSelectHint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSelectHint.java @@ -29,10 +29,8 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; @@ -44,9 +42,9 @@ public class LogicalSelectHint extends LogicalUnary implements BlockFuncDepsPropagation { - private final Map hints; + private final ImmutableList hints; - public LogicalSelectHint(Map hints, CHILD_TYPE child) { + public LogicalSelectHint(ImmutableList hints, CHILD_TYPE child) { this(hints, Optional.empty(), Optional.empty(), child); } @@ -57,19 +55,29 @@ public LogicalSelectHint(Map hints, CHILD_TYPE child) { * @param logicalProperties logicalProperties is use for compute output * @param child child plan */ - public LogicalSelectHint(Map hints, + public LogicalSelectHint(ImmutableList hints, Optional groupExpression, Optional logicalProperties, CHILD_TYPE child) { super(PlanType.LOGICAL_SELECT_HINT, groupExpression, logicalProperties, child); - this.hints = ImmutableMap.copyOf(Objects.requireNonNull(hints, "hints can not be null")); + this.hints = ImmutableList.copyOf(Objects.requireNonNull(hints, "hints can not be null")); } - public Map getHints() { + public List getHints() { return hints; } - public boolean isIncludeLeading() { - return hints.containsKey("leading"); + /** + * check if current select hint include some hint + * @param hintName hint name + * @return boolean which indicate have hint + */ + public boolean isIncludeHint(String hintName) { + for (SelectHint hint : hints) { + if (hint.getHintName().equalsIgnoreCase(hintName)) { + return true; + } + } + return false; } @Override @@ -107,9 +115,9 @@ public List computeOutput() { @Override public String toString() { - String hintStr = this.hints.entrySet() + String hintStr = this.hints .stream() - .map(entry -> entry.getValue().toString()) + .map(hint -> hint.toString()) .collect(Collectors.joining(", ")); return "LogicalSelectHint (" + hintStr + ")"; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java index 50b0f5a026909c..a92cac7b510260 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java @@ -739,8 +739,20 @@ public int getScanRangeNum() { } public boolean shouldUseOneInstance(ConnectContext ctx) { - long limitRowsForSingleInstance = ctx == null ? 10000 : ctx.getSessionVariable().limitRowsForSingleInstance; - return hasLimit() && getLimit() < limitRowsForSingleInstance && conjuncts.isEmpty(); + int adaptivePipelineTaskSerialReadOnLimit = 10000; + + if (ctx != null) { + if (ctx.getSessionVariable().enableAdaptivePipelineTaskSerialReadOnLimit) { + adaptivePipelineTaskSerialReadOnLimit = ctx.getSessionVariable().adaptivePipelineTaskSerialReadOnLimit; + } else { + return false; + } + } else { + // No connection context, typically for broker load. + } + + // For UniqueKey table, we will use multiple instance. + return hasLimit() && getLimit() <= adaptivePipelineTaskSerialReadOnLimit && conjuncts.isEmpty(); } // In cloud mode, meta read lock is not enough to keep a snapshot of the partition versions. diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 4140835b657004..c1d3c7a3efb0ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -654,6 +654,11 @@ public class SessionVariable implements Serializable, Writable { public static final String IN_LIST_VALUE_COUNT_THRESHOLD = "in_list_value_count_threshold"; + public static final String ENABLE_ADAPTIVE_PIPELINE_TASK_SERIAL_READ_ON_LIMIT = + "enable_adaptive_pipeline_task_serial_read_on_limit"; + public static final String ADAPTIVE_PIPELINE_TASK_SERIAL_READ_ON_LIMIT = + "adaptive_pipeline_task_serial_read_on_limit"; + /** * If set false, user couldn't submit analyze SQL and FE won't allocate any related resources. */ @@ -2115,6 +2120,7 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { }) public boolean enableFallbackOnMissingInvertedIndex = true; + @VariableMgr.VarAttr(name = IN_LIST_VALUE_COUNT_THRESHOLD, description = { "in条件value数量大于这个threshold后将不会走fast_execute", "When the number of values in the IN condition exceeds this threshold," @@ -2122,6 +2128,22 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { }) public int inListValueCountThreshold = 10; + @VariableMgr.VarAttr(name = ENABLE_ADAPTIVE_PIPELINE_TASK_SERIAL_READ_ON_LIMIT, needForward = true, description = { + "开启后将会允许自动调整 pipeline task 的并发数。当 scan 节点没有过滤条件,且 limit 参数小于" + + "adaptive_pipeline_task_serial_read_on_limit 中指定的行数时,scan 的并行度将会被设置为 1", + "When enabled, the pipeline task concurrency will be adjusted automatically. When the scan node has no filter " + + "conditions and the limit parameter is less than the number of rows specified in " + + "adaptive_pipeline_task_serial_read_on_limit, the parallelism of the scan will be set to 1." + }) + public boolean enableAdaptivePipelineTaskSerialReadOnLimit = true; + + @VariableMgr.VarAttr(name = ADAPTIVE_PIPELINE_TASK_SERIAL_READ_ON_LIMIT, needForward = true, description = { + "当 enable_adaptive_pipeline_task_serial_read_on_limit 开启时,scan 的并行度将会被设置为 1 的行数阈值", + "When enable_adaptive_pipeline_task_serial_read_on_limit is enabled, " + + "the number of rows at which the parallelism of the scan will be set to 1." + }) + public int adaptivePipelineTaskSerialReadOnLimit = 10000; + public void setEnableEsParallelScroll(boolean enableESParallelScroll) { this.enableESParallelScroll = enableESParallelScroll; } @@ -3700,6 +3722,9 @@ public TQueryOptions toThrift() { tResult.setKeepCarriageReturn(keepCarriageReturn); tResult.setEnableSegmentCache(enableSegmentCache); + + tResult.setEnableAdaptivePipelineTaskSerialReadOnLimit(enableAdaptivePipelineTaskSerialReadOnLimit); + tResult.setAdaptivePipelineTaskSerialReadOnLimit(adaptivePipelineTaskSerialReadOnLimit); tResult.setInListValueCountThreshold(inListValueCountThreshold); return tResult; } 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..ca9735b56654b1 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,8 +179,7 @@ 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; @@ -188,9 +191,18 @@ public StatisticRange cover(StatisticRange other) { 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 +253,6 @@ public double getDistinctValues() { @Override public String toString() { - return "(" + lowExpr + "," + highExpr + ")"; + return "range=(" + lowExpr + "," + highExpr + "), ndv=" + distinctValues; } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/joinhint/DistributeHintTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/joinhint/DistributeHintTest.java index 0b2ed8069ade83..f96fe7e918c410 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/joinhint/DistributeHintTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/joinhint/DistributeHintTest.java @@ -28,13 +28,12 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; -import com.google.common.collect.Maps; +import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Set; public class DistributeHintTest extends TPCHTestBase { @@ -85,15 +84,15 @@ public void testHintJoin() { } private Plan generateLeadingHintPlan(int tableNum, Plan childPlan) { - Map hints = Maps.newLinkedHashMap(); + ImmutableList.Builder hints = ImmutableList.builder(); List leadingParameters = new ArrayList(); for (int i = 0; i < tableNum; i++) { leadingParameters.add(String.valueOf(i)); } Collections.shuffle(leadingParameters); System.out.println("LeadingHint: " + leadingParameters.toString()); - hints.put("leading", new SelectHintLeading("leading", leadingParameters)); - return new LogicalSelectHint<>(hints, childPlan); + hints.add(new SelectHintLeading("Leading", leadingParameters)); + return new LogicalSelectHint<>(hints.build(), childPlan); } private void randomTest(int tableNum, int edgeNum, boolean withJoinHint, boolean withLeading) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/MaxMinFilterPushDownTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/MaxMinFilterPushDownTest.java new file mode 100644 index 00000000000000..bc7d32fb3fb24a --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/MaxMinFilterPushDownTest.java @@ -0,0 +1,115 @@ +// 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.util.MemoPatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.utframe.TestWithFeService; + +import org.junit.jupiter.api.Test; + +public class MaxMinFilterPushDownTest extends TestWithFeService implements MemoPatternMatchSupported { + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + connectContext.setDatabase("test"); + createTable("CREATE TABLE IF NOT EXISTS max_t(\n" + + "`id` int(32),\n" + + "`score` int(64) NULL,\n" + + "`name` varchar(64) NULL\n" + + ") properties('replication_num'='1');"); + connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + } + + @Test + public void testMaxRewrite() { + String sql = "select id, max(score) from max_t group by id having max(score)>10"; + PlanChecker.from(connectContext).analyze(sql).rewrite() + .matches(logicalFilter(logicalOlapScan()).when(filter -> filter.getConjuncts().size() == 1)); + } + + @Test + public void testMinRewrite() { + String sql = "select id, min(score) from max_t group by id having min(score)<10"; + PlanChecker.from(connectContext).analyze(sql).rewrite() + .matches(logicalFilter(logicalOlapScan()).when(filter -> filter.getConjuncts().size() == 1)); + } + + @Test + public void testNotRewriteBecauseFuncIsMoreThanOne1() { + String sql = "select id, min(score), max(name) from max_t group by id having min(score)<10 and max(name)>'abc'"; + PlanChecker.from(connectContext).analyze(sql).rewrite() + .nonMatch(logicalFilter(logicalOlapScan())); + } + + @Test + public void testNotRewriteBecauseFuncIsMoreThanOne2() { + String sql = "select id, min(score), min(name) from max_t group by id having min(score)<10 and min(name)<'abc'"; + PlanChecker.from(connectContext).analyze(sql).rewrite() + .nonMatch(logicalFilter(logicalOlapScan())); + } + + @Test + public void testMaxNotRewriteBecauseLessThan() { + String sql = "select id, max(score) from max_t group by id having max(score)<10"; + PlanChecker.from(connectContext).analyze(sql).rewrite() + .nonMatch(logicalFilter(logicalOlapScan())); + } + + @Test + public void testMinNotRewriteBecauseGreaterThan() { + String sql = "select id, min(score) from max_t group by id having min(score)>10"; + PlanChecker.from(connectContext).analyze(sql).rewrite() + .nonMatch(logicalFilter(logicalOlapScan())); + } + + @Test + public void testMinNotRewriteBecauseHasMaxFunc() { + String sql = "select id, min(score), max(score) from max_t group by id having min(score)<10"; + PlanChecker.from(connectContext).analyze(sql).rewrite() + .nonMatch(logicalFilter(logicalOlapScan())); + } + + @Test + public void testMinNotRewriteBecauseHasCountFunc() { + String sql = "select id, min(score), count(score) from max_t group by id having min(score)<10"; + PlanChecker.from(connectContext).analyze(sql).rewrite() + .nonMatch(logicalFilter(logicalOlapScan())); + } + + @Test + public void testNotRewriteBecauseConjunctLeftNotSlot() { + String sql = "select id, max(score) from max_t group by id having abs(max(score))>10"; + PlanChecker.from(connectContext).analyze(sql).rewrite() + .nonMatch(logicalFilter(logicalOlapScan())); + } + + @Test + public void testRewriteAggFuncHasExpr() { + String sql = "select id, max(score+1) from max_t group by id having max(score+1)>10"; + PlanChecker.from(connectContext).analyze(sql).rewrite() + .matches(logicalFilter(logicalOlapScan()).when(filter -> filter.getConjuncts().size() == 1)); + } + + @Test + public void testNotRewriteScalarAgg() { + String sql = "select max(score+1) from max_t having max(score+1)>10"; + PlanChecker.from(connectContext).analyze(sql).rewrite() + .nonMatch(logicalFilter(logicalOlapScan())); + } +} 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 0158dd9587c70b..6e76c3f6a33d1c 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 @@ -1394,4 +1394,39 @@ public void testLargeRange() { out = estimation.estimate(greater, stats); Assertions.assertEquals(out.getRowCount(), row * FilterEstimation.RANGE_SELECTIVITY_THRESHOLD); } + + @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/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 85e4ade4ca4adc..9da87117154dcc 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -335,6 +335,9 @@ struct TQueryOptions { 127: optional i32 in_list_value_count_threshold = 10; + 128: optional bool enable_adaptive_pipeline_task_serial_read_on_limit = true; + 129: optional i32 adaptive_pipeline_task_serial_read_on_limit = 10000; + // For cloud, to control if the content would be written into file cache // In write path, to control if the content would be written into file cache. // In read path, read from file cache or remote storage when execute query. diff --git a/regression-test/data/inverted_index_p0/test_ignore_above_in_index.out b/regression-test/data/inverted_index_p0/test_ignore_above_in_index.out index 718bd29e5d9875..66d0935327dd4b 100644 --- a/regression-test/data/inverted_index_p0/test_ignore_above_in_index.out +++ b/regression-test/data/inverted_index_p0/test_ignore_above_in_index.out @@ -5,3 +5,6 @@ -- !sql -- 772 +-- !sql -- +971 + diff --git a/regression-test/data/nereids_p0/sql_functions/encryption_digest/test_encryption_function.out b/regression-test/data/nereids_p0/sql_functions/encryption_digest/test_encryption_function.out index c1f7d7b1c9d407..7a91c1dbf720ed 100644 --- a/regression-test/data/nereids_p0/sql_functions/encryption_digest/test_encryption_function.out +++ b/regression-test/data/nereids_p0/sql_functions/encryption_digest/test_encryption_function.out @@ -11,33 +11,6 @@ text -- !sql -- \N --- !sql -- -wr2JEDVXzL9+2XtRhgIloA== - --- !sql -- -wr2JEDVXzL9+2XtRhgIloA== - --- !sql -- -text - --- !sql -- -\N - --- !sql -- -wr2JEDVXzL9+2XtRhgIloA== - --- !sql -- -BO2vxHeUcw5BQQalSBbo1w== - --- !sql -- -text - --- !sql -- -\N - --- !sql -- -text - -- !sql -- BO2vxHeUcw5BQQalSBbo1w== @@ -54,25 +27,7 @@ text text -- !sql -- -wr2JEDVXzL9+2XtRhgIloA== - --- !sql -- -tsmK1HzbpnEdR2//WhO+MA== - --- !sql -- -ciacXDLHMNG7CD9Kws8png== - --- !sql -- -text - --- !sql -- -\N - --- !sql -- -text - --- !sql -- -text +3dym0E7/+1zbrLIaBVNHSw== -- !sql -- tsmK1HzbpnEdR2//WhO+MA== @@ -83,45 +38,12 @@ ciacXDLHMNG7CD9Kws8png== -- !sql -- \N --- !sql -- -text - --- !sql -- -text - --- !sql -- -aDjwRflBrDjhBZIOFNw3Tg== - --- !sql -- -1Y4NGIukSbv9OrkZnRD1bQ== - --- !sql -- -G5POcFAJwiZHeTtN6DjInQ== - --- !sql -- -text - --- !sql -- -text - -- !sql -- \N -- !sql -- text --- !sql -- -1Y4NGIukSbv9OrkZnRD1bQ== - --- !sql -- -G5POcFAJwiZHeTtN6DjInQ== - --- !sql -- -text - --- !sql -- -\N - -- !sql -- text diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out index 281de8ea61b88e..79b4ed890ded95 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out @@ -91,11 +91,11 @@ PhysicalResultSink -- !groupby_pushdown_having -- PhysicalResultSink ---filter((max(score) > 100)) -----hashAgg[GLOBAL] -------hashAgg[LOCAL] ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[max_t] +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------filter((t1.score > 100)) ----------PhysicalOlapScan[max_t] -- !groupby_pushdown_mixed_aggregates -- @@ -366,11 +366,11 @@ SyntaxError: -- !with_hint_groupby_pushdown_having -- PhysicalResultSink ---filter((max(score) > 100)) -----hashAgg[GLOBAL] -------hashAgg[LOCAL] ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[max_t] +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------filter((t1.score > 100)) ----------PhysicalOlapScan[max_t] Hint log: diff --git a/regression-test/data/nereids_rules_p0/max_min_filter_push_down/max_min_filter_push_down.out b/regression-test/data/nereids_rules_p0/max_min_filter_push_down/max_min_filter_push_down.out new file mode 100644 index 00000000000000..2e0ac41d5ebd31 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/max_min_filter_push_down/max_min_filter_push_down.out @@ -0,0 +1,290 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !scalar_agg_empty_table -- +PhysicalResultSink +--filter((min(value1) < 20)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalEmptyRelation + +-- !min -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down1.value1 < 20)) +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !max -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down1.value1 > 40)) +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !min_expr -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((cast(value1 as BIGINT) < 19)) +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !max_expr -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((abs(value1) > 39)) +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !min_commute -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down1.value1 < 40)) +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !max -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down1.value1 > 40)) +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !min_equal -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down1.value1 <= 20)) +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !max_equal -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down1.value1 >= 40)) +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !min_commute_equal -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down1.value1 <= 40)) +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !max_commute_equal -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down1.value1 >= 40)) +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !has_other_agg_func -- +PhysicalResultSink +--filter((max(value1) >= 40)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !min_scalar_agg -- +PhysicalResultSink +--filter((min(value1) < 40)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalStorageLayerAggregate[max_min_filter_push_down1] + +-- !max_scalar_agg -- +PhysicalResultSink +--filter((max(value1) > 40)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalStorageLayerAggregate[max_min_filter_push_down1] + +-- !max_scalar_agg -- +PhysicalResultSink +--filter((max(value1) > 40)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalStorageLayerAggregate[max_min_filter_push_down1] + +-- !min_equal_scalar_agg -- +PhysicalResultSink +--filter((min(value1) <= 20)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalStorageLayerAggregate[max_min_filter_push_down1] + +-- !max_equal_scalar_agg -- +PhysicalResultSink +--filter((max(value1) >= 40)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalStorageLayerAggregate[max_min_filter_push_down1] + +-- !depend_prune_column -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down1.value1 < 10)) +--------PhysicalOlapScan[max_min_filter_push_down1] + +-- !scalar_agg_empty_table_res -- + +-- !min_res -- +1 10 +2 19 + +-- !max_res -- +2 73 +3 61 +4 45 + +-- !min_expr_res -- +1 11 + +-- !max_expr_res -- +2 74 +3 62 +4 46 + +-- !min_commute_res -- +1 10 +2 19 +3 30 + +-- !max_res -- +2 73 +3 61 +4 45 + +-- !min_equal_res -- +1 10 +2 19 + +-- !max_equal_res -- +2 73 +3 61 +4 45 + +-- !min_commute_equal_res -- +1 10 +2 19 +3 30 +4 40 + +-- !max_commute_equal_res -- +2 73 +3 61 +4 45 + +-- !has_other_agg_func_res -- +2 73 19 +3 61 30 +4 45 40 + +-- !min_scalar_agg_res -- +10 + +-- !max_scalar_agg_res -- +73 + +-- !max_scalar_agg_res -- +73 + +-- !min_equal_scalar_agg_res -- +10 + +-- !max_equal_scalar_agg_res -- +73 + +-- !depend_prune_column_res -- +10 +19 + +-- !smallint -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down2.d_smallint > 10)) +--------PhysicalOlapScan[max_min_filter_push_down2] + +-- !tinyint -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down2.d_tinyint < 10)) +--------PhysicalOlapScan[max_min_filter_push_down2] + +-- !char100 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down2.d_char100 > 'ab')) +--------PhysicalOlapScan[max_min_filter_push_down2] + +-- !char100_cmp_num_cannot_rewrite -- +PhysicalResultSink +--filter((cast(min(d_char100) as DOUBLE) < 10.0)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalOlapScan[max_min_filter_push_down2] + +-- !datetimev2 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down2.d_datetimev2 < '2020-01-09 00:00:00')) +--------PhysicalOlapScan[max_min_filter_push_down2] + +-- !datev2 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down2.d_datev2 > '2020-01-09')) +--------PhysicalOlapScan[max_min_filter_push_down2] + +-- !smallint_group_by_key -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down2.d_smallint > 10)) +--------PhysicalOlapScan[max_min_filter_push_down2] + +-- !tinyint_group_by_key -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down2.d_tinyint < 10)) +--------PhysicalOlapScan[max_min_filter_push_down2] + +-- !char100_group_by_key -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((max_min_filter_push_down2.d_char100 > 'ab')) +--------PhysicalOlapScan[max_min_filter_push_down2] + +-- !smallint_res -- +14 32 + +-- !tinyint_res -- +1 3 + +-- !char100_res -- + +-- !char100_cmp_num_cannot_rewrite_res -- + +-- !datetimev2_res -- +1 2020-01-07T10:00:01 +14 2020-01-07T10:00:01 + +-- !datev2_res -- +1 2020-01-11 +14 2020-01-11 + +-- !smallint_group_by_key_res -- +29 +32 + +-- !tinyint_group_by_key_res -- +3 + +-- !char100_group_by_key_res -- + diff --git a/regression-test/data/nereids_rules_p0/mv/ssb/mv_ssb_test.out b/regression-test/data/nereids_rules_p0/mv/ssb/mv_ssb_test.out index ddb04ee99511fa..b3d45a8b45bdf1 100644 --- a/regression-test/data/nereids_rules_p0/mv/ssb/mv_ssb_test.out +++ b/regression-test/data/nereids_rules_p0/mv/ssb/mv_ssb_test.out @@ -1,3955 +1,617 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !query1_1_before -- -446031203850 +\N -- !query1_1_after -- -446031203850 +\N -- !query1_2_before -- -98714004603 +\N -- !query1_2_after -- -98714004603 +\N -- !query1_3before -- -26232701005 +\N -- !query1_3_after -- -26232701005 +\N -- !query2_1before -- -244739231 1998 MFGR#1238 -251549955 1998 MFGR#1228 -290208878 1998 MFGR#1225 -296330561 1998 MFGR#123 -304873002 1998 MFGR#1214 -312440027 1998 MFGR#122 -317068168 1998 MFGR#1211 -318769573 1998 MFGR#129 -320623334 1998 MFGR#1218 -330082371 1998 MFGR#124 -332740903 1998 MFGR#1213 -339845398 1998 MFGR#1221 -341618569 1998 MFGR#1237 -341657580 1998 MFGR#126 -344575925 1998 MFGR#121 -344889822 1998 MFGR#1223 -346182862 1998 MFGR#1219 -348123961 1998 MFGR#1220 -352277781 1998 MFGR#1235 -355416161 1998 MFGR#1222 -358466340 1998 MFGR#1227 -359508497 1998 MFGR#1217 -360289624 1998 MFGR#125 -361416497 1998 MFGR#128 -361827086 1998 MFGR#1236 -366101132 1998 MFGR#1215 -374341516 1998 MFGR#1212 -377507061 1998 MFGR#127 -379133898 1998 MFGR#1216 -383138860 1998 MFGR#1229 -396906691 1998 MFGR#1224 -398944492 1998 MFGR#1231 -401953419 1997 MFGR#1238 -406967188 1998 MFGR#1233 -414151803 1998 MFGR#1239 -415312453 1998 MFGR#1240 -417152416 1998 MFGR#1210 -419415707 1998 MFGR#1226 -424062455 1998 MFGR#1232 -424306670 1997 MFGR#1213 -428867240 1998 MFGR#1234 -437181243 1998 MFGR#1230 -444614010 1993 MFGR#1238 -450402292 1997 MFGR#1225 -466676148 1996 MFGR#1238 -468535087 1992 MFGR#1230 -473007381 1993 MFGR#1225 -476864333 1996 MFGR#125 -477423770 1992 MFGR#1238 -478648074 1996 MFGR#1235 -478777095 1992 MFGR#125 -479099365 1997 MFGR#125 -483838085 1996 MFGR#1227 -493270412 1994 MFGR#1230 -496297087 1996 MFGR#1213 -499319414 1996 MFGR#1221 -499852146 1995 MFGR#1231 -501892561 1992 MFGR#1236 -502601790 1997 MFGR#1231 -506011570 1997 MFGR#1227 -508749401 1995 MFGR#129 -511971910 1997 MFGR#1214 -515571416 1993 MFGR#1233 -517956131 1995 MFGR#1220 -518444215 1996 MFGR#1220 -519659003 1995 MFGR#1219 -523879145 1996 MFGR#121 -528843086 1996 MFGR#1218 -528879998 1996 MFGR#1230 -528988960 1997 MFGR#1222 -530361300 1995 MFGR#1227 -530830127 1993 MFGR#1220 -533544350 1993 MFGR#1223 -534002330 1994 MFGR#1214 -534455976 1997 MFGR#1233 -534541525 1994 MFGR#129 -535448651 1992 MFGR#1223 -537098211 1997 MFGR#1223 -538043594 1992 MFGR#1219 -538246872 1995 MFGR#125 -540262882 1992 MFGR#1220 -542306646 1992 MFGR#128 -542569815 1994 MFGR#1237 -543248087 1997 MFGR#124 -543346337 1993 MFGR#1221 -546674347 1994 MFGR#1218 -547960244 1995 MFGR#1229 -548048395 1993 MFGR#129 -548588761 1997 MFGR#1211 -549318912 1995 MFGR#1238 -550769662 1992 MFGR#1211 -551010618 1996 MFGR#128 -555134404 1994 MFGR#1223 -556120633 1992 MFGR#1221 -557149571 1995 MFGR#1233 -558030884 1996 MFGR#126 -558646341 1994 MFGR#1216 -560488304 1994 MFGR#125 -560570630 1996 MFGR#129 -560667719 1996 MFGR#124 -561169527 1997 MFGR#1219 -562570804 1995 MFGR#1214 -562582172 1994 MFGR#1220 -564368410 1995 MFGR#1222 -564405648 1994 MFGR#121 -566217852 1996 MFGR#123 -567272942 1994 MFGR#1219 -567838207 1992 MFGR#121 -568249365 1996 MFGR#1236 -568332348 1994 MFGR#1212 -570696568 1997 MFGR#126 -570745955 1994 MFGR#1225 -570832868 1992 MFGR#1225 -571149450 1996 MFGR#1234 -571800941 1997 MFGR#1230 -572354196 1992 MFGR#124 -572568748 1995 MFGR#1226 -572847270 1995 MFGR#1237 -573510781 1994 MFGR#1238 -573693547 1994 MFGR#1228 -575779480 1993 MFGR#1235 -579855853 1994 MFGR#1239 -580327635 1993 MFGR#1231 -580449592 1994 MFGR#123 -581759388 1992 MFGR#1227 -581987352 1992 MFGR#129 -583074592 1997 MFGR#127 -583249505 1996 MFGR#1214 -583507058 1993 MFGR#1228 -585421815 1995 MFGR#124 -585578737 1997 MFGR#1228 -586246330 1996 MFGR#1219 -586845664 1994 MFGR#128 -587013207 1997 MFGR#121 -588848171 1997 MFGR#1221 -589481194 1996 MFGR#1231 -589593892 1997 MFGR#1212 -590762777 1992 MFGR#1222 -591481503 1992 MFGR#1237 -591878667 1993 MFGR#1236 -592174616 1992 MFGR#126 -592178887 1993 MFGR#1227 -592436656 1992 MFGR#1231 -592616167 1996 MFGR#1237 -593851712 1995 MFGR#1239 -594466157 1997 MFGR#1220 -595065339 1996 MFGR#1211 -595256327 1993 MFGR#1239 -598618997 1994 MFGR#1221 -598680959 1994 MFGR#1235 -599586479 1995 MFGR#1213 -600202070 1992 MFGR#1218 -600665149 1995 MFGR#1223 -601016441 1994 MFGR#1222 -601809334 1996 MFGR#1216 -602735858 1995 MFGR#1230 -602892803 1995 MFGR#1210 -603546148 1994 MFGR#1231 -605799021 1995 MFGR#126 -609618576 1993 MFGR#1237 -609855391 1996 MFGR#1228 -610663790 1992 MFGR#1210 -610756714 1997 MFGR#1239 -613289283 1995 MFGR#1212 -613885100 1992 MFGR#1233 -614061593 1992 MFGR#1226 -615726097 1994 MFGR#1236 -616224539 1993 MFGR#125 -616287892 1997 MFGR#1210 -616837237 1993 MFGR#1214 -617126754 1993 MFGR#126 -617453491 1993 MFGR#1229 -618877179 1997 MFGR#1236 -622425239 1995 MFGR#1235 -622532984 1993 MFGR#122 -622571183 1994 MFGR#127 -622744016 1997 MFGR#1229 -624031241 1992 MFGR#1213 -625534310 1993 MFGR#1230 -628762754 1996 MFGR#1223 -631620635 1994 MFGR#1211 -631772246 1997 MFGR#1215 -633152470 1992 MFGR#1215 -633357085 1995 MFGR#1228 -634565501 1995 MFGR#1236 -634687975 1993 MFGR#1215 -634743898 1993 MFGR#1211 -635741351 1995 MFGR#1221 -635873891 1996 MFGR#127 -637863868 1993 MFGR#123 -638255029 1993 MFGR#1213 -638259374 1992 MFGR#1239 -638353900 1993 MFGR#1216 -638982238 1994 MFGR#1233 -639638057 1997 MFGR#1237 -640290070 1992 MFGR#1235 -640858430 1992 MFGR#1229 -643645053 1996 MFGR#1210 -644642592 1992 MFGR#1228 -645166092 1995 MFGR#1211 -645404849 1994 MFGR#1210 -646503168 1997 MFGR#123 -646950033 1993 MFGR#1219 -646960956 1995 MFGR#128 -647918373 1994 MFGR#1229 -648160706 1993 MFGR#1210 -649205856 1992 MFGR#1212 -651707481 1994 MFGR#1227 -651935758 1993 MFGR#1234 -654400242 1994 MFGR#1215 -654438324 1993 MFGR#127 -655326672 1992 MFGR#122 -655638776 1997 MFGR#129 -657026635 1995 MFGR#1218 -657036514 1994 MFGR#126 -658959557 1996 MFGR#1229 -659884062 1994 MFGR#122 -660586237 1993 MFGR#124 -660620587 1996 MFGR#1225 -660711077 1995 MFGR#123 -663372951 1993 MFGR#1217 -663679947 1996 MFGR#1233 -664275152 1992 MFGR#1232 -664916245 1997 MFGR#122 -665978112 1995 MFGR#127 -667399281 1992 MFGR#1234 -667674729 1996 MFGR#1226 -669000972 1995 MFGR#1216 -670488468 1992 MFGR#1214 -670693719 1996 MFGR#1239 -671669586 1995 MFGR#1225 -672528755 1995 MFGR#1215 -674626440 1996 MFGR#1212 -674763166 1997 MFGR#1224 -674846781 1992 MFGR#1216 -675093435 1992 MFGR#1217 -675132692 1997 MFGR#1240 -677924656 1997 MFGR#1232 -678785857 1994 MFGR#1213 -679469356 1996 MFGR#1222 -683985855 1993 MFGR#1218 -684573322 1994 MFGR#124 -687845641 1994 MFGR#1217 -692135140 1997 MFGR#1216 -695133104 1997 MFGR#128 -697373098 1993 MFGR#1232 -701360722 1997 MFGR#1226 -702184857 1996 MFGR#1215 -702805896 1996 MFGR#1232 -703752611 1992 MFGR#1224 -704898387 1996 MFGR#1217 -706151632 1992 MFGR#127 -706469511 1995 MFGR#121 -707053720 1997 MFGR#1218 -707207888 1995 MFGR#1240 -710023059 1995 MFGR#1234 -712110492 1996 MFGR#122 -714934715 1997 MFGR#1234 -715300753 1995 MFGR#1232 -719865331 1994 MFGR#1232 -724727741 1995 MFGR#122 -724844856 1996 MFGR#1224 -725362449 1995 MFGR#1217 -731657001 1993 MFGR#128 -737422302 1994 MFGR#1224 -739036124 1993 MFGR#1226 -740479248 1992 MFGR#1240 -743247677 1994 MFGR#1234 -746302245 1994 MFGR#1226 -756921203 1993 MFGR#1222 -762700351 1995 MFGR#1224 -767151420 1997 MFGR#1235 -777994957 1997 MFGR#1217 -785639283 1993 MFGR#1212 -788730059 1993 MFGR#1240 -789755835 1992 MFGR#123 -821167950 1996 MFGR#1240 -823087702 1993 MFGR#121 -873735737 1994 MFGR#1240 -915916085 1993 MFGR#1224 +15288453 1992 MFGR#1213 +16558051 1992 MFGR#1222 +17199862 1992 MFGR#123 +19716439 1992 MFGR#1216 +21732451 1992 MFGR#129 +22246540 1992 MFGR#1215 +22759602 1992 MFGR#1218 +23120066 1992 MFGR#1210 +23318799 1992 MFGR#1219 +24245603 1992 MFGR#124 +24678908 1992 MFGR#1230 +24946678 1992 MFGR#1233 +26231337 1992 MFGR#1231 +26690787 1992 MFGR#1223 +27691433 1992 MFGR#1227 +28194770 1992 MFGR#125 +29165996 1992 MFGR#121 +30954680 1992 MFGR#1212 +31379822 1992 MFGR#1239 +31443810 1992 MFGR#1237 +32513490 1992 MFGR#1228 +32608903 1992 MFGR#1225 +35514258 1992 MFGR#1229 +35881895 1992 MFGR#128 +36027836 1992 MFGR#127 +36330900 1992 MFGR#1232 +36431683 1992 MFGR#1234 +38878674 1992 MFGR#1221 +39368479 1992 MFGR#1235 +40503844 1992 MFGR#126 +43666251 1992 MFGR#1217 +44456974 1992 MFGR#1236 +47636685 1992 MFGR#1226 +49003021 1992 MFGR#1238 +49870826 1992 MFGR#1240 +51050565 1992 MFGR#1220 +52982362 1992 MFGR#1211 +74056106 1992 MFGR#122 +76498594 1992 MFGR#1224 +7655070 1992 MFGR#1214 -- !query2_1_after -- -244739231 1998 MFGR#1238 -251549955 1998 MFGR#1228 -290208878 1998 MFGR#1225 -296330561 1998 MFGR#123 -304873002 1998 MFGR#1214 -312440027 1998 MFGR#122 -317068168 1998 MFGR#1211 -318769573 1998 MFGR#129 -320623334 1998 MFGR#1218 -330082371 1998 MFGR#124 -332740903 1998 MFGR#1213 -339845398 1998 MFGR#1221 -341618569 1998 MFGR#1237 -341657580 1998 MFGR#126 -344575925 1998 MFGR#121 -344889822 1998 MFGR#1223 -346182862 1998 MFGR#1219 -348123961 1998 MFGR#1220 -352277781 1998 MFGR#1235 -355416161 1998 MFGR#1222 -358466340 1998 MFGR#1227 -359508497 1998 MFGR#1217 -360289624 1998 MFGR#125 -361416497 1998 MFGR#128 -361827086 1998 MFGR#1236 -366101132 1998 MFGR#1215 -374341516 1998 MFGR#1212 -377507061 1998 MFGR#127 -379133898 1998 MFGR#1216 -383138860 1998 MFGR#1229 -396906691 1998 MFGR#1224 -398944492 1998 MFGR#1231 -401953419 1997 MFGR#1238 -406967188 1998 MFGR#1233 -414151803 1998 MFGR#1239 -415312453 1998 MFGR#1240 -417152416 1998 MFGR#1210 -419415707 1998 MFGR#1226 -424062455 1998 MFGR#1232 -424306670 1997 MFGR#1213 -428867240 1998 MFGR#1234 -437181243 1998 MFGR#1230 -444614010 1993 MFGR#1238 -450402292 1997 MFGR#1225 -466676148 1996 MFGR#1238 -468535087 1992 MFGR#1230 -473007381 1993 MFGR#1225 -476864333 1996 MFGR#125 -477423770 1992 MFGR#1238 -478648074 1996 MFGR#1235 -478777095 1992 MFGR#125 -479099365 1997 MFGR#125 -483838085 1996 MFGR#1227 -493270412 1994 MFGR#1230 -496297087 1996 MFGR#1213 -499319414 1996 MFGR#1221 -499852146 1995 MFGR#1231 -501892561 1992 MFGR#1236 -502601790 1997 MFGR#1231 -506011570 1997 MFGR#1227 -508749401 1995 MFGR#129 -511971910 1997 MFGR#1214 -515571416 1993 MFGR#1233 -517956131 1995 MFGR#1220 -518444215 1996 MFGR#1220 -519659003 1995 MFGR#1219 -523879145 1996 MFGR#121 -528843086 1996 MFGR#1218 -528879998 1996 MFGR#1230 -528988960 1997 MFGR#1222 -530361300 1995 MFGR#1227 -530830127 1993 MFGR#1220 -533544350 1993 MFGR#1223 -534002330 1994 MFGR#1214 -534455976 1997 MFGR#1233 -534541525 1994 MFGR#129 -535448651 1992 MFGR#1223 -537098211 1997 MFGR#1223 -538043594 1992 MFGR#1219 -538246872 1995 MFGR#125 -540262882 1992 MFGR#1220 -542306646 1992 MFGR#128 -542569815 1994 MFGR#1237 -543248087 1997 MFGR#124 -543346337 1993 MFGR#1221 -546674347 1994 MFGR#1218 -547960244 1995 MFGR#1229 -548048395 1993 MFGR#129 -548588761 1997 MFGR#1211 -549318912 1995 MFGR#1238 -550769662 1992 MFGR#1211 -551010618 1996 MFGR#128 -555134404 1994 MFGR#1223 -556120633 1992 MFGR#1221 -557149571 1995 MFGR#1233 -558030884 1996 MFGR#126 -558646341 1994 MFGR#1216 -560488304 1994 MFGR#125 -560570630 1996 MFGR#129 -560667719 1996 MFGR#124 -561169527 1997 MFGR#1219 -562570804 1995 MFGR#1214 -562582172 1994 MFGR#1220 -564368410 1995 MFGR#1222 -564405648 1994 MFGR#121 -566217852 1996 MFGR#123 -567272942 1994 MFGR#1219 -567838207 1992 MFGR#121 -568249365 1996 MFGR#1236 -568332348 1994 MFGR#1212 -570696568 1997 MFGR#126 -570745955 1994 MFGR#1225 -570832868 1992 MFGR#1225 -571149450 1996 MFGR#1234 -571800941 1997 MFGR#1230 -572354196 1992 MFGR#124 -572568748 1995 MFGR#1226 -572847270 1995 MFGR#1237 -573510781 1994 MFGR#1238 -573693547 1994 MFGR#1228 -575779480 1993 MFGR#1235 -579855853 1994 MFGR#1239 -580327635 1993 MFGR#1231 -580449592 1994 MFGR#123 -581759388 1992 MFGR#1227 -581987352 1992 MFGR#129 -583074592 1997 MFGR#127 -583249505 1996 MFGR#1214 -583507058 1993 MFGR#1228 -585421815 1995 MFGR#124 -585578737 1997 MFGR#1228 -586246330 1996 MFGR#1219 -586845664 1994 MFGR#128 -587013207 1997 MFGR#121 -588848171 1997 MFGR#1221 -589481194 1996 MFGR#1231 -589593892 1997 MFGR#1212 -590762777 1992 MFGR#1222 -591481503 1992 MFGR#1237 -591878667 1993 MFGR#1236 -592174616 1992 MFGR#126 -592178887 1993 MFGR#1227 -592436656 1992 MFGR#1231 -592616167 1996 MFGR#1237 -593851712 1995 MFGR#1239 -594466157 1997 MFGR#1220 -595065339 1996 MFGR#1211 -595256327 1993 MFGR#1239 -598618997 1994 MFGR#1221 -598680959 1994 MFGR#1235 -599586479 1995 MFGR#1213 -600202070 1992 MFGR#1218 -600665149 1995 MFGR#1223 -601016441 1994 MFGR#1222 -601809334 1996 MFGR#1216 -602735858 1995 MFGR#1230 -602892803 1995 MFGR#1210 -603546148 1994 MFGR#1231 -605799021 1995 MFGR#126 -609618576 1993 MFGR#1237 -609855391 1996 MFGR#1228 -610663790 1992 MFGR#1210 -610756714 1997 MFGR#1239 -613289283 1995 MFGR#1212 -613885100 1992 MFGR#1233 -614061593 1992 MFGR#1226 -615726097 1994 MFGR#1236 -616224539 1993 MFGR#125 -616287892 1997 MFGR#1210 -616837237 1993 MFGR#1214 -617126754 1993 MFGR#126 -617453491 1993 MFGR#1229 -618877179 1997 MFGR#1236 -622425239 1995 MFGR#1235 -622532984 1993 MFGR#122 -622571183 1994 MFGR#127 -622744016 1997 MFGR#1229 -624031241 1992 MFGR#1213 -625534310 1993 MFGR#1230 -628762754 1996 MFGR#1223 -631620635 1994 MFGR#1211 -631772246 1997 MFGR#1215 -633152470 1992 MFGR#1215 -633357085 1995 MFGR#1228 -634565501 1995 MFGR#1236 -634687975 1993 MFGR#1215 -634743898 1993 MFGR#1211 -635741351 1995 MFGR#1221 -635873891 1996 MFGR#127 -637863868 1993 MFGR#123 -638255029 1993 MFGR#1213 -638259374 1992 MFGR#1239 -638353900 1993 MFGR#1216 -638982238 1994 MFGR#1233 -639638057 1997 MFGR#1237 -640290070 1992 MFGR#1235 -640858430 1992 MFGR#1229 -643645053 1996 MFGR#1210 -644642592 1992 MFGR#1228 -645166092 1995 MFGR#1211 -645404849 1994 MFGR#1210 -646503168 1997 MFGR#123 -646950033 1993 MFGR#1219 -646960956 1995 MFGR#128 -647918373 1994 MFGR#1229 -648160706 1993 MFGR#1210 -649205856 1992 MFGR#1212 -651707481 1994 MFGR#1227 -651935758 1993 MFGR#1234 -654400242 1994 MFGR#1215 -654438324 1993 MFGR#127 -655326672 1992 MFGR#122 -655638776 1997 MFGR#129 -657026635 1995 MFGR#1218 -657036514 1994 MFGR#126 -658959557 1996 MFGR#1229 -659884062 1994 MFGR#122 -660586237 1993 MFGR#124 -660620587 1996 MFGR#1225 -660711077 1995 MFGR#123 -663372951 1993 MFGR#1217 -663679947 1996 MFGR#1233 -664275152 1992 MFGR#1232 -664916245 1997 MFGR#122 -665978112 1995 MFGR#127 -667399281 1992 MFGR#1234 -667674729 1996 MFGR#1226 -669000972 1995 MFGR#1216 -670488468 1992 MFGR#1214 -670693719 1996 MFGR#1239 -671669586 1995 MFGR#1225 -672528755 1995 MFGR#1215 -674626440 1996 MFGR#1212 -674763166 1997 MFGR#1224 -674846781 1992 MFGR#1216 -675093435 1992 MFGR#1217 -675132692 1997 MFGR#1240 -677924656 1997 MFGR#1232 -678785857 1994 MFGR#1213 -679469356 1996 MFGR#1222 -683985855 1993 MFGR#1218 -684573322 1994 MFGR#124 -687845641 1994 MFGR#1217 -692135140 1997 MFGR#1216 -695133104 1997 MFGR#128 -697373098 1993 MFGR#1232 -701360722 1997 MFGR#1226 -702184857 1996 MFGR#1215 -702805896 1996 MFGR#1232 -703752611 1992 MFGR#1224 -704898387 1996 MFGR#1217 -706151632 1992 MFGR#127 -706469511 1995 MFGR#121 -707053720 1997 MFGR#1218 -707207888 1995 MFGR#1240 -710023059 1995 MFGR#1234 -712110492 1996 MFGR#122 -714934715 1997 MFGR#1234 -715300753 1995 MFGR#1232 -719865331 1994 MFGR#1232 -724727741 1995 MFGR#122 -724844856 1996 MFGR#1224 -725362449 1995 MFGR#1217 -731657001 1993 MFGR#128 -737422302 1994 MFGR#1224 -739036124 1993 MFGR#1226 -740479248 1992 MFGR#1240 -743247677 1994 MFGR#1234 -746302245 1994 MFGR#1226 -756921203 1993 MFGR#1222 -762700351 1995 MFGR#1224 -767151420 1997 MFGR#1235 -777994957 1997 MFGR#1217 -785639283 1993 MFGR#1212 -788730059 1993 MFGR#1240 -789755835 1992 MFGR#123 -821167950 1996 MFGR#1240 -823087702 1993 MFGR#121 -873735737 1994 MFGR#1240 -915916085 1993 MFGR#1224 +15288453 1992 MFGR#1213 +16558051 1992 MFGR#1222 +17199862 1992 MFGR#123 +19716439 1992 MFGR#1216 +21732451 1992 MFGR#129 +22246540 1992 MFGR#1215 +22759602 1992 MFGR#1218 +23120066 1992 MFGR#1210 +23318799 1992 MFGR#1219 +24245603 1992 MFGR#124 +24678908 1992 MFGR#1230 +24946678 1992 MFGR#1233 +26231337 1992 MFGR#1231 +26690787 1992 MFGR#1223 +27691433 1992 MFGR#1227 +28194770 1992 MFGR#125 +29165996 1992 MFGR#121 +30954680 1992 MFGR#1212 +31379822 1992 MFGR#1239 +31443810 1992 MFGR#1237 +32513490 1992 MFGR#1228 +32608903 1992 MFGR#1225 +35514258 1992 MFGR#1229 +35881895 1992 MFGR#128 +36027836 1992 MFGR#127 +36330900 1992 MFGR#1232 +36431683 1992 MFGR#1234 +38878674 1992 MFGR#1221 +39368479 1992 MFGR#1235 +40503844 1992 MFGR#126 +43666251 1992 MFGR#1217 +44456974 1992 MFGR#1236 +47636685 1992 MFGR#1226 +49003021 1992 MFGR#1238 +49870826 1992 MFGR#1240 +51050565 1992 MFGR#1220 +52982362 1992 MFGR#1211 +74056106 1992 MFGR#122 +76498594 1992 MFGR#1224 +7655070 1992 MFGR#1214 -- !query2_2before -- -335304504 1998 MFGR#2221 -390506405 1998 MFGR#2227 -397939103 1998 MFGR#2228 -409347137 1998 MFGR#2222 -410402095 1998 MFGR#2225 -413318072 1998 MFGR#2224 -453515044 1998 MFGR#2226 -459109577 1998 MFGR#2223 -589765707 1995 MFGR#2224 -630422081 1997 MFGR#2224 -637832575 1995 MFGR#2223 -646528589 1992 MFGR#2225 -651488962 1992 MFGR#2224 -656095314 1996 MFGR#2223 -656859917 1996 MFGR#2224 -664533779 1997 MFGR#2222 -666524807 1994 MFGR#2222 -669662707 1993 MFGR#2224 -680880216 1993 MFGR#2228 -685777518 1994 MFGR#2221 -691475597 1993 MFGR#2222 -700010008 1996 MFGR#2221 -707869040 1994 MFGR#2224 -708290039 1995 MFGR#2225 -709524929 1992 MFGR#2221 -709650548 1992 MFGR#2226 -720837128 1994 MFGR#2227 -721251967 1994 MFGR#2225 -727342382 1997 MFGR#2221 -728857899 1997 MFGR#2227 -729563303 1996 MFGR#2227 -733993590 1994 MFGR#2223 -737087518 1993 MFGR#2226 -745556316 1992 MFGR#2227 -747356383 1995 MFGR#2226 -747889257 1997 MFGR#2226 -748288392 1997 MFGR#2223 -756901875 1992 MFGR#2228 -757391203 1997 MFGR#2225 -758220752 1993 MFGR#2223 -761354792 1995 MFGR#2222 -765052002 1992 MFGR#2223 -765820896 1996 MFGR#2225 -766521103 1993 MFGR#2221 -773854228 1993 MFGR#2225 -775312985 1997 MFGR#2228 -775437074 1995 MFGR#2221 -777310085 1996 MFGR#2222 -781967766 1993 MFGR#2227 -783846394 1992 MFGR#2222 -802502540 1995 MFGR#2227 -808177734 1996 MFGR#2226 -819665874 1996 MFGR#2228 -822495919 1994 MFGR#2226 -826225350 1994 MFGR#2228 -895936786 1995 MFGR#2228 +20416501 1992 MFGR#2224 +28235270 1992 MFGR#2221 +39273349 1992 MFGR#2227 +48591160 1992 MFGR#2223 +60628045 1992 MFGR#2226 +64071827 1992 MFGR#2222 +66658087 1992 MFGR#2228 +74950776 1992 MFGR#2225 -- !query2_2_after -- -335304504 1998 MFGR#2221 -390506405 1998 MFGR#2227 -397939103 1998 MFGR#2228 -409347137 1998 MFGR#2222 -410402095 1998 MFGR#2225 -413318072 1998 MFGR#2224 -453515044 1998 MFGR#2226 -459109577 1998 MFGR#2223 -589765707 1995 MFGR#2224 -630422081 1997 MFGR#2224 -637832575 1995 MFGR#2223 -646528589 1992 MFGR#2225 -651488962 1992 MFGR#2224 -656095314 1996 MFGR#2223 -656859917 1996 MFGR#2224 -664533779 1997 MFGR#2222 -666524807 1994 MFGR#2222 -669662707 1993 MFGR#2224 -680880216 1993 MFGR#2228 -685777518 1994 MFGR#2221 -691475597 1993 MFGR#2222 -700010008 1996 MFGR#2221 -707869040 1994 MFGR#2224 -708290039 1995 MFGR#2225 -709524929 1992 MFGR#2221 -709650548 1992 MFGR#2226 -720837128 1994 MFGR#2227 -721251967 1994 MFGR#2225 -727342382 1997 MFGR#2221 -728857899 1997 MFGR#2227 -729563303 1996 MFGR#2227 -733993590 1994 MFGR#2223 -737087518 1993 MFGR#2226 -745556316 1992 MFGR#2227 -747356383 1995 MFGR#2226 -747889257 1997 MFGR#2226 -748288392 1997 MFGR#2223 -756901875 1992 MFGR#2228 -757391203 1997 MFGR#2225 -758220752 1993 MFGR#2223 -761354792 1995 MFGR#2222 -765052002 1992 MFGR#2223 -765820896 1996 MFGR#2225 -766521103 1993 MFGR#2221 -773854228 1993 MFGR#2225 -775312985 1997 MFGR#2228 -775437074 1995 MFGR#2221 -777310085 1996 MFGR#2222 -781967766 1993 MFGR#2227 -783846394 1992 MFGR#2222 -802502540 1995 MFGR#2227 -808177734 1996 MFGR#2226 -819665874 1996 MFGR#2228 -822495919 1994 MFGR#2226 -826225350 1994 MFGR#2228 -895936786 1995 MFGR#2228 +20416501 1992 MFGR#2224 +28235270 1992 MFGR#2221 +39273349 1992 MFGR#2227 +48591160 1992 MFGR#2223 +60628045 1992 MFGR#2226 +64071827 1992 MFGR#2222 +66658087 1992 MFGR#2228 +74950776 1992 MFGR#2225 -- !query2_3before -- -380255731 1998 MFGR#2239 -664145134 1994 MFGR#2239 -705238959 1993 MFGR#2239 -713839336 1995 MFGR#2239 -726427486 1992 MFGR#2239 -728492865 1996 MFGR#2239 -733896532 1997 MFGR#2239 +89380397 1992 MFGR#2239 -- !query2_3_after -- -380255731 1998 MFGR#2239 -664145134 1994 MFGR#2239 -705238959 1993 MFGR#2239 -713839336 1995 MFGR#2239 -726427486 1992 MFGR#2239 -728492865 1996 MFGR#2239 -733896532 1997 MFGR#2239 +89380397 1992 MFGR#2239 -- !query3_1before -- -CHINA CHINA 1992 6456431477 -CHINA CHINA 1993 6444513787 -CHINA CHINA 1994 6648942965 -CHINA CHINA 1995 6624666612 -CHINA CHINA 1996 6591697915 -CHINA CHINA 1997 6463794795 -CHINA INDIA 1992 6189831351 -CHINA INDIA 1993 6421694121 -CHINA INDIA 1994 6297582811 -CHINA INDIA 1995 6246656374 -CHINA INDIA 1996 6153787365 -CHINA INDIA 1997 6084283983 -CHINA INDONESIA 1992 6616078869 -CHINA INDONESIA 1993 6581513366 -CHINA INDONESIA 1994 6234787896 -CHINA INDONESIA 1995 7074539444 -CHINA INDONESIA 1996 6657125075 -CHINA INDONESIA 1997 6653234696 -CHINA JAPAN 1992 6157678893 -CHINA JAPAN 1993 6195466884 -CHINA JAPAN 1994 6349037021 -CHINA JAPAN 1995 6021854130 -CHINA JAPAN 1996 6105566324 -CHINA JAPAN 1997 6284733518 -CHINA VIETNAM 1992 5412322026 -CHINA VIETNAM 1993 5885023228 -CHINA VIETNAM 1994 5885318733 -CHINA VIETNAM 1995 5489936063 -CHINA VIETNAM 1996 5434772818 -CHINA VIETNAM 1997 5908254176 -INDIA CHINA 1992 6066461811 -INDIA CHINA 1993 6209353792 -INDIA CHINA 1994 5815022714 -INDIA CHINA 1995 6010532793 -INDIA CHINA 1996 6198567819 -INDIA CHINA 1997 6153282214 -INDIA INDIA 1992 5361243755 -INDIA INDIA 1993 5571575799 -INDIA INDIA 1994 5584936515 -INDIA INDIA 1995 5565995550 -INDIA INDIA 1996 5532073103 -INDIA INDIA 1997 5693028907 -INDIA INDONESIA 1992 6238807354 -INDIA INDONESIA 1993 6331058594 -INDIA INDONESIA 1994 6168189859 -INDIA INDONESIA 1995 5975518597 -INDIA INDONESIA 1996 6189242058 -INDIA INDONESIA 1997 5983439300 -INDIA JAPAN 1992 5421959281 -INDIA JAPAN 1993 5995208985 -INDIA JAPAN 1994 5394134974 -INDIA JAPAN 1995 5469203427 -INDIA JAPAN 1996 5841664805 -INDIA JAPAN 1997 5587827913 -INDIA VIETNAM 1992 5245330621 -INDIA VIETNAM 1993 5233871487 -INDIA VIETNAM 1994 5251039815 -INDIA VIETNAM 1995 5207063284 -INDIA VIETNAM 1996 5293891531 -INDIA VIETNAM 1997 5003624837 -INDONESIA CHINA 1992 6310709533 -INDONESIA CHINA 1993 6612891249 -INDONESIA CHINA 1994 6483600910 -INDONESIA CHINA 1995 6928620620 -INDONESIA CHINA 1996 6613439112 -INDONESIA CHINA 1997 6577396314 -INDONESIA INDIA 1992 5770203218 -INDONESIA INDIA 1993 6151664306 -INDONESIA INDIA 1994 6198412245 -INDONESIA INDIA 1995 6173805183 -INDONESIA INDIA 1996 6138810863 -INDONESIA INDIA 1997 5789367105 -INDONESIA INDONESIA 1992 6905698988 -INDONESIA INDONESIA 1993 6931363262 -INDONESIA INDONESIA 1994 6315331086 -INDONESIA INDONESIA 1995 6268531321 -INDONESIA INDONESIA 1996 6750056283 -INDONESIA INDONESIA 1997 6398427821 -INDONESIA JAPAN 1992 5663765610 -INDONESIA JAPAN 1993 6147605249 -INDONESIA JAPAN 1994 6141195969 -INDONESIA JAPAN 1995 6019093871 -INDONESIA JAPAN 1996 6055638222 -INDONESIA JAPAN 1997 6001427956 -INDONESIA VIETNAM 1992 5302638372 -INDONESIA VIETNAM 1993 5840287722 -INDONESIA VIETNAM 1994 5624346368 -INDONESIA VIETNAM 1995 5574974066 -INDONESIA VIETNAM 1996 5438514387 -INDONESIA VIETNAM 1997 5665678132 -JAPAN CHINA 1992 6067100655 -JAPAN CHINA 1993 6292293995 -JAPAN CHINA 1994 6320009218 -JAPAN CHINA 1995 6519889385 -JAPAN CHINA 1996 6088985564 -JAPAN CHINA 1997 5973849687 -JAPAN INDIA 1992 5802749985 -JAPAN INDIA 1993 5823343157 -JAPAN INDIA 1994 5645818872 -JAPAN INDIA 1995 5730743126 -JAPAN INDIA 1996 5644579616 -JAPAN INDIA 1997 5796921992 -JAPAN INDONESIA 1992 6222565638 -JAPAN INDONESIA 1993 6053946313 -JAPAN INDONESIA 1994 6013034618 -JAPAN INDONESIA 1995 5947000735 -JAPAN INDONESIA 1996 6227447856 -JAPAN INDONESIA 1997 6429270126 -JAPAN JAPAN 1992 5648034711 -JAPAN JAPAN 1993 5430318583 -JAPAN JAPAN 1994 5753821482 -JAPAN JAPAN 1995 5952279129 -JAPAN JAPAN 1996 5566204795 -JAPAN JAPAN 1997 5790450039 -JAPAN VIETNAM 1992 5401692555 -JAPAN VIETNAM 1993 5406569653 -JAPAN VIETNAM 1994 5275821637 -JAPAN VIETNAM 1995 5501313775 -JAPAN VIETNAM 1996 5334277549 -JAPAN VIETNAM 1997 5249874614 -VIETNAM CHINA 1992 6332844390 -VIETNAM CHINA 1993 6413579999 -VIETNAM CHINA 1994 6709883442 -VIETNAM CHINA 1995 6332731268 -VIETNAM CHINA 1996 6428653890 -VIETNAM CHINA 1997 6164218723 -VIETNAM INDIA 1992 5698511401 -VIETNAM INDIA 1993 6161174477 -VIETNAM INDIA 1994 5789398691 -VIETNAM INDIA 1995 6198441957 -VIETNAM INDIA 1996 5807486983 -VIETNAM INDIA 1997 5730121608 -VIETNAM INDONESIA 1992 6340181707 -VIETNAM INDONESIA 1993 6511639352 -VIETNAM INDONESIA 1994 6347441931 -VIETNAM INDONESIA 1995 6057820148 -VIETNAM INDONESIA 1996 6295626732 -VIETNAM INDONESIA 1997 6363583120 -VIETNAM JAPAN 1992 5863786349 -VIETNAM JAPAN 1993 5874266867 -VIETNAM JAPAN 1994 6231973047 -VIETNAM JAPAN 1995 6263957020 -VIETNAM JAPAN 1996 5919997150 -VIETNAM JAPAN 1997 5689345230 -VIETNAM VIETNAM 1992 5111731778 -VIETNAM VIETNAM 1993 5554161183 -VIETNAM VIETNAM 1994 5418498159 -VIETNAM VIETNAM 1995 5754727607 -VIETNAM VIETNAM 1996 5860842302 -VIETNAM VIETNAM 1997 5620251324 +CHINA CHINA 1992 614550901 +CHINA INDIA 1992 348359904 +CHINA INDONESIA 1992 352903905 +CHINA JAPAN 1992 327558220 +CHINA VIETNAM 1992 324763767 +INDIA CHINA 1992 561966207 +INDIA INDIA 1992 329354089 +INDIA INDONESIA 1992 487449629 +INDIA JAPAN 1992 412186106 +INDIA VIETNAM 1992 334582962 +INDONESIA CHINA 1992 621316255 +INDONESIA INDIA 1992 310417666 +INDONESIA INDONESIA 1992 477417717 +INDONESIA JAPAN 1992 465870469 +INDONESIA VIETNAM 1992 278083418 +JAPAN CHINA 1992 637991852 +JAPAN INDIA 1992 393835589 +JAPAN INDONESIA 1992 476513261 +JAPAN JAPAN 1992 399179790 +JAPAN VIETNAM 1992 468999429 +VIETNAM CHINA 1992 621845377 +VIETNAM INDIA 1992 296225919 +VIETNAM INDONESIA 1992 462424521 +VIETNAM JAPAN 1992 395247587 +VIETNAM VIETNAM 1992 342176333 -- !query3_1_after -- -CHINA CHINA 1992 6456431477 -CHINA CHINA 1993 6444513787 -CHINA CHINA 1994 6648942965 -CHINA CHINA 1995 6624666612 -CHINA CHINA 1996 6591697915 -CHINA CHINA 1997 6463794795 -CHINA INDIA 1992 6189831351 -CHINA INDIA 1993 6421694121 -CHINA INDIA 1994 6297582811 -CHINA INDIA 1995 6246656374 -CHINA INDIA 1996 6153787365 -CHINA INDIA 1997 6084283983 -CHINA INDONESIA 1992 6616078869 -CHINA INDONESIA 1993 6581513366 -CHINA INDONESIA 1994 6234787896 -CHINA INDONESIA 1995 7074539444 -CHINA INDONESIA 1996 6657125075 -CHINA INDONESIA 1997 6653234696 -CHINA JAPAN 1992 6157678893 -CHINA JAPAN 1993 6195466884 -CHINA JAPAN 1994 6349037021 -CHINA JAPAN 1995 6021854130 -CHINA JAPAN 1996 6105566324 -CHINA JAPAN 1997 6284733518 -CHINA VIETNAM 1992 5412322026 -CHINA VIETNAM 1993 5885023228 -CHINA VIETNAM 1994 5885318733 -CHINA VIETNAM 1995 5489936063 -CHINA VIETNAM 1996 5434772818 -CHINA VIETNAM 1997 5908254176 -INDIA CHINA 1992 6066461811 -INDIA CHINA 1993 6209353792 -INDIA CHINA 1994 5815022714 -INDIA CHINA 1995 6010532793 -INDIA CHINA 1996 6198567819 -INDIA CHINA 1997 6153282214 -INDIA INDIA 1992 5361243755 -INDIA INDIA 1993 5571575799 -INDIA INDIA 1994 5584936515 -INDIA INDIA 1995 5565995550 -INDIA INDIA 1996 5532073103 -INDIA INDIA 1997 5693028907 -INDIA INDONESIA 1992 6238807354 -INDIA INDONESIA 1993 6331058594 -INDIA INDONESIA 1994 6168189859 -INDIA INDONESIA 1995 5975518597 -INDIA INDONESIA 1996 6189242058 -INDIA INDONESIA 1997 5983439300 -INDIA JAPAN 1992 5421959281 -INDIA JAPAN 1993 5995208985 -INDIA JAPAN 1994 5394134974 -INDIA JAPAN 1995 5469203427 -INDIA JAPAN 1996 5841664805 -INDIA JAPAN 1997 5587827913 -INDIA VIETNAM 1992 5245330621 -INDIA VIETNAM 1993 5233871487 -INDIA VIETNAM 1994 5251039815 -INDIA VIETNAM 1995 5207063284 -INDIA VIETNAM 1996 5293891531 -INDIA VIETNAM 1997 5003624837 -INDONESIA CHINA 1992 6310709533 -INDONESIA CHINA 1993 6612891249 -INDONESIA CHINA 1994 6483600910 -INDONESIA CHINA 1995 6928620620 -INDONESIA CHINA 1996 6613439112 -INDONESIA CHINA 1997 6577396314 -INDONESIA INDIA 1992 5770203218 -INDONESIA INDIA 1993 6151664306 -INDONESIA INDIA 1994 6198412245 -INDONESIA INDIA 1995 6173805183 -INDONESIA INDIA 1996 6138810863 -INDONESIA INDIA 1997 5789367105 -INDONESIA INDONESIA 1992 6905698988 -INDONESIA INDONESIA 1993 6931363262 -INDONESIA INDONESIA 1994 6315331086 -INDONESIA INDONESIA 1995 6268531321 -INDONESIA INDONESIA 1996 6750056283 -INDONESIA INDONESIA 1997 6398427821 -INDONESIA JAPAN 1992 5663765610 -INDONESIA JAPAN 1993 6147605249 -INDONESIA JAPAN 1994 6141195969 -INDONESIA JAPAN 1995 6019093871 -INDONESIA JAPAN 1996 6055638222 -INDONESIA JAPAN 1997 6001427956 -INDONESIA VIETNAM 1992 5302638372 -INDONESIA VIETNAM 1993 5840287722 -INDONESIA VIETNAM 1994 5624346368 -INDONESIA VIETNAM 1995 5574974066 -INDONESIA VIETNAM 1996 5438514387 -INDONESIA VIETNAM 1997 5665678132 -JAPAN CHINA 1992 6067100655 -JAPAN CHINA 1993 6292293995 -JAPAN CHINA 1994 6320009218 -JAPAN CHINA 1995 6519889385 -JAPAN CHINA 1996 6088985564 -JAPAN CHINA 1997 5973849687 -JAPAN INDIA 1992 5802749985 -JAPAN INDIA 1993 5823343157 -JAPAN INDIA 1994 5645818872 -JAPAN INDIA 1995 5730743126 -JAPAN INDIA 1996 5644579616 -JAPAN INDIA 1997 5796921992 -JAPAN INDONESIA 1992 6222565638 -JAPAN INDONESIA 1993 6053946313 -JAPAN INDONESIA 1994 6013034618 -JAPAN INDONESIA 1995 5947000735 -JAPAN INDONESIA 1996 6227447856 -JAPAN INDONESIA 1997 6429270126 -JAPAN JAPAN 1992 5648034711 -JAPAN JAPAN 1993 5430318583 -JAPAN JAPAN 1994 5753821482 -JAPAN JAPAN 1995 5952279129 -JAPAN JAPAN 1996 5566204795 -JAPAN JAPAN 1997 5790450039 -JAPAN VIETNAM 1992 5401692555 -JAPAN VIETNAM 1993 5406569653 -JAPAN VIETNAM 1994 5275821637 -JAPAN VIETNAM 1995 5501313775 -JAPAN VIETNAM 1996 5334277549 -JAPAN VIETNAM 1997 5249874614 -VIETNAM CHINA 1992 6332844390 -VIETNAM CHINA 1993 6413579999 -VIETNAM CHINA 1994 6709883442 -VIETNAM CHINA 1995 6332731268 -VIETNAM CHINA 1996 6428653890 -VIETNAM CHINA 1997 6164218723 -VIETNAM INDIA 1992 5698511401 -VIETNAM INDIA 1993 6161174477 -VIETNAM INDIA 1994 5789398691 -VIETNAM INDIA 1995 6198441957 -VIETNAM INDIA 1996 5807486983 -VIETNAM INDIA 1997 5730121608 -VIETNAM INDONESIA 1992 6340181707 -VIETNAM INDONESIA 1993 6511639352 -VIETNAM INDONESIA 1994 6347441931 -VIETNAM INDONESIA 1995 6057820148 -VIETNAM INDONESIA 1996 6295626732 -VIETNAM INDONESIA 1997 6363583120 -VIETNAM JAPAN 1992 5863786349 -VIETNAM JAPAN 1993 5874266867 -VIETNAM JAPAN 1994 6231973047 -VIETNAM JAPAN 1995 6263957020 -VIETNAM JAPAN 1996 5919997150 -VIETNAM JAPAN 1997 5689345230 -VIETNAM VIETNAM 1992 5111731778 -VIETNAM VIETNAM 1993 5554161183 -VIETNAM VIETNAM 1994 5418498159 -VIETNAM VIETNAM 1995 5754727607 -VIETNAM VIETNAM 1996 5860842302 -VIETNAM VIETNAM 1997 5620251324 +CHINA CHINA 1992 614550901 +CHINA INDIA 1992 348359904 +CHINA INDONESIA 1992 352903905 +CHINA JAPAN 1992 327558220 +CHINA VIETNAM 1992 324763767 +INDIA CHINA 1992 561966207 +INDIA INDIA 1992 329354089 +INDIA INDONESIA 1992 487449629 +INDIA JAPAN 1992 412186106 +INDIA VIETNAM 1992 334582962 +INDONESIA CHINA 1992 621316255 +INDONESIA INDIA 1992 310417666 +INDONESIA INDONESIA 1992 477417717 +INDONESIA JAPAN 1992 465870469 +INDONESIA VIETNAM 1992 278083418 +JAPAN CHINA 1992 637991852 +JAPAN INDIA 1992 393835589 +JAPAN INDONESIA 1992 476513261 +JAPAN JAPAN 1992 399179790 +JAPAN VIETNAM 1992 468999429 +VIETNAM CHINA 1992 621845377 +VIETNAM INDIA 1992 296225919 +VIETNAM INDONESIA 1992 462424521 +VIETNAM JAPAN 1992 395247587 +VIETNAM VIETNAM 1992 342176333 -- !query3_2before -- -UNITED ST0 UNITED ST0 1992 75622122 -UNITED ST0 UNITED ST0 1993 106742548 -UNITED ST0 UNITED ST0 1994 83526877 -UNITED ST0 UNITED ST0 1995 38984502 -UNITED ST0 UNITED ST0 1996 61528855 -UNITED ST0 UNITED ST0 1997 66308516 -UNITED ST0 UNITED ST1 1992 77267396 -UNITED ST0 UNITED ST1 1993 91196593 -UNITED ST0 UNITED ST1 1994 76613379 -UNITED ST0 UNITED ST1 1995 90763599 -UNITED ST0 UNITED ST1 1996 91948777 -UNITED ST0 UNITED ST1 1997 80167139 -UNITED ST0 UNITED ST2 1992 27718733 -UNITED ST0 UNITED ST2 1993 15042145 -UNITED ST0 UNITED ST2 1994 22017954 -UNITED ST0 UNITED ST2 1995 4776460 -UNITED ST0 UNITED ST2 1996 6907199 -UNITED ST0 UNITED ST2 1997 23584229 -UNITED ST0 UNITED ST3 1992 56359317 -UNITED ST0 UNITED ST3 1993 76171283 -UNITED ST0 UNITED ST3 1994 27327829 -UNITED ST0 UNITED ST3 1995 56360708 -UNITED ST0 UNITED ST3 1996 29795665 -UNITED ST0 UNITED ST3 1997 40078508 -UNITED ST0 UNITED ST4 1992 49676807 -UNITED ST0 UNITED ST4 1993 49260067 -UNITED ST0 UNITED ST4 1994 61519744 -UNITED ST0 UNITED ST4 1995 29170038 -UNITED ST0 UNITED ST4 1996 103228749 -UNITED ST0 UNITED ST4 1997 41193803 -UNITED ST0 UNITED ST5 1992 40120886 -UNITED ST0 UNITED ST5 1993 52701052 -UNITED ST0 UNITED ST5 1994 41808301 -UNITED ST0 UNITED ST5 1995 11453709 -UNITED ST0 UNITED ST5 1996 57762895 -UNITED ST0 UNITED ST5 1997 47988418 -UNITED ST0 UNITED ST6 1992 42471806 -UNITED ST0 UNITED ST6 1993 50432282 -UNITED ST0 UNITED ST6 1994 61288778 -UNITED ST0 UNITED ST6 1995 40718942 -UNITED ST0 UNITED ST6 1996 37232062 -UNITED ST0 UNITED ST6 1997 22924046 -UNITED ST0 UNITED ST7 1992 50583409 -UNITED ST0 UNITED ST7 1993 48796360 -UNITED ST0 UNITED ST7 1994 61480820 -UNITED ST0 UNITED ST7 1995 33395313 -UNITED ST0 UNITED ST7 1996 31391181 -UNITED ST0 UNITED ST7 1997 72577490 -UNITED ST0 UNITED ST8 1992 49732640 -UNITED ST0 UNITED ST8 1993 42002682 -UNITED ST0 UNITED ST8 1994 31845433 -UNITED ST0 UNITED ST8 1995 16393249 -UNITED ST0 UNITED ST8 1996 15168262 -UNITED ST0 UNITED ST8 1997 30241370 -UNITED ST0 UNITED ST9 1992 63979996 -UNITED ST0 UNITED ST9 1993 109834040 -UNITED ST0 UNITED ST9 1994 89865202 -UNITED ST0 UNITED ST9 1995 63253499 -UNITED ST0 UNITED ST9 1996 75935209 -UNITED ST0 UNITED ST9 1997 119856268 -UNITED ST1 UNITED ST0 1992 96989491 -UNITED ST1 UNITED ST0 1993 99305952 -UNITED ST1 UNITED ST0 1994 43225201 -UNITED ST1 UNITED ST0 1995 81676623 -UNITED ST1 UNITED ST0 1996 94861554 -UNITED ST1 UNITED ST0 1997 77667866 -UNITED ST1 UNITED ST1 1992 64669070 -UNITED ST1 UNITED ST1 1993 47538610 -UNITED ST1 UNITED ST1 1994 54383174 -UNITED ST1 UNITED ST1 1995 64199332 -UNITED ST1 UNITED ST1 1996 87265848 -UNITED ST1 UNITED ST1 1997 76340444 -UNITED ST1 UNITED ST2 1992 7706728 -UNITED ST1 UNITED ST2 1993 16844538 -UNITED ST1 UNITED ST2 1994 23556586 -UNITED ST1 UNITED ST2 1995 7991676 -UNITED ST1 UNITED ST2 1996 10082536 -UNITED ST1 UNITED ST2 1997 20599130 -UNITED ST1 UNITED ST3 1992 55727474 -UNITED ST1 UNITED ST3 1993 52097946 -UNITED ST1 UNITED ST3 1994 75992459 -UNITED ST1 UNITED ST3 1995 45823056 -UNITED ST1 UNITED ST3 1996 77610905 -UNITED ST1 UNITED ST3 1997 85175638 -UNITED ST1 UNITED ST4 1992 39326346 -UNITED ST1 UNITED ST4 1993 62884347 -UNITED ST1 UNITED ST4 1994 41381558 -UNITED ST1 UNITED ST4 1995 57535599 -UNITED ST1 UNITED ST4 1996 50135547 -UNITED ST1 UNITED ST4 1997 83092569 -UNITED ST1 UNITED ST5 1992 33971251 -UNITED ST1 UNITED ST5 1993 40693513 -UNITED ST1 UNITED ST5 1994 24939138 -UNITED ST1 UNITED ST5 1995 31857452 -UNITED ST1 UNITED ST5 1996 51189649 -UNITED ST1 UNITED ST5 1997 51052110 -UNITED ST1 UNITED ST6 1992 35404990 -UNITED ST1 UNITED ST6 1993 44728540 -UNITED ST1 UNITED ST6 1994 31287326 -UNITED ST1 UNITED ST6 1995 27139403 -UNITED ST1 UNITED ST6 1996 54056366 -UNITED ST1 UNITED ST6 1997 18205326 -UNITED ST1 UNITED ST7 1992 71472973 -UNITED ST1 UNITED ST7 1993 54851183 -UNITED ST1 UNITED ST7 1994 53853106 -UNITED ST1 UNITED ST7 1995 61515741 -UNITED ST1 UNITED ST7 1996 41721506 -UNITED ST1 UNITED ST7 1997 77045722 -UNITED ST1 UNITED ST8 1992 28565579 -UNITED ST1 UNITED ST8 1993 66146198 -UNITED ST1 UNITED ST8 1994 30245153 -UNITED ST1 UNITED ST8 1995 34918793 -UNITED ST1 UNITED ST8 1996 39430882 -UNITED ST1 UNITED ST8 1997 15375258 -UNITED ST1 UNITED ST9 1992 75367059 -UNITED ST1 UNITED ST9 1993 52514902 -UNITED ST1 UNITED ST9 1994 43000037 -UNITED ST1 UNITED ST9 1995 72470870 -UNITED ST1 UNITED ST9 1996 69823769 -UNITED ST1 UNITED ST9 1997 48944111 -UNITED ST2 UNITED ST0 1992 72210218 -UNITED ST2 UNITED ST0 1993 42845485 -UNITED ST2 UNITED ST0 1994 60182153 -UNITED ST2 UNITED ST0 1995 52560838 -UNITED ST2 UNITED ST0 1996 67095122 -UNITED ST2 UNITED ST0 1997 77591470 -UNITED ST2 UNITED ST1 1992 81839165 -UNITED ST2 UNITED ST1 1993 69631701 -UNITED ST2 UNITED ST1 1994 81300089 -UNITED ST2 UNITED ST1 1995 45437340 -UNITED ST2 UNITED ST1 1996 88256155 -UNITED ST2 UNITED ST1 1997 65396135 -UNITED ST2 UNITED ST2 1992 25743654 -UNITED ST2 UNITED ST2 1993 14521995 -UNITED ST2 UNITED ST2 1994 18068657 -UNITED ST2 UNITED ST2 1995 13455031 -UNITED ST2 UNITED ST2 1997 37034333 -UNITED ST2 UNITED ST3 1992 35981963 -UNITED ST2 UNITED ST3 1993 66972438 -UNITED ST2 UNITED ST3 1994 65620760 -UNITED ST2 UNITED ST3 1995 65218129 -UNITED ST2 UNITED ST3 1996 59648598 -UNITED ST2 UNITED ST3 1997 65429453 -UNITED ST2 UNITED ST4 1992 91180500 -UNITED ST2 UNITED ST4 1993 56650391 -UNITED ST2 UNITED ST4 1994 55679377 -UNITED ST2 UNITED ST4 1995 82432305 -UNITED ST2 UNITED ST4 1996 57487057 -UNITED ST2 UNITED ST4 1997 57012611 -UNITED ST2 UNITED ST5 1992 57021365 -UNITED ST2 UNITED ST5 1993 37406845 -UNITED ST2 UNITED ST5 1994 37640317 -UNITED ST2 UNITED ST5 1995 43382417 -UNITED ST2 UNITED ST5 1996 47028976 -UNITED ST2 UNITED ST5 1997 40662886 -UNITED ST2 UNITED ST6 1992 27813739 -UNITED ST2 UNITED ST6 1993 26482226 -UNITED ST2 UNITED ST6 1994 41526105 -UNITED ST2 UNITED ST6 1995 37146288 -UNITED ST2 UNITED ST6 1996 45416649 -UNITED ST2 UNITED ST6 1997 34274467 -UNITED ST2 UNITED ST7 1992 33702779 -UNITED ST2 UNITED ST7 1993 64185014 -UNITED ST2 UNITED ST7 1994 22182758 -UNITED ST2 UNITED ST7 1995 60000208 -UNITED ST2 UNITED ST7 1996 80468477 -UNITED ST2 UNITED ST7 1997 76483802 -UNITED ST2 UNITED ST8 1992 36046449 -UNITED ST2 UNITED ST8 1993 33445446 -UNITED ST2 UNITED ST8 1994 33767774 -UNITED ST2 UNITED ST8 1995 30504189 -UNITED ST2 UNITED ST8 1996 46865113 -UNITED ST2 UNITED ST8 1997 46391909 -UNITED ST2 UNITED ST9 1992 83705618 -UNITED ST2 UNITED ST9 1993 48188507 -UNITED ST2 UNITED ST9 1994 69671054 -UNITED ST2 UNITED ST9 1995 65055178 -UNITED ST2 UNITED ST9 1996 53491415 -UNITED ST2 UNITED ST9 1997 69543984 -UNITED ST3 UNITED ST0 1992 62441656 -UNITED ST3 UNITED ST0 1993 66626926 -UNITED ST3 UNITED ST0 1994 65522727 -UNITED ST3 UNITED ST0 1995 84462733 -UNITED ST3 UNITED ST0 1996 60226840 -UNITED ST3 UNITED ST0 1997 54050172 -UNITED ST3 UNITED ST1 1992 73637490 -UNITED ST3 UNITED ST1 1993 78230278 -UNITED ST3 UNITED ST1 1994 85523653 -UNITED ST3 UNITED ST1 1995 64762815 -UNITED ST3 UNITED ST1 1996 74746699 -UNITED ST3 UNITED ST1 1997 54150683 -UNITED ST3 UNITED ST2 1992 1097208 -UNITED ST3 UNITED ST2 1993 21559073 -UNITED ST3 UNITED ST2 1994 9995234 -UNITED ST3 UNITED ST2 1995 12886829 -UNITED ST3 UNITED ST2 1996 4311800 -UNITED ST3 UNITED ST2 1997 7891878 -UNITED ST3 UNITED ST3 1992 45087216 -UNITED ST3 UNITED ST3 1993 21777144 -UNITED ST3 UNITED ST3 1994 45627153 -UNITED ST3 UNITED ST3 1995 48158089 -UNITED ST3 UNITED ST3 1996 49263796 -UNITED ST3 UNITED ST3 1997 48385233 -UNITED ST3 UNITED ST4 1992 63144772 -UNITED ST3 UNITED ST4 1993 92493920 -UNITED ST3 UNITED ST4 1994 63412365 -UNITED ST3 UNITED ST4 1995 30422346 -UNITED ST3 UNITED ST4 1996 54838259 -UNITED ST3 UNITED ST4 1997 58356141 -UNITED ST3 UNITED ST5 1992 64239048 -UNITED ST3 UNITED ST5 1993 55907305 -UNITED ST3 UNITED ST5 1994 26197875 -UNITED ST3 UNITED ST5 1995 40057815 -UNITED ST3 UNITED ST5 1996 33717439 -UNITED ST3 UNITED ST5 1997 32811963 -UNITED ST3 UNITED ST6 1992 38798715 -UNITED ST3 UNITED ST6 1993 58584808 -UNITED ST3 UNITED ST6 1994 36290257 -UNITED ST3 UNITED ST6 1995 16738530 -UNITED ST3 UNITED ST6 1996 41881628 -UNITED ST3 UNITED ST6 1997 36075844 -UNITED ST3 UNITED ST7 1992 39028579 -UNITED ST3 UNITED ST7 1993 50135448 -UNITED ST3 UNITED ST7 1994 40935131 -UNITED ST3 UNITED ST7 1995 51579548 -UNITED ST3 UNITED ST7 1996 53987218 -UNITED ST3 UNITED ST7 1997 61095300 -UNITED ST3 UNITED ST8 1992 41275764 -UNITED ST3 UNITED ST8 1993 42251848 -UNITED ST3 UNITED ST8 1994 15936051 -UNITED ST3 UNITED ST8 1995 22505239 -UNITED ST3 UNITED ST8 1996 33924732 -UNITED ST3 UNITED ST8 1997 29102220 -UNITED ST3 UNITED ST9 1992 86142048 -UNITED ST3 UNITED ST9 1993 54762596 -UNITED ST3 UNITED ST9 1994 97476185 -UNITED ST3 UNITED ST9 1995 37364623 -UNITED ST3 UNITED ST9 1996 49612304 -UNITED ST3 UNITED ST9 1997 51370308 -UNITED ST4 UNITED ST0 1992 94644242 -UNITED ST4 UNITED ST0 1993 86918458 -UNITED ST4 UNITED ST0 1994 73719340 -UNITED ST4 UNITED ST0 1995 46142899 -UNITED ST4 UNITED ST0 1996 106897563 -UNITED ST4 UNITED ST0 1997 69875804 -UNITED ST4 UNITED ST1 1992 102027455 -UNITED ST4 UNITED ST1 1993 88521423 -UNITED ST4 UNITED ST1 1994 72219557 -UNITED ST4 UNITED ST1 1995 90878653 -UNITED ST4 UNITED ST1 1996 72819867 -UNITED ST4 UNITED ST1 1997 84611188 -UNITED ST4 UNITED ST2 1992 12061845 -UNITED ST4 UNITED ST2 1993 27012678 -UNITED ST4 UNITED ST2 1994 23035892 -UNITED ST4 UNITED ST2 1995 22787003 -UNITED ST4 UNITED ST2 1996 12451867 -UNITED ST4 UNITED ST2 1997 2848157 -UNITED ST4 UNITED ST3 1992 69001787 -UNITED ST4 UNITED ST3 1993 33129412 -UNITED ST4 UNITED ST3 1994 31599433 -UNITED ST4 UNITED ST3 1995 37356396 -UNITED ST4 UNITED ST3 1996 46460940 -UNITED ST4 UNITED ST3 1997 59634153 -UNITED ST4 UNITED ST4 1992 70111335 -UNITED ST4 UNITED ST4 1993 98150465 -UNITED ST4 UNITED ST4 1994 76837381 -UNITED ST4 UNITED ST4 1995 74587071 -UNITED ST4 UNITED ST4 1996 32787995 -UNITED ST4 UNITED ST4 1997 62532346 -UNITED ST4 UNITED ST5 1992 65057872 -UNITED ST4 UNITED ST5 1993 29119594 -UNITED ST4 UNITED ST5 1994 59397028 -UNITED ST4 UNITED ST5 1995 62081505 -UNITED ST4 UNITED ST5 1996 42808412 -UNITED ST4 UNITED ST5 1997 31995868 -UNITED ST4 UNITED ST6 1992 81886374 -UNITED ST4 UNITED ST6 1993 30699255 -UNITED ST4 UNITED ST6 1994 58633482 -UNITED ST4 UNITED ST6 1995 38109524 -UNITED ST4 UNITED ST6 1996 62189328 -UNITED ST4 UNITED ST6 1997 46236849 -UNITED ST4 UNITED ST7 1992 62529376 -UNITED ST4 UNITED ST7 1993 88172198 -UNITED ST4 UNITED ST7 1994 76954140 -UNITED ST4 UNITED ST7 1995 69264407 -UNITED ST4 UNITED ST7 1996 83677814 -UNITED ST4 UNITED ST7 1997 84606996 -UNITED ST4 UNITED ST8 1992 21928442 -UNITED ST4 UNITED ST8 1993 41657582 -UNITED ST4 UNITED ST8 1994 20685352 -UNITED ST4 UNITED ST8 1995 23053236 -UNITED ST4 UNITED ST8 1996 53035662 -UNITED ST4 UNITED ST8 1997 32599736 -UNITED ST4 UNITED ST9 1992 67837970 -UNITED ST4 UNITED ST9 1993 73296959 -UNITED ST4 UNITED ST9 1994 83822393 -UNITED ST4 UNITED ST9 1995 99490068 -UNITED ST4 UNITED ST9 1996 54134840 -UNITED ST4 UNITED ST9 1997 79697927 -UNITED ST5 UNITED ST0 1992 90432300 -UNITED ST5 UNITED ST0 1993 106822597 -UNITED ST5 UNITED ST0 1994 78648057 -UNITED ST5 UNITED ST0 1995 77615746 -UNITED ST5 UNITED ST0 1996 104501979 -UNITED ST5 UNITED ST0 1997 121457766 -UNITED ST5 UNITED ST1 1992 59945287 -UNITED ST5 UNITED ST1 1993 71454623 -UNITED ST5 UNITED ST1 1994 73508418 -UNITED ST5 UNITED ST1 1995 119793669 -UNITED ST5 UNITED ST1 1996 65346389 -UNITED ST5 UNITED ST1 1997 106900161 -UNITED ST5 UNITED ST2 1992 20495817 -UNITED ST5 UNITED ST2 1993 20610242 -UNITED ST5 UNITED ST2 1994 7162796 -UNITED ST5 UNITED ST2 1995 14951023 -UNITED ST5 UNITED ST2 1996 8544233 -UNITED ST5 UNITED ST2 1997 13863438 -UNITED ST5 UNITED ST3 1992 53963553 -UNITED ST5 UNITED ST3 1993 34031811 -UNITED ST5 UNITED ST3 1994 65334911 -UNITED ST5 UNITED ST3 1995 78099130 -UNITED ST5 UNITED ST3 1996 32919926 -UNITED ST5 UNITED ST3 1997 56789932 -UNITED ST5 UNITED ST4 1992 57625627 -UNITED ST5 UNITED ST4 1993 63415639 -UNITED ST5 UNITED ST4 1994 51799593 -UNITED ST5 UNITED ST4 1995 81016761 -UNITED ST5 UNITED ST4 1996 33424859 -UNITED ST5 UNITED ST4 1997 103432927 -UNITED ST5 UNITED ST5 1992 29279568 -UNITED ST5 UNITED ST5 1993 50357189 -UNITED ST5 UNITED ST5 1994 56235213 -UNITED ST5 UNITED ST5 1995 50394007 -UNITED ST5 UNITED ST5 1996 43020722 -UNITED ST5 UNITED ST5 1997 32568888 -UNITED ST5 UNITED ST6 1992 62628093 -UNITED ST5 UNITED ST6 1993 25357679 -UNITED ST5 UNITED ST6 1994 38658451 -UNITED ST5 UNITED ST6 1995 30204737 -UNITED ST5 UNITED ST6 1996 38332044 -UNITED ST5 UNITED ST6 1997 49030016 -UNITED ST5 UNITED ST7 1992 68333651 -UNITED ST5 UNITED ST7 1993 52745538 -UNITED ST5 UNITED ST7 1994 89089961 -UNITED ST5 UNITED ST7 1995 53740759 -UNITED ST5 UNITED ST7 1996 40946330 -UNITED ST5 UNITED ST7 1997 54534315 -UNITED ST5 UNITED ST8 1992 33878053 -UNITED ST5 UNITED ST8 1993 74359279 -UNITED ST5 UNITED ST8 1994 18931891 -UNITED ST5 UNITED ST8 1995 13346932 -UNITED ST5 UNITED ST8 1996 32415682 -UNITED ST5 UNITED ST8 1997 32910946 -UNITED ST5 UNITED ST9 1992 50191091 -UNITED ST5 UNITED ST9 1993 76286539 -UNITED ST5 UNITED ST9 1994 53367331 -UNITED ST5 UNITED ST9 1995 62364897 -UNITED ST5 UNITED ST9 1996 72614114 -UNITED ST5 UNITED ST9 1997 97392640 -UNITED ST6 UNITED ST0 1992 78696956 -UNITED ST6 UNITED ST0 1993 68247539 -UNITED ST6 UNITED ST0 1994 53728853 -UNITED ST6 UNITED ST0 1995 91718471 -UNITED ST6 UNITED ST0 1996 132286763 -UNITED ST6 UNITED ST0 1997 55824919 -UNITED ST6 UNITED ST1 1992 69408688 -UNITED ST6 UNITED ST1 1993 83345939 -UNITED ST6 UNITED ST1 1994 96937147 -UNITED ST6 UNITED ST1 1995 38328352 -UNITED ST6 UNITED ST1 1996 89705870 -UNITED ST6 UNITED ST1 1997 55117709 -UNITED ST6 UNITED ST2 1992 11820698 -UNITED ST6 UNITED ST2 1993 18814367 -UNITED ST6 UNITED ST2 1994 25809000 -UNITED ST6 UNITED ST2 1995 11363551 -UNITED ST6 UNITED ST2 1996 20447847 -UNITED ST6 UNITED ST2 1997 14746680 -UNITED ST6 UNITED ST3 1992 78778218 -UNITED ST6 UNITED ST3 1993 83417161 -UNITED ST6 UNITED ST3 1994 56123238 -UNITED ST6 UNITED ST3 1995 67513062 -UNITED ST6 UNITED ST3 1996 82168229 -UNITED ST6 UNITED ST3 1997 69725506 -UNITED ST6 UNITED ST4 1992 71086944 -UNITED ST6 UNITED ST4 1993 69646913 -UNITED ST6 UNITED ST4 1994 59829989 -UNITED ST6 UNITED ST4 1995 47697241 -UNITED ST6 UNITED ST4 1996 69296005 -UNITED ST6 UNITED ST4 1997 57215022 -UNITED ST6 UNITED ST5 1992 85882966 -UNITED ST6 UNITED ST5 1993 34229082 -UNITED ST6 UNITED ST5 1994 66461534 -UNITED ST6 UNITED ST5 1995 45428152 -UNITED ST6 UNITED ST5 1996 46030104 -UNITED ST6 UNITED ST5 1997 37571788 -UNITED ST6 UNITED ST6 1992 18137105 -UNITED ST6 UNITED ST6 1993 33050663 -UNITED ST6 UNITED ST6 1994 46199722 -UNITED ST6 UNITED ST6 1995 50954677 -UNITED ST6 UNITED ST6 1996 42923032 -UNITED ST6 UNITED ST6 1997 24983291 -UNITED ST6 UNITED ST7 1992 87658267 -UNITED ST6 UNITED ST7 1993 78197294 -UNITED ST6 UNITED ST7 1994 37057808 -UNITED ST6 UNITED ST7 1995 23833977 -UNITED ST6 UNITED ST7 1996 57090619 -UNITED ST6 UNITED ST7 1997 49795849 -UNITED ST6 UNITED ST8 1992 42463690 -UNITED ST6 UNITED ST8 1993 36456741 -UNITED ST6 UNITED ST8 1994 26592996 -UNITED ST6 UNITED ST8 1995 41973920 -UNITED ST6 UNITED ST8 1996 52136784 -UNITED ST6 UNITED ST8 1997 37988445 -UNITED ST6 UNITED ST9 1992 40591721 -UNITED ST6 UNITED ST9 1993 54900942 -UNITED ST6 UNITED ST9 1994 94683309 -UNITED ST6 UNITED ST9 1995 63457105 -UNITED ST6 UNITED ST9 1996 63256182 -UNITED ST6 UNITED ST9 1997 58231263 -UNITED ST7 UNITED ST0 1992 96078010 -UNITED ST7 UNITED ST0 1993 109788024 -UNITED ST7 UNITED ST0 1994 82265536 -UNITED ST7 UNITED ST0 1995 98031295 -UNITED ST7 UNITED ST0 1996 75602445 -UNITED ST7 UNITED ST0 1997 69354312 -UNITED ST7 UNITED ST1 1992 50167208 -UNITED ST7 UNITED ST1 1993 34391159 -UNITED ST7 UNITED ST1 1994 75320799 -UNITED ST7 UNITED ST1 1995 79964362 -UNITED ST7 UNITED ST1 1996 48784175 -UNITED ST7 UNITED ST1 1997 101326940 -UNITED ST7 UNITED ST2 1992 4943204 -UNITED ST7 UNITED ST2 1993 20219304 -UNITED ST7 UNITED ST2 1994 19655081 -UNITED ST7 UNITED ST2 1995 3973565 -UNITED ST7 UNITED ST2 1996 8390470 -UNITED ST7 UNITED ST2 1997 31409590 -UNITED ST7 UNITED ST3 1992 83765666 -UNITED ST7 UNITED ST3 1993 39523861 -UNITED ST7 UNITED ST3 1994 29424840 -UNITED ST7 UNITED ST3 1995 43699734 -UNITED ST7 UNITED ST3 1996 52356886 -UNITED ST7 UNITED ST3 1997 35168922 -UNITED ST7 UNITED ST4 1992 64900888 -UNITED ST7 UNITED ST4 1993 61656613 -UNITED ST7 UNITED ST4 1994 64778715 -UNITED ST7 UNITED ST4 1995 49899545 -UNITED ST7 UNITED ST4 1996 77941011 -UNITED ST7 UNITED ST4 1997 60123494 -UNITED ST7 UNITED ST5 1992 57566802 -UNITED ST7 UNITED ST5 1993 44763692 -UNITED ST7 UNITED ST5 1994 45895364 -UNITED ST7 UNITED ST5 1995 39015293 -UNITED ST7 UNITED ST5 1996 32784224 -UNITED ST7 UNITED ST5 1997 36889736 -UNITED ST7 UNITED ST6 1992 65513205 -UNITED ST7 UNITED ST6 1993 27100502 -UNITED ST7 UNITED ST6 1994 24631937 -UNITED ST7 UNITED ST6 1995 65020671 -UNITED ST7 UNITED ST6 1996 36535628 -UNITED ST7 UNITED ST6 1997 27539931 -UNITED ST7 UNITED ST7 1992 60121386 -UNITED ST7 UNITED ST7 1993 68109820 -UNITED ST7 UNITED ST7 1994 46446442 -UNITED ST7 UNITED ST7 1995 86571624 -UNITED ST7 UNITED ST7 1996 74040644 -UNITED ST7 UNITED ST7 1997 42566603 -UNITED ST7 UNITED ST8 1992 11895702 -UNITED ST7 UNITED ST8 1993 43902628 -UNITED ST7 UNITED ST8 1994 42776467 -UNITED ST7 UNITED ST8 1995 26444490 -UNITED ST7 UNITED ST8 1996 24348851 -UNITED ST7 UNITED ST8 1997 45476266 -UNITED ST7 UNITED ST9 1992 60326042 -UNITED ST7 UNITED ST9 1993 80310701 -UNITED ST7 UNITED ST9 1994 50144085 -UNITED ST7 UNITED ST9 1995 75317709 -UNITED ST7 UNITED ST9 1996 25028091 -UNITED ST7 UNITED ST9 1997 55039970 -UNITED ST8 UNITED ST0 1992 44368839 -UNITED ST8 UNITED ST0 1993 88356371 -UNITED ST8 UNITED ST0 1994 99220789 -UNITED ST8 UNITED ST0 1995 55231664 -UNITED ST8 UNITED ST0 1996 39845427 -UNITED ST8 UNITED ST0 1997 89434845 -UNITED ST8 UNITED ST1 1992 35680752 -UNITED ST8 UNITED ST1 1993 44415434 -UNITED ST8 UNITED ST1 1994 40152096 -UNITED ST8 UNITED ST1 1995 34217780 -UNITED ST8 UNITED ST1 1996 78971756 -UNITED ST8 UNITED ST1 1997 43284139 -UNITED ST8 UNITED ST2 1992 10971561 -UNITED ST8 UNITED ST2 1993 9984366 -UNITED ST8 UNITED ST2 1994 3789626 -UNITED ST8 UNITED ST2 1995 18282990 -UNITED ST8 UNITED ST2 1996 7028334 -UNITED ST8 UNITED ST2 1997 12053706 -UNITED ST8 UNITED ST3 1992 30949324 -UNITED ST8 UNITED ST3 1993 54049874 -UNITED ST8 UNITED ST3 1994 55203841 -UNITED ST8 UNITED ST3 1995 29349938 -UNITED ST8 UNITED ST3 1996 39535940 -UNITED ST8 UNITED ST3 1997 66918394 -UNITED ST8 UNITED ST4 1992 46757019 -UNITED ST8 UNITED ST4 1993 51419356 -UNITED ST8 UNITED ST4 1994 62141681 -UNITED ST8 UNITED ST4 1995 19287311 -UNITED ST8 UNITED ST4 1996 51033054 -UNITED ST8 UNITED ST4 1997 36600584 -UNITED ST8 UNITED ST5 1992 35187610 -UNITED ST8 UNITED ST5 1993 52237837 -UNITED ST8 UNITED ST5 1994 44553056 -UNITED ST8 UNITED ST5 1995 44994863 -UNITED ST8 UNITED ST5 1996 43480473 -UNITED ST8 UNITED ST5 1997 26673743 -UNITED ST8 UNITED ST6 1992 60540440 -UNITED ST8 UNITED ST6 1993 54633062 -UNITED ST8 UNITED ST6 1994 36776569 -UNITED ST8 UNITED ST6 1995 58990404 -UNITED ST8 UNITED ST6 1996 59746655 -UNITED ST8 UNITED ST6 1997 20073000 -UNITED ST8 UNITED ST7 1992 41897901 -UNITED ST8 UNITED ST7 1993 36902218 -UNITED ST8 UNITED ST7 1994 37600687 -UNITED ST8 UNITED ST7 1995 30918202 -UNITED ST8 UNITED ST7 1996 57523925 -UNITED ST8 UNITED ST7 1997 40878429 -UNITED ST8 UNITED ST8 1992 29309108 -UNITED ST8 UNITED ST8 1993 38790235 -UNITED ST8 UNITED ST8 1994 10023412 -UNITED ST8 UNITED ST8 1995 11285867 -UNITED ST8 UNITED ST8 1996 16340603 -UNITED ST8 UNITED ST8 1997 40749400 -UNITED ST8 UNITED ST9 1992 22684471 -UNITED ST8 UNITED ST9 1993 38516361 -UNITED ST8 UNITED ST9 1994 58666906 -UNITED ST8 UNITED ST9 1995 88855089 -UNITED ST8 UNITED ST9 1996 60466691 -UNITED ST8 UNITED ST9 1997 45852796 -UNITED ST9 UNITED ST0 1992 94813294 -UNITED ST9 UNITED ST0 1993 32653305 -UNITED ST9 UNITED ST0 1994 87670573 -UNITED ST9 UNITED ST0 1995 51548621 -UNITED ST9 UNITED ST0 1996 49590701 -UNITED ST9 UNITED ST0 1997 66066791 -UNITED ST9 UNITED ST1 1992 80795302 -UNITED ST9 UNITED ST1 1993 74059419 -UNITED ST9 UNITED ST1 1994 90440948 -UNITED ST9 UNITED ST1 1995 82938967 -UNITED ST9 UNITED ST1 1996 101983827 -UNITED ST9 UNITED ST1 1997 62418406 -UNITED ST9 UNITED ST2 1992 11623177 -UNITED ST9 UNITED ST2 1993 17718390 -UNITED ST9 UNITED ST2 1994 9700989 -UNITED ST9 UNITED ST2 1995 12653362 -UNITED ST9 UNITED ST2 1996 8108035 -UNITED ST9 UNITED ST2 1997 2661652 -UNITED ST9 UNITED ST3 1992 47681165 -UNITED ST9 UNITED ST3 1993 46943647 -UNITED ST9 UNITED ST3 1994 35846500 -UNITED ST9 UNITED ST3 1995 17539102 -UNITED ST9 UNITED ST3 1996 47021018 -UNITED ST9 UNITED ST3 1997 46121724 -UNITED ST9 UNITED ST4 1992 61489513 -UNITED ST9 UNITED ST4 1993 30644687 -UNITED ST9 UNITED ST4 1994 43910761 -UNITED ST9 UNITED ST4 1995 72525686 -UNITED ST9 UNITED ST4 1996 69616614 -UNITED ST9 UNITED ST4 1997 31765478 -UNITED ST9 UNITED ST5 1992 28075004 -UNITED ST9 UNITED ST5 1993 39685269 -UNITED ST9 UNITED ST5 1994 37146726 -UNITED ST9 UNITED ST5 1995 36009924 -UNITED ST9 UNITED ST5 1996 52825922 -UNITED ST9 UNITED ST5 1997 56091920 -UNITED ST9 UNITED ST6 1992 15879568 -UNITED ST9 UNITED ST6 1993 43867160 -UNITED ST9 UNITED ST6 1994 29212927 -UNITED ST9 UNITED ST6 1995 43482494 -UNITED ST9 UNITED ST6 1996 48685063 -UNITED ST9 UNITED ST6 1997 37596882 -UNITED ST9 UNITED ST7 1992 56883359 -UNITED ST9 UNITED ST7 1993 66625483 -UNITED ST9 UNITED ST7 1994 31586173 -UNITED ST9 UNITED ST7 1995 46464589 -UNITED ST9 UNITED ST7 1996 48798759 -UNITED ST9 UNITED ST7 1997 69052630 -UNITED ST9 UNITED ST8 1992 21386100 -UNITED ST9 UNITED ST8 1993 37184696 -UNITED ST9 UNITED ST8 1994 39278808 -UNITED ST9 UNITED ST8 1995 21292527 -UNITED ST9 UNITED ST8 1996 36112301 -UNITED ST9 UNITED ST8 1997 31272385 -UNITED ST9 UNITED ST9 1992 36048343 -UNITED ST9 UNITED ST9 1993 63260763 -UNITED ST9 UNITED ST9 1994 51980849 -UNITED ST9 UNITED ST9 1995 76515958 -UNITED ST9 UNITED ST9 1996 77301740 -UNITED ST9 UNITED ST9 1997 106222795 +UNITED ST0 UNITED ST0 1992 18293852 +UNITED ST0 UNITED ST3 1992 3201624 +UNITED ST0 UNITED ST7 1992 7137641 +UNITED ST1 UNITED ST0 1992 10878084 +UNITED ST1 UNITED ST2 1992 8004700 +UNITED ST1 UNITED ST3 1992 13938002 +UNITED ST1 UNITED ST6 1992 11000283 +UNITED ST1 UNITED ST7 1992 17410800 +UNITED ST1 UNITED ST9 1992 25644597 +UNITED ST2 UNITED ST0 1992 23943154 +UNITED ST2 UNITED ST2 1992 8313714 +UNITED ST2 UNITED ST3 1992 15331073 +UNITED ST2 UNITED ST7 1992 7759164 +UNITED ST2 UNITED ST9 1992 12343455 +UNITED ST3 UNITED ST3 1992 4080199 +UNITED ST3 UNITED ST9 1992 2614811 +UNITED ST4 UNITED ST0 1992 34626982 +UNITED ST4 UNITED ST2 1992 11696334 +UNITED ST4 UNITED ST3 1992 29767238 +UNITED ST4 UNITED ST6 1992 11369008 +UNITED ST4 UNITED ST7 1992 10151573 +UNITED ST4 UNITED ST9 1992 21189183 +UNITED ST5 UNITED ST0 1992 12301234 +UNITED ST5 UNITED ST2 1992 9917834 +UNITED ST5 UNITED ST3 1992 5862031 +UNITED ST5 UNITED ST6 1992 12398029 +UNITED ST5 UNITED ST7 1992 3936271 +UNITED ST5 UNITED ST9 1992 14448179 +UNITED ST6 UNITED ST0 1992 11900889 +UNITED ST6 UNITED ST2 1992 291566 +UNITED ST6 UNITED ST3 1992 1873819 +UNITED ST6 UNITED ST6 1992 8685228 +UNITED ST6 UNITED ST9 1992 6597261 +UNITED ST7 UNITED ST0 1992 1882015 +UNITED ST7 UNITED ST2 1992 6125476 +UNITED ST7 UNITED ST3 1992 17996772 +UNITED ST7 UNITED ST6 1992 6058017 +UNITED ST7 UNITED ST7 1992 9715656 +UNITED ST7 UNITED ST9 1992 5403152 +UNITED ST8 UNITED ST0 1992 3574169 +UNITED ST8 UNITED ST7 1992 2373825 +UNITED ST8 UNITED ST9 1992 5690491 +UNITED ST9 UNITED ST0 1992 4816370 +UNITED ST9 UNITED ST2 1992 12370917 +UNITED ST9 UNITED ST3 1992 17863433 +UNITED ST9 UNITED ST6 1992 2066609 +UNITED ST9 UNITED ST7 1992 6703890 +UNITED ST9 UNITED ST9 1992 4234523 -- !query3_2_after -- -UNITED ST0 UNITED ST0 1992 75622122 -UNITED ST0 UNITED ST0 1993 106742548 -UNITED ST0 UNITED ST0 1994 83526877 -UNITED ST0 UNITED ST0 1995 38984502 -UNITED ST0 UNITED ST0 1996 61528855 -UNITED ST0 UNITED ST0 1997 66308516 -UNITED ST0 UNITED ST1 1992 77267396 -UNITED ST0 UNITED ST1 1993 91196593 -UNITED ST0 UNITED ST1 1994 76613379 -UNITED ST0 UNITED ST1 1995 90763599 -UNITED ST0 UNITED ST1 1996 91948777 -UNITED ST0 UNITED ST1 1997 80167139 -UNITED ST0 UNITED ST2 1992 27718733 -UNITED ST0 UNITED ST2 1993 15042145 -UNITED ST0 UNITED ST2 1994 22017954 -UNITED ST0 UNITED ST2 1995 4776460 -UNITED ST0 UNITED ST2 1996 6907199 -UNITED ST0 UNITED ST2 1997 23584229 -UNITED ST0 UNITED ST3 1992 56359317 -UNITED ST0 UNITED ST3 1993 76171283 -UNITED ST0 UNITED ST3 1994 27327829 -UNITED ST0 UNITED ST3 1995 56360708 -UNITED ST0 UNITED ST3 1996 29795665 -UNITED ST0 UNITED ST3 1997 40078508 -UNITED ST0 UNITED ST4 1992 49676807 -UNITED ST0 UNITED ST4 1993 49260067 -UNITED ST0 UNITED ST4 1994 61519744 -UNITED ST0 UNITED ST4 1995 29170038 -UNITED ST0 UNITED ST4 1996 103228749 -UNITED ST0 UNITED ST4 1997 41193803 -UNITED ST0 UNITED ST5 1992 40120886 -UNITED ST0 UNITED ST5 1993 52701052 -UNITED ST0 UNITED ST5 1994 41808301 -UNITED ST0 UNITED ST5 1995 11453709 -UNITED ST0 UNITED ST5 1996 57762895 -UNITED ST0 UNITED ST5 1997 47988418 -UNITED ST0 UNITED ST6 1992 42471806 -UNITED ST0 UNITED ST6 1993 50432282 -UNITED ST0 UNITED ST6 1994 61288778 -UNITED ST0 UNITED ST6 1995 40718942 -UNITED ST0 UNITED ST6 1996 37232062 -UNITED ST0 UNITED ST6 1997 22924046 -UNITED ST0 UNITED ST7 1992 50583409 -UNITED ST0 UNITED ST7 1993 48796360 -UNITED ST0 UNITED ST7 1994 61480820 -UNITED ST0 UNITED ST7 1995 33395313 -UNITED ST0 UNITED ST7 1996 31391181 -UNITED ST0 UNITED ST7 1997 72577490 -UNITED ST0 UNITED ST8 1992 49732640 -UNITED ST0 UNITED ST8 1993 42002682 -UNITED ST0 UNITED ST8 1994 31845433 -UNITED ST0 UNITED ST8 1995 16393249 -UNITED ST0 UNITED ST8 1996 15168262 -UNITED ST0 UNITED ST8 1997 30241370 -UNITED ST0 UNITED ST9 1992 63979996 -UNITED ST0 UNITED ST9 1993 109834040 -UNITED ST0 UNITED ST9 1994 89865202 -UNITED ST0 UNITED ST9 1995 63253499 -UNITED ST0 UNITED ST9 1996 75935209 -UNITED ST0 UNITED ST9 1997 119856268 -UNITED ST1 UNITED ST0 1992 96989491 -UNITED ST1 UNITED ST0 1993 99305952 -UNITED ST1 UNITED ST0 1994 43225201 -UNITED ST1 UNITED ST0 1995 81676623 -UNITED ST1 UNITED ST0 1996 94861554 -UNITED ST1 UNITED ST0 1997 77667866 -UNITED ST1 UNITED ST1 1992 64669070 -UNITED ST1 UNITED ST1 1993 47538610 -UNITED ST1 UNITED ST1 1994 54383174 -UNITED ST1 UNITED ST1 1995 64199332 -UNITED ST1 UNITED ST1 1996 87265848 -UNITED ST1 UNITED ST1 1997 76340444 -UNITED ST1 UNITED ST2 1992 7706728 -UNITED ST1 UNITED ST2 1993 16844538 -UNITED ST1 UNITED ST2 1994 23556586 -UNITED ST1 UNITED ST2 1995 7991676 -UNITED ST1 UNITED ST2 1996 10082536 -UNITED ST1 UNITED ST2 1997 20599130 -UNITED ST1 UNITED ST3 1992 55727474 -UNITED ST1 UNITED ST3 1993 52097946 -UNITED ST1 UNITED ST3 1994 75992459 -UNITED ST1 UNITED ST3 1995 45823056 -UNITED ST1 UNITED ST3 1996 77610905 -UNITED ST1 UNITED ST3 1997 85175638 -UNITED ST1 UNITED ST4 1992 39326346 -UNITED ST1 UNITED ST4 1993 62884347 -UNITED ST1 UNITED ST4 1994 41381558 -UNITED ST1 UNITED ST4 1995 57535599 -UNITED ST1 UNITED ST4 1996 50135547 -UNITED ST1 UNITED ST4 1997 83092569 -UNITED ST1 UNITED ST5 1992 33971251 -UNITED ST1 UNITED ST5 1993 40693513 -UNITED ST1 UNITED ST5 1994 24939138 -UNITED ST1 UNITED ST5 1995 31857452 -UNITED ST1 UNITED ST5 1996 51189649 -UNITED ST1 UNITED ST5 1997 51052110 -UNITED ST1 UNITED ST6 1992 35404990 -UNITED ST1 UNITED ST6 1993 44728540 -UNITED ST1 UNITED ST6 1994 31287326 -UNITED ST1 UNITED ST6 1995 27139403 -UNITED ST1 UNITED ST6 1996 54056366 -UNITED ST1 UNITED ST6 1997 18205326 -UNITED ST1 UNITED ST7 1992 71472973 -UNITED ST1 UNITED ST7 1993 54851183 -UNITED ST1 UNITED ST7 1994 53853106 -UNITED ST1 UNITED ST7 1995 61515741 -UNITED ST1 UNITED ST7 1996 41721506 -UNITED ST1 UNITED ST7 1997 77045722 -UNITED ST1 UNITED ST8 1992 28565579 -UNITED ST1 UNITED ST8 1993 66146198 -UNITED ST1 UNITED ST8 1994 30245153 -UNITED ST1 UNITED ST8 1995 34918793 -UNITED ST1 UNITED ST8 1996 39430882 -UNITED ST1 UNITED ST8 1997 15375258 -UNITED ST1 UNITED ST9 1992 75367059 -UNITED ST1 UNITED ST9 1993 52514902 -UNITED ST1 UNITED ST9 1994 43000037 -UNITED ST1 UNITED ST9 1995 72470870 -UNITED ST1 UNITED ST9 1996 69823769 -UNITED ST1 UNITED ST9 1997 48944111 -UNITED ST2 UNITED ST0 1992 72210218 -UNITED ST2 UNITED ST0 1993 42845485 -UNITED ST2 UNITED ST0 1994 60182153 -UNITED ST2 UNITED ST0 1995 52560838 -UNITED ST2 UNITED ST0 1996 67095122 -UNITED ST2 UNITED ST0 1997 77591470 -UNITED ST2 UNITED ST1 1992 81839165 -UNITED ST2 UNITED ST1 1993 69631701 -UNITED ST2 UNITED ST1 1994 81300089 -UNITED ST2 UNITED ST1 1995 45437340 -UNITED ST2 UNITED ST1 1996 88256155 -UNITED ST2 UNITED ST1 1997 65396135 -UNITED ST2 UNITED ST2 1992 25743654 -UNITED ST2 UNITED ST2 1993 14521995 -UNITED ST2 UNITED ST2 1994 18068657 -UNITED ST2 UNITED ST2 1995 13455031 -UNITED ST2 UNITED ST2 1997 37034333 -UNITED ST2 UNITED ST3 1992 35981963 -UNITED ST2 UNITED ST3 1993 66972438 -UNITED ST2 UNITED ST3 1994 65620760 -UNITED ST2 UNITED ST3 1995 65218129 -UNITED ST2 UNITED ST3 1996 59648598 -UNITED ST2 UNITED ST3 1997 65429453 -UNITED ST2 UNITED ST4 1992 91180500 -UNITED ST2 UNITED ST4 1993 56650391 -UNITED ST2 UNITED ST4 1994 55679377 -UNITED ST2 UNITED ST4 1995 82432305 -UNITED ST2 UNITED ST4 1996 57487057 -UNITED ST2 UNITED ST4 1997 57012611 -UNITED ST2 UNITED ST5 1992 57021365 -UNITED ST2 UNITED ST5 1993 37406845 -UNITED ST2 UNITED ST5 1994 37640317 -UNITED ST2 UNITED ST5 1995 43382417 -UNITED ST2 UNITED ST5 1996 47028976 -UNITED ST2 UNITED ST5 1997 40662886 -UNITED ST2 UNITED ST6 1992 27813739 -UNITED ST2 UNITED ST6 1993 26482226 -UNITED ST2 UNITED ST6 1994 41526105 -UNITED ST2 UNITED ST6 1995 37146288 -UNITED ST2 UNITED ST6 1996 45416649 -UNITED ST2 UNITED ST6 1997 34274467 -UNITED ST2 UNITED ST7 1992 33702779 -UNITED ST2 UNITED ST7 1993 64185014 -UNITED ST2 UNITED ST7 1994 22182758 -UNITED ST2 UNITED ST7 1995 60000208 -UNITED ST2 UNITED ST7 1996 80468477 -UNITED ST2 UNITED ST7 1997 76483802 -UNITED ST2 UNITED ST8 1992 36046449 -UNITED ST2 UNITED ST8 1993 33445446 -UNITED ST2 UNITED ST8 1994 33767774 -UNITED ST2 UNITED ST8 1995 30504189 -UNITED ST2 UNITED ST8 1996 46865113 -UNITED ST2 UNITED ST8 1997 46391909 -UNITED ST2 UNITED ST9 1992 83705618 -UNITED ST2 UNITED ST9 1993 48188507 -UNITED ST2 UNITED ST9 1994 69671054 -UNITED ST2 UNITED ST9 1995 65055178 -UNITED ST2 UNITED ST9 1996 53491415 -UNITED ST2 UNITED ST9 1997 69543984 -UNITED ST3 UNITED ST0 1992 62441656 -UNITED ST3 UNITED ST0 1993 66626926 -UNITED ST3 UNITED ST0 1994 65522727 -UNITED ST3 UNITED ST0 1995 84462733 -UNITED ST3 UNITED ST0 1996 60226840 -UNITED ST3 UNITED ST0 1997 54050172 -UNITED ST3 UNITED ST1 1992 73637490 -UNITED ST3 UNITED ST1 1993 78230278 -UNITED ST3 UNITED ST1 1994 85523653 -UNITED ST3 UNITED ST1 1995 64762815 -UNITED ST3 UNITED ST1 1996 74746699 -UNITED ST3 UNITED ST1 1997 54150683 -UNITED ST3 UNITED ST2 1992 1097208 -UNITED ST3 UNITED ST2 1993 21559073 -UNITED ST3 UNITED ST2 1994 9995234 -UNITED ST3 UNITED ST2 1995 12886829 -UNITED ST3 UNITED ST2 1996 4311800 -UNITED ST3 UNITED ST2 1997 7891878 -UNITED ST3 UNITED ST3 1992 45087216 -UNITED ST3 UNITED ST3 1993 21777144 -UNITED ST3 UNITED ST3 1994 45627153 -UNITED ST3 UNITED ST3 1995 48158089 -UNITED ST3 UNITED ST3 1996 49263796 -UNITED ST3 UNITED ST3 1997 48385233 -UNITED ST3 UNITED ST4 1992 63144772 -UNITED ST3 UNITED ST4 1993 92493920 -UNITED ST3 UNITED ST4 1994 63412365 -UNITED ST3 UNITED ST4 1995 30422346 -UNITED ST3 UNITED ST4 1996 54838259 -UNITED ST3 UNITED ST4 1997 58356141 -UNITED ST3 UNITED ST5 1992 64239048 -UNITED ST3 UNITED ST5 1993 55907305 -UNITED ST3 UNITED ST5 1994 26197875 -UNITED ST3 UNITED ST5 1995 40057815 -UNITED ST3 UNITED ST5 1996 33717439 -UNITED ST3 UNITED ST5 1997 32811963 -UNITED ST3 UNITED ST6 1992 38798715 -UNITED ST3 UNITED ST6 1993 58584808 -UNITED ST3 UNITED ST6 1994 36290257 -UNITED ST3 UNITED ST6 1995 16738530 -UNITED ST3 UNITED ST6 1996 41881628 -UNITED ST3 UNITED ST6 1997 36075844 -UNITED ST3 UNITED ST7 1992 39028579 -UNITED ST3 UNITED ST7 1993 50135448 -UNITED ST3 UNITED ST7 1994 40935131 -UNITED ST3 UNITED ST7 1995 51579548 -UNITED ST3 UNITED ST7 1996 53987218 -UNITED ST3 UNITED ST7 1997 61095300 -UNITED ST3 UNITED ST8 1992 41275764 -UNITED ST3 UNITED ST8 1993 42251848 -UNITED ST3 UNITED ST8 1994 15936051 -UNITED ST3 UNITED ST8 1995 22505239 -UNITED ST3 UNITED ST8 1996 33924732 -UNITED ST3 UNITED ST8 1997 29102220 -UNITED ST3 UNITED ST9 1992 86142048 -UNITED ST3 UNITED ST9 1993 54762596 -UNITED ST3 UNITED ST9 1994 97476185 -UNITED ST3 UNITED ST9 1995 37364623 -UNITED ST3 UNITED ST9 1996 49612304 -UNITED ST3 UNITED ST9 1997 51370308 -UNITED ST4 UNITED ST0 1992 94644242 -UNITED ST4 UNITED ST0 1993 86918458 -UNITED ST4 UNITED ST0 1994 73719340 -UNITED ST4 UNITED ST0 1995 46142899 -UNITED ST4 UNITED ST0 1996 106897563 -UNITED ST4 UNITED ST0 1997 69875804 -UNITED ST4 UNITED ST1 1992 102027455 -UNITED ST4 UNITED ST1 1993 88521423 -UNITED ST4 UNITED ST1 1994 72219557 -UNITED ST4 UNITED ST1 1995 90878653 -UNITED ST4 UNITED ST1 1996 72819867 -UNITED ST4 UNITED ST1 1997 84611188 -UNITED ST4 UNITED ST2 1992 12061845 -UNITED ST4 UNITED ST2 1993 27012678 -UNITED ST4 UNITED ST2 1994 23035892 -UNITED ST4 UNITED ST2 1995 22787003 -UNITED ST4 UNITED ST2 1996 12451867 -UNITED ST4 UNITED ST2 1997 2848157 -UNITED ST4 UNITED ST3 1992 69001787 -UNITED ST4 UNITED ST3 1993 33129412 -UNITED ST4 UNITED ST3 1994 31599433 -UNITED ST4 UNITED ST3 1995 37356396 -UNITED ST4 UNITED ST3 1996 46460940 -UNITED ST4 UNITED ST3 1997 59634153 -UNITED ST4 UNITED ST4 1992 70111335 -UNITED ST4 UNITED ST4 1993 98150465 -UNITED ST4 UNITED ST4 1994 76837381 -UNITED ST4 UNITED ST4 1995 74587071 -UNITED ST4 UNITED ST4 1996 32787995 -UNITED ST4 UNITED ST4 1997 62532346 -UNITED ST4 UNITED ST5 1992 65057872 -UNITED ST4 UNITED ST5 1993 29119594 -UNITED ST4 UNITED ST5 1994 59397028 -UNITED ST4 UNITED ST5 1995 62081505 -UNITED ST4 UNITED ST5 1996 42808412 -UNITED ST4 UNITED ST5 1997 31995868 -UNITED ST4 UNITED ST6 1992 81886374 -UNITED ST4 UNITED ST6 1993 30699255 -UNITED ST4 UNITED ST6 1994 58633482 -UNITED ST4 UNITED ST6 1995 38109524 -UNITED ST4 UNITED ST6 1996 62189328 -UNITED ST4 UNITED ST6 1997 46236849 -UNITED ST4 UNITED ST7 1992 62529376 -UNITED ST4 UNITED ST7 1993 88172198 -UNITED ST4 UNITED ST7 1994 76954140 -UNITED ST4 UNITED ST7 1995 69264407 -UNITED ST4 UNITED ST7 1996 83677814 -UNITED ST4 UNITED ST7 1997 84606996 -UNITED ST4 UNITED ST8 1992 21928442 -UNITED ST4 UNITED ST8 1993 41657582 -UNITED ST4 UNITED ST8 1994 20685352 -UNITED ST4 UNITED ST8 1995 23053236 -UNITED ST4 UNITED ST8 1996 53035662 -UNITED ST4 UNITED ST8 1997 32599736 -UNITED ST4 UNITED ST9 1992 67837970 -UNITED ST4 UNITED ST9 1993 73296959 -UNITED ST4 UNITED ST9 1994 83822393 -UNITED ST4 UNITED ST9 1995 99490068 -UNITED ST4 UNITED ST9 1996 54134840 -UNITED ST4 UNITED ST9 1997 79697927 -UNITED ST5 UNITED ST0 1992 90432300 -UNITED ST5 UNITED ST0 1993 106822597 -UNITED ST5 UNITED ST0 1994 78648057 -UNITED ST5 UNITED ST0 1995 77615746 -UNITED ST5 UNITED ST0 1996 104501979 -UNITED ST5 UNITED ST0 1997 121457766 -UNITED ST5 UNITED ST1 1992 59945287 -UNITED ST5 UNITED ST1 1993 71454623 -UNITED ST5 UNITED ST1 1994 73508418 -UNITED ST5 UNITED ST1 1995 119793669 -UNITED ST5 UNITED ST1 1996 65346389 -UNITED ST5 UNITED ST1 1997 106900161 -UNITED ST5 UNITED ST2 1992 20495817 -UNITED ST5 UNITED ST2 1993 20610242 -UNITED ST5 UNITED ST2 1994 7162796 -UNITED ST5 UNITED ST2 1995 14951023 -UNITED ST5 UNITED ST2 1996 8544233 -UNITED ST5 UNITED ST2 1997 13863438 -UNITED ST5 UNITED ST3 1992 53963553 -UNITED ST5 UNITED ST3 1993 34031811 -UNITED ST5 UNITED ST3 1994 65334911 -UNITED ST5 UNITED ST3 1995 78099130 -UNITED ST5 UNITED ST3 1996 32919926 -UNITED ST5 UNITED ST3 1997 56789932 -UNITED ST5 UNITED ST4 1992 57625627 -UNITED ST5 UNITED ST4 1993 63415639 -UNITED ST5 UNITED ST4 1994 51799593 -UNITED ST5 UNITED ST4 1995 81016761 -UNITED ST5 UNITED ST4 1996 33424859 -UNITED ST5 UNITED ST4 1997 103432927 -UNITED ST5 UNITED ST5 1992 29279568 -UNITED ST5 UNITED ST5 1993 50357189 -UNITED ST5 UNITED ST5 1994 56235213 -UNITED ST5 UNITED ST5 1995 50394007 -UNITED ST5 UNITED ST5 1996 43020722 -UNITED ST5 UNITED ST5 1997 32568888 -UNITED ST5 UNITED ST6 1992 62628093 -UNITED ST5 UNITED ST6 1993 25357679 -UNITED ST5 UNITED ST6 1994 38658451 -UNITED ST5 UNITED ST6 1995 30204737 -UNITED ST5 UNITED ST6 1996 38332044 -UNITED ST5 UNITED ST6 1997 49030016 -UNITED ST5 UNITED ST7 1992 68333651 -UNITED ST5 UNITED ST7 1993 52745538 -UNITED ST5 UNITED ST7 1994 89089961 -UNITED ST5 UNITED ST7 1995 53740759 -UNITED ST5 UNITED ST7 1996 40946330 -UNITED ST5 UNITED ST7 1997 54534315 -UNITED ST5 UNITED ST8 1992 33878053 -UNITED ST5 UNITED ST8 1993 74359279 -UNITED ST5 UNITED ST8 1994 18931891 -UNITED ST5 UNITED ST8 1995 13346932 -UNITED ST5 UNITED ST8 1996 32415682 -UNITED ST5 UNITED ST8 1997 32910946 -UNITED ST5 UNITED ST9 1992 50191091 -UNITED ST5 UNITED ST9 1993 76286539 -UNITED ST5 UNITED ST9 1994 53367331 -UNITED ST5 UNITED ST9 1995 62364897 -UNITED ST5 UNITED ST9 1996 72614114 -UNITED ST5 UNITED ST9 1997 97392640 -UNITED ST6 UNITED ST0 1992 78696956 -UNITED ST6 UNITED ST0 1993 68247539 -UNITED ST6 UNITED ST0 1994 53728853 -UNITED ST6 UNITED ST0 1995 91718471 -UNITED ST6 UNITED ST0 1996 132286763 -UNITED ST6 UNITED ST0 1997 55824919 -UNITED ST6 UNITED ST1 1992 69408688 -UNITED ST6 UNITED ST1 1993 83345939 -UNITED ST6 UNITED ST1 1994 96937147 -UNITED ST6 UNITED ST1 1995 38328352 -UNITED ST6 UNITED ST1 1996 89705870 -UNITED ST6 UNITED ST1 1997 55117709 -UNITED ST6 UNITED ST2 1992 11820698 -UNITED ST6 UNITED ST2 1993 18814367 -UNITED ST6 UNITED ST2 1994 25809000 -UNITED ST6 UNITED ST2 1995 11363551 -UNITED ST6 UNITED ST2 1996 20447847 -UNITED ST6 UNITED ST2 1997 14746680 -UNITED ST6 UNITED ST3 1992 78778218 -UNITED ST6 UNITED ST3 1993 83417161 -UNITED ST6 UNITED ST3 1994 56123238 -UNITED ST6 UNITED ST3 1995 67513062 -UNITED ST6 UNITED ST3 1996 82168229 -UNITED ST6 UNITED ST3 1997 69725506 -UNITED ST6 UNITED ST4 1992 71086944 -UNITED ST6 UNITED ST4 1993 69646913 -UNITED ST6 UNITED ST4 1994 59829989 -UNITED ST6 UNITED ST4 1995 47697241 -UNITED ST6 UNITED ST4 1996 69296005 -UNITED ST6 UNITED ST4 1997 57215022 -UNITED ST6 UNITED ST5 1992 85882966 -UNITED ST6 UNITED ST5 1993 34229082 -UNITED ST6 UNITED ST5 1994 66461534 -UNITED ST6 UNITED ST5 1995 45428152 -UNITED ST6 UNITED ST5 1996 46030104 -UNITED ST6 UNITED ST5 1997 37571788 -UNITED ST6 UNITED ST6 1992 18137105 -UNITED ST6 UNITED ST6 1993 33050663 -UNITED ST6 UNITED ST6 1994 46199722 -UNITED ST6 UNITED ST6 1995 50954677 -UNITED ST6 UNITED ST6 1996 42923032 -UNITED ST6 UNITED ST6 1997 24983291 -UNITED ST6 UNITED ST7 1992 87658267 -UNITED ST6 UNITED ST7 1993 78197294 -UNITED ST6 UNITED ST7 1994 37057808 -UNITED ST6 UNITED ST7 1995 23833977 -UNITED ST6 UNITED ST7 1996 57090619 -UNITED ST6 UNITED ST7 1997 49795849 -UNITED ST6 UNITED ST8 1992 42463690 -UNITED ST6 UNITED ST8 1993 36456741 -UNITED ST6 UNITED ST8 1994 26592996 -UNITED ST6 UNITED ST8 1995 41973920 -UNITED ST6 UNITED ST8 1996 52136784 -UNITED ST6 UNITED ST8 1997 37988445 -UNITED ST6 UNITED ST9 1992 40591721 -UNITED ST6 UNITED ST9 1993 54900942 -UNITED ST6 UNITED ST9 1994 94683309 -UNITED ST6 UNITED ST9 1995 63457105 -UNITED ST6 UNITED ST9 1996 63256182 -UNITED ST6 UNITED ST9 1997 58231263 -UNITED ST7 UNITED ST0 1992 96078010 -UNITED ST7 UNITED ST0 1993 109788024 -UNITED ST7 UNITED ST0 1994 82265536 -UNITED ST7 UNITED ST0 1995 98031295 -UNITED ST7 UNITED ST0 1996 75602445 -UNITED ST7 UNITED ST0 1997 69354312 -UNITED ST7 UNITED ST1 1992 50167208 -UNITED ST7 UNITED ST1 1993 34391159 -UNITED ST7 UNITED ST1 1994 75320799 -UNITED ST7 UNITED ST1 1995 79964362 -UNITED ST7 UNITED ST1 1996 48784175 -UNITED ST7 UNITED ST1 1997 101326940 -UNITED ST7 UNITED ST2 1992 4943204 -UNITED ST7 UNITED ST2 1993 20219304 -UNITED ST7 UNITED ST2 1994 19655081 -UNITED ST7 UNITED ST2 1995 3973565 -UNITED ST7 UNITED ST2 1996 8390470 -UNITED ST7 UNITED ST2 1997 31409590 -UNITED ST7 UNITED ST3 1992 83765666 -UNITED ST7 UNITED ST3 1993 39523861 -UNITED ST7 UNITED ST3 1994 29424840 -UNITED ST7 UNITED ST3 1995 43699734 -UNITED ST7 UNITED ST3 1996 52356886 -UNITED ST7 UNITED ST3 1997 35168922 -UNITED ST7 UNITED ST4 1992 64900888 -UNITED ST7 UNITED ST4 1993 61656613 -UNITED ST7 UNITED ST4 1994 64778715 -UNITED ST7 UNITED ST4 1995 49899545 -UNITED ST7 UNITED ST4 1996 77941011 -UNITED ST7 UNITED ST4 1997 60123494 -UNITED ST7 UNITED ST5 1992 57566802 -UNITED ST7 UNITED ST5 1993 44763692 -UNITED ST7 UNITED ST5 1994 45895364 -UNITED ST7 UNITED ST5 1995 39015293 -UNITED ST7 UNITED ST5 1996 32784224 -UNITED ST7 UNITED ST5 1997 36889736 -UNITED ST7 UNITED ST6 1992 65513205 -UNITED ST7 UNITED ST6 1993 27100502 -UNITED ST7 UNITED ST6 1994 24631937 -UNITED ST7 UNITED ST6 1995 65020671 -UNITED ST7 UNITED ST6 1996 36535628 -UNITED ST7 UNITED ST6 1997 27539931 -UNITED ST7 UNITED ST7 1992 60121386 -UNITED ST7 UNITED ST7 1993 68109820 -UNITED ST7 UNITED ST7 1994 46446442 -UNITED ST7 UNITED ST7 1995 86571624 -UNITED ST7 UNITED ST7 1996 74040644 -UNITED ST7 UNITED ST7 1997 42566603 -UNITED ST7 UNITED ST8 1992 11895702 -UNITED ST7 UNITED ST8 1993 43902628 -UNITED ST7 UNITED ST8 1994 42776467 -UNITED ST7 UNITED ST8 1995 26444490 -UNITED ST7 UNITED ST8 1996 24348851 -UNITED ST7 UNITED ST8 1997 45476266 -UNITED ST7 UNITED ST9 1992 60326042 -UNITED ST7 UNITED ST9 1993 80310701 -UNITED ST7 UNITED ST9 1994 50144085 -UNITED ST7 UNITED ST9 1995 75317709 -UNITED ST7 UNITED ST9 1996 25028091 -UNITED ST7 UNITED ST9 1997 55039970 -UNITED ST8 UNITED ST0 1992 44368839 -UNITED ST8 UNITED ST0 1993 88356371 -UNITED ST8 UNITED ST0 1994 99220789 -UNITED ST8 UNITED ST0 1995 55231664 -UNITED ST8 UNITED ST0 1996 39845427 -UNITED ST8 UNITED ST0 1997 89434845 -UNITED ST8 UNITED ST1 1992 35680752 -UNITED ST8 UNITED ST1 1993 44415434 -UNITED ST8 UNITED ST1 1994 40152096 -UNITED ST8 UNITED ST1 1995 34217780 -UNITED ST8 UNITED ST1 1996 78971756 -UNITED ST8 UNITED ST1 1997 43284139 -UNITED ST8 UNITED ST2 1992 10971561 -UNITED ST8 UNITED ST2 1993 9984366 -UNITED ST8 UNITED ST2 1994 3789626 -UNITED ST8 UNITED ST2 1995 18282990 -UNITED ST8 UNITED ST2 1996 7028334 -UNITED ST8 UNITED ST2 1997 12053706 -UNITED ST8 UNITED ST3 1992 30949324 -UNITED ST8 UNITED ST3 1993 54049874 -UNITED ST8 UNITED ST3 1994 55203841 -UNITED ST8 UNITED ST3 1995 29349938 -UNITED ST8 UNITED ST3 1996 39535940 -UNITED ST8 UNITED ST3 1997 66918394 -UNITED ST8 UNITED ST4 1992 46757019 -UNITED ST8 UNITED ST4 1993 51419356 -UNITED ST8 UNITED ST4 1994 62141681 -UNITED ST8 UNITED ST4 1995 19287311 -UNITED ST8 UNITED ST4 1996 51033054 -UNITED ST8 UNITED ST4 1997 36600584 -UNITED ST8 UNITED ST5 1992 35187610 -UNITED ST8 UNITED ST5 1993 52237837 -UNITED ST8 UNITED ST5 1994 44553056 -UNITED ST8 UNITED ST5 1995 44994863 -UNITED ST8 UNITED ST5 1996 43480473 -UNITED ST8 UNITED ST5 1997 26673743 -UNITED ST8 UNITED ST6 1992 60540440 -UNITED ST8 UNITED ST6 1993 54633062 -UNITED ST8 UNITED ST6 1994 36776569 -UNITED ST8 UNITED ST6 1995 58990404 -UNITED ST8 UNITED ST6 1996 59746655 -UNITED ST8 UNITED ST6 1997 20073000 -UNITED ST8 UNITED ST7 1992 41897901 -UNITED ST8 UNITED ST7 1993 36902218 -UNITED ST8 UNITED ST7 1994 37600687 -UNITED ST8 UNITED ST7 1995 30918202 -UNITED ST8 UNITED ST7 1996 57523925 -UNITED ST8 UNITED ST7 1997 40878429 -UNITED ST8 UNITED ST8 1992 29309108 -UNITED ST8 UNITED ST8 1993 38790235 -UNITED ST8 UNITED ST8 1994 10023412 -UNITED ST8 UNITED ST8 1995 11285867 -UNITED ST8 UNITED ST8 1996 16340603 -UNITED ST8 UNITED ST8 1997 40749400 -UNITED ST8 UNITED ST9 1992 22684471 -UNITED ST8 UNITED ST9 1993 38516361 -UNITED ST8 UNITED ST9 1994 58666906 -UNITED ST8 UNITED ST9 1995 88855089 -UNITED ST8 UNITED ST9 1996 60466691 -UNITED ST8 UNITED ST9 1997 45852796 -UNITED ST9 UNITED ST0 1992 94813294 -UNITED ST9 UNITED ST0 1993 32653305 -UNITED ST9 UNITED ST0 1994 87670573 -UNITED ST9 UNITED ST0 1995 51548621 -UNITED ST9 UNITED ST0 1996 49590701 -UNITED ST9 UNITED ST0 1997 66066791 -UNITED ST9 UNITED ST1 1992 80795302 -UNITED ST9 UNITED ST1 1993 74059419 -UNITED ST9 UNITED ST1 1994 90440948 -UNITED ST9 UNITED ST1 1995 82938967 -UNITED ST9 UNITED ST1 1996 101983827 -UNITED ST9 UNITED ST1 1997 62418406 -UNITED ST9 UNITED ST2 1992 11623177 -UNITED ST9 UNITED ST2 1993 17718390 -UNITED ST9 UNITED ST2 1994 9700989 -UNITED ST9 UNITED ST2 1995 12653362 -UNITED ST9 UNITED ST2 1996 8108035 -UNITED ST9 UNITED ST2 1997 2661652 -UNITED ST9 UNITED ST3 1992 47681165 -UNITED ST9 UNITED ST3 1993 46943647 -UNITED ST9 UNITED ST3 1994 35846500 -UNITED ST9 UNITED ST3 1995 17539102 -UNITED ST9 UNITED ST3 1996 47021018 -UNITED ST9 UNITED ST3 1997 46121724 -UNITED ST9 UNITED ST4 1992 61489513 -UNITED ST9 UNITED ST4 1993 30644687 -UNITED ST9 UNITED ST4 1994 43910761 -UNITED ST9 UNITED ST4 1995 72525686 -UNITED ST9 UNITED ST4 1996 69616614 -UNITED ST9 UNITED ST4 1997 31765478 -UNITED ST9 UNITED ST5 1992 28075004 -UNITED ST9 UNITED ST5 1993 39685269 -UNITED ST9 UNITED ST5 1994 37146726 -UNITED ST9 UNITED ST5 1995 36009924 -UNITED ST9 UNITED ST5 1996 52825922 -UNITED ST9 UNITED ST5 1997 56091920 -UNITED ST9 UNITED ST6 1992 15879568 -UNITED ST9 UNITED ST6 1993 43867160 -UNITED ST9 UNITED ST6 1994 29212927 -UNITED ST9 UNITED ST6 1995 43482494 -UNITED ST9 UNITED ST6 1996 48685063 -UNITED ST9 UNITED ST6 1997 37596882 -UNITED ST9 UNITED ST7 1992 56883359 -UNITED ST9 UNITED ST7 1993 66625483 -UNITED ST9 UNITED ST7 1994 31586173 -UNITED ST9 UNITED ST7 1995 46464589 -UNITED ST9 UNITED ST7 1996 48798759 -UNITED ST9 UNITED ST7 1997 69052630 -UNITED ST9 UNITED ST8 1992 21386100 -UNITED ST9 UNITED ST8 1993 37184696 -UNITED ST9 UNITED ST8 1994 39278808 -UNITED ST9 UNITED ST8 1995 21292527 -UNITED ST9 UNITED ST8 1996 36112301 -UNITED ST9 UNITED ST8 1997 31272385 -UNITED ST9 UNITED ST9 1992 36048343 -UNITED ST9 UNITED ST9 1993 63260763 -UNITED ST9 UNITED ST9 1994 51980849 -UNITED ST9 UNITED ST9 1995 76515958 -UNITED ST9 UNITED ST9 1996 77301740 -UNITED ST9 UNITED ST9 1997 106222795 +UNITED ST0 UNITED ST0 1992 18293852 +UNITED ST0 UNITED ST3 1992 3201624 +UNITED ST0 UNITED ST7 1992 7137641 +UNITED ST1 UNITED ST0 1992 10878084 +UNITED ST1 UNITED ST2 1992 8004700 +UNITED ST1 UNITED ST3 1992 13938002 +UNITED ST1 UNITED ST6 1992 11000283 +UNITED ST1 UNITED ST7 1992 17410800 +UNITED ST1 UNITED ST9 1992 25644597 +UNITED ST2 UNITED ST0 1992 23943154 +UNITED ST2 UNITED ST2 1992 8313714 +UNITED ST2 UNITED ST3 1992 15331073 +UNITED ST2 UNITED ST7 1992 7759164 +UNITED ST2 UNITED ST9 1992 12343455 +UNITED ST3 UNITED ST3 1992 4080199 +UNITED ST3 UNITED ST9 1992 2614811 +UNITED ST4 UNITED ST0 1992 34626982 +UNITED ST4 UNITED ST2 1992 11696334 +UNITED ST4 UNITED ST3 1992 29767238 +UNITED ST4 UNITED ST6 1992 11369008 +UNITED ST4 UNITED ST7 1992 10151573 +UNITED ST4 UNITED ST9 1992 21189183 +UNITED ST5 UNITED ST0 1992 12301234 +UNITED ST5 UNITED ST2 1992 9917834 +UNITED ST5 UNITED ST3 1992 5862031 +UNITED ST5 UNITED ST6 1992 12398029 +UNITED ST5 UNITED ST7 1992 3936271 +UNITED ST5 UNITED ST9 1992 14448179 +UNITED ST6 UNITED ST0 1992 11900889 +UNITED ST6 UNITED ST2 1992 291566 +UNITED ST6 UNITED ST3 1992 1873819 +UNITED ST6 UNITED ST6 1992 8685228 +UNITED ST6 UNITED ST9 1992 6597261 +UNITED ST7 UNITED ST0 1992 1882015 +UNITED ST7 UNITED ST2 1992 6125476 +UNITED ST7 UNITED ST3 1992 17996772 +UNITED ST7 UNITED ST6 1992 6058017 +UNITED ST7 UNITED ST7 1992 9715656 +UNITED ST7 UNITED ST9 1992 5403152 +UNITED ST8 UNITED ST0 1992 3574169 +UNITED ST8 UNITED ST7 1992 2373825 +UNITED ST8 UNITED ST9 1992 5690491 +UNITED ST9 UNITED ST0 1992 4816370 +UNITED ST9 UNITED ST2 1992 12370917 +UNITED ST9 UNITED ST3 1992 17863433 +UNITED ST9 UNITED ST6 1992 2066609 +UNITED ST9 UNITED ST7 1992 6703890 +UNITED ST9 UNITED ST9 1992 4234523 -- !query3_3before -- -UNITED KI1 UNITED KI1 1992 46735995 -UNITED KI1 UNITED KI1 1993 45669535 -UNITED KI1 UNITED KI1 1994 34868441 -UNITED KI1 UNITED KI1 1995 35881186 -UNITED KI1 UNITED KI1 1996 59923037 -UNITED KI1 UNITED KI1 1997 70343133 -UNITED KI1 UNITED KI5 1992 21917846 -UNITED KI1 UNITED KI5 1993 21444206 -UNITED KI1 UNITED KI5 1994 21221780 -UNITED KI1 UNITED KI5 1995 17176510 -UNITED KI1 UNITED KI5 1996 13700254 -UNITED KI1 UNITED KI5 1997 16410668 -UNITED KI5 UNITED KI1 1992 36277055 -UNITED KI5 UNITED KI1 1993 61017607 -UNITED KI5 UNITED KI1 1994 36282163 -UNITED KI5 UNITED KI1 1995 82955896 -UNITED KI5 UNITED KI1 1996 65767049 -UNITED KI5 UNITED KI1 1997 64978359 -UNITED KI5 UNITED KI5 1992 25355267 -UNITED KI5 UNITED KI5 1993 34099392 -UNITED KI5 UNITED KI5 1994 34507284 -UNITED KI5 UNITED KI5 1995 20539805 -UNITED KI5 UNITED KI5 1996 46077342 -UNITED KI5 UNITED KI5 1997 27332027 +UNITED KI5 UNITED KI1 1992 4397192 -- !query3_3_after -- -UNITED KI1 UNITED KI1 1992 46735995 -UNITED KI1 UNITED KI1 1993 45669535 -UNITED KI1 UNITED KI1 1994 34868441 -UNITED KI1 UNITED KI1 1995 35881186 -UNITED KI1 UNITED KI1 1996 59923037 -UNITED KI1 UNITED KI1 1997 70343133 -UNITED KI1 UNITED KI5 1992 21917846 -UNITED KI1 UNITED KI5 1993 21444206 -UNITED KI1 UNITED KI5 1994 21221780 -UNITED KI1 UNITED KI5 1995 17176510 -UNITED KI1 UNITED KI5 1996 13700254 -UNITED KI1 UNITED KI5 1997 16410668 -UNITED KI5 UNITED KI1 1992 36277055 -UNITED KI5 UNITED KI1 1993 61017607 -UNITED KI5 UNITED KI1 1994 36282163 -UNITED KI5 UNITED KI1 1995 82955896 -UNITED KI5 UNITED KI1 1996 65767049 -UNITED KI5 UNITED KI1 1997 64978359 -UNITED KI5 UNITED KI5 1992 25355267 -UNITED KI5 UNITED KI5 1993 34099392 -UNITED KI5 UNITED KI5 1994 34507284 -UNITED KI5 UNITED KI5 1995 20539805 -UNITED KI5 UNITED KI5 1996 46077342 -UNITED KI5 UNITED KI5 1997 27332027 +UNITED KI5 UNITED KI1 1992 4397192 -- !query3_4before -- -UNITED KI1 UNITED KI5 1997 1870070 -UNITED KI5 UNITED KI1 1997 9117846 -UNITED KI5 UNITED KI5 1997 6203801 +UNITED KI5 UNITED KI1 1992 3856441 -- !query3_4_after -- -UNITED KI1 UNITED KI5 1997 1870070 -UNITED KI5 UNITED KI1 1997 9117846 -UNITED KI5 UNITED KI5 1997 6203801 +UNITED KI5 UNITED KI1 1992 3856441 -- !query4_1before -- -1992 ARGENTINA 9658964469 -1992 BRAZIL 9226983055 -1992 CANADA 9643176787 -1992 PERU 9410676597 -1992 UNITED STATES 9849427653 -1993 ARGENTINA 9976332853 -1993 BRAZIL 9468799229 -1993 CANADA 9897302420 -1993 PERU 9309445836 -1993 UNITED STATES 10003985086 -1994 ARGENTINA 9940305215 -1994 BRAZIL 9348651677 -1994 CANADA 9582647596 -1994 PERU 9295265013 -1994 UNITED STATES 9519880425 -1995 ARGENTINA 10143841380 -1995 BRAZIL 9156077350 -1995 CANADA 9562612160 -1995 PERU 9778087211 -1995 UNITED STATES 9310565244 -1996 ARGENTINA 10001927895 -1996 BRAZIL 9168485151 -1996 CANADA 10061604203 -1996 PERU 9355135674 -1996 UNITED STATES 9769712174 -1997 ARGENTINA 9854560261 -1997 BRAZIL 9121571047 -1997 CANADA 10097371778 -1997 PERU 9315525917 -1997 UNITED STATES 10506723879 -1998 ARGENTINA 5834240407 -1998 BRAZIL 5356398095 -1998 CANADA 5423211196 -1998 PERU 5726185970 -1998 UNITED STATES 6009462019 +1992 ARGENTINA 533196600 +1992 BRAZIL 684224630 +1992 CANADA 532686194 +1992 PERU 586223155 +1992 UNITED STATES 682387184 -- !query4_1_after -- -1992 ARGENTINA 9658964469 -1992 BRAZIL 9226983055 -1992 CANADA 9643176787 -1992 PERU 9410676597 -1992 UNITED STATES 9849427653 -1993 ARGENTINA 9976332853 -1993 BRAZIL 9468799229 -1993 CANADA 9897302420 -1993 PERU 9309445836 -1993 UNITED STATES 10003985086 -1994 ARGENTINA 9940305215 -1994 BRAZIL 9348651677 -1994 CANADA 9582647596 -1994 PERU 9295265013 -1994 UNITED STATES 9519880425 -1995 ARGENTINA 10143841380 -1995 BRAZIL 9156077350 -1995 CANADA 9562612160 -1995 PERU 9778087211 -1995 UNITED STATES 9310565244 -1996 ARGENTINA 10001927895 -1996 BRAZIL 9168485151 -1996 CANADA 10061604203 -1996 PERU 9355135674 -1996 UNITED STATES 9769712174 -1997 ARGENTINA 9854560261 -1997 BRAZIL 9121571047 -1997 CANADA 10097371778 -1997 PERU 9315525917 -1997 UNITED STATES 10506723879 -1998 ARGENTINA 5834240407 -1998 BRAZIL 5356398095 -1998 CANADA 5423211196 -1998 PERU 5726185970 -1998 UNITED STATES 6009462019 +1992 ARGENTINA 533196600 +1992 BRAZIL 684224630 +1992 CANADA 532686194 +1992 PERU 586223155 +1992 UNITED STATES 682387184 -- !query4_2before -- -1997 ARGENTINA MFGR#11 907071566 -1997 ARGENTINA MFGR#12 924115562 -1997 ARGENTINA MFGR#13 972861821 -1997 ARGENTINA MFGR#14 975410345 -1997 ARGENTINA MFGR#15 938867375 -1997 ARGENTINA MFGR#21 1014782574 -1997 ARGENTINA MFGR#22 873016783 -1997 ARGENTINA MFGR#23 1030357302 -1997 ARGENTINA MFGR#24 893460579 -1997 ARGENTINA MFGR#25 1006311403 -1997 BRAZIL MFGR#11 1073219328 -1997 BRAZIL MFGR#12 989858833 -1997 BRAZIL MFGR#13 1128480379 -1997 BRAZIL MFGR#14 1194256722 -1997 BRAZIL MFGR#15 1094419124 -1997 BRAZIL MFGR#21 910026832 -1997 BRAZIL MFGR#22 993142048 -1997 BRAZIL MFGR#23 1107672874 -1997 BRAZIL MFGR#24 1058013649 -1997 BRAZIL MFGR#25 994733764 -1997 CANADA MFGR#11 854725020 -1997 CANADA MFGR#12 938718164 -1997 CANADA MFGR#13 959265890 -1997 CANADA MFGR#14 1002812450 -1997 CANADA MFGR#15 848183013 -1997 CANADA MFGR#21 999805272 -1997 CANADA MFGR#22 778419763 -1997 CANADA MFGR#23 928359645 -1997 CANADA MFGR#24 849895128 -1997 CANADA MFGR#25 904587965 -1997 PERU MFGR#11 1100242877 -1997 PERU MFGR#12 994365350 -1997 PERU MFGR#13 847486105 -1997 PERU MFGR#14 947769683 -1997 PERU MFGR#15 999395678 -1997 PERU MFGR#21 867923394 -1997 PERU MFGR#22 988747459 -1997 PERU MFGR#23 1066645086 -1997 PERU MFGR#24 935665725 -1997 PERU MFGR#25 981454129 -1997 UNITED STATES MFGR#11 1046706048 -1997 UNITED STATES MFGR#12 909213709 -1997 UNITED STATES MFGR#13 1096278906 -1997 UNITED STATES MFGR#14 934282111 -1997 UNITED STATES MFGR#15 962760843 -1997 UNITED STATES MFGR#21 1000676474 -1997 UNITED STATES MFGR#22 1076947615 -1997 UNITED STATES MFGR#23 937288102 -1997 UNITED STATES MFGR#24 1003386363 -1997 UNITED STATES MFGR#25 1053666052 -1998 ARGENTINA MFGR#11 567612227 -1998 ARGENTINA MFGR#12 527025042 -1998 ARGENTINA MFGR#13 582791792 -1998 ARGENTINA MFGR#14 523726368 -1998 ARGENTINA MFGR#15 522078267 -1998 ARGENTINA MFGR#21 504712685 -1998 ARGENTINA MFGR#22 506061736 -1998 ARGENTINA MFGR#23 560479801 -1998 ARGENTINA MFGR#24 524579118 -1998 ARGENTINA MFGR#25 547840211 -1998 BRAZIL MFGR#11 638578488 -1998 BRAZIL MFGR#12 646312681 -1998 BRAZIL MFGR#13 655161772 -1998 BRAZIL MFGR#14 552676170 -1998 BRAZIL MFGR#15 663812709 -1998 BRAZIL MFGR#21 668822448 -1998 BRAZIL MFGR#22 591791918 -1998 BRAZIL MFGR#23 690648877 -1998 BRAZIL MFGR#24 562362220 -1998 BRAZIL MFGR#25 704182376 -1998 CANADA MFGR#11 509086125 -1998 CANADA MFGR#12 488089772 -1998 CANADA MFGR#13 486533297 -1998 CANADA MFGR#14 527837227 -1998 CANADA MFGR#15 535869299 -1998 CANADA MFGR#21 455868651 -1998 CANADA MFGR#22 594277308 -1998 CANADA MFGR#23 574125070 -1998 CANADA MFGR#24 508530424 -1998 CANADA MFGR#25 547757992 -1998 PERU MFGR#11 567838547 -1998 PERU MFGR#12 540544757 -1998 PERU MFGR#13 591331883 -1998 PERU MFGR#14 481335064 -1998 PERU MFGR#15 570246138 -1998 PERU MFGR#21 533733330 -1998 PERU MFGR#22 527790656 -1998 PERU MFGR#23 636393221 -1998 PERU MFGR#24 589075262 -1998 PERU MFGR#25 543251115 -1998 UNITED STATES MFGR#11 556276232 -1998 UNITED STATES MFGR#12 612385982 -1998 UNITED STATES MFGR#13 622413927 -1998 UNITED STATES MFGR#14 555006887 -1998 UNITED STATES MFGR#15 525119569 -1998 UNITED STATES MFGR#21 563406836 -1998 UNITED STATES MFGR#22 580478735 -1998 UNITED STATES MFGR#23 656080465 -1998 UNITED STATES MFGR#24 538445058 -1998 UNITED STATES MFGR#25 589111952 +1992 ARGENTINA MFGR#11 70016467 +1992 ARGENTINA MFGR#12 57130735 +1992 ARGENTINA MFGR#13 89600962 +1992 ARGENTINA MFGR#14 54217587 +1992 ARGENTINA MFGR#15 45034625 +1992 ARGENTINA MFGR#21 43792906 +1992 ARGENTINA MFGR#22 49248625 +1992 ARGENTINA MFGR#23 79718572 +1992 ARGENTINA MFGR#24 51057067 +1992 ARGENTINA MFGR#25 47431621 +1992 BRAZIL MFGR#11 49035302 +1992 BRAZIL MFGR#12 45469986 +1992 BRAZIL MFGR#13 68191869 +1992 BRAZIL MFGR#14 54781608 +1992 BRAZIL MFGR#15 40204144 +1992 BRAZIL MFGR#21 54385184 +1992 BRAZIL MFGR#22 73332346 +1992 BRAZIL MFGR#23 47288876 +1992 BRAZIL MFGR#24 61293205 +1992 BRAZIL MFGR#25 77335065 +1992 CANADA MFGR#11 20381804 +1992 CANADA MFGR#12 30863219 +1992 CANADA MFGR#13 52567971 +1992 CANADA MFGR#14 65603736 +1992 CANADA MFGR#15 49669956 +1992 CANADA MFGR#21 45613232 +1992 CANADA MFGR#22 32289959 +1992 CANADA MFGR#23 23226684 +1992 CANADA MFGR#24 49284224 +1992 CANADA MFGR#25 44418525 +1992 PERU MFGR#11 60890044 +1992 PERU MFGR#12 57092894 +1992 PERU MFGR#13 54461960 +1992 PERU MFGR#14 39217023 +1992 PERU MFGR#15 21170135 +1992 PERU MFGR#21 33391866 +1992 PERU MFGR#22 59295631 +1992 PERU MFGR#23 69486455 +1992 PERU MFGR#24 34232078 +1992 PERU MFGR#25 45525878 +1992 UNITED STATES MFGR#11 91427459 +1992 UNITED STATES MFGR#12 100744805 +1992 UNITED STATES MFGR#13 101592664 +1992 UNITED STATES MFGR#14 121042999 +1992 UNITED STATES MFGR#15 99396680 +1992 UNITED STATES MFGR#21 72692399 +1992 UNITED STATES MFGR#22 92929921 +1992 UNITED STATES MFGR#23 101853653 +1992 UNITED STATES MFGR#24 96327671 +1992 UNITED STATES MFGR#25 93459486 -- !query4_2_after -- -1997 ARGENTINA MFGR#11 907071566 -1997 ARGENTINA MFGR#12 924115562 -1997 ARGENTINA MFGR#13 972861821 -1997 ARGENTINA MFGR#14 975410345 -1997 ARGENTINA MFGR#15 938867375 -1997 ARGENTINA MFGR#21 1014782574 -1997 ARGENTINA MFGR#22 873016783 -1997 ARGENTINA MFGR#23 1030357302 -1997 ARGENTINA MFGR#24 893460579 -1997 ARGENTINA MFGR#25 1006311403 -1997 BRAZIL MFGR#11 1073219328 -1997 BRAZIL MFGR#12 989858833 -1997 BRAZIL MFGR#13 1128480379 -1997 BRAZIL MFGR#14 1194256722 -1997 BRAZIL MFGR#15 1094419124 -1997 BRAZIL MFGR#21 910026832 -1997 BRAZIL MFGR#22 993142048 -1997 BRAZIL MFGR#23 1107672874 -1997 BRAZIL MFGR#24 1058013649 -1997 BRAZIL MFGR#25 994733764 -1997 CANADA MFGR#11 854725020 -1997 CANADA MFGR#12 938718164 -1997 CANADA MFGR#13 959265890 -1997 CANADA MFGR#14 1002812450 -1997 CANADA MFGR#15 848183013 -1997 CANADA MFGR#21 999805272 -1997 CANADA MFGR#22 778419763 -1997 CANADA MFGR#23 928359645 -1997 CANADA MFGR#24 849895128 -1997 CANADA MFGR#25 904587965 -1997 PERU MFGR#11 1100242877 -1997 PERU MFGR#12 994365350 -1997 PERU MFGR#13 847486105 -1997 PERU MFGR#14 947769683 -1997 PERU MFGR#15 999395678 -1997 PERU MFGR#21 867923394 -1997 PERU MFGR#22 988747459 -1997 PERU MFGR#23 1066645086 -1997 PERU MFGR#24 935665725 -1997 PERU MFGR#25 981454129 -1997 UNITED STATES MFGR#11 1046706048 -1997 UNITED STATES MFGR#12 909213709 -1997 UNITED STATES MFGR#13 1096278906 -1997 UNITED STATES MFGR#14 934282111 -1997 UNITED STATES MFGR#15 962760843 -1997 UNITED STATES MFGR#21 1000676474 -1997 UNITED STATES MFGR#22 1076947615 -1997 UNITED STATES MFGR#23 937288102 -1997 UNITED STATES MFGR#24 1003386363 -1997 UNITED STATES MFGR#25 1053666052 -1998 ARGENTINA MFGR#11 567612227 -1998 ARGENTINA MFGR#12 527025042 -1998 ARGENTINA MFGR#13 582791792 -1998 ARGENTINA MFGR#14 523726368 -1998 ARGENTINA MFGR#15 522078267 -1998 ARGENTINA MFGR#21 504712685 -1998 ARGENTINA MFGR#22 506061736 -1998 ARGENTINA MFGR#23 560479801 -1998 ARGENTINA MFGR#24 524579118 -1998 ARGENTINA MFGR#25 547840211 -1998 BRAZIL MFGR#11 638578488 -1998 BRAZIL MFGR#12 646312681 -1998 BRAZIL MFGR#13 655161772 -1998 BRAZIL MFGR#14 552676170 -1998 BRAZIL MFGR#15 663812709 -1998 BRAZIL MFGR#21 668822448 -1998 BRAZIL MFGR#22 591791918 -1998 BRAZIL MFGR#23 690648877 -1998 BRAZIL MFGR#24 562362220 -1998 BRAZIL MFGR#25 704182376 -1998 CANADA MFGR#11 509086125 -1998 CANADA MFGR#12 488089772 -1998 CANADA MFGR#13 486533297 -1998 CANADA MFGR#14 527837227 -1998 CANADA MFGR#15 535869299 -1998 CANADA MFGR#21 455868651 -1998 CANADA MFGR#22 594277308 -1998 CANADA MFGR#23 574125070 -1998 CANADA MFGR#24 508530424 -1998 CANADA MFGR#25 547757992 -1998 PERU MFGR#11 567838547 -1998 PERU MFGR#12 540544757 -1998 PERU MFGR#13 591331883 -1998 PERU MFGR#14 481335064 -1998 PERU MFGR#15 570246138 -1998 PERU MFGR#21 533733330 -1998 PERU MFGR#22 527790656 -1998 PERU MFGR#23 636393221 -1998 PERU MFGR#24 589075262 -1998 PERU MFGR#25 543251115 -1998 UNITED STATES MFGR#11 556276232 -1998 UNITED STATES MFGR#12 612385982 -1998 UNITED STATES MFGR#13 622413927 -1998 UNITED STATES MFGR#14 555006887 -1998 UNITED STATES MFGR#15 525119569 -1998 UNITED STATES MFGR#21 563406836 -1998 UNITED STATES MFGR#22 580478735 -1998 UNITED STATES MFGR#23 656080465 -1998 UNITED STATES MFGR#24 538445058 -1998 UNITED STATES MFGR#25 589111952 +1992 ARGENTINA MFGR#11 70016467 +1992 ARGENTINA MFGR#12 57130735 +1992 ARGENTINA MFGR#13 89600962 +1992 ARGENTINA MFGR#14 54217587 +1992 ARGENTINA MFGR#15 45034625 +1992 ARGENTINA MFGR#21 43792906 +1992 ARGENTINA MFGR#22 49248625 +1992 ARGENTINA MFGR#23 79718572 +1992 ARGENTINA MFGR#24 51057067 +1992 ARGENTINA MFGR#25 47431621 +1992 BRAZIL MFGR#11 49035302 +1992 BRAZIL MFGR#12 45469986 +1992 BRAZIL MFGR#13 68191869 +1992 BRAZIL MFGR#14 54781608 +1992 BRAZIL MFGR#15 40204144 +1992 BRAZIL MFGR#21 54385184 +1992 BRAZIL MFGR#22 73332346 +1992 BRAZIL MFGR#23 47288876 +1992 BRAZIL MFGR#24 61293205 +1992 BRAZIL MFGR#25 77335065 +1992 CANADA MFGR#11 20381804 +1992 CANADA MFGR#12 30863219 +1992 CANADA MFGR#13 52567971 +1992 CANADA MFGR#14 65603736 +1992 CANADA MFGR#15 49669956 +1992 CANADA MFGR#21 45613232 +1992 CANADA MFGR#22 32289959 +1992 CANADA MFGR#23 23226684 +1992 CANADA MFGR#24 49284224 +1992 CANADA MFGR#25 44418525 +1992 PERU MFGR#11 60890044 +1992 PERU MFGR#12 57092894 +1992 PERU MFGR#13 54461960 +1992 PERU MFGR#14 39217023 +1992 PERU MFGR#15 21170135 +1992 PERU MFGR#21 33391866 +1992 PERU MFGR#22 59295631 +1992 PERU MFGR#23 69486455 +1992 PERU MFGR#24 34232078 +1992 PERU MFGR#25 45525878 +1992 UNITED STATES MFGR#11 91427459 +1992 UNITED STATES MFGR#12 100744805 +1992 UNITED STATES MFGR#13 101592664 +1992 UNITED STATES MFGR#14 121042999 +1992 UNITED STATES MFGR#15 99396680 +1992 UNITED STATES MFGR#21 72692399 +1992 UNITED STATES MFGR#22 92929921 +1992 UNITED STATES MFGR#23 101853653 +1992 UNITED STATES MFGR#24 96327671 +1992 UNITED STATES MFGR#25 93459486 -- !query4_3before -- -1997 UNITED ST0 MFGR#141 15456288 -1997 UNITED ST0 MFGR#1410 32961113 -1997 UNITED ST0 MFGR#1411 36166400 -1997 UNITED ST0 MFGR#1412 18237451 -1997 UNITED ST0 MFGR#1413 40752813 -1997 UNITED ST0 MFGR#1414 24237588 -1997 UNITED ST0 MFGR#1415 6419001 -1997 UNITED ST0 MFGR#1416 15639876 -1997 UNITED ST0 MFGR#1417 15476211 -1997 UNITED ST0 MFGR#1418 8169956 -1997 UNITED ST0 MFGR#1419 18217252 -1997 UNITED ST0 MFGR#142 29361222 -1997 UNITED ST0 MFGR#1420 34587603 -1997 UNITED ST0 MFGR#1421 28560921 -1997 UNITED ST0 MFGR#1422 25154181 -1997 UNITED ST0 MFGR#1423 18144008 -1997 UNITED ST0 MFGR#1424 28064267 -1997 UNITED ST0 MFGR#1425 15255845 -1997 UNITED ST0 MFGR#1426 5610353 -1997 UNITED ST0 MFGR#1427 5750693 -1997 UNITED ST0 MFGR#1428 14081707 -1997 UNITED ST0 MFGR#1429 9024512 -1997 UNITED ST0 MFGR#143 5325193 -1997 UNITED ST0 MFGR#1430 16961739 -1997 UNITED ST0 MFGR#1431 19015253 -1997 UNITED ST0 MFGR#1432 6324917 -1997 UNITED ST0 MFGR#1433 6734461 -1997 UNITED ST0 MFGR#1434 19206393 -1997 UNITED ST0 MFGR#1435 679925 -1997 UNITED ST0 MFGR#1436 13834012 -1997 UNITED ST0 MFGR#1437 23575407 -1997 UNITED ST0 MFGR#1438 24756074 -1997 UNITED ST0 MFGR#1439 19644072 -1997 UNITED ST0 MFGR#144 20285157 -1997 UNITED ST0 MFGR#1440 14068558 -1997 UNITED ST0 MFGR#145 22646138 -1997 UNITED ST0 MFGR#146 16381159 -1997 UNITED ST0 MFGR#147 12267827 -1997 UNITED ST0 MFGR#148 30968614 -1997 UNITED ST0 MFGR#149 8640669 -1997 UNITED ST1 MFGR#141 10152087 -1997 UNITED ST1 MFGR#1410 32583300 -1997 UNITED ST1 MFGR#1411 15690352 -1997 UNITED ST1 MFGR#1412 16671403 -1997 UNITED ST1 MFGR#1413 4942775 -1997 UNITED ST1 MFGR#1414 9391597 -1997 UNITED ST1 MFGR#1415 26937296 -1997 UNITED ST1 MFGR#1416 25782871 -1997 UNITED ST1 MFGR#1417 23349641 -1997 UNITED ST1 MFGR#1418 10934620 -1997 UNITED ST1 MFGR#1419 22673030 -1997 UNITED ST1 MFGR#142 14048276 -1997 UNITED ST1 MFGR#1420 18100795 -1997 UNITED ST1 MFGR#1421 25405678 -1997 UNITED ST1 MFGR#1423 5713800 -1997 UNITED ST1 MFGR#1424 4131923 -1997 UNITED ST1 MFGR#1425 17190425 -1997 UNITED ST1 MFGR#1426 22314044 -1997 UNITED ST1 MFGR#1427 13635057 -1997 UNITED ST1 MFGR#1428 17735811 -1997 UNITED ST1 MFGR#1429 28740911 -1997 UNITED ST1 MFGR#143 27418033 -1997 UNITED ST1 MFGR#1430 18655292 -1997 UNITED ST1 MFGR#1431 11074399 -1997 UNITED ST1 MFGR#1432 7490170 -1997 UNITED ST1 MFGR#1433 28997266 -1997 UNITED ST1 MFGR#1434 4612211 -1997 UNITED ST1 MFGR#1435 16327124 -1997 UNITED ST1 MFGR#1436 17409665 -1997 UNITED ST1 MFGR#1437 33391635 -1997 UNITED ST1 MFGR#1438 37321134 -1997 UNITED ST1 MFGR#1439 10279941 -1997 UNITED ST1 MFGR#144 12304723 -1997 UNITED ST1 MFGR#1440 28921230 -1997 UNITED ST1 MFGR#145 17933684 -1997 UNITED ST1 MFGR#146 8709103 -1997 UNITED ST1 MFGR#147 20145983 -1997 UNITED ST1 MFGR#148 16940400 -1997 UNITED ST1 MFGR#149 10284949 -1997 UNITED ST2 MFGR#1412 6328240 -1997 UNITED ST2 MFGR#1415 5022682 -1997 UNITED ST2 MFGR#1416 4102214 -1997 UNITED ST2 MFGR#1417 5284444 -1997 UNITED ST2 MFGR#1418 2649908 -1997 UNITED ST2 MFGR#1419 4664673 -1997 UNITED ST2 MFGR#142 10784835 -1997 UNITED ST2 MFGR#1423 4095190 -1997 UNITED ST2 MFGR#1424 5376355 -1997 UNITED ST2 MFGR#1425 3949957 -1997 UNITED ST2 MFGR#143 2859829 -1997 UNITED ST2 MFGR#1430 4618266 -1997 UNITED ST2 MFGR#1432 5183060 -1997 UNITED ST2 MFGR#1433 10531371 -1997 UNITED ST2 MFGR#1434 77646 -1997 UNITED ST2 MFGR#1436 6538508 -1997 UNITED ST2 MFGR#1438 12324747 -1997 UNITED ST2 MFGR#144 2002870 -1997 UNITED ST2 MFGR#1440 12247229 -1997 UNITED ST2 MFGR#145 202685 -1997 UNITED ST2 MFGR#148 5998932 -1997 UNITED ST3 MFGR#141 12866083 -1997 UNITED ST3 MFGR#1410 31051703 -1997 UNITED ST3 MFGR#1411 14099543 -1997 UNITED ST3 MFGR#1412 4970551 -1997 UNITED ST3 MFGR#1413 26129762 -1997 UNITED ST3 MFGR#1414 22364787 -1997 UNITED ST3 MFGR#1415 27883428 -1997 UNITED ST3 MFGR#1416 25289630 -1997 UNITED ST3 MFGR#1417 5524810 -1997 UNITED ST3 MFGR#1419 3616608 -1997 UNITED ST3 MFGR#142 10776059 -1997 UNITED ST3 MFGR#1420 12958851 -1997 UNITED ST3 MFGR#1421 5815494 -1997 UNITED ST3 MFGR#1422 17694270 -1997 UNITED ST3 MFGR#1423 10285142 -1997 UNITED ST3 MFGR#1424 21555317 -1997 UNITED ST3 MFGR#1425 5214517 -1997 UNITED ST3 MFGR#1426 16810700 -1997 UNITED ST3 MFGR#1427 10086430 -1997 UNITED ST3 MFGR#1428 7733716 -1997 UNITED ST3 MFGR#1429 11922430 -1997 UNITED ST3 MFGR#1430 20105292 -1997 UNITED ST3 MFGR#1431 14716979 -1997 UNITED ST3 MFGR#1432 15779247 -1997 UNITED ST3 MFGR#1433 15308773 -1997 UNITED ST3 MFGR#1434 8515037 -1997 UNITED ST3 MFGR#1435 25486899 -1997 UNITED ST3 MFGR#1436 12739448 -1997 UNITED ST3 MFGR#1437 11249180 -1997 UNITED ST3 MFGR#1438 4156824 -1997 UNITED ST3 MFGR#1439 18358277 -1997 UNITED ST3 MFGR#144 10097718 -1997 UNITED ST3 MFGR#1440 22920494 -1997 UNITED ST3 MFGR#145 20285859 -1997 UNITED ST3 MFGR#146 18406423 -1997 UNITED ST3 MFGR#147 17295990 -1997 UNITED ST3 MFGR#148 26127145 -1997 UNITED ST3 MFGR#149 19895538 -1997 UNITED ST4 MFGR#141 8135477 -1997 UNITED ST4 MFGR#1410 5503247 -1997 UNITED ST4 MFGR#1411 6096435 -1997 UNITED ST4 MFGR#1412 19976043 -1997 UNITED ST4 MFGR#1413 22581282 -1997 UNITED ST4 MFGR#1414 10101379 -1997 UNITED ST4 MFGR#1415 8877444 -1997 UNITED ST4 MFGR#1416 17943398 -1997 UNITED ST4 MFGR#1417 9224501 -1997 UNITED ST4 MFGR#1418 16255176 -1997 UNITED ST4 MFGR#1419 683069 -1997 UNITED ST4 MFGR#142 23512944 -1997 UNITED ST4 MFGR#1420 27954262 -1997 UNITED ST4 MFGR#1421 54209185 -1997 UNITED ST4 MFGR#1422 1173957 -1997 UNITED ST4 MFGR#1423 20410710 -1997 UNITED ST4 MFGR#1424 2889270 -1997 UNITED ST4 MFGR#1425 13361796 -1997 UNITED ST4 MFGR#1426 18678819 -1997 UNITED ST4 MFGR#1427 18206066 -1997 UNITED ST4 MFGR#1428 18153974 -1997 UNITED ST4 MFGR#1429 12545076 -1997 UNITED ST4 MFGR#143 6158834 -1997 UNITED ST4 MFGR#1430 16128370 -1997 UNITED ST4 MFGR#1431 11153717 -1997 UNITED ST4 MFGR#1432 7745636 -1997 UNITED ST4 MFGR#1433 15983634 -1997 UNITED ST4 MFGR#1434 3662246 -1997 UNITED ST4 MFGR#1435 3060180 -1997 UNITED ST4 MFGR#1436 11524592 -1997 UNITED ST4 MFGR#1437 36824174 -1997 UNITED ST4 MFGR#1438 8226666 -1997 UNITED ST4 MFGR#1439 9872470 -1997 UNITED ST4 MFGR#1440 401207 -1997 UNITED ST4 MFGR#145 11698312 -1997 UNITED ST4 MFGR#146 18259956 -1997 UNITED ST4 MFGR#147 10256110 -1997 UNITED ST4 MFGR#148 15892628 -1997 UNITED ST4 MFGR#149 4322491 -1997 UNITED ST5 MFGR#141 8534275 -1997 UNITED ST5 MFGR#1411 2427261 -1997 UNITED ST5 MFGR#1412 9032422 -1997 UNITED ST5 MFGR#1413 6343588 -1997 UNITED ST5 MFGR#1414 2961025 -1997 UNITED ST5 MFGR#1415 21186568 -1997 UNITED ST5 MFGR#1416 9354080 -1997 UNITED ST5 MFGR#1417 11786246 -1997 UNITED ST5 MFGR#1418 15297245 -1997 UNITED ST5 MFGR#1420 15787608 -1997 UNITED ST5 MFGR#1421 17858301 -1997 UNITED ST5 MFGR#1422 14080987 -1997 UNITED ST5 MFGR#1423 12535186 -1997 UNITED ST5 MFGR#1424 3221960 -1997 UNITED ST5 MFGR#1425 8950317 -1997 UNITED ST5 MFGR#1426 13161486 -1997 UNITED ST5 MFGR#1427 6149737 -1997 UNITED ST5 MFGR#1428 6796520 -1997 UNITED ST5 MFGR#1429 26026300 -1997 UNITED ST5 MFGR#143 6712572 -1997 UNITED ST5 MFGR#1430 16092131 -1997 UNITED ST5 MFGR#1431 3425291 -1997 UNITED ST5 MFGR#1432 14630175 -1997 UNITED ST5 MFGR#1433 4967585 -1997 UNITED ST5 MFGR#1434 6716261 -1997 UNITED ST5 MFGR#1435 6290790 -1997 UNITED ST5 MFGR#1436 6787699 -1997 UNITED ST5 MFGR#1437 6045168 -1997 UNITED ST5 MFGR#1438 1082877 -1997 UNITED ST5 MFGR#1439 5268611 -1997 UNITED ST5 MFGR#144 8900840 -1997 UNITED ST5 MFGR#1440 2644336 -1997 UNITED ST5 MFGR#145 18054844 -1997 UNITED ST5 MFGR#146 4893668 -1997 UNITED ST5 MFGR#147 9615891 -1997 UNITED ST5 MFGR#148 1426124 -1997 UNITED ST5 MFGR#149 7320726 -1997 UNITED ST6 MFGR#141 10051337 -1997 UNITED ST6 MFGR#1410 23352963 -1997 UNITED ST6 MFGR#1411 1537303 -1997 UNITED ST6 MFGR#1412 4332646 -1997 UNITED ST6 MFGR#1413 14686752 -1997 UNITED ST6 MFGR#1414 7226409 -1997 UNITED ST6 MFGR#1415 15062686 -1997 UNITED ST6 MFGR#1416 8606480 -1997 UNITED ST6 MFGR#1417 7662761 -1997 UNITED ST6 MFGR#1418 1810860 -1997 UNITED ST6 MFGR#1419 20179568 -1997 UNITED ST6 MFGR#142 15506072 -1997 UNITED ST6 MFGR#1420 3844271 -1997 UNITED ST6 MFGR#1421 10608709 -1997 UNITED ST6 MFGR#1422 16083699 -1997 UNITED ST6 MFGR#1423 6789571 -1997 UNITED ST6 MFGR#1424 17631950 -1997 UNITED ST6 MFGR#1425 8477221 -1997 UNITED ST6 MFGR#1426 16847029 -1997 UNITED ST6 MFGR#1428 17314796 -1997 UNITED ST6 MFGR#1429 19175871 -1997 UNITED ST6 MFGR#143 8775848 -1997 UNITED ST6 MFGR#1431 5319063 -1997 UNITED ST6 MFGR#1432 7875606 -1997 UNITED ST6 MFGR#1433 9305264 -1997 UNITED ST6 MFGR#1434 8050404 -1997 UNITED ST6 MFGR#1435 17585836 -1997 UNITED ST6 MFGR#1436 13745758 -1997 UNITED ST6 MFGR#1438 14536324 -1997 UNITED ST6 MFGR#1439 11451329 -1997 UNITED ST6 MFGR#144 10794837 -1997 UNITED ST6 MFGR#1440 5719238 -1997 UNITED ST6 MFGR#145 25449058 -1997 UNITED ST6 MFGR#146 17821996 -1997 UNITED ST6 MFGR#147 18922333 -1997 UNITED ST6 MFGR#148 10825718 -1997 UNITED ST6 MFGR#149 14950868 -1997 UNITED ST7 MFGR#141 8291476 -1997 UNITED ST7 MFGR#1410 4622953 -1997 UNITED ST7 MFGR#1411 14885507 -1997 UNITED ST7 MFGR#1412 19011375 -1997 UNITED ST7 MFGR#1413 31178370 -1997 UNITED ST7 MFGR#1414 16429613 -1997 UNITED ST7 MFGR#1415 3091832 -1997 UNITED ST7 MFGR#1416 17008243 -1997 UNITED ST7 MFGR#1417 15609283 -1997 UNITED ST7 MFGR#1418 5059394 -1997 UNITED ST7 MFGR#1419 7234721 -1997 UNITED ST7 MFGR#142 6898938 -1997 UNITED ST7 MFGR#1420 6426590 -1997 UNITED ST7 MFGR#1421 6043221 -1997 UNITED ST7 MFGR#1422 13273021 -1997 UNITED ST7 MFGR#1423 30537249 -1997 UNITED ST7 MFGR#1424 9224296 -1997 UNITED ST7 MFGR#1425 12799069 -1997 UNITED ST7 MFGR#1426 30277891 -1997 UNITED ST7 MFGR#1427 8331493 -1997 UNITED ST7 MFGR#1428 16205423 -1997 UNITED ST7 MFGR#1429 14396627 -1997 UNITED ST7 MFGR#143 10209623 -1997 UNITED ST7 MFGR#1430 16556758 -1997 UNITED ST7 MFGR#1431 15775327 -1997 UNITED ST7 MFGR#1432 24602989 -1997 UNITED ST7 MFGR#1433 22927717 -1997 UNITED ST7 MFGR#1434 9486462 -1997 UNITED ST7 MFGR#1435 14254290 -1997 UNITED ST7 MFGR#1436 8645365 -1997 UNITED ST7 MFGR#1437 19026166 -1997 UNITED ST7 MFGR#1438 11291550 -1997 UNITED ST7 MFGR#1439 9590292 -1997 UNITED ST7 MFGR#144 14110404 -1997 UNITED ST7 MFGR#1440 21877355 -1997 UNITED ST7 MFGR#145 8246717 -1997 UNITED ST7 MFGR#146 9785781 -1997 UNITED ST7 MFGR#147 13412691 -1997 UNITED ST7 MFGR#148 14235939 -1997 UNITED ST7 MFGR#149 23450309 -1997 UNITED ST8 MFGR#141 7323416 -1997 UNITED ST8 MFGR#1410 7191961 -1997 UNITED ST8 MFGR#1411 2425849 -1997 UNITED ST8 MFGR#1412 26725641 -1997 UNITED ST8 MFGR#1413 7915896 -1997 UNITED ST8 MFGR#1414 14359044 -1997 UNITED ST8 MFGR#1415 15834927 -1997 UNITED ST8 MFGR#1416 7464470 -1997 UNITED ST8 MFGR#1417 15274496 -1997 UNITED ST8 MFGR#1418 3261391 -1997 UNITED ST8 MFGR#1419 8020386 -1997 UNITED ST8 MFGR#142 5985550 -1997 UNITED ST8 MFGR#1420 9441461 -1997 UNITED ST8 MFGR#1421 2998735 -1997 UNITED ST8 MFGR#1423 1565415 -1997 UNITED ST8 MFGR#1424 5613094 -1997 UNITED ST8 MFGR#1425 7186734 -1997 UNITED ST8 MFGR#1426 3153117 -1997 UNITED ST8 MFGR#1427 11303043 -1997 UNITED ST8 MFGR#1428 3564003 -1997 UNITED ST8 MFGR#143 6357477 -1997 UNITED ST8 MFGR#1430 6633950 -1997 UNITED ST8 MFGR#1431 867285 -1997 UNITED ST8 MFGR#1432 9962884 -1997 UNITED ST8 MFGR#1433 13708011 -1997 UNITED ST8 MFGR#1434 7357067 -1997 UNITED ST8 MFGR#1435 11993453 -1997 UNITED ST8 MFGR#1436 7231938 -1997 UNITED ST8 MFGR#1437 18632056 -1997 UNITED ST8 MFGR#1438 3609384 -1997 UNITED ST8 MFGR#144 4822734 -1997 UNITED ST8 MFGR#1440 2521917 -1997 UNITED ST8 MFGR#145 11683385 -1997 UNITED ST8 MFGR#146 3350415 -1997 UNITED ST8 MFGR#147 8425393 -1997 UNITED ST8 MFGR#148 15005341 -1997 UNITED ST8 MFGR#149 9018101 -1997 UNITED ST9 MFGR#141 21838850 -1997 UNITED ST9 MFGR#1410 8455387 -1997 UNITED ST9 MFGR#1411 27984689 -1997 UNITED ST9 MFGR#1412 26491401 -1997 UNITED ST9 MFGR#1413 30237705 -1997 UNITED ST9 MFGR#1414 8957476 -1997 UNITED ST9 MFGR#1415 35391583 -1997 UNITED ST9 MFGR#1416 8923782 -1997 UNITED ST9 MFGR#1417 3210984 -1997 UNITED ST9 MFGR#1418 12102607 -1997 UNITED ST9 MFGR#1419 14468627 -1997 UNITED ST9 MFGR#142 33363582 -1997 UNITED ST9 MFGR#1420 13139731 -1997 UNITED ST9 MFGR#1421 8410650 -1997 UNITED ST9 MFGR#1422 11268410 -1997 UNITED ST9 MFGR#1423 2846614 -1997 UNITED ST9 MFGR#1424 2305872 -1997 UNITED ST9 MFGR#1425 4896060 -1997 UNITED ST9 MFGR#1426 31544772 -1997 UNITED ST9 MFGR#1427 4113085 -1997 UNITED ST9 MFGR#1428 11137081 -1997 UNITED ST9 MFGR#1429 10633230 -1997 UNITED ST9 MFGR#143 11438203 -1997 UNITED ST9 MFGR#1430 2727389 -1997 UNITED ST9 MFGR#1431 10548538 -1997 UNITED ST9 MFGR#1433 26564495 -1997 UNITED ST9 MFGR#1434 11948206 -1997 UNITED ST9 MFGR#1435 2201909 -1997 UNITED ST9 MFGR#1436 10347631 -1997 UNITED ST9 MFGR#1437 40051690 -1997 UNITED ST9 MFGR#1438 2662945 -1997 UNITED ST9 MFGR#1439 18011455 -1997 UNITED ST9 MFGR#144 10202784 -1997 UNITED ST9 MFGR#1440 11123196 -1997 UNITED ST9 MFGR#145 11326624 -1997 UNITED ST9 MFGR#146 27825456 -1997 UNITED ST9 MFGR#147 17579502 -1997 UNITED ST9 MFGR#148 17001116 -1998 UNITED ST0 MFGR#141 12484786 -1998 UNITED ST0 MFGR#1410 5225999 -1998 UNITED ST0 MFGR#1411 2633395 -1998 UNITED ST0 MFGR#1412 9479536 -1998 UNITED ST0 MFGR#1413 18468711 -1998 UNITED ST0 MFGR#1414 19440601 -1998 UNITED ST0 MFGR#1415 16894643 -1998 UNITED ST0 MFGR#1416 7520117 -1998 UNITED ST0 MFGR#1417 464576 -1998 UNITED ST0 MFGR#1418 5171747 -1998 UNITED ST0 MFGR#1419 5970985 -1998 UNITED ST0 MFGR#142 23830409 -1998 UNITED ST0 MFGR#1420 2568835 -1998 UNITED ST0 MFGR#1421 6096442 -1998 UNITED ST0 MFGR#1422 10099456 -1998 UNITED ST0 MFGR#1423 24617746 -1998 UNITED ST0 MFGR#1424 22853888 -1998 UNITED ST0 MFGR#1425 25622112 -1998 UNITED ST0 MFGR#1426 9630701 -1998 UNITED ST0 MFGR#1427 4952625 -1998 UNITED ST0 MFGR#1428 4675471 -1998 UNITED ST0 MFGR#1429 19713527 -1998 UNITED ST0 MFGR#143 4971265 -1998 UNITED ST0 MFGR#1430 13948705 -1998 UNITED ST0 MFGR#1431 2955181 -1998 UNITED ST0 MFGR#1432 17946759 -1998 UNITED ST0 MFGR#1433 23325027 -1998 UNITED ST0 MFGR#1434 8580738 -1998 UNITED ST0 MFGR#1435 6663914 -1998 UNITED ST0 MFGR#1436 13468582 -1998 UNITED ST0 MFGR#1437 9419979 -1998 UNITED ST0 MFGR#1438 16164522 -1998 UNITED ST0 MFGR#1439 8692866 -1998 UNITED ST0 MFGR#144 17338711 -1998 UNITED ST0 MFGR#1440 8513736 -1998 UNITED ST0 MFGR#145 6212569 -1998 UNITED ST0 MFGR#146 2702349 -1998 UNITED ST0 MFGR#149 8844540 -1998 UNITED ST1 MFGR#141 13449110 -1998 UNITED ST1 MFGR#1410 5224626 -1998 UNITED ST1 MFGR#1411 16046399 -1998 UNITED ST1 MFGR#1412 8424392 -1998 UNITED ST1 MFGR#1413 7634982 -1998 UNITED ST1 MFGR#1414 8711782 -1998 UNITED ST1 MFGR#1415 5991201 -1998 UNITED ST1 MFGR#1416 11059645 -1998 UNITED ST1 MFGR#1417 8108087 -1998 UNITED ST1 MFGR#1418 1710875 -1998 UNITED ST1 MFGR#1419 452298 -1998 UNITED ST1 MFGR#142 6186526 -1998 UNITED ST1 MFGR#1421 4766963 -1998 UNITED ST1 MFGR#1422 5556563 -1998 UNITED ST1 MFGR#1423 12110257 -1998 UNITED ST1 MFGR#1424 8924948 -1998 UNITED ST1 MFGR#1425 5684710 -1998 UNITED ST1 MFGR#1426 6844444 -1998 UNITED ST1 MFGR#1427 7625525 -1998 UNITED ST1 MFGR#1428 5227395 -1998 UNITED ST1 MFGR#1429 2855272 -1998 UNITED ST1 MFGR#143 6964811 -1998 UNITED ST1 MFGR#1430 16952380 -1998 UNITED ST1 MFGR#1431 7655883 -1998 UNITED ST1 MFGR#1432 2975960 -1998 UNITED ST1 MFGR#1433 10175594 -1998 UNITED ST1 MFGR#1434 26358848 -1998 UNITED ST1 MFGR#1435 11408623 -1998 UNITED ST1 MFGR#1436 4438242 -1998 UNITED ST1 MFGR#1437 10014906 -1998 UNITED ST1 MFGR#1438 18817171 -1998 UNITED ST1 MFGR#1439 2129973 -1998 UNITED ST1 MFGR#144 9512727 -1998 UNITED ST1 MFGR#1440 20185179 -1998 UNITED ST1 MFGR#145 823605 -1998 UNITED ST1 MFGR#146 9358026 -1998 UNITED ST1 MFGR#147 6552312 -1998 UNITED ST1 MFGR#148 9144461 -1998 UNITED ST1 MFGR#149 2149639 -1998 UNITED ST2 MFGR#1410 8044368 -1998 UNITED ST2 MFGR#1413 727343 -1998 UNITED ST2 MFGR#1414 251088 -1998 UNITED ST2 MFGR#1416 5224696 -1998 UNITED ST2 MFGR#1419 7917142 -1998 UNITED ST2 MFGR#1420 9848986 -1998 UNITED ST2 MFGR#1427 4816976 -1998 UNITED ST2 MFGR#1429 4196069 -1998 UNITED ST2 MFGR#143 10073044 -1998 UNITED ST2 MFGR#1431 877123 -1998 UNITED ST2 MFGR#1433 3740836 -1998 UNITED ST2 MFGR#1434 857691 -1998 UNITED ST2 MFGR#1435 2277870 -1998 UNITED ST2 MFGR#145 4906521 -1998 UNITED ST2 MFGR#149 842348 -1998 UNITED ST3 MFGR#141 4181582 -1998 UNITED ST3 MFGR#1410 3169761 -1998 UNITED ST3 MFGR#1412 6590440 -1998 UNITED ST3 MFGR#1413 2015369 -1998 UNITED ST3 MFGR#1414 4408466 -1998 UNITED ST3 MFGR#1415 3963826 -1998 UNITED ST3 MFGR#1416 3179540 -1998 UNITED ST3 MFGR#1417 11440999 -1998 UNITED ST3 MFGR#1418 8725751 -1998 UNITED ST3 MFGR#1419 10646990 -1998 UNITED ST3 MFGR#142 5056172 -1998 UNITED ST3 MFGR#1420 3583150 -1998 UNITED ST3 MFGR#1421 7085689 -1998 UNITED ST3 MFGR#1422 20858904 -1998 UNITED ST3 MFGR#1423 12825743 -1998 UNITED ST3 MFGR#1424 3533518 -1998 UNITED ST3 MFGR#1425 4938324 -1998 UNITED ST3 MFGR#1426 15164664 -1998 UNITED ST3 MFGR#1427 2619162 -1998 UNITED ST3 MFGR#1428 7723699 -1998 UNITED ST3 MFGR#143 24362893 -1998 UNITED ST3 MFGR#1430 7139650 -1998 UNITED ST3 MFGR#1431 10306394 -1998 UNITED ST3 MFGR#1432 14769004 -1998 UNITED ST3 MFGR#1433 9610545 -1998 UNITED ST3 MFGR#1435 6611760 -1998 UNITED ST3 MFGR#1436 941348 -1998 UNITED ST3 MFGR#1437 4802162 -1998 UNITED ST3 MFGR#1438 5079750 -1998 UNITED ST3 MFGR#1439 5905118 -1998 UNITED ST3 MFGR#144 5436839 -1998 UNITED ST3 MFGR#1440 9815517 -1998 UNITED ST3 MFGR#145 7580271 -1998 UNITED ST3 MFGR#146 14051404 -1998 UNITED ST3 MFGR#147 7786017 -1998 UNITED ST3 MFGR#149 1271813 -1998 UNITED ST4 MFGR#141 26703183 -1998 UNITED ST4 MFGR#1410 18094129 -1998 UNITED ST4 MFGR#1411 615999 -1998 UNITED ST4 MFGR#1412 13988709 -1998 UNITED ST4 MFGR#1413 10115647 -1998 UNITED ST4 MFGR#1414 1586551 -1998 UNITED ST4 MFGR#1415 14357244 -1998 UNITED ST4 MFGR#1416 9430036 -1998 UNITED ST4 MFGR#1417 17361773 -1998 UNITED ST4 MFGR#1418 10187338 -1998 UNITED ST4 MFGR#1419 7116094 -1998 UNITED ST4 MFGR#142 18454502 -1998 UNITED ST4 MFGR#1420 1621340 -1998 UNITED ST4 MFGR#1422 13861291 -1998 UNITED ST4 MFGR#1423 4208006 -1998 UNITED ST4 MFGR#1424 3242576 -1998 UNITED ST4 MFGR#1425 1470642 -1998 UNITED ST4 MFGR#1426 11987205 -1998 UNITED ST4 MFGR#1427 12975399 -1998 UNITED ST4 MFGR#1428 11756155 -1998 UNITED ST4 MFGR#1429 6956191 -1998 UNITED ST4 MFGR#143 11499476 -1998 UNITED ST4 MFGR#1430 4355096 -1998 UNITED ST4 MFGR#1431 20166612 -1998 UNITED ST4 MFGR#1432 8849789 -1998 UNITED ST4 MFGR#1433 10786037 -1998 UNITED ST4 MFGR#1435 13208151 -1998 UNITED ST4 MFGR#1436 4747808 -1998 UNITED ST4 MFGR#1437 12999351 -1998 UNITED ST4 MFGR#1438 17360787 -1998 UNITED ST4 MFGR#1439 2557841 -1998 UNITED ST4 MFGR#144 22030911 -1998 UNITED ST4 MFGR#145 5272280 -1998 UNITED ST4 MFGR#146 4307958 -1998 UNITED ST4 MFGR#147 7995050 -1998 UNITED ST4 MFGR#148 2348889 -1998 UNITED ST5 MFGR#141 8396190 -1998 UNITED ST5 MFGR#1410 17422889 -1998 UNITED ST5 MFGR#1411 764246 -1998 UNITED ST5 MFGR#1413 8051561 -1998 UNITED ST5 MFGR#1414 1500867 -1998 UNITED ST5 MFGR#1416 3874315 -1998 UNITED ST5 MFGR#1417 7550747 -1998 UNITED ST5 MFGR#1418 14241755 -1998 UNITED ST5 MFGR#1419 4893846 -1998 UNITED ST5 MFGR#142 5418299 -1998 UNITED ST5 MFGR#1420 3810727 -1998 UNITED ST5 MFGR#1421 3321336 -1998 UNITED ST5 MFGR#1423 5021100 -1998 UNITED ST5 MFGR#1424 6125645 -1998 UNITED ST5 MFGR#1426 880680 -1998 UNITED ST5 MFGR#1427 1284890 -1998 UNITED ST5 MFGR#1428 3813108 -1998 UNITED ST5 MFGR#1429 9429868 -1998 UNITED ST5 MFGR#143 15699981 -1998 UNITED ST5 MFGR#1430 7977654 -1998 UNITED ST5 MFGR#1431 5545969 -1998 UNITED ST5 MFGR#1432 1803434 -1998 UNITED ST5 MFGR#1435 12957056 -1998 UNITED ST5 MFGR#1436 20666755 -1998 UNITED ST5 MFGR#1437 9536733 -1998 UNITED ST5 MFGR#1439 11784214 -1998 UNITED ST5 MFGR#1440 613089 -1998 UNITED ST5 MFGR#145 8237944 -1998 UNITED ST5 MFGR#147 5370351 -1998 UNITED ST6 MFGR#141 6683977 -1998 UNITED ST6 MFGR#1411 11589040 -1998 UNITED ST6 MFGR#1412 4800954 -1998 UNITED ST6 MFGR#1416 5848432 -1998 UNITED ST6 MFGR#1417 1804324 -1998 UNITED ST6 MFGR#1418 4642727 -1998 UNITED ST6 MFGR#1419 10079621 -1998 UNITED ST6 MFGR#142 2533570 -1998 UNITED ST6 MFGR#1420 9016258 -1998 UNITED ST6 MFGR#1421 7636150 -1998 UNITED ST6 MFGR#1422 1272436 -1998 UNITED ST6 MFGR#1423 9600097 -1998 UNITED ST6 MFGR#1424 7036762 -1998 UNITED ST6 MFGR#1425 10867426 -1998 UNITED ST6 MFGR#1426 7033994 -1998 UNITED ST6 MFGR#1427 6481679 -1998 UNITED ST6 MFGR#1428 6269185 -1998 UNITED ST6 MFGR#1429 12055631 -1998 UNITED ST6 MFGR#143 6500459 -1998 UNITED ST6 MFGR#1430 6816243 -1998 UNITED ST6 MFGR#1432 7752973 -1998 UNITED ST6 MFGR#1436 6292813 -1998 UNITED ST6 MFGR#1437 2334894 -1998 UNITED ST6 MFGR#1438 4459455 -1998 UNITED ST6 MFGR#144 640220 -1998 UNITED ST6 MFGR#1440 748200 -1998 UNITED ST6 MFGR#145 7948794 -1998 UNITED ST6 MFGR#146 2734333 -1998 UNITED ST6 MFGR#147 2995632 -1998 UNITED ST6 MFGR#148 10695287 -1998 UNITED ST6 MFGR#149 1475864 -1998 UNITED ST7 MFGR#141 4816637 -1998 UNITED ST7 MFGR#1410 24393912 -1998 UNITED ST7 MFGR#1411 1283038 -1998 UNITED ST7 MFGR#1412 12607459 -1998 UNITED ST7 MFGR#1413 15421293 -1998 UNITED ST7 MFGR#1414 7681668 -1998 UNITED ST7 MFGR#1415 7138377 -1998 UNITED ST7 MFGR#1416 9246990 -1998 UNITED ST7 MFGR#1417 5242972 -1998 UNITED ST7 MFGR#1419 13548941 -1998 UNITED ST7 MFGR#142 17758762 -1998 UNITED ST7 MFGR#1420 11980437 -1998 UNITED ST7 MFGR#1421 11166952 -1998 UNITED ST7 MFGR#1422 9231883 -1998 UNITED ST7 MFGR#1424 5796389 -1998 UNITED ST7 MFGR#1425 4943582 -1998 UNITED ST7 MFGR#1426 16361100 -1998 UNITED ST7 MFGR#1427 9659296 -1998 UNITED ST7 MFGR#1428 2998020 -1998 UNITED ST7 MFGR#1429 5766768 -1998 UNITED ST7 MFGR#143 5526167 -1998 UNITED ST7 MFGR#1430 4191892 -1998 UNITED ST7 MFGR#1431 12979559 -1998 UNITED ST7 MFGR#1432 5832068 -1998 UNITED ST7 MFGR#1433 11410227 -1998 UNITED ST7 MFGR#1434 10525792 -1998 UNITED ST7 MFGR#1435 5982420 -1998 UNITED ST7 MFGR#1436 7142934 -1998 UNITED ST7 MFGR#1437 7499671 -1998 UNITED ST7 MFGR#1438 9177564 -1998 UNITED ST7 MFGR#1439 1974444 -1998 UNITED ST7 MFGR#144 2428463 -1998 UNITED ST7 MFGR#1440 3300838 -1998 UNITED ST7 MFGR#146 1771645 -1998 UNITED ST7 MFGR#147 4681835 -1998 UNITED ST7 MFGR#148 7124367 -1998 UNITED ST7 MFGR#149 11662970 -1998 UNITED ST8 MFGR#141 12753354 -1998 UNITED ST8 MFGR#1410 1097292 -1998 UNITED ST8 MFGR#1411 4069881 -1998 UNITED ST8 MFGR#1414 9474831 -1998 UNITED ST8 MFGR#1415 6406304 -1998 UNITED ST8 MFGR#1418 2392133 -1998 UNITED ST8 MFGR#1419 2953230 -1998 UNITED ST8 MFGR#142 10702563 -1998 UNITED ST8 MFGR#1420 3719432 -1998 UNITED ST8 MFGR#1421 3854657 -1998 UNITED ST8 MFGR#1422 9711968 -1998 UNITED ST8 MFGR#1424 1102516 -1998 UNITED ST8 MFGR#1426 8937532 -1998 UNITED ST8 MFGR#1427 5255637 -1998 UNITED ST8 MFGR#1428 1695112 -1998 UNITED ST8 MFGR#143 6231349 -1998 UNITED ST8 MFGR#1430 4678664 -1998 UNITED ST8 MFGR#1431 2459741 -1998 UNITED ST8 MFGR#1433 1724339 -1998 UNITED ST8 MFGR#1434 8222328 -1998 UNITED ST8 MFGR#1435 4087165 -1998 UNITED ST8 MFGR#1437 7379288 -1998 UNITED ST8 MFGR#1439 1947163 -1998 UNITED ST8 MFGR#144 21846266 -1998 UNITED ST8 MFGR#146 13768113 -1998 UNITED ST8 MFGR#147 5279643 -1998 UNITED ST8 MFGR#149 15221410 -1998 UNITED ST9 MFGR#141 5544000 -1998 UNITED ST9 MFGR#1410 7971104 -1998 UNITED ST9 MFGR#1411 26426609 -1998 UNITED ST9 MFGR#1412 1701777 -1998 UNITED ST9 MFGR#1413 18786672 -1998 UNITED ST9 MFGR#1414 15787780 -1998 UNITED ST9 MFGR#1415 11214818 -1998 UNITED ST9 MFGR#1416 2076120 -1998 UNITED ST9 MFGR#1417 14310334 -1998 UNITED ST9 MFGR#1418 3558355 -1998 UNITED ST9 MFGR#1419 6134867 -1998 UNITED ST9 MFGR#142 257897 -1998 UNITED ST9 MFGR#1420 7716338 -1998 UNITED ST9 MFGR#1421 10291775 -1998 UNITED ST9 MFGR#1422 6105716 -1998 UNITED ST9 MFGR#1423 11378613 -1998 UNITED ST9 MFGR#1424 1904550 -1998 UNITED ST9 MFGR#1425 18831147 -1998 UNITED ST9 MFGR#1426 18562642 -1998 UNITED ST9 MFGR#1427 12565890 -1998 UNITED ST9 MFGR#1428 10720914 -1998 UNITED ST9 MFGR#1429 21940718 -1998 UNITED ST9 MFGR#143 21480722 -1998 UNITED ST9 MFGR#1430 30935173 -1998 UNITED ST9 MFGR#1431 7793012 -1998 UNITED ST9 MFGR#1432 10026104 -1998 UNITED ST9 MFGR#1433 17874435 -1998 UNITED ST9 MFGR#1434 5493785 -1998 UNITED ST9 MFGR#1435 10738769 -1998 UNITED ST9 MFGR#1436 43504 -1998 UNITED ST9 MFGR#1437 13559646 -1998 UNITED ST9 MFGR#1438 12654258 -1998 UNITED ST9 MFGR#1439 5972923 -1998 UNITED ST9 MFGR#144 15702123 -1998 UNITED ST9 MFGR#1440 6681029 -1998 UNITED ST9 MFGR#145 920718 -1998 UNITED ST9 MFGR#146 10435931 -1998 UNITED ST9 MFGR#147 4759326 -1998 UNITED ST9 MFGR#148 9970848 -1998 UNITED ST9 MFGR#149 7510940 +1992 UNITED ST0 MFGR#141 555641 +1992 UNITED ST0 MFGR#1411 7079182 +1992 UNITED ST0 MFGR#1412 2872721 +1992 UNITED ST0 MFGR#1413 5471501 +1992 UNITED ST0 MFGR#1414 6180491 +1992 UNITED ST0 MFGR#1415 3832532 +1992 UNITED ST0 MFGR#1417 5090445 +1992 UNITED ST0 MFGR#142 4603254 +1992 UNITED ST0 MFGR#1421 717404 +1992 UNITED ST0 MFGR#1422 6204274 +1992 UNITED ST0 MFGR#1424 5518896 +1992 UNITED ST0 MFGR#1425 5763764 +1992 UNITED ST0 MFGR#1427 6180274 +1992 UNITED ST0 MFGR#1432 6428730 +1992 UNITED ST0 MFGR#1433 833533 +1992 UNITED ST0 MFGR#1434 13713612 +1992 UNITED ST0 MFGR#145 13934235 +1992 UNITED ST2 MFGR#1411 758915 +1992 UNITED ST2 MFGR#142 4221010 +1992 UNITED ST2 MFGR#1421 4315917 +1992 UNITED ST2 MFGR#1424 57589 +1992 UNITED ST2 MFGR#1427 1685113 +1992 UNITED ST2 MFGR#143 1865250 +1992 UNITED ST2 MFGR#1430 6454140 +1992 UNITED ST3 MFGR#141 10985538 +1992 UNITED ST3 MFGR#1410 5661337 +1992 UNITED ST3 MFGR#1411 2913347 +1992 UNITED ST3 MFGR#1412 3456611 +1992 UNITED ST3 MFGR#1413 1843964 +1992 UNITED ST3 MFGR#1415 741657 +1992 UNITED ST3 MFGR#1416 12592059 +1992 UNITED ST3 MFGR#1417 343888 +1992 UNITED ST3 MFGR#1419 2221452 +1992 UNITED ST3 MFGR#142 579832 +1992 UNITED ST3 MFGR#1421 2176380 +1992 UNITED ST3 MFGR#1422 1841100 +1992 UNITED ST3 MFGR#1423 121681 +1992 UNITED ST3 MFGR#1425 3025877 +1992 UNITED ST3 MFGR#1426 2812288 +1992 UNITED ST3 MFGR#1427 7624182 +1992 UNITED ST3 MFGR#1428 9484717 +1992 UNITED ST3 MFGR#1429 13290434 +1992 UNITED ST3 MFGR#1430 5263185 +1992 UNITED ST3 MFGR#1431 1057990 +1992 UNITED ST3 MFGR#1433 3172704 +1992 UNITED ST3 MFGR#1436 3292838 +1992 UNITED ST3 MFGR#1438 10257150 +1992 UNITED ST3 MFGR#144 4992624 +1992 UNITED ST3 MFGR#145 4555258 +1992 UNITED ST3 MFGR#146 5155417 +1992 UNITED ST3 MFGR#147 6044237 +1992 UNITED ST3 MFGR#148 12453066 +1992 UNITED ST3 MFGR#149 4283724 +1992 UNITED ST6 MFGR#1413 4420349 +1992 UNITED ST6 MFGR#1415 6201665 +1992 UNITED ST6 MFGR#1416 2588026 +1992 UNITED ST6 MFGR#142 55047 +1992 UNITED ST6 MFGR#1420 1288451 +1992 UNITED ST6 MFGR#1421 2567818 +1992 UNITED ST6 MFGR#1423 583540 +1992 UNITED ST6 MFGR#1426 6446245 +1992 UNITED ST6 MFGR#1429 1693359 +1992 UNITED ST6 MFGR#1430 954778 +1992 UNITED ST6 MFGR#1436 2419392 +1992 UNITED ST6 MFGR#1437 7302587 +1992 UNITED ST6 MFGR#144 8794123 +1992 UNITED ST6 MFGR#1440 3993789 +1992 UNITED ST7 MFGR#141 6582861 +1992 UNITED ST7 MFGR#1411 692197 +1992 UNITED ST7 MFGR#1412 169740 +1992 UNITED ST7 MFGR#1413 4359125 +1992 UNITED ST7 MFGR#1420 2500664 +1992 UNITED ST7 MFGR#1424 5152200 +1992 UNITED ST7 MFGR#1426 8490217 +1992 UNITED ST7 MFGR#143 5620388 +1992 UNITED ST7 MFGR#1431 1235687 +1992 UNITED ST7 MFGR#1432 6829944 +1992 UNITED ST7 MFGR#1440 6253049 +1992 UNITED ST7 MFGR#145 3732260 +1992 UNITED ST7 MFGR#147 2421233 +1992 UNITED ST7 MFGR#149 1866816 +1992 UNITED ST9 MFGR#141 147955 +1992 UNITED ST9 MFGR#1413 5392214 +1992 UNITED ST9 MFGR#1414 6319659 +1992 UNITED ST9 MFGR#1415 1084942 +1992 UNITED ST9 MFGR#1416 9714488 +1992 UNITED ST9 MFGR#1421 5856389 +1992 UNITED ST9 MFGR#1422 5439287 +1992 UNITED ST9 MFGR#1423 7040305 +1992 UNITED ST9 MFGR#1426 6595331 +1992 UNITED ST9 MFGR#1428 446309 +1992 UNITED ST9 MFGR#1429 3993195 +1992 UNITED ST9 MFGR#143 2320022 +1992 UNITED ST9 MFGR#1433 8938527 +1992 UNITED ST9 MFGR#1434 3280337 +1992 UNITED ST9 MFGR#1435 11735187 +1992 UNITED ST9 MFGR#1438 2940085 +1992 UNITED ST9 MFGR#145 2145463 +1992 UNITED ST9 MFGR#147 5798259 +1992 UNITED ST9 MFGR#148 4292580 -- !query4_3_after -- -1997 UNITED ST0 MFGR#141 15456288 -1997 UNITED ST0 MFGR#1410 32961113 -1997 UNITED ST0 MFGR#1411 36166400 -1997 UNITED ST0 MFGR#1412 18237451 -1997 UNITED ST0 MFGR#1413 40752813 -1997 UNITED ST0 MFGR#1414 24237588 -1997 UNITED ST0 MFGR#1415 6419001 -1997 UNITED ST0 MFGR#1416 15639876 -1997 UNITED ST0 MFGR#1417 15476211 -1997 UNITED ST0 MFGR#1418 8169956 -1997 UNITED ST0 MFGR#1419 18217252 -1997 UNITED ST0 MFGR#142 29361222 -1997 UNITED ST0 MFGR#1420 34587603 -1997 UNITED ST0 MFGR#1421 28560921 -1997 UNITED ST0 MFGR#1422 25154181 -1997 UNITED ST0 MFGR#1423 18144008 -1997 UNITED ST0 MFGR#1424 28064267 -1997 UNITED ST0 MFGR#1425 15255845 -1997 UNITED ST0 MFGR#1426 5610353 -1997 UNITED ST0 MFGR#1427 5750693 -1997 UNITED ST0 MFGR#1428 14081707 -1997 UNITED ST0 MFGR#1429 9024512 -1997 UNITED ST0 MFGR#143 5325193 -1997 UNITED ST0 MFGR#1430 16961739 -1997 UNITED ST0 MFGR#1431 19015253 -1997 UNITED ST0 MFGR#1432 6324917 -1997 UNITED ST0 MFGR#1433 6734461 -1997 UNITED ST0 MFGR#1434 19206393 -1997 UNITED ST0 MFGR#1435 679925 -1997 UNITED ST0 MFGR#1436 13834012 -1997 UNITED ST0 MFGR#1437 23575407 -1997 UNITED ST0 MFGR#1438 24756074 -1997 UNITED ST0 MFGR#1439 19644072 -1997 UNITED ST0 MFGR#144 20285157 -1997 UNITED ST0 MFGR#1440 14068558 -1997 UNITED ST0 MFGR#145 22646138 -1997 UNITED ST0 MFGR#146 16381159 -1997 UNITED ST0 MFGR#147 12267827 -1997 UNITED ST0 MFGR#148 30968614 -1997 UNITED ST0 MFGR#149 8640669 -1997 UNITED ST1 MFGR#141 10152087 -1997 UNITED ST1 MFGR#1410 32583300 -1997 UNITED ST1 MFGR#1411 15690352 -1997 UNITED ST1 MFGR#1412 16671403 -1997 UNITED ST1 MFGR#1413 4942775 -1997 UNITED ST1 MFGR#1414 9391597 -1997 UNITED ST1 MFGR#1415 26937296 -1997 UNITED ST1 MFGR#1416 25782871 -1997 UNITED ST1 MFGR#1417 23349641 -1997 UNITED ST1 MFGR#1418 10934620 -1997 UNITED ST1 MFGR#1419 22673030 -1997 UNITED ST1 MFGR#142 14048276 -1997 UNITED ST1 MFGR#1420 18100795 -1997 UNITED ST1 MFGR#1421 25405678 -1997 UNITED ST1 MFGR#1423 5713800 -1997 UNITED ST1 MFGR#1424 4131923 -1997 UNITED ST1 MFGR#1425 17190425 -1997 UNITED ST1 MFGR#1426 22314044 -1997 UNITED ST1 MFGR#1427 13635057 -1997 UNITED ST1 MFGR#1428 17735811 -1997 UNITED ST1 MFGR#1429 28740911 -1997 UNITED ST1 MFGR#143 27418033 -1997 UNITED ST1 MFGR#1430 18655292 -1997 UNITED ST1 MFGR#1431 11074399 -1997 UNITED ST1 MFGR#1432 7490170 -1997 UNITED ST1 MFGR#1433 28997266 -1997 UNITED ST1 MFGR#1434 4612211 -1997 UNITED ST1 MFGR#1435 16327124 -1997 UNITED ST1 MFGR#1436 17409665 -1997 UNITED ST1 MFGR#1437 33391635 -1997 UNITED ST1 MFGR#1438 37321134 -1997 UNITED ST1 MFGR#1439 10279941 -1997 UNITED ST1 MFGR#144 12304723 -1997 UNITED ST1 MFGR#1440 28921230 -1997 UNITED ST1 MFGR#145 17933684 -1997 UNITED ST1 MFGR#146 8709103 -1997 UNITED ST1 MFGR#147 20145983 -1997 UNITED ST1 MFGR#148 16940400 -1997 UNITED ST1 MFGR#149 10284949 -1997 UNITED ST2 MFGR#1412 6328240 -1997 UNITED ST2 MFGR#1415 5022682 -1997 UNITED ST2 MFGR#1416 4102214 -1997 UNITED ST2 MFGR#1417 5284444 -1997 UNITED ST2 MFGR#1418 2649908 -1997 UNITED ST2 MFGR#1419 4664673 -1997 UNITED ST2 MFGR#142 10784835 -1997 UNITED ST2 MFGR#1423 4095190 -1997 UNITED ST2 MFGR#1424 5376355 -1997 UNITED ST2 MFGR#1425 3949957 -1997 UNITED ST2 MFGR#143 2859829 -1997 UNITED ST2 MFGR#1430 4618266 -1997 UNITED ST2 MFGR#1432 5183060 -1997 UNITED ST2 MFGR#1433 10531371 -1997 UNITED ST2 MFGR#1434 77646 -1997 UNITED ST2 MFGR#1436 6538508 -1997 UNITED ST2 MFGR#1438 12324747 -1997 UNITED ST2 MFGR#144 2002870 -1997 UNITED ST2 MFGR#1440 12247229 -1997 UNITED ST2 MFGR#145 202685 -1997 UNITED ST2 MFGR#148 5998932 -1997 UNITED ST3 MFGR#141 12866083 -1997 UNITED ST3 MFGR#1410 31051703 -1997 UNITED ST3 MFGR#1411 14099543 -1997 UNITED ST3 MFGR#1412 4970551 -1997 UNITED ST3 MFGR#1413 26129762 -1997 UNITED ST3 MFGR#1414 22364787 -1997 UNITED ST3 MFGR#1415 27883428 -1997 UNITED ST3 MFGR#1416 25289630 -1997 UNITED ST3 MFGR#1417 5524810 -1997 UNITED ST3 MFGR#1419 3616608 -1997 UNITED ST3 MFGR#142 10776059 -1997 UNITED ST3 MFGR#1420 12958851 -1997 UNITED ST3 MFGR#1421 5815494 -1997 UNITED ST3 MFGR#1422 17694270 -1997 UNITED ST3 MFGR#1423 10285142 -1997 UNITED ST3 MFGR#1424 21555317 -1997 UNITED ST3 MFGR#1425 5214517 -1997 UNITED ST3 MFGR#1426 16810700 -1997 UNITED ST3 MFGR#1427 10086430 -1997 UNITED ST3 MFGR#1428 7733716 -1997 UNITED ST3 MFGR#1429 11922430 -1997 UNITED ST3 MFGR#1430 20105292 -1997 UNITED ST3 MFGR#1431 14716979 -1997 UNITED ST3 MFGR#1432 15779247 -1997 UNITED ST3 MFGR#1433 15308773 -1997 UNITED ST3 MFGR#1434 8515037 -1997 UNITED ST3 MFGR#1435 25486899 -1997 UNITED ST3 MFGR#1436 12739448 -1997 UNITED ST3 MFGR#1437 11249180 -1997 UNITED ST3 MFGR#1438 4156824 -1997 UNITED ST3 MFGR#1439 18358277 -1997 UNITED ST3 MFGR#144 10097718 -1997 UNITED ST3 MFGR#1440 22920494 -1997 UNITED ST3 MFGR#145 20285859 -1997 UNITED ST3 MFGR#146 18406423 -1997 UNITED ST3 MFGR#147 17295990 -1997 UNITED ST3 MFGR#148 26127145 -1997 UNITED ST3 MFGR#149 19895538 -1997 UNITED ST4 MFGR#141 8135477 -1997 UNITED ST4 MFGR#1410 5503247 -1997 UNITED ST4 MFGR#1411 6096435 -1997 UNITED ST4 MFGR#1412 19976043 -1997 UNITED ST4 MFGR#1413 22581282 -1997 UNITED ST4 MFGR#1414 10101379 -1997 UNITED ST4 MFGR#1415 8877444 -1997 UNITED ST4 MFGR#1416 17943398 -1997 UNITED ST4 MFGR#1417 9224501 -1997 UNITED ST4 MFGR#1418 16255176 -1997 UNITED ST4 MFGR#1419 683069 -1997 UNITED ST4 MFGR#142 23512944 -1997 UNITED ST4 MFGR#1420 27954262 -1997 UNITED ST4 MFGR#1421 54209185 -1997 UNITED ST4 MFGR#1422 1173957 -1997 UNITED ST4 MFGR#1423 20410710 -1997 UNITED ST4 MFGR#1424 2889270 -1997 UNITED ST4 MFGR#1425 13361796 -1997 UNITED ST4 MFGR#1426 18678819 -1997 UNITED ST4 MFGR#1427 18206066 -1997 UNITED ST4 MFGR#1428 18153974 -1997 UNITED ST4 MFGR#1429 12545076 -1997 UNITED ST4 MFGR#143 6158834 -1997 UNITED ST4 MFGR#1430 16128370 -1997 UNITED ST4 MFGR#1431 11153717 -1997 UNITED ST4 MFGR#1432 7745636 -1997 UNITED ST4 MFGR#1433 15983634 -1997 UNITED ST4 MFGR#1434 3662246 -1997 UNITED ST4 MFGR#1435 3060180 -1997 UNITED ST4 MFGR#1436 11524592 -1997 UNITED ST4 MFGR#1437 36824174 -1997 UNITED ST4 MFGR#1438 8226666 -1997 UNITED ST4 MFGR#1439 9872470 -1997 UNITED ST4 MFGR#1440 401207 -1997 UNITED ST4 MFGR#145 11698312 -1997 UNITED ST4 MFGR#146 18259956 -1997 UNITED ST4 MFGR#147 10256110 -1997 UNITED ST4 MFGR#148 15892628 -1997 UNITED ST4 MFGR#149 4322491 -1997 UNITED ST5 MFGR#141 8534275 -1997 UNITED ST5 MFGR#1411 2427261 -1997 UNITED ST5 MFGR#1412 9032422 -1997 UNITED ST5 MFGR#1413 6343588 -1997 UNITED ST5 MFGR#1414 2961025 -1997 UNITED ST5 MFGR#1415 21186568 -1997 UNITED ST5 MFGR#1416 9354080 -1997 UNITED ST5 MFGR#1417 11786246 -1997 UNITED ST5 MFGR#1418 15297245 -1997 UNITED ST5 MFGR#1420 15787608 -1997 UNITED ST5 MFGR#1421 17858301 -1997 UNITED ST5 MFGR#1422 14080987 -1997 UNITED ST5 MFGR#1423 12535186 -1997 UNITED ST5 MFGR#1424 3221960 -1997 UNITED ST5 MFGR#1425 8950317 -1997 UNITED ST5 MFGR#1426 13161486 -1997 UNITED ST5 MFGR#1427 6149737 -1997 UNITED ST5 MFGR#1428 6796520 -1997 UNITED ST5 MFGR#1429 26026300 -1997 UNITED ST5 MFGR#143 6712572 -1997 UNITED ST5 MFGR#1430 16092131 -1997 UNITED ST5 MFGR#1431 3425291 -1997 UNITED ST5 MFGR#1432 14630175 -1997 UNITED ST5 MFGR#1433 4967585 -1997 UNITED ST5 MFGR#1434 6716261 -1997 UNITED ST5 MFGR#1435 6290790 -1997 UNITED ST5 MFGR#1436 6787699 -1997 UNITED ST5 MFGR#1437 6045168 -1997 UNITED ST5 MFGR#1438 1082877 -1997 UNITED ST5 MFGR#1439 5268611 -1997 UNITED ST5 MFGR#144 8900840 -1997 UNITED ST5 MFGR#1440 2644336 -1997 UNITED ST5 MFGR#145 18054844 -1997 UNITED ST5 MFGR#146 4893668 -1997 UNITED ST5 MFGR#147 9615891 -1997 UNITED ST5 MFGR#148 1426124 -1997 UNITED ST5 MFGR#149 7320726 -1997 UNITED ST6 MFGR#141 10051337 -1997 UNITED ST6 MFGR#1410 23352963 -1997 UNITED ST6 MFGR#1411 1537303 -1997 UNITED ST6 MFGR#1412 4332646 -1997 UNITED ST6 MFGR#1413 14686752 -1997 UNITED ST6 MFGR#1414 7226409 -1997 UNITED ST6 MFGR#1415 15062686 -1997 UNITED ST6 MFGR#1416 8606480 -1997 UNITED ST6 MFGR#1417 7662761 -1997 UNITED ST6 MFGR#1418 1810860 -1997 UNITED ST6 MFGR#1419 20179568 -1997 UNITED ST6 MFGR#142 15506072 -1997 UNITED ST6 MFGR#1420 3844271 -1997 UNITED ST6 MFGR#1421 10608709 -1997 UNITED ST6 MFGR#1422 16083699 -1997 UNITED ST6 MFGR#1423 6789571 -1997 UNITED ST6 MFGR#1424 17631950 -1997 UNITED ST6 MFGR#1425 8477221 -1997 UNITED ST6 MFGR#1426 16847029 -1997 UNITED ST6 MFGR#1428 17314796 -1997 UNITED ST6 MFGR#1429 19175871 -1997 UNITED ST6 MFGR#143 8775848 -1997 UNITED ST6 MFGR#1431 5319063 -1997 UNITED ST6 MFGR#1432 7875606 -1997 UNITED ST6 MFGR#1433 9305264 -1997 UNITED ST6 MFGR#1434 8050404 -1997 UNITED ST6 MFGR#1435 17585836 -1997 UNITED ST6 MFGR#1436 13745758 -1997 UNITED ST6 MFGR#1438 14536324 -1997 UNITED ST6 MFGR#1439 11451329 -1997 UNITED ST6 MFGR#144 10794837 -1997 UNITED ST6 MFGR#1440 5719238 -1997 UNITED ST6 MFGR#145 25449058 -1997 UNITED ST6 MFGR#146 17821996 -1997 UNITED ST6 MFGR#147 18922333 -1997 UNITED ST6 MFGR#148 10825718 -1997 UNITED ST6 MFGR#149 14950868 -1997 UNITED ST7 MFGR#141 8291476 -1997 UNITED ST7 MFGR#1410 4622953 -1997 UNITED ST7 MFGR#1411 14885507 -1997 UNITED ST7 MFGR#1412 19011375 -1997 UNITED ST7 MFGR#1413 31178370 -1997 UNITED ST7 MFGR#1414 16429613 -1997 UNITED ST7 MFGR#1415 3091832 -1997 UNITED ST7 MFGR#1416 17008243 -1997 UNITED ST7 MFGR#1417 15609283 -1997 UNITED ST7 MFGR#1418 5059394 -1997 UNITED ST7 MFGR#1419 7234721 -1997 UNITED ST7 MFGR#142 6898938 -1997 UNITED ST7 MFGR#1420 6426590 -1997 UNITED ST7 MFGR#1421 6043221 -1997 UNITED ST7 MFGR#1422 13273021 -1997 UNITED ST7 MFGR#1423 30537249 -1997 UNITED ST7 MFGR#1424 9224296 -1997 UNITED ST7 MFGR#1425 12799069 -1997 UNITED ST7 MFGR#1426 30277891 -1997 UNITED ST7 MFGR#1427 8331493 -1997 UNITED ST7 MFGR#1428 16205423 -1997 UNITED ST7 MFGR#1429 14396627 -1997 UNITED ST7 MFGR#143 10209623 -1997 UNITED ST7 MFGR#1430 16556758 -1997 UNITED ST7 MFGR#1431 15775327 -1997 UNITED ST7 MFGR#1432 24602989 -1997 UNITED ST7 MFGR#1433 22927717 -1997 UNITED ST7 MFGR#1434 9486462 -1997 UNITED ST7 MFGR#1435 14254290 -1997 UNITED ST7 MFGR#1436 8645365 -1997 UNITED ST7 MFGR#1437 19026166 -1997 UNITED ST7 MFGR#1438 11291550 -1997 UNITED ST7 MFGR#1439 9590292 -1997 UNITED ST7 MFGR#144 14110404 -1997 UNITED ST7 MFGR#1440 21877355 -1997 UNITED ST7 MFGR#145 8246717 -1997 UNITED ST7 MFGR#146 9785781 -1997 UNITED ST7 MFGR#147 13412691 -1997 UNITED ST7 MFGR#148 14235939 -1997 UNITED ST7 MFGR#149 23450309 -1997 UNITED ST8 MFGR#141 7323416 -1997 UNITED ST8 MFGR#1410 7191961 -1997 UNITED ST8 MFGR#1411 2425849 -1997 UNITED ST8 MFGR#1412 26725641 -1997 UNITED ST8 MFGR#1413 7915896 -1997 UNITED ST8 MFGR#1414 14359044 -1997 UNITED ST8 MFGR#1415 15834927 -1997 UNITED ST8 MFGR#1416 7464470 -1997 UNITED ST8 MFGR#1417 15274496 -1997 UNITED ST8 MFGR#1418 3261391 -1997 UNITED ST8 MFGR#1419 8020386 -1997 UNITED ST8 MFGR#142 5985550 -1997 UNITED ST8 MFGR#1420 9441461 -1997 UNITED ST8 MFGR#1421 2998735 -1997 UNITED ST8 MFGR#1423 1565415 -1997 UNITED ST8 MFGR#1424 5613094 -1997 UNITED ST8 MFGR#1425 7186734 -1997 UNITED ST8 MFGR#1426 3153117 -1997 UNITED ST8 MFGR#1427 11303043 -1997 UNITED ST8 MFGR#1428 3564003 -1997 UNITED ST8 MFGR#143 6357477 -1997 UNITED ST8 MFGR#1430 6633950 -1997 UNITED ST8 MFGR#1431 867285 -1997 UNITED ST8 MFGR#1432 9962884 -1997 UNITED ST8 MFGR#1433 13708011 -1997 UNITED ST8 MFGR#1434 7357067 -1997 UNITED ST8 MFGR#1435 11993453 -1997 UNITED ST8 MFGR#1436 7231938 -1997 UNITED ST8 MFGR#1437 18632056 -1997 UNITED ST8 MFGR#1438 3609384 -1997 UNITED ST8 MFGR#144 4822734 -1997 UNITED ST8 MFGR#1440 2521917 -1997 UNITED ST8 MFGR#145 11683385 -1997 UNITED ST8 MFGR#146 3350415 -1997 UNITED ST8 MFGR#147 8425393 -1997 UNITED ST8 MFGR#148 15005341 -1997 UNITED ST8 MFGR#149 9018101 -1997 UNITED ST9 MFGR#141 21838850 -1997 UNITED ST9 MFGR#1410 8455387 -1997 UNITED ST9 MFGR#1411 27984689 -1997 UNITED ST9 MFGR#1412 26491401 -1997 UNITED ST9 MFGR#1413 30237705 -1997 UNITED ST9 MFGR#1414 8957476 -1997 UNITED ST9 MFGR#1415 35391583 -1997 UNITED ST9 MFGR#1416 8923782 -1997 UNITED ST9 MFGR#1417 3210984 -1997 UNITED ST9 MFGR#1418 12102607 -1997 UNITED ST9 MFGR#1419 14468627 -1997 UNITED ST9 MFGR#142 33363582 -1997 UNITED ST9 MFGR#1420 13139731 -1997 UNITED ST9 MFGR#1421 8410650 -1997 UNITED ST9 MFGR#1422 11268410 -1997 UNITED ST9 MFGR#1423 2846614 -1997 UNITED ST9 MFGR#1424 2305872 -1997 UNITED ST9 MFGR#1425 4896060 -1997 UNITED ST9 MFGR#1426 31544772 -1997 UNITED ST9 MFGR#1427 4113085 -1997 UNITED ST9 MFGR#1428 11137081 -1997 UNITED ST9 MFGR#1429 10633230 -1997 UNITED ST9 MFGR#143 11438203 -1997 UNITED ST9 MFGR#1430 2727389 -1997 UNITED ST9 MFGR#1431 10548538 -1997 UNITED ST9 MFGR#1433 26564495 -1997 UNITED ST9 MFGR#1434 11948206 -1997 UNITED ST9 MFGR#1435 2201909 -1997 UNITED ST9 MFGR#1436 10347631 -1997 UNITED ST9 MFGR#1437 40051690 -1997 UNITED ST9 MFGR#1438 2662945 -1997 UNITED ST9 MFGR#1439 18011455 -1997 UNITED ST9 MFGR#144 10202784 -1997 UNITED ST9 MFGR#1440 11123196 -1997 UNITED ST9 MFGR#145 11326624 -1997 UNITED ST9 MFGR#146 27825456 -1997 UNITED ST9 MFGR#147 17579502 -1997 UNITED ST9 MFGR#148 17001116 -1998 UNITED ST0 MFGR#141 12484786 -1998 UNITED ST0 MFGR#1410 5225999 -1998 UNITED ST0 MFGR#1411 2633395 -1998 UNITED ST0 MFGR#1412 9479536 -1998 UNITED ST0 MFGR#1413 18468711 -1998 UNITED ST0 MFGR#1414 19440601 -1998 UNITED ST0 MFGR#1415 16894643 -1998 UNITED ST0 MFGR#1416 7520117 -1998 UNITED ST0 MFGR#1417 464576 -1998 UNITED ST0 MFGR#1418 5171747 -1998 UNITED ST0 MFGR#1419 5970985 -1998 UNITED ST0 MFGR#142 23830409 -1998 UNITED ST0 MFGR#1420 2568835 -1998 UNITED ST0 MFGR#1421 6096442 -1998 UNITED ST0 MFGR#1422 10099456 -1998 UNITED ST0 MFGR#1423 24617746 -1998 UNITED ST0 MFGR#1424 22853888 -1998 UNITED ST0 MFGR#1425 25622112 -1998 UNITED ST0 MFGR#1426 9630701 -1998 UNITED ST0 MFGR#1427 4952625 -1998 UNITED ST0 MFGR#1428 4675471 -1998 UNITED ST0 MFGR#1429 19713527 -1998 UNITED ST0 MFGR#143 4971265 -1998 UNITED ST0 MFGR#1430 13948705 -1998 UNITED ST0 MFGR#1431 2955181 -1998 UNITED ST0 MFGR#1432 17946759 -1998 UNITED ST0 MFGR#1433 23325027 -1998 UNITED ST0 MFGR#1434 8580738 -1998 UNITED ST0 MFGR#1435 6663914 -1998 UNITED ST0 MFGR#1436 13468582 -1998 UNITED ST0 MFGR#1437 9419979 -1998 UNITED ST0 MFGR#1438 16164522 -1998 UNITED ST0 MFGR#1439 8692866 -1998 UNITED ST0 MFGR#144 17338711 -1998 UNITED ST0 MFGR#1440 8513736 -1998 UNITED ST0 MFGR#145 6212569 -1998 UNITED ST0 MFGR#146 2702349 -1998 UNITED ST0 MFGR#149 8844540 -1998 UNITED ST1 MFGR#141 13449110 -1998 UNITED ST1 MFGR#1410 5224626 -1998 UNITED ST1 MFGR#1411 16046399 -1998 UNITED ST1 MFGR#1412 8424392 -1998 UNITED ST1 MFGR#1413 7634982 -1998 UNITED ST1 MFGR#1414 8711782 -1998 UNITED ST1 MFGR#1415 5991201 -1998 UNITED ST1 MFGR#1416 11059645 -1998 UNITED ST1 MFGR#1417 8108087 -1998 UNITED ST1 MFGR#1418 1710875 -1998 UNITED ST1 MFGR#1419 452298 -1998 UNITED ST1 MFGR#142 6186526 -1998 UNITED ST1 MFGR#1421 4766963 -1998 UNITED ST1 MFGR#1422 5556563 -1998 UNITED ST1 MFGR#1423 12110257 -1998 UNITED ST1 MFGR#1424 8924948 -1998 UNITED ST1 MFGR#1425 5684710 -1998 UNITED ST1 MFGR#1426 6844444 -1998 UNITED ST1 MFGR#1427 7625525 -1998 UNITED ST1 MFGR#1428 5227395 -1998 UNITED ST1 MFGR#1429 2855272 -1998 UNITED ST1 MFGR#143 6964811 -1998 UNITED ST1 MFGR#1430 16952380 -1998 UNITED ST1 MFGR#1431 7655883 -1998 UNITED ST1 MFGR#1432 2975960 -1998 UNITED ST1 MFGR#1433 10175594 -1998 UNITED ST1 MFGR#1434 26358848 -1998 UNITED ST1 MFGR#1435 11408623 -1998 UNITED ST1 MFGR#1436 4438242 -1998 UNITED ST1 MFGR#1437 10014906 -1998 UNITED ST1 MFGR#1438 18817171 -1998 UNITED ST1 MFGR#1439 2129973 -1998 UNITED ST1 MFGR#144 9512727 -1998 UNITED ST1 MFGR#1440 20185179 -1998 UNITED ST1 MFGR#145 823605 -1998 UNITED ST1 MFGR#146 9358026 -1998 UNITED ST1 MFGR#147 6552312 -1998 UNITED ST1 MFGR#148 9144461 -1998 UNITED ST1 MFGR#149 2149639 -1998 UNITED ST2 MFGR#1410 8044368 -1998 UNITED ST2 MFGR#1413 727343 -1998 UNITED ST2 MFGR#1414 251088 -1998 UNITED ST2 MFGR#1416 5224696 -1998 UNITED ST2 MFGR#1419 7917142 -1998 UNITED ST2 MFGR#1420 9848986 -1998 UNITED ST2 MFGR#1427 4816976 -1998 UNITED ST2 MFGR#1429 4196069 -1998 UNITED ST2 MFGR#143 10073044 -1998 UNITED ST2 MFGR#1431 877123 -1998 UNITED ST2 MFGR#1433 3740836 -1998 UNITED ST2 MFGR#1434 857691 -1998 UNITED ST2 MFGR#1435 2277870 -1998 UNITED ST2 MFGR#145 4906521 -1998 UNITED ST2 MFGR#149 842348 -1998 UNITED ST3 MFGR#141 4181582 -1998 UNITED ST3 MFGR#1410 3169761 -1998 UNITED ST3 MFGR#1412 6590440 -1998 UNITED ST3 MFGR#1413 2015369 -1998 UNITED ST3 MFGR#1414 4408466 -1998 UNITED ST3 MFGR#1415 3963826 -1998 UNITED ST3 MFGR#1416 3179540 -1998 UNITED ST3 MFGR#1417 11440999 -1998 UNITED ST3 MFGR#1418 8725751 -1998 UNITED ST3 MFGR#1419 10646990 -1998 UNITED ST3 MFGR#142 5056172 -1998 UNITED ST3 MFGR#1420 3583150 -1998 UNITED ST3 MFGR#1421 7085689 -1998 UNITED ST3 MFGR#1422 20858904 -1998 UNITED ST3 MFGR#1423 12825743 -1998 UNITED ST3 MFGR#1424 3533518 -1998 UNITED ST3 MFGR#1425 4938324 -1998 UNITED ST3 MFGR#1426 15164664 -1998 UNITED ST3 MFGR#1427 2619162 -1998 UNITED ST3 MFGR#1428 7723699 -1998 UNITED ST3 MFGR#143 24362893 -1998 UNITED ST3 MFGR#1430 7139650 -1998 UNITED ST3 MFGR#1431 10306394 -1998 UNITED ST3 MFGR#1432 14769004 -1998 UNITED ST3 MFGR#1433 9610545 -1998 UNITED ST3 MFGR#1435 6611760 -1998 UNITED ST3 MFGR#1436 941348 -1998 UNITED ST3 MFGR#1437 4802162 -1998 UNITED ST3 MFGR#1438 5079750 -1998 UNITED ST3 MFGR#1439 5905118 -1998 UNITED ST3 MFGR#144 5436839 -1998 UNITED ST3 MFGR#1440 9815517 -1998 UNITED ST3 MFGR#145 7580271 -1998 UNITED ST3 MFGR#146 14051404 -1998 UNITED ST3 MFGR#147 7786017 -1998 UNITED ST3 MFGR#149 1271813 -1998 UNITED ST4 MFGR#141 26703183 -1998 UNITED ST4 MFGR#1410 18094129 -1998 UNITED ST4 MFGR#1411 615999 -1998 UNITED ST4 MFGR#1412 13988709 -1998 UNITED ST4 MFGR#1413 10115647 -1998 UNITED ST4 MFGR#1414 1586551 -1998 UNITED ST4 MFGR#1415 14357244 -1998 UNITED ST4 MFGR#1416 9430036 -1998 UNITED ST4 MFGR#1417 17361773 -1998 UNITED ST4 MFGR#1418 10187338 -1998 UNITED ST4 MFGR#1419 7116094 -1998 UNITED ST4 MFGR#142 18454502 -1998 UNITED ST4 MFGR#1420 1621340 -1998 UNITED ST4 MFGR#1422 13861291 -1998 UNITED ST4 MFGR#1423 4208006 -1998 UNITED ST4 MFGR#1424 3242576 -1998 UNITED ST4 MFGR#1425 1470642 -1998 UNITED ST4 MFGR#1426 11987205 -1998 UNITED ST4 MFGR#1427 12975399 -1998 UNITED ST4 MFGR#1428 11756155 -1998 UNITED ST4 MFGR#1429 6956191 -1998 UNITED ST4 MFGR#143 11499476 -1998 UNITED ST4 MFGR#1430 4355096 -1998 UNITED ST4 MFGR#1431 20166612 -1998 UNITED ST4 MFGR#1432 8849789 -1998 UNITED ST4 MFGR#1433 10786037 -1998 UNITED ST4 MFGR#1435 13208151 -1998 UNITED ST4 MFGR#1436 4747808 -1998 UNITED ST4 MFGR#1437 12999351 -1998 UNITED ST4 MFGR#1438 17360787 -1998 UNITED ST4 MFGR#1439 2557841 -1998 UNITED ST4 MFGR#144 22030911 -1998 UNITED ST4 MFGR#145 5272280 -1998 UNITED ST4 MFGR#146 4307958 -1998 UNITED ST4 MFGR#147 7995050 -1998 UNITED ST4 MFGR#148 2348889 -1998 UNITED ST5 MFGR#141 8396190 -1998 UNITED ST5 MFGR#1410 17422889 -1998 UNITED ST5 MFGR#1411 764246 -1998 UNITED ST5 MFGR#1413 8051561 -1998 UNITED ST5 MFGR#1414 1500867 -1998 UNITED ST5 MFGR#1416 3874315 -1998 UNITED ST5 MFGR#1417 7550747 -1998 UNITED ST5 MFGR#1418 14241755 -1998 UNITED ST5 MFGR#1419 4893846 -1998 UNITED ST5 MFGR#142 5418299 -1998 UNITED ST5 MFGR#1420 3810727 -1998 UNITED ST5 MFGR#1421 3321336 -1998 UNITED ST5 MFGR#1423 5021100 -1998 UNITED ST5 MFGR#1424 6125645 -1998 UNITED ST5 MFGR#1426 880680 -1998 UNITED ST5 MFGR#1427 1284890 -1998 UNITED ST5 MFGR#1428 3813108 -1998 UNITED ST5 MFGR#1429 9429868 -1998 UNITED ST5 MFGR#143 15699981 -1998 UNITED ST5 MFGR#1430 7977654 -1998 UNITED ST5 MFGR#1431 5545969 -1998 UNITED ST5 MFGR#1432 1803434 -1998 UNITED ST5 MFGR#1435 12957056 -1998 UNITED ST5 MFGR#1436 20666755 -1998 UNITED ST5 MFGR#1437 9536733 -1998 UNITED ST5 MFGR#1439 11784214 -1998 UNITED ST5 MFGR#1440 613089 -1998 UNITED ST5 MFGR#145 8237944 -1998 UNITED ST5 MFGR#147 5370351 -1998 UNITED ST6 MFGR#141 6683977 -1998 UNITED ST6 MFGR#1411 11589040 -1998 UNITED ST6 MFGR#1412 4800954 -1998 UNITED ST6 MFGR#1416 5848432 -1998 UNITED ST6 MFGR#1417 1804324 -1998 UNITED ST6 MFGR#1418 4642727 -1998 UNITED ST6 MFGR#1419 10079621 -1998 UNITED ST6 MFGR#142 2533570 -1998 UNITED ST6 MFGR#1420 9016258 -1998 UNITED ST6 MFGR#1421 7636150 -1998 UNITED ST6 MFGR#1422 1272436 -1998 UNITED ST6 MFGR#1423 9600097 -1998 UNITED ST6 MFGR#1424 7036762 -1998 UNITED ST6 MFGR#1425 10867426 -1998 UNITED ST6 MFGR#1426 7033994 -1998 UNITED ST6 MFGR#1427 6481679 -1998 UNITED ST6 MFGR#1428 6269185 -1998 UNITED ST6 MFGR#1429 12055631 -1998 UNITED ST6 MFGR#143 6500459 -1998 UNITED ST6 MFGR#1430 6816243 -1998 UNITED ST6 MFGR#1432 7752973 -1998 UNITED ST6 MFGR#1436 6292813 -1998 UNITED ST6 MFGR#1437 2334894 -1998 UNITED ST6 MFGR#1438 4459455 -1998 UNITED ST6 MFGR#144 640220 -1998 UNITED ST6 MFGR#1440 748200 -1998 UNITED ST6 MFGR#145 7948794 -1998 UNITED ST6 MFGR#146 2734333 -1998 UNITED ST6 MFGR#147 2995632 -1998 UNITED ST6 MFGR#148 10695287 -1998 UNITED ST6 MFGR#149 1475864 -1998 UNITED ST7 MFGR#141 4816637 -1998 UNITED ST7 MFGR#1410 24393912 -1998 UNITED ST7 MFGR#1411 1283038 -1998 UNITED ST7 MFGR#1412 12607459 -1998 UNITED ST7 MFGR#1413 15421293 -1998 UNITED ST7 MFGR#1414 7681668 -1998 UNITED ST7 MFGR#1415 7138377 -1998 UNITED ST7 MFGR#1416 9246990 -1998 UNITED ST7 MFGR#1417 5242972 -1998 UNITED ST7 MFGR#1419 13548941 -1998 UNITED ST7 MFGR#142 17758762 -1998 UNITED ST7 MFGR#1420 11980437 -1998 UNITED ST7 MFGR#1421 11166952 -1998 UNITED ST7 MFGR#1422 9231883 -1998 UNITED ST7 MFGR#1424 5796389 -1998 UNITED ST7 MFGR#1425 4943582 -1998 UNITED ST7 MFGR#1426 16361100 -1998 UNITED ST7 MFGR#1427 9659296 -1998 UNITED ST7 MFGR#1428 2998020 -1998 UNITED ST7 MFGR#1429 5766768 -1998 UNITED ST7 MFGR#143 5526167 -1998 UNITED ST7 MFGR#1430 4191892 -1998 UNITED ST7 MFGR#1431 12979559 -1998 UNITED ST7 MFGR#1432 5832068 -1998 UNITED ST7 MFGR#1433 11410227 -1998 UNITED ST7 MFGR#1434 10525792 -1998 UNITED ST7 MFGR#1435 5982420 -1998 UNITED ST7 MFGR#1436 7142934 -1998 UNITED ST7 MFGR#1437 7499671 -1998 UNITED ST7 MFGR#1438 9177564 -1998 UNITED ST7 MFGR#1439 1974444 -1998 UNITED ST7 MFGR#144 2428463 -1998 UNITED ST7 MFGR#1440 3300838 -1998 UNITED ST7 MFGR#146 1771645 -1998 UNITED ST7 MFGR#147 4681835 -1998 UNITED ST7 MFGR#148 7124367 -1998 UNITED ST7 MFGR#149 11662970 -1998 UNITED ST8 MFGR#141 12753354 -1998 UNITED ST8 MFGR#1410 1097292 -1998 UNITED ST8 MFGR#1411 4069881 -1998 UNITED ST8 MFGR#1414 9474831 -1998 UNITED ST8 MFGR#1415 6406304 -1998 UNITED ST8 MFGR#1418 2392133 -1998 UNITED ST8 MFGR#1419 2953230 -1998 UNITED ST8 MFGR#142 10702563 -1998 UNITED ST8 MFGR#1420 3719432 -1998 UNITED ST8 MFGR#1421 3854657 -1998 UNITED ST8 MFGR#1422 9711968 -1998 UNITED ST8 MFGR#1424 1102516 -1998 UNITED ST8 MFGR#1426 8937532 -1998 UNITED ST8 MFGR#1427 5255637 -1998 UNITED ST8 MFGR#1428 1695112 -1998 UNITED ST8 MFGR#143 6231349 -1998 UNITED ST8 MFGR#1430 4678664 -1998 UNITED ST8 MFGR#1431 2459741 -1998 UNITED ST8 MFGR#1433 1724339 -1998 UNITED ST8 MFGR#1434 8222328 -1998 UNITED ST8 MFGR#1435 4087165 -1998 UNITED ST8 MFGR#1437 7379288 -1998 UNITED ST8 MFGR#1439 1947163 -1998 UNITED ST8 MFGR#144 21846266 -1998 UNITED ST8 MFGR#146 13768113 -1998 UNITED ST8 MFGR#147 5279643 -1998 UNITED ST8 MFGR#149 15221410 -1998 UNITED ST9 MFGR#141 5544000 -1998 UNITED ST9 MFGR#1410 7971104 -1998 UNITED ST9 MFGR#1411 26426609 -1998 UNITED ST9 MFGR#1412 1701777 -1998 UNITED ST9 MFGR#1413 18786672 -1998 UNITED ST9 MFGR#1414 15787780 -1998 UNITED ST9 MFGR#1415 11214818 -1998 UNITED ST9 MFGR#1416 2076120 -1998 UNITED ST9 MFGR#1417 14310334 -1998 UNITED ST9 MFGR#1418 3558355 -1998 UNITED ST9 MFGR#1419 6134867 -1998 UNITED ST9 MFGR#142 257897 -1998 UNITED ST9 MFGR#1420 7716338 -1998 UNITED ST9 MFGR#1421 10291775 -1998 UNITED ST9 MFGR#1422 6105716 -1998 UNITED ST9 MFGR#1423 11378613 -1998 UNITED ST9 MFGR#1424 1904550 -1998 UNITED ST9 MFGR#1425 18831147 -1998 UNITED ST9 MFGR#1426 18562642 -1998 UNITED ST9 MFGR#1427 12565890 -1998 UNITED ST9 MFGR#1428 10720914 -1998 UNITED ST9 MFGR#1429 21940718 -1998 UNITED ST9 MFGR#143 21480722 -1998 UNITED ST9 MFGR#1430 30935173 -1998 UNITED ST9 MFGR#1431 7793012 -1998 UNITED ST9 MFGR#1432 10026104 -1998 UNITED ST9 MFGR#1433 17874435 -1998 UNITED ST9 MFGR#1434 5493785 -1998 UNITED ST9 MFGR#1435 10738769 -1998 UNITED ST9 MFGR#1436 43504 -1998 UNITED ST9 MFGR#1437 13559646 -1998 UNITED ST9 MFGR#1438 12654258 -1998 UNITED ST9 MFGR#1439 5972923 -1998 UNITED ST9 MFGR#144 15702123 -1998 UNITED ST9 MFGR#1440 6681029 -1998 UNITED ST9 MFGR#145 920718 -1998 UNITED ST9 MFGR#146 10435931 -1998 UNITED ST9 MFGR#147 4759326 -1998 UNITED ST9 MFGR#148 9970848 -1998 UNITED ST9 MFGR#149 7510940 +1992 UNITED ST0 MFGR#141 555641 +1992 UNITED ST0 MFGR#1411 7079182 +1992 UNITED ST0 MFGR#1412 2872721 +1992 UNITED ST0 MFGR#1413 5471501 +1992 UNITED ST0 MFGR#1414 6180491 +1992 UNITED ST0 MFGR#1415 3832532 +1992 UNITED ST0 MFGR#1417 5090445 +1992 UNITED ST0 MFGR#142 4603254 +1992 UNITED ST0 MFGR#1421 717404 +1992 UNITED ST0 MFGR#1422 6204274 +1992 UNITED ST0 MFGR#1424 5518896 +1992 UNITED ST0 MFGR#1425 5763764 +1992 UNITED ST0 MFGR#1427 6180274 +1992 UNITED ST0 MFGR#1432 6428730 +1992 UNITED ST0 MFGR#1433 833533 +1992 UNITED ST0 MFGR#1434 13713612 +1992 UNITED ST0 MFGR#145 13934235 +1992 UNITED ST2 MFGR#1411 758915 +1992 UNITED ST2 MFGR#142 4221010 +1992 UNITED ST2 MFGR#1421 4315917 +1992 UNITED ST2 MFGR#1424 57589 +1992 UNITED ST2 MFGR#1427 1685113 +1992 UNITED ST2 MFGR#143 1865250 +1992 UNITED ST2 MFGR#1430 6454140 +1992 UNITED ST3 MFGR#141 10985538 +1992 UNITED ST3 MFGR#1410 5661337 +1992 UNITED ST3 MFGR#1411 2913347 +1992 UNITED ST3 MFGR#1412 3456611 +1992 UNITED ST3 MFGR#1413 1843964 +1992 UNITED ST3 MFGR#1415 741657 +1992 UNITED ST3 MFGR#1416 12592059 +1992 UNITED ST3 MFGR#1417 343888 +1992 UNITED ST3 MFGR#1419 2221452 +1992 UNITED ST3 MFGR#142 579832 +1992 UNITED ST3 MFGR#1421 2176380 +1992 UNITED ST3 MFGR#1422 1841100 +1992 UNITED ST3 MFGR#1423 121681 +1992 UNITED ST3 MFGR#1425 3025877 +1992 UNITED ST3 MFGR#1426 2812288 +1992 UNITED ST3 MFGR#1427 7624182 +1992 UNITED ST3 MFGR#1428 9484717 +1992 UNITED ST3 MFGR#1429 13290434 +1992 UNITED ST3 MFGR#1430 5263185 +1992 UNITED ST3 MFGR#1431 1057990 +1992 UNITED ST3 MFGR#1433 3172704 +1992 UNITED ST3 MFGR#1436 3292838 +1992 UNITED ST3 MFGR#1438 10257150 +1992 UNITED ST3 MFGR#144 4992624 +1992 UNITED ST3 MFGR#145 4555258 +1992 UNITED ST3 MFGR#146 5155417 +1992 UNITED ST3 MFGR#147 6044237 +1992 UNITED ST3 MFGR#148 12453066 +1992 UNITED ST3 MFGR#149 4283724 +1992 UNITED ST6 MFGR#1413 4420349 +1992 UNITED ST6 MFGR#1415 6201665 +1992 UNITED ST6 MFGR#1416 2588026 +1992 UNITED ST6 MFGR#142 55047 +1992 UNITED ST6 MFGR#1420 1288451 +1992 UNITED ST6 MFGR#1421 2567818 +1992 UNITED ST6 MFGR#1423 583540 +1992 UNITED ST6 MFGR#1426 6446245 +1992 UNITED ST6 MFGR#1429 1693359 +1992 UNITED ST6 MFGR#1430 954778 +1992 UNITED ST6 MFGR#1436 2419392 +1992 UNITED ST6 MFGR#1437 7302587 +1992 UNITED ST6 MFGR#144 8794123 +1992 UNITED ST6 MFGR#1440 3993789 +1992 UNITED ST7 MFGR#141 6582861 +1992 UNITED ST7 MFGR#1411 692197 +1992 UNITED ST7 MFGR#1412 169740 +1992 UNITED ST7 MFGR#1413 4359125 +1992 UNITED ST7 MFGR#1420 2500664 +1992 UNITED ST7 MFGR#1424 5152200 +1992 UNITED ST7 MFGR#1426 8490217 +1992 UNITED ST7 MFGR#143 5620388 +1992 UNITED ST7 MFGR#1431 1235687 +1992 UNITED ST7 MFGR#1432 6829944 +1992 UNITED ST7 MFGR#1440 6253049 +1992 UNITED ST7 MFGR#145 3732260 +1992 UNITED ST7 MFGR#147 2421233 +1992 UNITED ST7 MFGR#149 1866816 +1992 UNITED ST9 MFGR#141 147955 +1992 UNITED ST9 MFGR#1413 5392214 +1992 UNITED ST9 MFGR#1414 6319659 +1992 UNITED ST9 MFGR#1415 1084942 +1992 UNITED ST9 MFGR#1416 9714488 +1992 UNITED ST9 MFGR#1421 5856389 +1992 UNITED ST9 MFGR#1422 5439287 +1992 UNITED ST9 MFGR#1423 7040305 +1992 UNITED ST9 MFGR#1426 6595331 +1992 UNITED ST9 MFGR#1428 446309 +1992 UNITED ST9 MFGR#1429 3993195 +1992 UNITED ST9 MFGR#143 2320022 +1992 UNITED ST9 MFGR#1433 8938527 +1992 UNITED ST9 MFGR#1434 3280337 +1992 UNITED ST9 MFGR#1435 11735187 +1992 UNITED ST9 MFGR#1438 2940085 +1992 UNITED ST9 MFGR#145 2145463 +1992 UNITED ST9 MFGR#147 5798259 +1992 UNITED ST9 MFGR#148 4292580 diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out index 8f376d296af9e4..1946f8020ded90 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out @@ -56,7 +56,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 +------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() --------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() @@ -69,7 +72,4 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalProject ----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out index 531ca06c5a80bb..01b1d2752e59b5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out @@ -40,10 +40,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() ------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() --------------------PhysicalProject ----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) @@ -51,4 +48,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalProject +--------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out index 3d35d56d81d4e1..63f30b978f4d68 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out @@ -51,20 +51,14 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() ------------------PhysicalProject --------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 ------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() ---------------------------PhysicalProject -----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() --------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() ----------------------------PhysicalProject ------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) @@ -72,4 +66,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) --------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalProject +----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalProject +--------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out index fa8d314e00019d..b44acb09519210 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out @@ -40,10 +40,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() ------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() --------------------PhysicalProject ----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) @@ -51,4 +48,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalProject +--------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out index 27803aec4d8f64..2ab0ebfe7fb09c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out @@ -51,20 +51,14 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() ------------------PhysicalProject --------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 ------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() ---------------------------PhysicalProject -----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() --------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() ----------------------------PhysicalProject ------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) @@ -72,4 +66,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) --------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalProject +----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalProject +--------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out index a6b03c32eee3df..61bce070b34aaa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out @@ -56,7 +56,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 +------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() --------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() @@ -69,7 +72,4 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalProject ----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 8f376d296af9e4..1946f8020ded90 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 @@ -56,7 +56,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 +------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() --------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() @@ -69,7 +72,4 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalProject ----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out index 531ca06c5a80bb..01b1d2752e59b5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out @@ -40,10 +40,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() ------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() --------------------PhysicalProject ----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) @@ -51,4 +48,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalProject +--------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out index 3d35d56d81d4e1..63f30b978f4d68 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out @@ -51,20 +51,14 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() ------------------PhysicalProject --------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 ------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() ---------------------------PhysicalProject -----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() --------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() ----------------------------PhysicalProject ------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) @@ -72,4 +66,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) --------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalProject +----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalProject +--------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out index fa8d314e00019d..b44acb09519210 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out @@ -40,10 +40,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() ------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() --------------------PhysicalProject ----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) @@ -51,4 +48,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalProject +--------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out index 27803aec4d8f64..2ab0ebfe7fb09c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out @@ -51,20 +51,14 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() ------------------PhysicalProject --------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 ------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() ---------------------------PhysicalProject -----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() --------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() ----------------------------PhysicalProject ------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) @@ -72,4 +66,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) --------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalProject +----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalProject +--------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out index a6b03c32eee3df..61bce070b34aaa 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out @@ -56,7 +56,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 +------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() --------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() @@ -69,7 +72,4 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalProject ----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out index 8f376d296af9e4..1946f8020ded90 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out @@ -56,7 +56,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 +------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() --------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() @@ -69,7 +72,4 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalProject ----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out index 8f376d296af9e4..1946f8020ded90 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out @@ -56,7 +56,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((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 +------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() --------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() @@ -69,7 +72,4 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalProject ----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/partition_p0/auto_partition/test_auto_dynamic.out b/regression-test/data/partition_p0/auto_partition/test_auto_dynamic.out new file mode 100644 index 00000000000000..0b588c49759ad4 --- /dev/null +++ b/regression-test/data/partition_p0/auto_partition/test_auto_dynamic.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql_dynamic_auto -- +2024-01-01T00:00 +2900-01-01T00:00 +3000-01-01T00:00 + diff --git a/regression-test/data/partition_p0/auto_partition/test_auto_partition_behavior.out b/regression-test/data/partition_p0/auto_partition/test_auto_partition_behavior.out index 27d77c500f2864..133a86cddab91d 100644 --- a/regression-test/data/partition_p0/auto_partition/test_auto_partition_behavior.out +++ b/regression-test/data/partition_p0/auto_partition/test_auto_partition_behavior.out @@ -114,8 +114,3 @@ Xxx -- !sql_non_order3 -- 3 2013-12-12T00:00 --- !sql_dynamic_auto -- -2024-01-01T00:00 -2900-01-01T00:00 -3000-01-01T00:00 - diff --git a/regression-test/data/query_p0/sql_functions/encryption_digest/test_encryption_function.out b/regression-test/data/query_p0/sql_functions/encryption_digest/test_encryption_function.out index 9e920596f894cf..721412dc0364c1 100644 --- a/regression-test/data/query_p0/sql_functions/encryption_digest/test_encryption_function.out +++ b/regression-test/data/query_p0/sql_functions/encryption_digest/test_encryption_function.out @@ -11,33 +11,6 @@ text -- !sql -- \N --- !sql -- -wr2JEDVXzL9+2XtRhgIloA== - --- !sql -- -wr2JEDVXzL9+2XtRhgIloA== - --- !sql -- -text - --- !sql -- -\N - --- !sql -- -wr2JEDVXzL9+2XtRhgIloA== - --- !sql -- -BO2vxHeUcw5BQQalSBbo1w== - --- !sql -- -text - --- !sql -- -\N - --- !sql -- -text - -- !sql -- BO2vxHeUcw5BQQalSBbo1w== @@ -54,25 +27,7 @@ text text -- !sql -- -wr2JEDVXzL9+2XtRhgIloA== - --- !sql -- -tsmK1HzbpnEdR2//WhO+MA== - --- !sql -- -ciacXDLHMNG7CD9Kws8png== - --- !sql -- -text - --- !sql -- -\N - --- !sql -- -text - --- !sql -- -text +3dym0E7/+1zbrLIaBVNHSw== -- !sql -- tsmK1HzbpnEdR2//WhO+MA== @@ -93,7 +48,7 @@ text text -- !sql -- -aDjwRflBrDjhBZIOFNw3Tg== +FSYstvOmH2cXy7B/072Mug== -- !sql -- 1Y4NGIukSbv9OrkZnRD1bQ== @@ -101,27 +56,12 @@ aDjwRflBrDjhBZIOFNw3Tg== -- !sql -- G5POcFAJwiZHeTtN6DjInQ== --- !sql -- -text - --- !sql -- -text - -- !sql -- \N -- !sql -- text --- !sql -- -1Y4NGIukSbv9OrkZnRD1bQ== - --- !sql -- -G5POcFAJwiZHeTtN6DjInQ== - --- !sql -- -text - -- !sql -- \N diff --git a/regression-test/data/query_p0/sql_functions/math_functions/test_math_unary_always_nullable.out b/regression-test/data/query_p0/sql_functions/math_functions/test_math_unary_always_nullable.out new file mode 100644 index 00000000000000..0a190f0bd6b2f9 --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/math_functions/test_math_unary_always_nullable.out @@ -0,0 +1,95 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !acos_1 -- +\N true + +-- !acos_2 -- +\N true + +-- !acos_3 -- +\N true 0 +\N true 1 +\N true 2 +\N true 3 +\N true 4 +\N true 5 +\N true 6 +\N true 7 +\N true 8 +\N true 9 + +-- !asin_1 -- +\N true + +-- !asin_2 -- +\N true + +-- !asin_3 -- +\N true 0 +\N true 1 +\N true 2 +\N true 3 +\N true 4 +\N true 5 +\N true 6 +\N true 7 +\N true 8 +\N true 9 + +-- !sqrt_1 -- +\N true + +-- !sqrt_2 -- +\N true + +-- !sqrt_3 -- +\N true 0 +\N true 1 +\N true 2 +\N true 3 +\N true 4 +\N true 5 +\N true 6 +\N true 7 +\N true 8 +\N true 9 + +-- !acos_tbl_1 -- +1 \N true +2 \N true +3 1.5707963267948966 false +4 \N true +5 \N true +6 \N true +7 \N true +8 \N true + +-- !asin_tbl_1 -- +1 \N true +2 \N true +3 0.0 false +4 \N true +5 \N true +6 \N true +7 \N true +8 \N true + +-- !sqrt_tbl_1 -- +1 1.0488088481701516 false +2 \N true +3 0.0 false +4 \N true +5 \N true +6 \N true +7 \N true +8 \N true + +-- !dsqrt_tbl_1 -- +1 1.0488088481701516 false +2 \N true +3 0.0 false +4 \N true +5 \N true +6 \N true +7 \N true +8 \N true + diff --git a/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_null.groovy b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_null.groovy index f882cda1522004..5bf1ec1773b559 100644 --- a/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_null.groovy +++ b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_null.groovy @@ -120,6 +120,7 @@ suite("test_index_compaction_null", "nonConcurrent") { } def run_sql = { -> + sql """ set enable_common_expr_pushdown=true """ // select all data qt_select_0 "SELECT * FROM ${tableName} ORDER BY id" diff --git a/regression-test/suites/inverted_index_p0/test_ignore_above_in_index.groovy b/regression-test/suites/inverted_index_p0/test_ignore_above_in_index.groovy index d6b426614414af..bde764fa332e1f 100644 --- a/regression-test/suites/inverted_index_p0/test_ignore_above_in_index.groovy +++ b/regression-test/suites/inverted_index_p0/test_ignore_above_in_index.groovy @@ -84,6 +84,7 @@ suite("test_ignore_above_in_index", "p0") { assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) } } - + sql """ set enable_common_expr_pushdown = true; """ qt_sql "select count() from ${tableName2} where clientip > '17.0';" + qt_sql "select count() from ${tableName2} where clientip > '17.0' or status = 200;" } diff --git a/regression-test/suites/nereids_function_p0/scalar_function/S.groovy b/regression-test/suites/nereids_function_p0/scalar_function/S.groovy index ea6cc5c817e2a2..98a8685a88840b 100644 --- a/regression-test/suites/nereids_function_p0/scalar_function/S.groovy +++ b/regression-test/suites/nereids_function_p0/scalar_function/S.groovy @@ -97,26 +97,11 @@ suite("nereids_scalar_fn_S") { qt_sql_sm3sum_Varchar_notnull "select sm3sum(kvchrs1) from fn_test_not_nullable order by kvchrs1" qt_sql_sm3sum_String "select sm3sum(kstr) from fn_test order by kstr" qt_sql_sm3sum_String_notnull "select sm3sum(kstr) from fn_test_not_nullable order by kstr" + sql "select sm4_decrypt(kvchrs1, kvchrs1) from fn_test order by kvchrs1, kvchrs1" - test { - sql "select sm4_decrypt_v2(kvchrs1, kvchrs1) from fn_test order by kvchrs1, kvchrs1" - exception "Incorrect parameter count in the call to native function 'sm4_decrypt'" - } sql "select sm4_decrypt(kvchrs1, kvchrs1) from fn_test_not_nullable order by kvchrs1, kvchrs1" - test { - sql "select sm4_decrypt_v2(kvchrs1, kvchrs1) from fn_test_not_nullable order by kvchrs1, kvchrs1" - exception "Incorrect parameter count in the call to native function 'sm4_decrypt'" - } sql "select sm4_decrypt(kstr, kstr) from fn_test order by kstr, kstr" - test { - sql "select sm4_decrypt_v2(kstr, kstr) from fn_test order by kstr, kstr" - exception "Incorrect parameter count in the call to native function 'sm4_decrypt'" - } sql "select sm4_decrypt(kstr, kstr) from fn_test_not_nullable order by kstr, kstr" - test { - sql "select sm4_decrypt_v2(kstr, kstr) from fn_test_not_nullable order by kstr, kstr" - exception "Incorrect parameter count in the call to native function 'sm4_decrypt'" - } sql "select sm4_decrypt(kvchrs1, kvchrs1, kvchrs1) from fn_test order by kvchrs1, kvchrs1, kvchrs1" sql "select sm4_decrypt(kvchrs1, kvchrs1, kvchrs1) from fn_test_not_nullable order by kvchrs1, kvchrs1, kvchrs1" sql "select sm4_decrypt(kstr, kstr, kstr) from fn_test order by kstr, kstr, kstr" @@ -125,34 +110,11 @@ suite("nereids_scalar_fn_S") { sql "select sm4_decrypt(kvchrs1, kvchrs1, kvchrs1, 'SM4_128_ECB') from fn_test_not_nullable order by kvchrs1, kvchrs1, kvchrs1" sql "select sm4_decrypt(kstr, kstr, kstr, 'SM4_128_ECB') from fn_test order by kstr, kstr, kstr" sql "select sm4_decrypt(kstr, kstr, kstr, 'SM4_128_ECB') from fn_test_not_nullable order by kstr, kstr, kstr" - sql "select sm4_decrypt_v2(kvchrs1, kvchrs1, kvchrs1) from fn_test order by kvchrs1, kvchrs1, kvchrs1" - sql "select sm4_decrypt_v2(kvchrs1, kvchrs1, kvchrs1) from fn_test_not_nullable order by kvchrs1, kvchrs1, kvchrs1" - sql "select sm4_decrypt_v2(kstr, kstr, kstr) from fn_test order by kstr, kstr, kstr" - sql "select sm4_decrypt_v2(kstr, kstr, kstr) from fn_test_not_nullable order by kstr, kstr, kstr" - sql "select sm4_decrypt_v2(kvchrs1, kvchrs1, kvchrs1, 'SM4_128_ECB') from fn_test order by kvchrs1, kvchrs1, kvchrs1" - sql "select sm4_decrypt_v2(kvchrs1, kvchrs1, kvchrs1, 'SM4_128_ECB') from fn_test_not_nullable order by kvchrs1, kvchrs1, kvchrs1" - sql "select sm4_decrypt_v2(kstr, kstr, kstr, 'SM4_128_ECB') from fn_test order by kstr, kstr, kstr" - sql "select sm4_decrypt_v2(kstr, kstr, kstr, 'SM4_128_ECB') from fn_test_not_nullable order by kstr, kstr, kstr" + sql "select sm4_encrypt(kvchrs1, kvchrs1) from fn_test order by kvchrs1, kvchrs1" - test { - sql "select sm4_encrypt_v2(kvchrs1, kvchrs1) from fn_test order by kvchrs1, kvchrs1" - exception "Incorrect parameter count in the call to native function 'sm4_encrypt'" - } sql "select sm4_encrypt(kvchrs1, kvchrs1) from fn_test_not_nullable order by kvchrs1, kvchrs1" - test { - sql "select sm4_encrypt_v2(kvchrs1, kvchrs1) from fn_test_not_nullable order by kvchrs1, kvchrs1" - exception "Incorrect parameter count in the call to native function 'sm4_encrypt'" - } sql "select sm4_encrypt(kstr, kstr) from fn_test order by kstr, kstr" - test { - sql "select sm4_encrypt_v2(kstr, kstr) from fn_test order by kstr, kstr" - exception "Incorrect parameter count in the call to native function 'sm4_encrypt'" - } sql "select sm4_encrypt(kstr, kstr) from fn_test_not_nullable order by kstr, kstr" - test { - sql "select sm4_encrypt_v2(kstr, kstr) from fn_test_not_nullable order by kstr, kstr" - exception "Incorrect parameter count in the call to native function 'sm4_encrypt'" - } sql "select sm4_encrypt(kvchrs1, kvchrs1, kvchrs1) from fn_test order by kvchrs1, kvchrs1, kvchrs1" sql "select sm4_encrypt(kvchrs1, kvchrs1, kvchrs1) from fn_test_not_nullable order by kvchrs1, kvchrs1, kvchrs1" sql "select sm4_encrypt(kstr, kstr, kstr) from fn_test order by kstr, kstr, kstr" @@ -161,14 +123,7 @@ suite("nereids_scalar_fn_S") { sql "select sm4_encrypt(kvchrs1, kvchrs1, kvchrs1, 'SM4_128_ECB') from fn_test_not_nullable order by kvchrs1, kvchrs1, kvchrs1" sql "select sm4_encrypt(kstr, kstr, kstr, 'SM4_128_ECB') from fn_test order by kstr, kstr, kstr" sql "select sm4_encrypt(kstr, kstr, kstr, 'SM4_128_ECB') from fn_test_not_nullable order by kstr, kstr, kstr" - sql "select sm4_encrypt_v2(kvchrs1, kvchrs1, kvchrs1) from fn_test order by kvchrs1, kvchrs1, kvchrs1" - sql "select sm4_encrypt_v2(kvchrs1, kvchrs1, kvchrs1) from fn_test_not_nullable order by kvchrs1, kvchrs1, kvchrs1" - sql "select sm4_encrypt_v2(kstr, kstr, kstr) from fn_test order by kstr, kstr, kstr" - sql "select sm4_encrypt_v2(kstr, kstr, kstr) from fn_test_not_nullable order by kstr, kstr, kstr" - sql "select sm4_encrypt_v2(kvchrs1, kvchrs1, kvchrs1, 'SM4_128_ECB') from fn_test order by kvchrs1, kvchrs1, kvchrs1" - sql "select sm4_encrypt_v2(kvchrs1, kvchrs1, kvchrs1, 'SM4_128_ECB') from fn_test_not_nullable order by kvchrs1, kvchrs1, kvchrs1" - sql "select sm4_encrypt_v2(kstr, kstr, kstr, 'SM4_128_ECB') from fn_test order by kstr, kstr, kstr" - sql "select sm4_encrypt_v2(kstr, kstr, kstr, 'SM4_128_ECB') from fn_test_not_nullable order by kstr, kstr, kstr" + sql "select space(10) from fn_test order by kint" sql "select space(10) from fn_test_not_nullable order by kint" qt_sql_split_part_Varchar_Varchar_Integer "select split_part(kvchrs1, ' ', 1) from fn_test order by kvchrs1" diff --git a/regression-test/suites/nereids_p0/hint/test_use_mv.groovy b/regression-test/suites/nereids_p0/hint/test_use_mv.groovy new file mode 100644 index 00000000000000..e511ccc11ae071 --- /dev/null +++ b/regression-test/suites/nereids_p0/hint/test_use_mv.groovy @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +suite("test_use_mv") { + // create database and tables + sql 'DROP DATABASE IF EXISTS test_use_mv' + sql 'CREATE DATABASE IF NOT EXISTS test_use_mv' + sql 'use test_use_mv' + + // setting planner to nereids + sql 'set exec_mem_limit=21G' + sql 'set be_number_for_test=1' + sql 'set parallel_pipeline_task_num=1' + sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' + sql "set ignore_shape_nodes='PhysicalProject'" + sql 'set enable_fallback_to_original_planner=false' + sql 'set runtime_filter_mode=OFF' + + sql """drop table if exists t1;""" + // create tables + sql """ + CREATE TABLE `t1` ( + `k1` int(11) NULL, + `k2` int(11) NULL, + `v1` int(11) SUM NULL + ) ENGINE=OLAP + AGGREGATE KEY(`k1`, `k2`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "in_memory" = "false", + "storage_format" = "V2", + "disable_auto_compaction" = "false" + ); + """ + sql """ alter table t1 add rollup r1(k2, k1); """ + waitForRollUpJob("t1", 5000, 1) + sql """ alter table t1 add rollup r2(k2); """ + waitForRollUpJob("t1", 5000, 1) + createMV("create materialized view k1_k2_sumk3 as select k1, k2, sum(v1) from t1 group by k1, k2;") + sql """set enable_sync_mv_cost_based_rewrite = false""" + explain { + sql """select k1 from t1;""" + contains("t1(r1)") + } + sql """set enable_sync_mv_cost_based_rewrite = true""" + explain { + sql """select /*+ no_use_mv */ k1 from t1;""" + notContains("t1(r1)") + } + explain { + sql """select /*+ no_use_mv(t1) */ k1 from t1;""" + contains("parameter of no_use_mv hint must be in pairs") + } + explain { + sql """select /*+ no_use_mv(t1.`*`) */ k1 from t1;""" + contains("t1(t1)") + } + explain { + sql """select /*+ use_mv(t1.`*`) */ k1 from t1;""" + contains("use_mv hint should only have one mv in one table") + } + explain { + sql """select /*+ use_mv(t1.r1,t1.r2) */ k1 from t1;""" + contains("use_mv hint should only have one mv in one table") + } + explain { + sql """select /*+ use_mv(t1.r1) use_mv(t1.r2) */ k1 from t1;""" + contains("one use_mv hint is allowed") + } + explain { + sql """select /*+ no_use_mv(t1.r1) no_use_mv(t1.r2) */ k1 from t1;""" + contains("only one no_use_mv hint is allowed") + } + explain { + sql """select /*+ no_use_mv(t1.r3) */ k1 from t1;""" + contains("do not have mv: r3 in table: t1") + } + explain { + sql """select /*+ use_mv(t1.r1) no_use_mv(t1.r1) */ k1 from t1;""" + contains("conflict mv exist in use_mv and no_use_mv in the same time") + } + explain { + sql """select /*+ use_mv(t1.k1_k2_sumk3) */ k1, k2, sum(v1) from t1 group by k1, k2;""" + contains("t1(k1_k2_sumk3)") + } + explain { + sql """select /*+ use_mv(t1.k1_k2_sumk3) */ k1, k2, min(v1) from t1 group by k1, k2;""" + notContains("t1(k1_k2_sumk3)") + } + +} diff --git a/regression-test/suites/nereids_p0/sql_functions/encryption_digest/test_encryption_function.groovy b/regression-test/suites/nereids_p0/sql_functions/encryption_digest/test_encryption_function.groovy index 84b67091b1e02d..0a2d42610012dc 100644 --- a/regression-test/suites/nereids_p0/sql_functions/encryption_digest/test_encryption_function.groovy +++ b/regression-test/suites/nereids_p0/sql_functions/encryption_digest/test_encryption_function.groovy @@ -16,10 +16,9 @@ // under the License. suite("test_encryption_function") { - def tableName = "dwd_candidates" - sql "DROP TABLE IF EXISTS ${tableName}" + sql "DROP TABLE IF EXISTS dwd_candidates" sql """ - CREATE TABLE IF NOT EXISTS ${tableName} ( + CREATE TABLE IF NOT EXISTS dwd_candidates ( c_int INT, `name` varchar(65530) NULL COMMENT "" ) @@ -29,26 +28,12 @@ suite("test_encryption_function") { ) """ - sql "SET enable_nereids_planner=true" - - sql "set batch_size = 4096;" - sql "set block_encryption_mode=\"AES_128_ECB\";" qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3'));" // 'wr2JEDVXzL9+2XtRhgIloA==' qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // wr2JEDVXzL9+2XtRhgIloA== qt_sql "SELECT AES_DECRYPT(FROM_BASE64('wr2JEDVXzL9+2XtRhgIloA=='),'F3229A0B371ED2D9441B830D21A390C3');" // text qt_sql "SELECT AES_DECRYPT(FROM_BASE64('mvZT1KJw7N0RJf27aipUpg=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // NULL - sql "set block_encryption_mode=\"AES_128_ECB\";" - qt_sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" // 'wr2JEDVXzL9+2XtRhgIloA==' - qt_sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // wr2JEDVXzL9+2XtRhgIloA== - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('wr2JEDVXzL9+2XtRhgIloA=='),'F3229A0B371ED2D9441B830D21A390C3');" // text - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('mvZT1KJw7N0RJf27aipUpg=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // NULL - test { - sql "SELECT TO_BASE64(SM4_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" - exception "session variable block_encryption_mode is invalid with sm4" - } - sql "set block_encryption_mode=\"AES_256_ECB\";" qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3'));" // 'wr2JEDVXzL9+2XtRhgIloA==' qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // 'BO2vxHeUcw5BQQalSBbo1w==' @@ -56,13 +41,6 @@ suite("test_encryption_function") { qt_sql "SELECT AES_DECRYPT(FROM_BASE64('BO2vxHeUcw5BQQalSBbo1w=='),'F3229A0B371ED2D9441B830D21A390C3');" // NULL qt_sql "SELECT AES_DECRYPT(FROM_BASE64('BO2vxHeUcw5BQQalSBbo1w=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // text - sql "set block_encryption_mode=\"AES_256_ECB\";" - qt_sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" // 'BO2vxHeUcw5BQQalSBbo1w==' - qt_sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // 'BO2vxHeUcw5BQQalSBbo1w==' - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('wr2JEDVXzL9+2XtRhgIloA=='),'F3229A0B371ED2D9441B830D21A390C3');" // NULL - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('BO2vxHeUcw5BQQalSBbo1w=='),'F3229A0B371ED2D9441B830D21A390C3');" // text - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('BO2vxHeUcw5BQQalSBbo1w=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // text - sql "set block_encryption_mode=\"AES_256_CBC\";" qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3'));" // 'wr2JEDVXzL9+2XtRhgIloA==' qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // tsmK1HzbpnEdR2//WhO+MA== @@ -76,53 +54,6 @@ suite("test_encryption_function") { "GROUP BY x0;" } - sql "set block_encryption_mode=\"AES_256_CBC\";" - test { - sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" - exception "Incorrect parameter count in the call to native function 'aes_encrypt'" - } - qt_sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // tsmK1HzbpnEdR2//WhO+MA== - qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789ff'));" // ciacXDLHMNG7CD9Kws8png== - test { - sql "SELECT AES_DECRYPT_v2(FROM_BASE64('wr2JEDVXzL9+2XtRhgIloA=='),'F3229A0B371ED2D9441B830D21A390C3');" - exception "Incorrect parameter count in the call to native function 'aes_decrypt'" - } - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('mvZT1KJw7N0RJf27aipUpg=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // NULL - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('tsmK1HzbpnEdR2//WhO+MA=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // text - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('ciacXDLHMNG7CD9Kws8png=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789ff');" // text - explain { - sql "SELECT AES_DECRYPT_v2(UNHEX(r_2_3.`name`), 'namePnhe3E0MWyfZivUnVzDy12caymnrKp', '0123456789') AS x0 FROM dwd_candidates AS r_2_3\n" + - "GROUP BY x0;" - } - - sql "set block_encryption_mode=\"SM4_128_CBC\";" - qt_sql "SELECT TO_BASE64(SM4_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3'));" // aDjwRflBrDjhBZIOFNw3Tg== - qt_sql "SELECT TO_BASE64(SM4_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // 1Y4NGIukSbv9OrkZnRD1bQ== - qt_sql "SELECT TO_BASE64(SM4_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789ff'));" // G5POcFAJwiZHeTtN6DjInQ== - qt_sql "SELECT SM4_DECRYPT(FROM_BASE64('aDjwRflBrDjhBZIOFNw3Tg=='),'F3229A0B371ED2D9441B830D21A390C3');" // text - qt_sql "SELECT SM4_DECRYPT(FROM_BASE64('1Y4NGIukSbv9OrkZnRD1bQ=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // text - qt_sql "SELECT SM4_DECRYPT(FROM_BASE64('G5POcFAJwiZHeTtN6DjInQ=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // NULL - qt_sql "SELECT SM4_DECRYPT(FROM_BASE64('G5POcFAJwiZHeTtN6DjInQ=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789ff');" // text - - sql "set block_encryption_mode=\"SM4_128_CBC\";" - test { - sql "SELECT TO_BASE64(SM4_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" - exception "Incorrect parameter count in the call to native function 'sm4_encrypt'" - } - qt_sql "SELECT TO_BASE64(SM4_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // 1Y4NGIukSbv9OrkZnRD1bQ== - qt_sql "SELECT TO_BASE64(SM4_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789ff'));" // G5POcFAJwiZHeTtN6DjInQ== - test { - sql "SELECT SM4_DECRYPT_v2(FROM_BASE64('aDjwRflBrDjhBZIOFNw3Tg=='),'F3229A0B371ED2D9441B830D21A390C3');" - exception "Incorrect parameter count in the call to native function 'sm4_decrypt'" - } - qt_sql "SELECT SM4_DECRYPT_v2(FROM_BASE64('1Y4NGIukSbv9OrkZnRD1bQ=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // text - qt_sql "SELECT SM4_DECRYPT_v2(FROM_BASE64('G5POcFAJwiZHeTtN6DjInQ=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // NULL - qt_sql "SELECT SM4_DECRYPT_v2(FROM_BASE64('G5POcFAJwiZHeTtN6DjInQ=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789ff');" // text - test { - sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" - exception "session variable block_encryption_mode is invalid with aes" - } - qt_sql "SELECT SM3(\"abc\");" qt_sql "select sm3(\"abcd\");" qt_sql "select sm3sum(\"ab\",\"cd\");" diff --git a/regression-test/suites/nereids_rules_p0/max_min_filter_push_down/max_min_filter_push_down.groovy b/regression-test/suites/nereids_rules_p0/max_min_filter_push_down/max_min_filter_push_down.groovy new file mode 100644 index 00000000000000..47610f2e125701 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/max_min_filter_push_down/max_min_filter_push_down.groovy @@ -0,0 +1,214 @@ +// 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("max_min_filter_push_down") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + + sql "drop table if exists max_min_filter_push_down1" + sql""" + CREATE TABLE max_min_filter_push_down1 ( + id INT, + value1 INT, + value2 VARCHAR(50) + ) properties("replication_num"="1"); + """ + + sql """ + INSERT INTO max_min_filter_push_down1 (id, value1, value2) VALUES + (1, 10, 'A'),(1, 11, 'A'),(2, 20, 'B'),(2, 73, 'B'),(2, 19, 'B'),(3, 30, 'C'),(3, 61, 'C'),(4, 40, 'D'),(4, 43, 'D'),(4, 45, 'D'); + """ + sql "drop table if exists max_min_filter_push_down_empty" + sql "create table max_min_filter_push_down_empty like max_min_filter_push_down1" + + qt_scalar_agg_empty_table """ + explain shape plan + select min(value1) from max_min_filter_push_down_empty having min(value1) <40 and min(value1) <20; + """ + qt_min """ + explain shape plan + select id,min(value1) from max_min_filter_push_down1 group by id having min(value1) <40 and min(value1) <20; + """ + qt_max """ + explain shape plan + select id,max(value1) from max_min_filter_push_down1 group by id having max(value1) >40; + """ + + qt_min_expr """ + explain shape plan + select id,min(value1+1) from max_min_filter_push_down1 group by id having min(value1+1) <40 and min(value1+1) <20; + """ + qt_max_expr """ + explain shape plan + select id,max(abs(value1)+1) from max_min_filter_push_down1 group by id having max(abs(value1)+1) >40; + """ + + qt_min_commute """ + explain shape plan + select id,min(value1) from max_min_filter_push_down1 group by id having 40>min(value1); + """ + qt_max """ + explain shape plan + select id,max(value1) from max_min_filter_push_down1 group by id having 40=40; + """ + + qt_min_commute_equal """ + explain shape plan + select id,min(value1) from max_min_filter_push_down1 group by id having 40>=min(value1); + """ + qt_max_commute_equal """ + explain shape plan + select id,max(value1) from max_min_filter_push_down1 group by id having 40<=max(value1); + """ + + qt_has_other_agg_func """ + explain shape plan + select id,max(value1),min(value1) from max_min_filter_push_down1 group by id having 40<=max(value1); + """ + + qt_min_scalar_agg """ + explain shape plan + select min(value1) from max_min_filter_push_down1 having min(value1) <40; + """ + qt_max_scalar_agg """ + explain shape plan + select max(value1) from max_min_filter_push_down1 having max(value1) >40; + """ + qt_max_scalar_agg """ + explain shape plan + select max(value1) from max_min_filter_push_down1 having 40=40; + """ + + qt_depend_prune_column """ + explain shape plan + select c1 from (select min(value1) c1,max(value2) from max_min_filter_push_down1 group by id having min(value1)<10) t + """ + + qt_scalar_agg_empty_table_res """ + select min(value1) from max_min_filter_push_down_empty having min(value1) <40 and min(value1) <20; + """ + qt_min_res """ + select id,min(value1) from max_min_filter_push_down1 group by id having min(value1) <40 and min(value1) <20 order by 1,2; + """ + qt_max_res """ + select id,max(value1) from max_min_filter_push_down1 group by id having max(value1) >40 order by 1,2; + """ + qt_min_expr_res """ + select id,min(value1+1) from max_min_filter_push_down1 group by id having min(value1+1) <40 and min(value1+1) <20 order by 1,2; + """ + qt_max_expr_res """ + select id,max(abs(value1)+1) from max_min_filter_push_down1 group by id having max(abs(value1)+1) >40 order by 1,2; + """ + qt_min_commute_res """ + select id,min(value1) from max_min_filter_push_down1 group by id having 40>min(value1) order by 1,2; + """ + qt_max_res """ + select id,max(value1) from max_min_filter_push_down1 group by id having 40=40 order by 1,2; + """ + + qt_min_commute_equal_res """ + select id,min(value1) from max_min_filter_push_down1 group by id having 40>=min(value1) order by 1,2; + """ + qt_max_commute_equal_res """ + select id,max(value1) from max_min_filter_push_down1 group by id having 40<=max(value1) order by 1,2; + """ + + qt_has_other_agg_func_res """ + select id,max(value1),min(value1) from max_min_filter_push_down1 group by id having 40<=max(value1) order by 1,2; + """ + + qt_min_scalar_agg_res """ + select min(value1) from max_min_filter_push_down1 having min(value1) <40; + """ + qt_max_scalar_agg_res """ + select max(value1) from max_min_filter_push_down1 having max(value1) >40; + """ + qt_max_scalar_agg_res """ + select max(value1) from max_min_filter_push_down1 having 40=40; + """ + qt_depend_prune_column_res """ + select c1 from (select min(value1) c1,max(value2) from max_min_filter_push_down1 group by id having min(value1)<20) t order by c1 + """ + + sql "drop table if exists max_min_filter_push_down2" + sql """create table max_min_filter_push_down2(d_int int, d_char100 char(100), d_smallint smallint, d_tinyint tinyint, d_char10 char(10),d_datetimev2 datetimev2, d_datev2 datev2) + properties("replication_num"="1");""" + sql """insert into max_min_filter_push_down2 values(1,'01234567890123456789', 3,3,'0123456789','2020-01-09 10:00:00.99','2020-01-09') + ,(14,'01234567890123456789', 29,23,'0123456789','2020-01-7 10:00:00.99','2020-01-11'),(1,'01234567890123456789', 7,23,'0123456789','2020-01-7 10:00:00.99','2020-01-11') + ,(14,'01234567890123456789', 32,23,'0123456789','2020-01-11 10:00:00.99','2020-01-11'),(1,'01234567890123456789', 8,23,'0123456789','2020-01-11 10:00:00.99','2020-01-11');""" + + qt_smallint """explain shape plan + select d_int,max(d_smallint) from max_min_filter_push_down2 group by d_int having max(d_smallint)>10;""" + qt_tinyint """explain shape plan + select d_int,min(d_tinyint) from max_min_filter_push_down2 group by d_int having min(d_tinyint)<10;""" + qt_char100 """explain shape plan + select d_int,max(d_char100) from max_min_filter_push_down2 group by d_int having max(d_char100)>'ab';""" + qt_char100_cmp_num_cannot_rewrite """explain shape plan + select d_int,min(d_char100) from max_min_filter_push_down2 group by d_int having min(d_char100)<10;""" + qt_datetimev2 """explain shape plan + select d_int,min(d_datetimev2) from max_min_filter_push_down2 group by d_int having min(d_datetimev2)<'2020-01-09';""" + qt_datev2 """explain shape plan + select d_int,max(d_datev2) from max_min_filter_push_down2 group by d_int having max(d_datev2)>'2020-01-09 10:00:00';""" + qt_smallint_group_by_key """explain shape plan + select max(d_smallint) from max_min_filter_push_down2 group by d_smallint having max(d_smallint)>10;""" + qt_tinyint_group_by_key """explain shape plan + select min(d_tinyint) from max_min_filter_push_down2 group by d_tinyint having min(d_tinyint)<10;""" + qt_char100_group_by_key """explain shape plan + select max(d_char100) from max_min_filter_push_down2 group by d_char100 having max(d_char100)>'ab';""" + + qt_smallint_res """select d_int,max(d_smallint) from max_min_filter_push_down2 group by d_int having max(d_smallint)>10 order by 1,2;""" + qt_tinyint_res """select d_int,min(d_tinyint) from max_min_filter_push_down2 group by d_int having min(d_tinyint)<10 order by 1,2;""" + qt_char100_res """select d_int,max(d_char100) from max_min_filter_push_down2 group by d_int having max(d_char100)>'ab' order by 1,2;""" + qt_char100_cmp_num_cannot_rewrite_res """select d_int,min(d_char100) from max_min_filter_push_down2 group by d_int having min(d_char100)<10 order by 1,2;""" + qt_datetimev2_res """select d_int,min(d_datetimev2) from max_min_filter_push_down2 group by d_int having min(d_datetimev2)<'2020-01-09' order by 1,2;""" + qt_datev2_res """select d_int,max(d_datev2) from max_min_filter_push_down2 group by d_int having max(d_datev2)>'2020-01-09 10:00:00' order by 1,2;""" + qt_smallint_group_by_key_res """select max(d_smallint) from max_min_filter_push_down2 group by d_smallint having max(d_smallint)>10 order by 1;""" + qt_tinyint_group_by_key_res """select min(d_tinyint) from max_min_filter_push_down2 group by d_tinyint having min(d_tinyint)<10 order by 1;""" + qt_char100_group_by_key_res """select max(d_char100) from max_min_filter_push_down2 group by d_char100 having max(d_char100)>'ab' order by 1;""" +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/mv/ssb/mv_ssb_test.groovy b/regression-test/suites/nereids_rules_p0/mv/ssb/mv_ssb_test.groovy index 17849eea2cbf88..a80690a7bbef5a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/ssb/mv_ssb_test.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/ssb/mv_ssb_test.groovy @@ -61,7 +61,7 @@ suite("mv_ssb_test") { // relate to ${DORIS_HOME}/regression-test/data/demo/streamload_input.csv. // also, you can stream load a http stream, e.g. http://xxx/some.csv - file """${getS3Url()}/regression/ssb/sf1/${tableName}.tbl.gz""" + file """${getS3Url()}/regression/ssb/sf0.1/${tableName}.tbl.gz""" time 10000 // limit inflight 10s @@ -358,7 +358,7 @@ suite("mv_ssb_test") { OR c_city='UNITED KI5') AND (s_city='UNITED KI1' OR s_city='UNITED KI5') - AND d_yearmonth = 'Dec1997' + AND d_yearmonth = 'Jul1992' GROUP BY c_city, s_city, d_year ORDER BY d_year ASC, REVENUE DESC; """ @@ -373,7 +373,7 @@ suite("mv_ssb_test") { OR c_city='UNITED KI5') AND (s_city='UNITED KI1' OR s_city='UNITED KI5') - AND d_yearmonth = 'Dec1997' + AND d_yearmonth = 'Jul1992' GROUP BY c_city, s_city, d_year ORDER BY d_year ASC, REVENUE DESC; """ @@ -428,7 +428,7 @@ suite("mv_ssb_test") { AND lo_orderdate = d_datekey AND c_region = 'AMERICA' AND s_region = 'AMERICA' - AND (d_year = 1997 OR d_year = 1998) + AND (d_year = 1992 OR d_year = 1993) AND (p_mfgr = 'MFGR#1' OR p_mfgr = 'MFGR#2') GROUP BY d_year, s_nation, p_category @@ -444,7 +444,7 @@ suite("mv_ssb_test") { AND lo_orderdate = d_datekey AND c_region = 'AMERICA' AND s_region = 'AMERICA' - AND (d_year = 1997 OR d_year = 1998) + AND (d_year = 1992 OR d_year = 1993) AND (p_mfgr = 'MFGR#1' OR p_mfgr = 'MFGR#2') GROUP BY d_year, s_nation, p_category @@ -465,7 +465,7 @@ suite("mv_ssb_test") { AND lo_partkey = p_partkey AND lo_orderdate = d_datekey AND s_nation = 'UNITED STATES' - AND (d_year = 1997 OR d_year = 1998) + AND (d_year = 1992 OR d_year = 1993) AND p_category = 'MFGR#14' GROUP BY d_year, s_city, p_brand ORDER BY d_year, s_city, p_brand; @@ -479,7 +479,7 @@ suite("mv_ssb_test") { AND lo_partkey = p_partkey AND lo_orderdate = d_datekey AND s_nation = 'UNITED STATES' - AND (d_year = 1997 OR d_year = 1998) + AND (d_year = 1992 OR d_year = 1993) AND p_category = 'MFGR#14' GROUP BY d_year, s_city, p_brand ORDER BY d_year, s_city, p_brand; diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_dynamic.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_dynamic.groovy new file mode 100644 index 00000000000000..c5fa8a736dbf74 --- /dev/null +++ b/regression-test/suites/partition_p0/auto_partition/test_auto_dynamic.groovy @@ -0,0 +1,130 @@ + + +// 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_auto_dynamic", "nonConcurrent") { + // PROHIBIT different timeunit of interval when use both auto & dynamic partition + test{ + sql """ + CREATE TABLE tbl3 + ( + k1 DATETIME NOT NULL, + col1 int + ) + auto partition by range (date_trunc(`k1`, 'year')) () + DISTRIBUTED BY HASH(k1) + PROPERTIES + ( + "replication_num" = "1", + "dynamic_partition.create_history_partition"="true", + "dynamic_partition.enable" = "true", + "dynamic_partition.time_unit" = "HOUR", + "dynamic_partition.start" = "-2", + "dynamic_partition.end" = "2", + "dynamic_partition.prefix" = "p", + "dynamic_partition.buckets" = "8" + ); + """ + exception "If support auto partition and dynamic partition at same time, they must have the same interval unit." + } + + sql " drop table if exists test_dynamic " + sql """ + create table test_dynamic( + k0 DATE not null + ) + auto partition by range (date_trunc(k0, 'year')) () + DISTRIBUTED BY HASH(`k0`) BUCKETS auto + properties("replication_num" = "1"); + """ + test { + sql """ + ALTER TABLE test_dynamic set ( + "dynamic_partition.enable" = "true", + "dynamic_partition.time_unit" = "DAY", + "dynamic_partition.end" = "3", + "dynamic_partition.prefix" = "p", + "dynamic_partition.buckets" = "32" + ); + """ + exception "If support auto partition and dynamic partition at same time, they must have the same interval unit." + } + sql """ + ALTER TABLE test_dynamic set ( + "dynamic_partition.enable" = "true", + "dynamic_partition.time_unit" = "YeAr", + "dynamic_partition.end" = "3", + "dynamic_partition.prefix" = "p", + "dynamic_partition.buckets" = "32" + ); + """ + + sql " drop table if exists auto_dynamic " + sql """ + create table auto_dynamic( + k0 datetime(6) NOT NULL + ) + auto partition by range (date_trunc(k0, 'hour')) + ( + ) + DISTRIBUTED BY HASH(`k0`) BUCKETS 2 + properties( + "dynamic_partition.enable" = "true", + "dynamic_partition.prefix" = "p", + "dynamic_partition.create_history_partition" = "true", + "dynamic_partition.start" = "-5", + "dynamic_partition.end" = "0", + "dynamic_partition.time_unit" = "hour", + "replication_num" = "1" + ); + """ + def part_result = sql " show partitions from auto_dynamic " + assertEquals(part_result.size, 6) + + sql " drop table if exists auto_dynamic " + sql """ + create table auto_dynamic( + k0 datetime(6) NOT NULL + ) + auto partition by range (date_trunc(k0, 'year')) + ( + ) + DISTRIBUTED BY HASH(`k0`) BUCKETS 2 + properties( + "dynamic_partition.enable" = "true", + "dynamic_partition.prefix" = "p", + "dynamic_partition.start" = "-50", + "dynamic_partition.end" = "0", + "dynamic_partition.time_unit" = "year", + "replication_num" = "1" + ); + """ + part_result = sql " show partitions from auto_dynamic " + assertEquals(part_result.size, 1) + + sql " insert into auto_dynamic values ('2024-01-01'), ('2900-01-01'), ('1900-01-01'), ('3000-01-01'); " + sql """ admin set frontend config ('dynamic_partition_check_interval_seconds' = '1') """ + sleep(2000) + part_result = sql " show partitions from auto_dynamic " + log.info("${part_result}".toString()) + assertEquals(part_result.size, 3) + + qt_sql_dynamic_auto "select * from auto_dynamic order by k0;" + + sql """ admin set frontend config ('dynamic_partition_check_interval_seconds' = '600') """ +} \ No newline at end of file diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy index e5ce52af31e395..01fdfd4d634261 100644 --- a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy +++ b/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy @@ -217,117 +217,6 @@ suite("test_auto_partition_behavior") { - // dynamic + auto partition - sql """ admin set frontend config ('dynamic_partition_check_interval_seconds' = '1') """ - // PROHIBIT different timeunit of interval when use both auto & dynamic partition - test{ - sql """ - CREATE TABLE tbl3 - ( - k1 DATETIME NOT NULL, - col1 int - ) - auto partition by range (date_trunc(`k1`, 'year')) () - DISTRIBUTED BY HASH(k1) - PROPERTIES - ( - "replication_num" = "1", - "dynamic_partition.create_history_partition"="true", - "dynamic_partition.enable" = "true", - "dynamic_partition.time_unit" = "HOUR", - "dynamic_partition.start" = "-2", - "dynamic_partition.end" = "2", - "dynamic_partition.prefix" = "p", - "dynamic_partition.buckets" = "8" - ); - """ - exception "If support auto partition and dynamic partition at same time, they must have the same interval unit." - } - - sql " drop table if exists test_dynamic " - sql """ - create table test_dynamic( - k0 DATE not null - ) - auto partition by range (date_trunc(k0, 'year')) () - DISTRIBUTED BY HASH(`k0`) BUCKETS auto - properties("replication_num" = "1"); - """ - test { - sql """ - ALTER TABLE test_dynamic set ( - "dynamic_partition.enable" = "true", - "dynamic_partition.time_unit" = "DAY", - "dynamic_partition.end" = "3", - "dynamic_partition.prefix" = "p", - "dynamic_partition.buckets" = "32" - ); - """ - exception "If support auto partition and dynamic partition at same time, they must have the same interval unit." - } - sql """ - ALTER TABLE test_dynamic set ( - "dynamic_partition.enable" = "true", - "dynamic_partition.time_unit" = "YeAr", - "dynamic_partition.end" = "3", - "dynamic_partition.prefix" = "p", - "dynamic_partition.buckets" = "32" - ); - """ - - sql " drop table if exists auto_dynamic " - sql """ - create table auto_dynamic( - k0 datetime(6) NOT NULL - ) - auto partition by range (date_trunc(k0, 'hour')) - ( - ) - DISTRIBUTED BY HASH(`k0`) BUCKETS 2 - properties( - "dynamic_partition.enable" = "true", - "dynamic_partition.prefix" = "p", - "dynamic_partition.create_history_partition" = "true", - "dynamic_partition.start" = "-5", - "dynamic_partition.end" = "0", - "dynamic_partition.time_unit" = "hour", - "replication_num" = "1" - ); - """ - def part_result = sql " show partitions from auto_dynamic " - assertEquals(part_result.size, 6) - - sql " drop table if exists auto_dynamic " - sql """ - create table auto_dynamic( - k0 datetime(6) NOT NULL - ) - auto partition by range (date_trunc(k0, 'year')) - ( - ) - DISTRIBUTED BY HASH(`k0`) BUCKETS 2 - properties( - "dynamic_partition.enable" = "true", - "dynamic_partition.prefix" = "p", - "dynamic_partition.start" = "-50", - "dynamic_partition.end" = "0", - "dynamic_partition.time_unit" = "year", - "replication_num" = "1" - ); - """ - part_result = sql " show partitions from auto_dynamic " - assertEquals(part_result.size, 1) - sql " insert into auto_dynamic values ('2024-01-01'), ('2900-01-01'), ('1900-01-01'), ('3000-01-01'); " - sleep(3000) - part_result = sql " show partitions from auto_dynamic " - log.info("${part_result}".toString()) - assertEquals(part_result.size, 3) - qt_sql_dynamic_auto "select * from auto_dynamic order by k0;" - sql """ admin set frontend config ('dynamic_partition_check_interval_seconds' = '600') """ - - - - // prohibit too long value for partition column sql "drop table if exists `long_value`" sql """ diff --git a/regression-test/suites/query_p0/sql_functions/encryption_digest/test_encryption_function.groovy b/regression-test/suites/query_p0/sql_functions/encryption_digest/test_encryption_function.groovy index 4b0fd3d497d420..f4a67b052c3750 100644 --- a/regression-test/suites/query_p0/sql_functions/encryption_digest/test_encryption_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/encryption_digest/test_encryption_function.groovy @@ -16,10 +16,9 @@ // under the License. suite("test_encryption_function") { - def tableName = "dwd_candidates" - sql "DROP TABLE IF EXISTS ${tableName}" + sql "DROP TABLE IF EXISTS dwd_candidates" sql """ - CREATE TABLE IF NOT EXISTS ${tableName} ( + CREATE TABLE IF NOT EXISTS dwd_candidates ( c_int INT, `name` varchar(65530) NULL COMMENT "" ) @@ -28,7 +27,6 @@ suite("test_encryption_function") { "replication_num" = "1" ) """ - sql "set batch_size = 4096;" sql "set block_encryption_mode=\"AES_128_ECB\";" qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3'));" // 'wr2JEDVXzL9+2XtRhgIloA==' @@ -40,16 +38,6 @@ suite("test_encryption_function") { exception "session variable block_encryption_mode is invalid with sm4" } - sql "set block_encryption_mode=\"AES_128_ECB\";" - qt_sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" // 'wr2JEDVXzL9+2XtRhgIloA==' - qt_sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // wr2JEDVXzL9+2XtRhgIloA== - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('wr2JEDVXzL9+2XtRhgIloA=='),'F3229A0B371ED2D9441B830D21A390C3');" // text - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('mvZT1KJw7N0RJf27aipUpg=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // NULL - test { - sql "SELECT TO_BASE64(SM4_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" - exception "session variable block_encryption_mode is invalid with sm4" - } - sql "set block_encryption_mode=\"AES_256_ECB\";" qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3'));" // 'wr2JEDVXzL9+2XtRhgIloA==' qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // 'BO2vxHeUcw5BQQalSBbo1w==' @@ -57,13 +45,6 @@ suite("test_encryption_function") { qt_sql "SELECT AES_DECRYPT(FROM_BASE64('BO2vxHeUcw5BQQalSBbo1w=='),'F3229A0B371ED2D9441B830D21A390C3');" // NULL qt_sql "SELECT AES_DECRYPT(FROM_BASE64('BO2vxHeUcw5BQQalSBbo1w=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // text - sql "set block_encryption_mode=\"AES_256_ECB\";" - qt_sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" // 'BO2vxHeUcw5BQQalSBbo1w==' - qt_sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // 'BO2vxHeUcw5BQQalSBbo1w==' - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('wr2JEDVXzL9+2XtRhgIloA=='),'F3229A0B371ED2D9441B830D21A390C3');" // NULL - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('BO2vxHeUcw5BQQalSBbo1w=='),'F3229A0B371ED2D9441B830D21A390C3');" // text - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('BO2vxHeUcw5BQQalSBbo1w=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // text - sql "set block_encryption_mode=\"AES_256_CBC\";" qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3'));" // 'wr2JEDVXzL9+2XtRhgIloA==' qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // tsmK1HzbpnEdR2//WhO+MA== @@ -77,26 +58,6 @@ suite("test_encryption_function") { "GROUP BY x0;" } - sql "set block_encryption_mode=\"AES_256_CBC\";" - test { - sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" - exception "Incorrect parameter count in the call to native function 'aes_encrypt'" - } - qt_sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // tsmK1HzbpnEdR2//WhO+MA== - qt_sql "SELECT TO_BASE64(AES_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789ff'));" // ciacXDLHMNG7CD9Kws8png== - test { - sql "SELECT AES_DECRYPT_v2(FROM_BASE64('wr2JEDVXzL9+2XtRhgIloA=='),'F3229A0B371ED2D9441B830D21A390C3');" - exception "Incorrect parameter count in the call to native function 'aes_decrypt'" - } - qt_sql "SELECT AES_DECRYPT(FROM_BASE64('mvZT1KJw7N0RJf27aipUpg=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // NULL - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('mvZT1KJw7N0RJf27aipUpg=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // NULL - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('tsmK1HzbpnEdR2//WhO+MA=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // text - qt_sql "SELECT AES_DECRYPT_v2(FROM_BASE64('ciacXDLHMNG7CD9Kws8png=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789ff');" // text - explain { - sql "SELECT AES_DECRYPT_v2(UNHEX(r_2_3.`name`), 'namePnhe3E0MWyfZivUnVzDy12caymnrKp', '0123456789') AS x0 FROM dwd_candidates AS r_2_3\n" + - "GROUP BY x0;" - } - sql "set block_encryption_mode=\"SM4_128_CBC\";" qt_sql "SELECT TO_BASE64(SM4_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3'));" // aDjwRflBrDjhBZIOFNw3Tg== qt_sql "SELECT TO_BASE64(SM4_ENCRYPT('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // 1Y4NGIukSbv9OrkZnRD1bQ== @@ -110,25 +71,6 @@ suite("test_encryption_function") { exception "session variable block_encryption_mode is invalid with aes" } - sql "set block_encryption_mode=\"SM4_128_CBC\";" - test { - sql "SELECT TO_BASE64(SM4_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" - exception "Incorrect parameter count in the call to native function 'sm4_encrypt'" - } - qt_sql "SELECT TO_BASE64(SM4_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789'));" // 1Y4NGIukSbv9OrkZnRD1bQ== - qt_sql "SELECT TO_BASE64(SM4_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3', '0123456789ff'));" // G5POcFAJwiZHeTtN6DjInQ== - test { - sql "SELECT SM4_DECRYPT_v2(FROM_BASE64('aDjwRflBrDjhBZIOFNw3Tg=='),'F3229A0B371ED2D9441B830D21A390C3');" - exception "Incorrect parameter count in the call to native function 'sm4_decrypt'" - } - qt_sql "SELECT SM4_DECRYPT_v2(FROM_BASE64('1Y4NGIukSbv9OrkZnRD1bQ=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // text - qt_sql "SELECT SM4_DECRYPT_v2(FROM_BASE64('G5POcFAJwiZHeTtN6DjInQ=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789');" // NULL - qt_sql "SELECT SM4_DECRYPT_v2(FROM_BASE64('G5POcFAJwiZHeTtN6DjInQ=='),'F3229A0B371ED2D9441B830D21A390C3', '0123456789ff');" // text - test { - sql "SELECT TO_BASE64(AES_ENCRYPT_v2('text','F3229A0B371ED2D9441B830D21A390C3'));" - exception "session variable block_encryption_mode is invalid with aes" - } - qt_sql "SELECT SM3(\"abc\");" qt_sql "select sm3(\"abcd\");" qt_sql "select sm3sum(\"ab\",\"cd\");" diff --git a/regression-test/suites/query_p0/sql_functions/math_functions/test_math_unary_always_nullable.groovy b/regression-test/suites/query_p0/sql_functions/math_functions/test_math_unary_always_nullable.groovy new file mode 100644 index 00000000000000..282d4e3c5754e4 --- /dev/null +++ b/regression-test/suites/query_p0/sql_functions/math_functions/test_math_unary_always_nullable.groovy @@ -0,0 +1,85 @@ +// 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_math_unary_alway_nullable") { + sql """ + set debug_skip_fold_constant=true; + """ + + qt_acos_1 """ + select acos(1.1), acos(1.1) is null; + """ + qt_acos_2 """ + select acos(-1.1), acos(-1.1) is null; + """ + qt_acos_3 """ + select acos(-1.1), acos(-1.1) is NULL, number from numbers("number"="10") + """ + + qt_asin_1 """ + select asin(1.1), asin(1.1) is null; + """ + qt_asin_2 """ + select asin(-1.1), asin(-1.1) is null; + """ + qt_asin_3 """ + select asin(-1.1), asin(-1.1) is NULL, number from numbers("number"="10") + """ + + qt_sqrt_1 """ + select sqrt(-1), sqrt(-1) is null; + """ + qt_sqrt_2 """ + select sqrt(-1.1), sqrt(-1.1) is null; + """ + qt_sqrt_3 """ + select sqrt(-1.1), sqrt(-1.1) is NULL, number from numbers("number"="10") + """ + + sql "drop table if exists test_math_unary_alway_nullable" + + sql """ + create table if not exists test_math_unary_alway_nullable (rowid int, val double NULL) + distributed by hash(rowid) properties ("replication_num"="1"); + """ + + sql """ + insert into test_math_unary_alway_nullable values + (1, 1.1), (2, -1.1), (3, 0), (4, NULL) + """ + sql """ + insert into test_math_unary_alway_nullable values + (5, NULL), (6, NULL), (7, NULL), (8, NULL) + """ + + qt_acos_tbl_1 """ + select rowid, acos(val), acos(val) is null from test_math_unary_alway_nullable order by rowid; + """ + + qt_asin_tbl_1 """ + select rowid, asin(val), asin(val) is null from test_math_unary_alway_nullable order by rowid; + """ + + qt_sqrt_tbl_1 """ + select rowid, sqrt(val), sqrt(val) is null from test_math_unary_alway_nullable order by rowid; + """ + + qt_dsqrt_tbl_1 """ + select rowid, dsqrt(val), dsqrt(val) is null from test_math_unary_alway_nullable order by rowid; + """ + +} \ No newline at end of file diff --git a/regression-test/suites/query_profile/adaptive_pipeline_task_serial_read_on_limit.groovy b/regression-test/suites/query_profile/adaptive_pipeline_task_serial_read_on_limit.groovy new file mode 100644 index 00000000000000..15481fe7c9a8dc --- /dev/null +++ b/regression-test/suites/query_profile/adaptive_pipeline_task_serial_read_on_limit.groovy @@ -0,0 +1,175 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import groovy.json.JsonOutput +import groovy.json.JsonSlurper +import groovy.json.StringEscapeUtils + + +def getProfileList = { + def dst = 'http://' + context.config.feHttpAddress + def conn = new URL(dst + "/rest/v1/query_profile").openConnection() + conn.setRequestMethod("GET") + def encoding = Base64.getEncoder().encodeToString((context.config.feHttpUser + ":" + + (context.config.feHttpPassword == null ? "" : context.config.feHttpPassword)).getBytes("UTF-8")) + conn.setRequestProperty("Authorization", "Basic ${encoding}") + return conn.getInputStream().getText() +} + + +def getProfile = { id -> + def dst = 'http://' + context.config.feHttpAddress + def conn = new URL(dst + "/api/profile/text/?query_id=$id").openConnection() + conn.setRequestMethod("GET") + def encoding = Base64.getEncoder().encodeToString((context.config.feHttpUser + ":" + + (context.config.feHttpPassword == null ? "" : context.config.feHttpPassword)).getBytes("UTF-8")) + conn.setRequestProperty("Authorization", "Basic ${encoding}") + return conn.getInputStream().getText() +} + +suite('adaptive_pipeline_task_serial_read_on_limit') { + sql """ + DROP TABLE IF EXISTS adaptive_pipeline_task_serial_read_on_limit; + """ + sql """ + CREATE TABLE if not exists `adaptive_pipeline_task_serial_read_on_limit` ( + `id` INT, + `name` varchar(32) + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + // Insert data to table + sql """ + insert into adaptive_pipeline_task_serial_read_on_limit values + (1, "A"),(2, "B"),(3, "C"),(4, "D"),(5,"E"),(6,"F"),(7,"G"),(8,"H"),(9,"K"); + """ + sql """ + insert into adaptive_pipeline_task_serial_read_on_limit values + (10, "A"),(20, "B"),(30, "C"),(40, "D"),(50,"E"),(60,"F"),(70,"G"),(80,"H"),(90,"K"); + """ + sql """ + insert into adaptive_pipeline_task_serial_read_on_limit values + (101, "A"),(201, "B"),(301, "C"),(401, "D"),(501,"E"),(601,"F"),(701,"G"),(801,"H"),(901,"K"); + """ + sql """ + insert into adaptive_pipeline_task_serial_read_on_limit values + (1010, "A"),(2010, "B"),(3010, "C"),(4010, "D"),(5010,"E"),(6010,"F"),(7010,"G"),(8010,"H"),(9010,"K"); + """ + + def uuidString = UUID.randomUUID().toString() + sql "set enable_profile=true" + // set parallel_pipeline_task_num to 1 so that only one scan node, + // and we can check MaxScannerThreadNum in profile. + sql "set parallel_pipeline_task_num=1;" + // no limit, MaxScannerThreadNum = TabletNum + sql """ + select "no_limit_1_${uuidString}", * from adaptive_pipeline_task_serial_read_on_limit; + """ + sql "set parallel_pipeline_task_num=0;" + // With Limit, MaxScannerThreadNum = 1 + sql """ + select "with_limit_1_${uuidString}", * from adaptive_pipeline_task_serial_read_on_limit limit 10000; + """ + // With Limit, but bigger then adaptive_pipeline_task_serial_read_on_limit, MaxScannerThreadNum = TabletNum + sql """ + select "with_limit_2_${uuidString}", * from adaptive_pipeline_task_serial_read_on_limit limit 10001; + """ + sql """ + set enable_adaptive_pipeline_task_serial_read_on_limit=false; + """ + sql "set parallel_pipeline_task_num=1;" + // Forbid the strategy, with limit, MaxScannerThreadNum = TabletNum + sql """ + select "not_enable_limit_${uuidString}", * from adaptive_pipeline_task_serial_read_on_limit limit 100; + """ + + sql "set parallel_pipeline_task_num=0;" + + // Enable the strategy, with limit 20, MaxScannerThreadNum = 1 + sql """ + set enable_adaptive_pipeline_task_serial_read_on_limit=true; + """ + sql """ + set adaptive_pipeline_task_serial_read_on_limit=10; + """ + sql """ + select "modify_to_20_${uuidString}", * from adaptive_pipeline_task_serial_read_on_limit limit 15; + """ + + sql "set enable_profile=false" + + def wholeString = getProfileList() + List profileData = new JsonSlurper().parseText(wholeString).data.rows + String queryIdNoLimit1 = ""; + String queryIdWithLimit1 = ""; + String queryIdWithLimit2 = ""; + String queryIDNotEnableLimit = ""; + String queryIdModifyTo20 = ""; + + logger.info("{}", uuidString) + + for (def profileItem in profileData) { + if (profileItem["Sql Statement"].toString().contains("no_limit_1_${uuidString}")) { + queryIdNoLimit1 = profileItem["Profile ID"].toString() + logger.info("profileItem: {}", profileItem) + } + if (profileItem["Sql Statement"].toString().contains("with_limit_1_${uuidString}")) { + queryIdWithLimit1 = profileItem["Profile ID"].toString() + logger.info("profileItem: {}", profileItem) + } + if (profileItem["Sql Statement"].toString().contains("with_limit_2_${uuidString}")) { + queryIdWithLimit2 = profileItem["Profile ID"].toString() + logger.info("profileItem: {}", profileItem) + } + if (profileItem["Sql Statement"].toString().contains("not_enable_limit_${uuidString}")) { + queryIDNotEnableLimit = profileItem["Profile ID"].toString() + logger.info("profileItem: {}", profileItem) + } + if (profileItem["Sql Statement"].toString().contains("modify_to_20_${uuidString}")) { + queryIdModifyTo20 = profileItem["Profile ID"].toString() + logger.info("profileItem: {}", profileItem) + } + } + + logger.info("queryIdNoLimit1_${uuidString}: {}", queryIdNoLimit1) + logger.info("queryIdWithLimit1_${uuidString}: {}", queryIdWithLimit1) + logger.info("queryIdWithLimit2_${uuidString}: {}", queryIdWithLimit2) + logger.info("queryIDNotEnableLimit_${uuidString}: {}", queryIDNotEnableLimit) + logger.info("queryIdModifyTo20_${uuidString}: {}", queryIdModifyTo20) + + assertTrue(queryIdNoLimit1 != "") + assertTrue(queryIdWithLimit1 != "") + assertTrue(queryIdWithLimit2 != "") + assertTrue(queryIDNotEnableLimit != "") + assertTrue(queryIdModifyTo20 != "") + + def String profileNoLimit1 = getProfile(queryIdNoLimit1).toString() + def String profileWithLimit1 = getProfile(queryIdWithLimit1).toString() + def String profileWithLimit2 = getProfile(queryIdWithLimit2).toString() + def String profileNotEnableLimit = getProfile(queryIDNotEnableLimit).toString() + def String profileModifyTo20 = getProfile(queryIdModifyTo20).toString() + + assertTrue(profileNoLimit1.contains("- MaxScannerThreadNum: 10")) + assertTrue(profileWithLimit1.contains("- MaxScannerThreadNum: 1")) + assertTrue(profileWithLimit2.contains("- MaxScannerThreadNum: 10")) + assertTrue(profileNotEnableLimit.contains("- MaxScannerThreadNum: 10")) + assertTrue(profileModifyTo20.contains("- MaxScannerThreadNum: 1")) +} \ No newline at end of file diff --git a/regression-test/suites/variant_p0/load.groovy b/regression-test/suites/variant_p0/load.groovy index d74155b182ddfa..967ff63f0baa94 100644 --- a/regression-test/suites/variant_p0/load.groovy +++ b/regression-test/suites/variant_p0/load.groovy @@ -290,7 +290,7 @@ suite("regression_test_variant", "p0"){ sql """insert into ${table_name} values (5, '{"i" : 1}'), (1, '{"a" : 1}')""" sql """insert into ${table_name} values (6, '{"j" : 1}'), (1, '{"a" : 1}')""" sql """insert into ${table_name} values (6, '{"k" : 1}'), (1, '{"a" : 1}')""" - sql "select * from ${table_name}" + sql "select /*+SET_VAR(batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=1,parallel_pipeline_task_num=4,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=16,parallel_scan_min_rows_per_scanner=128,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=2,enable_parallel_result_sink=false,sort_phase_num=0,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=true,enable_local_exchange=true,partitioned_hash_join_rows_threshold=1048576,partitioned_hash_agg_rows_threshold=8,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=true,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=false,enable_delete_sub_predicate_v2=true,min_revocable_mem=33554432,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_join_spill=false,enable_sort_spill=false,enable_agg_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5) */ * from ${table_name}" qt_sql_36_1 "select cast(v['a'] as int), cast(v['b'] as int), cast(v['c'] as int) from ${table_name} order by k limit 10" sql "DELETE FROM ${table_name} WHERE k=1" sql "select * from ${table_name}" diff --git a/samples/datalake/deltalake_and_kudu/README.md b/samples/datalake/deltalake_and_kudu/README.md new file mode 100644 index 00000000000000..382a20ee0e4a2d --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/README.md @@ -0,0 +1,147 @@ + + +# Doris + DeltaLake + Kudu + MINIO Environments +Launch spark / doris / hive / deltalake / kudu /minio test environments, and give examples to query deltalake and kudu tables in Doris. + +## Launch Docker Compose +**Create Network** +```shell +sudo docker network create -d bridge trinoconnector-net +``` +**Launch all components in docker** +```shell +sudo sh start-trinoconnector-compose.sh +``` +**Login into Spark** +```shell +sudo sh login-spark.sh +``` +**Login into Doris** +```shell +sudo sh login-doris.sh +``` + +## Prepare DeltaLake Data +There's already a deltalake table named `customer` in default database. + +## Create Catalog +The Doris Cluster has created two catalogs called `delta_lake` and `kudu_catalog`. You can view both of them by using the `SHOW CATALOGS` command or the `SHOW CREATE CATALOG ${catalog_name}` command after you log in to the Doris. Here are the creation statements for the two catalogs: + +```sql +-- The catalog has been created, and no further action is required. +create catalog delta_lake properties ( + "type"="trino-connector", + "trino.connector.name"="delta_lake", + "trino.hive.metastore.uri"="thrift://hive-metastore:9083", + "trino.hive.s3.endpoint"="http://minio:9000", + "trino.hive.s3.region"="us-east-1", + "trino.hive.s3.aws-access-key"="minio", + "trino.hive.s3.aws-secret-key"="minio123", + "trino.hive.s3.path-style-access"="true" +); + +-- The catalog has been created, and no further action is required. +CREATE CATALOG `kudu_catalog` PROPERTIES ( + "type" = "trino-connector", + "trino.connector.name" = "kudu", + "trino.kudu.authentication.type" = "NONE", + "trino.kudu.client.master-addresses" = "kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251" +); +``` + +## Query Catalog Data +The data of `Delta Lake` and `Kudu` have been prepared in Doris Cluster. You can select these data directly in Doris. + +- select Delta Lake data + +```sql +mysql> switch delta_lake; +Query OK, 0 rows affected (0.00 sec) + +mysql> use default; +Reading table information for completion of table and column names +You can turn off this feature to get a quicker startup with -A + +Database changed +mysql> select * from customer limit 10; ++-----------+--------------------+------------------------------------+-------------+-----------------+-----------+--------------+---------------------------------------------------------------------------------------------------------------+ +| c_custkey | c_name | c_address | c_nationkey | c_phone | c_acctbal | c_mktsegment | c_comment | ++-----------+--------------------+------------------------------------+-------------+-----------------+-----------+--------------+---------------------------------------------------------------------------------------------------------------+ +| 2 | Customer#000000002 | XSTf4,NCwDVaWNe6tEgvwfmRchLXak | 13 | 23-768-687-3665 | 121.65 | AUTOMOBILE | l accounts. blithely ironic theodolites integrate boldly: caref | +| 34 | Customer#000000034 | Q6G9wZ6dnczmtOx509xgE,M2KV | 15 | 25-344-968-5422 | 8589.70 | HOUSEHOLD | nder against the even, pending accounts. even | +| 66 | Customer#000000066 | XbsEqXH1ETbJYYtA1A | 22 | 32-213-373-5094 | 242.77 | HOUSEHOLD | le slyly accounts. carefully silent packages benea | +| 98 | Customer#000000098 | 7yiheXNSpuEAwbswDW | 12 | 22-885-845-6889 | -551.37 | BUILDING | ages. furiously pending accounts are quickly carefully final foxes: busily pe | +| 130 | Customer#000000130 | RKPx2OfZy0Vn 8wGWZ7F2EAvmMORl1k8iH | 9 | 19-190-993-9281 | 5073.58 | HOUSEHOLD | ix slowly. express packages along the furiously ironic requests integrate daringly deposits. fur | +| 162 | Customer#000000162 | JE398sXZt2QuKXfJd7poNpyQFLFtth | 8 | 18-131-101-2267 | 6268.99 | MACHINERY | accounts along the doggedly special asymptotes boost blithely during the quickly regular theodolites. slyly | +| 194 | Customer#000000194 | mksKhdWuQ1pjbc4yffHp8rRmLOMcJ | 16 | 26-597-636-3003 | 6696.49 | HOUSEHOLD | quickly across the fluffily dogged requests. regular platelets around the ironic, even requests cajole quickl | +| 226 | Customer#000000226 | ToEmqB90fM TkLqyEgX8MJ8T8NkK | 3 | 13-452-318-7709 | 9008.61 | AUTOMOBILE | ic packages. ideas cajole furiously slyly special theodolites: carefully express pinto beans acco | +| 258 | Customer#000000258 | 7VbADek8qYezQYotxNUmnNI | 12 | 22-278-425-9944 | 6022.27 | MACHINERY | about the regular, bold accounts; pending packages use furiously stealthy warhorses. bold accounts sleep fur | +| 290 | Customer#000000290 | 8OlPT9G 8UqVXmVZNbmxVTPO8 | 4 | 14-458-625-5633 | 1811.35 | MACHINERY | sts. blithely pending requests sleep fluffily on the regular excuses. carefully expre | ++-----------+--------------------+------------------------------------+-------------+-----------------+-----------+--------------+---------------------------------------------------------------------------------------------------------------+ +10 rows in set (0.12 sec) +``` + +- select Kudu data + +```sql +mysql> switch kudu_catalog; +Query OK, 0 rows affected (0.00 sec) + +mysql> use default; +Reading table information for completion of table and column names +You can turn off this feature to get a quicker startup with -A + +Database changed + +mysql> select * from test_table limit 10; ++------+----------+--------+ +| key | value | added | ++------+----------+--------+ +| 0 | NULL | 12.345 | +| 4 | NULL | 12.345 | +| 20 | NULL | 12.345 | +| 26 | NULL | 12.345 | +| 29 | value 29 | 12.345 | +| 42 | NULL | 12.345 | +| 50 | NULL | 12.345 | +| 56 | NULL | 12.345 | +| 66 | NULL | 12.345 | +| 74 | NULL | 12.345 | ++------+----------+--------+ +10 rows in set (1.49 sec) +``` + +- federation query + +```sql +mysql> select * from delta_lake.`default`.customer c join kudu_catalog.`default`.test_table t on c.c_custkey = t.`key` where c.c_custkey < 50; ++-----------+--------------------+---------------------------------------+-------------+-----------------+-----------+--------------+--------------------------------------------------------------------------------------------------------+------+----------+--------+ +| c_custkey | c_name | c_address | c_nationkey | c_phone | c_acctbal | c_mktsegment | c_comment | key | value | added | ++-----------+--------------------+---------------------------------------+-------------+-----------------+-----------+--------------+--------------------------------------------------------------------------------------------------------+------+----------+--------+ +| 1 | Customer#000000001 | IVhzIApeRb ot,c,E | 15 | 25-989-741-2988 | 711.56 | BUILDING | to the even, regular platelets. regular, ironic epitaphs nag e | 1 | value 1 | 12.345 | +| 33 | Customer#000000033 | qFSlMuLucBmx9xnn5ib2csWUweg D | 17 | 27-375-391-1280 | -78.56 | AUTOMOBILE | s. slyly regular accounts are furiously. carefully pending requests | 33 | value 33 | 12.345 | +| 3 | Customer#000000003 | MG9kdTD2WBHm | 1 | 11-719-748-3364 | 7498.12 | AUTOMOBILE | deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov | 3 | value 3 | 12.345 | +| 35 | Customer#000000035 | TEjWGE4nBzJL2 | 17 | 27-566-888-7431 | 1228.24 | HOUSEHOLD | requests. special, express requests nag slyly furiousl | 35 | value 35 | 12.345 | +| 2 | Customer#000000002 | XSTf4,NCwDVaWNe6tEgvwfmRchLXak | 13 | 23-768-687-3665 | 121.65 | AUTOMOBILE | l accounts. blithely ironic theodolites integrate boldly: caref | 2 | NULL | 12.345 | +| 34 | Customer#000000034 | Q6G9wZ6dnczmtOx509xgE,M2KV | 15 | 25-344-968-5422 | 8589.70 | HOUSEHOLD | nder against the even, pending accounts. even | 34 | NULL | 12.345 | +| 32 | Customer#000000032 | jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J | 15 | 25-430-914-2194 | 3471.53 | BUILDING | cial ideas. final, furious requests across the e | 32 | NULL | 12.345 | ++-----------+--------------------+---------------------------------------+-------------+-----------------+-----------+--------------+--------------------------------------------------------------------------------------------------------+------+----------+--------+ +7 rows in set (0.13 sec) +``` diff --git a/samples/datalake/deltalake_and_kudu/data/customer/_delta_log/00000000000000000000.json b/samples/datalake/deltalake_and_kudu/data/customer/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000000..a5d9fc6052bd21 --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/data/customer/_delta_log/00000000000000000000.json @@ -0,0 +1,7 @@ +{"commitInfo":{"timestamp":1724747485883,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"4","numOutputRows":"18751","numOutputBytes":"1564827"},"engineInfo":"Apache-Spark/3.4.2 Delta-Lake/2.4.0","txnId":"1646d68c-f6f2-4da5-a9bf-56318b2b7216"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"421eb35b-e9ec-44ed-92fd-25e0fda91036","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"c_custkey\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c_address\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c_nationkey\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c_phone\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c_acctbal\",\"type\":\"decimal(12,2)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c_mktsegment\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c_comment\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1724747483772}} +{"add":{"path":"part-00000-44ff362c-110d-44ca-aed8-93ed65c19492-c000.snappy.parquet","partitionValues":{},"size":392744,"modificationTime":1724747485000,"dataChange":true,"stats":"{\"numRecords\":4688,\"minValues\":{\"c_custkey\":2,\"c_name\":\"Customer#000000002\",\"c_address\":\" UfkcgKnrSL0VRSDuuXjXW,\",\"c_nationkey\":0,\"c_phone\":\"10-103-318-6809\",\"c_acctbal\":-998.90,\"c_mktsegment\":\"AUTOMOBILE\",\"c_comment\":\" Tiresias detect always about \"},\"maxValues\":{\"c_custkey\":149986,\"c_name\":\"Customer#000149986\",\"c_address\":\"zyTHzirSOvDeqwIs4R7qn76825FPYr8Y�\",\"c_nationkey\":24,\"c_phone\":\"34-999-195-7029\",\"c_acctbal\":9994.63,\"c_mktsegment\":\"MACHINERY\",\"c_comment\":\"ze! special, even deposits nag q�\"},\"nullCount\":{\"c_custkey\":0,\"c_name\":0,\"c_address\":0,\"c_nationkey\":0,\"c_phone\":0,\"c_acctbal\":0,\"c_mktsegment\":0,\"c_comment\":0}}"}} +{"add":{"path":"part-00001-749ded2d-a84b-4e2c-9e6f-5ac6a59ee91d-c000.snappy.parquet","partitionValues":{},"size":392284,"modificationTime":1724747485000,"dataChange":true,"stats":"{\"numRecords\":4687,\"minValues\":{\"c_custkey\":32,\"c_name\":\"Customer#000000032\",\"c_address\":\" FjVZqAg2Pd9jhTN8pVD4DkvmxlCxMm\",\"c_nationkey\":0,\"c_phone\":\"10-105-777-9167\",\"c_acctbal\":-994.43,\"c_mktsegment\":\"AUTOMOBILE\",\"c_comment\":\" about the fluffily bold ideas. \"},\"maxValues\":{\"c_custkey\":149984,\"c_name\":\"Customer#000149984\",\"c_address\":\"zz5LSqGU2QoyQTcMzkOxVqWrHedmhqQ6�\",\"c_nationkey\":24,\"c_phone\":\"34-997-204-5897\",\"c_acctbal\":9998.01,\"c_mktsegment\":\"MACHINERY\",\"c_comment\":\"zzle quickly bold packages. sile�\"},\"nullCount\":{\"c_custkey\":0,\"c_name\":0,\"c_address\":0,\"c_nationkey\":0,\"c_phone\":0,\"c_acctbal\":0,\"c_mktsegment\":0,\"c_comment\":0}}"}} +{"add":{"path":"part-00002-137a1b68-bafd-46a7-b231-400a174b520c-c000.snappy.parquet","partitionValues":{},"size":390594,"modificationTime":1724747485000,"dataChange":true,"stats":"{\"numRecords\":4688,\"minValues\":{\"c_custkey\":1,\"c_name\":\"Customer#000000001\",\"c_address\":\" NUi8asf651zG096JTGeXdh\",\"c_nationkey\":0,\"c_phone\":\"10-100-220-4520\",\"c_acctbal\":-999.55,\"c_mktsegment\":\"AUTOMOBILE\",\"c_comment\":\" Tiresias detect slyly according\"},\"maxValues\":{\"c_custkey\":149985,\"c_name\":\"Customer#000149985\",\"c_address\":\"zzbUlYAy9rhCprBVHlzA\",\"c_nationkey\":24,\"c_phone\":\"34-999-363-7145\",\"c_acctbal\":9997.80,\"c_mktsegment\":\"MACHINERY\",\"c_comment\":\"zzle blithely against the carefu�\"},\"nullCount\":{\"c_custkey\":0,\"c_name\":0,\"c_address\":0,\"c_nationkey\":0,\"c_phone\":0,\"c_acctbal\":0,\"c_mktsegment\":0,\"c_comment\":0}}"}} +{"add":{"path":"part-00003-75203d54-ef95-4fbe-95c9-2012fd9dbaed-c000.snappy.parquet","partitionValues":{},"size":389205,"modificationTime":1724747485000,"dataChange":true,"stats":"{\"numRecords\":4688,\"minValues\":{\"c_custkey\":3,\"c_name\":\"Customer#000000003\",\"c_address\":\" 821GWWou3sOyp,\",\"c_nationkey\":0,\"c_phone\":\"10-105-204-5643\",\"c_acctbal\":-999.95,\"c_mktsegment\":\"AUTOMOBILE\",\"c_comment\":\" about the fluffily regular asym\"},\"maxValues\":{\"c_custkey\":149987,\"c_name\":\"Customer#000149987\",\"c_address\":\"zwrbKxxY yL Go\",\"c_nationkey\":24,\"c_phone\":\"34-999-283-6448\",\"c_acctbal\":9997.73,\"c_mktsegment\":\"MACHINERY\",\"c_comment\":\"ymptotes. unusual theodolites ab�\"},\"nullCount\":{\"c_custkey\":0,\"c_name\":0,\"c_address\":0,\"c_nationkey\":0,\"c_phone\":0,\"c_acctbal\":0,\"c_mktsegment\":0,\"c_comment\":0}}"}} diff --git a/samples/datalake/deltalake_and_kudu/data/customer/part-00000-44ff362c-110d-44ca-aed8-93ed65c19492-c000.snappy.parquet b/samples/datalake/deltalake_and_kudu/data/customer/part-00000-44ff362c-110d-44ca-aed8-93ed65c19492-c000.snappy.parquet new file mode 100644 index 00000000000000..db5f6570c5cfec Binary files /dev/null and b/samples/datalake/deltalake_and_kudu/data/customer/part-00000-44ff362c-110d-44ca-aed8-93ed65c19492-c000.snappy.parquet differ diff --git a/samples/datalake/deltalake_and_kudu/data/customer/part-00001-749ded2d-a84b-4e2c-9e6f-5ac6a59ee91d-c000.snappy.parquet b/samples/datalake/deltalake_and_kudu/data/customer/part-00001-749ded2d-a84b-4e2c-9e6f-5ac6a59ee91d-c000.snappy.parquet new file mode 100644 index 00000000000000..7abc7ccd604c71 Binary files /dev/null and b/samples/datalake/deltalake_and_kudu/data/customer/part-00001-749ded2d-a84b-4e2c-9e6f-5ac6a59ee91d-c000.snappy.parquet differ diff --git a/samples/datalake/deltalake_and_kudu/data/customer/part-00002-137a1b68-bafd-46a7-b231-400a174b520c-c000.snappy.parquet b/samples/datalake/deltalake_and_kudu/data/customer/part-00002-137a1b68-bafd-46a7-b231-400a174b520c-c000.snappy.parquet new file mode 100644 index 00000000000000..020abd85fc8bfb Binary files /dev/null and b/samples/datalake/deltalake_and_kudu/data/customer/part-00002-137a1b68-bafd-46a7-b231-400a174b520c-c000.snappy.parquet differ diff --git a/samples/datalake/deltalake_and_kudu/data/customer/part-00003-75203d54-ef95-4fbe-95c9-2012fd9dbaed-c000.snappy.parquet b/samples/datalake/deltalake_and_kudu/data/customer/part-00003-75203d54-ef95-4fbe-95c9-2012fd9dbaed-c000.snappy.parquet new file mode 100644 index 00000000000000..458f5c5f4d2608 Binary files /dev/null and b/samples/datalake/deltalake_and_kudu/data/customer/part-00003-75203d54-ef95-4fbe-95c9-2012fd9dbaed-c000.snappy.parquet differ diff --git a/samples/datalake/deltalake_and_kudu/login-doris.sh b/samples/datalake/deltalake_and_kudu/login-doris.sh new file mode 100755 index 00000000000000..90aaaa100bc5d9 --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/login-doris.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +# 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. + +docker exec -it spark-hive mysql -u root -h doris-env -P 9030 diff --git a/samples/datalake/deltalake_and_kudu/login-spark.sh b/samples/datalake/deltalake_and_kudu/login-spark.sh new file mode 100755 index 00000000000000..6a55539dcefc1d --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/login-spark.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +# 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. + +docker exec -it spark-hive /opt/scripts/spark-delta.sh diff --git a/samples/datalake/deltalake_and_kudu/scripts/create-delta-table.sh b/samples/datalake/deltalake_and_kudu/scripts/create-delta-table.sh new file mode 100644 index 00000000000000..9cf5a14ca3a45a --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/scripts/create-delta-table.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash + +# 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. + +export SPARK_HOME=/opt/spark +export HIVE_HOME=/opt/apache-hive-3.1.2-bin +export HADOOP_HOME=/opt/hadoop-3.3.1 + +if [[ ! -d "${SPARK_HOME}" ]]; then + cp -r /opt/spark-3.4.2-bin-hadoop3 "${SPARK_HOME}" +fi + +cp "${HIVE_HOME}"/conf/hive-site.xml "${SPARK_HOME}"/conf/ +cp "${HIVE_HOME}"/lib/postgresql-jdbc.jar "${SPARK_HOME}"/jars/ +cp "${HADOOP_HOME}"/etc/hadoop/core-site.xml "${SPARK_HOME}"/conf/ + +"${SPARK_HOME}"/bin/spark-sql \ + --master local[*] \ + --name "spark-delta-sql" \ + --conf "spark.sql.extensions=io.delta.sql.DeltaSparkSessionExtension" \ + --conf "spark.sql.catalog.spark_catalog=org.apache.spark.sql.delta.catalog.DeltaCatalog" \ + -f /opt/scripts/spark-delta.sql diff --git a/samples/datalake/deltalake_and_kudu/scripts/doris-sql.sql b/samples/datalake/deltalake_and_kudu/scripts/doris-sql.sql new file mode 100644 index 00000000000000..8d003a1f31706c --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/scripts/doris-sql.sql @@ -0,0 +1,20 @@ +create catalog delta_lake properties ( + "type"="trino-connector", + "trino.connector.name"="delta_lake", + "trino.hive.metastore.uri"="thrift://hive-metastore:9083", + "trino.hive.s3.endpoint"="http://minio:9000", + "trino.hive.s3.region"="us-east-1", + "trino.hive.s3.aws-access-key"="minio", + "trino.hive.s3.aws-secret-key"="minio123", + "trino.hive.s3.path-style-access"="true" +); + + +CREATE CATALOG `kudu_catalog` PROPERTIES ( + "type" = "trino-connector", + "trino.connector.name" = "kudu", + "trino.kudu.authentication.type" = "NONE", + "trino.kudu.client.master-addresses" = "kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251" +); + +ALTER SYSTEM ADD BACKEND 'doris-env:9050'; diff --git a/samples/datalake/deltalake_and_kudu/scripts/spark-delta.sh b/samples/datalake/deltalake_and_kudu/scripts/spark-delta.sh new file mode 100755 index 00000000000000..f4a64072658826 --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/scripts/spark-delta.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# 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. + +export SPARK_HOME=/opt/spark +export HIVE_HOME=/opt/apache-hive-3.1.2-bin +export HADOOP_HOME=/opt/hadoop-3.3.1 + +if [[ ! -d "${SPARK_HOME}" ]]; then + cp -r /opt/spark-3.4.2-bin-hadoop3 "${SPARK_HOME}" +fi + +cp "${HIVE_HOME}"/conf/hive-site.xml "${SPARK_HOME}"/conf/ +cp "${HIVE_HOME}"/lib/postgresql-jdbc.jar "${SPARK_HOME}"/jars/ +cp "${HADOOP_HOME}"/etc/hadoop/core-site.xml "${SPARK_HOME}"/conf/ + +"${SPARK_HOME}"/bin/spark-sql \ + --master local[*] \ + --name "spark-delta-sql" \ + --conf "spark.sql.extensions=io.delta.sql.DeltaSparkSessionExtension" \ + --conf "spark.sql.catalog.spark_catalog=org.apache.spark.sql.delta.catalog.DeltaCatalog" diff --git a/samples/datalake/deltalake_and_kudu/scripts/spark-delta.sql b/samples/datalake/deltalake_and_kudu/scripts/spark-delta.sql new file mode 100644 index 00000000000000..25f8757889f66d --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/scripts/spark-delta.sql @@ -0,0 +1 @@ +CREATE TABLE default.customer USING DELTA LOCATION 's3://data/customer/'; \ No newline at end of file diff --git a/samples/datalake/deltalake_and_kudu/scripts/start_doris.sh b/samples/datalake/deltalake_and_kudu/scripts/start_doris.sh new file mode 100755 index 00000000000000..91965d4e4ed683 --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/scripts/start_doris.sh @@ -0,0 +1,28 @@ +#!/usr/bin/env bash + +# 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. + +export JAVA_HOME=/opt/jdk-17.0.2 + +cp -r /opt/doris-bin /opt/doris +echo "trino_connector_plugin_dir=/opt/connectors/" >> /opt/doris/fe/conf/fe.conf +echo "trino_connector_plugin_dir=/opt/connectors/" >> /opt/doris/be/conf/be.conf + +/opt/doris/fe/bin/start_fe.sh --daemon +/opt/doris/be/bin/start_be.sh --daemon +tail -F /dev/null diff --git a/samples/datalake/deltalake_and_kudu/start-trinoconnector-compose.sh b/samples/datalake/deltalake_and_kudu/start-trinoconnector-compose.sh new file mode 100755 index 00000000000000..96fc6c739b016c --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/start-trinoconnector-compose.sh @@ -0,0 +1,181 @@ +#!/usr/bin/env bash + +# 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. + +DORIS_PACKAGE=apache-doris-3.0.1-bin-x64 +DORIS_DOWNLOAD_URL=https://apache-doris-releases.oss-accelerate.aliyuncs.com + +md5_aws_java_sdk="452d1e00efb11bff0ee17c42a6a44a0a" +md5_hadoop_aws="a3e19d42cadd1a6862a41fd276f94382" +md5_jdk17="0930efa680ac61e833699ccc36bfc739" +md5_spark="b393d314ffbc03facdc85575197c5db9" +md5_doris="fecd81c2d043542363422de6f111dbdb" +delta_core="65b8dec752d4984b7958d644848e3978" +delta_storage="a83011a52c66e081d4f53a7dc5c9708a" +antlr4_runtime="718f199bafa6574ffa1111fa3e10276a" +kudu_plugin="2d58bfcac5b84218c5d1055af189e30c" +delta_plugin="6b33448ce42d3d05e7b500ccafbe9698" +hdfs_plugin="ff4a3e3b32dcce27f4df58f17938abde" +kudu_java_example="1afe0a890785e8d0011ea7342ae5e43d" + + +download_source_file() { + local FILE_PATH="$1" + local EXPECTED_MD5="$2" + local DOWNLOAD_URL="$3" + + echo "Download ${FILE_PATH}" + + if [[ -f "${FILE_PATH}" ]]; then + local FILE_MD5 + FILE_MD5=$(md5sum "${FILE_PATH}" | awk '{ print $1 }') + echo "${FILE_PATH} 's md5sum is = ${FILE_MD5} ; expected is = ${EXPECTED_MD5}" + if [[ "${FILE_MD5}" = "${EXPECTED_MD5}" ]]; then + echo "${FILE_PATH} is ready!" + else + echo "${FILE_PATH} is broken, Redownloading ..." + rm "${FILE_PATH}" + wget "${DOWNLOAD_URL}"/"${FILE_PATH}" + fi + else + echo "Downloading ${FILE_PATH} ..." + wget "${DOWNLOAD_URL}"/"${FILE_PATH}" + fi +} + +curdir="$(cd "$(dirname "${BASH_SOURCE[0]}")" &>/dev/null && pwd)" +cd "${curdir}" || exit + +if [[ ! -d "packages" ]]; then + mkdir packages +fi +cd packages || exit + +download_source_file "aws-java-sdk-bundle-1.12.48.jar" "${md5_aws_java_sdk}" "https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-bundle/1.12.48" +download_source_file "hadoop-aws-3.3.1.jar" "${md5_hadoop_aws}" "https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-aws/3.3.1" +download_source_file "openjdk-17.0.2_linux-x64_bin.tar.gz" "${md5_jdk17}" "https://download.java.net/java/GA/jdk17.0.2/dfd4a8d0985749f896bed50d7138ee7f/8/GPL" +download_source_file "spark-3.4.2-bin-hadoop3.tgz" "${md5_spark}" "https://archive.apache.org/dist/spark/spark-3.4.2" +download_source_file "${DORIS_PACKAGE}.tar.gz" "${md5_doris}" "${DORIS_DOWNLOAD_URL}" +download_source_file "delta-core_2.12-2.4.0.jar" "${delta_core}" "https://repo1.maven.org/maven2/io/delta/delta-core_2.12/2.4.0" +download_source_file "delta-storage-2.4.0.jar" "${delta_storage}" "https://repo1.maven.org/maven2/io/delta/delta-storage/2.4.0" +download_source_file "antlr4-runtime-4.9.3.jar" "${antlr4_runtime}" "https://repo1.maven.org/maven2/org/antlr/antlr4-runtime/4.9.3" +download_source_file "trino-delta-lake-435-20240724.tar.gz" "${delta_plugin}" "https://github.com/apache/doris-thirdparty/releases/download/trino-435-20240724" +download_source_file "trino-kudu-435-20240724.tar.gz" "${kudu_plugin}" "https://github.com/apache/doris-thirdparty/releases/download/trino-435-20240724" +download_source_file "trino-hdfs-435-20240724.tar.gz" "${hdfs_plugin}" "https://github.com/apache/doris-thirdparty/releases/download/trino-435-20240724" +download_source_file "kudu-java-example-1.0-SNAPSHOT.jar" "${kudu_java_example}" "https://github.com/apache/doris-thirdparty/releases/download/trino-435-20240724" + + + + +if [[ ! -f "jdk-17.0.2/SUCCESS" ]]; then + echo "Prepare jdk17 environment" + if [[ -d "jdk-17.0.2" ]]; then + echo "Remove broken jdk-17.0.2" + rm -rf jdk-17.0.2 + fi + echo "Unpackage jdk-17.0.2" + tar xzf openjdk-17.0.2_linux-x64_bin.tar.gz + touch jdk-17.0.2/SUCCESS +fi +if [[ ! -f "spark-3.4.2-bin-hadoop3/SUCCESS" ]]; then + echo "Prepare spark3.4 environment" + if [[ -d "spark-3.4.2-bin-hadoop3" ]]; then + echo "Remove broken spark-3.4.2-bin-hadoop3" + rm -rf spark-3.4.2-bin-hadoop3 + fi + echo "Unpackage spark-3.4.2-bin-hadoop3" + tar -xf spark-3.4.2-bin-hadoop3.tgz + cp aws-java-sdk-bundle-1.12.48.jar spark-3.4.2-bin-hadoop3/jars/ + cp hadoop-aws-3.3.1.jar spark-3.4.2-bin-hadoop3/jars/ + cp delta-core_2.12-2.4.0.jar spark-3.4.2-bin-hadoop3/jars/ + cp delta-storage-2.4.0.jar spark-3.4.2-bin-hadoop3/jars/ + cp antlr4-runtime-4.9.3.jar spark-3.4.2-bin-hadoop3/jars/ + touch spark-3.4.2-bin-hadoop3/SUCCESS +fi +if [[ ! -f "doris-bin/SUCCESS" ]]; then + echo "Prepare ${DORIS_PACKAGE} environment" + if [[ -d "doris-bin" ]]; then + echo "Remove broken ${DORIS_PACKAGE}" + rm -rf doris-bin + fi + echo "Unpackage ${DORIS_PACKAGE}" + tar xzf "${DORIS_PACKAGE}".tar.gz + mv "${DORIS_PACKAGE}" doris-bin + touch doris-bin/SUCCESS +fi + +mkdir connectors +if [[ ! -f "connectors/trino-delta-lake-435/SUCCESS" ]]; then + echo "Prepare trino-delta-lake-435 plugin" + if [[ -d "connectors/trino-delta-lake-435" ]]; then + echo "Remove broken trino-delta-lake-435" + rm -rf connectors/trino-delta-lake-435 + fi + echo "Unpackage trino-delta-lake-435" + tar xzf trino-delta-lake-435-20240724.tar.gz + mv trino-delta-lake-435 connectors/trino-delta-lake-435 + touch connectors/trino-delta-lake-435/SUCCESS +fi + +if [[ ! -f "connectors/trino-kudu-435/SUCCESS" ]]; then + echo "Prepare trino-kudu-435 plugin" + if [[ -d "connectors/trino-kudu-435" ]]; then + echo "Remove broken trino-kudu-435" + rm -rf connectors/trino-kudu-435 + fi + echo "Unpackage trino-kudu-435" + tar xzf trino-kudu-435-20240724.tar.gz + mv trino-kudu-435 connectors/trino-kudu-435 + touch connectors/trino-kudu-435/SUCCESS +fi + +if [[ ! -f "connectors/trino-delta-lake-435/hdfs/SUCCESS" ]]; then + echo "Prepare hdfs plugin" + if [[ -d "connectors/trino-delta-lake-435/hdfs" ]]; then + echo "Remove broken connectors/trino-delta-lake-435/hdfs" + rm -rf connectors/trino-delta-lake-435/hdfs + fi + echo "Unpackage trino-delta-lake-435/hdfs" + tar xzf trino-hdfs-435-20240724.tar.gz + mv hdfs connectors/trino-delta-lake-435/hdfs + touch connectors/trino-delta-lake-435/hdfs/SUCCESS +fi + + +cd ../ + +export KUDU_QUICKSTART_IP=$(ifconfig | grep "inet " | grep -Fv 127.0.0.1 | awk '{print $2}' | tail -1) + +docker compose -f trinoconnector-compose.yml --env-file trinoconnector-compose.env up -d +echo "Create hive table ..." +sleep 5 +docker exec -it spark-hive sh -c "/opt/hadoop-3.3.1/bin/hadoop fs -chmod 777 /tmp/hive" +docker exec -it spark-hive sh -c "sh /opt/scripts/create-delta-table.sh" +sleep 5 +echo "Build hive catalog in Doris ..." +docker exec -it spark-hive sh -c "mysql -u root -h doris-env -P 9030 < /opt/scripts/doris-sql.sql" +sleep 10 +echo "Create kudu table ..." +docker exec -it kudu-master-1 sh -c "/opt/jdk-17.0.2/bin/java -DkuduMasters=kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251 -jar /opt/kudu-java-example-1.0-SNAPSHOT.jar" +sleep 10 + +echo "======================================================" +echo "Success to launch spark+doris+deltalake+kudu+minio environments!" +echo "./login-spark.sh to login into spark" +echo "./login-doris.sh to login into doris" +echo "======================================================" diff --git a/samples/datalake/deltalake_and_kudu/stop-trinoconnector-compose.sh b/samples/datalake/deltalake_and_kudu/stop-trinoconnector-compose.sh new file mode 100755 index 00000000000000..838630d0249297 --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/stop-trinoconnector-compose.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash + +# 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. + +export KUDU_QUICKSTART_IP=$(ifconfig | grep "inet " | grep -Fv 127.0.0.1 | awk '{print $2}' | tail -1) + +docker compose -f trinoconnector-compose.yml --env-file trinoconnector-compose.env down diff --git a/samples/datalake/deltalake_and_kudu/trinoconnector-compose.env b/samples/datalake/deltalake_and_kudu/trinoconnector-compose.env new file mode 100644 index 00000000000000..8abf3700ba1fe7 --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/trinoconnector-compose.env @@ -0,0 +1,22 @@ +#!/bin/bash +# 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. + +HIVE_THRIFT_PORT=9771 +MINIO_SERVICE_PORT=9772 +DORIS_QUERY_PORT=9773 +KUDU_QUICKSTART_VERSION="1.17.0" \ No newline at end of file diff --git a/samples/datalake/deltalake_and_kudu/trinoconnector-compose.yml b/samples/datalake/deltalake_and_kudu/trinoconnector-compose.yml new file mode 100644 index 00000000000000..e88348b3e6f4b1 --- /dev/null +++ b/samples/datalake/deltalake_and_kudu/trinoconnector-compose.yml @@ -0,0 +1,323 @@ +# 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. + +version: "3.9" +services: + metastore_db: + image: postgres:11 + hostname: metastore_db + environment: + POSTGRES_USER: hive + POSTGRES_PASSWORD: hive + POSTGRES_DB: metastore + + hive-metastore: + hostname: hive-metastore + container_name: spark-hive + image: 'starburstdata/hive:3.1.2-e.18' + volumes: + - './packages/spark-3.4.2-bin-hadoop3:/opt/spark-3.4.2-bin-hadoop3' + - './scripts:/opt/scripts' + ports: + - '${HIVE_THRIFT_PORT}:9083' # Metastore Thrift + environment: + HIVE_METASTORE_DRIVER: org.postgresql.Driver + HIVE_METASTORE_JDBC_URL: jdbc:postgresql://metastore_db:5432/metastore + HIVE_METASTORE_USER: hive + HIVE_METASTORE_PASSWORD: hive + HIVE_METASTORE_WAREHOUSE_DIR: s3://datalake/ + S3_ENDPOINT: http://minio:9000 + S3_ACCESS_KEY: minio + S3_SECRET_KEY: minio123 + S3_PATH_STYLE_ACCESS: "true" + REGION: "" + GOOGLE_CLOUD_KEY_FILE_PATH: "" + AZURE_ADL_CLIENT_ID: "" + AZURE_ADL_CREDENTIAL: "" + AZURE_ADL_REFRESH_URL: "" + AZURE_ABFS_STORAGE_ACCOUNT: "" + AZURE_ABFS_ACCESS_KEY: "" + AZURE_WASB_STORAGE_ACCOUNT: "" + AZURE_ABFS_OAUTH: "" + AZURE_ABFS_OAUTH_TOKEN_PROVIDER: "" + AZURE_ABFS_OAUTH_CLIENT_ID: "" + AZURE_ABFS_OAUTH_SECRET: "" + AZURE_ABFS_OAUTH_ENDPOINT: "" + AZURE_WASB_ACCESS_KEY: "" + HIVE_METASTORE_USERS_IN_ADMIN_ROLE: "hive" + depends_on: + - metastore_db + + doris-env: + hostname: doris-env + container_name: doris-env + image: 'apache/hadoop:3.3.6' + environment: + LD_LIBRARY_PATH: /opt/doris/be/lib + ports: + - '${DORIS_QUERY_PORT}:9030' + volumes: + - './packages/jdk-17.0.2:/opt/jdk-17.0.2' + - './packages/doris-bin:/opt/doris-bin' + - './scripts:/opt/scripts' + - './packages/connectors:/opt/connectors' + command: sh /opt/scripts/start_doris.sh + + minio: + hostname: minio + image: 'minio/minio:RELEASE.2022-05-26T05-48-41Z' + container_name: minio-storage + ports: + - '${MINIO_SERVICE_PORT}:9000' + environment: + MINIO_ACCESS_KEY: minio + MINIO_SECRET_KEY: minio123 + command: server /data --console-address ":9001" + + # This job will create the "datalake" bucket on Minio + mc-job: + image: 'minio/mc:RELEASE.2022-05-09T04-08-26Z' + volumes: + - './data:/data' + entrypoint: | + /bin/bash -c " + sleep 5; + /usr/bin/mc config --quiet host add myminio http://minio:9000 minio minio123; + /usr/bin/mc mb --quiet myminio/datalake; + /usr/bin/mc mb --quiet myminio/data; + /usr/bin/mc mirror /data myminio/data + " + depends_on: + - minio + + kudu-master-1: + image: apache/kudu:${KUDU_QUICKSTART_VERSION:-latest} + hostname: kudu-master-1 + container_name: kudu-master-1 + ports: + - "7051:7051" + - "8051:8051" + command: ["master"] + volumes: + - kudu-master-1:/var/lib/kudu + - './packages/jdk-17.0.2:/opt/jdk-17.0.2' + - './packages/kudu-java-example-1.0-SNAPSHOT.jar:/opt/kudu-java-example-1.0-SNAPSHOT.jar' + environment: + - KUDU_MASTERS=kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251 + # TODO: Use `host.docker.internal` instead of KUDU_QUICKSTART_IP when it + # works on Linux (https://github.com/docker/for-linux/issues/264) + - > + MASTER_ARGS=--fs_wal_dir=/var/lib/kudu/master + --rpc_bind_addresses=0.0.0.0:7051 + --rpc_advertised_addresses=${KUDU_QUICKSTART_IP:?Please set KUDU_QUICKSTART_IP environment variable}:7051 + --webserver_port=8051 + --webserver_advertised_addresses=${KUDU_QUICKSTART_IP}:8051 + --webserver_doc_root=/opt/kudu/www + --stderrthreshold=0 + --use_hybrid_clock=false + --unlock_unsafe_flags=true + # command: /opt/jdk-17.0.2/bin/java -DkuduMasters=kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251 -jar /opt/kudu-java-example-1.0-SNAPSHOT.jar + + kudu-master-2: + image: apache/kudu:${KUDU_QUICKSTART_VERSION:-latest} + hostname: kudu-master-2 + container_name: kudu-master-2 + ports: + - "7151:7151" + - "8151:8151" + command: ["master"] + volumes: + - kudu-master-2:/var/lib/kudu + environment: + - KUDU_MASTERS=kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251 + - > + MASTER_ARGS=--fs_wal_dir=/var/lib/kudu/master + --rpc_bind_addresses=0.0.0.0:7151 + --rpc_advertised_addresses=${KUDU_QUICKSTART_IP}:7151 + --webserver_port=8151 + --webserver_advertised_addresses=${KUDU_QUICKSTART_IP}:8151 + --webserver_doc_root=/opt/kudu/www + --stderrthreshold=0 + --use_hybrid_clock=false + --unlock_unsafe_flags=true + kudu-master-3: + image: apache/kudu:${KUDU_QUICKSTART_VERSION:-latest} + hostname: kudu-master-3 + container_name: kudu-master-3 + ports: + - "7251:7251" + - "8251:8251" + command: ["master"] + volumes: + - kudu-master-3:/var/lib/kudu + environment: + - KUDU_MASTERS=kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251 + - > + MASTER_ARGS=--fs_wal_dir=/var/lib/kudu/master + --rpc_bind_addresses=0.0.0.0:7251 + --rpc_advertised_addresses=${KUDU_QUICKSTART_IP}:7251 + --webserver_port=8251 + --webserver_advertised_addresses=${KUDU_QUICKSTART_IP}:8251 + --webserver_doc_root=/opt/kudu/www + --stderrthreshold=0 + --use_hybrid_clock=false + --unlock_unsafe_flags=true + kudu-tserver-1: + image: apache/kudu:${KUDU_QUICKSTART_VERSION:-latest} + hostname: kudu-tserver-1 + container_name: kudu-tserver-1 + depends_on: + - kudu-master-1 + - kudu-master-2 + - kudu-master-3 + ports: + - "7050:7050" + - "8050:8050" + command: ["tserver"] + volumes: + - kudu-tserver-1:/var/lib/kudu + environment: + - KUDU_MASTERS=kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251 + - > + TSERVER_ARGS=--fs_wal_dir=/var/lib/kudu/tserver + --rpc_bind_addresses=0.0.0.0:7050 + --rpc_advertised_addresses=${KUDU_QUICKSTART_IP}:7050 + --webserver_port=8050 + --webserver_advertised_addresses=${KUDU_QUICKSTART_IP}:8050 + --webserver_doc_root=/opt/kudu/www + --stderrthreshold=0 + --use_hybrid_clock=false + --unlock_unsafe_flags=true + kudu-tserver-2: + image: apache/kudu:${KUDU_QUICKSTART_VERSION:-latest} + hostname: kudu-tserver-2 + container_name: kudu-tserver-2 + depends_on: + - kudu-master-1 + - kudu-master-2 + - kudu-master-3 + ports: + - "7150:7150" + - "8150:8150" + command: ["tserver"] + volumes: + - kudu-tserver-2:/var/lib/kudu + environment: + - KUDU_MASTERS=kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251 + - > + TSERVER_ARGS=--fs_wal_dir=/var/lib/kudu/tserver + --rpc_bind_addresses=0.0.0.0:7150 + --rpc_advertised_addresses=${KUDU_QUICKSTART_IP}:7150 + --webserver_port=8150 + --webserver_advertised_addresses=${KUDU_QUICKSTART_IP}:8150 + --webserver_doc_root=/opt/kudu/www + --stderrthreshold=0 + --use_hybrid_clock=false + --unlock_unsafe_flags=true + kudu-tserver-3: + image: apache/kudu:${KUDU_QUICKSTART_VERSION:-latest} + hostname: kudu-tserver-3 + container_name: kudu-tserver-3 + depends_on: + - kudu-master-1 + - kudu-master-2 + - kudu-master-3 + ports: + - "7250:7250" + - "8250:8250" + command: ["tserver"] + volumes: + - kudu-tserver-3:/var/lib/kudu + environment: + - KUDU_MASTERS=kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251 + - > + TSERVER_ARGS=--fs_wal_dir=/var/lib/kudu/tserver + --rpc_bind_addresses=0.0.0.0:7250 + --rpc_advertised_addresses=${KUDU_QUICKSTART_IP}:7250 + --webserver_port=8250 + --webserver_advertised_addresses=${KUDU_QUICKSTART_IP}:8250 + --webserver_doc_root=/opt/kudu/www + --stderrthreshold=0 + --use_hybrid_clock=false + --unlock_unsafe_flags=true + kudu-tserver-4: + image: apache/kudu:${KUDU_QUICKSTART_VERSION:-latest} + hostname: kudu-tserver-4 + container_name: kudu-tserver-4 + depends_on: + - kudu-master-1 + - kudu-master-2 + - kudu-master-3 + ports: + - "7350:7350" + - "8350:8350" + command: ["tserver"] + volumes: + - kudu-tserver-4:/var/lib/kudu + environment: + - KUDU_MASTERS=kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251 + - > + TSERVER_ARGS=--fs_wal_dir=/var/lib/kudu/tserver + --rpc_bind_addresses=0.0.0.0:7350 + --rpc_advertised_addresses=${KUDU_QUICKSTART_IP}:7350 + --webserver_port=8350 + --webserver_advertised_addresses=${KUDU_QUICKSTART_IP}:8350 + --webserver_doc_root=/opt/kudu/www + --stderrthreshold=0 + --use_hybrid_clock=false + --unlock_unsafe_flags=true + kudu-tserver-5: + image: apache/kudu:${KUDU_QUICKSTART_VERSION:-latest} + hostname: kudu-tserver-5 + container_name: kudu-tserver-5 + depends_on: + - kudu-master-1 + - kudu-master-2 + - kudu-master-3 + ports: + - "7450:7450" + - "8450:8450" + command: ["tserver"] + volumes: + - kudu-tserver-5:/var/lib/kudu + environment: + - KUDU_MASTERS=kudu-master-1:7051,kudu-master-2:7151,kudu-master-3:7251 + - > + TSERVER_ARGS=--fs_wal_dir=/var/lib/kudu/tserver + --rpc_bind_addresses=0.0.0.0:7450 + --rpc_advertised_addresses=${KUDU_QUICKSTART_IP}:7450 + --webserver_port=8450 + --webserver_advertised_addresses=${KUDU_QUICKSTART_IP}:8450 + --webserver_doc_root=/opt/kudu/www + --stderrthreshold=0 + --use_hybrid_clock=false + --unlock_unsafe_flags=true + +volumes: + kudu-master-1: + kudu-master-2: + kudu-master-3: + kudu-tserver-1: + kudu-tserver-2: + kudu-tserver-3: + kudu-tserver-4: + kudu-tserver-5: + +networks: + default: + name: trinoconnector-net + external: true