You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2022/06/01 15:28:55 UTC

[incubator-doris] branch dev-1.0.1 updated (96ad450eac -> 07a119abfb)

This is an automated email from the ASF dual-hosted git repository.

morningman pushed a change to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git


    from 96ad450eac [hotfix](expr) fix memory leak in expr (#9820)
     new a9d4fbcde9 [refactor][storage format] Forbidden rowset v1 (#9248)
     new b01bd5b72e [fix](vectorized) fix vcast expr input wrong row number (#9520)
     new f5b421eaaf [improvement](load) reduce useless err_msg format in VOlapTableSink send (#9531)
     new 1a43e0d412 [bugfix] Fix create table like when having hidden columns (#9694)
     new 0224db0417 [Bug] [Bitmap] change to_bitmap to always_not_nullable (#9716)
     new bda73c9ba5 [improvement](script) Add jvm parameters and the process will automatically stop when oom occurs in fe. (#9765)
     new e57aa42fa7 [fix](sql-block-rule) sql block rule NPE (#9778)
     new d40fcdc58d [Improvement] optimize  scannode concurrency query performance in vectorized engine. (#9792)
     new 99e76032c0 [API changed](parser) Remove merge join syntax (#9795)
     new 741f97829b [Improvement] Simplify expressions for _vconjunct_ctx_ptr (#9816)
     new a7c6ca63c4 [bugfix]handle ComlumnDictionary in evaluate_and and evaluate_or (#9818)
     new 60ed43e621 [Bugfix(Vec)] Fix some memory leak issues (#9824)
     new f21002f767 [feature] Support compression prop (#8923)
     new fa6b72617c [fix](ut) fix BE ut (#9831)
     new fe32d8ffff [Bugfix(Vec)] Close result_sink properly (#9849)
     new fceff7f98b [bugfix]fix TableFunctionNode memory leak (#9853)
     new f5386cb8e7 [Bug][Fix] One Rowset have same key output in unique table (#9858)
     new 9f14de2127 [BUG] return NULL for invalid date value (#9862)
     new 9094188ba6 fix core dump on vcase_expr::close (#9875)
     new c1abaec0d5 [Bug][Vectorized] fix core dump on vcase_expr::close (#9893)
     new 03748e4b17 [Feature] add `weekday` function on vectorized engine (#9901)
     new 07a119abfb [fix](routine-load) fix bug that routine load task can not find backend (#9902)

The 22 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/common/config.h                             |   4 +
 be/src/exec/CMakeLists.txt                         |   1 -
 be/src/exec/exec_node.cpp                          |   5 -
 be/src/exec/merge_join_node.cpp                    | 335 ---------------
 be/src/exec/merge_join_node.h                      | 102 -----
 be/src/exec/olap_scan_node.cpp                     |  18 +-
 be/src/exec/olap_scan_node.h                       |   9 +-
 be/src/exec/olap_scanner.cpp                       |   1 -
 be/src/exec/scan_node.cpp                          |  10 +-
 be/src/exec/table_function_node.cpp                |   1 +
 be/src/exec/tablet_sink.cpp                        |   5 +-
 be/src/exprs/bitmap_function.cpp                   |  20 +-
 be/src/olap/bloom_filter_predicate.h               |  17 +-
 be/src/olap/collect_iterator.cpp                   |   2 +-
 be/src/olap/comparison_predicate.cpp               | 451 ++++++++++-----------
 be/src/olap/generic_iterators.cpp                  |  52 ++-
 be/src/olap/generic_iterators.h                    |   2 +-
 be/src/olap/reader.cpp                             |   1 +
 be/src/olap/rowset/beta_rowset_reader.cpp          |   4 +-
 be/src/olap/rowset/rowset_reader_context.h         |   1 +
 be/src/olap/rowset/segment_v2/column_writer.cpp    |   4 +-
 be/src/olap/rowset/segment_v2/segment_writer.cpp   |  10 +-
 be/src/olap/rowset/segment_v2/segment_writer.h     |   3 +-
 be/src/olap/schema_change.cpp                      |   2 +
 be/src/olap/storage_engine.cpp                     |   9 +-
 be/src/olap/tablet_meta.cpp                        |  43 +-
 be/src/olap/tablet_meta.h                          |   3 +-
 be/src/olap/tablet_schema.cpp                      |   2 +
 be/src/olap/tablet_schema.h                        |   3 +
 be/src/olap/tuple_reader.cpp                       |  25 +-
 be/src/util/block_compression.cpp                  |   4 +-
 be/src/vec/exec/join/vhash_join_node.cpp           |   2 +
 be/src/vec/exec/join/vhash_join_node.h             |  45 +-
 be/src/vec/exec/vaggregation_node.cpp              |   5 +-
 be/src/vec/exec/vanalytic_eval_node.cpp            |  12 +-
 be/src/vec/exec/vexchange_node.cpp                 |   3 +
 be/src/vec/exec/volap_scan_node.cpp                |  44 +-
 be/src/vec/exec/volap_scanner.cpp                  |   3 -
 be/src/vec/exec/vsort_node.cpp                     |   3 +-
 be/src/vec/exprs/vcase_expr.cpp                    |  18 +-
 be/src/vec/exprs/vcast_expr.cpp                    |  16 +-
 be/src/vec/exprs/vexpr.cpp                         |   8 +-
 be/src/vec/exprs/vexpr.h                           |  22 +-
 be/src/vec/exprs/vexpr_context.cpp                 |   9 +-
 be/src/vec/exprs/vexpr_context.h                   |   1 +
 be/src/vec/exprs/vinfo_func.cpp                    |   9 +-
 be/src/vec/exprs/vliteral.cpp                      |  10 +-
 be/src/vec/functions/date_time_transforms.h        |   1 +
 be/src/vec/functions/function_bitmap.cpp           |  10 +-
 be/src/vec/functions/function_case.h               |   5 +-
 be/src/vec/functions/function_cast.h               |  54 +--
 be/src/vec/functions/time_of_function.cpp          |   2 +
 be/src/vec/olap/vcollect_iterator.cpp              |   8 +-
 be/src/vec/olap/vgeneric_iterators.cpp             |  44 +-
 be/src/vec/olap/vgeneric_iterators.h               |   2 +-
 be/src/vec/runtime/vdatetime_value.cpp             |   8 +-
 be/src/vec/sink/result_sink.cpp                    |  17 +-
 be/src/vec/sink/vtablet_sink.cpp                   |  43 +-
 be/src/vec/utils/util.hpp                          |  13 +-
 be/test/exprs/bitmap_function_test.cpp             |   6 +-
 be/test/olap/cumulative_compaction_policy_test.cpp |  12 +-
 be/test/olap/generic_iterators_test.cpp            |  38 +-
 be/test/olap/tablet_meta_test.cpp                  |   5 +-
 be/test/olap/tablet_test.cpp                       |   6 +-
 be/test/olap/test_data/header_without_inc_rs.txt   |   3 +-
 be/test/vec/exec/vgeneric_iterators_test.cpp       |  61 ++-
 be/test/vec/function/function_time_test.cpp        |  23 ++
 bin/start_fe.sh                                    |   4 +-
 docs/en/administrator-guide/config/fe_config.md    |   6 -
 docs/zh-CN/administrator-guide/config/fe_config.md |   6 -
 fe/fe-core/src/main/cup/sql_parser.cup             |   2 -
 .../java/org/apache/doris/alter/RollupJobV2.java   |   3 +-
 .../org/apache/doris/alter/SchemaChangeJobV2.java  |   3 +-
 .../org/apache/doris/analysis/JoinOperator.java    |   1 -
 .../java/org/apache/doris/analysis/TableRef.java   |   2 -
 .../java/org/apache/doris/backup/RestoreJob.java   |   3 +-
 .../org/apache/doris/blockrule/SqlBlockRule.java   |  37 +-
 .../apache/doris/blockrule/SqlBlockRuleMgr.java    |  84 +++-
 .../java/org/apache/doris/catalog/Catalog.java     |  53 ++-
 .../java/org/apache/doris/catalog/OlapTable.java   |  16 +
 .../org/apache/doris/catalog/TableProperty.java    |  20 +-
 .../main/java/org/apache/doris/common/Config.java  |   8 -
 .../apache/doris/common/util/PropertyAnalyzer.java |  39 +-
 .../doris/load/routineload/RoutineLoadManager.java |   5 +-
 .../org/apache/doris/master/ReportHandler.java     |   3 +-
 .../java/org/apache/doris/qe/StmtExecutor.java     |   7 +-
 .../org/apache/doris/system/HeartbeatFlags.java    |  27 +-
 .../org/apache/doris/system/SystemInfoService.java |  21 +-
 .../java/org/apache/doris/task/AgentBatchTask.java |  21 -
 .../org/apache/doris/task/CreateReplicaTask.java   |  18 +-
 .../org/apache/doris/task/CreateRollupTask.java    | 157 -------
 .../org/apache/doris/task/SchemaChangeTask.java    | 141 -------
 .../org/apache/doris/alter/AlterJobV2Test.java     |  56 ---
 .../doris/blockrule/SqlBlockRuleMgrTest.java       | 382 -----------------
 .../org/apache/doris/catalog/CreateTableTest.java  |  10 +
 .../apache/doris/system/SystemInfoServiceTest.java |  13 +-
 .../java/org/apache/doris/task/AgentTaskTest.java  |  45 +-
 gensrc/proto/olap_file.proto                       |   2 +
 gensrc/script/doris_builtins_functions.py          |   4 +-
 gensrc/thrift/AgentService.thrift                  |  31 ++
 gensrc/thrift/PlanNodes.thrift                     |   4 +-
 .../data_model/unique/test_unique_table_like.out   |  15 +
 .../unique/test_unique_table_like.groovy           |  52 +++
 103 files changed, 1140 insertions(+), 1881 deletions(-)
 delete mode 100644 be/src/exec/merge_join_node.cpp
 delete mode 100644 be/src/exec/merge_join_node.h
 delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/task/CreateRollupTask.java
 delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/task/SchemaChangeTask.java
 delete mode 100644 fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java
 create mode 100644 regression-test/data/data_model/unique/test_unique_table_like.out
 create mode 100644 regression-test/suites/data_model/unique/test_unique_table_like.groovy


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 12/22: [Bugfix(Vec)] Fix some memory leak issues (#9824)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 60ed43e621dee87c6e0514a2ce9ad022bd6cb192
Author: Amos Bird <am...@gmail.com>
AuthorDate: Sun May 29 23:04:11 2022 +0800

    [Bugfix(Vec)] Fix some memory leak issues (#9824)
---
 be/src/vec/exec/join/vhash_join_node.cpp |  2 ++
 be/src/vec/exec/vaggregation_node.cpp    |  5 +++--
 be/src/vec/exec/vanalytic_eval_node.cpp  | 12 ++++++++++--
 be/src/vec/exec/vexchange_node.cpp       |  3 +++
 be/src/vec/exec/vsort_node.cpp           |  3 +--
 be/src/vec/exprs/vexpr_context.cpp       |  9 ++++++++-
 be/src/vec/exprs/vexpr_context.h         |  1 +
 7 files changed, 28 insertions(+), 7 deletions(-)

diff --git a/be/src/vec/exec/join/vhash_join_node.cpp b/be/src/vec/exec/join/vhash_join_node.cpp
index a350f99ef4..d91ff2d22b 100644
--- a/be/src/vec/exec/join/vhash_join_node.cpp
+++ b/be/src/vec/exec/join/vhash_join_node.cpp
@@ -808,6 +808,8 @@ Status HashJoinNode::close(RuntimeState* state) {
         return Status::OK();
     }
 
+    VExpr::close(_build_expr_ctxs, state);
+    VExpr::close(_probe_expr_ctxs, state);
     if (_vother_join_conjunct_ptr) (*_vother_join_conjunct_ptr)->close(state);
 
     _mem_tracker->Release(_mem_used);
diff --git a/be/src/vec/exec/vaggregation_node.cpp b/be/src/vec/exec/vaggregation_node.cpp
index 76b4b349ef..dcd7b375d7 100644
--- a/be/src/vec/exec/vaggregation_node.cpp
+++ b/be/src/vec/exec/vaggregation_node.cpp
@@ -402,10 +402,11 @@ Status AggregationNode::get_next(RuntimeState* state, Block* block, bool* eos) {
 Status AggregationNode::close(RuntimeState* state) {
     if (is_closed()) return Status::OK();
 
-    RETURN_IF_ERROR(ExecNode::close(state));
+    for (auto* aggregate_evaluator : _aggregate_evaluators) aggregate_evaluator->close(state);
     VExpr::close(_probe_expr_ctxs, state);
     if (_executor.close) _executor.close();
-    return Status::OK();
+
+    return ExecNode::close(state);
 }
 
 Status AggregationNode::_create_agg_status(AggregateDataPtr data) {
diff --git a/be/src/vec/exec/vanalytic_eval_node.cpp b/be/src/vec/exec/vanalytic_eval_node.cpp
index 4d69716216..65765d15d3 100644
--- a/be/src/vec/exec/vanalytic_eval_node.cpp
+++ b/be/src/vec/exec/vanalytic_eval_node.cpp
@@ -216,6 +216,9 @@ Status VAnalyticEvalNode::open(RuntimeState* state) {
     for (size_t i = 0; i < _agg_functions_size; ++i) {
         RETURN_IF_ERROR(VExpr::open(_agg_expr_ctxs[i], state));
     }
+    for (auto* agg_function : _agg_functions) {
+        RETURN_IF_ERROR(agg_function->open(state));
+    }
     return Status::OK();
 }
 
@@ -223,9 +226,14 @@ Status VAnalyticEvalNode::close(RuntimeState* state) {
     if (is_closed()) {
         return Status::OK();
     }
-    ExecNode::close(state);
+
+    VExpr::close(_partition_by_eq_expr_ctxs, state);
+    VExpr::close(_order_by_eq_expr_ctxs, state);
+    for (size_t i = 0; i < _agg_functions_size; ++i) VExpr::close(_agg_expr_ctxs[i], state);
+    for (auto* agg_function : _agg_functions) agg_function->close(state);
+
     _destory_agg_status();
-    return Status::OK();
+    return ExecNode::close(state);
 }
 
 Status VAnalyticEvalNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) {
diff --git a/be/src/vec/exec/vexchange_node.cpp b/be/src/vec/exec/vexchange_node.cpp
index 91b107904d..83c9e31b56 100644
--- a/be/src/vec/exec/vexchange_node.cpp
+++ b/be/src/vec/exec/vexchange_node.cpp
@@ -98,6 +98,9 @@ Status VExchangeNode::close(RuntimeState* state) {
     if (_stream_recvr != nullptr) {
         _stream_recvr->close();
     }
+
+    if (_is_merging) _vsort_exec_exprs.close(state);
+
     return ExecNode::close(state);
 }
 
diff --git a/be/src/vec/exec/vsort_node.cpp b/be/src/vec/exec/vsort_node.cpp
index 734af91baa..99c8090fde 100644
--- a/be/src/vec/exec/vsort_node.cpp
+++ b/be/src/vec/exec/vsort_node.cpp
@@ -104,8 +104,7 @@ Status VSortNode::close(RuntimeState* state) {
     }
     _mem_tracker->Release(_total_mem_usage);
     _vsort_exec_exprs.close(state);
-    ExecNode::close(state);
-    return Status::OK();
+    return ExecNode::close(state);
 }
 
 void VSortNode::debug_string(int indentation_level, stringstream* out) const {
diff --git a/be/src/vec/exprs/vexpr_context.cpp b/be/src/vec/exprs/vexpr_context.cpp
index b8c7f37b4a..bb8f615dc5 100644
--- a/be/src/vec/exprs/vexpr_context.cpp
+++ b/be/src/vec/exprs/vexpr_context.cpp
@@ -29,6 +29,14 @@ VExprContext::VExprContext(VExpr* expr)
           _closed(false),
           _last_result_column_id(-1) {}
 
+VExprContext::~VExprContext() {
+    DCHECK(!_prepared || _closed);
+
+    for (int i = 0; i < _fn_contexts.size(); ++i) {
+        delete _fn_contexts[i];
+    }
+}
+
 doris::Status VExprContext::execute(doris::vectorized::Block* block, int* result_column_id) {
     Status st = _root->execute(this, block, result_column_id);
     _last_result_column_id = *result_column_id;
@@ -64,7 +72,6 @@ void VExprContext::close(doris::RuntimeState* state) {
 
     for (int i = 0; i < _fn_contexts.size(); ++i) {
         _fn_contexts[i]->impl()->close();
-        delete _fn_contexts[i];
     }
     // _pool can be NULL if Prepare() was never called
     if (_pool != NULL) {
diff --git a/be/src/vec/exprs/vexpr_context.h b/be/src/vec/exprs/vexpr_context.h
index 9d2d467dd4..54b47d7d1b 100644
--- a/be/src/vec/exprs/vexpr_context.h
+++ b/be/src/vec/exprs/vexpr_context.h
@@ -27,6 +27,7 @@ class VExpr;
 class VExprContext {
 public:
     VExprContext(VExpr* expr);
+    ~VExprContext();
     Status prepare(RuntimeState* state, const RowDescriptor& row_desc,
                    const std::shared_ptr<MemTracker>& tracker);
     Status open(RuntimeState* state);


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 06/22: [improvement](script) Add jvm parameters and the process will automatically stop when oom occurs in fe. (#9765)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit bda73c9ba5cc089ad9dc9e73e0e24593b6c4d461
Author: smallhibiscus <84...@qq.com>
AuthorDate: Mon May 30 09:44:12 2022 +0800

    [improvement](script) Add jvm parameters and the process will automatically stop when oom occurs in fe. (#9765)
---
 bin/start_fe.sh | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/bin/start_fe.sh b/bin/start_fe.sh
index 37a474ca4b..5c83f8991e 100755
--- a/bin/start_fe.sh
+++ b/bin/start_fe.sh
@@ -164,10 +164,10 @@ if [ x"$HELPER" != x"" ]; then
 fi
 
 if [ ${RUN_DAEMON} -eq 1 ]; then
-    nohup $LIMIT $JAVA $final_java_opt org.apache.doris.PaloFe ${HELPER} "$@" >> $LOG_DIR/fe.out 2>&1 < /dev/null &
+    nohup $LIMIT $JAVA $final_java_opt -XX:OnOutOfMemoryError="kill -9 %p" org.apache.doris.PaloFe ${HELPER} "$@" >> $LOG_DIR/fe.out 2>&1 < /dev/null &
 else
     export DORIS_LOG_TO_STDERR=1
-    $LIMIT $JAVA $final_java_opt org.apache.doris.PaloFe ${HELPER} "$@" < /dev/null
+    $LIMIT $JAVA $final_java_opt -XX:OnOutOfMemoryError="kill -9 %p" org.apache.doris.PaloFe ${HELPER} "$@" < /dev/null
 fi
 
 echo $! > $pidfile


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 18/22: [BUG] return NULL for invalid date value (#9862)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 9f14de2127149d1cab53453cf98377ca6c386c1c
Author: Gabriel <ga...@gmail.com>
AuthorDate: Mon May 30 21:35:41 2022 +0800

    [BUG] return NULL for invalid date value (#9862)
---
 be/src/vec/runtime/vdatetime_value.cpp | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/be/src/vec/runtime/vdatetime_value.cpp b/be/src/vec/runtime/vdatetime_value.cpp
index ec40d27b00..97cec16e0f 100644
--- a/be/src/vec/runtime/vdatetime_value.cpp
+++ b/be/src/vec/runtime/vdatetime_value.cpp
@@ -61,11 +61,11 @@ bool VecDateTimeValue::check_range(uint32_t year, uint32_t month, uint32_t day,
 }
 
 bool VecDateTimeValue::check_date(uint32_t year, uint32_t month, uint32_t day) {
-    if (month != 0 && month <= 12 && day > s_days_in_month[month]) {
-        // Feb 29 in leap year is valid.
-        if (!(month == 2 && day == 29 && is_leap(year))) return true;
+    if (month == 2 && day == 29 && is_leap(year)) return false;
+    if (year > 9999 || month == 0 || month > 12 || day > s_days_in_month[month] || day == 0) {
+        return true;
     }
-    return year > 9999 || month > 12 || day > 31;
+    return false;
 }
 
 // The interval format is that with no delimiters


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 01/22: [refactor][storage format] Forbidden rowset v1 (#9248)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit a9d4fbcde94678d1b3547da535d49287b6f101d6
Author: yiguolei <67...@qq.com>
AuthorDate: Wed May 4 17:32:20 2022 +0800

    [refactor][storage format] Forbidden rowset v1 (#9248)
    
    - Force change the existing olaptable's storage format from V1 to V2
    - Forbidden to create new olap table with storage format == v1 OR do schema change that want to create new v1 format
---
 be/src/olap/storage_engine.cpp                     |   9 +-
 docs/en/administrator-guide/config/fe_config.md    |   6 -
 docs/zh-CN/administrator-guide/config/fe_config.md |   6 -
 .../org/apache/doris/catalog/TableProperty.java    |   4 +
 .../main/java/org/apache/doris/common/Config.java  |   8 --
 .../apache/doris/common/util/PropertyAnalyzer.java |   7 +-
 .../org/apache/doris/system/HeartbeatFlags.java    |  27 ++--
 .../java/org/apache/doris/task/AgentBatchTask.java |  21 ---
 .../org/apache/doris/task/CreateReplicaTask.java   |   8 +-
 .../org/apache/doris/task/CreateRollupTask.java    | 157 ---------------------
 .../org/apache/doris/task/SchemaChangeTask.java    | 141 ------------------
 .../org/apache/doris/alter/AlterJobV2Test.java     |  56 --------
 .../java/org/apache/doris/task/AgentTaskTest.java  |  42 +-----
 13 files changed, 37 insertions(+), 455 deletions(-)

diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp
index 209ef86fb4..1e28d05562 100644
--- a/be/src/olap/storage_engine.cpp
+++ b/be/src/olap/storage_engine.cpp
@@ -121,7 +121,7 @@ StorageEngine::StorageEngine(const EngineOptions& options)
           _txn_manager(new TxnManager(config::txn_map_shard_size, config::txn_shard_size)),
           _rowset_id_generator(new UniqueRowsetIdGenerator(options.backend_uid)),
           _memtable_flush_executor(nullptr),
-          _default_rowset_type(ALPHA_ROWSET),
+          _default_rowset_type(BETA_ROWSET),
           _heartbeat_flags(nullptr),
           _stream_load_recorder(nullptr) {
     if (_s_instance == nullptr) {
@@ -859,8 +859,13 @@ void StorageEngine::_parse_default_rowset_type() {
     boost::to_upper(default_rowset_type_config);
     if (default_rowset_type_config == "BETA") {
         _default_rowset_type = BETA_ROWSET;
-    } else {
+    } else if (default_rowset_type_config == "ALPHA") {
         _default_rowset_type = ALPHA_ROWSET;
+        LOG(WARNING) << "default_rowset_type in be.conf should be set to beta, alpha is not "
+                        "supported any more";
+    } else {
+        LOG(FATAL) << "unknown value " << default_rowset_type_config
+                   << " in default_rowset_type in be.conf";
     }
 }
 
diff --git a/docs/en/administrator-guide/config/fe_config.md b/docs/en/administrator-guide/config/fe_config.md
index 6df6807f89..1a4f90a4ea 100644
--- a/docs/en/administrator-guide/config/fe_config.md
+++ b/docs/en/administrator-guide/config/fe_config.md
@@ -172,12 +172,6 @@ Default:false
 
 If set to true, FE will be started in BDBJE debug mode
 
-### enable_alpha_rowset
-
-Default:false
-
-Whether to support the creation of alpha rowset tables.  The default is false and should only be used in emergency situations,  this config should be remove in some future version
-
 ### enable_http_server_v2
 
 Default:The default is true after the official 0.14.0 version is released, and the default is false before
diff --git a/docs/zh-CN/administrator-guide/config/fe_config.md b/docs/zh-CN/administrator-guide/config/fe_config.md
index 0f1f6436d6..2636c6b940 100644
--- a/docs/zh-CN/administrator-guide/config/fe_config.md
+++ b/docs/zh-CN/administrator-guide/config/fe_config.md
@@ -171,12 +171,6 @@ FE 的配置项有两种方式进行配置:
 
 如果设置为 true,FE 将在 BDBJE 调试模式下启动,在 Web 页面 `System->bdbje` 可以查看相关信息,否则不可以查看
 
-### `enable_alpha_rowset`
-
-默认值:false
-
-是否支持创建 alpha rowset。默认为 false,只应在紧急情况下使用,此配置应在未来的某个版本中删除
-
 ### `enable_http_server_v2`
 
 默认值:从官方 0.14.0 release 版之后默认是 true,之前默认 false
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
index f007df52a2..844e4032c3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
@@ -199,6 +199,10 @@ public class TableProperty implements Writable {
     }
 
     public TStorageFormat getStorageFormat() {
+    	// Force convert all V1 table to V2 table
+    	if (TStorageFormat.V1 == storageFormat) {
+    		return TStorageFormat.V2;
+    	}
         return storageFormat;
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java
index 45cfac5e2d..80b144cc04 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java
@@ -1421,14 +1421,6 @@ public class Config extends ConfigBase {
     @ConfField
     public static String http_api_extra_base_path = "";
 
-    /**
-     * Whether to support the creation of alpha rowset tables.
-     * The default is false and should only be used in emergency situations,
-     * this config should be remove in some future version
-     */
-    @ConfField
-    public static boolean enable_alpha_rowset = false;
-
     /**
      * If set to true, FE will be started in BDBJE debug mode
      */
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
index ceac147bae..a4684f09c9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
@@ -397,11 +397,8 @@ public class PropertyAnalyzer {
         }
 
         if (storageFormat.equalsIgnoreCase("v1")) {
-            if (!Config.enable_alpha_rowset) {
-                throw new AnalysisException("Storage format V1 has been deprecated since version 0.14," +
-                        " please use V2 instead");
-            }
-            return TStorageFormat.V1;
+            throw new AnalysisException("Storage format V1 has been deprecated since version 0.14, "
+            		+ "please use V2 instead");
         } else if (storageFormat.equalsIgnoreCase("v2")) {
             return TStorageFormat.V2;
         } else if (storageFormat.equalsIgnoreCase("default")) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatFlags.java b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatFlags.java
index 7fd5f0fe63..a74ea444b3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatFlags.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatFlags.java
@@ -28,18 +28,23 @@ import org.apache.logging.log4j.Logger;
 // Now the flag is represented by 64-bit long type, each bit can be used to control
 // one behavior. The first bit is used for set default rowset type to beta flag.
 public class HeartbeatFlags {
-    private static final Logger LOG = LogManager.getLogger(HeartbeatFlags.class);
+	private static final Logger LOG = LogManager.getLogger(HeartbeatFlags.class);
 
-    public static boolean isValidRowsetType(String rowsetType) {
-        return "alpha".equalsIgnoreCase(rowsetType) || "beta".equalsIgnoreCase(rowsetType);
-    }
+	public static boolean isValidRowsetType(String rowsetType) {
+		return "alpha".equalsIgnoreCase(rowsetType) || "beta".equalsIgnoreCase(rowsetType);
+	}
 
-    public long getHeartbeatFlags() {
-        long heartbeatFlags = 0;
-        if ("beta".equalsIgnoreCase(GlobalVariable.defaultRowsetType)) {
-            heartbeatFlags |= HeartbeatServiceConstants.IS_SET_DEFAULT_ROWSET_TO_BETA_BIT;
-        }
+	public long getHeartbeatFlags() {
+		long heartbeatFlags = 0;
+		// If user set default rowset type to ALPHA, then convert it to beta, because 
+		// alpha rowset will be removed
+		if ("beta".equalsIgnoreCase(GlobalVariable.defaultRowsetType) 
+				|| "alpha".equalsIgnoreCase(GlobalVariable.defaultRowsetType)) {
+			heartbeatFlags |= HeartbeatServiceConstants.IS_SET_DEFAULT_ROWSET_TO_BETA_BIT;
+		} else {
+			throw new IllegalArgumentException("unknown DEFAULT_ROWSET_TYPE in global variable");
+		}
 
-        return heartbeatFlags;
-    }
+		return heartbeatFlags;
+	}
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java
index 9a40c0a810..d9f7d20c35 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java
@@ -24,7 +24,6 @@ import org.apache.doris.system.Backend;
 import org.apache.doris.thrift.BackendService;
 import org.apache.doris.thrift.TAgentServiceVersion;
 import org.apache.doris.thrift.TAgentTaskRequest;
-import org.apache.doris.thrift.TAlterTabletReq;
 import org.apache.doris.thrift.TAlterTabletReqV2;
 import org.apache.doris.thrift.TCheckConsistencyReq;
 import org.apache.doris.thrift.TClearAlterTaskRequest;
@@ -235,26 +234,6 @@ public class AgentBatchTask implements Runnable {
                 tAgentTaskRequest.setCloneReq(request);
                 return tAgentTaskRequest;
             }
-            case ROLLUP: {
-                CreateRollupTask rollupTask = (CreateRollupTask) task;
-                TAlterTabletReq request = rollupTask.toThrift();
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug(request.toString());
-                }
-                tAgentTaskRequest.setAlterTabletReq(request);
-                tAgentTaskRequest.setResourceInfo(rollupTask.getResourceInfo());
-                return tAgentTaskRequest;
-            }
-            case SCHEMA_CHANGE: {
-                SchemaChangeTask schemaChangeTask = (SchemaChangeTask) task;
-                TAlterTabletReq request = schemaChangeTask.toThrift();
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug(request.toString());
-                }
-                tAgentTaskRequest.setAlterTabletReq(request);
-                tAgentTaskRequest.setResourceInfo(schemaChangeTask.getResourceInfo());
-                return tAgentTaskRequest;
-            }
             case STORAGE_MEDIUM_MIGRATE: {
                 StorageMediaMigrationTask migrationTask = (StorageMediaMigrationTask) task;
                 TStorageMediumMigrateReq request = migrationTask.toThrift();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java
index 620791c18e..7ecf25979d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java
@@ -76,8 +76,10 @@ public class CreateReplicaTask extends AgentTask {
     // if base tablet id is set, BE will create the replica on same disk as this base tablet
     private long baseTabletId = -1;
     private int baseSchemaHash = -1;
-
-    private TStorageFormat storageFormat = null;
+    
+    // V2 is beta rowset, v1 is alpha rowset
+    // TODO should unify the naming of v1(alpha rowset), v2(beta rowset), it is very confused to read code
+    private TStorageFormat storageFormat = TStorageFormat.V2;	
 
     // true if this task is created by recover request(See comment of Config.recover_with_empty_tablet)
     private boolean isRecoverTask = false;
@@ -187,7 +189,7 @@ public class CreateReplicaTask extends AgentTask {
     }
 
     public void setStorageFormat(TStorageFormat storageFormat) {
-        this.storageFormat = storageFormat;
+    	this.storageFormat = storageFormat;
     }
 
     public TCreateTabletReq toThrift() {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/CreateRollupTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/CreateRollupTask.java
deleted file mode 100644
index a4fb118d06..0000000000
--- a/fe/fe-core/src/main/java/org/apache/doris/task/CreateRollupTask.java
+++ /dev/null
@@ -1,157 +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.task;
-
-import org.apache.doris.catalog.Column;
-import org.apache.doris.thrift.TAlterTabletReq;
-import org.apache.doris.thrift.TColumn;
-import org.apache.doris.thrift.TCreateTabletReq;
-import org.apache.doris.thrift.TKeysType;
-import org.apache.doris.thrift.TResourceInfo;
-import org.apache.doris.thrift.TStorageType;
-import org.apache.doris.thrift.TTabletSchema;
-import org.apache.doris.thrift.TTaskType;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-
-@Deprecated
-public class CreateRollupTask extends AgentTask {
-
-    private long baseTableId;
-    private long baseTabletId;
-
-    private long rollupReplicaId;
-
-    private int rollupSchemaHash;
-    private int baseSchemaHash;
-
-    private short shortKeyColumnCount;
-    private TStorageType storageType;
-    private TKeysType keysType;
-
-    private List<Column> rollupColumns;
-
-    // bloom filter columns
-    private Set<String> bfColumns;
-    private double bfFpp;
-
-    public CreateRollupTask(TResourceInfo resourceInfo, long backendId, long dbId, long tableId,
-                            long partitionId, long rollupIndexId, long baseIndexId, long rollupTabletId,
-                            long baseTabletId, long rollupReplicaId, short shortKeyColumnCount,
-                            int rollupSchemaHash, int baseSchemaHash, TStorageType storageType,
-                            List<Column> rollupColumns, Set<String> bfColumns, double bfFpp, TKeysType keysType) {
-        super(resourceInfo, backendId, TTaskType.ROLLUP, dbId, tableId, partitionId, rollupIndexId, rollupTabletId);
-
-        this.baseTableId = baseIndexId;
-        this.baseTabletId = baseTabletId;
-        this.rollupReplicaId = rollupReplicaId;
-
-        this.rollupSchemaHash = rollupSchemaHash;
-        this.baseSchemaHash = baseSchemaHash;
-
-        this.shortKeyColumnCount = shortKeyColumnCount;
-        this.storageType = storageType;
-        this.keysType = keysType;
-
-        this.rollupColumns = rollupColumns;
-
-        this.bfColumns = bfColumns;
-        this.bfFpp = bfFpp;
-    }
-
-    public TAlterTabletReq toThrift() {
-        TAlterTabletReq tAlterTabletReq = new TAlterTabletReq();
-        tAlterTabletReq.setBaseTabletId(baseTabletId);
-        tAlterTabletReq.setBaseSchemaHash(baseSchemaHash);
-
-        // make 1 TCreateTableReq
-        TCreateTabletReq createTabletReq = new TCreateTabletReq();
-        createTabletReq.setTabletId(tabletId);
-
-        // no need to set version
-        // schema
-        TTabletSchema tSchema = new TTabletSchema();
-        tSchema.setShortKeyColumnCount(shortKeyColumnCount);
-        tSchema.setSchemaHash(rollupSchemaHash);
-        tSchema.setStorageType(storageType);
-        tSchema.setKeysType(keysType);
-
-        List<TColumn> tColumns = new ArrayList<TColumn>();
-        int deleteSign = -1;
-        for (int i = 0; i < rollupColumns.size(); i++) {
-            Column column = rollupColumns.get(i);
-            TColumn tColumn = column.toThrift();
-            // is bloom filter column
-            if (bfColumns != null && bfColumns.contains(column.getName())) {
-                tColumn.setIsBloomFilterColumn(true);
-            }
-            tColumn.setVisible(column.isVisible());
-            if (column.isDeleteSignColumn()) {
-                deleteSign = i;
-            }
-            tColumns.add(tColumn);
-        }
-        tSchema.setColumns(tColumns);
-        tSchema.setDeleteSignIdx(deleteSign);
-
-        if (bfColumns != null) {
-            tSchema.setBloomFilterFpp(bfFpp);
-        }
-        createTabletReq.setTabletSchema(tSchema);
-        createTabletReq.setTableId(tableId);
-        createTabletReq.setPartitionId(partitionId);
-
-        tAlterTabletReq.setNewTabletReq(createTabletReq);
-
-        return tAlterTabletReq;
-    }
-
-    public long getBaseTableId() {
-        return baseTableId;
-    }
-
-    public long getBaseTabletId() {
-        return baseTabletId;
-    }
-
-    public long getRollupReplicaId() {
-        return rollupReplicaId;
-    }
-
-    public int getRollupSchemaHash() {
-        return rollupSchemaHash;
-    }
-
-    public int getBaseSchemaHash() {
-        return baseSchemaHash;
-    }
-
-    public short getShortKeyColumnCount() {
-        return shortKeyColumnCount;
-    }
-
-    public TStorageType getStorageType() {
-        return storageType;
-    }
-
-    public List<Column> getRollupColumns() {
-        return rollupColumns;
-    }
-}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/SchemaChangeTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/SchemaChangeTask.java
deleted file mode 100644
index 59616ccaff..0000000000
--- a/fe/fe-core/src/main/java/org/apache/doris/task/SchemaChangeTask.java
+++ /dev/null
@@ -1,141 +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.task;
-
-import org.apache.doris.catalog.Column;
-import org.apache.doris.thrift.TAlterTabletReq;
-import org.apache.doris.thrift.TColumn;
-import org.apache.doris.thrift.TCreateTabletReq;
-import org.apache.doris.thrift.TKeysType;
-import org.apache.doris.thrift.TResourceInfo;
-import org.apache.doris.thrift.TStorageType;
-import org.apache.doris.thrift.TTabletSchema;
-import org.apache.doris.thrift.TTaskType;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-
-@Deprecated
-public class SchemaChangeTask extends AgentTask {
-
-    private long baseReplicaId;
-    private int baseSchemaHash;
-    private TStorageType storageType;
-    private TKeysType keysType;
-
-    private int newSchemaHash;
-    private short newShortKeyColumnCount;
-    private List<Column> newColumns;
-
-    // bloom filter columns
-    private Set<String> bfColumns;
-    private double bfFpp;
-
-    public SchemaChangeTask(TResourceInfo resourceInfo, long backendId, long dbId, long tableId,
-                            long partitionId, long indexId, long baseTabletId, long baseReplicaId,
-                            List<Column> newColumns, int newSchemaHash, int baseSchemaHash,
-                            short newShortKeyColumnCount, TStorageType storageType,
-                            Set<String> bfColumns, double bfFpp, TKeysType keysType) {
-        super(resourceInfo, backendId, TTaskType.SCHEMA_CHANGE, dbId, tableId, partitionId, indexId, baseTabletId);
-
-        this.baseReplicaId = baseReplicaId;
-        this.baseSchemaHash = baseSchemaHash;
-        this.storageType = storageType;
-        this.keysType = keysType;
-
-        this.newSchemaHash = newSchemaHash;
-        this.newShortKeyColumnCount = newShortKeyColumnCount;
-        this.newColumns = newColumns;
-
-        this.bfColumns = bfColumns;
-        this.bfFpp = bfFpp;
-    }
-
-    public TAlterTabletReq toThrift() {
-        TAlterTabletReq tAlterTabletReq = new TAlterTabletReq();
-
-        tAlterTabletReq.setBaseTabletId(tabletId);
-        tAlterTabletReq.setBaseSchemaHash(baseSchemaHash);
-
-        // make 1 TCreateTableReq
-        TCreateTabletReq createTabletReq = new TCreateTabletReq();
-        createTabletReq.setTabletId(tabletId);
-
-        // no need to set version
-        // schema
-        TTabletSchema tSchema = new TTabletSchema();
-        tSchema.setShortKeyColumnCount(newShortKeyColumnCount);
-        tSchema.setSchemaHash(newSchemaHash);
-        tSchema.setStorageType(storageType);
-        tSchema.setKeysType(keysType);
-        int deleteSign = -1;
-        List<TColumn> tColumns = new ArrayList<TColumn>();
-        for (int i = 0; i < newColumns.size(); i++) {
-            Column column = newColumns.get(i);
-            TColumn tColumn = column.toThrift();
-            // is bloom filter column
-            if (bfColumns != null && bfColumns.contains(column.getName())) {
-                tColumn.setIsBloomFilterColumn(true);
-            }
-            tColumn.setVisible(column.isVisible());
-            if (column.isDeleteSignColumn()) {
-                deleteSign = i;
-            }
-            tColumns.add(tColumn);
-        }
-        tSchema.setColumns(tColumns);
-        tSchema.setDeleteSignIdx(deleteSign);
-
-        if (bfColumns != null) {
-            tSchema.setBloomFilterFpp(bfFpp);
-        }
-        createTabletReq.setTabletSchema(tSchema);
-        createTabletReq.setTableId(tableId);
-        createTabletReq.setPartitionId(partitionId);
-
-        tAlterTabletReq.setNewTabletReq(createTabletReq);
-
-        return tAlterTabletReq;
-    }
-
-    public long getReplicaId() {
-        return baseReplicaId;
-    }
-
-    public int getSchemaHash() {
-        return newSchemaHash;
-    }
-
-    public int getBaseSchemaHash() {
-        return baseSchemaHash;
-    }
-
-    public short getNewShortKeyColumnCount() {
-        return newShortKeyColumnCount;
-    }
-
-    public TStorageType getStorageType() {
-        return storageType;
-    }
-
-    public List<Column> getColumns() {
-        return newColumns;
-    }
-
-}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
index 20f44dc60a..289e45c285 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
@@ -57,7 +57,6 @@ public class AlterJobV2Test {
         FeConstants.runningUnitTest = true;
 
         UtFrameUtils.createDorisCluster(runningDir);
-        Config.enable_alpha_rowset = true;
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
@@ -67,8 +66,6 @@ public class AlterJobV2Test {
         Catalog.getCurrentCatalog().createDb(createDbStmt);
 
         createTable("CREATE TABLE test.schema_change_test(k1 int, k2 int, k3 int) distributed by hash(k1) buckets 3 properties('replication_num' = '1');");
-
-        createTable("CREATE TABLE test.segmentv2(k1 int, k2 int, v1 int sum) distributed by hash(k1) buckets 3 properties('replication_num' = '1', 'storage_format' = 'v1');");
     }
 
     @AfterClass
@@ -145,59 +142,6 @@ public class AlterJobV2Test {
         System.out.println(showResultSet.getResultRows());
     }
 
-    @Test
-    @Deprecated
-    public void testAlterSegmentV2() throws Exception {
-        // TODO this test should remove after we disable segment v1 completely
-        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
-        OlapTable tbl = db.getTableOrMetaException("segmentv2", Table.TableType.OLAP);
-        Assert.assertEquals(TStorageFormat.V1, tbl.getTableProperty().getStorageFormat());
-
-        // 1. create a rollup r1
-        String alterStmtStr = "alter table test.segmentv2 add rollup r1(k2, v1)";
-        AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseAndAnalyzeStmt(alterStmtStr, connectContext);
-        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(alterTableStmt);
-        Map<Long, AlterJobV2> alterJobs = Catalog.getCurrentCatalog().getMaterializedViewHandler().getAlterJobsV2();
-        waitAlterJobDone(alterJobs);
-
-        String sql = "select k2, sum(v1) from test.segmentv2 group by k2";
-        String explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "explain " + sql);
-        Assert.assertTrue(explainString.contains("rollup: r1"));
-
-        // 2. create a rollup with segment v2
-        alterStmtStr = "alter table test.segmentv2 add rollup segmentv2(k2, v1) properties('storage_format' = 'v2')";
-        alterTableStmt = (AlterTableStmt) UtFrameUtils.parseAndAnalyzeStmt(alterStmtStr, connectContext);
-        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(alterTableStmt);
-        alterJobs = Catalog.getCurrentCatalog().getMaterializedViewHandler().getAlterJobsV2();
-        waitAlterJobDone(alterJobs);
-
-        explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "explain " + sql);
-        Assert.assertTrue(explainString.contains("rollup: r1"));
-
-        // set use_v2_rollup = true;
-        connectContext.getSessionVariable().setUseV2Rollup(true);
-        explainString = UtFrameUtils.getSQLPlanOrErrorMsg(connectContext, "explain " + sql);
-        Assert.assertTrue(explainString.contains("rollup: __v2_segmentv2"));
-
-        // 3. process alter segment v2
-        alterStmtStr = "alter table test.segmentv2 set ('storage_format' = 'v2');";
-        alterTableStmt = (AlterTableStmt) UtFrameUtils.parseAndAnalyzeStmt(alterStmtStr, connectContext);
-        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(alterTableStmt);
-        // 4. check alter job
-        alterJobs = Catalog.getCurrentCatalog().getSchemaChangeHandler().getAlterJobsV2();
-        waitAlterJobDone(alterJobs);
-        // 5. check storage format of table
-        Assert.assertEquals(TStorageFormat.V2, tbl.getTableProperty().getStorageFormat());
-
-        // 6. alter again, that no job will be created
-        try {
-            Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(alterTableStmt);
-            Assert.fail();
-        } catch (DdlException e) {
-            Assert.assertTrue(e.getMessage().contains("Nothing is changed"));
-        }
-    }
-
     @Test
     public void testDupTableSchemaChange() throws Exception {
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java
index 375c86fc01..ca31e8de72 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java
@@ -28,7 +28,6 @@ import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.MarkedCountDownLatch;
 import org.apache.doris.thrift.TAgentTaskRequest;
 import org.apache.doris.thrift.TBackend;
-import org.apache.doris.thrift.TKeysType;
 import org.apache.doris.thrift.TPriority;
 import org.apache.doris.thrift.TPushType;
 import org.apache.doris.thrift.TStorageMedium;
@@ -86,8 +85,6 @@ public class AgentTaskTest {
     private AgentTask dropTask;
     private AgentTask pushTask;
     private AgentTask cloneTask;
-    private AgentTask rollupTask;
-    private AgentTask schemaChangeTask;
     private AgentTask cancelDeleteTask;
 
     @Before
@@ -129,17 +126,6 @@ public class AgentTaskTest {
                 new CloneTask(backendId1, dbId, tableId, partitionId, indexId1, tabletId1, schemaHash1,
                         Arrays.asList(new TBackend("host1", 8290, 8390)), TStorageMedium.HDD, -1, 3600);
 
-        // rollup
-        rollupTask =
-                new CreateRollupTask(null, backendId1, dbId, tableId, partitionId, indexId2, indexId1,
-                                     tabletId2, tabletId1, replicaId2, shortKeyNum, schemaHash2, schemaHash1,
-                                     storageType, columns, null, 0, TKeysType.AGG_KEYS);
-
-        // schemaChange
-        schemaChangeTask =
-                new SchemaChangeTask(null, backendId1, dbId, tableId, partitionId, indexId1, 
-                                     tabletId1, replicaId1, columns, schemaHash2, schemaHash1, 
-                                     shortKeyNum, storageType, null, 0, TKeysType.AGG_KEYS);
     }
 
     @Test
@@ -152,17 +138,12 @@ public class AgentTaskTest {
         agentBatchTask.addTask(createReplicaTask);
         Assert.assertEquals(1, agentBatchTask.getTaskNum());
 
-        agentBatchTask.addTask(rollupTask);
-        Assert.assertEquals(2, agentBatchTask.getTaskNum());
-
         List<AgentTask> allTasks = agentBatchTask.getAllTasks();
-        Assert.assertEquals(2, allTasks.size());
+        Assert.assertEquals(1, allTasks.size());
 
         for (AgentTask agentTask : allTasks) {
             if (agentTask instanceof CreateReplicaTask) {
                 Assert.assertEquals(createReplicaTask, agentTask);
-            } else if (agentTask instanceof CreateRollupTask) {
-                Assert.assertEquals(rollupTask, agentTask);
             } else {
                 Assert.fail();
             }
@@ -200,17 +181,6 @@ public class AgentTaskTest {
         Assert.assertEquals(cloneTask.getSignature(), request4.getSignature());
         Assert.assertNotNull(request4.getCloneReq());
 
-        // rollup
-        TAgentTaskRequest request5 = (TAgentTaskRequest) toAgentTaskRequest.invoke(agentBatchTask, rollupTask);
-        Assert.assertEquals(TTaskType.ROLLUP, request5.getTaskType());
-        Assert.assertEquals(rollupTask.getSignature(), request5.getSignature());
-        Assert.assertNotNull(request5.getAlterTabletReq());
-
-        // schemaChange
-        TAgentTaskRequest request6 = (TAgentTaskRequest) toAgentTaskRequest.invoke(agentBatchTask, schemaChangeTask);
-        Assert.assertEquals(TTaskType.SCHEMA_CHANGE, request6.getTaskType());
-        Assert.assertEquals(schemaChangeTask.getSignature(), request6.getSignature());
-        Assert.assertNotNull(request6.getAlterTabletReq());
     }
 
     @Test
@@ -227,24 +197,18 @@ public class AgentTaskTest {
         AgentTask task = AgentTaskQueue.getTask(backendId1, TTaskType.CREATE, createReplicaTask.getSignature());
         Assert.assertEquals(createReplicaTask, task);
 
-        // diff
-        AgentTaskQueue.addTask(rollupTask);
-
         Map<TTaskType, Set<Long>> runningTasks = new HashMap<TTaskType, Set<Long>>();
         List<AgentTask> diffTasks = AgentTaskQueue.getDiffTasks(backendId1, runningTasks);
-        Assert.assertEquals(2, diffTasks.size());
+        Assert.assertEquals(1, diffTasks.size());
 
         Set<Long> set = new HashSet<Long>();
         set.add(createReplicaTask.getSignature());
         runningTasks.put(TTaskType.CREATE, set);
         diffTasks = AgentTaskQueue.getDiffTasks(backendId1, runningTasks);
-        Assert.assertEquals(1, diffTasks.size());
-        Assert.assertEquals(rollupTask, diffTasks.get(0));
+        Assert.assertEquals(0, diffTasks.size());
 
         // remove
         AgentTaskQueue.removeTask(backendId1, TTaskType.CREATE, createReplicaTask.getSignature());
-        Assert.assertEquals(1, AgentTaskQueue.getTaskNum());
-        AgentTaskQueue.removeTask(backendId1, TTaskType.ROLLUP, rollupTask.getSignature());
         Assert.assertEquals(0, AgentTaskQueue.getTaskNum());
     }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 21/22: [Feature] add `weekday` function on vectorized engine (#9901)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 03748e4b17a9fafcce8b26a5f6ea59e5229909fd
Author: Gabriel <ga...@gmail.com>
AuthorDate: Wed Jun 1 14:47:37 2022 +0800

    [Feature] add `weekday` function on vectorized engine (#9901)
---
 be/src/vec/functions/date_time_transforms.h |  1 +
 be/src/vec/functions/time_of_function.cpp   |  2 ++
 be/test/vec/function/function_time_test.cpp | 23 +++++++++++++++++++++++
 3 files changed, 26 insertions(+)

diff --git a/be/src/vec/functions/date_time_transforms.h b/be/src/vec/functions/date_time_transforms.h
index 7640a2707d..ef54798025 100644
--- a/be/src/vec/functions/date_time_transforms.h
+++ b/be/src/vec/functions/date_time_transforms.h
@@ -57,6 +57,7 @@ TIME_FUNCTION_IMPL(WeekOfYearImpl, weekofyear, week(mysql_week_mode(3)));
 TIME_FUNCTION_IMPL(DayOfYearImpl, dayofyear, day_of_year());
 TIME_FUNCTION_IMPL(DayOfMonthImpl, dayofmonth, day());
 TIME_FUNCTION_IMPL(DayOfWeekImpl, dayofweek, day_of_week());
+TIME_FUNCTION_IMPL(WeekDayImpl, weekday, weekday());
 // TODO: the method should be always not nullable
 TIME_FUNCTION_IMPL(ToDaysImpl, to_days, daynr());
 
diff --git a/be/src/vec/functions/time_of_function.cpp b/be/src/vec/functions/time_of_function.cpp
index 1d364f528d..071704826d 100644
--- a/be/src/vec/functions/time_of_function.cpp
+++ b/be/src/vec/functions/time_of_function.cpp
@@ -27,6 +27,7 @@ using FunctionDayOfYear = FunctionDateOrDateTimeToSomething<DataTypeInt32, DayOf
 using FunctionDayOfWeek = FunctionDateOrDateTimeToSomething<DataTypeInt32, DayOfWeekImpl>;
 using FunctionDayOfMonth = FunctionDateOrDateTimeToSomething<DataTypeInt32, DayOfMonthImpl>;
 using FunctionYearWeek = FunctionDateOrDateTimeToSomething<DataTypeInt32, ToYearWeekOneArgImpl>;
+using FunctionWeekDay = FunctionDateOrDateTimeToSomething<DataTypeInt32, WeekDayImpl>;
 
 void register_function_time_of_function(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionDayOfWeek>();
@@ -34,5 +35,6 @@ void register_function_time_of_function(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionDayOfYear>();
     factory.register_function<FunctionWeekOfYear>();
     factory.register_function<FunctionYearWeek>();
+    factory.register_function<FunctionWeekDay>();
 }
 } // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/test/vec/function/function_time_test.cpp b/be/test/vec/function/function_time_test.cpp
index 7e45295c4d..70841072e6 100644
--- a/be/test/vec/function/function_time_test.cpp
+++ b/be/test/vec/function/function_time_test.cpp
@@ -551,6 +551,29 @@ TEST(TimestampFunctionsTest, convert_tz_test) {
 
     check_function<DataTypeDate, true>(func_name, input_types, data_set);
 }
+
+TEST(VTimestampFunctionsTest, weekday_test) {
+    std::string func_name = "weekday";
+
+    {
+        InputTypeSet input_types = {TypeIndex::DateTime};
+
+        DataSet data_set = {{{std::string("2001-02-03 12:34:56")}, 5},
+                            {{std::string("2019-06-25")}, 1},
+                            {{std::string("2020-00-01 00:00:00")}, Null()},
+                            {{std::string("2020-01-00 00:00:00")}, Null()}};
+
+        check_function<DataTypeInt32, true>(func_name, input_types, data_set);
+    }
+    InputTypeSet input_types = {TypeIndex::Date};
+
+    DataSet data_set = {{{std::string("2001-02-03")}, 5},
+                        {{std::string("2019-06-25")}, 1},
+                        {{std::string("2020-00-01")}, Null()},
+                        {{std::string("2020-01-00")}, Null()}};
+
+    check_function<DataTypeInt32, true>(func_name, input_types, data_set);
+}
 } // namespace doris::vectorized
 
 int main(int argc, char** argv) {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 22/22: [fix](routine-load) fix bug that routine load task can not find backend (#9902)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 07a119abfbac0ce29981f048cd8d9df326f1c4e7
Author: Mingyu Chen <mo...@gmail.com>
AuthorDate: Wed Jun 1 17:55:30 2022 +0800

    [fix](routine-load) fix bug that routine load task can not find backend (#9902)
    
    Introduced from #9492.
---
 .../doris/load/routineload/RoutineLoadManager.java  |  5 +++--
 .../org/apache/doris/system/SystemInfoService.java  | 21 ++++++++++++++++-----
 .../apache/doris/system/SystemInfoServiceTest.java  | 13 +++++++++----
 3 files changed, 28 insertions(+), 11 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java
index ed220f88b8..9dac66b996 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java
@@ -496,13 +496,14 @@ public class RoutineLoadManager implements Writable {
         } else {
             tags = Catalog.getCurrentCatalog().getAuth().getResourceTags(job.getUserIdentity().getQualifiedUser());
             if (tags == UserProperty.INVALID_RESOURCE_TAGS) {
-                // user may be dropped. Here we fall back to use replica tag
+                // user may be dropped, or may not set resource tag property.
+                // Here we fall back to use replica tag
                 tags = getTagsFromReplicaAllocation(job.getDbId(), job.getTableId());
             }
         }
         BeSelectionPolicy policy = new BeSelectionPolicy.Builder().needLoadAvailable().setCluster(cluster)
                 .addTags(tags).build();
-        return Catalog.getCurrentSystemInfo().selectBackendIdsByPolicy(policy, 20000);
+        return Catalog.getCurrentSystemInfo().selectBackendIdsByPolicy(policy, -1 /* as many as possible */);
     }
 
     private Set<Tag> getTagsFromReplicaAllocation(long dbId, long tblId) throws LoadException {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java
index 82571cd3d8..ef31b53e19 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java
@@ -776,14 +776,16 @@ public class SystemInfoService {
      * Select a set of backends by the given policy.
      *
      * @param policy
-     * @param number number of backends which need to be selected.
+     * @param number number of backends which need to be selected. -1 means return as many as possible.
      * @return return #number of backend ids,
      * or empty set if no backends match the policy, or the number of matched backends is less than "number";
      */
     public List<Long> selectBackendIdsByPolicy(BeSelectionPolicy policy, int number) {
+        Preconditions.checkArgument(number >= -1);
         List<Backend> candidates =
                 idToBackendRef.values().stream().filter(policy::isMatch).collect(Collectors.toList());
-        if (candidates.size() < number) {
+        if ((number != -1 && candidates.size() < number) || candidates.isEmpty()) {
+            LOG.debug("Not match policy: {}. candidates num: {}, expected: {}", policy, candidates.size(), number);
             return Lists.newArrayList();
         }
         // If only need one Backend, just return a random one.
@@ -794,7 +796,11 @@ public class SystemInfoService {
 
         if (policy.allowOnSameHost) {
             Collections.shuffle(candidates);
-            return candidates.subList(0, number).stream().map(b -> b.getId()).collect(Collectors.toList());
+            if (number == -1) {
+                return candidates.stream().map(b -> b.getId()).collect(Collectors.toList());
+            } else {
+                return candidates.subList(0, number).stream().map(b -> b.getId()).collect(Collectors.toList());
+            }
         }
 
         // for each host, random select one backend.
@@ -813,11 +819,16 @@ public class SystemInfoService {
             Collections.shuffle(list);
             candidates.add(list.get(0));
         }
-        if (candidates.size() < number) {
+        if (number != -1 && candidates.size() < number) {
+            LOG.debug("Not match policy: {}. candidates num: {}, expected: {}", policy, candidates.size(), number);
             return Lists.newArrayList();
         }
         Collections.shuffle(candidates);
-        return candidates.subList(0, number).stream().map(b -> b.getId()).collect(Collectors.toList());
+        if (number != -1) {
+            return candidates.subList(0, number).stream().map(b -> b.getId()).collect(Collectors.toList());
+        } else {
+            return candidates.stream().map(b -> b.getId()).collect(Collectors.toList());
+        }
     }
 
     public ImmutableMap<Long, Backend> getIdToBackend() {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/system/SystemInfoServiceTest.java b/fe/fe-core/src/test/java/org/apache/doris/system/SystemInfoServiceTest.java
index b2570095a0..9134c66c44 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/system/SystemInfoServiceTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/system/SystemInfoServiceTest.java
@@ -185,15 +185,20 @@ public class SystemInfoServiceTest {
         BeSelectionPolicy policy10 = new BeSelectionPolicy.Builder().addTags(Sets.newHashSet(taga, tagb))
                 .setStorageMedium(TStorageMedium.SSD).build();
         Assert.assertEquals(4, infoService.selectBackendIdsByPolicy(policy10, 4).size());
+        Assert.assertEquals(3, infoService.selectBackendIdsByPolicy(policy10, 3).size());
+        // check return as many as possible
+        Assert.assertEquals(4, infoService.selectBackendIdsByPolicy(policy10, -1).size());
         Assert.assertEquals(0, infoService.selectBackendIdsByPolicy(policy10, 5).size());
 
-        BeSelectionPolicy policy11 = new BeSelectionPolicy.Builder().addTags(Sets.newHashSet(tagb))
-                .setStorageMedium(TStorageMedium.HDD).build();
+        BeSelectionPolicy policy11 =
+                new BeSelectionPolicy.Builder().addTags(Sets.newHashSet(tagb)).setStorageMedium(TStorageMedium.HDD)
+                        .build();
         Assert.assertEquals(0, infoService.selectBackendIdsByPolicy(policy11, 1).size());
 
         // 7. check disk usage
-        BeSelectionPolicy policy12 = new BeSelectionPolicy.Builder().addTags(Sets.newHashSet(taga))
-                .setStorageMedium(TStorageMedium.HDD).build();
+        BeSelectionPolicy policy12 =
+                new BeSelectionPolicy.Builder().addTags(Sets.newHashSet(taga)).setStorageMedium(TStorageMedium.HDD)
+                        .build();
         Assert.assertEquals(1, infoService.selectBackendIdsByPolicy(policy12, 1).size());
         BeSelectionPolicy policy13 = new BeSelectionPolicy.Builder().addTags(Sets.newHashSet(taga))
                 .setStorageMedium(TStorageMedium.HDD).needCheckDiskUsage().build();


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 17/22: [Bug][Fix] One Rowset have same key output in unique table (#9858)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit f5386cb8e7b38471d6f19c56b3e51779dd6d3260
Author: HappenLee <ha...@hotmail.com>
AuthorDate: Tue May 31 12:29:16 2022 +0800

    [Bug][Fix] One Rowset have same key output in unique table (#9858)
    
    Co-authored-by: lihaopeng <li...@baidu.com>
---
 be/src/olap/collect_iterator.cpp             |  2 +-
 be/src/olap/generic_iterators.cpp            | 52 ++++++++++++++++--------
 be/src/olap/generic_iterators.h              |  2 +-
 be/src/olap/reader.cpp                       |  1 +
 be/src/olap/rowset/beta_rowset_reader.cpp    |  4 +-
 be/src/olap/rowset/rowset_reader_context.h   |  1 +
 be/src/olap/schema_change.cpp                |  2 +
 be/src/olap/tuple_reader.cpp                 | 25 ++++++++----
 be/src/vec/olap/vcollect_iterator.cpp        |  8 +---
 be/src/vec/olap/vgeneric_iterators.cpp       | 44 +++++++++++++-------
 be/src/vec/olap/vgeneric_iterators.h         |  2 +-
 be/test/olap/generic_iterators_test.cpp      | 38 ++++++++++++++++-
 be/test/vec/exec/vgeneric_iterators_test.cpp | 61 ++++++++++++++++++++++------
 13 files changed, 175 insertions(+), 67 deletions(-)

diff --git a/be/src/olap/collect_iterator.cpp b/be/src/olap/collect_iterator.cpp
index 404da7e35d..df1f3231a8 100644
--- a/be/src/olap/collect_iterator.cpp
+++ b/be/src/olap/collect_iterator.cpp
@@ -117,7 +117,7 @@ bool CollectIterator::LevelIteratorComparator::operator()(const LevelIterator* a
         return cmp_res > 0;
     }
 
-    // Second: If sequence_id_idx != 0 means we need to compare sequence. sequence only use
+    // Second: If _sequence_id_idx != 0 means we need to compare sequence. sequence only use
     // in unique key. so keep reverse order here
     if (_sequence_id_idx != -1) {
         auto seq_first_cell = first->cell(_sequence_id_idx);
diff --git a/be/src/olap/generic_iterators.cpp b/be/src/olap/generic_iterators.cpp
index 1b8f176637..ba4cc5559d 100644
--- a/be/src/olap/generic_iterators.cpp
+++ b/be/src/olap/generic_iterators.cpp
@@ -150,6 +150,10 @@ public:
 
     uint64_t data_id() const { return _iter->data_id(); }
 
+    bool need_skip() const { return _skip; }
+
+    void set_skip(bool skip) const { _skip = skip; }
+
 private:
     // Load next block into _block
     Status _load_next_block();
@@ -161,6 +165,7 @@ private:
     RowBlockV2 _block;
 
     bool _valid = false;
+    mutable bool _skip = false;
     size_t _index_in_block = -1;
 };
 
@@ -174,6 +179,7 @@ Status MergeIteratorContext::init(const StorageReadOptions& opts) {
 }
 
 Status MergeIteratorContext::advance() {
+    _skip = false;
     // NOTE: we increase _index_in_block directly to valid one check
     do {
         _index_in_block++;
@@ -207,8 +213,8 @@ Status MergeIteratorContext::_load_next_block() {
 class MergeIterator : public RowwiseIterator {
 public:
     // MergeIterator takes the ownership of input iterators
-    MergeIterator(std::vector<RowwiseIterator*> iters, std::shared_ptr<MemTracker> parent, int sequence_id_idx)
-        : _origin_iters(std::move(iters)), _sequence_id_idx(sequence_id_idx), _merge_heap(MergeContextComparator(_sequence_id_idx)) {
+    MergeIterator(std::vector<RowwiseIterator*> iters, std::shared_ptr<MemTracker> parent, int sequence_id_idx, bool is_unique)
+        : _origin_iters(std::move(iters)), _sequence_id_idx(sequence_id_idx), _is_unique(is_unique), _merge_heap(MergeContextComparator(_sequence_id_idx, is_unique)) {
         // use for count the mem use of Block use in Merge
         _mem_tracker = MemTracker::CreateTracker(-1, "MergeIterator", std::move(parent), false);
     }
@@ -232,11 +238,13 @@ private:
     std::vector<RowwiseIterator*> _origin_iters;
 
     int _sequence_id_idx;
+    bool _is_unique;
 
     std::unique_ptr<Schema> _schema;
 
     struct MergeContextComparator {
-        explicit MergeContextComparator(int idx) : sequence_id_idx(idx) {};
+        MergeContextComparator(int idx, bool is_unique)
+                : _sequence_id_idx(idx), _is_unique(is_unique) {};
 
         bool operator()(const MergeIteratorContext* lhs, const MergeIteratorContext* rhs) const {
             auto lhs_row = lhs->current_row();
@@ -245,23 +253,29 @@ private:
             if (cmp_res != 0) {
                 return cmp_res > 0;
             }
-            
-            // Second: If sequence_id_idx != 0 means we need to compare sequence. sequence only use
+            auto res = 0;
+            // Second: If _sequence_id_idx != 0 means we need to compare sequence. sequence only use
             // in unique key. so keep reverse order of sequence id here
-            if (sequence_id_idx != -1) {
-                auto l_cell = lhs_row.cell(sequence_id_idx);
-                auto r_cell = rhs_row.cell(sequence_id_idx);
-                auto res = lhs_row.schema()->column(sequence_id_idx)->compare_cell(l_cell, r_cell);
-                if (res != 0) return res < 0;
+            if (_sequence_id_idx != -1) {
+                auto l_cell = lhs_row.cell(_sequence_id_idx);
+                auto r_cell = rhs_row.cell(_sequence_id_idx);
+                res = lhs_row.schema()->column(_sequence_id_idx)->compare_cell(l_cell, r_cell);
             }
+
             // if row cursors equal, compare segment id.
             // here we sort segment id in reverse order, because of the row order in AGG_KEYS
             // dose no matter, but in UNIQUE_KEYS table we only read the latest is one, so we
             // return the row in reverse order of segment id
-            return lhs->data_id() < rhs->data_id();
+            bool result = res == 0 ? lhs->data_id() < rhs->data_id() : res < 0;
+            if (_is_unique) {
+                result ? lhs->set_skip(true) : rhs->set_skip(true);
+            }
+
+            return result;
         }
 
-        int sequence_id_idx;
+        int _sequence_id_idx;
+        bool _is_unique;
     };
 
     using MergeHeap = std::priority_queue<MergeIteratorContext*, 
@@ -292,13 +306,15 @@ Status MergeIterator::init(const StorageReadOptions& opts) {
 
 Status MergeIterator::next_batch(RowBlockV2* block) {
     size_t row_idx = 0;
-    for (; row_idx < block->capacity() && !_merge_heap.empty(); ++row_idx) {
+    for (; row_idx < block->capacity() && !_merge_heap.empty();) {
         auto ctx = _merge_heap.top();
         _merge_heap.pop();
 
-        RowBlockRow dst_row = block->row(row_idx);
-        // copy current row to block
-        copy_row(&dst_row, ctx->current_row(), block->pool());
+        if (!ctx->need_skip()) {
+            RowBlockRow dst_row = block->row(row_idx++);
+            // copy current row to block
+            copy_row(&dst_row, ctx->current_row(), block->pool());
+        }
 
         RETURN_IF_ERROR(ctx->advance());
         if (ctx->valid()) {
@@ -374,11 +390,11 @@ Status UnionIterator::next_batch(RowBlockV2* block) {
     return Status::EndOfFile("End of UnionIterator");
 }
 
-RowwiseIterator* new_merge_iterator(std::vector<RowwiseIterator*> inputs, std::shared_ptr<MemTracker> parent, int sequence_id_idx) {
+RowwiseIterator* new_merge_iterator(std::vector<RowwiseIterator*> inputs, std::shared_ptr<MemTracker> parent, int sequence_id_idx, bool is_unique) {
     if (inputs.size() == 1) {
         return *(inputs.begin());
     }
-    return new MergeIterator(std::move(inputs), parent, sequence_id_idx);
+    return new MergeIterator(std::move(inputs), parent, sequence_id_idx, is_unique);
 }
 
 RowwiseIterator* new_union_iterator(std::vector<RowwiseIterator*>& inputs, std::shared_ptr<MemTracker> parent) {
diff --git a/be/src/olap/generic_iterators.h b/be/src/olap/generic_iterators.h
index e8f4528885..1a6ca90030 100644
--- a/be/src/olap/generic_iterators.h
+++ b/be/src/olap/generic_iterators.h
@@ -25,7 +25,7 @@ namespace doris {
 //
 // Inputs iterators' ownership is taken by created merge iterator. And client
 // should delete returned iterator after usage.
-RowwiseIterator* new_merge_iterator(std::vector<RowwiseIterator*> inputs, std::shared_ptr<MemTracker> parent, int sequence_id_idx);
+RowwiseIterator* new_merge_iterator(std::vector<RowwiseIterator*> inputs, std::shared_ptr<MemTracker> parent, int sequence_id_idx, bool is_unique);
 
 // Create a union iterator for input iterators. Union iterator will read
 // input iterators one by one.
diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp
index 4542ae3f65..819e84a9a1 100644
--- a/be/src/olap/reader.cpp
+++ b/be/src/olap/reader.cpp
@@ -223,6 +223,7 @@ OLAPStatus TabletReader::_capture_rs_readers(const ReaderParams& read_params,
     _reader_context.use_page_cache = read_params.use_page_cache;
     _reader_context.sequence_id_idx = _sequence_col_idx;
     _reader_context.batch_size = _batch_size;
+    _reader_context.is_unique = tablet()->keys_type() == UNIQUE_KEYS;
 
     *valid_rs_readers = *rs_readers;
 
diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp b/be/src/olap/rowset/beta_rowset_reader.cpp
index 4e475fa9b1..3148656dd6 100644
--- a/be/src/olap/rowset/beta_rowset_reader.cpp
+++ b/be/src/olap/rowset/beta_rowset_reader.cpp
@@ -120,13 +120,13 @@ OLAPStatus BetaRowsetReader::init(RowsetReaderContext* read_context) {
     RowwiseIterator* final_iterator;
     if (config::enable_storage_vectorization && read_context->is_vec) {
         if (read_context->need_ordered_result && _rowset->rowset_meta()->is_segments_overlapping()) {
-            final_iterator = vectorized::new_merge_iterator(iterators, _parent_tracker, read_context->sequence_id_idx);
+            final_iterator = vectorized::new_merge_iterator(iterators, _parent_tracker, read_context->sequence_id_idx, read_context->is_unique);
         } else {
             final_iterator = vectorized::new_union_iterator(iterators, _parent_tracker);
         }
     } else {
         if (read_context->need_ordered_result && _rowset->rowset_meta()->is_segments_overlapping()) {
-            final_iterator = new_merge_iterator(iterators, _parent_tracker, read_context->sequence_id_idx);
+            final_iterator = new_merge_iterator(iterators, _parent_tracker, read_context->sequence_id_idx, read_context->is_unique);
         } else {
             final_iterator = new_union_iterator(iterators, _parent_tracker);
         }
diff --git a/be/src/olap/rowset/rowset_reader_context.h b/be/src/olap/rowset/rowset_reader_context.h
index 07d9340fdf..0ae42f6cf4 100644
--- a/be/src/olap/rowset/rowset_reader_context.h
+++ b/be/src/olap/rowset/rowset_reader_context.h
@@ -63,6 +63,7 @@ struct RowsetReaderContext {
     int sequence_id_idx = -1;
     int batch_size = 1024;
     bool is_vec = false;
+    bool is_unique = false;
 };
 
 } // namespace doris
diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp
index 0cd9ae7567..b27fea5f71 100644
--- a/be/src/olap/schema_change.cpp
+++ b/be/src/olap/schema_change.cpp
@@ -1505,6 +1505,7 @@ OLAPStatus SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletRe
         // for schema change, seek_columns is the same to return_columns
         reader_context.seek_columns = &return_columns;
         reader_context.sequence_id_idx = reader_context.tablet_schema->sequence_col_idx();
+        reader_context.is_unique = base_tablet->keys_type() == UNIQUE_KEYS;
 
         auto mem_tracker = MemTracker::CreateTracker(-1, "AlterTablet:" + std::to_string(base_tablet->tablet_id()) + "-"
                                                          + std::to_string(new_tablet->tablet_id()), _mem_tracker, true, false, MemTrackerLevel::TASK);
@@ -1724,6 +1725,7 @@ OLAPStatus SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tabl
     reader_context.return_columns = &return_columns;
     reader_context.seek_columns = &return_columns;
     reader_context.sequence_id_idx = reader_context.tablet_schema->sequence_col_idx();
+    reader_context.is_unique = base_tablet->keys_type() == UNIQUE_KEYS;
 
     RowsetReaderSharedPtr rowset_reader;
     RETURN_NOT_OK((*base_rowset)->create_reader(_mem_tracker, &rowset_reader));
diff --git a/be/src/olap/tuple_reader.cpp b/be/src/olap/tuple_reader.cpp
index 5c15c2b42f..33640777f5 100644
--- a/be/src/olap/tuple_reader.cpp
+++ b/be/src/olap/tuple_reader.cpp
@@ -185,15 +185,24 @@ OLAPStatus TupleReader::_unique_key_next_row(RowCursor* row_cursor, MemPool* mem
         // in UNIQUE_KEY highest version is the final result, there is no need to
         // merge the lower versions
         direct_copy_row(row_cursor, *_next_key);
-        // skip the lower version rows;
-        auto res = _collect_iter.next(&_next_key, &_next_delete_flag);
-        if (LIKELY(res != OLAP_ERR_DATA_EOF)) {
-            if (UNLIKELY(res != OLAP_SUCCESS)) {
-                LOG(WARNING) << "next failed: " << res;
-                return res;
+        while (_next_key) {
+            // skip the lower version rows;
+            auto res = _collect_iter.next(&_next_key, &_next_delete_flag);
+            if (LIKELY(res != OLAP_ERR_DATA_EOF)) {
+                if (UNLIKELY(res != OLAP_SUCCESS)) {
+                    LOG(WARNING) << "next failed: " << res;
+                    return res;
+                }
+
+                if (!equal_row(_key_cids, *row_cursor, *_next_key)) {
+                    agg_finalize_row(_value_cids, row_cursor, mem_pool);
+                    break;
+                }
+                _merged_rows++;
+                cur_delete_flag = _next_delete_flag;
+            } else {
+                break;
             }
-            agg_finalize_row(_value_cids, row_cursor, mem_pool);
-            cur_delete_flag = _next_delete_flag;
         }
 
         // if reader needs to filter delete row and current delete_flag is true,
diff --git a/be/src/vec/olap/vcollect_iterator.cpp b/be/src/vec/olap/vcollect_iterator.cpp
index 411c96f4e5..0d351a55fb 100644
--- a/be/src/vec/olap/vcollect_iterator.cpp
+++ b/be/src/vec/olap/vcollect_iterator.cpp
@@ -142,14 +142,8 @@ bool VCollectIterator::LevelIteratorComparator::operator()(LevelIterator* lhs, L
     // for UNIQUE_KEYS just read the highest version and no need agg_update.
     // for AGG_KEYS if a version is deleted, the lower version no need to agg_update
     bool lower = (cmp_res != 0) ? (cmp_res < 0) : (lhs->version() < rhs->version());
+    lower ? lhs->set_same(true) : rhs->set_same(true);
 
-    // if lhs or rhs set same is true, means some same value already output, so need to
-    // set another is same
-    if (lower) {
-        lhs->is_same() ? rhs->set_same(true) : lhs->set_same(true);
-    } else {
-        rhs->is_same() ? lhs->set_same(true) : rhs->set_same(true);
-    }
     return lower;
 }
 
diff --git a/be/src/vec/olap/vgeneric_iterators.cpp b/be/src/vec/olap/vgeneric_iterators.cpp
index 19e7be78b0..2ffb4e224f 100644
--- a/be/src/vec/olap/vgeneric_iterators.cpp
+++ b/be/src/vec/olap/vgeneric_iterators.cpp
@@ -120,7 +120,7 @@ Status VAutoIncrementIterator::init(const StorageReadOptions& opts) {
 //      }
 class VMergeIteratorContext {
 public:
-    VMergeIteratorContext(RowwiseIterator* iter, int sequence_id_idx) : _iter(iter), _sequence_id_idx(sequence_id_idx) {}
+    VMergeIteratorContext(RowwiseIterator* iter, int sequence_id_idx, bool is_unique) : _iter(iter), _sequence_id_idx(sequence_id_idx), _is_unique(is_unique) {}
     VMergeIteratorContext(const VMergeIteratorContext&) = delete;
     VMergeIteratorContext(VMergeIteratorContext&&) = delete;
     VMergeIteratorContext& operator=(const VMergeIteratorContext&) = delete;
@@ -165,14 +165,18 @@ public:
         if (cmp_res != 0) {
             return cmp_res > 0;
         }
-        
+
+        auto col_cmp_res = 0;
         if (_sequence_id_idx != -1) {
-            int col_cmp_res = this->_block.compare_column_at(_index_in_block, rhs._index_in_block, _sequence_id_idx, rhs._block, -1);
-            if (col_cmp_res != 0) {
-                return col_cmp_res < 0;
-            }
+            col_cmp_res = this->_block.compare_column_at(_index_in_block, rhs._index_in_block,
+                                                         _sequence_id_idx, rhs._block, -1);
         }
-        return this->data_id() < rhs.data_id();
+        auto result = col_cmp_res == 0 ? this->data_id() < rhs.data_id() : col_cmp_res < 0;
+
+        if (_is_unique) {
+            result ? this->set_skip(true) : rhs.set_skip(true);
+        }
+        return result;
     }
 
     void copy_row(vectorized::Block* block) {
@@ -203,6 +207,10 @@ public:
 
     uint64_t data_id() const { return _iter->data_id(); }
 
+    bool need_skip() const { return _skip; }
+
+    void set_skip(bool skip) const { _skip = skip; }
+
 private:
     // Load next block into _block
     Status _load_next_block();
@@ -212,10 +220,12 @@ private:
     // used to store data load from iterator->next_batch(Vectorized::Block*)
     vectorized::Block _block;
 
+    int _sequence_id_idx = -1;
+    bool _is_unique = false;
     bool _valid = false;
+    mutable bool _skip = false;
     size_t _index_in_block = -1;
     int _block_row_max = 4096;
-    int _sequence_id_idx = -1;
 };
 
 Status VMergeIteratorContext::init(const StorageReadOptions& opts) {
@@ -230,6 +240,7 @@ Status VMergeIteratorContext::init(const StorageReadOptions& opts) {
 }
 
 Status VMergeIteratorContext::advance() {
+    _skip = false;
     // NOTE: we increase _index_in_block directly to valid one check
     do {
         _index_in_block++;
@@ -263,8 +274,8 @@ Status VMergeIteratorContext::_load_next_block() {
 class VMergeIterator : public RowwiseIterator {
 public:
     // VMergeIterator takes the ownership of input iterators
-    VMergeIterator(std::vector<RowwiseIterator*>& iters, std::shared_ptr<MemTracker> parent, int sequence_id_idx) : 
-        _origin_iters(iters),_sequence_id_idx(sequence_id_idx) {
+    VMergeIterator(std::vector<RowwiseIterator*>& iters, std::shared_ptr<MemTracker> parent, int sequence_id_idx, bool is_unique) : 
+        _origin_iters(iters),_sequence_id_idx(sequence_id_idx), _is_unique(is_unique) {
         // use for count the mem use of Block use in Merge
         _mem_tracker = MemTracker::CreateTracker(-1, "VMergeIterator", parent, false);
     }
@@ -303,6 +314,7 @@ private:
 
     int block_row_max = 0;
     int _sequence_id_idx = -1;
+    bool _is_unique = false;
 };
 
 Status VMergeIterator::init(const StorageReadOptions& opts) {
@@ -312,7 +324,7 @@ Status VMergeIterator::init(const StorageReadOptions& opts) {
     _schema = &(*_origin_iters.begin())->schema();
 
     for (auto iter : _origin_iters) {
-        auto ctx = std::make_unique<VMergeIteratorContext>(iter, _sequence_id_idx);
+        auto ctx = std::make_unique<VMergeIteratorContext>(iter, _sequence_id_idx, _is_unique);
         RETURN_IF_ERROR(ctx->init(opts));
         if (!ctx->valid()) {
             continue;
@@ -335,8 +347,10 @@ Status VMergeIterator::next_batch(vectorized::Block* block) {
         auto ctx = _merge_heap.top();
         _merge_heap.pop();
 
-        // copy current row to block
-        ctx->copy_row(block);
+        if (!ctx->need_skip()) {
+            // copy current row to block
+            ctx->copy_row(block);
+        }
 
         RETURN_IF_ERROR(ctx->advance());
         if (ctx->valid()) {
@@ -412,11 +426,11 @@ Status VUnionIterator::next_batch(vectorized::Block* block) {
 }
 
 
-RowwiseIterator* new_merge_iterator(std::vector<RowwiseIterator*>& inputs, std::shared_ptr<MemTracker> parent, int sequence_id_idx) {
+RowwiseIterator* new_merge_iterator(std::vector<RowwiseIterator*>& inputs, std::shared_ptr<MemTracker> parent, int sequence_id_idx, bool is_unique) {
     if (inputs.size() == 1) {
         return *(inputs.begin());
     }
-    return new VMergeIterator(inputs, parent, sequence_id_idx);
+    return new VMergeIterator(inputs, parent, sequence_id_idx, is_unique);
 }
 
 RowwiseIterator* new_union_iterator(std::vector<RowwiseIterator*>& inputs, std::shared_ptr<MemTracker> parent) {
diff --git a/be/src/vec/olap/vgeneric_iterators.h b/be/src/vec/olap/vgeneric_iterators.h
index af9733bf44..063d07da51 100644
--- a/be/src/vec/olap/vgeneric_iterators.h
+++ b/be/src/vec/olap/vgeneric_iterators.h
@@ -27,7 +27,7 @@ namespace vectorized {
 //
 // Inputs iterators' ownership is taken by created merge iterator. And client
 // should delete returned iterator after usage.
-RowwiseIterator* new_merge_iterator(std::vector<RowwiseIterator*>& inputs, std::shared_ptr<MemTracker> parent, int sequence_id_idx);
+RowwiseIterator* new_merge_iterator(std::vector<RowwiseIterator*>& inputs, std::shared_ptr<MemTracker> parent, int sequence_id_idx, bool is_unique);
 
 // Create a union iterator for input iterators. Union iterator will read
 // input iterators one by one.
diff --git a/be/test/olap/generic_iterators_test.cpp b/be/test/olap/generic_iterators_test.cpp
index b73ad0271e..de25b071bb 100644
--- a/be/test/olap/generic_iterators_test.cpp
+++ b/be/test/olap/generic_iterators_test.cpp
@@ -115,7 +115,7 @@ TEST(GenericIteratorsTest, Union) {
     delete iter;
 }
 
-TEST(GenericIteratorsTest, Merge) {
+TEST(GenericIteratorsTest, MergeAgg) {
     auto schema = create_schema();
     std::vector<RowwiseIterator*> inputs;
 
@@ -124,7 +124,7 @@ TEST(GenericIteratorsTest, Merge) {
     inputs.push_back(new_auto_increment_iterator(schema, 300));
 
     auto iter = new_merge_iterator(
-            std::move(inputs), MemTracker::CreateTracker(-1, "MergeIterator", nullptr, false), -1);
+            std::move(inputs), MemTracker::CreateTracker(-1, "MergeIterator", nullptr, false), -1, false);
     StorageReadOptions opts;
     auto st = iter->init(opts);
     ASSERT_TRUE(st.ok());
@@ -158,6 +158,40 @@ TEST(GenericIteratorsTest, Merge) {
     delete iter;
 }
 
+TEST(GenericIteratorsTest, MergeUnique) {
+    auto schema = create_schema();
+    std::vector<RowwiseIterator*> inputs;
+
+    inputs.push_back(new_auto_increment_iterator(schema, 100));
+    inputs.push_back(new_auto_increment_iterator(schema, 200));
+    inputs.push_back(new_auto_increment_iterator(schema, 300));
+
+    auto iter = new_merge_iterator(std::move(inputs), -1, true);
+    StorageReadOptions opts;
+    auto st = iter->init(opts);
+    EXPECT_TRUE(st.ok());
+
+    RowBlockV2 block(schema, 128);
+
+    size_t row_count = 0;
+    do {
+        block.clear();
+        st = iter->next_batch(&block);
+        for (int i = 0; i < block.num_rows(); ++i) {
+            size_t base_value = row_count;
+            auto row = block.row(i);
+            EXPECT_EQ(base_value, *(int16_t*)row.cell_ptr(0));
+            EXPECT_EQ(base_value + 1, *(int32_t*)row.cell_ptr(1));
+            EXPECT_EQ(base_value + 2, *(int64_t*)row.cell_ptr(2));
+            row_count++;
+        }
+    } while (st.ok());
+    EXPECT_TRUE(st.is_end_of_file());
+    EXPECT_EQ(300, row_count);
+
+    delete iter;
+}
+
 } // namespace doris
 
 int main(int argc, char** argv) {
diff --git a/be/test/vec/exec/vgeneric_iterators_test.cpp b/be/test/vec/exec/vgeneric_iterators_test.cpp
index 405c9a9103..6189fdd821 100644
--- a/be/test/vec/exec/vgeneric_iterators_test.cpp
+++ b/be/test/vec/exec/vgeneric_iterators_test.cpp
@@ -140,8 +140,8 @@ TEST(VGenericIteratorsTest, Union) {
     delete iter;
 }
 
-TEST(VGenericIteratorsTest, Merge) {
-    ASSERT_TRUE(1);
+TEST(VGenericIteratorsTest, MergeAgg) {
+    EXPECT_TRUE(1);
     auto schema = create_schema();
     std::vector<RowwiseIterator*> inputs;
 
@@ -149,7 +149,7 @@ TEST(VGenericIteratorsTest, Merge) {
     inputs.push_back(vectorized::new_auto_increment_iterator(schema, 200));
     inputs.push_back(vectorized::new_auto_increment_iterator(schema, 300));
 
-    auto iter = vectorized::new_merge_iterator(inputs, MemTracker::CreateTracker(-1, "VMergeIterator", nullptr, false), -1);
+    auto iter = vectorized::new_merge_iterator(inputs, MemTracker::CreateTracker(-1, "VMergeIterator", nullptr, false), -1, false);
     StorageReadOptions opts;
     auto st = iter->init(opts);
     ASSERT_TRUE(st.ok());
@@ -189,6 +189,47 @@ TEST(VGenericIteratorsTest, Merge) {
     delete iter;
 }
 
+TEST(VGenericIteratorsTest, MergeUnique) {
+    EXPECT_TRUE(1);
+    auto schema = create_schema();
+    std::vector<RowwiseIterator*> inputs;
+
+    inputs.push_back(vectorized::new_auto_increment_iterator(schema, 100));
+    inputs.push_back(vectorized::new_auto_increment_iterator(schema, 200));
+    inputs.push_back(vectorized::new_auto_increment_iterator(schema, 300));
+
+    auto iter = vectorized::new_merge_iterator(inputs, -1, true);
+    StorageReadOptions opts;
+    auto st = iter->init(opts);
+    EXPECT_TRUE(st.ok());
+
+    vectorized::Block block;
+    create_block(schema, block);
+
+    do {
+        st = iter->next_batch(&block);
+    } while (st.ok());
+
+    EXPECT_TRUE(st.is_end_of_file());
+    EXPECT_EQ(block.rows(), 300);
+
+    auto c0 = block.get_by_position(0).column;
+    auto c1 = block.get_by_position(1).column;
+    auto c2 = block.get_by_position(2).column;
+
+    size_t row_count = 0;
+    for (size_t i = 0; i < block.rows(); ++i) {
+        size_t base_value = row_count;
+
+        EXPECT_EQ(base_value, (*c0)[i].get<int>());
+        EXPECT_EQ(base_value + 1, (*c1)[i].get<int>());
+        EXPECT_EQ(base_value + 2, (*c2)[i].get<int>());
+        row_count++;
+    }
+
+    delete iter;
+}
+
 // only used for Seq Column UT
 class SeqColumnUtIterator : public RowwiseIterator {
 public:
@@ -276,7 +317,7 @@ TEST(VGenericIteratorsTest, MergeWithSeqColumn) {
         inputs.push_back(new SeqColumnUtIterator(schema, num_rows, rows_begin, seq_column_id, seq_id_in_every_file));
     }
 
-    auto iter = vectorized::new_merge_iterator(inputs, MemTracker::CreateTracker(-1, "VMergeIterator", nullptr, false), seq_column_id);
+    auto iter = vectorized::new_merge_iterator(inputs, MemTracker::CreateTracker(-1, "VMergeIterator", nullptr, false), seq_column_id, true);
     StorageReadOptions opts;
     auto st = iter->init(opts);
     ASSERT_TRUE(st.ok());
@@ -288,18 +329,14 @@ TEST(VGenericIteratorsTest, MergeWithSeqColumn) {
         st = iter->next_batch(&block);
     } while (st.ok());
 
-    ASSERT_TRUE(st.is_end_of_file());
-    ASSERT_EQ(block.rows(), seg_iter_num);
+    EXPECT_TRUE(st.is_end_of_file());
+    EXPECT_EQ(block.rows(), 1);
 
     auto col0 = block.get_by_position(0).column;
     auto col1 = block.get_by_position(1).column;
     auto seq_col = block.get_by_position(seq_column_id).column;
-
-    for (size_t i = 0; i < seg_iter_num; i++) {
-        size_t expected_value = seg_iter_num - i - 1; // in Descending 
-        size_t actual_value = (*seq_col)[i].get<int>();
-        ASSERT_EQ(expected_value, actual_value);
-    }
+    size_t actual_value = (*seq_col)[0].get<int>();
+    EXPECT_EQ(seg_iter_num - 1, actual_value);
 
     delete iter;
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 10/22: [Improvement] Simplify expressions for _vconjunct_ctx_ptr (#9816)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 741f97829bf5487c602c610eede62644f56caf72
Author: Gabriel <ga...@gmail.com>
AuthorDate: Sun May 29 23:05:21 2022 +0800

    [Improvement] Simplify expressions for _vconjunct_ctx_ptr (#9816)
---
 be/src/exec/olap_scan_node.cpp |  4 ++--
 be/src/exec/scan_node.cpp      | 10 +++++-----
 2 files changed, 7 insertions(+), 7 deletions(-)

diff --git a/be/src/exec/olap_scan_node.cpp b/be/src/exec/olap_scan_node.cpp
index 7ea7ca3510..de8fa57726 100644
--- a/be/src/exec/olap_scan_node.cpp
+++ b/be/src/exec/olap_scan_node.cpp
@@ -536,8 +536,8 @@ void OlapScanNode::remove_pushed_conjuncts(RuntimeState* state) {
 
     // set vconjunct_ctx is empty, if all conjunct
     if (_direct_conjunct_size == 0) {
-        if (_vconjunct_ctx_ptr.get() != nullptr) {
-            (*_vconjunct_ctx_ptr.get())->close(state);
+        if (_vconjunct_ctx_ptr != nullptr) {
+            (*_vconjunct_ctx_ptr)->close(state);
             _vconjunct_ctx_ptr = nullptr;
         }
     }
diff --git a/be/src/exec/scan_node.cpp b/be/src/exec/scan_node.cpp
index 8fb838f0e3..d3a5367593 100644
--- a/be/src/exec/scan_node.cpp
+++ b/be/src/exec/scan_node.cpp
@@ -48,20 +48,20 @@ Status ScanNode::prepare(RuntimeState* state) {
 // Expr tree specific forms do not require requirements.
 std::string ScanNode::_peel_pushed_vconjunct(RuntimeState* state,
                                              const std::function<bool(int)>& checker) {
-    if (_vconjunct_ctx_ptr.get() == nullptr) {
+    if (_vconjunct_ctx_ptr == nullptr) {
         return "null";
     }
 
     int leaf_index = 0;
-    vectorized::VExpr* conjunct_expr_root = (*_vconjunct_ctx_ptr.get())->root();
+    vectorized::VExpr* conjunct_expr_root = (*_vconjunct_ctx_ptr)->root();
 
     if (conjunct_expr_root != nullptr) {
         vectorized::VExpr* new_conjunct_expr_root = vectorized::VectorizedUtils::dfs_peel_conjunct(
-                state, *_vconjunct_ctx_ptr.get(), conjunct_expr_root, leaf_index, checker);
+                state, *_vconjunct_ctx_ptr, conjunct_expr_root, leaf_index, checker);
         if (new_conjunct_expr_root == nullptr) {
-            _vconjunct_ctx_ptr = nullptr;
+            _vconjunct_ctx_ptr.reset(nullptr);
         } else {
-            (*_vconjunct_ctx_ptr.get())->set_root(new_conjunct_expr_root);
+            (*_vconjunct_ctx_ptr)->set_root(new_conjunct_expr_root);
             return new_conjunct_expr_root->debug_string();
         }
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 08/22: [Improvement] optimize scannode concurrency query performance in vectorized engine. (#9792)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit d40fcdc58d2be1031534c669aa61f8a96379ca58
Author: yiguolei <67...@qq.com>
AuthorDate: Mon May 30 16:04:40 2022 +0800

    [Improvement] optimize  scannode concurrency query performance in vectorized engine. (#9792)
---
 be/src/common/config.h               |  4 ++++
 be/src/exec/olap_scan_node.cpp       | 14 ++++++++----
 be/src/exec/olap_scan_node.h         |  9 ++++++--
 be/src/exec/olap_scanner.cpp         |  1 -
 be/src/olap/bloom_filter_predicate.h | 17 +++++++++++++-
 be/src/vec/exec/volap_scan_node.cpp  | 44 ++++++++++++++++++++++++++++--------
 6 files changed, 70 insertions(+), 19 deletions(-)

diff --git a/be/src/common/config.h b/be/src/common/config.h
index 48582beabe..90faf93006 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -718,6 +718,10 @@ CONF_Int32(object_pool_buffer_size, "100");
 // ParquetReaderWrap prefetch buffer size
 CONF_Int32(parquet_reader_max_buffer_size, "50");
 
+// When the rows number reached this limit, will check the filter rate the of bloomfilter
+// if it is lower than a specific threshold, the predicate will be disabled.
+CONF_mInt32(bloom_filter_predicate_check_row_num, "1000");
+
 } // namespace config
 
 } // namespace doris
diff --git a/be/src/exec/olap_scan_node.cpp b/be/src/exec/olap_scan_node.cpp
index 0d40d40b3a..7ea7ca3510 100644
--- a/be/src/exec/olap_scan_node.cpp
+++ b/be/src/exec/olap_scan_node.cpp
@@ -175,6 +175,8 @@ Status OlapScanNode::prepare(RuntimeState* state) {
     // create scanner profile
     // create timer
     _tablet_counter = ADD_COUNTER(runtime_profile(), "TabletCount ", TUnit::UNIT);
+    _scanner_sched_counter = ADD_COUNTER(runtime_profile(), "ScannerSchedCount ", TUnit::UNIT);
+
     _rows_pushed_cond_filtered_counter =
             ADD_COUNTER(_scanner_profile, "RowsPushedCondFiltered", TUnit::UNIT);
     _init_counter(state);
@@ -679,11 +681,11 @@ Status OlapScanNode::build_scan_key() {
     return Status::OK();
 }
 
-static Status get_hints(TabletSharedPtr table, const TPaloScanRange& scan_range,
-                        int block_row_count, bool is_begin_include, bool is_end_include,
-                        const std::vector<std::unique_ptr<OlapScanRange>>& scan_key_range,
-                        std::vector<std::unique_ptr<OlapScanRange>>* sub_scan_range,
-                        RuntimeProfile* profile) {
+Status OlapScanNode::get_hints(TabletSharedPtr table, const TPaloScanRange& scan_range,
+                               int block_row_count, bool is_begin_include, bool is_end_include,
+                               const std::vector<std::unique_ptr<OlapScanRange>>& scan_key_range,
+                               std::vector<std::unique_ptr<OlapScanRange>>* sub_scan_range,
+                               RuntimeProfile* profile) {
     RuntimeProfile::Counter* show_hints_timer = profile->get_counter("ShowHintsTime_V1");
     std::vector<std::vector<OlapTuple>> ranges;
     bool have_valid_range = false;
@@ -1439,6 +1441,7 @@ void OlapScanNode::transfer_thread(RuntimeState* state) {
                         std::bind(&OlapScanNode::scanner_thread, this, *iter));
                 if (s.ok()) {
                     (*iter)->start_wait_worker_timer();
+                    COUNTER_UPDATE(_scanner_sched_counter, 1);
                     olap_scanners.erase(iter++);
                 } else {
                     LOG(FATAL) << "Failed to assign scanner task to thread pool! "
@@ -1453,6 +1456,7 @@ void OlapScanNode::transfer_thread(RuntimeState* state) {
                 task.priority = _nice;
                 task.queue_id = state->exec_env()->store_path_to_index((*iter)->scan_disk());
                 (*iter)->start_wait_worker_timer();
+                COUNTER_UPDATE(_scanner_sched_counter, 1);
                 if (thread_pool->offer(task)) {
                     olap_scanners.erase(iter++);
                 } else {
diff --git a/be/src/exec/olap_scan_node.h b/be/src/exec/olap_scan_node.h
index 1220a742e0..4cd2902b93 100644
--- a/be/src/exec/olap_scan_node.h
+++ b/be/src/exec/olap_scan_node.h
@@ -57,7 +57,12 @@ public:
     Status collect_query_statistics(QueryStatistics* statistics) override;
     Status close(RuntimeState* state) override;
     Status set_scan_ranges(const std::vector<TScanRangeParams>& scan_ranges) override;
-    inline void set_no_agg_finalize() { _need_agg_finalize = false; }
+    void set_no_agg_finalize() { _need_agg_finalize = false; }
+    Status get_hints(TabletSharedPtr table, const TPaloScanRange& scan_range, int block_row_count,
+                     bool is_begin_include, bool is_end_include,
+                     const std::vector<std::unique_ptr<OlapScanRange>>& scan_key_range,
+                     std::vector<std::unique_ptr<OlapScanRange>>* sub_scan_range,
+                     RuntimeProfile* profile);
 
 protected:
     struct HeapType {
@@ -246,7 +251,7 @@ protected:
     RuntimeProfile::Counter* _tablet_counter;
     RuntimeProfile::Counter* _rows_pushed_cond_filtered_counter = nullptr;
     RuntimeProfile::Counter* _reader_init_timer = nullptr;
-
+    RuntimeProfile::Counter* _scanner_sched_counter = nullptr;
     TResourceInfo* _resource_info;
 
     std::atomic<int64_t> _buffered_bytes;
diff --git a/be/src/exec/olap_scanner.cpp b/be/src/exec/olap_scanner.cpp
index 9baef00001..4e2003ae0b 100644
--- a/be/src/exec/olap_scanner.cpp
+++ b/be/src/exec/olap_scanner.cpp
@@ -308,7 +308,6 @@ Status OlapScanner::get_batch(RuntimeState* state, RowBatch* batch, bool* eof) {
             if (UNLIKELY(*eof)) {
                 break;
             }
-
             _num_rows_read++;
 
             _convert_row_to_tuple(tuple);
diff --git a/be/src/olap/bloom_filter_predicate.h b/be/src/olap/bloom_filter_predicate.h
index c8fbdab94c..a7671f2724 100644
--- a/be/src/olap/bloom_filter_predicate.h
+++ b/be/src/olap/bloom_filter_predicate.h
@@ -71,6 +71,9 @@ public:
 private:
     std::shared_ptr<IBloomFilterFuncBase> _filter;
     SpecificFilter* _specific_filter; // owned by _filter
+    mutable uint64_t _evaluated_rows = 1;
+    mutable uint64_t _passed_rows = 0;
+    mutable bool _enable_pred = true;
 };
 
 // bloom filter column predicate do not support in segment v1
@@ -113,7 +116,9 @@ void BloomFilterColumnPredicate<T>::evaluate(vectorized::IColumn& column, uint16
                                                 uint16_t* size) const {
     uint16_t new_size = 0;
     using FT = typename PredicatePrimitiveTypeTraits<T>::PredicateFieldType;
-
+    if (!_enable_pred) {
+        return;
+    }
     if (column.is_nullable()) {
         auto* nullable_col = vectorized::check_and_get_column<vectorized::ColumnNullable>(column);
         auto& null_map_data = nullable_col->get_null_map_column().get_data();
@@ -158,6 +163,16 @@ void BloomFilterColumnPredicate<T>::evaluate(vectorized::IColumn& column, uint16
             new_size += _specific_filter->find_olap_engine(cell_value);
         }
     }
+    // If the pass rate is very high, for example > 50%, then the bloomfilter is useless.
+    // Some bloomfilter is useless, for example ssb 4.3, it consumes a lot of cpu but it is
+    // useless.
+    _evaluated_rows += *size;
+    _passed_rows += new_size;
+    if (_evaluated_rows > config::bloom_filter_predicate_check_row_num) {
+        if (_passed_rows / (_evaluated_rows * 1.0) > 0.5) {
+            _enable_pred = false;
+        }
+    }
     *size = new_size;
 }
 
diff --git a/be/src/vec/exec/volap_scan_node.cpp b/be/src/vec/exec/volap_scan_node.cpp
index 3156324d66..c54312d16d 100644
--- a/be/src/vec/exec/volap_scan_node.cpp
+++ b/be/src/vec/exec/volap_scan_node.cpp
@@ -196,9 +196,13 @@ void VOlapScanNode::scanner_thread(VOlapScanner* scanner) {
     int64_t raw_bytes_read = 0;
     int64_t raw_bytes_threshold = config::doris_scanner_row_bytes;
     bool get_free_block = true;
+    int num_rows_in_block = 0;
 
-    while (!eos && raw_rows_read < raw_rows_threshold && raw_bytes_read < raw_bytes_threshold &&
-           get_free_block) {
+    // Has to wait at least one full block, or it will cause a lot of schedule task in priority
+    // queue, it will affect query latency and query concurrency for example ssb 3.3.
+    while (!eos && ((raw_rows_read < raw_rows_threshold && raw_bytes_read < raw_bytes_threshold &&
+                     get_free_block) ||
+                    num_rows_in_block < _runtime_state->batch_size())) {
         if (UNLIKELY(_transfer_done)) {
             eos = true;
             status = Status::Cancelled("Cancelled");
@@ -218,7 +222,7 @@ void VOlapScanNode::scanner_thread(VOlapScanner* scanner) {
         }
 
         raw_bytes_read += block->allocated_bytes();
-
+        num_rows_in_block += block->rows();
         // 4. if status not ok, change status_.
         if (UNLIKELY(block->rows() == 0)) {
             std::lock_guard<std::mutex> l(_free_blocks_lock);
@@ -324,6 +328,12 @@ Status VOlapScanNode::start_scan_thread(RuntimeState* state) {
     if (cond_ranges.empty()) {
         cond_ranges.emplace_back(new OlapScanRange());
     }
+    bool need_split = true;
+    // If we have ranges more than 64, there is no need to call
+    // ShowHint to split ranges
+    if (limit() != -1 || cond_ranges.size() > 64) {
+        need_split = false;
+    }
     int scanners_per_tablet = std::max(1, 64 / (int)_scan_ranges.size());
 
     std::unordered_set<std::string> disk_set;
@@ -341,6 +351,16 @@ Status VOlapScanNode::start_scan_thread(RuntimeState* state) {
             return Status::InternalError(ss.str());
         }
 
+        std::vector<std::unique_ptr<OlapScanRange>>* ranges = &cond_ranges;
+        std::vector<std::unique_ptr<OlapScanRange>> split_ranges;
+        if (need_split && !tablet->all_beta()) {
+            auto st = get_hints(tablet, *scan_range, config::doris_scan_range_row_count,
+                                _scan_keys.begin_include(), _scan_keys.end_include(), cond_ranges,
+                                &split_ranges, _runtime_profile.get());
+            if (st.ok()) {
+                ranges = &split_ranges;
+            }
+        }
         int size_based_scanners_per_tablet = 1;
 
         if (config::doris_scan_range_max_mb > 0) {
@@ -349,17 +369,17 @@ Status VOlapScanNode::start_scan_thread(RuntimeState* state) {
         }
 
         int ranges_per_scanner =
-                std::max(1, (int)cond_ranges.size() /
+                std::max(1, (int)ranges->size() /
                                     std::min(scanners_per_tablet, size_based_scanners_per_tablet));
-        int num_ranges = cond_ranges.size();
+        int num_ranges = ranges->size();
         for (int i = 0; i < num_ranges;) {
             std::vector<OlapScanRange*> scanner_ranges;
-            scanner_ranges.push_back(cond_ranges[i].get());
+            scanner_ranges.push_back((*ranges)[i].get());
             ++i;
             for (int j = 1; i < num_ranges && j < ranges_per_scanner &&
-                            cond_ranges[i]->end_include == cond_ranges[i - 1]->end_include;
+                            (*ranges)[i]->end_include == (*ranges)[i - 1]->end_include;
                  ++j, ++i) {
-                scanner_ranges.push_back(cond_ranges[i].get());
+                scanner_ranges.push_back((*ranges)[i].get());
             }
             VOlapScanner* scanner = new VOlapScanner(state, this, _olap_scan_node.is_preaggregation,
                                                      _need_agg_finalize, *scan_range);
@@ -551,8 +571,11 @@ Block* VOlapScanNode::_alloc_block(bool& get_free_block) {
 int VOlapScanNode::_start_scanner_thread_task(RuntimeState* state, int block_per_scanner) {
     std::list<VOlapScanner*> olap_scanners;
     int assigned_thread_num = _running_thread;
-    size_t max_thread = std::min(_volap_scanners.size(),
-                                 static_cast<size_t>(config::doris_scanner_thread_pool_thread_num));
+    size_t max_thread = config::doris_scanner_queue_size;
+    if (config::doris_scanner_row_num > state->batch_size()) {
+        max_thread /= config::doris_scanner_row_num / state->batch_size();
+        if (max_thread <= 0) max_thread = 1;
+    }
     // copy to local
     {
         // How many thread can apply to this query
@@ -606,6 +629,7 @@ int VOlapScanNode::_start_scanner_thread_task(RuntimeState* state, int block_per
         task.priority = _nice;
         task.queue_id = state->exec_env()->store_path_to_index((*iter)->scan_disk());
         (*iter)->start_wait_worker_timer();
+        COUNTER_UPDATE(_scanner_sched_counter, 1);
         if (thread_pool->offer(task)) {
             olap_scanners.erase(iter++);
         } else {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 02/22: [fix](vectorized) fix vcast expr input wrong row number (#9520)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit b01bd5b72eb90c8795f0db0c3f2d9a31e20ae244
Author: Pxl <px...@qq.com>
AuthorDate: Wed Jun 1 15:19:31 2022 +0800

    [fix](vectorized) fix vcast expr input wrong row number (#9520)
---
 be/src/vec/exprs/vcast_expr.cpp      | 16 +++++------
 be/src/vec/exprs/vexpr.h             | 11 ++++++++
 be/src/vec/exprs/vinfo_func.cpp      |  9 ++----
 be/src/vec/exprs/vliteral.cpp        | 10 ++-----
 be/src/vec/functions/function_cast.h | 54 ++++++++++--------------------------
 5 files changed, 39 insertions(+), 61 deletions(-)

diff --git a/be/src/vec/exprs/vcast_expr.cpp b/be/src/vec/exprs/vcast_expr.cpp
index a1a441b7b8..f910dfd466 100644
--- a/be/src/vec/exprs/vcast_expr.cpp
+++ b/be/src/vec/exprs/vcast_expr.cpp
@@ -19,8 +19,10 @@
 
 #include <string_view>
 
+#include "common/status.h"
 #include "vec/core/field.h"
 #include "vec/data_types/data_type_factory.hpp"
+#include "vec/exprs/vexpr.h"
 #include "vec/functions/simple_function_factory.h"
 
 namespace doris::vectorized {
@@ -72,21 +74,19 @@ void VCastExpr::close(doris::RuntimeState* state, VExprContext* context,
 doris::Status VCastExpr::execute(VExprContext* context, doris::vectorized::Block* block,
                                  int* result_column_id) {
     // for each child call execute
-    doris::vectorized::ColumnNumbers arguments(2);
-    int column_id = -1;
+    int column_id = 0;
     _children[0]->execute(context, block, &column_id);
-    arguments[0] = column_id;
 
-    size_t const_param_id = block->columns();
-    block->insert({_cast_param, _cast_param_data_type, _target_data_type_name});
-    arguments[1] = const_param_id;
+    size_t const_param_id = VExpr::insert_param(
+            block, {_cast_param, _cast_param_data_type, _target_data_type_name}, block->rows());
 
     // call function
     size_t num_columns_without_result = block->columns();
     // prepare a column to save result
     block->insert({nullptr, _data_type, _expr_name});
-    _function->execute(context->fn_context(_fn_context_index), *block, arguments,
-                       num_columns_without_result, block->rows(), false);
+    RETURN_IF_ERROR(_function->execute(context->fn_context(_fn_context_index), *block,
+                                       {static_cast<size_t>(column_id), const_param_id},
+                                       num_columns_without_result, block->rows(), false));
     *result_column_id = num_columns_without_result;
     return Status::OK();
 }
diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h
index 0bbce41af8..8958b91e00 100644
--- a/be/src/vec/exprs/vexpr.h
+++ b/be/src/vec/exprs/vexpr.h
@@ -33,8 +33,19 @@ namespace vectorized {
 
 class VExpr {
 public:
+    // resize inserted param column to make sure column size equal to block.rows()
+    // and return param column index
+    static size_t insert_param(Block* block, ColumnWithTypeAndName&& elem, size_t size) {
+        // usualy elem.column always is const column, so we just clone it.
+        elem.column = elem.column->clone_resized(size);
+        block->insert(std::move(elem));
+        return block->columns() - 1;
+    }
+
     VExpr(const TExprNode& node);
     VExpr(const TypeDescriptor& type, bool is_slotref, bool is_nullable);
+    // only used for test
+    VExpr() = default;
     virtual ~VExpr() = default;
 
     virtual VExpr* clone(ObjectPool* pool) const = 0;
diff --git a/be/src/vec/exprs/vinfo_func.cpp b/be/src/vec/exprs/vinfo_func.cpp
index d703c3790f..d347d97240 100644
--- a/be/src/vec/exprs/vinfo_func.cpp
+++ b/be/src/vec/exprs/vinfo_func.cpp
@@ -47,12 +47,9 @@ VInfoFunc::VInfoFunc(const TExprNode& node) : VExpr(node) {
 }
 
 Status VInfoFunc::execute(VExprContext* context, vectorized::Block* block, int* result_column_id) {
-    int rows = block->rows();
-    if (rows < 1) {
-        rows = 1;
-    }
-    *result_column_id = block->columns();
-    block->insert({_column_ptr->clone_resized(rows), _data_type, _expr_name});
+    // Info function should return least one row, e.g. select current_user().
+    size_t row_size = std::max(block->rows(), size_t(1));
+    *result_column_id = VExpr::insert_param(block, {_column_ptr, _data_type, _expr_name}, row_size);
     return Status::OK();
 }
 
diff --git a/be/src/vec/exprs/vliteral.cpp b/be/src/vec/exprs/vliteral.cpp
index 38ec4e1bf6..37088d0a9f 100644
--- a/be/src/vec/exprs/vliteral.cpp
+++ b/be/src/vec/exprs/vliteral.cpp
@@ -126,13 +126,9 @@ VLiteral::VLiteral(const TExprNode& node) : VExpr(node) {
 VLiteral::~VLiteral() {}
 
 Status VLiteral::execute(VExprContext* context, vectorized::Block* block, int* result_column_id) {
-    int rows = block->rows();
-    if (rows < 1) {
-        rows = 1;
-    }
-    size_t res = block->columns();
-    block->insert({_column_ptr->clone_resized(rows), _data_type, _expr_name});
-    *result_column_id = res;
+    // Literal expr should return least one row.
+    size_t row_size = std::max(block->rows(), size_t(1));
+    *result_column_id = VExpr::insert_param(block, {_column_ptr, _data_type, _expr_name}, row_size);
     return Status::OK();
 }
 } // namespace doris::vectorized
diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h
index 092842f5fa..893b146840 100644
--- a/be/src/vec/functions/function_cast.h
+++ b/be/src/vec/functions/function_cast.h
@@ -718,11 +718,7 @@ struct ConvertThroughParsing {
 
     using ToFieldType = typename ToDataType::FieldType;
 
-    static bool is_all_read(ReadBuffer& in) {
-        if (in.eof()) return true;
-
-        return false;
-    }
+    static bool is_all_read(ReadBuffer& in) { return in.eof(); }
 
     template <typename Additions = void*>
     static Status execute(Block& block, const ColumnNumbers& arguments, size_t result,
@@ -779,19 +775,9 @@ struct ConvertThroughParsing {
 
             ReadBuffer read_buffer(&(*chars)[current_offset], string_size);
 
-            {
-                bool parsed;
-
-                {
-                    parsed = try_parse_impl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
-
-                    parsed = parsed && is_all_read(read_buffer);
-                }
-
-                if (!parsed) vec_to[i] = 0;
-
-                (*vec_null_map_to)[i] = !parsed;
-            }
+            (*vec_null_map_to)[i] =
+                    !try_parse_impl<ToDataType>(vec_to[i], read_buffer, local_time_zone) ||
+                    !is_all_read(read_buffer);
 
             current_offset = next_offset;
         }
@@ -836,27 +822,16 @@ public:
                         size_t result, size_t input_rows_count) override {
         const IDataType* from_type = block.get_by_position(arguments[0]).type.get();
 
-        bool ok = true;
-
-        {
-            if (check_and_get_data_type<DataTypeString>(from_type)) {
-                return ConvertThroughParsing<DataTypeString, ToDataType, Name>::execute(
-                        block, arguments, result, input_rows_count);
-            }
-
-            else
-                ok = false;
-        }
-
-        if (!ok) {
-            return Status::RuntimeError(fmt::format(
-                    "Illegal type {} of argument of function {} . Only String or FixedString "
-                    "argument is accepted for try-conversion function. For other arguments, use "
-                    "function without 'orZero' or 'orNull'.",
-                    block.get_by_position(arguments[0]).type->get_name(), get_name()));
+        if (check_and_get_data_type<DataTypeString>(from_type)) {
+            return ConvertThroughParsing<DataTypeString, ToDataType, Name>::execute(
+                    block, arguments, result, input_rows_count);
         }
 
-        return Status::OK();
+        return Status::RuntimeError(fmt::format(
+                "Illegal type {} of argument of function {} . Only String or FixedString "
+                "argument is accepted for try-conversion function. For other arguments, use "
+                "function without 'orZero' or 'orNull'.",
+                block.get_by_position(arguments[0]).type->get_name(), get_name()));
     }
 };
 
@@ -877,8 +852,7 @@ public:
     // This function should not be called for get DateType Ptr
     // using the FunctionCast::get_return_type_impl
     DataTypePtr get_return_type_impl(const ColumnsWithTypeAndName& arguments) const override {
-        auto res = std::make_shared<ToDataType>();
-        return res;
+        return std::make_shared<ToDataType>();
     }
 
     Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
@@ -1084,7 +1058,7 @@ private:
             };
         }
 
-        bool skip_not_null_check = false;
+        constexpr bool skip_not_null_check = false;
 
         auto wrapper = prepare_remove_nullable(from_nested, to_nested, skip_not_null_check);
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 09/22: [API changed](parser) Remove merge join syntax (#9795)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 99e76032c0c634e22d9e52b558c823ad12439a3f
Author: EmmyMiao87 <52...@qq.com>
AuthorDate: Mon May 30 09:04:21 2022 +0800

    [API changed](parser) Remove merge join syntax (#9795)
    
    Remove merge join sql and merge join node
---
 be/src/exec/CMakeLists.txt                         |   1 -
 be/src/exec/exec_node.cpp                          |   5 -
 be/src/exec/merge_join_node.cpp                    | 335 ---------------------
 be/src/exec/merge_join_node.h                      | 102 -------
 be/src/vec/exec/join/vhash_join_node.h             |  45 ++-
 fe/fe-core/src/main/cup/sql_parser.cup             |   2 -
 .../org/apache/doris/analysis/JoinOperator.java    |   1 -
 .../java/org/apache/doris/analysis/TableRef.java   |   2 -
 gensrc/thrift/PlanNodes.thrift                     |   4 +-
 9 files changed, 24 insertions(+), 473 deletions(-)

diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt
index 709cb4de84..ad629d9f62 100644
--- a/be/src/exec/CMakeLists.txt
+++ b/be/src/exec/CMakeLists.txt
@@ -41,7 +41,6 @@ set(EXEC_FILES
     hash_table.cpp
     local_file_reader.cpp
     merge_node.cpp
-    merge_join_node.cpp
     scan_node.cpp
     select_node.cpp
     text_converter.cpp
diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp
index 5141ddff06..89c806b4c6 100644
--- a/be/src/exec/exec_node.cpp
+++ b/be/src/exec/exec_node.cpp
@@ -35,7 +35,6 @@
 #include "exec/exchange_node.h"
 #include "exec/hash_join_node.h"
 #include "exec/intersect_node.h"
-#include "exec/merge_join_node.h"
 #include "exec/merge_node.h"
 #include "exec/mysql_scan_node.h"
 #include "exec/odbc_scan_node.h"
@@ -478,10 +477,6 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN
         }
         return Status::OK();
 
-    case TPlanNodeType::MERGE_JOIN_NODE:
-        *node = pool->add(new MergeJoinNode(pool, tnode, descs));
-        return Status::OK();
-
     case TPlanNodeType::EMPTY_SET_NODE:
         if (state->enable_vectorized_exec()) {
             *node = pool->add(new vectorized::VEmptySetNode(pool, tnode, descs));
diff --git a/be/src/exec/merge_join_node.cpp b/be/src/exec/merge_join_node.cpp
deleted file mode 100644
index d83e872507..0000000000
--- a/be/src/exec/merge_join_node.cpp
+++ /dev/null
@@ -1,335 +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.
-
-#include "exec/merge_join_node.h"
-
-#include <sstream>
-
-#include "exprs/expr.h"
-#include "exprs/expr_context.h"
-#include "exprs/in_predicate.h"
-#include "gen_cpp/PlanNodes_types.h"
-#include "runtime/row_batch.h"
-#include "runtime/runtime_state.h"
-#include "util/debug_util.h"
-#include "util/runtime_profile.h"
-
-namespace doris {
-
-template <class T>
-int compare_value(const void* left_value, const void* right_value) {
-    if (*(T*)left_value < *(T*)right_value) {
-        return -1;
-    } else if (*(T*)left_value == *(T*)right_value) {
-        return 0;
-    } else {
-        return 1;
-    }
-}
-
-template <class T>
-int compare_value(const StringValue* left_value, const StringValue* right_value) {
-    return left_value->compare(*right_value);
-}
-
-MergeJoinNode::MergeJoinNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
-        : ExecNode(pool, tnode, descs), _out_batch(nullptr) {}
-
-MergeJoinNode::~MergeJoinNode() {}
-
-Status MergeJoinNode::init(const TPlanNode& tnode, RuntimeState* state) {
-    DCHECK(tnode.__isset.merge_join_node);
-    RETURN_IF_ERROR(ExecNode::init(tnode, state));
-    const std::vector<TEqJoinCondition>& cmp_conjuncts = tnode.merge_join_node.cmp_conjuncts;
-
-    for (int i = 0; i < cmp_conjuncts.size(); ++i) {
-        ExprContext* ctx = nullptr;
-        RETURN_IF_ERROR(Expr::create_expr_tree(_pool, cmp_conjuncts[i].left, &ctx));
-        _left_expr_ctxs.push_back(ctx);
-        RETURN_IF_ERROR(Expr::create_expr_tree(_pool, cmp_conjuncts[i].right, &ctx));
-        _right_expr_ctxs.push_back(ctx);
-    }
-
-    RETURN_IF_ERROR(Expr::create_expr_trees(_pool, tnode.merge_join_node.other_join_conjuncts,
-                                            &_other_join_conjunct_ctxs));
-    return Status::OK();
-}
-
-Status MergeJoinNode::prepare(RuntimeState* state) {
-    RETURN_IF_ERROR(ExecNode::prepare(state));
-
-    // build and probe exprs are evaluated in the context of the rows produced by our
-    // right and left children, respectively
-    RETURN_IF_ERROR(
-            Expr::prepare(_left_expr_ctxs, state, child(0)->row_desc(), expr_mem_tracker()));
-    RETURN_IF_ERROR(
-            Expr::prepare(_right_expr_ctxs, state, child(1)->row_desc(), expr_mem_tracker()));
-
-    for (int i = 0; i < _left_expr_ctxs.size(); ++i) {
-        switch (_left_expr_ctxs[i]->root()->type().type) {
-        case TYPE_TINYINT:
-            _cmp_func.push_back(compare_value<int8_t>);
-            break;
-
-        case TYPE_SMALLINT:
-            _cmp_func.push_back(compare_value<int16_t>);
-            break;
-
-        case TYPE_INT:
-            _cmp_func.push_back(compare_value<int32_t>);
-            break;
-
-        case TYPE_BIGINT:
-            _cmp_func.push_back(compare_value<int64_t>);
-            break;
-
-        case TYPE_LARGEINT:
-            _cmp_func.push_back(compare_value<__int128>);
-            break;
-
-        case TYPE_CHAR:
-        case TYPE_VARCHAR:
-        case TYPE_STRING:
-            _cmp_func.push_back(compare_value<StringValue>);
-            break;
-
-        default:
-            return Status::InternalError("unsupported compare type.");
-            break;
-        }
-    }
-
-    // _other_join_conjuncts are evaluated in the context of the rows produced by this node
-    RETURN_IF_ERROR(
-            Expr::prepare(_other_join_conjunct_ctxs, state, _row_descriptor, expr_mem_tracker()));
-
-    _result_tuple_row_size = _row_descriptor.tuple_descriptors().size() * sizeof(Tuple*);
-    // pre-compute the tuple index of build tuples in the output row
-
-    _left_tuple_size = child(0)->row_desc().tuple_descriptors().size();
-    _right_tuple_size = child(1)->row_desc().tuple_descriptors().size();
-    _right_tuple_idx.reserve(_right_tuple_size);
-
-    for (int i = 0; i < _right_tuple_size; ++i) {
-        TupleDescriptor* right_tuple_desc = child(1)->row_desc().tuple_descriptors()[i];
-        _right_tuple_idx.push_back(_row_descriptor.get_tuple_idx(right_tuple_desc->id()));
-    }
-
-    _left_child_ctx.reset(
-            new ChildReaderContext(row_desc(), state->batch_size(), state->instance_mem_tracker()));
-    _right_child_ctx.reset(
-            new ChildReaderContext(row_desc(), state->batch_size(), state->instance_mem_tracker()));
-
-    return Status::OK();
-}
-
-Status MergeJoinNode::close(RuntimeState* state) {
-    if (is_closed()) {
-        return Status::OK();
-    }
-    RETURN_IF_ERROR(exec_debug_action(TExecNodePhase::CLOSE));
-    Expr::close(_left_expr_ctxs, state);
-    Expr::close(_right_expr_ctxs, state);
-    Expr::close(_other_join_conjunct_ctxs, state);
-    return ExecNode::close(state);
-}
-
-Status MergeJoinNode::open(RuntimeState* state) {
-    SCOPED_TIMER(_runtime_profile->total_time_counter());
-    RETURN_IF_CANCELLED(state);
-    RETURN_IF_ERROR(ExecNode::open(state));
-    RETURN_IF_ERROR(Expr::open(_left_expr_ctxs, state));
-    RETURN_IF_ERROR(Expr::open(_right_expr_ctxs, state));
-    RETURN_IF_ERROR(Expr::open(_other_join_conjunct_ctxs, state));
-
-    _eos = false;
-    // Open the probe-side child so that it may perform any initialisation in parallel.
-    // Don't exit even if we see an error, we still need to wait for the build thread
-    // to finish.
-    RETURN_IF_ERROR(child(0)->open(state));
-    RETURN_IF_ERROR(child(1)->open(state));
-
-    RETURN_IF_ERROR(get_input_row(state, 0));
-    RETURN_IF_ERROR(get_input_row(state, 1));
-
-    return Status::OK();
-}
-
-Status MergeJoinNode::get_next(RuntimeState* state, RowBatch* out_batch, bool* eos) {
-    RETURN_IF_ERROR(exec_debug_action(TExecNodePhase::GETNEXT));
-    RETURN_IF_CANCELLED(state);
-    SCOPED_TIMER(_runtime_profile->total_time_counter());
-
-    if (reached_limit() || _eos) {
-        *eos = true;
-        return Status::OK();
-    }
-
-    while (true) {
-        int row_idx = out_batch->add_row();
-        DCHECK(row_idx != RowBatch::INVALID_ROW_INDEX);
-        TupleRow* row = out_batch->get_row(row_idx);
-
-        _out_batch = out_batch;
-        RETURN_IF_ERROR(get_next_row(state, row, eos));
-
-        if (*eos) {
-            _eos = true;
-            return Status::OK();
-        }
-
-        if (eval_conjuncts(&_other_join_conjunct_ctxs[0], _other_join_conjunct_ctxs.size(), row)) {
-            out_batch->commit_last_row();
-            ++_num_rows_returned;
-            COUNTER_SET(_rows_returned_counter, _num_rows_returned);
-        }
-
-        if (out_batch->is_full() || out_batch->at_resource_limit() || reached_limit()) {
-            break;
-        }
-    }
-
-    return Status::OK();
-}
-
-void MergeJoinNode::create_output_row(TupleRow* out, TupleRow* left, TupleRow* right) {
-    if (left == nullptr) {
-        memset(out, 0, _left_tuple_size);
-    } else {
-        memcpy(out, left, _left_tuple_size);
-    }
-
-    if (right != nullptr) {
-        for (int i = 0; i < _right_tuple_size; ++i) {
-            out->set_tuple(_right_tuple_idx[i], right->get_tuple(i));
-        }
-    } else {
-        for (int i = 0; i < _right_tuple_size; ++i) {
-            out->set_tuple(_right_tuple_idx[i], nullptr);
-        }
-    }
-}
-
-Status MergeJoinNode::compare_row(TupleRow* left_row, TupleRow* right_row, bool* is_lt) {
-    if (left_row == nullptr) {
-        *is_lt = false;
-        return Status::OK();
-    } else if (right_row == nullptr) {
-        *is_lt = true;
-        return Status::OK();
-    }
-
-    for (int i = 0; i < _left_expr_ctxs.size(); ++i) {
-        void* left_value = _left_expr_ctxs[i]->get_value(left_row);
-        void* right_value = _right_expr_ctxs[i]->get_value(right_row);
-        int cmp_val = _cmp_func[i](left_value, right_value);
-
-        if (cmp_val < 0) {
-            *is_lt = true;
-            return Status::OK();
-        } else if (cmp_val == 0) {
-            // do nothing
-        } else {
-            *is_lt = false;
-            return Status::OK();
-        }
-    }
-
-    // equal
-    *is_lt = false;
-
-    return Status::OK();
-}
-
-Status MergeJoinNode::get_next_row(RuntimeState* state, TupleRow* out_row, bool* eos) {
-    TupleRow* left_row = _left_child_ctx->current_row;
-    TupleRow* right_row = _right_child_ctx->current_row;
-
-    if (left_row == nullptr && right_row == nullptr) {
-        *eos = true;
-        return Status::OK();
-    }
-
-    bool is_lt = true;
-    RETURN_IF_ERROR(compare_row(left_row, right_row, &is_lt));
-
-    if (is_lt) {
-        create_output_row(out_row, left_row, nullptr);
-        RETURN_IF_ERROR(get_input_row(state, 0));
-    } else {
-        create_output_row(out_row, nullptr, right_row);
-        RETURN_IF_ERROR(get_input_row(state, 1));
-    }
-
-    return Status::OK();
-}
-
-Status MergeJoinNode::get_input_row(RuntimeState* state, int child_idx) {
-    ChildReaderContext* ctx = nullptr;
-
-    if (child_idx == 0) {
-        ctx = _left_child_ctx.get();
-    } else {
-        ctx = _right_child_ctx.get();
-    }
-
-    // loop util read a valid data
-    while (!ctx->is_eos && ctx->row_idx >= ctx->batch.num_rows()) {
-        // transfer ownership before get new batch
-        if (nullptr != _out_batch) {
-            ctx->batch.transfer_resource_ownership(_out_batch);
-        }
-
-        if (child_idx == 0) {
-            _left_child_ctx.reset(new ChildReaderContext(child(child_idx)->row_desc(),
-                                                         state->batch_size(),
-                                                         state->instance_mem_tracker()));
-            ctx = _left_child_ctx.get();
-        } else {
-            _right_child_ctx.reset(new ChildReaderContext(child(child_idx)->row_desc(),
-                                                          state->batch_size(),
-                                                          state->instance_mem_tracker()));
-            ctx = _right_child_ctx.get();
-        }
-
-        RETURN_IF_ERROR(child(child_idx)->get_next(state, &ctx->batch, &ctx->is_eos));
-    }
-
-    if (ctx->row_idx >= ctx->batch.num_rows()) {
-        ctx->current_row = nullptr;
-        return Status::OK();
-    }
-
-    ctx->current_row = ctx->batch.get_row(ctx->row_idx++);
-    return Status::OK();
-}
-
-void MergeJoinNode::debug_string(int indentation_level, std::stringstream* out) const {
-    *out << string(indentation_level * 2, ' ');
-    *out << "MergeJoin(eos=" << (_eos ? "true" : "false")
-         << " _left_child_pos=" << (_left_child_ctx.get() ? _left_child_ctx->row_idx : -1)
-         << " _right_child_pos=" << (_right_child_ctx.get() ? _right_child_ctx->row_idx : -1)
-         << " join_conjuncts=";
-    *out << "Conjunct(";
-    // << " left_exprs=" << Expr::debug_string(_left_exprs)
-    // << " right_exprs=" << Expr::debug_string(_right_exprs);
-    *out << ")";
-    ExecNode::debug_string(indentation_level, out);
-    *out << ")";
-}
-
-} // namespace doris
diff --git a/be/src/exec/merge_join_node.h b/be/src/exec/merge_join_node.h
deleted file mode 100644
index d8b294ea18..0000000000
--- a/be/src/exec/merge_join_node.h
+++ /dev/null
@@ -1,102 +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.
-
-#ifndef DORIS_BE_SRC_QUERY_EXEC_MERGE_JOIN_NODE_H
-#define DORIS_BE_SRC_QUERY_EXEC_MERGE_JOIN_NODE_H
-
-#include <string>
-#include <thread>
-#include <unordered_set>
-
-#include "exec/exec_node.h"
-#include "gen_cpp/PlanNodes_types.h" // for TJoinOp
-#include "runtime/row_batch.h"
-
-namespace doris {
-
-class MemPool;
-class TupleRow;
-
-// Node for in-memory merge joins:
-// find the minimal tuple and output
-class MergeJoinNode : public ExecNode {
-public:
-    MergeJoinNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
-
-    ~MergeJoinNode();
-
-    virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr);
-    virtual Status prepare(RuntimeState* state);
-    virtual Status open(RuntimeState* state);
-    virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos);
-    virtual Status close(RuntimeState* state);
-
-protected:
-    void debug_string(int indentation_level, std::stringstream* out) const;
-
-private:
-    // our equi-join predicates "<lhs> = <rhs>" are separated into
-    // _left_exprs (over child(0)) and _right_exprs (over child(1))
-    // check which expr is min
-    std::vector<ExprContext*> _left_expr_ctxs;
-    std::vector<ExprContext*> _right_expr_ctxs;
-
-    // non-equi-join conjuncts from the JOIN clause
-    std::vector<ExprContext*> _other_join_conjunct_ctxs;
-
-    bool _eos; // if true, nothing left to return in get_next()
-
-    struct ChildReaderContext {
-        RowBatch batch;
-        int row_idx;
-        bool is_eos;
-        TupleRow* current_row;
-        ChildReaderContext(const RowDescriptor& desc, int batch_size,
-                           const std::shared_ptr<MemTracker>& mem_tracker)
-                : batch(desc, batch_size, mem_tracker.get()),
-                  row_idx(0),
-                  is_eos(false),
-                  current_row(nullptr) {}
-    };
-    // _left_batch must be cleared before calling get_next().  used cache child(0)'s data
-    // _right_batch must be cleared before calling get_next().  used cache child(1)'s data
-    // does not initialize all tuple ptrs in the row, only the ones that it
-    // is responsible for.
-    std::unique_ptr<ChildReaderContext> _left_child_ctx;
-    std::unique_ptr<ChildReaderContext> _right_child_ctx;
-    // _build_tuple_idx[i] is the tuple index of child(1)'s tuple[i] in the output row
-    std::vector<int> _right_tuple_idx;
-    int _right_tuple_size;
-    int _left_tuple_size;
-    RowBatch* _out_batch;
-
-    typedef int (*CompareFn)(const void*, const void*);
-    std::vector<CompareFn> _cmp_func;
-
-    // byte size of result tuple row (sum of the tuple ptrs, not the tuple data).
-    // This should be the same size as the probe tuple row.
-    int _result_tuple_row_size;
-
-    void create_output_row(TupleRow* out, TupleRow* left, TupleRow* right);
-    Status compare_row(TupleRow* left_row, TupleRow* right_row, bool* is_lt);
-    Status get_next_row(RuntimeState* state, TupleRow* out_row, bool* eos);
-    Status get_input_row(RuntimeState* state, int child_idx);
-};
-
-} // namespace doris
-
-#endif
diff --git a/be/src/vec/exec/join/vhash_join_node.h b/be/src/vec/exec/join/vhash_join_node.h
index e2762e64f8..caf4803231 100644
--- a/be/src/vec/exec/join/vhash_join_node.h
+++ b/be/src/vec/exec/join/vhash_join_node.h
@@ -115,29 +115,28 @@ using HashTableVariants =
                      I128FixedKeyHashTableContext<false>, I256FixedKeyHashTableContext<true>,
                      I256FixedKeyHashTableContext<false>>;
 
-using JoinOpVariants = std::variant<std::integral_constant<TJoinOp::type, TJoinOp::INNER_JOIN>,
-                                    std::integral_constant<TJoinOp::type, TJoinOp::LEFT_SEMI_JOIN>,
-                                    std::integral_constant<TJoinOp::type, TJoinOp::LEFT_ANTI_JOIN>,
-                                    std::integral_constant<TJoinOp::type, TJoinOp::LEFT_OUTER_JOIN>,
-                                    std::integral_constant<TJoinOp::type, TJoinOp::FULL_OUTER_JOIN>,
-                                    std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_OUTER_JOIN>,
-                                    std::integral_constant<TJoinOp::type, TJoinOp::CROSS_JOIN>,
-                                    std::integral_constant<TJoinOp::type, TJoinOp::MERGE_JOIN>,
-                                    std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_SEMI_JOIN>,
-                                    std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_ANTI_JOIN>,
-                                    std::integral_constant<TJoinOp::type, TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN>>;
-
-#define APPLY_FOR_JOINOP_VARIANTS(M)        \
-    M(INNER_JOIN)                           \
-    M(LEFT_SEMI_JOIN)                       \
-    M(LEFT_ANTI_JOIN)                       \
-    M(LEFT_OUTER_JOIN)                      \
-    M(FULL_OUTER_JOIN)                      \
-    M(RIGHT_OUTER_JOIN)                     \
-    M(CROSS_JOIN)                           \
-    M(MERGE_JOIN)                           \
-    M(RIGHT_SEMI_JOIN)                      \
-    M(RIGHT_ANTI_JOIN)                      \
+using JoinOpVariants =
+        std::variant<std::integral_constant<TJoinOp::type, TJoinOp::INNER_JOIN>,
+                     std::integral_constant<TJoinOp::type, TJoinOp::LEFT_SEMI_JOIN>,
+                     std::integral_constant<TJoinOp::type, TJoinOp::LEFT_ANTI_JOIN>,
+                     std::integral_constant<TJoinOp::type, TJoinOp::LEFT_OUTER_JOIN>,
+                     std::integral_constant<TJoinOp::type, TJoinOp::FULL_OUTER_JOIN>,
+                     std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_OUTER_JOIN>,
+                     std::integral_constant<TJoinOp::type, TJoinOp::CROSS_JOIN>,
+                     std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_SEMI_JOIN>,
+                     std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_ANTI_JOIN>,
+                     std::integral_constant<TJoinOp::type, TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN>>;
+
+#define APPLY_FOR_JOINOP_VARIANTS(M) \
+    M(INNER_JOIN)                    \
+    M(LEFT_SEMI_JOIN)                \
+    M(LEFT_ANTI_JOIN)                \
+    M(LEFT_OUTER_JOIN)               \
+    M(FULL_OUTER_JOIN)               \
+    M(RIGHT_OUTER_JOIN)              \
+    M(CROSS_JOIN)                    \
+    M(RIGHT_SEMI_JOIN)               \
+    M(RIGHT_ANTI_JOIN)               \
     M(NULL_AWARE_LEFT_ANTI_JOIN)
 
 class VExprContext;
diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index e60e78c0b4..5098d332fa 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -4304,8 +4304,6 @@ join_operator ::=
   {: RESULT = JoinOperator.INNER_JOIN; :}
   | KW_LEFT opt_outer KW_JOIN
   {: RESULT = JoinOperator.LEFT_OUTER_JOIN; :}
-  | KW_MERGE KW_JOIN
-  {: RESULT = JoinOperator.MERGE_JOIN; :}
   | KW_RIGHT opt_outer KW_JOIN
   {: RESULT = JoinOperator.RIGHT_OUTER_JOIN; :}
   | KW_FULL opt_outer KW_JOIN
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/JoinOperator.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/JoinOperator.java
index 49d9e9f7b4..54c52109dd 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/JoinOperator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/JoinOperator.java
@@ -29,7 +29,6 @@ public enum JoinOperator {
     RIGHT_ANTI_JOIN("RIGHT ANTI JOIN", TJoinOp.RIGHT_ANTI_JOIN),
     RIGHT_OUTER_JOIN("RIGHT OUTER JOIN", TJoinOp.RIGHT_OUTER_JOIN),
     FULL_OUTER_JOIN("FULL OUTER JOIN", TJoinOp.FULL_OUTER_JOIN),
-    MERGE_JOIN("MERGE JOIN", TJoinOp.MERGE_JOIN),
     CROSS_JOIN("CROSS JOIN", TJoinOp.CROSS_JOIN),
     // Variant of the LEFT ANTI JOIN that is used for the equal of
     // NOT IN subqueries. It can have a single equality join conjunct
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java
index d78aac9c46..16d666822b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java
@@ -605,8 +605,6 @@ public class TableRef implements ParseNode, Writable {
                 return "FULL OUTER JOIN";
             case CROSS_JOIN:
                 return "CROSS JOIN";
-            case MERGE_JOIN:
-                return "MERGE JOIN";
             default:
                 return "bad join op: " + joinOp.toString();
         }
diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift
index ef599e4bb9..c46b0f3c24 100644
--- a/gensrc/thrift/PlanNodes.thrift
+++ b/gensrc/thrift/PlanNodes.thrift
@@ -29,7 +29,7 @@ enum TPlanNodeType {
   CSV_SCAN_NODE, // deprecated
   SCHEMA_SCAN_NODE,
   HASH_JOIN_NODE,
-  MERGE_JOIN_NODE,
+  MERGE_JOIN_NODE, // deprecated
   AGGREGATION_NODE,
   PRE_AGGREGATION_NODE,
   SORT_NODE,
@@ -364,7 +364,7 @@ enum TJoinOp {
   RIGHT_OUTER_JOIN,
   FULL_OUTER_JOIN,
   CROSS_JOIN,
-  MERGE_JOIN,
+  MERGE_JOIN, // deprecated
 
   RIGHT_SEMI_JOIN,
   LEFT_ANTI_JOIN,


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 11/22: [bugfix]handle ComlumnDictionary in evaluate_and and evaluate_or (#9818)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit a7c6ca63c45e553973f91a5e9b69c00e6f5af5a1
Author: morningman <mo...@163.com>
AuthorDate: Wed Jun 1 22:10:58 2022 +0800

    [bugfix]handle ComlumnDictionary in evaluate_and and evaluate_or (#9818)
    
    * handle ColumnDictory in evaluate_or
    
    We need to handle ComlumnDictory in evaluate_or, otherwise delete handler
    would triger a core dump.
    
    * handle ComlumnDictionary in evaluate_and
    
    Because there is only one difference between evaluate_and and
    evaluate_or, that is or and delete, I merge two macros into one.
    
    Delete handlers also trigger evaluate_and, i am not sure if column
    dictionary would be used in evaluate_and.
    
    * clang format
    
    * fix short circut for evaluate_and and evaluate_or
    
    * clang format
---
 be/src/olap/comparison_predicate.cpp | 451 +++++++++++++++++------------------
 1 file changed, 217 insertions(+), 234 deletions(-)

diff --git a/be/src/olap/comparison_predicate.cpp b/be/src/olap/comparison_predicate.cpp
index 363c75e5b8..2223d1fe19 100644
--- a/be/src/olap/comparison_predicate.cpp
+++ b/be/src/olap/comparison_predicate.cpp
@@ -28,8 +28,8 @@
 
 namespace doris {
 
-#define COMPARISON_PRED_CONSTRUCTOR(CLASS) \
-    template <class T>                     \
+#define COMPARISON_PRED_CONSTRUCTOR(CLASS)                             \
+    template <class T>                                                 \
     CLASS<T>::CLASS(uint32_t column_id, const T& value, bool opposite) \
             : ColumnPredicate(column_id, opposite), _value(value) {}
 
@@ -55,55 +55,54 @@ COMPARISON_PRED_CONSTRUCTOR_STRING(LessEqualPredicate)
 COMPARISON_PRED_CONSTRUCTOR_STRING(GreaterPredicate)
 COMPARISON_PRED_CONSTRUCTOR_STRING(GreaterEqualPredicate)
 
-#define COMPARISON_PRED_EVALUATE(CLASS, OP)                                           \
-    template <class T>                                                                \
-    void CLASS<T>::evaluate(VectorizedRowBatch* batch) const {                        \
-        uint16_t n = batch->size();                                                   \
-        if (n == 0) {                                                                 \
-            return;                                                                   \
-        }                                                                             \
-        uint16_t* sel = batch->selected();                                            \
-        const T* col_vector =                                                         \
-                reinterpret_cast<const T*>(batch->column(_column_id)->col_data());    \
-        uint16_t new_size = 0;                                                        \
-        if (batch->column(_column_id)->no_nulls()) {                                  \
-            if (batch->selected_in_use()) {                                           \
-                for (uint16_t j = 0; j != n; ++j) {                                   \
-                    uint16_t i = sel[j];                                              \
-                    sel[new_size] = i;                                                \
-                    new_size += (col_vector[i] OP _value);                            \
-                }                                                                     \
-                batch->set_size(new_size);                                            \
-            } else {                                                                  \
-                for (uint16_t i = 0; i != n; ++i) {                                   \
-                    sel[new_size] = i;                                                \
-                    new_size += (col_vector[i] OP _value);                            \
-                }                                                                     \
-                if (new_size < n) {                                                   \
-                    batch->set_size(new_size);                                        \
-                    batch->set_selected_in_use(true);                                 \
-                }                                                                     \
-            }                                                                         \
-        } else {                                                                      \
-            bool* is_null = batch->column(_column_id)->is_null();                     \
-            if (batch->selected_in_use()) {                                           \
-                for (uint16_t j = 0; j != n; ++j) {                                   \
-                    uint16_t i = sel[j];                                              \
-                    sel[new_size] = i;                                                \
-                    new_size += (!is_null[i] && (col_vector[i] OP _value));           \
-                }                                                                     \
-                batch->set_size(new_size);                                            \
-            } else {                                                                  \
-                for (uint16_t i = 0; i != n; ++i) {                                   \
-                    sel[new_size] = i;                                                \
-                    new_size += (!is_null[i] && (col_vector[i] OP _value));           \
-                }                                                                     \
-                if (new_size < n) {                                                   \
-                    batch->set_size(new_size);                                        \
-                    batch->set_selected_in_use(true);                                 \
-                }                                                                     \
-            }                                                                         \
-        }                                                                             \
+#define COMPARISON_PRED_EVALUATE(CLASS, OP)                                                      \
+    template <class T>                                                                           \
+    void CLASS<T>::evaluate(VectorizedRowBatch* batch) const {                                   \
+        uint16_t n = batch->size();                                                              \
+        if (n == 0) {                                                                            \
+            return;                                                                              \
+        }                                                                                        \
+        uint16_t* sel = batch->selected();                                                       \
+        const T* col_vector = reinterpret_cast<const T*>(batch->column(_column_id)->col_data()); \
+        uint16_t new_size = 0;                                                                   \
+        if (batch->column(_column_id)->no_nulls()) {                                             \
+            if (batch->selected_in_use()) {                                                      \
+                for (uint16_t j = 0; j != n; ++j) {                                              \
+                    uint16_t i = sel[j];                                                         \
+                    sel[new_size] = i;                                                           \
+                    new_size += (col_vector[i] OP _value);                                       \
+                }                                                                                \
+                batch->set_size(new_size);                                                       \
+            } else {                                                                             \
+                for (uint16_t i = 0; i != n; ++i) {                                              \
+                    sel[new_size] = i;                                                           \
+                    new_size += (col_vector[i] OP _value);                                       \
+                }                                                                                \
+                if (new_size < n) {                                                              \
+                    batch->set_size(new_size);                                                   \
+                    batch->set_selected_in_use(true);                                            \
+                }                                                                                \
+            }                                                                                    \
+        } else {                                                                                 \
+            bool* is_null = batch->column(_column_id)->is_null();                                \
+            if (batch->selected_in_use()) {                                                      \
+                for (uint16_t j = 0; j != n; ++j) {                                              \
+                    uint16_t i = sel[j];                                                         \
+                    sel[new_size] = i;                                                           \
+                    new_size += (!is_null[i] && (col_vector[i] OP _value));                      \
+                }                                                                                \
+                batch->set_size(new_size);                                                       \
+            } else {                                                                             \
+                for (uint16_t i = 0; i != n; ++i) {                                              \
+                    sel[new_size] = i;                                                           \
+                    new_size += (!is_null[i] && (col_vector[i] OP _value));                      \
+                }                                                                                \
+                if (new_size < n) {                                                              \
+                    batch->set_size(new_size);                                                   \
+                    batch->set_selected_in_use(true);                                            \
+                }                                                                                \
+            }                                                                                    \
+        }                                                                                        \
     }
 
 COMPARISON_PRED_EVALUATE(EqualPredicate, ==)
@@ -113,30 +112,28 @@ COMPARISON_PRED_EVALUATE(LessEqualPredicate, <=)
 COMPARISON_PRED_EVALUATE(GreaterPredicate, >)
 COMPARISON_PRED_EVALUATE(GreaterEqualPredicate, >=)
 
-#define COMPARISON_PRED_COLUMN_BLOCK_EVALUATE(CLASS, OP)                                  \
-    template <class T>                                                                    \
-    void CLASS<T>::evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) const {    \
-        uint16_t new_size = 0;                                                            \
-        if (block->is_nullable()) {                                                       \
-            for (uint16_t i = 0; i < *size; ++i) {                                        \
-                uint16_t idx = sel[i];                                                    \
-                sel[new_size] = idx;                                                      \
-                const T* cell_value =                                                     \
-                        reinterpret_cast<const T*>(block->cell(idx).cell_ptr());          \
-                auto result = (!block->cell(idx).is_null() && (*cell_value OP _value));   \
-                new_size += _opposite ? !result : result;                                 \
-            }                                                                             \
-        } else {                                                                          \
-            for (uint16_t i = 0; i < *size; ++i) {                                        \
-                uint16_t idx = sel[i];                                                    \
-                sel[new_size] = idx;                                                      \
-                const T* cell_value =                                                     \
-                        reinterpret_cast<const T*>(block->cell(idx).cell_ptr());          \
-                auto result = (*cell_value OP _value);                                    \
-                new_size += _opposite ? !result : result;                                 \
-            }                                                                             \
-        }                                                                                 \
-        *size = new_size;                                                                 \
+#define COMPARISON_PRED_COLUMN_BLOCK_EVALUATE(CLASS, OP)                                       \
+    template <class T>                                                                         \
+    void CLASS<T>::evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) const {         \
+        uint16_t new_size = 0;                                                                 \
+        if (block->is_nullable()) {                                                            \
+            for (uint16_t i = 0; i < *size; ++i) {                                             \
+                uint16_t idx = sel[i];                                                         \
+                sel[new_size] = idx;                                                           \
+                const T* cell_value = reinterpret_cast<const T*>(block->cell(idx).cell_ptr()); \
+                auto result = (!block->cell(idx).is_null() && (*cell_value OP _value));        \
+                new_size += _opposite ? !result : result;                                      \
+            }                                                                                  \
+        } else {                                                                               \
+            for (uint16_t i = 0; i < *size; ++i) {                                             \
+                uint16_t idx = sel[i];                                                         \
+                sel[new_size] = idx;                                                           \
+                const T* cell_value = reinterpret_cast<const T*>(block->cell(idx).cell_ptr()); \
+                auto result = (*cell_value OP _value);                                         \
+                new_size += _opposite ? !result : result;                                      \
+            }                                                                                  \
+        }                                                                                      \
+        *size = new_size;                                                                      \
     }
 
 COMPARISON_PRED_COLUMN_BLOCK_EVALUATE(EqualPredicate, ==)
@@ -193,7 +190,8 @@ COMPARISON_PRED_COLUMN_BLOCK_EVALUATE(GreaterEqualPredicate, >=)
         } else if (column.is_column_dictionary()) {                                                \
             if constexpr (std::is_same_v<T, StringValue>) {                                        \
                 auto& dict_col =                                                                   \
-                        reinterpret_cast<vectorized::ColumnDictionary<vectorized::Int32>&>(column);\
+                        reinterpret_cast<vectorized::ColumnDictionary<vectorized::Int32>&>(        \
+                                column);                                                           \
                 auto& data_array = dict_col.get_data();                                            \
                 auto dict_code = IS_RANGE ? dict_col.find_code_by_bound(_value, 1 OP 0, 1 OP 1)    \
                                           : dict_col.find_code(_value);                            \
@@ -201,13 +199,12 @@ COMPARISON_PRED_COLUMN_BLOCK_EVALUATE(GreaterEqualPredicate, >=)
                     uint16_t idx = sel[i];                                                         \
                     sel[new_size] = idx;                                                           \
                     const auto& cell_value = data_array[idx];                                      \
-                    bool ret = cell_value OP dict_code;                                           \
+                    bool ret = cell_value OP dict_code;                                            \
                     new_size += _opposite ? !ret : ret;                                            \
                 }                                                                                  \
             }                                                                                      \
         } else {                                                                                   \
-            auto& pred_column_ref =                                                                \
-                    reinterpret_cast<vectorized::PredicateColumnType<T>&>(column);                 \
+            auto& pred_column_ref = reinterpret_cast<vectorized::PredicateColumnType<T>&>(column); \
             auto& data_array = pred_column_ref.get_data();                                         \
             for (uint16_t i = 0; i < *size; i++) {                                                 \
                 uint16_t idx = sel[i];                                                             \
@@ -220,7 +217,6 @@ COMPARISON_PRED_COLUMN_BLOCK_EVALUATE(GreaterEqualPredicate, >=)
         *size = new_size;                                                                          \
     }
 
-
 COMPARISON_PRED_COLUMN_EVALUATE(EqualPredicate, ==, false)
 COMPARISON_PRED_COLUMN_EVALUATE(NotEqualPredicate, !=, false)
 COMPARISON_PRED_COLUMN_EVALUATE(LessPredicate, <, true)
@@ -228,35 +224,34 @@ COMPARISON_PRED_COLUMN_EVALUATE(LessEqualPredicate, <=, true)
 COMPARISON_PRED_COLUMN_EVALUATE(GreaterPredicate, >, true)
 COMPARISON_PRED_COLUMN_EVALUATE(GreaterEqualPredicate, >=, true)
 
-#define COMPARISON_PRED_COLUMN_EVALUATE_VEC(CLASS, OP)                                         \
-    template <class T>                                                                         \
-    void CLASS<T>::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags)       \
-            const {                                                                            \
-        if (column.is_nullable()) {                                                            \
-            auto* nullable_column =                                                            \
-                    vectorized::check_and_get_column<vectorized::ColumnNullable>(column);      \
-            auto& data_array = reinterpret_cast<const vectorized::PredicateColumnType<T>&>(    \
-                                       nullable_column->get_nested_column())                   \
-                                       .get_data();                                            \
-            auto& null_bitmap = reinterpret_cast<const vectorized::ColumnVector<uint8_t>&>(    \
-                                        *(nullable_column->get_null_map_column_ptr()))         \
-                                        .get_data();                                           \
-            for (uint16_t i = 0; i < size; i++) {                                              \
-                flags[i] = (data_array[i] OP _value) && (!null_bitmap[i]);                     \
-            }                                                                                  \
-        } else {                                                                               \
-            auto& predicate_column =                                                           \
-                    reinterpret_cast<vectorized::PredicateColumnType<T>&>(column);             \
-            auto& data_array = predicate_column.get_data();                                    \
-            for (uint16_t i = 0; i < size; i++) {                                              \
-                flags[i] = data_array[i] OP _value;                                            \
-            }                                                                                  \
-        }                                                                                      \
-        if (_opposite) {                                                                       \
-            for (uint16_t i = 0; i < size; i++) {                                              \
-                flags[i] = !flags[i];                                                          \
-            }                                                                                  \
-        }                                                                                      \
+#define COMPARISON_PRED_COLUMN_EVALUATE_VEC(CLASS, OP)                                           \
+    template <class T>                                                                           \
+    void CLASS<T>::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) const { \
+        if (column.is_nullable()) {                                                              \
+            auto* nullable_column =                                                              \
+                    vectorized::check_and_get_column<vectorized::ColumnNullable>(column);        \
+            auto& data_array = reinterpret_cast<const vectorized::PredicateColumnType<T>&>(      \
+                                       nullable_column->get_nested_column())                     \
+                                       .get_data();                                              \
+            auto& null_bitmap = reinterpret_cast<const vectorized::ColumnVector<uint8_t>&>(      \
+                                        *(nullable_column->get_null_map_column_ptr()))           \
+                                        .get_data();                                             \
+            for (uint16_t i = 0; i < size; i++) {                                                \
+                flags[i] = (data_array[i] OP _value) && (!null_bitmap[i]);                       \
+            }                                                                                    \
+        } else {                                                                                 \
+            auto& predicate_column =                                                             \
+                    reinterpret_cast<vectorized::PredicateColumnType<T>&>(column);               \
+            auto& data_array = predicate_column.get_data();                                      \
+            for (uint16_t i = 0; i < size; i++) {                                                \
+                flags[i] = data_array[i] OP _value;                                              \
+            }                                                                                    \
+        }                                                                                        \
+        if (_opposite) {                                                                         \
+            for (uint16_t i = 0; i < size; i++) {                                                \
+                flags[i] = !flags[i];                                                            \
+            }                                                                                    \
+        }                                                                                        \
     }
 
 COMPARISON_PRED_COLUMN_EVALUATE_VEC(EqualPredicate, ==)
@@ -266,133 +261,121 @@ COMPARISON_PRED_COLUMN_EVALUATE_VEC(LessEqualPredicate, <=)
 COMPARISON_PRED_COLUMN_EVALUATE_VEC(GreaterPredicate, >)
 COMPARISON_PRED_COLUMN_EVALUATE_VEC(GreaterEqualPredicate, >=)
 
-#define COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_OR(CLASS, OP)                                      \
-    template <class T>                                                                           \
-    void CLASS<T>::evaluate_or(ColumnBlock* block, uint16_t* sel, uint16_t size, bool* flags)    \
-            const {                                                                              \
-        if (block->is_nullable()) {                                                              \
-            for (uint16_t i = 0; i < size; ++i) {                                                \
-                if (flags[i]) continue;                                                          \
-                uint16_t idx = sel[i];                                                           \
-                const T* cell_value =                                                            \
-                        reinterpret_cast<const T*>(block->cell(idx).cell_ptr());                 \
-                auto result = (!block->cell(idx).is_null() && (*cell_value OP _value));          \
-                flags[i] |= _opposite ? !result : result;                                        \
-            }                                                                                    \
-        } else {                                                                                 \
-            for (uint16_t i = 0; i < size; ++i) {                                                \
-                if (flags[i]) continue;                                                          \
-                uint16_t idx = sel[i];                                                           \
-                const T* cell_value =                                                            \
-                        reinterpret_cast<const T*>(block->cell(idx).cell_ptr());                 \
-                auto result = (*cell_value OP _value);                                           \
-                flags[i] |= _opposite ? !result : result;                                        \
-            }                                                                                    \
-        }                                                                                        \
+#define COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_BOOL(CLASS, OP, BOOL_NAME, BOOL_OP, SHORT_OP)    \
+    template <class T>                                                                         \
+    void CLASS<T>::evaluate_##BOOL_NAME(ColumnBlock* block, uint16_t* sel, uint16_t size,      \
+                                        bool* flags) const {                                   \
+        if (block->is_nullable()) {                                                            \
+            for (uint16_t i = 0; i < size; ++i) {                                              \
+                if (SHORT_OP(flags[i])) continue;                                              \
+                uint16_t idx = sel[i];                                                         \
+                const T* cell_value = reinterpret_cast<const T*>(block->cell(idx).cell_ptr()); \
+                auto result = (!block->cell(idx).is_null() && (*cell_value OP _value));        \
+                flags[i] = flags[i] BOOL_OP(_opposite ? !result : result);                     \
+            }                                                                                  \
+        } else {                                                                               \
+            for (uint16_t i = 0; i < size; ++i) {                                              \
+                if (flags[i]) continue;                                                        \
+                uint16_t idx = sel[i];                                                         \
+                const T* cell_value = reinterpret_cast<const T*>(block->cell(idx).cell_ptr()); \
+                auto result = (*cell_value OP _value);                                         \
+                flags[i] = flags[i] BOOL_OP(_opposite ? !result : result);                     \
+            }                                                                                  \
+        }                                                                                      \
     }
 
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_OR(EqualPredicate, ==)
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_OR(NotEqualPredicate, !=)
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_OR(LessPredicate, <)
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_OR(LessEqualPredicate, <=)
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_OR(GreaterPredicate, >)
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_OR(GreaterEqualPredicate, >=)
-
-#define COMPARISON_PRED_COLUMN_EVALUATE_OR(CLASS, OP)                                                                                                \
-    template <class T>                                                                                                                               \
-    void CLASS<T>::evaluate_or(vectorized::IColumn& column, uint16_t* sel, uint16_t size, bool* flags) const {                                       \
-        if (column.is_nullable()) {                                                                                                                  \
-            auto* nullable_column = vectorized::check_and_get_column<vectorized::ColumnNullable>(column);                                            \
-            auto& data_array = reinterpret_cast<const vectorized::PredicateColumnType<T>&>(nullable_column->get_nested_column()).get_data();         \
-            auto& null_bitmap = reinterpret_cast<const vectorized::ColumnVector<uint8_t>&>(*(nullable_column->get_null_map_column_ptr())).get_data();\
-            for (uint16_t i = 0; i < size; i++) {                                                                                                    \
-                if (flags[i]) continue;                                                                                                              \
-                uint16_t idx = sel[i];                                                                                                               \
-                bool ret = !null_bitmap[idx] && (data_array[idx] OP _value);                                                                         \
-                flags[i] |= _opposite ? !ret : ret;                                                                                                  \
-            }                                                                                                                                        \
-        } else {                                                                                                                                     \
-            auto& predicate_column = reinterpret_cast<vectorized::PredicateColumnType<T>&>(column);                                                  \
-            auto& data_array = predicate_column.get_data();                                                                                          \
-            for (uint16_t i = 0; i < size; ++i) {                                                                                                    \
-                if (flags[i]) continue;                                                                                                              \
-                uint16_t idx = sel[i];                                                                                                               \
-                bool ret = data_array[idx] OP _value;                                                                                                \
-                flags[i] |= _opposite ? !ret : ret;                                                                                                  \
-            }                                                                                                                                        \
-        }                                                                                                                                            \
-    }
- 
-COMPARISON_PRED_COLUMN_EVALUATE_OR(EqualPredicate, ==)
-COMPARISON_PRED_COLUMN_EVALUATE_OR(NotEqualPredicate, !=)
-COMPARISON_PRED_COLUMN_EVALUATE_OR(LessPredicate, <)
-COMPARISON_PRED_COLUMN_EVALUATE_OR(LessEqualPredicate, <=)
-COMPARISON_PRED_COLUMN_EVALUATE_OR(GreaterPredicate, >)
-COMPARISON_PRED_COLUMN_EVALUATE_OR(GreaterEqualPredicate, >=)
-
-#define COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_AND(CLASS, OP)                                      \
-    template <class T>                                                                            \
-    void CLASS<T>::evaluate_and(ColumnBlock* block, uint16_t* sel, uint16_t size, bool* flags)    \
-            const {                                                                               \
-        if (block->is_nullable()) {                                                               \
-            for (uint16_t i = 0; i < size; ++i) {                                                 \
-                if (!flags[i]) continue;                                                          \
-                uint16_t idx = sel[i];                                                            \
-                const T* cell_value =                                                             \
-                        reinterpret_cast<const T*>(block->cell(idx).cell_ptr());                  \
-                auto result = (!block->cell(idx).is_null() && (*cell_value OP _value));           \
-                flags[i] &= _opposite ? !result : result;                                         \
-            }                                                                                     \
-        } else {                                                                                  \
-            for (uint16_t i = 0; i < size; ++i) {                                                 \
-                if (!flags[i]) continue;                                                          \
-                uint16_t idx = sel[i];                                                            \
-                const T* cell_value =                                                             \
-                        reinterpret_cast<const T*>(block->cell(idx).cell_ptr());                  \
-                auto result = (*cell_value OP _value);                                            \
-                flags[i] &= _opposite ? !result : result;                                         \
-            }                                                                                     \
-        }                                                                                         \
+#define COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_BOOL2(BOOL_NAME, BOOL_OP, SHORT_OP)                 \
+    COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_BOOL(EqualPredicate, ==, BOOL_NAME, BOOL_OP, SHORT_OP)  \
+    COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_BOOL(NotEqualPredicate, !=, BOOL_NAME, BOOL_OP,         \
+                                               SHORT_OP)                                          \
+    COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_BOOL(LessPredicate, <, BOOL_NAME, BOOL_OP, SHORT_OP)    \
+    COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_BOOL(LessEqualPredicate, <=, BOOL_NAME, BOOL_OP,        \
+                                               SHORT_OP)                                          \
+    COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_BOOL(GreaterPredicate, >, BOOL_NAME, BOOL_OP, SHORT_OP) \
+    COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_BOOL(GreaterEqualPredicate, >=, BOOL_NAME, BOOL_OP,     \
+                                               SHORT_OP)
+
+COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_BOOL2(or, |, )
+COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_BOOL2(and, &, !)
+
+#define COMPARISON_PRED_COLUMN_EVALUATE_BOOL(CLASS, OP, IS_RANGE, BOOL_NAME, BOOL_OP, SHORT_OP)    \
+    template <class T>                                                                             \
+    void CLASS<T>::evaluate_##BOOL_NAME(vectorized::IColumn& column, uint16_t* sel, uint16_t size, \
+                                        bool* flags) const {                                       \
+        if (column.is_nullable()) {                                                                \
+            auto* nullable_column =                                                                \
+                    vectorized::check_and_get_column<vectorized::ColumnNullable>(column);          \
+            auto& nested_col = nullable_column->get_nested_column();                               \
+            auto& null_bitmap = reinterpret_cast<const vectorized::ColumnVector<uint8_t>&>(        \
+                                        *(nullable_column->get_null_map_column_ptr()))             \
+                                        .get_data();                                               \
+            if (nested_col.is_column_dictionary()) {                                               \
+                if constexpr (std::is_same_v<T, StringValue>) {                                    \
+                    auto* nested_col_ptr = vectorized::check_and_get_column<                       \
+                            vectorized::ColumnDictionary<vectorized::Int32>>(nested_col);          \
+                    auto& data_array = nested_col_ptr->get_data();                                 \
+                    auto dict_code =                                                               \
+                            IS_RANGE ? nested_col_ptr->find_code_by_bound(_value, 1 OP 0, 1 OP 1)  \
+                                     : nested_col_ptr->find_code(_value);                          \
+                    for (uint16_t i = 0; i < size; i++) {                                          \
+                        if (SHORT_OP(flags[i])) continue;                                          \
+                        uint16_t idx = sel[i];                                                     \
+                        bool ret = !null_bitmap[idx] && (data_array[idx] OP dict_code);            \
+                        flags[i] = flags[i] BOOL_OP(_opposite ? !ret : ret);                       \
+                    }                                                                              \
+                }                                                                                  \
+            } else {                                                                               \
+                auto& data_array =                                                                 \
+                        reinterpret_cast<const vectorized::PredicateColumnType<T>&>(nested_col)    \
+                                .get_data();                                                       \
+                for (uint16_t i = 0; i < size; i++) {                                              \
+                    if (SHORT_OP(flags[i])) continue;                                              \
+                    uint16_t idx = sel[i];                                                         \
+                    bool ret = !null_bitmap[idx] && (data_array[idx] OP _value);                   \
+                    flags[i] = flags[i] BOOL_OP(_opposite ? !ret : ret);                           \
+                }                                                                                  \
+            }                                                                                      \
+        } else if (column.is_column_dictionary()) {                                                \
+            if constexpr (std::is_same_v<T, StringValue>) {                                        \
+                auto& dict_col =                                                                   \
+                        reinterpret_cast<vectorized::ColumnDictionary<vectorized::Int32>&>(        \
+                                column);                                                           \
+                auto& data_array = dict_col.get_data();                                            \
+                auto dict_code = IS_RANGE ? dict_col.find_code_by_bound(_value, 1 OP 0, 1 OP 1)    \
+                                          : dict_col.find_code(_value);                            \
+                for (uint16_t i = 0; i < size; i++) {                                              \
+                    if (SHORT_OP(flags[i])) continue;                                              \
+                    uint16_t idx = sel[i];                                                         \
+                    bool ret = data_array[idx] OP dict_code;                                       \
+                    flags[i] = flags[i] BOOL_OP(_opposite ? !ret : ret);                           \
+                }                                                                                  \
+            }                                                                                      \
+        } else {                                                                                   \
+            auto& predicate_column =                                                               \
+                    reinterpret_cast<vectorized::PredicateColumnType<T>&>(column);                 \
+            auto& data_array = predicate_column.get_data();                                        \
+            for (uint16_t i = 0; i < size; ++i) {                                                  \
+                if (SHORT_OP(flags[i])) continue;                                                  \
+                uint16_t idx = sel[i];                                                             \
+                bool ret = data_array[idx] OP _value;                                              \
+                flags[i] = flags[i] BOOL_OP(_opposite ? !ret : ret);                               \
+            }                                                                                      \
+        }                                                                                          \
     }
 
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_AND(EqualPredicate, ==)
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_AND(NotEqualPredicate, !=)
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_AND(LessPredicate, <)
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_AND(LessEqualPredicate, <=)
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_AND(GreaterPredicate, >)
-COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_AND(GreaterEqualPredicate, >=)
-
-#define COMPARISON_PRED_COLUMN_EVALUATE_AND(CLASS, OP)                                                                                               \
-    template <class T>                                                                                                                               \
-    void CLASS<T>::evaluate_and(vectorized::IColumn& column, uint16_t* sel, uint16_t size, bool* flags) const {                                      \
-        if (column.is_nullable()) {                                                                                                                  \
-            auto* nullable_column = vectorized::check_and_get_column<vectorized::ColumnNullable>(column);                                            \
-            auto& data_array = reinterpret_cast<const vectorized::PredicateColumnType<T>&>(nullable_column->get_nested_column()).get_data();         \
-            auto& null_bitmap = reinterpret_cast<const vectorized::ColumnVector<uint8_t>&>(*(nullable_column->get_null_map_column_ptr())).get_data();\
-            for (uint16_t i = 0; i < size; i++) {                                                                                                    \
-                if (!flags[i]) continue;                                                                                                             \
-                uint16_t idx = sel[i];                                                                                                               \
-                bool ret = !null_bitmap[idx] && (data_array[idx] OP _value);                                                                         \
-                flags[i] &= _opposite ? !ret : ret;                                                                                                  \
-            }                                                                                                                                        \
-        } else {                                                                                                                                     \
-            auto& predicate_column = reinterpret_cast<vectorized::PredicateColumnType<T>&>(column);                                                  \
-            auto& data_array = predicate_column.get_data();                                                                                          \
-            for (uint16_t i = 0; i < size; ++i) {                                                                                                    \
-                if (!flags[i]) continue;                                                                                                             \
-                uint16_t idx = sel[i];                                                                                                               \
-                bool ret = data_array[idx] OP _value;                                                                                                \
-                flags[i] &= _opposite ? !ret : ret;                                                                                                  \
-            }                                                                                                                                        \
-        }                                                                                                                                            \
-    }
- 
-COMPARISON_PRED_COLUMN_EVALUATE_AND(EqualPredicate, ==)
-COMPARISON_PRED_COLUMN_EVALUATE_AND(NotEqualPredicate, !=)
-COMPARISON_PRED_COLUMN_EVALUATE_AND(LessPredicate, <)
-COMPARISON_PRED_COLUMN_EVALUATE_AND(LessEqualPredicate, <=)
-COMPARISON_PRED_COLUMN_EVALUATE_AND(GreaterPredicate, >)
-COMPARISON_PRED_COLUMN_EVALUATE_AND(GreaterEqualPredicate, >=)
+#define COMPARISON_PRED_COLUMN_EVALUATE_BOOL2(BOOL_NAME, BOOL_OP, SHORT_OP)                       \
+    COMPARISON_PRED_COLUMN_EVALUATE_BOOL(EqualPredicate, ==, false, BOOL_NAME, BOOL_OP, SHORT_OP) \
+    COMPARISON_PRED_COLUMN_EVALUATE_BOOL(NotEqualPredicate, !=, false, BOOL_NAME, BOOL_OP,        \
+                                         SHORT_OP)                                                \
+    COMPARISON_PRED_COLUMN_EVALUATE_BOOL(LessPredicate, <, true, BOOL_NAME, BOOL_OP, SHORT_OP)    \
+    COMPARISON_PRED_COLUMN_EVALUATE_BOOL(LessEqualPredicate, <=, true, BOOL_NAME, BOOL_OP,        \
+                                         SHORT_OP)                                                \
+    COMPARISON_PRED_COLUMN_EVALUATE_BOOL(GreaterPredicate, >, true, BOOL_NAME, BOOL_OP, SHORT_OP) \
+    COMPARISON_PRED_COLUMN_EVALUATE_BOOL(GreaterEqualPredicate, >=, true, BOOL_NAME, BOOL_OP,     \
+                                         SHORT_OP)
+
+COMPARISON_PRED_COLUMN_EVALUATE_BOOL2(or, |, )
+COMPARISON_PRED_COLUMN_EVALUATE_BOOL2(and, &, !)
 
 #define BITMAP_COMPARE_EqualPredicate(s, exact_match, seeked_ordinal, iterator, bitmap, roaring) \
     do {                                                                                         \
@@ -479,8 +462,8 @@ COMPARISON_PRED_COLUMN_EVALUATE_AND(GreaterEqualPredicate, >=)
 #define COMPARISON_PRED_BITMAP_EVALUATE(CLASS, OP)                                        \
     template <class T>                                                                    \
     Status CLASS<T>::evaluate(const Schema& schema,                                       \
-                                 const std::vector<BitmapIndexIterator*>& iterators,      \
-                                 uint32_t num_rows, roaring::Roaring* bitmap) const {     \
+                              const std::vector<BitmapIndexIterator*>& iterators,         \
+                              uint32_t num_rows, roaring::Roaring* bitmap) const {        \
         BitmapIndexIterator* iterator = iterators[_column_id];                            \
         if (iterator == nullptr) {                                                        \
             return Status::OK();                                                          \


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 20/22: [Bug][Vectorized] fix core dump on vcase_expr::close (#9893)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit c1abaec0d5538910559fa7b0671c2f9b51217eb4
Author: HappenLee <ha...@hotmail.com>
AuthorDate: Wed Jun 1 08:05:09 2022 +0800

    [Bug][Vectorized] fix core dump on vcase_expr::close (#9893)
    
    Co-authored-by: lihaopeng <li...@baidu.com>
---
 be/src/vec/exec/volap_scanner.cpp    |  3 ---
 be/src/vec/exprs/vcase_expr.cpp      | 16 ++++++++++++----
 be/src/vec/exprs/vexpr.h             |  5 +----
 be/src/vec/functions/function_case.h | 11 +----------
 be/src/vec/utils/util.hpp            | 13 +++++++++++--
 5 files changed, 25 insertions(+), 23 deletions(-)

diff --git a/be/src/vec/exec/volap_scanner.cpp b/be/src/vec/exec/volap_scanner.cpp
index 4c86830b30..635b0f4a60 100644
--- a/be/src/vec/exec/volap_scanner.cpp
+++ b/be/src/vec/exec/volap_scanner.cpp
@@ -20,10 +20,7 @@
 #include <memory>
 
 #include "runtime/runtime_state.h"
-#include "vec/columns/column_complex.h"
-#include "vec/columns/column_nullable.h"
 #include "vec/columns/column_string.h"
-#include "vec/columns/column_vector.h"
 #include "vec/common/assert_cast.h"
 #include "vec/core/block.h"
 #include "vec/exec/volap_scan_node.h"
diff --git a/be/src/vec/exprs/vcase_expr.cpp b/be/src/vec/exprs/vcase_expr.cpp
index 20ad1fe340..2683e33bba 100644
--- a/be/src/vec/exprs/vcase_expr.cpp
+++ b/be/src/vec/exprs/vcase_expr.cpp
@@ -68,15 +68,23 @@ Status VCaseExpr::open(RuntimeState* state, VExprContext* context,
                        FunctionContext::FunctionStateScope scope) {
     RETURN_IF_ERROR(VExpr::open(state, context, scope));
     RETURN_IF_ERROR(VExpr::init_function_context(context, scope, _function));
-
-    CaseState* case_state = new CaseState {_data_type};
-    context->fn_context(_fn_context_index)->set_function_state(scope, case_state);
-
+    if (scope == doris_udf::FunctionContext::FRAGMENT_LOCAL) {
+        auto* case_state = new CaseState {_data_type};
+        context->fn_context(_fn_context_index)
+                ->set_function_state(FunctionContext::FRAGMENT_LOCAL, case_state);
+    }
     return Status::OK();
 }
 
 void VCaseExpr::close(RuntimeState* state, VExprContext* context,
                       FunctionContext::FunctionStateScope scope) {
+    if (scope == doris_udf::FunctionContext::FRAGMENT_LOCAL) {
+        auto* case_state = reinterpret_cast<CaseState*>(
+                context->fn_context(_fn_context_index)
+                        ->get_function_state(FunctionContext::FRAGMENT_LOCAL));
+        delete case_state;
+    }
+
     VExpr::close_function_context(context, scope, _function);
     VExpr::close(state, context, scope);
 }
diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h
index 31ccc8efba..7ebe597a0f 100644
--- a/be/src/vec/exprs/vexpr.h
+++ b/be/src/vec/exprs/vexpr.h
@@ -117,10 +117,7 @@ public:
                                           VExpr* parent, int* node_idx, VExpr** root_expr,
                                           VExprContext** ctx);
     const std::vector<VExpr*>& children() const { return _children; }
-    void set_children(RuntimeState* state, VExprContext* ctx, std::vector<VExpr*> children) {
-        close(state, ctx, ctx->get_function_state_scope());
-        _children = children;
-    }
+    void set_children(std::vector<VExpr*> children) { _children = children; }
     virtual std::string debug_string() const;
     static std::string debug_string(const std::vector<VExpr*>& exprs);
     static std::string debug_string(const std::vector<VExprContext*>& ctxs);
diff --git a/be/src/vec/functions/function_case.h b/be/src/vec/functions/function_case.h
index 5210ac80ad..47e33f58ff 100644
--- a/be/src/vec/functions/function_case.h
+++ b/be/src/vec/functions/function_case.h
@@ -327,21 +327,12 @@ public:
 
     Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
                         size_t result, size_t input_rows_count) override {
-        CaseState* case_state = reinterpret_cast<CaseState*>(
+        auto* case_state = reinterpret_cast<CaseState*>(
                 context->get_function_state(FunctionContext::FRAGMENT_LOCAL));
 
         return execute_get_type(case_state->result_type, block, arguments, result,
                                 input_rows_count);
     }
-
-    Status close(FunctionContext* context, FunctionContext::FunctionStateScope scope) override {
-        if (scope == FunctionContext::THREAD_LOCAL) {
-            auto* state = reinterpret_cast<CaseState*>(
-                    context->get_function_state(FunctionContext::THREAD_LOCAL));
-            delete state;
-        }
-        return Status::OK();
-    }
 };
 
 } // namespace doris::vectorized
diff --git a/be/src/vec/utils/util.hpp b/be/src/vec/utils/util.hpp
index 8e50d12b5d..0df02ddd6a 100644
--- a/be/src/vec/utils/util.hpp
+++ b/be/src/vec/utils/util.hpp
@@ -69,7 +69,11 @@ public:
         static constexpr auto is_leaf = [](VExpr* expr) { return !expr->is_and_expr(); };
 
         if (is_leaf(expr)) {
-            return checker(leaf_index++) ? nullptr : expr;
+            if (checker(leaf_index++)) {
+                expr->close(state, context, context->get_function_state_scope());
+                return nullptr;
+            }
+            return expr;
         } else {
             VExpr* left_child =
                     dfs_peel_conjunct(state, context, expr->children()[0], leaf_index, checker);
@@ -77,9 +81,14 @@ public:
                     dfs_peel_conjunct(state, context, expr->children()[1], leaf_index, checker);
 
             if (left_child != nullptr && right_child != nullptr) {
-                expr->set_children(state, context, {left_child, right_child});
+                expr->set_children({left_child, right_child});
                 return expr;
+            } else {
+                // here only close the and expr self, do not close the child
+                expr->set_children({});
+                expr->close(state, context, context->get_function_state_scope());
             }
+
             // here do not close Expr* now
             return left_child != nullptr ? left_child : right_child;
         }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 03/22: [improvement](load) reduce useless err_msg format in VOlapTableSink send (#9531)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit f5b421eaaf757c204d926668e571e0717f73ef41
Author: yixiutt <10...@users.noreply.github.com>
AuthorDate: Sun May 29 16:02:57 2022 +0800

    [improvement](load) reduce useless err_msg format in VOlapTableSink send (#9531)
---
 be/src/vec/sink/vtablet_sink.cpp | 43 ++++++++++++++++++++--------------------
 1 file changed, 21 insertions(+), 22 deletions(-)

diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp
index ba3da67b99..6de189099e 100644
--- a/be/src/vec/sink/vtablet_sink.cpp
+++ b/be/src/vec/sink/vtablet_sink.cpp
@@ -193,26 +193,25 @@ Status VOlapTableSink::_validate_data(RuntimeState* state, vectorized::Block* bl
                 if (!filter_bitmap->Get(j)) {
                     auto str_val = column_string->get_data_at(j);
                     bool invalid = str_val.size > limit;
-
-                    error_msg.clear();
-                    if (str_val.size > desc->type().len) {
-                        fmt::format_to(error_msg, "{}",
-                                       "the length of input is too long than schema. ");
-                        fmt::format_to(error_msg, "column_name: {}; ", desc->col_name());
-                        fmt::format_to(error_msg, "input str: [{}] ", str_val.to_prefix(10));
-                        fmt::format_to(error_msg, "schema length: {}; ", desc->type().len);
-                        fmt::format_to(error_msg, "actual length: {}; ", str_val.size);
-                    } else if (str_val.size > limit) {
-                        fmt::format_to(error_msg, "{}",
-                                       "the length of input string is too long than vec schema. ");
-                        fmt::format_to(error_msg, "column_name: {}; ", desc->col_name());
-                        fmt::format_to(error_msg, "input str: [{}] ", str_val.to_prefix(10));
-                        fmt::format_to(error_msg, "schema length: {}; ", desc->type().len);
-                        fmt::format_to(error_msg, "limit length: {}; ", limit);
-                        fmt::format_to(error_msg, "actual length: {}; ", str_val.size);
-                    }
-
                     if (invalid) {
+                        error_msg.clear();
+                        if (str_val.size > desc->type().len) {
+                            fmt::format_to(error_msg, "{}",
+                                           "the length of input is too long than schema. ");
+                            fmt::format_to(error_msg, "column_name: {}; ", desc->col_name());
+                            fmt::format_to(error_msg, "input str: [{}] ", str_val.to_prefix(10));
+                            fmt::format_to(error_msg, "schema length: {}; ", desc->type().len);
+                            fmt::format_to(error_msg, "actual length: {}; ", str_val.size);
+                        } else if (str_val.size > limit) {
+                            fmt::format_to(
+                                    error_msg, "{}",
+                                    "the length of input string is too long than vec schema. ");
+                            fmt::format_to(error_msg, "column_name: {}; ", desc->col_name());
+                            fmt::format_to(error_msg, "input str: [{}] ", str_val.to_prefix(10));
+                            fmt::format_to(error_msg, "schema length: {}; ", desc->type().len);
+                            fmt::format_to(error_msg, "limit length: {}; ", limit);
+                            fmt::format_to(error_msg, "actual length: {}; ", str_val.size);
+                        }
                         RETURN_IF_ERROR(set_invalid_and_append_error_msg(j));
                     }
                 }
@@ -270,10 +269,10 @@ Status VOlapTableSink::_validate_data(RuntimeState* state, vectorized::Block* bl
         if ((!desc->is_nullable() || desc->type() == TYPE_OBJECT) && column_ptr) {
             const auto& null_map = column_ptr->get_null_map_data();
             for (int j = 0; j < null_map.size(); ++j) {
-                fmt::format_to(error_msg,
-                               "null value for not null column/or bitmap column, column={}; ",
-                               desc->col_name());
                 if (null_map[j] && !filter_bitmap->Get(j)) {
+                    error_msg.clear();
+                    fmt::format_to(error_msg, "null value for not null column, column={}; ",
+                                   desc->col_name());
                     RETURN_IF_ERROR(set_invalid_and_append_error_msg(j));
                 }
             }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 04/22: [bugfix] Fix create table like when having hidden columns (#9694)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 1a43e0d4125bff42f9b79a72caaf9580a3716322
Author: xueweizhang <zx...@163.com>
AuthorDate: Sun May 29 18:02:16 2022 +0800

    [bugfix] Fix create table like when having hidden columns (#9694)
---
 .../java/org/apache/doris/catalog/Catalog.java     | 24 +++++++---
 .../data_model/unique/test_unique_table_like.out   | 15 +++++++
 .../unique/test_unique_table_like.groovy           | 52 ++++++++++++++++++++++
 3 files changed, 86 insertions(+), 5 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
index ffded947bf..bf04888bf3 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
@@ -3087,7 +3087,7 @@ public class Catalog {
                     throw new DdlException("Table[" + table.getName() + "] is external, not support rollup copy");
                 }
 
-                Catalog.getDdlStmt(stmt, stmt.getDbName(), table, createTableStmt, null, null, false, false);
+                Catalog.getDdlStmt(stmt, stmt.getDbName(), table, createTableStmt, null, null, false, false, true);
                 if (createTableStmt.isEmpty()) {
                     ErrorReport.reportDdlException(ErrorCode.ERROR_CREATE_TABLE_LIKE_EMPTY, "CREATE");
                 }
@@ -4076,11 +4076,18 @@ public class Catalog {
 
     public static void getDdlStmt(Table table, List<String> createTableStmt, List<String> addPartitionStmt,
                                   List<String> createRollupStmt, boolean separatePartition, boolean hidePassword) {
-        getDdlStmt(null, null, table, createTableStmt, addPartitionStmt, createRollupStmt, separatePartition, hidePassword);
+        getDdlStmt(null, null, table, createTableStmt, addPartitionStmt, createRollupStmt,
+                separatePartition, hidePassword, false);
     }
 
-    public static void getDdlStmt(DdlStmt ddlStmt, String dbName, Table table, List<String> createTableStmt, List<String> addPartitionStmt,
-                                  List<String> createRollupStmt, boolean separatePartition, boolean hidePassword) {
+    /**
+     * Get table ddl stmt.
+     *
+     * @param getDdlForLike Get schema for 'create table like' or not. when true, without hidden columns.
+     */
+    public static void getDdlStmt(DdlStmt ddlStmt, String dbName, Table table, List<String> createTableStmt,
+                                  List<String> addPartitionStmt, List<String> createRollupStmt,
+                                  boolean separatePartition, boolean hidePassword, boolean getDdlForLike) {
         StringBuilder sb = new StringBuilder();
 
         // 1. create table
@@ -4105,7 +4112,14 @@ public class Catalog {
         }
         sb.append("`").append(table.getName()).append("` (\n");
         int idx = 0;
-        for (Column column : table.getBaseSchema()) {
+        List<Column> columns;
+        // when 'create table B like A', always return schema of A without hidden columns
+        if (getDdlForLike) {
+            columns = table.getBaseSchema(false);
+        } else {
+            columns = table.getBaseSchema();
+        }
+        for (Column column : columns) {
             if (idx++ != 0) {
                 sb.append(",\n");
             }
diff --git a/regression-test/data/data_model/unique/test_unique_table_like.out b/regression-test/data/data_model/unique/test_unique_table_like.out
new file mode 100644
index 0000000000..ffcbaf18cc
--- /dev/null
+++ b/regression-test/data/data_model/unique/test_unique_table_like.out
@@ -0,0 +1,15 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !desc_uniq_table --
+k	INT	Yes	true	\N	
+int_value	INT	Yes	false	\N	REPLACE
+char_value	CHAR(10)	Yes	false	\N	REPLACE
+date_value	DATE	Yes	false	\N	REPLACE
+__DORIS_DELETE_SIGN__	TINYINT	No	false	0	REPLACE
+
+-- !desc_uniq_table --
+k	INT	Yes	true	\N	
+int_value	INT	Yes	false	\N	REPLACE
+char_value	CHAR(10)	Yes	false	\N	REPLACE
+date_value	DATE	Yes	false	\N	REPLACE
+__DORIS_DELETE_SIGN__	TINYINT	No	false	0	REPLACE
+
diff --git a/regression-test/suites/data_model/unique/test_unique_table_like.groovy b/regression-test/suites/data_model/unique/test_unique_table_like.groovy
new file mode 100644
index 0000000000..6250b61138
--- /dev/null
+++ b/regression-test/suites/data_model/unique/test_unique_table_like.groovy
@@ -0,0 +1,52 @@
+// 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_unique_table_like", "data_model") {
+    def dbName = "test_unique_db"
+    List<List<Object>> db = sql "show databases like '${dbName}'"
+    if (db.size() == 0) {
+        sql "CREATE DATABASE  ${dbName}"
+    }
+    sql "use ${dbName}"
+
+    // test uniq table like 
+    def tbNameA = "test_uniq"
+    def tbNameB = "test_uniq_like"
+    sql "ADMIN SET FRONTEND CONFIG ('enable_batch_delete_by_default' = 'true')"
+    sql "SET show_hidden_columns=true"
+    sql "DROP TABLE IF EXISTS ${tbNameA}"
+    sql """
+            CREATE TABLE IF NOT EXISTS ${tbNameA} (
+                k int,
+                int_value int,
+                char_value char(10),
+                date_value date
+            )
+            ENGINE=OLAP
+            UNIQUE KEY(k)
+            DISTRIBUTED BY HASH(k) BUCKETS 5 properties("replication_num" = "1");
+        """
+    qt_desc_uniq_table "desc ${tbNameA}"    
+    sql """
+            CREATE TABLE IF NOT EXISTS ${tbNameB} LIKE ${tbNameA};
+        """
+    
+    qt_desc_uniq_table "desc ${tbNameB}"
+    sql "DROP TABLE ${tbNameA}"
+    sql "DROP TABLE ${tbNameB}"
+}
+


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 16/22: [bugfix]fix TableFunctionNode memory leak (#9853)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit fceff7f98b1e12ccbc9c3b319d5543d3e9a95e29
Author: Pxl <px...@qq.com>
AuthorDate: Tue May 31 19:20:22 2022 +0800

    [bugfix]fix TableFunctionNode memory leak (#9853)
---
 be/src/exec/table_function_node.cpp | 1 +
 be/src/util/block_compression.cpp   | 4 ++--
 2 files changed, 3 insertions(+), 2 deletions(-)

diff --git a/be/src/exec/table_function_node.cpp b/be/src/exec/table_function_node.cpp
index 1a06c0cb30..c2ed1f44a3 100644
--- a/be/src/exec/table_function_node.cpp
+++ b/be/src/exec/table_function_node.cpp
@@ -359,6 +359,7 @@ Status TableFunctionNode::close(RuntimeState* state) {
     }
     RETURN_IF_ERROR(exec_debug_action(TExecNodePhase::CLOSE));
     Expr::close(_fn_ctxs, state);
+    vectorized::VExpr::close(_vfn_ctxs, state);
 
     if (_num_rows_filtered_counter != nullptr) {
         COUNTER_SET(_num_rows_filtered_counter, static_cast<int64_t>(_num_rows_filtered));
diff --git a/be/src/util/block_compression.cpp b/be/src/util/block_compression.cpp
index 51f485f348..01c3734948 100644
--- a/be/src/util/block_compression.cpp
+++ b/be/src/util/block_compression.cpp
@@ -418,7 +418,7 @@ public:
 
     // follow ZSTD official example
     //  https://github.com/facebook/zstd/blob/dev/examples/streaming_compression.c
-    Status compress(const std::vector<Slice>& inputs, Slice* output) const {
+    Status compress(const std::vector<Slice>& inputs, Slice* output) const override {
         if (!ctx_c) return Status::InvalidArgument("compression context NOT initialized");
 
         // reset ctx to start new compress session
@@ -479,7 +479,7 @@ public:
 
     // follow ZSTD official example
     //  https://github.com/facebook/zstd/blob/dev/examples/streaming_decompression.c
-    Status decompress(const Slice& input, Slice* output) const {
+    Status decompress(const Slice& input, Slice* output) const override {
         if (!ctx_d) return Status::InvalidArgument("decompression context NOT initialized");
 
         // reset ctx to start a new decompress session


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 15/22: [Bugfix(Vec)] Close result_sink properly (#9849)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit fe32d8ffff7f4542ea46536c481adb4efe9b007b
Author: Amos Bird <am...@gmail.com>
AuthorDate: Mon May 30 19:03:33 2022 +0800

    [Bugfix(Vec)] Close result_sink properly (#9849)
    
    Close result_sink properly so that error code is reported and
    expr_context is always closed.
---
 be/src/vec/sink/result_sink.cpp | 17 ++++++++++-------
 1 file changed, 10 insertions(+), 7 deletions(-)

diff --git a/be/src/vec/sink/result_sink.cpp b/be/src/vec/sink/result_sink.cpp
index bc714470eb..e3db9fef32 100644
--- a/be/src/vec/sink/result_sink.cpp
+++ b/be/src/vec/sink/result_sink.cpp
@@ -103,21 +103,24 @@ Status VResultSink::send(RuntimeState* state, Block* block) {
 }
 
 Status VResultSink::close(RuntimeState* state, Status exec_status) {
-    if (_closed || _writer == nullptr|| _sender == nullptr) {
+    if (_closed) {
         return Status::OK();
     }
 
     Status final_status = exec_status;
-    // close the writer
-    Status st = _writer->close();
-    if (!st.ok() && exec_status.ok()) {
-        // close file writer failed, should return this error to client
-        final_status = st;
+
+    if (_writer) {
+        // close the writer
+        Status st = _writer->close();
+        if (!st.ok() && exec_status.ok()) {
+            // close file writer failed, should return this error to client
+            final_status = st;
+        }
     }
 
     // close sender, this is normal path end
     if (_sender) {
-        _sender->update_num_written_rows(_writer->get_written_rows());
+        if (_writer) _sender->update_num_written_rows(_writer->get_written_rows());
         _sender->close(final_status);
     }
     state->exec_env()->result_mgr()->cancel_at_time(


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 13/22: [feature] Support compression prop (#8923)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit f21002f767f0ce52e81e57984ca6c84f8b6fc884
Author: Lightman <31...@users.noreply.github.com>
AuthorDate: Fri May 27 21:52:05 2022 +0800

    [feature] Support compression prop (#8923)
---
 be/src/olap/rowset/segment_v2/column_writer.cpp    |  4 +-
 be/src/olap/rowset/segment_v2/segment_writer.cpp   | 10 +++--
 be/src/olap/rowset/segment_v2/segment_writer.h     |  3 +-
 be/src/olap/tablet_meta.cpp                        | 43 ++++++++++++++++++----
 be/src/olap/tablet_meta.h                          |  3 +-
 be/src/olap/tablet_schema.cpp                      |  2 +
 be/src/olap/tablet_schema.h                        |  3 ++
 .../java/org/apache/doris/alter/RollupJobV2.java   |  3 +-
 .../org/apache/doris/alter/SchemaChangeJobV2.java  |  3 +-
 .../java/org/apache/doris/backup/RestoreJob.java   |  3 +-
 .../java/org/apache/doris/catalog/Catalog.java     | 29 +++++++++++++--
 .../java/org/apache/doris/catalog/OlapTable.java   | 16 ++++++++
 .../org/apache/doris/catalog/TableProperty.java    | 16 +++++++-
 .../apache/doris/common/util/PropertyAnalyzer.java | 32 ++++++++++++++++
 .../org/apache/doris/master/ReportHandler.java     |  3 +-
 .../org/apache/doris/task/CreateReplicaTask.java   | 10 ++++-
 .../org/apache/doris/catalog/CreateTableTest.java  | 10 +++++
 .../java/org/apache/doris/task/AgentTaskTest.java  |  3 +-
 gensrc/proto/olap_file.proto                       |  2 +
 gensrc/thrift/AgentService.thrift                  | 31 ++++++++++++++++
 20 files changed, 202 insertions(+), 27 deletions(-)

diff --git a/be/src/olap/rowset/segment_v2/column_writer.cpp b/be/src/olap/rowset/segment_v2/column_writer.cpp
index bcf2b2a338..65696a816f 100644
--- a/be/src/olap/rowset/segment_v2/column_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/column_writer.cpp
@@ -120,7 +120,7 @@ Status ColumnWriter::create(const ColumnWriterOptions& opts, const TabletColumn*
             length_options.meta->set_is_nullable(false);
             length_options.meta->set_length(get_scalar_type_info<OLAP_FIELD_TYPE_UNSIGNED_INT>()->size());
             length_options.meta->set_encoding(DEFAULT_ENCODING);
-            length_options.meta->set_compression(LZ4F);
+            length_options.meta->set_compression(opts.meta->compression());
 
             length_options.need_zone_map = false;
             length_options.need_bloom_filter = false;
@@ -147,7 +147,7 @@ Status ColumnWriter::create(const ColumnWriterOptions& opts, const TabletColumn*
                 null_options.meta->set_is_nullable(false);
                 null_options.meta->set_length(get_scalar_type_info<OLAP_FIELD_TYPE_TINYINT>()->size());
                 null_options.meta->set_encoding(DEFAULT_ENCODING);
-                null_options.meta->set_compression(LZ4F);
+                null_options.meta->set_compression(opts.meta->compression());
 
                 null_options.need_zone_map = false;
                 null_options.need_bloom_filter = false;
diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp
index 669c770655..896c5e793c 100644
--- a/be/src/olap/rowset/segment_v2/segment_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp
@@ -51,17 +51,19 @@ SegmentWriter::~SegmentWriter() {
 };
 
 void SegmentWriter::init_column_meta(ColumnMetaPB* meta, uint32_t* column_id,
-                                      const TabletColumn& column) {
+                                     const TabletColumn& column,
+                                     const TabletSchema* tablet_schema) {
     // TODO(zc): Do we need this column_id??
     meta->set_column_id((*column_id)++);
     meta->set_unique_id(column.unique_id());
     meta->set_type(column.type());
     meta->set_length(column.length());
     meta->set_encoding(DEFAULT_ENCODING);
-    meta->set_compression(LZ4F);
+    meta->set_compression(tablet_schema->compression_type());
     meta->set_is_nullable(column.is_nullable());
     for (uint32_t i = 0; i < column.get_subtype_count(); ++i) {
-        init_column_meta(meta->add_children_columns(), column_id, column.get_sub_column(i));
+        init_column_meta(meta->add_children_columns(), column_id, column.get_sub_column(i),
+                         tablet_schema);
     }
 }
 
@@ -72,7 +74,7 @@ Status SegmentWriter::init(uint32_t write_mbytes_per_sec __attribute__((unused))
         ColumnWriterOptions opts;
         opts.meta = _footer.add_columns();
 
-        init_column_meta(opts.meta, &column_id, column);
+        init_column_meta(opts.meta, &column_id, column, _tablet_schema);
 
         // now we create zone map for key columns in AGG_KEYS or all column in UNIQUE_KEYS or DUP_KEYS
         // and not support zone map for array type.
diff --git a/be/src/olap/rowset/segment_v2/segment_writer.h b/be/src/olap/rowset/segment_v2/segment_writer.h
index ebba30fe70..ad3b608dd6 100644
--- a/be/src/olap/rowset/segment_v2/segment_writer.h
+++ b/be/src/olap/rowset/segment_v2/segment_writer.h
@@ -68,7 +68,8 @@ public:
 
     Status finalize(uint64_t* segment_file_size, uint64_t* index_size);
 
-    static void init_column_meta(ColumnMetaPB* meta, uint32_t* column_id, const TabletColumn& column);
+    static void init_column_meta(ColumnMetaPB* meta, uint32_t* column_id,
+                                 const TabletColumn& column, const TabletSchema* tablet_schema);
 
 private:
     DISALLOW_COPY_AND_ASSIGN(SegmentWriter);
diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp
index 5ad3aa71a7..c4696d10c8 100644
--- a/be/src/olap/tablet_meta.cpp
+++ b/be/src/olap/tablet_meta.cpp
@@ -43,7 +43,7 @@ OLAPStatus TabletMeta::create(const TCreateTabletReq& request, const TabletUid&
             request.tablet_schema.schema_hash, shard_id, request.tablet_schema, next_unique_id,
             col_ordinal_to_unique_id, tablet_uid,
             request.__isset.tablet_type ? request.tablet_type : TTabletType::TABLET_TYPE_DISK,
-            request.storage_medium));
+            request.storage_medium, request.compression_type));
     return OLAP_SUCCESS;
 }
 
@@ -54,7 +54,8 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id
                        uint32_t next_unique_id,
                        const std::unordered_map<uint32_t, uint32_t>& col_ordinal_to_unique_id,
                        TabletUid tablet_uid, TTabletType::type tabletType,
-                       TStorageMedium::type t_storage_medium)
+                       TStorageMedium::type t_storage_medium,
+                       TCompressionType::type compression_type)
         : _tablet_uid(0, 0), _schema(new TabletSchema) {
     TabletMetaPB tablet_meta_pb;
     tablet_meta_pb.set_table_id(table_id);
@@ -88,14 +89,40 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id
         LOG(WARNING) << "unknown tablet keys type";
         break;
     }
+    // compress_kind used to compress segment files
     schema->set_compress_kind(COMPRESS_LZ4);
 
-    switch(tablet_schema.sort_type) {
-        case TSortType::type::ZORDER:
-            schema->set_sort_type(SortType::ZORDER);
-            break;
-        default:
-            schema->set_sort_type(SortType::LEXICAL);
+    // compression_type used to compress segment page
+    switch (compression_type) {
+    case TCompressionType::NO_COMPRESSION:
+        schema->set_compression_type(NO_COMPRESSION);
+        break;
+    case TCompressionType::SNAPPY:
+        schema->set_compression_type(SNAPPY);
+        break;
+    case TCompressionType::LZ4:
+        schema->set_compression_type(LZ4);
+        break;
+    case TCompressionType::LZ4F:
+        schema->set_compression_type(LZ4F);
+        break;
+    case TCompressionType::ZLIB:
+        schema->set_compression_type(ZLIB);
+        break;
+    case TCompressionType::ZSTD:
+        schema->set_compression_type(ZSTD);
+        break;
+    default:
+        schema->set_compression_type(LZ4F);
+        break;
+    }
+
+    switch (tablet_schema.sort_type) {
+    case TSortType::type::ZORDER:
+        schema->set_sort_type(SortType::ZORDER);
+        break;
+    default:
+        schema->set_sort_type(SortType::LEXICAL);
     }
     schema->set_sort_col_num(tablet_schema.sort_col_num);
     tablet_meta_pb.set_in_restore_mode(false);
diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h
index b43e7d0727..5654fee29e 100644
--- a/be/src/olap/tablet_meta.h
+++ b/be/src/olap/tablet_meta.h
@@ -83,7 +83,8 @@ public:
                uint64_t shard_id, const TTabletSchema& tablet_schema, uint32_t next_unique_id,
                const std::unordered_map<uint32_t, uint32_t>& col_ordinal_to_unique_id,
                TabletUid tablet_uid, TTabletType::type tabletType,
-               TStorageMedium::type t_storage_medium);
+               TStorageMedium::type t_storage_medium,
+               TCompressionType::type compression_type);
     // If need add a filed in TableMeta, filed init copy in copy construct function
     TabletMeta(const TabletMeta& tablet_meta);
     TabletMeta(TabletMeta&& tablet_meta) = delete;
diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp
index 782ae1a7ac..8f67d7f512 100644
--- a/be/src/olap/tablet_schema.cpp
+++ b/be/src/olap/tablet_schema.cpp
@@ -426,6 +426,7 @@ void TabletSchema::init_from_pb(const TabletSchemaPB& schema) {
     _sequence_col_idx = schema.sequence_col_idx();
     _sort_type = schema.sort_type();
     _sort_col_num = schema.sort_col_num();
+    _compression_type = schema.compression_type();
 }
 
 void TabletSchema::to_schema_pb(TabletSchemaPB* tablet_meta_pb) {
@@ -446,6 +447,7 @@ void TabletSchema::to_schema_pb(TabletSchemaPB* tablet_meta_pb) {
     tablet_meta_pb->set_sequence_col_idx(_sequence_col_idx);
     tablet_meta_pb->set_sort_type(_sort_type);
     tablet_meta_pb->set_sort_col_num(_sort_col_num);
+    tablet_meta_pb->set_compression_type(_compression_type);
 }
 
 uint32_t TabletSchema::mem_size() const {
diff --git a/be/src/olap/tablet_schema.h b/be/src/olap/tablet_schema.h
index f9bd519cc1..4fbdf3b2e0 100644
--- a/be/src/olap/tablet_schema.h
+++ b/be/src/olap/tablet_schema.h
@@ -21,6 +21,7 @@
 #include <vector>
 
 #include "gen_cpp/olap_file.pb.h"
+#include "gen_cpp/segment_v2.pb.h"
 #include "olap/olap_define.h"
 #include "olap/types.h"
 
@@ -143,6 +144,7 @@ public:
     inline void set_delete_sign_idx(int32_t delete_sign_idx) { _delete_sign_idx = delete_sign_idx; }
     inline bool has_sequence_col() const { return _sequence_col_idx != -1; }
     inline int32_t sequence_col_idx() const { return _sequence_col_idx; }
+    inline segment_v2::CompressionTypePB compression_type() const { return _compression_type; }
     vectorized::Block create_block(const std::vector<uint32_t>& return_columns,
             const std::unordered_set<uint32_t>* tablet_columns_need_convert_null = nullptr) const;
 
@@ -165,6 +167,7 @@ private:
     size_t _num_short_key_columns = 0;
     size_t _num_rows_per_row_block = 0;
     CompressKind _compress_kind = COMPRESS_NONE;
+    segment_v2::CompressionTypePB _compression_type = segment_v2::CompressionTypePB::LZ4F;
     size_t _next_column_unique_id = 0;
 
     bool _has_bf_fpp = false;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
index a328fb0400..6c1ea1148d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
@@ -232,7 +232,8 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
                                 rollupSchema, tbl.getCopiedBfColumns(), tbl.getBfFpp(), countDownLatch,
                                 tbl.getCopiedIndexes(),
                                 tbl.isInMemory(),
-                                tabletType);
+                                tabletType,
+                                tbl.getCompressionType());
                         createReplicaTask.setBaseTablet(tabletIdMap.get(rollupTabletId), baseSchemaHash);
                         if (this.storageFormat != null) {
                             createReplicaTask.setStorageFormat(this.storageFormat);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
index 856df527ea..897ac2cd4b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
@@ -254,7 +254,8 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
                                     originKeysType, TStorageType.COLUMN, storageMedium,
                                     shadowSchema, bfColumns, bfFpp, countDownLatch, indexes,
                                     tbl.isInMemory(),
-                                    tbl.getPartitionInfo().getTabletType(partitionId));
+                                    tbl.getPartitionInfo().getTabletType(partitionId),
+                                    tbl.getCompressionType());
                             createReplicaTask.setBaseTablet(partitionIndexTabletMap.get(partitionId, shadowIdxId).get(shadowTabletId), originSchemaHash);
                             if (this.storageFormat != null) {
                                 createReplicaTask.setStorageFormat(this.storageFormat);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
index f29ab14be1..b30e13d52f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
@@ -963,7 +963,8 @@ public class RestoreJob extends AbstractJob {
                             indexMeta.getSchema(), bfColumns, bfFpp, null,
                             localTbl.getCopiedIndexes(),
                             localTbl.isInMemory(),
-                            localTbl.getPartitionInfo().getTabletType(restorePart.getId()));
+                            localTbl.getPartitionInfo().getTabletType(restorePart.getId()),
+                            localTbl.getCompressionType());
                     task.setInRestoreMode(true);
                     batchTask.addTask(task);
                 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
index bf04888bf3..7685327ef1 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
@@ -241,6 +241,7 @@ import org.apache.doris.task.CreateReplicaTask;
 import org.apache.doris.task.DropReplicaTask;
 import org.apache.doris.task.MasterTaskExecutor;
 import org.apache.doris.thrift.BackendService;
+import org.apache.doris.thrift.TCompressionType;
 import org.apache.doris.thrift.TNetworkAddress;
 import org.apache.doris.thrift.TStorageFormat;
 import org.apache.doris.thrift.TStorageMedium;
@@ -3003,6 +3004,7 @@ public class Catalog {
      *     6.2. replicationNum
      *     6.3. inMemory
      *     6.4. storageFormat
+     *     6.5. compressionType
      * 7. set index meta
      * 8. check colocation properties
      * 9. create tablet in BE
@@ -3281,6 +3283,7 @@ public class Catalog {
                     singlePartitionDesc.isInMemory(),
                     olapTable.getStorageFormat(),
                     singlePartitionDesc.getTabletType(),
+                    olapTable.getCompressionType(),
                     olapTable.getDataSortInfo()
             );
 
@@ -3512,6 +3515,7 @@ public class Catalog {
                                                  boolean isInMemory,
                                                  TStorageFormat storageFormat,
                                                  TTabletType tabletType,
+                                                 TCompressionType compressionType,
                                                  DataSortInfo dataSortInfo) throws DdlException {
         // create base index first.
         Preconditions.checkArgument(baseIndexId != -1);
@@ -3579,7 +3583,8 @@ public class Catalog {
                             indexes,
                             isInMemory,
                             tabletType,
-                            dataSortInfo);
+                            dataSortInfo,
+                            compressionType);
                     task.setStorageFormat(storageFormat);
                     batchTask.addTask(task);
                     // add to AgentTaskQueue for handling finish report.
@@ -3697,6 +3702,15 @@ public class Catalog {
         }
         olapTable.setStorageFormat(storageFormat);
 
+        // get compression type
+        TCompressionType compressionType = TCompressionType.LZ4;
+        try {
+            compressionType = PropertyAnalyzer.analyzeCompressionType(properties);
+        } catch (AnalysisException e) {
+            throw new DdlException(e.getMessage());
+        }
+        olapTable.setCompressionType(compressionType);
+
         // check data sort properties
         DataSortInfo dataSortInfo = PropertyAnalyzer.analyzeDataSortInfo(properties, keysType,
                 keysDesc.keysColumnSize(), storageFormat);
@@ -3741,6 +3755,7 @@ public class Catalog {
             throw new DdlException(e.getMessage());
         }
 
+
         if (partitionInfo.getType() == PartitionType.UNPARTITIONED) {
             // if this is an unpartitioned table, we should analyze data property and replication num here.
             // if this is a partitioned table, there properties are already analyzed in RangePartitionDesc analyze phase.
@@ -3877,7 +3892,7 @@ public class Catalog {
                         partitionInfo.getReplicaAllocation(partitionId),
                         versionInfo, bfColumns, bfFpp,
                         tabletIdSet, olapTable.getCopiedIndexes(),
-                        isInMemory, storageFormat, tabletType, olapTable.getDataSortInfo());
+                        isInMemory, storageFormat, tabletType, compressionType, olapTable.getDataSortInfo());
                 olapTable.addPartition(partition);
             } else if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) {
                 try {
@@ -3928,7 +3943,8 @@ public class Catalog {
                             versionInfo, bfColumns, bfFpp,
                             tabletIdSet, olapTable.getCopiedIndexes(),
                             isInMemory, storageFormat,
-                            partitionInfo.getTabletType(entry.getValue()), olapTable.getDataSortInfo());
+                            partitionInfo.getTabletType(entry.getValue()),
+                            compressionType, olapTable.getDataSortInfo());
                     olapTable.addPartition(partition);
                 }
             } else {
@@ -4272,6 +4288,12 @@ public class Catalog {
             sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_STORAGE_FORMAT).append("\" = \"");
             sb.append(olapTable.getStorageFormat()).append("\"");
 
+            // compression type
+            if (olapTable.getCompressionType() != TCompressionType.LZ4F) {
+                sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_COMPRESSION).append("\" = \"");
+                sb.append(olapTable.getCompressionType()).append("\"");
+            }
+
             sb.append("\n)");
         } else if (table.getType() == TableType.MYSQL) {
             MysqlTable mysqlTable = (MysqlTable) table;
@@ -6800,6 +6822,7 @@ public class Catalog {
                         copiedTbl.isInMemory(),
                         copiedTbl.getStorageFormat(),
                         copiedTbl.getPartitionInfo().getTabletType(oldPartitionId),
+                        copiedTbl.getCompressionType(),
                         copiedTbl.getDataSortInfo());
                 newPartitions.add(newPartition);
             }
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 253d53521e..77e22eb118 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
@@ -48,6 +48,7 @@ import org.apache.doris.qe.OriginStatement;
 import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
+import org.apache.doris.thrift.TCompressionType;
 import org.apache.doris.thrift.TOlapTable;
 import org.apache.doris.thrift.TSortType;
 import org.apache.doris.thrift.TStorageFormat;
@@ -1672,6 +1673,14 @@ public class OlapTable extends Table {
         return !tempPartitions.isEmpty();
     }
 
+    public void setCompressionType(TCompressionType compressionType) {
+        if (tableProperty == null) {
+            tableProperty = new TableProperty(new HashMap<>());
+        }
+        tableProperty.modifyTableProperties(PropertyAnalyzer.PROPERTIES_COMPRESSION, compressionType.name());
+        tableProperty.buildCompressionType();
+    }
+
     public void setStorageFormat(TStorageFormat storageFormat) {
         if (tableProperty == null) {
             tableProperty = new TableProperty(new HashMap<>());
@@ -1687,6 +1696,13 @@ public class OlapTable extends Table {
         return tableProperty.getStorageFormat();
     }
 
+    public TCompressionType getCompressionType() {
+        if (tableProperty == null) {
+            return TCompressionType.LZ4F;
+        }
+        return tableProperty.getCompressionType();
+    }
+
     public DataSortInfo getDataSortInfo() {
         if (tableProperty == null) {
             return new DataSortInfo(TSortType.LEXICAL, this.getKeysNum());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
index 844e4032c3..91ef9b0c32 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
@@ -25,6 +25,7 @@ import org.apache.doris.common.io.Writable;
 import org.apache.doris.common.util.PropertyAnalyzer;
 import org.apache.doris.persist.OperationType;
 import org.apache.doris.persist.gson.GsonUtils;
+import org.apache.doris.thrift.TCompressionType;
 import org.apache.doris.thrift.TStorageFormat;
 
 import com.google.common.base.Strings;
@@ -68,6 +69,8 @@ public class TableProperty implements Writable {
      */
     private TStorageFormat storageFormat = TStorageFormat.DEFAULT;
 
+    private TCompressionType compressionType = TCompressionType.LZ4F;
+
     private DataSortInfo dataSortInfo = new DataSortInfo();
 
     public TableProperty(Map<String, String> properties) {
@@ -145,6 +148,12 @@ public class TableProperty implements Writable {
         return this;
     }
 
+    public TableProperty buildCompressionType() {
+        compressionType = TCompressionType.valueOf(properties.getOrDefault(PropertyAnalyzer.PROPERTIES_COMPRESSION,
+                TCompressionType.LZ4F.name()));
+        return this;
+    }
+
     public TableProperty buildStorageFormat() {
         storageFormat = TStorageFormat.valueOf(properties.getOrDefault(PropertyAnalyzer.PROPERTIES_STORAGE_FORMAT,
                 TStorageFormat.DEFAULT.name()));
@@ -210,6 +219,10 @@ public class TableProperty implements Writable {
         return dataSortInfo;
     }
 
+    public TCompressionType getCompressionType() {
+        return compressionType;
+    }
+
     public void buildReplicaAllocation() {
         try {
             // Must copy the properties because "analyzeReplicaAllocation" with remove the property
@@ -233,7 +246,8 @@ public class TableProperty implements Writable {
                 .executeBuildDynamicProperty()
                 .buildInMemory()
                 .buildStorageFormat()
-                .buildDataSortInfo();
+                .buildDataSortInfo()
+                .buildCompressionType();
         if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) {
             // get replica num from property map and create replica allocation
             String repNum = tableProperty.properties.remove(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
index a4684f09c9..c4e186c4e1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
@@ -30,6 +30,8 @@ import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
 import org.apache.doris.resource.Tag;
+import org.apache.doris.thrift.TCompressionType;
+import org.apache.doris.thrift.TSortType;
 import org.apache.doris.thrift.TStorageFormat;
 import org.apache.doris.thrift.TStorageMedium;
 import org.apache.doris.thrift.TStorageType;
@@ -73,6 +75,7 @@ public class PropertyAnalyzer {
     public static final String PROPERTIES_COLOCATE_WITH = "colocate_with";
     
     public static final String PROPERTIES_TIMEOUT = "timeout";
+    public static final String PROPERTIES_COMPRESSION = "compression";
 
     public static final String PROPERTIES_DISTRIBUTION_TYPE = "distribution_type";
     public static final String PROPERTIES_SEND_CLEAR_ALTER_TASK = "send_clear_alter_tasks";
@@ -384,6 +387,35 @@ public class PropertyAnalyzer {
         return timeout;
     }
 
+    // analyzeCompressionType will parse the compression type from properties
+    public static TCompressionType analyzeCompressionType(Map<String, String> properties) throws  AnalysisException {
+        String compressionType = "";
+        if (properties != null && properties.containsKey(PROPERTIES_COMPRESSION)) {
+            compressionType = properties.get(PROPERTIES_COMPRESSION);
+            properties.remove(PROPERTIES_COMPRESSION);
+        } else {
+            return TCompressionType.LZ4F;
+        }
+
+        if (compressionType.equalsIgnoreCase("no_compression")) {
+            return TCompressionType.NO_COMPRESSION;
+        } else if (compressionType.equalsIgnoreCase("lz4")) {
+            return TCompressionType.LZ4;
+        } else if (compressionType.equalsIgnoreCase("lz4f")) {
+            return TCompressionType.LZ4F;
+        } else if (compressionType.equalsIgnoreCase("zlib")) {
+            return TCompressionType.ZLIB;
+        } else if (compressionType.equalsIgnoreCase("zstd")) {
+            return TCompressionType.ZSTD;
+        } else if (compressionType.equalsIgnoreCase("snappy")) {
+            return TCompressionType.SNAPPY;
+        } else if (compressionType.equalsIgnoreCase("default_compression")) {
+            return TCompressionType.LZ4F;
+        } else {
+            throw new AnalysisException("unknown compression type: " + compressionType);
+        }
+    }
+
     // analyzeStorageFormat will parse the storage format from properties
     // sql: alter table tablet_name set ("storage_format" = "v2")
     // Use this sql to convert all tablets(base and rollup index) to a new format segment
diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
index 05643aa7e2..d875fecbe6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
@@ -603,7 +603,8 @@ public class ReportHandler extends Daemon {
                                             TStorageMedium.HDD, indexMeta.getSchema(), bfColumns, bfFpp, null,
                                             olapTable.getCopiedIndexes(),
                                             olapTable.isInMemory(),
-                                            olapTable.getPartitionInfo().getTabletType(partitionId));
+                                            olapTable.getPartitionInfo().getTabletType(partitionId),
+                                            olapTable.getCompressionType());
                                     createReplicaTask.setIsRecoverTask(true);
                                     createReplicaBatchTask.addTask(createReplicaTask);
                                 } else {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java
index 7ecf25979d..5f59d5ff0d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java
@@ -25,6 +25,7 @@ import org.apache.doris.catalog.KeysType;
 import org.apache.doris.common.MarkedCountDownLatch;
 import org.apache.doris.common.Status;
 import org.apache.doris.thrift.TColumn;
+import org.apache.doris.thrift.TCompressionType;
 import org.apache.doris.thrift.TCreateTabletReq;
 import org.apache.doris.thrift.TOlapTableIndex;
 import org.apache.doris.thrift.TStatusCode;
@@ -54,6 +55,7 @@ public class CreateReplicaTask extends AgentTask {
     private KeysType keysType;
     private TStorageType storageType;
     private TStorageMedium storageMedium;
+    private TCompressionType compressionType;
 
     private List<Column> columns;
 
@@ -93,7 +95,7 @@ public class CreateReplicaTask extends AgentTask {
                              Set<String> bfColumns, double bfFpp, MarkedCountDownLatch<Long, Long> latch,
                              List<Index> indexes,
                              boolean isInMemory,
-                             TTabletType tabletType) {
+                             TTabletType tabletType, TCompressionType compressionType) {
         super(null, backendId, TTaskType.CREATE, dbId, tableId, partitionId, indexId, tabletId);
 
         this.shortKeyColumnCount = shortKeyColumnCount;
@@ -104,6 +106,7 @@ public class CreateReplicaTask extends AgentTask {
         this.keysType = keysType;
         this.storageType = storageType;
         this.storageMedium = storageMedium;
+        this.compressionType = compressionType;
 
         this.columns = columns;
 
@@ -125,7 +128,8 @@ public class CreateReplicaTask extends AgentTask {
                              List<Index> indexes,
                              boolean isInMemory,
                              TTabletType tabletType,
-                             DataSortInfo dataSortInfo) {
+                             DataSortInfo dataSortInfo,
+                             TCompressionType compressionType) {
         super(null, backendId, TTaskType.CREATE, dbId, tableId, partitionId, indexId, tabletId);
 
         this.shortKeyColumnCount = shortKeyColumnCount;
@@ -136,6 +140,7 @@ public class CreateReplicaTask extends AgentTask {
         this.keysType = keysType;
         this.storageType = storageType;
         this.storageMedium = storageMedium;
+        this.compressionType = compressionType;
 
         this.columns = columns;
 
@@ -267,6 +272,7 @@ public class CreateReplicaTask extends AgentTask {
         }
 
         createTabletReq.setTabletType(tabletType);
+        createTabletReq.setCompressionType(compressionType);
         return createTabletReq;
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
index e4f97607d9..082a05cf40 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
@@ -138,6 +138,16 @@ public class CreateTableTest {
                 .expectThrowsNoException(() -> createTable("create table test.tb7(key1 int, key2 varchar(10)) \n"
                         + "distributed by hash(key1) buckets 1 properties('replication_num' = '1', 'storage_medium' = 'ssd');"));
 
+        ExceptionChecker
+                .expectThrowsNoException(() -> createTable("create table test.compression1(key1 int, key2 varchar(10)) \n"
+                        + "distributed by hash(key1) buckets 1 \n"
+                        + "properties('replication_num' = '1', 'compression' = 'lz4f');"));
+
+        ExceptionChecker
+                .expectThrowsNoException(() -> createTable("create table test.compression2(key1 int, key2 varchar(10)) \n"
+                        + "distributed by hash(key1) buckets 1 \n"
+                        + "properties('replication_num' = '1', 'compression' = 'snappy');"));
+
         ExceptionChecker
                 .expectThrowsNoException(() -> createTable("create table test.tbl8\n" + "(k1 varchar(40), k2 int, v1 int)\n"
                         + "unique key(k1, k2)\n"
diff --git a/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java
index ca31e8de72..e52bcedce1 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java
@@ -28,6 +28,7 @@ import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.MarkedCountDownLatch;
 import org.apache.doris.thrift.TAgentTaskRequest;
 import org.apache.doris.thrift.TBackend;
+import org.apache.doris.thrift.TCompressionType;
 import org.apache.doris.thrift.TPriority;
 import org.apache.doris.thrift.TPushType;
 import org.apache.doris.thrift.TStorageMedium;
@@ -110,7 +111,7 @@ public class AgentTaskTest {
                                                   version, KeysType.AGG_KEYS,
                                                   storageType, TStorageMedium.SSD,
                                                   columns, null, 0, latch, null,
-                                                  false, TTabletType.TABLET_TYPE_DISK);
+                                                  false, TTabletType.TABLET_TYPE_DISK, TCompressionType.LZ4F);
 
         // drop
         dropTask = new DropReplicaTask(backendId1, tabletId1, schemaHash1);
diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto
index 026a99ef62..43cb8ac7a0 100644
--- a/gensrc/proto/olap_file.proto
+++ b/gensrc/proto/olap_file.proto
@@ -23,6 +23,7 @@ option java_package = "org.apache.doris.proto";
 
 import "olap_common.proto";
 import "types.proto";
+import "segment_v2.proto";
 
 message ZoneMap {
     required bytes min = 1;
@@ -194,6 +195,7 @@ message TabletSchemaPB {
     optional int32 sequence_col_idx = 10 [default= -1];
     optional SortType sort_type = 11;
     optional int32 sort_col_num = 12;
+    optional segment_v2.CompressionTypePB compression_type = 13;
 }
 
 enum TabletStatePB {
diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift
index aba8cc9580..62855abfe1 100644
--- a/gensrc/thrift/AgentService.thrift
+++ b/gensrc/thrift/AgentService.thrift
@@ -67,6 +67,35 @@ enum TTabletType {
     TABLET_TYPE_MEMORY = 1
 }
 
+struct TS3StorageParam {
+    1: optional string s3_endpoint
+    2: optional string s3_region
+    3: optional string s3_ak
+    4: optional string s3_sk
+    5: optional i32 s3_max_conn = 50
+    6: optional i32 s3_request_timeout_ms = 3000
+    7: optional i32 s3_conn_timeout_ms = 1000
+    8: optional string root_path
+}
+
+struct TStorageParam {
+    1: required Types.TStorageMedium storage_medium = TStorageMedium.HDD
+    2: required string storage_name = "";
+    3: optional TS3StorageParam s3_storage_param
+}
+
+enum TCompressionType {
+    UNKNOWN_COMPRESSION = 0,
+    DEFAULT_COMPRESSION = 1,
+    NO_COMPRESSION = 2,
+    SNAPPY = 3,
+    LZ4 = 4,
+    LZ4F = 5,
+    ZLIB = 6,
+    ZSTD = 7
+}
+
+
 struct TCreateTabletReq {
     1: required Types.TTabletId tablet_id
     2: required TTabletSchema tablet_schema
@@ -87,6 +116,8 @@ struct TCreateTabletReq {
     12: optional bool is_eco_mode
     13: optional TStorageFormat storage_format
     14: optional TTabletType tablet_type
+    15: optional TStorageParam storage_param
+    16: optional TCompressionType compression_type = TCompressionType.LZ4F
 }
 
 struct TDropTabletReq {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 07/22: [fix](sql-block-rule) sql block rule NPE (#9778)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit e57aa42fa7de132ee0bfd58306b42058a684bc46
Author: morningman <mo...@163.com>
AuthorDate: Wed Jun 1 18:21:54 2022 +0800

    [fix](sql-block-rule) sql block rule NPE (#9778)
---
 .../org/apache/doris/blockrule/SqlBlockRule.java   |  37 +-
 .../apache/doris/blockrule/SqlBlockRuleMgr.java    |  84 +++--
 .../java/org/apache/doris/qe/StmtExecutor.java     |   7 +-
 .../doris/blockrule/SqlBlockRuleMgrTest.java       | 382 ---------------------
 4 files changed, 92 insertions(+), 418 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRule.java b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRule.java
index 97304715dc..5601415ab3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRule.java
@@ -21,6 +21,7 @@ import org.apache.doris.analysis.AlterSqlBlockRuleStmt;
 import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
+import org.apache.doris.common.util.SqlBlockUtil;
 import org.apache.doris.persist.gson.GsonUtils;
 
 import com.google.common.collect.Lists;
@@ -34,12 +35,13 @@ import java.io.IOException;
 import java.util.List;
 import java.util.regex.Pattern;
 
+/**
+ * Use for block some sql by rule.
+ **/
 public class SqlBlockRule implements Writable {
 
     public static final String NAME_TYPE = "SQL BLOCK RULE NAME";
 
-    public static final String DEFAULT_USER = "default";
-
     // the rule name, cluster unique
     @SerializedName(value = "name")
     private String name;
@@ -73,11 +75,11 @@ public class SqlBlockRule implements Writable {
 
     private Pattern sqlPattern;
 
-    public SqlBlockRule(String name) {
-        this.name = name;
-    }
-
-    public SqlBlockRule(String name, String sql, String sqlHash, Long partitionNum, Long tabletNum, Long cardinality, Boolean global, Boolean enable) {
+    /**
+     * Create SqlBlockRule.
+     **/
+    public SqlBlockRule(String name, String sql, String sqlHash, Long partitionNum, Long tabletNum, Long cardinality,
+            Boolean global, Boolean enable) {
         this.name = name;
         this.sql = sql;
         this.sqlHash = sqlHash;
@@ -92,11 +94,13 @@ public class SqlBlockRule implements Writable {
     }
 
     public static SqlBlockRule fromCreateStmt(CreateSqlBlockRuleStmt stmt) {
-        return new SqlBlockRule(stmt.getRuleName(), stmt.getSql(), stmt.getSqlHash(), stmt.getPartitionNum(), stmt.getTabletNum(), stmt.getCardinality(), stmt.isGlobal(), stmt.isEnable());
+        return new SqlBlockRule(stmt.getRuleName(), stmt.getSql(), stmt.getSqlHash(), stmt.getPartitionNum(),
+                stmt.getTabletNum(), stmt.getCardinality(), stmt.isGlobal(), stmt.isEnable());
     }
 
     public static SqlBlockRule fromAlterStmt(AlterSqlBlockRuleStmt stmt) {
-        return new SqlBlockRule(stmt.getRuleName(), stmt.getSql(), stmt.getSqlHash(), stmt.getPartitionNum(), stmt.getTabletNum(), stmt.getCardinality(), stmt.getGlobal(), stmt.getEnable());
+        return new SqlBlockRule(stmt.getRuleName(), stmt.getSql(), stmt.getSqlHash(), stmt.getPartitionNum(),
+                stmt.getTabletNum(), stmt.getCardinality(), stmt.getGlobal(), stmt.getEnable());
     }
 
     public String getName() {
@@ -167,12 +171,15 @@ public class SqlBlockRule implements Writable {
         this.enable = enable;
     }
 
+    /**
+     * Show SqlBlockRule info.
+     **/
     public List<String> getShowInfo() {
         return Lists.newArrayList(this.name, this.sql, this.sqlHash,
                 this.partitionNum == null ? "0" : Long.toString(this.partitionNum),
                 this.tabletNum == null ? "0" : Long.toString(this.tabletNum),
-                this.cardinality == null ? "0" : Long.toString(this.cardinality),
-                String.valueOf(this.global), String.valueOf(this.enable));
+                this.cardinality == null ? "0" : Long.toString(this.cardinality), String.valueOf(this.global),
+                String.valueOf(this.enable));
     }
 
     @Override
@@ -180,10 +187,16 @@ public class SqlBlockRule implements Writable {
         Text.writeString(out, GsonUtils.GSON.toJson(this));
     }
 
+    /**
+     * Read data from file.
+     **/
     public static SqlBlockRule read(DataInput in) throws IOException {
         String json = Text.readString(in);
         SqlBlockRule sqlBlockRule = GsonUtils.GSON.fromJson(json, SqlBlockRule.class);
-        sqlBlockRule.setSqlPattern(Pattern.compile(sqlBlockRule.getSql()));
+        if (StringUtils.isNotEmpty(sqlBlockRule.getSql()) && !SqlBlockUtil.STRING_DEFAULT.equals(
+                sqlBlockRule.getSql())) {
+            sqlBlockRule.setSqlPattern(Pattern.compile(sqlBlockRule.getSql()));
+        }
         return sqlBlockRule;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
index c1ddf633df..46ac87aaf9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java
@@ -47,6 +47,9 @@ import java.util.Map;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.stream.Collectors;
 
+/**
+ * Manage SqlBlockRule.
+ **/
 public class SqlBlockRuleMgr implements Writable {
     private static final Logger LOG = LogManager.getLogger(SqlBlockRuleMgr.class);
 
@@ -63,10 +66,16 @@ public class SqlBlockRuleMgr implements Writable {
         lock.writeLock().unlock();
     }
 
+    /**
+     * Judge whether exist rule by ruleName.
+     **/
     public boolean existRule(String name) {
         return nameToSqlBlockRuleMap.containsKey(name);
     }
 
+    /**
+     * Get SqlBlockRule by show stmt.
+     **/
     public List<SqlBlockRule> getSqlBlockRule(ShowSqlBlockRuleStmt stmt) throws AnalysisException {
         String ruleName = stmt.getRuleName();
         if (StringUtils.isNotEmpty(ruleName)) {
@@ -79,9 +88,11 @@ public class SqlBlockRuleMgr implements Writable {
         return Lists.newArrayList(nameToSqlBlockRuleMap.values());
     }
 
-    // check limitation's  effectiveness of a sql_block_rule
-    public static void verifyLimitations(SqlBlockRule sqlBlockRule) throws DdlException {
-        if (sqlBlockRule.getPartitionNum() < 0){
+    /**
+     * Check limitation's  effectiveness of a SqlBlockRule.
+     **/
+    private static void verifyLimitations(SqlBlockRule sqlBlockRule) throws DdlException {
+        if (sqlBlockRule.getPartitionNum() < 0) {
             throw new DdlException("the value of partition_num can't be a negative");
         }
         if (sqlBlockRule.getTabletNum() < 0){
@@ -92,6 +103,9 @@ public class SqlBlockRuleMgr implements Writable {
         }
     }
 
+    /**
+     * Create SqlBlockRule for create stmt.
+     **/
     public void createSqlBlockRule(CreateSqlBlockRuleStmt stmt) throws UserException {
         writeLock();
         try {
@@ -108,11 +122,17 @@ public class SqlBlockRuleMgr implements Writable {
         }
     }
 
+    /**
+     * Add local cache when receive editLog.
+     **/
     public void replayCreate(SqlBlockRule sqlBlockRule) {
         unprotectedAdd(sqlBlockRule);
         LOG.info("replay create sql block rule: {}", sqlBlockRule);
     }
 
+    /**
+     * Alter SqlBlockRule for alter stmt.
+     **/
     public void alterSqlBlockRule(AlterSqlBlockRuleStmt stmt) throws AnalysisException, DdlException {
         writeLock();
         try {
@@ -159,14 +179,17 @@ public class SqlBlockRuleMgr implements Writable {
         LOG.info("replay alter sql block rule: {}", sqlBlockRule);
     }
 
-    public void unprotectedUpdate(SqlBlockRule sqlBlockRule) {
+    private void unprotectedUpdate(SqlBlockRule sqlBlockRule) {
         nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
     }
 
-    public void unprotectedAdd(SqlBlockRule sqlBlockRule) {
+    private void unprotectedAdd(SqlBlockRule sqlBlockRule) {
         nameToSqlBlockRuleMap.put(sqlBlockRule.getName(), sqlBlockRule);
     }
 
+    /**
+     * Drop SqlBlockRule for drop stmt.
+     **/
     public void dropSqlBlockRule(DropSqlBlockRuleStmt stmt) throws DdlException {
         writeLock();
         try {
@@ -192,9 +215,13 @@ public class SqlBlockRuleMgr implements Writable {
         ruleNames.forEach(name -> nameToSqlBlockRuleMap.remove(name));
     }
 
+    /**
+     * Match SQL according to rules.
+     **/
     public void matchSql(String originSql, String sqlHash, String user) throws AnalysisException {
         // match global rule
-        List<SqlBlockRule> globalRules = nameToSqlBlockRuleMap.values().stream().filter(SqlBlockRule::getGlobal).collect(Collectors.toList());
+        List<SqlBlockRule> globalRules =
+                nameToSqlBlockRuleMap.values().stream().filter(SqlBlockRule::getGlobal).collect(Collectors.toList());
         for (SqlBlockRule rule : globalRules) {
             matchSql(rule, originSql, sqlHash);
         }
@@ -209,25 +236,30 @@ public class SqlBlockRuleMgr implements Writable {
         }
     }
 
-    public void matchSql(SqlBlockRule rule, String originSql, String sqlHash) throws AnalysisException {
+    private void matchSql(SqlBlockRule rule, String originSql, String sqlHash) throws AnalysisException {
         if (rule.getEnable()) {
-            if (StringUtils.isNotEmpty(rule.getSqlHash()) &&
-                    (!CreateSqlBlockRuleStmt.STRING_NOT_SET.equals(rule.getSqlHash()) && rule.getSqlHash().equals(sqlHash))) {
+            if (StringUtils.isNotEmpty(rule.getSqlHash()) && !SqlBlockUtil.STRING_DEFAULT.equals(rule.getSqlHash())
+                    && rule.getSqlHash().equals(sqlHash)) {
                 MetricRepo.COUNTER_HIT_SQL_BLOCK_RULE.increase(1L);
                 throw new AnalysisException("sql match hash sql block rule: " + rule.getName());
-            } else if (StringUtils.isNotEmpty(rule.getSql()) &&
-                    (!CreateSqlBlockRuleStmt.STRING_NOT_SET.equals(rule.getSql()) && rule.getSqlPattern().matcher(originSql).find())) {
+            } else if (StringUtils.isNotEmpty(rule.getSql()) && !SqlBlockUtil.STRING_DEFAULT.equals(rule.getSql())
+                    && rule.getSqlPattern() != null && rule.getSqlPattern().matcher(originSql).find()) {
                 MetricRepo.COUNTER_HIT_SQL_BLOCK_RULE.increase(1L);
                 throw new AnalysisException("sql match regex sql block rule: " + rule.getName());
             }
         }
     }
 
-    public void checkLimitaions(Long partitionNum, Long tabletNum, Long cardinality, String user) throws AnalysisException {
+    /**
+     * Check number whether legal by user.
+     **/
+    public void checkLimitations(Long partitionNum, Long tabletNum, Long cardinality, String user)
+            throws AnalysisException {
         // match global rule
-        List<SqlBlockRule> globalRules = nameToSqlBlockRuleMap.values().stream().filter(SqlBlockRule::getGlobal).collect(Collectors.toList());
+        List<SqlBlockRule> globalRules =
+                nameToSqlBlockRuleMap.values().stream().filter(SqlBlockRule::getGlobal).collect(Collectors.toList());
         for (SqlBlockRule rule : globalRules) {
-            checkLimitaions(rule, partitionNum, tabletNum, cardinality);
+            checkLimitations(rule, partitionNum, tabletNum, cardinality);
         }
         // match user rule
         String[] bindSqlBlockRules = Catalog.getCurrentCatalog().getAuth().getSqlBlockRules(user);
@@ -236,24 +268,32 @@ public class SqlBlockRuleMgr implements Writable {
             if (rule == null) {
                 continue;
             }
-            checkLimitaions(rule, partitionNum, tabletNum, cardinality);
+            checkLimitations(rule, partitionNum, tabletNum, cardinality);
         }
     }
 
-    public void checkLimitaions(SqlBlockRule rule, Long partitionNum, Long tabletNum, Long cardinality) throws AnalysisException {
+    /**
+     * Check number whether legal by SqlBlockRule.
+     **/
+    private void checkLimitations(SqlBlockRule rule, Long partitionNum, Long tabletNum, Long cardinality)
+            throws AnalysisException {
         if (rule.getPartitionNum() == 0 && rule.getTabletNum() == 0 && rule.getCardinality() == 0) {
             return;
         } else if (rule.getEnable()) {
-            if ((rule.getPartitionNum() != 0 && rule.getPartitionNum() < partitionNum)
-                    || (rule.getTabletNum() != 0 && rule.getTabletNum() < tabletNum)
-                    || (rule.getCardinality() != 0 && rule.getCardinality() < cardinality)) {
+            if ((rule.getPartitionNum() != 0 && rule.getPartitionNum() < partitionNum) || (rule.getTabletNum() != 0
+                    && rule.getTabletNum() < tabletNum) || (rule.getCardinality() != 0
+                    && rule.getCardinality() < cardinality)) {
                 MetricRepo.COUNTER_HIT_SQL_BLOCK_RULE.increase(1L);
                 if (rule.getPartitionNum() < partitionNum && rule.getPartitionNum() != 0) {
-                    throw new AnalysisException("sql hits sql block rule: " + rule.getName() + ", reach partition_num : " + rule.getPartitionNum());
+                    throw new AnalysisException(
+                            "sql hits sql block rule: " + rule.getName() + ", reach partition_num : "
+                                    + rule.getPartitionNum());
                 } else if (rule.getTabletNum() < tabletNum && rule.getTabletNum() != 0) {
-                    throw new AnalysisException("sql hits sql block rule: " + rule.getName() + ", reach tablet_num : " + rule.getTabletNum());
+                    throw new AnalysisException("sql hits sql block rule: " + rule.getName() + ", reach tablet_num : "
+                            + rule.getTabletNum());
                 } else if (rule.getCardinality() < cardinality && rule.getCardinality() != 0) {
-                    throw new AnalysisException("sql hits sql block rule: " + rule.getName() + ", reach cardinality : " + rule.getCardinality());
+                    throw new AnalysisException("sql hits sql block rule: " + rule.getName() + ", reach cardinality : "
+                            + rule.getCardinality());
                 }
             }
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
index 6ba7364f1a..ec36508e0f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
@@ -371,8 +371,11 @@ public class StmtExecutor implements ProfileWriter {
                     for (ScanNode scanNode : scanNodeList) {
                         if (scanNode instanceof OlapScanNode) {
                             OlapScanNode olapScanNode = (OlapScanNode) scanNode;
-                            Catalog.getCurrentCatalog().getSqlBlockRuleMgr().checkLimitaions(olapScanNode.getSelectedPartitionNum().longValue(),
-                                    olapScanNode.getSelectedTabletsNum(), olapScanNode.getCardinality(), analyzer.getQualifiedUser());
+                            Catalog.getCurrentCatalog().getSqlBlockRuleMgr().checkLimitations(
+                                    olapScanNode.getSelectedPartitionNum().longValue(),
+                                    olapScanNode.getSelectedTabletsNum(),
+                                    olapScanNode.getCardinality(),
+                                    context.getQualifiedUser());
                         }
                     }
                 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java
deleted file mode 100644
index bf93eae895..0000000000
--- a/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java
+++ /dev/null
@@ -1,382 +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.blockrule;
-
-import org.apache.doris.analysis.AlterSqlBlockRuleStmt;
-import org.apache.doris.analysis.Analyzer;
-import org.apache.doris.analysis.CreateDbStmt;
-import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
-import org.apache.doris.analysis.CreateTableStmt;
-import org.apache.doris.analysis.SetUserPropertyStmt;
-import org.apache.doris.analysis.ShowSqlBlockRuleStmt;
-import org.apache.doris.catalog.Catalog;
-import org.apache.doris.common.AnalysisException;
-import org.apache.doris.common.DdlException;
-import org.apache.doris.common.ExceptionChecker;
-import org.apache.doris.common.UserException;
-import org.apache.doris.common.jmockit.Deencapsulation;
-import org.apache.doris.metric.MetricRepo;
-import org.apache.doris.planner.OlapScanNode;
-import org.apache.doris.planner.Planner;
-import org.apache.doris.planner.ScanNode;
-import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.qe.StmtExecutor;
-import org.apache.doris.utframe.UtFrameUtils;
-
-import org.apache.commons.codec.digest.DigestUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.File;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.HashMap;
-import java.util.Map;
-
-public class SqlBlockRuleMgrTest {
-
-    private static String runningDir = "fe/mocked/SqlBlockRuleMgrTest/" + UUID.randomUUID().toString() + "/";
-    
-    private static ConnectContext connectContext;
-    
-    @BeforeClass
-    public static void beforeClass() throws Exception {
-        UtFrameUtils.createDorisCluster(runningDir);
-        
-        // create connect context
-        connectContext = UtFrameUtils.createDefaultCtx();
-        
-        // create database
-        String createDbStmtStr = "create database test;";
-        CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseAndAnalyzeStmt(createDbStmtStr, connectContext);
-        Catalog.getCurrentCatalog().createDb(createDbStmt);
-        
-        MetricRepo.init();
-        createTable("create table test.table1\n" +
-                "(k1 int, k2 int) distributed by hash(k1) buckets 1\n" +
-                "properties(\"replication_num\" = \"1\");");
-
-        createTable("create table test.table2\n" +
-                "(k1 datetime, k2 int)\n" +
-                "ENGINE=OLAP\n" +
-                "PARTITION BY RANGE(k1)\n" +
-                "(\n" +
-                "PARTITION p20211213 VALUES [('2021-12-13 00:00:00'), ('2021-12-14 00:00:00')),\n" +
-                "PARTITION p20211214 VALUES [('2021-12-14 00:00:00'), ('2021-12-15 00:00:00')),\n" +
-                "PARTITION p20211215 VALUES [('2021-12-15 00:00:00'), ('2021-12-16 00:00:00')),\n" +
-                "PARTITION p20211216 VALUES [('2021-12-16 00:00:00'), ('2021-12-17 00:00:00'))\n" +
-                ")\n" +
-                "DISTRIBUTED BY HASH(k1)\n" +
-                "BUCKETS 10\n" +
-                "PROPERTIES (\n" +
-                "\"replication_num\" = \"1\"\n" +
-                ");");
-        
-    }
-    
-    @AfterClass
-    public static void tearDown() {
-        File file = new File(runningDir);
-        file.delete();
-    }
-    
-    private static void createTable(String sql) throws Exception {
-        CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext);
-        Catalog.getCurrentCatalog().createTable(createTableStmt);
-    }
-    
-    @Test
-    public void testUserMatchSql() throws Exception {
-        String sql = "select * from table1 limit 10";
-        String sqlHash = DigestUtils.md5Hex(sql);
-        SqlBlockRule sqlRule = new SqlBlockRule("test_rule1", null, sqlHash, 0L, 0L, 0L, false, true);
-        SqlBlockRuleMgr mgr = Catalog.getCurrentCatalog().getSqlBlockRuleMgr();
-        mgr.replayCreate(sqlRule);
-        // sql block rules
-        String setPropertyStr = "set property for \"root\" \"sql_block_rules\" = \"test_rule1\"";
-        SetUserPropertyStmt setUserPropertyStmt = (SetUserPropertyStmt) UtFrameUtils.parseAndAnalyzeStmt(setPropertyStr, connectContext);
-        Catalog.getCurrentCatalog().getAuth().updateUserProperty(setUserPropertyStmt);
-        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "sql match hash sql block rule: " + sqlRule.getName(),
-                () -> mgr.matchSql(sql, sqlHash, "root"));
-    }
-    
-    @Test
-    public void testGlobalMatchSql() throws AnalysisException {
-        String sql = "select * from test_table1 limit 10";
-        String sqlHash = DigestUtils.md5Hex(sql);
-        SqlBlockRule sqlRule = new SqlBlockRule("test_rule1", null, sqlHash, 0L, 0L, 0L, true, true);
-        SqlBlockRuleMgr mgr = new SqlBlockRuleMgr();
-        mgr.replayCreate(sqlRule);
-        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "sql match hash sql block rule: " + sqlRule.getName(),
-                () -> mgr.matchSql(sql, sqlHash, "test"));
-    }
-    
-    @Test
-    public void testRegexMatchSql() throws AnalysisException {
-        String sql = "select * from test_table1 tt1 join test_table2 tt2 on tt1.testId=tt2.testId limit 5";
-        String sqlHash = DigestUtils.md5Hex(sql);
-        SqlBlockRule sqlRule = new SqlBlockRule("test_rule1", ".* join .*", null, 0L, 0L, 0L, true, true);
-        SqlBlockRuleMgr mgr = new SqlBlockRuleMgr();
-        mgr.replayCreate(sqlRule);
-        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "sql match regex sql block rule: " + sqlRule.getName(),
-                () -> mgr.matchSql(sqlRule, sql, sqlHash));
-    }
-    
-    @Test
-    public void testHashMatchSql() throws AnalysisException {
-        String sql = "select * from test_table1 tt1 join test_table2 tt2 on tt1.testId=tt2.testId limit 5";
-        String sqlHash = DigestUtils.md5Hex(sql);
-        System.out.println(sqlHash);
-        SqlBlockRule sqlRule = new SqlBlockRule("test_rule1", null, sqlHash, 0L, 0L, 0L, true, true);
-        SqlBlockRuleMgr mgr = new SqlBlockRuleMgr();
-        mgr.replayCreate(sqlRule);
-        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "sql match hash sql block rule: " + sqlRule.getName(),
-                () -> mgr.matchSql(sqlRule, sql, sqlHash));
-    }
-
-    @Test
-    public void testReachLimitations() throws AnalysisException, Exception {
-        String sql = "select * from test.table2;";
-        StmtExecutor executor = UtFrameUtils.getSqlStmtExecutor(connectContext, sql);
-        Planner planner = executor.planner();
-        List<ScanNode> scanNodeList = planner.getScanNodes();
-        OlapScanNode olapScanNode = (OlapScanNode) scanNodeList.get(0);
-        Integer selectedPartition = Deencapsulation.getField(olapScanNode, "selectedPartitionNum");
-        long selectedPartitionLongValue = selectedPartition.longValue();
-        long selectedTablet = Deencapsulation.getField(olapScanNode, "selectedTabletsNum");
-        long cardinality = Deencapsulation.getField(olapScanNode, "cardinality");
-        Assert.assertEquals(0L, selectedPartitionLongValue);
-        Assert.assertEquals(0L, selectedTablet);
-        Assert.assertEquals(0L, cardinality);
-
-        SqlBlockRuleMgr mgr = Catalog.getCurrentCatalog().getSqlBlockRuleMgr();
-
-        // test reach partition_num :
-        // cuz there is no data in test.table2, so the selectedPartitionLongValue == 0;
-        // set sqlBlockRule.partition_num = -1, so it can be blocked.
-        SqlBlockRule sqlBlockRule = new SqlBlockRule("test_rule2", "NULL", "NULL", -1L, 0L, 0L, true, true);
-        mgr.replayCreate(sqlBlockRule);
-        Assert.assertEquals(true, mgr.existRule("test_rule2"));
-        Assert.assertEquals("NULL", sqlBlockRule.getSql());
-        Assert.assertEquals("NULL", sqlBlockRule.getSqlHash());
-        Assert.assertEquals(-1L, (long) sqlBlockRule.getPartitionNum());
-        Assert.assertEquals(0L, (long) sqlBlockRule.getTabletNum());
-        Assert.assertEquals(0L, (long) sqlBlockRule.getCardinality());
-
-        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "errCode = 2, detailMessage = sql hits sql block rule: "
-                        + sqlBlockRule.getName() + ", reach partition_num : " + sqlBlockRule.getPartitionNum(),
-                () -> mgr.checkLimitaions(sqlBlockRule, selectedPartitionLongValue, selectedTablet, cardinality));
-
-        // test reach tablet_num :
-        SqlBlockRule sqlBlockRule2 = new SqlBlockRule("test_rule3", "NULL", "NULL", 0L, -1L, 0L, true, true);
-        mgr.replayCreate(sqlBlockRule2);
-        Assert.assertEquals(true, mgr.existRule("test_rule3"));
-        Assert.assertEquals("NULL", sqlBlockRule2.getSql());
-        Assert.assertEquals("NULL", sqlBlockRule2.getSqlHash());
-        Assert.assertEquals(0L, (long) sqlBlockRule2.getPartitionNum());
-        Assert.assertEquals(-1L, (long) sqlBlockRule2.getTabletNum());
-        Assert.assertEquals(0L, (long) sqlBlockRule2.getCardinality());
-
-        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "errCode = 2, detailMessage = sql hits sql block rule: "
-                        + sqlBlockRule2.getName() + ", reach tablet_num : " + sqlBlockRule2.getTabletNum(),
-                () -> mgr.checkLimitaions(sqlBlockRule2, selectedPartitionLongValue, selectedTablet, cardinality));
-
-        // test reach cardinality :
-        SqlBlockRule sqlBlockRule3 = new SqlBlockRule("test_rule4", "NULL", "NULL", 0L, 0L, -1L, true, true);
-        mgr.replayCreate(sqlBlockRule3);
-        Assert.assertEquals(true, mgr.existRule("test_rule4"));
-        Assert.assertEquals("NULL", sqlBlockRule3.getSql());
-        Assert.assertEquals("NULL", sqlBlockRule3.getSqlHash());
-        Assert.assertEquals(0L, (long) sqlBlockRule3.getPartitionNum());
-        Assert.assertEquals(0L, (long) sqlBlockRule3.getTabletNum());
-        Assert.assertEquals(-1L, (long) sqlBlockRule3.getCardinality());
-
-        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "errCode = 2, detailMessage = sql hits sql block rule: "
-                        + sqlBlockRule3.getName() + ", reach cardinality : " + sqlBlockRule3.getCardinality(),
-                () -> mgr.checkLimitaions(sqlBlockRule3, selectedPartitionLongValue, selectedTablet, cardinality));
-    }
-
-    @Test
-    public void testAlterInvalid() throws Exception {
-        Analyzer analyzer = new Analyzer(Catalog.getCurrentCatalog(), connectContext);
-        SqlBlockRuleMgr mgr = Catalog.getCurrentCatalog().getSqlBlockRuleMgr();
-
-        // create : sql
-        // alter : sqlHash
-        // AnalysisException : Only sql or sqlHash can be configured
-        SqlBlockRule sqlBlockRule = new SqlBlockRule("test_rule", "select \\* from test_table", "NULL", 0L, 0L, 0L, true, true);
-        mgr.unprotectedAdd(sqlBlockRule);
-        Assert.assertEquals(true, mgr.existRule("test_rule"));
-
-        Map<String, String> properties = new HashMap<>();
-        properties.put(CreateSqlBlockRuleStmt.SQL_HASH_PROPERTY, "xxxx");
-        AlterSqlBlockRuleStmt stmt = new AlterSqlBlockRuleStmt("test_rule", properties);
-        stmt.analyze(analyzer);
-
-        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "Only sql or sqlHash can be configured",
-                () -> mgr.alterSqlBlockRule(stmt));
-
-        // create : sql
-        // alter : tabletNum
-        // AnalysisException : sql/sqlHash and partition_num/tablet_num/cardinality cannot be set in one rule.
-        Map<String, String> properties2 = new HashMap<>();
-        properties2.put(CreateSqlBlockRuleStmt.SCANNED_TABLET_NUM, "4");
-        AlterSqlBlockRuleStmt stmt2 = new AlterSqlBlockRuleStmt("test_rule", properties2);
-
-        stmt2.analyze(analyzer);
-        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "sql/sqlHash and partition_num/tablet_num/cardinality cannot be set in one rule.",
-                () -> mgr.alterSqlBlockRule(stmt2));
-
-        // create : cardinality
-        // alter : sqlHash
-        // AnalysisException : sql/sqlHash and partition_num/tablet_num/cardinality cannot be set in one rule.
-        SqlBlockRule sqlBlockRule2 = new SqlBlockRule("test_rule2", "NULL", "NULL", 0L, 0L, 10L, true, true);
-        mgr.unprotectedAdd(sqlBlockRule2);
-        Assert.assertEquals(true, mgr.existRule("test_rule2"));
-
-        Map<String, String> properties3 = new HashMap<>();
-        properties3.put(CreateSqlBlockRuleStmt.SQL_HASH_PROPERTY, "xxxx");
-        AlterSqlBlockRuleStmt stmt3 = new AlterSqlBlockRuleStmt("test_rule2", properties3);
-        stmt3.analyze(analyzer);
-        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "sql/sqlHash and partition_num/tablet_num/cardinality cannot be set in one rule.",
-                () -> mgr.alterSqlBlockRule(stmt3));
-    }
-
-    @Test
-    public void testNormalCreate() throws Exception {
-        String createSql = "CREATE SQL_BLOCK_RULE test_rule PROPERTIES(\"sql\"=\"select \\\\* from test_table\",\"enable\"=\"true\")";
-        CreateSqlBlockRuleStmt createSqlBlockRuleStmt = (CreateSqlBlockRuleStmt) UtFrameUtils.parseAndAnalyzeStmt(createSql, connectContext);
-    }
-
-    @Test
-    public void testOnlyBlockQuery() throws DdlException, UserException {
-        SqlBlockRuleMgr mgr = new SqlBlockRuleMgr();
-        Analyzer analyzer = new Analyzer(Catalog.getCurrentCatalog(), connectContext);
-
-        SqlBlockRule sqlRule = new SqlBlockRule("test_rule1", "test", null, 0L, 0L, 0L, true, true);
-        mgr.replayCreate(sqlRule);
-
-        Map<String, String> properties = new HashMap<>();
-        properties.put(CreateSqlBlockRuleStmt.SQL_PROPERTY, "select \\* from test_table");
-        AlterSqlBlockRuleStmt stmt = new AlterSqlBlockRuleStmt("test_rule1", properties);
-
-        stmt.analyze(analyzer);
-        mgr.alterSqlBlockRule(stmt);
-
-        ShowSqlBlockRuleStmt showStmt = new ShowSqlBlockRuleStmt("test_rule1");
-
-        Assert.assertEquals(1, mgr.getSqlBlockRule(showStmt).size());
-        Assert.assertEquals("select \\* from test_table", mgr.getSqlBlockRule(showStmt).get(0).getSql());
-    }
-
-    @Test
-    public void testLimitationsInvalid() throws Exception {
-        SqlBlockRuleMgr mgr = new SqlBlockRuleMgr();
-
-        // create sql_block_rule with partition_num = -1
-        // DdlException: the value of partition_num can't be a negative
-        String createSql = "CREATE SQL_BLOCK_RULE test_rule PROPERTIES(\"partition_num\"=\"-1\",\"enable\"=\"true\")";
-        CreateSqlBlockRuleStmt stmt = (CreateSqlBlockRuleStmt) UtFrameUtils.parseAndAnalyzeStmt(createSql, connectContext);
-        ExceptionChecker.expectThrowsWithMsg(DdlException.class, "the value of partition_num can't be a negative",
-                () -> mgr.createSqlBlockRule(stmt));
-
-        // create sql_block_rule with tablet_num = -1
-        // DdlException: the value of tablet_num can't be a negative
-        String createSql1 = "CREATE SQL_BLOCK_RULE test_rule PROPERTIES(\"tablet_num\"=\"-1\",\"enable\"=\"true\")";
-        CreateSqlBlockRuleStmt stmt1 = (CreateSqlBlockRuleStmt) UtFrameUtils.parseAndAnalyzeStmt(createSql1, connectContext);
-        ExceptionChecker.expectThrowsWithMsg(DdlException.class, "the value of tablet_num can't be a negative",
-                () -> mgr.createSqlBlockRule(stmt1));
-
-        // create sql_block_rule with cardinality = -1
-        // DdlException: the value of cardinality can't be a negative
-        String createSql2 = "CREATE SQL_BLOCK_RULE test_rule PROPERTIES(\"cardinality\"=\"-1\",\"enable\"=\"true\")";
-        CreateSqlBlockRuleStmt stmt2 = (CreateSqlBlockRuleStmt) UtFrameUtils.parseAndAnalyzeStmt(createSql2, connectContext);
-        ExceptionChecker.expectThrowsWithMsg(DdlException.class, "the value of cardinality can't be a negative",
-                () -> mgr.createSqlBlockRule(stmt2));
-    }
-
-    @Test
-    public void testUserPropertyInvalid() throws Exception {
-        // sql block rules
-        String ruleName = "test_rule_name";
-        String setPropertyStr = String.format("set property for \"root\" \"sql_block_rules\" = \"%s\"", ruleName);
-        SetUserPropertyStmt setUserPropertyStmt = (SetUserPropertyStmt) UtFrameUtils.parseAndAnalyzeStmt(setPropertyStr, connectContext);
-
-        ExceptionChecker.expectThrowsWithMsg(DdlException.class, String.format("the sql block rule %s not exist", ruleName),
-                () -> Catalog.getCurrentCatalog().getAuth().updateUserProperty(setUserPropertyStmt));
-
-    }
-
-    @Test
-    public void testAlterSqlBlock() throws Exception{
-        Analyzer analyzer = new Analyzer(Catalog.getCurrentCatalog(), connectContext);
-        SqlBlockRuleMgr mgr = Catalog.getCurrentCatalog().getSqlBlockRuleMgr();
-
-        // create : sql
-        // alter : global
-        SqlBlockRule sqlBlockRule = new SqlBlockRule("test_rule", "select \\* from test_table", "NULL", 0L, 0L, 0L, true, true);
-        mgr.unprotectedAdd(sqlBlockRule);
-        Assert.assertEquals(true, mgr.existRule("test_rule"));
-
-        Map<String, String> properties = new HashMap<>();
-        properties.put(CreateSqlBlockRuleStmt.GLOBAL_PROPERTY, "false");
-        AlterSqlBlockRuleStmt stmt = new AlterSqlBlockRuleStmt("test_rule", properties);
-        stmt.analyze(analyzer);
-        mgr.alterSqlBlockRule(stmt);
-
-        ShowSqlBlockRuleStmt showStmt = new ShowSqlBlockRuleStmt("test_rule");
-        SqlBlockRule alteredSqlBlockRule = mgr.getSqlBlockRule(showStmt).get(0);
-
-        Assert.assertEquals("select \\* from test_table", alteredSqlBlockRule.getSql());
-        Assert.assertEquals("NULL", alteredSqlBlockRule.getSqlHash());
-        Assert.assertEquals(0L, (long)alteredSqlBlockRule.getPartitionNum());
-        Assert.assertEquals(0L, (long)alteredSqlBlockRule.getTabletNum());
-        Assert.assertEquals(0L, (long)alteredSqlBlockRule.getCardinality());
-        Assert.assertEquals(false, alteredSqlBlockRule.getGlobal());
-        Assert.assertEquals(true, alteredSqlBlockRule.getEnable());
-
-        // create : partitionNum
-        // alter : tabletNum
-        SqlBlockRule sqlBlockRule2 = new SqlBlockRule("test_rule2", "NULL", "NULL", 100L, 0L, 0L, true, true);
-        mgr.unprotectedAdd(sqlBlockRule2);
-        Assert.assertEquals(true, mgr.existRule("test_rule2"));
-
-        Map<String, String> properties2 = new HashMap<>();
-        properties2.put(CreateSqlBlockRuleStmt.SCANNED_TABLET_NUM, "500");
-        AlterSqlBlockRuleStmt stmt2 = new AlterSqlBlockRuleStmt("test_rule2", properties2);
-        stmt2.analyze(analyzer);
-        mgr.alterSqlBlockRule(stmt2);
-
-        ShowSqlBlockRuleStmt showStmt2 = new ShowSqlBlockRuleStmt("test_rule2");
-        SqlBlockRule alteredSqlBlockRule2 = mgr.getSqlBlockRule(showStmt2).get(0);
-
-        Assert.assertEquals("NULL", alteredSqlBlockRule2.getSql());
-        Assert.assertEquals("NULL", alteredSqlBlockRule2.getSqlHash());
-        Assert.assertEquals(100L, (long)alteredSqlBlockRule2.getPartitionNum());
-        Assert.assertEquals(500L, (long)alteredSqlBlockRule2.getTabletNum());
-        Assert.assertEquals(0L, (long)alteredSqlBlockRule2.getCardinality());
-        Assert.assertEquals(true, alteredSqlBlockRule2.getGlobal());
-        Assert.assertEquals(true, alteredSqlBlockRule2.getEnable());
-
-
-    }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 19/22: fix core dump on vcase_expr::close (#9875)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 9094188ba6f9b22b88488fd1b782f9543266a1ba
Author: Pxl <px...@qq.com>
AuthorDate: Tue May 31 15:45:39 2022 +0800

    fix core dump on vcase_expr::close (#9875)
---
 be/src/vec/exprs/vcase_expr.cpp      | 10 +++-------
 be/src/vec/exprs/vexpr.cpp           |  8 ++++----
 be/src/vec/exprs/vexpr.h             |  6 +++---
 be/src/vec/functions/function_case.h | 12 +++++++++++-
 4 files changed, 21 insertions(+), 15 deletions(-)

diff --git a/be/src/vec/exprs/vcase_expr.cpp b/be/src/vec/exprs/vcase_expr.cpp
index 7ecf092574..20ad1fe340 100644
--- a/be/src/vec/exprs/vcase_expr.cpp
+++ b/be/src/vec/exprs/vcase_expr.cpp
@@ -68,19 +68,15 @@ Status VCaseExpr::open(RuntimeState* state, VExprContext* context,
                        FunctionContext::FunctionStateScope scope) {
     RETURN_IF_ERROR(VExpr::open(state, context, scope));
     RETURN_IF_ERROR(VExpr::init_function_context(context, scope, _function));
+
     CaseState* case_state = new CaseState {_data_type};
-    context->fn_context(_fn_context_index)
-            ->set_function_state(FunctionContext::FRAGMENT_LOCAL, case_state);
+    context->fn_context(_fn_context_index)->set_function_state(scope, case_state);
+
     return Status::OK();
 }
 
 void VCaseExpr::close(RuntimeState* state, VExprContext* context,
                       FunctionContext::FunctionStateScope scope) {
-    CaseState* case_state = reinterpret_cast<CaseState*>(
-            context->fn_context(_fn_context_index)
-                    ->get_function_state(FunctionContext::FRAGMENT_LOCAL));
-    delete case_state;
-
     VExpr::close_function_context(context, scope, _function);
     VExpr::close(state, context, scope);
 }
diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp
index 066808cb7d..a8a04d8d0d 100644
--- a/be/src/vec/exprs/vexpr.cpp
+++ b/be/src/vec/exprs/vexpr.cpp
@@ -181,7 +181,7 @@ Status VExpr::create_expr_tree(doris::ObjectPool* pool, const doris::TExpr& texp
     }
     int node_idx = 0;
     VExpr* e = nullptr;
-    Status status = create_tree_from_thrift(pool, texpr.nodes, NULL, &node_idx, &e, ctx);
+    Status status = create_tree_from_thrift(pool, texpr.nodes, nullptr, &node_idx, &e, ctx);
     if (status.ok() && node_idx + 1 != texpr.nodes.size()) {
         status = Status::InternalError(
                 "Expression tree only partially reconstructed. Not all thrift nodes were used.");
@@ -228,7 +228,7 @@ Status VExpr::open(const std::vector<VExprContext*>& ctxs, RuntimeState* state)
 
 Status VExpr::clone_if_not_exists(const std::vector<VExprContext*>& ctxs, RuntimeState* state,
                                   std::vector<VExprContext*>* new_ctxs) {
-    DCHECK(new_ctxs != NULL);
+    DCHECK(new_ctxs != nullptr);
     if (!new_ctxs->empty()) {
         // 'ctxs' was already cloned into '*new_ctxs', nothing to do.
         DCHECK_EQ(new_ctxs->size(), ctxs.size());
@@ -317,7 +317,7 @@ void VExpr::register_function_context(doris::RuntimeState* state, VExprContext*
 
 Status VExpr::init_function_context(VExprContext* context,
                                     FunctionContext::FunctionStateScope scope,
-                                    const FunctionBasePtr& function) {
+                                    const FunctionBasePtr& function) const {
     FunctionContext* fn_ctx = context->fn_context(_fn_context_index);
     if (scope == FunctionContext::FRAGMENT_LOCAL) {
         std::vector<ColumnPtrWrapper*> constant_cols;
@@ -335,7 +335,7 @@ Status VExpr::init_function_context(VExprContext* context,
 }
 
 void VExpr::close_function_context(VExprContext* context, FunctionContext::FunctionStateScope scope,
-                                   const FunctionBasePtr& function) {
+                                   const FunctionBasePtr& function) const {
     if (_fn_context_index != -1) {
         FunctionContext* fn_ctx = context->fn_context(_fn_context_index);
         function->close(fn_ctx, FunctionContext::THREAD_LOCAL);
diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h
index 8958b91e00..31ccc8efba 100644
--- a/be/src/vec/exprs/vexpr.h
+++ b/be/src/vec/exprs/vexpr.h
@@ -125,7 +125,7 @@ public:
     static std::string debug_string(const std::vector<VExpr*>& exprs);
     static std::string debug_string(const std::vector<VExprContext*>& ctxs);
 
-    bool is_and_expr() { return _fn.name.function_name == "and"; }
+    bool is_and_expr() const { return _fn.name.function_name == "and"; }
 
     const TFunction& fn() const { return _fn; }
 
@@ -157,12 +157,12 @@ protected:
     /// 2. Call function's prepare() to initialize function state, fragment-local or
     /// thread-local according the input `FunctionStateScope` argument.
     Status init_function_context(VExprContext* context, FunctionContext::FunctionStateScope scope,
-                                 const FunctionBasePtr& function);
+                                 const FunctionBasePtr& function) const;
 
     /// Helper function to close function context, fragment-local or thread-local according
     /// the input `FunctionStateScope` argument. Called in `close` phase of VExpr.
     void close_function_context(VExprContext* context, FunctionContext::FunctionStateScope scope,
-                                const FunctionBasePtr& function);
+                                const FunctionBasePtr& function) const;
 
     TExprNodeType::type _node_type;
     TypeDescriptor _type;
diff --git a/be/src/vec/functions/function_case.h b/be/src/vec/functions/function_case.h
index 7cea64ac9e..5210ac80ad 100644
--- a/be/src/vec/functions/function_case.h
+++ b/be/src/vec/functions/function_case.h
@@ -222,7 +222,8 @@ public:
     }
 
     template <typename ColumnType>
-    void update_result_auto_simd(MutableColumnPtr& result_column_ptr, uint8* __restrict then_idx,
+    void update_result_auto_simd(MutableColumnPtr& result_column_ptr,
+                                 const uint8* __restrict then_idx,
                                  CaseWhenColumnHolder& column_holder) {
         size_t rows_count = column_holder.rows_count;
         result_column_ptr->resize(rows_count);
@@ -332,6 +333,15 @@ public:
         return execute_get_type(case_state->result_type, block, arguments, result,
                                 input_rows_count);
     }
+
+    Status close(FunctionContext* context, FunctionContext::FunctionStateScope scope) override {
+        if (scope == FunctionContext::THREAD_LOCAL) {
+            auto* state = reinterpret_cast<CaseState*>(
+                    context->get_function_state(FunctionContext::THREAD_LOCAL));
+            delete state;
+        }
+        return Status::OK();
+    }
 };
 
 } // namespace doris::vectorized


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 05/22: [Bug] [Bitmap] change to_bitmap to always_not_nullable (#9716)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 0224db0417f06afc843c83abeb61aacd63bd59bc
Author: Pxl <px...@qq.com>
AuthorDate: Sat May 28 17:33:55 2022 +0800

    [Bug] [Bitmap] change to_bitmap to always_not_nullable (#9716)
---
 be/src/exec/tablet_sink.cpp               |  5 +++--
 be/src/exprs/bitmap_function.cpp          | 20 ++++++++++----------
 be/src/vec/functions/function_bitmap.cpp  | 10 ++++------
 be/test/exprs/bitmap_function_test.cpp    |  6 +++---
 gensrc/script/doris_builtins_functions.py |  4 ++--
 5 files changed, 22 insertions(+), 23 deletions(-)

diff --git a/be/src/exec/tablet_sink.cpp b/be/src/exec/tablet_sink.cpp
index 5604483012..f40d1b0211 100644
--- a/be/src/exec/tablet_sink.cpp
+++ b/be/src/exec/tablet_sink.cpp
@@ -1134,8 +1134,9 @@ Status OlapTableSink::_convert_batch(RuntimeState* state, RowBatch* input_batch,
                             []() -> std::string { return ""; },
                             [&]() -> std::string {
                                 fmt::memory_buffer buf;
-                                fmt::format_to(buf, "null value for not null column, column={}",
-                                               slot_desc->col_name());
+                                fmt::format_to(
+                                        buf, "null value for not null column, column={}, type={}",
+                                        slot_desc->col_name(), slot_desc->type().debug_string());
                                 return fmt::to_string(buf);
                             },
                             &stop_processing));
diff --git a/be/src/exprs/bitmap_function.cpp b/be/src/exprs/bitmap_function.cpp
index ed9c7b2825..d49863ab73 100644
--- a/be/src/exprs/bitmap_function.cpp
+++ b/be/src/exprs/bitmap_function.cpp
@@ -366,17 +366,17 @@ BigIntVal BitmapFunctions::bitmap_min(FunctionContext* ctx, const StringVal& src
 
 StringVal BitmapFunctions::to_bitmap(doris_udf::FunctionContext* ctx,
                                      const doris_udf::StringVal& src) {
-    if (src.is_null) {
-        return StringVal::null();
-    }
-    StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS;
-    uint64_t int_value = StringParser::string_to_unsigned_int<uint64_t>(
-            reinterpret_cast<char*>(src.ptr), src.len, &parse_result);
-    if (UNLIKELY(parse_result != StringParser::PARSE_SUCCESS)) {
-        return StringVal::null();
-    }
     BitmapValue bitmap;
-    bitmap.add(int_value);
+
+    if (!src.is_null) {
+        StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS;
+        uint64_t int_value = StringParser::string_to_unsigned_int<uint64_t>(
+                reinterpret_cast<char*>(src.ptr), src.len, &parse_result);
+        if (parse_result == StringParser::PARSE_SUCCESS) {
+            bitmap.add(int_value);
+        }
+    }
+
     return serialize(ctx, &bitmap);
 }
 
diff --git a/be/src/vec/functions/function_bitmap.cpp b/be/src/vec/functions/function_bitmap.cpp
index 614289e072..0b80c88b5d 100644
--- a/be/src/vec/functions/function_bitmap.cpp
+++ b/be/src/vec/functions/function_bitmap.cpp
@@ -45,18 +45,16 @@ struct ToBitmap {
         auto size = offsets.size();
         res.reserve(size);
         for (size_t i = 0; i < size; ++i) {
+            res.emplace_back();
+
             const char* raw_str = reinterpret_cast<const char*>(&data[offsets[i - 1]]);
             size_t str_size = offsets[i] - offsets[i - 1] - 1;
             StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS;
             uint64_t int_value = StringParser::string_to_unsigned_int<uint64_t>(raw_str, str_size,
                                                                                 &parse_result);
-            if (UNLIKELY(parse_result != StringParser::PARSE_SUCCESS)) {
-                res.emplace_back();
-                null_map[i] = 1;
-                continue;
+            if (LIKELY(parse_result == StringParser::PARSE_SUCCESS)) {
+                res.back().add(int_value);
             }
-            res.emplace_back();
-            res.back().add(int_value);
         }
         return Status::OK();
     }
diff --git a/be/test/exprs/bitmap_function_test.cpp b/be/test/exprs/bitmap_function_test.cpp
index cffc9d71b2..334ba7aa73 100644
--- a/be/test/exprs/bitmap_function_test.cpp
+++ b/be/test/exprs/bitmap_function_test.cpp
@@ -86,19 +86,19 @@ TEST_F(BitmapFunctionsTest, to_bitmap_null) {
     StringVal input = StringVal::null();
     StringVal result = BitmapFunctions::to_bitmap(ctx, input);
 
-    ASSERT_EQ(StringVal::null(), result);
+    EXPECT_EQ(BitmapFunctions::bitmap_empty(ctx), result);
 }
 
 TEST_F(BitmapFunctionsTest, to_bitmap_invalid_argument) {
     StringVal input = AnyValUtil::from_string_temp(ctx, std::string("-1"));
     StringVal result = BitmapFunctions::to_bitmap(ctx, input);
-    ASSERT_EQ(StringVal::null(), result);
+    EXPECT_EQ(BitmapFunctions::bitmap_empty(ctx), result);
 }
 
 TEST_F(BitmapFunctionsTest, to_bitmap_out_of_range) {
     StringVal input = AnyValUtil::from_string_temp(ctx, std::string("18446744073709551616"));
     StringVal result = BitmapFunctions::to_bitmap(ctx, input);
-    ASSERT_EQ(StringVal::null(), result);
+    EXPECT_EQ(BitmapFunctions::bitmap_empty(ctx), result);
 }
 
 TEST_F(BitmapFunctionsTest, bitmap_union_int) {
diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py
index 8e8797430e..4655856624 100755
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -1137,13 +1137,13 @@ visible_functions = [
 
     [['to_bitmap'], 'BITMAP', ['VARCHAR'],
         '_ZN5doris15BitmapFunctions9to_bitmapEPN9doris_udf15FunctionContextERKNS1_9StringValE',
-        '', '', 'vec', 'ALWAYS_NULLABLE'],
+        '', '', 'vec', 'ALWAYS_NOT_NULLABLE'],
     [['bitmap_hash'], 'BITMAP', ['VARCHAR'],
         '_ZN5doris15BitmapFunctions11bitmap_hashEPN9doris_udf15FunctionContextERKNS1_9StringValE',
         '', '', 'vec', 'ALWAYS_NOT_NULLABLE'],
     [['to_bitmap'], 'BITMAP', ['STRING'],
         '_ZN5doris15BitmapFunctions9to_bitmapEPN9doris_udf15FunctionContextERKNS1_9StringValE',
-        '', '', 'vec', 'ALWAYS_NULLABLE'],
+        '', '', 'vec', 'ALWAYS_NOT_NULLABLE'],
     [['bitmap_hash'], 'BITMAP', ['STRING'],
         '_ZN5doris15BitmapFunctions11bitmap_hashEPN9doris_udf15FunctionContextERKNS1_9StringValE',
         '', '', 'vec', 'ALWAYS_NOT_NULLABLE'],


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 14/22: [fix](ut) fix BE ut (#9831)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit fa6b72617c6415b57602eb7b9c49e235bf4055f1
Author: Mingyu Chen <mo...@gmail.com>
AuthorDate: Sun May 29 12:25:41 2022 +0800

    [fix](ut) fix BE ut (#9831)
    
    introduced from #8923, the github checks has some problem that failed to check BE ut in #8923
---
 be/test/olap/cumulative_compaction_policy_test.cpp | 12 ++++++------
 be/test/olap/tablet_meta_test.cpp                  |  5 +++--
 be/test/olap/tablet_test.cpp                       |  6 +++---
 be/test/olap/test_data/header_without_inc_rs.txt   |  3 ++-
 gensrc/proto/olap_file.proto                       |  2 +-
 5 files changed, 15 insertions(+), 13 deletions(-)

diff --git a/be/test/olap/cumulative_compaction_policy_test.cpp b/be/test/olap/cumulative_compaction_policy_test.cpp
index 950fcbf437..950cc18920 100644
--- a/be/test/olap/cumulative_compaction_policy_test.cpp
+++ b/be/test/olap/cumulative_compaction_policy_test.cpp
@@ -31,9 +31,9 @@ class TestNumBasedCumulativeCompactionPolicy : public testing::Test {
 public:
     TestNumBasedCumulativeCompactionPolicy() {}
     void SetUp() {
-        _tablet_meta = static_cast<TabletMetaSharedPtr>(
-                new TabletMeta(1, 2, 15673, 4, 5, TTabletSchema(), 6, {{7, 8}}, UniqueId(9, 10),
-                               TTabletType::TABLET_TYPE_DISK, TStorageMedium::HDD));
+        _tablet_meta = static_cast<TabletMetaSharedPtr>(new TabletMeta(
+                1, 2, 15673, 4, 5, TTabletSchema(), 6, {{7, 8}}, UniqueId(9, 10),
+                TTabletType::TABLET_TYPE_DISK, TStorageMedium::HDD, TCompressionType::LZ4F));
 
         _json_rowset_meta = R"({
             "rowset_id": 540081,
@@ -320,9 +320,9 @@ public:
         config::cumulative_size_based_promotion_min_size_mbytes = 64;
         config::cumulative_size_based_compaction_lower_size_mbytes = 64;
 
-        _tablet_meta = static_cast<TabletMetaSharedPtr>(
-                new TabletMeta(1, 2, 15673, 4, 5, TTabletSchema(), 6, {{7, 8}}, UniqueId(9, 10),
-                               TTabletType::TABLET_TYPE_DISK, TStorageMedium::HDD));
+        _tablet_meta = static_cast<TabletMetaSharedPtr>(new TabletMeta(
+                1, 2, 15673, 4, 5, TTabletSchema(), 6, {{7, 8}}, UniqueId(9, 10),
+                TTabletType::TABLET_TYPE_DISK, TStorageMedium::HDD, TCompressionType::LZ4F));
 
         _json_rowset_meta = R"({
             "rowset_id": 540081,
diff --git a/be/test/olap/tablet_meta_test.cpp b/be/test/olap/tablet_meta_test.cpp
index 001651f7fe..448e2ffb10 100644
--- a/be/test/olap/tablet_meta_test.cpp
+++ b/be/test/olap/tablet_meta_test.cpp
@@ -27,8 +27,9 @@ TEST(TabletMetaTest, SaveAndParse) {
     std::string meta_path = "./be/test/olap/test_data/tablet_meta_test.hdr";
 
     TabletMeta old_tablet_meta(1, 2, 3, 4, 5, TTabletSchema(), 6, {{7, 8}}, UniqueId(9, 10),
-                               TTabletType::TABLET_TYPE_DISK, TStorageMedium::HDD);
-    ASSERT_EQ(OLAP_SUCCESS, old_tablet_meta.save(meta_path));
+                               TTabletType::TABLET_TYPE_DISK, TStorageMedium::HDD,
+                               TCompressionType::LZ4F);
+    EXPECT_EQ(Status::OK(), old_tablet_meta.save(meta_path));
 
     {
         // Just to make stack space dirty
diff --git a/be/test/olap/tablet_test.cpp b/be/test/olap/tablet_test.cpp
index a3776d5fb9..ae8bcb6085 100644
--- a/be/test/olap/tablet_test.cpp
+++ b/be/test/olap/tablet_test.cpp
@@ -35,9 +35,9 @@ public:
     virtual ~TestTablet() {}
 
     virtual void SetUp() {
-        _tablet_meta = static_cast<TabletMetaSharedPtr>(
-                new TabletMeta(1, 2, 15673, 4, 5, TTabletSchema(), 6, {{7, 8}}, UniqueId(9, 10),
-                               TTabletType::TABLET_TYPE_DISK, TStorageMedium::HDD));
+        _tablet_meta = static_cast<TabletMetaSharedPtr>(new TabletMeta(
+                1, 2, 15673, 4, 5, TTabletSchema(), 6, {{7, 8}}, UniqueId(9, 10),
+                TTabletType::TABLET_TYPE_DISK, TStorageMedium::HDD, TCompressionType::LZ4F));
         _json_rowset_meta = R"({
             "rowset_id": 540081,
             "tablet_id": 15673,
diff --git a/be/test/olap/test_data/header_without_inc_rs.txt b/be/test/olap/test_data/header_without_inc_rs.txt
index 997abdba5c..15fa12797b 100644
--- a/be/test/olap/test_data/header_without_inc_rs.txt
+++ b/be/test/olap/test_data/header_without_inc_rs.txt
@@ -53,7 +53,8 @@
         "delete_sign_idx": -1,
         "sequence_col_idx": -1,
         "sort_type": "LEXICAL",
-        "sort_col_num": 0
+        "sort_col_num": 0,
+        "compression_type": "LZ4F"
     },
     "rs_metas": [
         {
diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto
index 43cb8ac7a0..61303f37c5 100644
--- a/gensrc/proto/olap_file.proto
+++ b/gensrc/proto/olap_file.proto
@@ -195,7 +195,7 @@ message TabletSchemaPB {
     optional int32 sequence_col_idx = 10 [default= -1];
     optional SortType sort_type = 11;
     optional int32 sort_col_num = 12;
-    optional segment_v2.CompressionTypePB compression_type = 13;
+    optional segment_v2.CompressionTypePB compression_type = 13 [default=LZ4F];
 }
 
 enum TabletStatePB {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org