You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2024/01/31 13:43:11 UTC

(doris) branch branch-2.1 updated (60c6dfc0073 -> af74c9d2aa5)

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

yiguolei pushed a change to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git


    from 60c6dfc0073 [feature](fe) support ANSI standard keyword for CURRENT_XXX #30550
     new 4418cac69d4 [Chore](workflow)delete auto-reply workflow (#30626)
     new 9012098bcaa [pipelinex](profile) improve memory counter of pipelineX (#30538)
     new b92cae51af5 [tools] update tpcds test tools (#30607)
     new 6634d9f2a98 [Chore](case) reset all variables at start on set_and_unset_variable case (#30580)
     new 7593c8daef6 [pipelineX](profile) Fix Tablet counter on pipelineX engine (#30613)
     new 47e86d77919 [typo](doc) Modify the alter table column document (#28645)
     new ebb04cd7f4e [fix](Nereids) div priority is not right (#30575)
     new ae3ee4b85c1 [docs](update) Update Doris-Streamloader docs (#30552)
     new 8154b8a3f0b [typo](doc)Update flink Spark connector documentation (#30565)
     new eacef8bee7c [Feature](executor)Insert select limited by WorkloadGroup #30610
     new 7234bdce336 [Refactor][Rf] remove unless code in RF (#30597)
     new b911ebc4ad8 [fix](join) incorrect result of mark join (#30543)
     new c1ac82f3b7b [fix][refactor] refactor schema init of externa table and some parquet issue (#30325)
     new e4fa19bb078 [fix](insert into) 'output_tuple_slot_num should be equal to output_expr_num' when insert into unique table with sequence column map (#30635)
     new 192c3ce43d8 [fix](statistics)Fix stats cleaner delete partition stats bug (#30648)
     new 39a6cb6e887 [fix](JDK17) The objects stored in `PriorityQueue` must implement the `Comparable` interface (#30050) (#30625)
     new be1d5a31514 [fix](compile)Fix Ambiguous regex Namespace Issue on MacOS Compilation (#30652)
     new 21d31c6dd9f Sync stats cache while task finished, doesn't need to query column_statistics table. (#30609)
     new af74c9d2aa5 [Fix](parquet-reader) Fix reading fixed length byte array decimal in parquet reader. (#30535)

The 19 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:
 .github/actions/create-or-update-comment           |   1 -
 .github/workflows/auto-pr-reply.yml                |  40 --
 .gitignore                                         |   2 +
 .gitmodules                                        |   3 -
 be/src/exprs/bloom_filter_func.h                   |   3 +-
 be/src/exprs/runtime_filter.cpp                    |  95 ++---
 be/src/exprs/runtime_filter.h                      |   6 +-
 be/src/exprs/runtime_filter_slots.h                |  13 +-
 be/src/exprs/runtime_filter_slots_cross.h          |   2 +-
 be/src/olap/delete_handler.cpp                     |   4 +-
 be/src/olap/delta_writer.cpp                       |   2 +-
 be/src/olap/delta_writer_v2.cpp                    |   6 +
 be/src/olap/memtable_flush_executor.cpp            |  12 +
 be/src/olap/memtable_flush_executor.h              |   3 +
 be/src/olap/memtable_writer.cpp                    |  11 +-
 be/src/olap/memtable_writer.h                      |   2 +-
 be/src/pipeline/exec/aggregation_sink_operator.cpp |   7 +-
 be/src/pipeline/exec/aggregation_sink_operator.h   |   1 -
 be/src/pipeline/exec/analytic_sink_operator.cpp    |   4 +-
 be/src/pipeline/exec/analytic_sink_operator.h      |   1 -
 be/src/pipeline/exec/analytic_source_operator.cpp  |   3 +-
 be/src/pipeline/exec/analytic_source_operator.h    |   1 -
 be/src/pipeline/exec/exchange_sink_operator.cpp    |   1 -
 be/src/pipeline/exec/exchange_sink_operator.h      |   1 -
 be/src/pipeline/exec/hashjoin_build_sink.cpp       |  28 +-
 be/src/pipeline/exec/hashjoin_build_sink.h         |   8 -
 be/src/pipeline/exec/hashjoin_probe_operator.cpp   |   5 +-
 be/src/pipeline/exec/hashjoin_probe_operator.h     |   7 -
 be/src/pipeline/exec/olap_scan_operator.cpp        |   4 +-
 be/src/pipeline/exec/olap_scan_operator.h          |   1 +
 be/src/pipeline/exec/scan_operator.cpp             |  11 +-
 be/src/pipeline/exec/sort_sink_operator.cpp        |  11 +-
 be/src/pipeline/exec/sort_sink_operator.h          |   2 +-
 be/src/pipeline/pipeline_x/operator.cpp            |   8 +-
 be/src/runtime/runtime_filter_mgr.cpp              |   2 +-
 be/src/util/runtime_profile.cpp                    |   5 +-
 be/src/util/runtime_profile.h                      |   6 +-
 be/src/vec/common/hash_table/hash_map_context.h    |  14 +
 be/src/vec/common/hash_table/join_hash_table.h     | 101 ++---
 .../exec/format/parquet/fix_length_plain_decoder.h |   6 +-
 be/src/vec/exec/format/parquet/vparquet_reader.cpp |   9 +-
 be/src/vec/exec/join/process_hash_table_probe.h    |  19 +-
 .../vec/exec/join/process_hash_table_probe_impl.h  | 174 ++++++---
 be/src/vec/exec/join/vhash_join_node.cpp           |   8 -
 be/src/vec/exec/join/vhash_join_node.h             |  22 +-
 be/src/vec/exec/scan/new_olap_scan_node.cpp        |   2 +
 be/src/vec/exec/scan/new_olap_scan_node.h          |   1 +
 be/src/vec/sink/writer/async_result_writer.cpp     |  30 +-
 .../hive/scripts/create_preinstalled_table.hql     |  19 +
 .../fixed_length_byte_array_decimal_table/000000_0 | Bin 0 -> 200910 bytes
 .../import/import-way/stream-load-manual.md        |  13 +
 docs/en/docs/ecosystem/doris-streamloader.md       | 250 +++++++++++++
 docs/en/docs/ecosystem/flink-doris-connector.md    |  18 +-
 docs/en/docs/ecosystem/spark-doris-connector.md    |   4 +-
 .../Alter/ALTER-TABLE-COLUMN.md                    | 398 ++++++++++++--------
 docs/sidebars.json                                 |   1 +
 .../import/import-way/stream-load-manual.md        |  12 +
 docs/zh-CN/docs/ecosystem/doris-streamloader.md    | 247 ++++++++++++
 docs/zh-CN/docs/ecosystem/flink-doris-connector.md |  18 +-
 docs/zh-CN/docs/ecosystem/spark-doris-connector.md |   4 +-
 .../Alter/ALTER-TABLE-COLUMN.md                    | 414 ++++++++++++---------
 .../antlr4/org/apache/doris/nereids/DorisParser.g4 |   4 +-
 .../apache/doris/analysis/NativeInsertStmt.java    |   5 +-
 .../doris/catalog/HiveMetaStoreClientHelper.java   |   2 +-
 .../doris/catalog/external/HMSExternalTable.java   |  73 ++--
 .../catalog/external/IcebergExternalTable.java     |  76 +---
 .../apache/doris/datasource/InternalCatalog.java   |   2 +-
 .../doris/datasource/hive/HiveMetaStoreCache.java  |  18 +-
 .../datasource/iceberg/IcebergExternalCatalog.java |   2 +-
 .../apache/doris/external/hive/util/HiveUtil.java  |  44 +--
 .../doris/external/iceberg/util/IcebergUtils.java  |  90 ++++-
 .../apache/doris/job/extensions/mtmv/MTMVTask.java |   4 +
 .../planner/external/iceberg/IcebergApiSource.java |   9 +-
 .../planner/external/iceberg/IcebergHMSSource.java |   4 +-
 .../external/iceberg/IcebergMetadataCache.java     |  80 +---
 .../planner/external/iceberg/IcebergScanNode.java  |   3 +-
 .../main/java/org/apache/doris/qe/Coordinator.java |  31 +-
 .../java/org/apache/doris/qe/StmtExecutor.java     |   5 -
 .../doris/resource/workloadgroup/QueueToken.java   |  10 +-
 .../apache/doris/service/FrontendServiceImpl.java  |  12 +-
 .../org/apache/doris/statistics/AnalysisJob.java   |  14 -
 .../apache/doris/statistics/AnalysisManager.java   |   2 +-
 .../apache/doris/statistics/BaseAnalysisTask.java  |  12 +-
 .../org/apache/doris/statistics/ColStatsData.java  |  71 +++-
 .../apache/doris/statistics/ColumnStatistic.java   |   1 +
 .../doris/statistics/ExternalAnalysisTask.java     |  10 -
 .../apache/doris/statistics/JdbcAnalysisTask.java  |   9 -
 .../apache/doris/statistics/StatisticsCache.java   |  39 +-
 .../apache/doris/statistics/StatisticsCleaner.java |  12 +-
 .../java/org/apache/doris/statistics/StatsId.java  |  11 +-
 .../doris/statistics/util/StatisticsUtil.java      |   2 +-
 .../apache/doris/statistics/AnalysisJobTest.java   |   7 -
 .../org/apache/doris/statistics/CacheTest.java     |   2 -
 .../apache/doris/statistics/ColStatsDataTest.java  |  71 ++++
 gensrc/thrift/FrontendService.thrift               |   3 +-
 .../external_table_p0/hive/test_hive_parquet.out   |  12 +
 regression-test/data/nereids_arith_p0/integer.out  | 414 +++++++++++----------
 .../data/nereids_p0/join/test_mark_join.out        |  43 +++
 .../data/variable_p0/set_and_unset_variable.out    |   6 +
 .../test_sequence_col_default_value.groovy         |   2 +-
 .../test_array_zip_array_enumerate_uniq.groovy     |   2 +-
 .../hive/test_hive_parquet.groovy                  |  16 +
 .../suites/nereids_arith_p0/integer.groovy         |  12 +
 .../suites/nereids_p0/join/test_mark_join.groovy   | 126 +++++++
 .../variable_p0/set_and_unset_variable.groovy      |   2 +
 tools/tpcds-tools/conf/opt/opt_sf1.sql             |   2 -
 tools/tpcds-tools/conf/opt/opt_sf100.sql           |   4 +-
 tools/tpcds-tools/conf/opt/opt_sf1000.sql          |   5 -
 tools/tpcds-tools/conf/opt/opt_sf10000.sql         |   5 -
 109 files changed, 2237 insertions(+), 1260 deletions(-)
 delete mode 160000 .github/actions/create-or-update-comment
 delete mode 100644 .github/workflows/auto-pr-reply.yml
 create mode 100644 docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/fixed_length_byte_array_decimal_table/000000_0
 create mode 100644 docs/en/docs/ecosystem/doris-streamloader.md
 create mode 100644 docs/zh-CN/docs/ecosystem/doris-streamloader.md
 create mode 100644 regression-test/data/nereids_p0/join/test_mark_join.out
 create mode 100644 regression-test/suites/nereids_p0/join/test_mark_join.groovy


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


(doris) 05/19: [pipelineX](profile) Fix Tablet counter on pipelineX engine (#30613)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 7593c8daef6fe1c38b138b33cc2bafe7438332fe
Author: Gabriel <ga...@gmail.com>
AuthorDate: Wed Jan 31 14:30:30 2024 +0800

    [pipelineX](profile) Fix Tablet counter on pipelineX engine (#30613)
---
 be/src/pipeline/exec/olap_scan_operator.cpp | 4 +++-
 be/src/pipeline/exec/olap_scan_operator.h   | 1 +
 be/src/pipeline/exec/scan_operator.cpp      | 2 +-
 be/src/vec/exec/scan/new_olap_scan_node.cpp | 2 ++
 be/src/vec/exec/scan/new_olap_scan_node.h   | 1 +
 5 files changed, 8 insertions(+), 2 deletions(-)

diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp b/be/src/pipeline/exec/olap_scan_operator.cpp
index b89ac7333c3..0aab714449e 100644
--- a/be/src/pipeline/exec/olap_scan_operator.cpp
+++ b/be/src/pipeline/exec/olap_scan_operator.cpp
@@ -136,6 +136,7 @@ Status OlapScanLocalState::_init_profile() {
     _filtered_segment_counter = ADD_COUNTER(_segment_profile, "NumSegmentFiltered", TUnit::UNIT);
     _total_segment_counter = ADD_COUNTER(_segment_profile, "NumSegmentTotal", TUnit::UNIT);
     _tablet_counter = ADD_COUNTER(_runtime_profile, "TabletNum", TUnit::UNIT);
+    _key_range_counter = ADD_COUNTER(_runtime_profile, "KeyRangesNum", TUnit::UNIT);
     _runtime_filter_info = ADD_LABEL_COUNTER_WITH_LEVEL(_runtime_profile, "RuntimeFilterInfo", 1);
     return Status::OK();
 }
@@ -323,6 +324,7 @@ Status OlapScanLocalState::_init_scanners(std::list<vectorized::VScannerSPtr>* s
 
     auto build_new_scanner = [&](BaseTabletSPtr tablet, int64_t version,
                                  const std::vector<OlapScanRange*>& key_ranges) {
+        COUNTER_UPDATE(_key_range_counter, key_ranges.size());
         auto scanner = vectorized::NewOlapScanner::create_shared(
                 this, vectorized::NewOlapScanner::Params {
                               state(),
@@ -380,7 +382,7 @@ void OlapScanLocalState::set_scan_ranges(RuntimeState* state,
     for (auto& scan_range : scan_ranges) {
         DCHECK(scan_range.scan_range.__isset.palo_scan_range);
         _scan_ranges.emplace_back(new TPaloScanRange(scan_range.scan_range.palo_scan_range));
-        //        COUNTER_UPDATE(_tablet_counter, 1);
+        COUNTER_UPDATE(_tablet_counter, 1);
     }
 }
 
diff --git a/be/src/pipeline/exec/olap_scan_operator.h b/be/src/pipeline/exec/olap_scan_operator.h
index ffd8fecd075..233283a59aa 100644
--- a/be/src/pipeline/exec/olap_scan_operator.h
+++ b/be/src/pipeline/exec/olap_scan_operator.h
@@ -104,6 +104,7 @@ private:
     RuntimeProfile::Counter* _num_disks_accessed_counter = nullptr;
 
     RuntimeProfile::Counter* _tablet_counter = nullptr;
+    RuntimeProfile::Counter* _key_range_counter = nullptr;
     RuntimeProfile::Counter* _rows_pushed_cond_filtered_counter = nullptr;
     RuntimeProfile::Counter* _reader_init_timer = nullptr;
     RuntimeProfile::Counter* _scanner_init_timer = nullptr;
diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp
index 88e6c880568..d9fced39b05 100644
--- a/be/src/pipeline/exec/scan_operator.cpp
+++ b/be/src/pipeline/exec/scan_operator.cpp
@@ -119,7 +119,6 @@ Status ScanLocalState<Derived>::init(RuntimeState* state, LocalStateInfo& info)
 
     _scan_dependency = dependency_sptr();
 
-    set_scan_ranges(state, info.scan_ranges);
     _common_expr_ctxs_push_down.resize(p._common_expr_ctxs_push_down.size());
     for (size_t i = 0; i < _common_expr_ctxs_push_down.size(); i++) {
         RETURN_IF_ERROR(
@@ -138,6 +137,7 @@ Status ScanLocalState<Derived>::init(RuntimeState* state, LocalStateInfo& info)
     // during pipeline mode with more instances, olap scan node maybe not new VScanner object,
     // so the profile of VScanner and SegmentIterator infos are always empty, could not init those.
     RETURN_IF_ERROR(_init_profile());
+    set_scan_ranges(state, info.scan_ranges);
     // if you want to add some profile in scan node, even it have not new VScanner object
     // could add here, not in the _init_profile() function
     _prepare_rf_timer(_runtime_profile.get());
diff --git a/be/src/vec/exec/scan/new_olap_scan_node.cpp b/be/src/vec/exec/scan/new_olap_scan_node.cpp
index 558169c775f..b11db29eb11 100644
--- a/be/src/vec/exec/scan/new_olap_scan_node.cpp
+++ b/be/src/vec/exec/scan/new_olap_scan_node.cpp
@@ -84,6 +84,7 @@ Status NewOlapScanNode::prepare(RuntimeState* state) {
     // if you want to add some profile in scan node, even it have not new VScanner object
     // could add here, not in the _init_profile() function
     _tablet_counter = ADD_COUNTER(_runtime_profile, "TabletNum", TUnit::UNIT);
+    _key_range_counter = ADD_COUNTER(_runtime_profile, "KeyRangesNum", TUnit::UNIT);
     return Status::OK();
 }
 
@@ -597,6 +598,7 @@ Status NewOlapScanNode::_init_scanners(std::list<VScannerSPtr>* scanners) {
     auto build_new_scanner = [&](BaseTabletSPtr tablet, int64_t version,
                                  const std::vector<OlapScanRange*>& key_ranges,
                                  TabletReader::ReadSource read_source) {
+        COUNTER_UPDATE(_key_range_counter, key_ranges.size());
         auto scanner =
                 NewOlapScanner::create_shared(this, NewOlapScanner::Params {
                                                             _state,
diff --git a/be/src/vec/exec/scan/new_olap_scan_node.h b/be/src/vec/exec/scan/new_olap_scan_node.h
index ca357b7eb7d..e1861a3b619 100644
--- a/be/src/vec/exec/scan/new_olap_scan_node.h
+++ b/be/src/vec/exec/scan/new_olap_scan_node.h
@@ -129,6 +129,7 @@ private:
     RuntimeProfile::Counter* _num_disks_accessed_counter = nullptr;
 
     RuntimeProfile::Counter* _tablet_counter = nullptr;
+    RuntimeProfile::Counter* _key_range_counter = nullptr;
     RuntimeProfile::Counter* _rows_pushed_cond_filtered_counter = nullptr;
     RuntimeProfile::Counter* _reader_init_timer = nullptr;
     RuntimeProfile::Counter* _scanner_init_timer = nullptr;


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


(doris) 15/19: [fix](statistics)Fix stats cleaner delete partition stats bug (#30648)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 192c3ce43d89858cd6954238683bb5c2bb521f4a
Author: Jibing-Li <64...@users.noreply.github.com>
AuthorDate: Wed Jan 31 20:50:33 2024 +0800

    [fix](statistics)Fix stats cleaner delete partition stats bug (#30648)
    
    When a partition in OlapTable is removed, we should use partition id to delete the related stats record in column_statistics. Before, it was using id, which may cause delete useful stats of other partition.
---
 .../java/org/apache/doris/statistics/StatisticsCleaner.java  | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCleaner.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCleaner.java
index 1af219baa6f..3a142351a28 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCleaner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCleaner.java
@@ -169,12 +169,15 @@ public class StatisticsCleaner extends MasterDaemon {
         doDelete("idx_id", expiredStats.expiredIdxId.stream()
                         .map(String::valueOf).collect(Collectors.toList()),
                 FeConstants.INTERNAL_DB_NAME + "." + tblName, false);
+        doDelete("part_id", expiredStats.expiredPartitionId.stream()
+                        .map(String::valueOf).collect(Collectors.toList()),
+                FeConstants.INTERNAL_DB_NAME + "." + tblName, false);
         doDelete("id", expiredStats.ids.stream()
                         .map(String::valueOf).collect(Collectors.toList()),
                 FeConstants.INTERNAL_DB_NAME + "." + tblName, false);
     }
 
-    private void doDelete(String/*col name*/ colName, List<String> pred, String tblName, boolean taskOnly) {
+    private void doDelete(String colName, List<String> pred, String tblName, boolean taskOnly) {
         String deleteTemplate = "DELETE FROM " + tblName + " WHERE ${left} IN (${right})";
         if (CollectionUtils.isEmpty(pred)) {
             return;
@@ -241,7 +244,7 @@ public class StatisticsCleaner extends MasterDaemon {
                         continue;
                     }
                     if (!olapTable.getPartitionIds().contains(Long.parseLong(partId))) {
-                        expiredStats.ids.add(id);
+                        expiredStats.expiredPartitionId.add(Long.parseLong(partId));
                     }
                 } catch (Exception e) {
                     LOG.warn("Error occurred when retrieving expired stats", e);
@@ -256,9 +259,8 @@ public class StatisticsCleaner extends MasterDaemon {
         Set<Long> expiredCatalog = new HashSet<>();
         Set<Long> expiredDatabase = new HashSet<>();
         Set<Long> expiredTable = new HashSet<>();
-
         Set<Long> expiredIdxId = new HashSet<>();
-
+        Set<Long> expiredPartitionId = new HashSet<>();
         Set<String> ids = new HashSet<>();
 
         public boolean isFull() {
@@ -266,6 +268,7 @@ public class StatisticsCleaner extends MasterDaemon {
                     || expiredDatabase.size() >= Config.max_allowed_in_element_num_of_delete
                     || expiredTable.size() >= Config.max_allowed_in_element_num_of_delete
                     || expiredIdxId.size() >= Config.max_allowed_in_element_num_of_delete
+                    || expiredPartitionId.size() >= Config.max_allowed_in_element_num_of_delete
                     || ids.size() >= Config.max_allowed_in_element_num_of_delete;
         }
 
@@ -274,6 +277,7 @@ public class StatisticsCleaner extends MasterDaemon {
                     && expiredDatabase.isEmpty()
                     && expiredTable.isEmpty()
                     && expiredIdxId.isEmpty()
+                    && expiredPartitionId.isEmpty()
                     && ids.size() < Config.max_allowed_in_element_num_of_delete / 10;
         }
     }


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


(doris) 19/19: [Fix](parquet-reader) Fix reading fixed length byte array decimal in parquet reader. (#30535)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit af74c9d2aa567378875f3ca5e1b80d7eed355a37
Author: Qi Chen <ka...@gmail.com>
AuthorDate: Wed Jan 31 21:37:59 2024 +0800

    [Fix](parquet-reader) Fix reading fixed length byte array decimal in parquet reader. (#30535)
---
 .../exec/format/parquet/fix_length_plain_decoder.h   |   6 ++----
 .../hive/scripts/create_preinstalled_table.hql       |  19 +++++++++++++++++++
 .../fixed_length_byte_array_decimal_table/000000_0   | Bin 0 -> 200910 bytes
 .../external_table_p0/hive/test_hive_parquet.out     |  12 ++++++++++++
 .../external_table_p0/hive/test_hive_parquet.groovy  |  16 ++++++++++++++++
 5 files changed, 49 insertions(+), 4 deletions(-)

diff --git a/be/src/vec/exec/format/parquet/fix_length_plain_decoder.h b/be/src/vec/exec/format/parquet/fix_length_plain_decoder.h
index c06e33c550c..cbf2ffa5c68 100644
--- a/be/src/vec/exec/format/parquet/fix_length_plain_decoder.h
+++ b/be/src/vec/exec/format/parquet/fix_length_plain_decoder.h
@@ -112,19 +112,17 @@ Status FixLengthPlainDecoder<PhysicalType>::_decode_string(MutableColumnPtr& dor
             size_t data_size = run_length * _type_length;
             size_t old_size = chars.size();
             chars.resize(old_size + data_size);
-            memcpy(chars.data() + old_size, _data->data, data_size);
+            memcpy(chars.data() + old_size, _data->data + _offset, data_size);
 
             // copy offsets
             offsets.resize(offsets.size() + run_length);
             auto* offsets_data = offsets.data() + offsets.size() - run_length;
 
-            int i = 0;
-            for (; i < run_length; i++) {
+            for (int i = 0; i < run_length; i++) {
                 bytes_size += _type_length;
                 *(offsets_data++) = bytes_size;
             }
 
-            //doris_column->insert_many_strings_fixed_length<_type_length>(&string_values[0], run_length);
             _offset += data_size;
             break;
         }
diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql
index cc16a408eec..9baf13020f6 100644
--- a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql
+++ b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_table.hql
@@ -1899,6 +1899,25 @@ LOCATION
 
 msck repair table parquet_decimal90_table;
 
+CREATE TABLE `fixed_length_byte_array_decimal_table`(
+  `decimal_col1` decimal(7,2),
+  `decimal_col2` decimal(7,2),
+  `decimal_col3` decimal(7,2),
+  `decimal_col4` decimal(7,2),
+  `decimal_col5` decimal(7,2))
+ROW FORMAT SERDE
+  'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+STORED AS INPUTFORMAT
+  'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+OUTPUTFORMAT
+  'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+LOCATION
+  '/user/doris/preinstalled_data/parquet_table/fixed_length_byte_array_decimal_table'
+TBLPROPERTIES (
+  'parquet.compress'='SNAPPY');
+
+msck repair table fixed_length_byte_array_decimal_table;
+
 show tables;
 
 
diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/fixed_length_byte_array_decimal_table/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/fixed_length_byte_array_decimal_table/000000_0
new file mode 100644
index 00000000000..c215cf5ac6c
Binary files /dev/null and b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/fixed_length_byte_array_decimal_table/000000_0 differ
diff --git a/regression-test/data/external_table_p0/hive/test_hive_parquet.out b/regression-test/data/external_table_p0/hive/test_hive_parquet.out
index a54a25520da..d0a22fe47b8 100644
--- a/regression-test/data/external_table_p0/hive/test_hive_parquet.out
+++ b/regression-test/data/external_table_p0/hive/test_hive_parquet.out
@@ -10145,3 +10145,15 @@ us	washington	1999
 -- !q21_avg --
 -495084140.9042
 
+-- !q22_max --
+27960.57	29917.80	41344.02	43556.47	18281.12
+
+-- !q22_min --
+0.00	0.00	0.00	0.00	-9884.00
+
+-- !q22_sum --
+2296013386.03	2399429017.59	3573059811.32	3676453127.65	-256727758.97
+
+-- !q22_avg --
+2296.5737	2399.9882	3573.0598	3676.4531	-256.7277
+
diff --git a/regression-test/suites/external_table_p0/hive/test_hive_parquet.groovy b/regression-test/suites/external_table_p0/hive/test_hive_parquet.groovy
index c60e1a4f0a6..c381a3c236b 100644
--- a/regression-test/suites/external_table_p0/hive/test_hive_parquet.groovy
+++ b/regression-test/suites/external_table_p0/hive/test_hive_parquet.groovy
@@ -154,6 +154,21 @@ suite("test_hive_parquet", "p0,external,hive,external_docker,external_docker_hiv
         """
     }
 
+    def q22 = {
+        qt_q22_max """
+        select max(decimal_col1), max(decimal_col2), max(decimal_col3), max(decimal_col4), max(decimal_col5) from fixed_length_byte_array_decimal_table;
+        """
+        qt_q22_min """
+        select min(decimal_col1), min(decimal_col2), min(decimal_col3), min(decimal_col4), min(decimal_col5) from fixed_length_byte_array_decimal_table;
+        """
+        qt_q22_sum """
+        select sum(decimal_col1), sum(decimal_col2), sum(decimal_col3), sum(decimal_col4), sum(decimal_col5) from fixed_length_byte_array_decimal_table;
+        """
+        qt_q22_avg """
+        select avg(decimal_col1), avg(decimal_col2), avg(decimal_col3), avg(decimal_col4), avg(decimal_col5) from fixed_length_byte_array_decimal_table;
+        """
+    }
+
     String enabled = context.config.otherConfigs.get("enableHiveTest")
     if (enabled != null && enabled.equalsIgnoreCase("true")) {
         try {
@@ -191,6 +206,7 @@ suite("test_hive_parquet", "p0,external,hive,external_docker,external_docker_hiv
             q19()
             q20()
             q21()
+            q22()
 
             sql """explain physical plan select l_partkey from partition_table
                 where (nation != 'cn' or city !='beijing') and (l_quantity > 28 or l_extendedprice > 30000)


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


(doris) 13/19: [fix][refactor] refactor schema init of externa table and some parquet issue (#30325)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit c1ac82f3b7b1440ee7234bbcea0ece43e334119c
Author: Mingyu Chen <mo...@163.com>
AuthorDate: Wed Jan 31 18:33:30 2024 +0800

    [fix][refactor] refactor schema init of externa table and some parquet issue (#30325)
    
    1. Skip parquet file which has only 4 bytes length: PAR1
    2. Refactor the schema init method of iceberg/hudi/hive table in hms catalog
        1. Remove some redundant methods of `getIcebergTable`
        2. Fix issue described in #23771
    3. Support HoodieParquetInputFormatBase, treat it as normal hive table format
    4. When listing file, skip all hidden dirs and files
---
 .gitignore                                         |  2 +
 be/src/vec/exec/format/parquet/vparquet_reader.cpp |  9 ++-
 .../doris/catalog/HiveMetaStoreClientHelper.java   |  2 +-
 .../doris/catalog/external/HMSExternalTable.java   | 73 +++++++++---------
 .../catalog/external/IcebergExternalTable.java     | 76 +-----------------
 .../doris/datasource/hive/HiveMetaStoreCache.java  | 18 ++---
 .../datasource/iceberg/IcebergExternalCatalog.java |  2 +-
 .../apache/doris/external/hive/util/HiveUtil.java  | 44 ++---------
 .../doris/external/iceberg/util/IcebergUtils.java  | 90 ++++++++++++++++++++--
 .../apache/doris/job/extensions/mtmv/MTMVTask.java |  4 +
 .../planner/external/iceberg/IcebergApiSource.java |  9 +--
 .../planner/external/iceberg/IcebergHMSSource.java |  4 +-
 .../external/iceberg/IcebergMetadataCache.java     | 80 ++++---------------
 .../planner/external/iceberg/IcebergScanNode.java  |  3 +-
 .../doris/statistics/util/StatisticsUtil.java      |  2 +-
 15 files changed, 175 insertions(+), 243 deletions(-)

diff --git a/.gitignore b/.gitignore
index 9a35bbe2258..c4d51db53db 100644
--- a/.gitignore
+++ b/.gitignore
@@ -54,6 +54,8 @@ thirdparty/installed*
 thirdparty/doris-thirdparty*.tar.xz
 
 docker/thirdparties/docker-compose/mysql/data
+docker/thirdparties/docker-compose/hive/scripts/tpch1.db/
+docker/thirdparties/docker-compose/hive/scripts/paimon1
 
 fe_plugins/output
 fe_plugins/**/.factorypath
diff --git a/be/src/vec/exec/format/parquet/vparquet_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_reader.cpp
index 6ae4ea2f5bc..dd8d01f4bb1 100644
--- a/be/src/vec/exec/format/parquet/vparquet_reader.cpp
+++ b/be/src/vec/exec/format/parquet/vparquet_reader.cpp
@@ -238,8 +238,11 @@ Status ParquetReader::_open_file() {
     }
     if (_file_metadata == nullptr) {
         SCOPED_RAW_TIMER(&_statistics.parse_footer_time);
-        if (_file_reader->size() == 0) {
-            return Status::EndOfFile("open file failed, empty parquet file: " + _scan_range.path);
+        if (_file_reader->size() <= sizeof(PARQUET_VERSION_NUMBER)) {
+            // Some system may generate parquet file with only 4 bytes: PAR1
+            // Should consider it as empty file.
+            return Status::EndOfFile("open file failed, empty parquet file {} with size: {}",
+                                     _scan_range.path, _file_reader->size());
         }
         size_t meta_size = 0;
         if (_meta_cache == nullptr) {
@@ -928,4 +931,4 @@ int64_t ParquetReader::_get_column_start_offset(const tparquet::ColumnMetaData&
     }
     return column.data_page_offset;
 }
-} // namespace doris::vectorized
\ No newline at end of file
+} // namespace doris::vectorized
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveMetaStoreClientHelper.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveMetaStoreClientHelper.java
index 96419cd0b0a..7d0419fcbf9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveMetaStoreClientHelper.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveMetaStoreClientHelper.java
@@ -770,7 +770,7 @@ public class HiveMetaStoreClientHelper {
         try {
             hudiSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaUtil.getTableAvroSchema());
         } catch (Exception e) {
-            throw new RuntimeException("Cannot get hudi table schema.");
+            throw new RuntimeException("Cannot get hudi table schema.", e);
         }
         return hudiSchema;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java
index e1037ffd025..028928eafce 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java
@@ -32,6 +32,7 @@ import org.apache.doris.datasource.HMSExternalCatalog;
 import org.apache.doris.datasource.hive.HMSCachedClient;
 import org.apache.doris.datasource.hive.HiveMetaStoreCache;
 import org.apache.doris.datasource.hive.HivePartition;
+import org.apache.doris.external.iceberg.util.IcebergUtils;
 import org.apache.doris.mtmv.MTMVRelatedTableIf;
 import org.apache.doris.nereids.exceptions.NotSupportedException;
 import org.apache.doris.statistics.AnalysisInfo;
@@ -62,8 +63,6 @@ import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.iceberg.Schema;
-import org.apache.iceberg.Table;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -87,8 +86,10 @@ import java.util.stream.Collectors;
 public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableIf {
     private static final Logger LOG = LogManager.getLogger(HMSExternalTable.class);
 
-    private static final Set<String> SUPPORTED_HIVE_FILE_FORMATS;
-    private static final Set<String> SUPPORTED_HIVE_TRANSACTIONAL_FILE_FORMATS;
+    public static final Set<String> SUPPORTED_HIVE_FILE_FORMATS;
+    public static final Set<String> SUPPORTED_HIVE_TRANSACTIONAL_FILE_FORMATS;
+    public static final Set<String> SUPPORTED_HUDI_FILE_FORMATS;
+
     private static final Map<StatsType, String> MAP_SPARK_STATS_TO_DORIS;
     private static final String TBL_PROP_TXN_PROPERTIES = "transactional_properties";
     private static final String TBL_PROP_INSERT_ONLY = "insert_only";
@@ -111,13 +112,16 @@ public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableI
         SUPPORTED_HIVE_FILE_FORMATS.add("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat");
         SUPPORTED_HIVE_FILE_FORMATS.add("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat");
         SUPPORTED_HIVE_FILE_FORMATS.add("org.apache.hadoop.mapred.TextInputFormat");
+        // Some hudi table use HoodieParquetInputFormatBase as input format
+        // But we can't treat it as hudi table.
+        // So add to SUPPORTED_HIVE_FILE_FORMATS and treat is as a hive table.
+        // Then Doris will just list the files from location and read parquet files directly.
+        SUPPORTED_HIVE_FILE_FORMATS.add("org.apache.hudi.hadoop.HoodieParquetInputFormatBase");
 
         SUPPORTED_HIVE_TRANSACTIONAL_FILE_FORMATS = Sets.newHashSet();
         SUPPORTED_HIVE_TRANSACTIONAL_FILE_FORMATS.add("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat");
     }
 
-    private static final Set<String> SUPPORTED_HUDI_FILE_FORMATS;
-
     static {
         SUPPORTED_HUDI_FILE_FORMATS = Sets.newHashSet();
         SUPPORTED_HUDI_FILE_FORMATS.add("org.apache.hudi.hadoop.HoodieParquetInputFormat");
@@ -405,10 +409,6 @@ public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableI
         return ((HMSExternalCatalog) catalog).getHiveMetastoreUris();
     }
 
-    public String getHiveVersion() {
-        return ((HMSExternalCatalog) catalog).getHiveVersion();
-    }
-
     public Map<String, String> getCatalogProperties() {
         return catalog.getProperties();
     }
@@ -454,32 +454,28 @@ public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableI
         return initSchema();
     }
 
-
     @Override
     public List<Column> initSchema() {
         makeSureInitialized();
         List<Column> columns;
-        List<FieldSchema> schema = ((HMSExternalCatalog) catalog).getClient().getSchema(dbName, name);
         if (dlaType.equals(DLAType.ICEBERG)) {
-            columns = getIcebergSchema(schema);
+            columns = getIcebergSchema();
         } else if (dlaType.equals(DLAType.HUDI)) {
-            columns = getHudiSchema(schema);
+            columns = getHudiSchema();
         } else {
-            List<Column> tmpSchema = Lists.newArrayListWithCapacity(schema.size());
-            for (FieldSchema field : schema) {
-                tmpSchema.add(new Column(field.getName().toLowerCase(Locale.ROOT),
-                        HiveMetaStoreClientHelper.hiveTypeToDorisType(field.getType()), true, null,
-                        true, field.getComment(), true, -1));
-            }
-            columns = tmpSchema;
+            columns = getHiveSchema();
         }
         initPartitionColumns(columns);
         return columns;
     }
 
-    public List<Column> getHudiSchema(List<FieldSchema> hmsSchema) {
+    private List<Column> getIcebergSchema() {
+        return IcebergUtils.getSchema(catalog, dbName, name);
+    }
+
+    private List<Column> getHudiSchema() {
         org.apache.avro.Schema hudiSchema = HiveMetaStoreClientHelper.getHudiTableSchema(this);
-        List<Column> tmpSchema = Lists.newArrayListWithCapacity(hmsSchema.size());
+        List<Column> tmpSchema = Lists.newArrayListWithCapacity(hudiSchema.getFields().size());
         for (org.apache.avro.Schema.Field hudiField : hudiSchema.getFields()) {
             String columnName = hudiField.name().toLowerCase(Locale.ROOT);
             tmpSchema.add(new Column(columnName, HudiUtils.fromAvroHudiTypeToDorisType(hudiField.schema()),
@@ -488,6 +484,19 @@ public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableI
         return tmpSchema;
     }
 
+    private List<Column> getHiveSchema() {
+        List<Column> columns;
+        List<FieldSchema> schema = ((HMSExternalCatalog) catalog).getClient().getSchema(dbName, name);
+        List<Column> tmpSchema = Lists.newArrayListWithCapacity(schema.size());
+        for (FieldSchema field : schema) {
+            tmpSchema.add(new Column(field.getName().toLowerCase(Locale.ROOT),
+                    HiveMetaStoreClientHelper.hiveTypeToDorisType(field.getType()), true, null,
+                    true, field.getComment(), true, -1));
+        }
+        columns = tmpSchema;
+        return columns;
+    }
+
     @Override
     public long getCacheRowCount() {
         //Cached accurate information
@@ -528,20 +537,6 @@ public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableI
         return 1;
     }
 
-    private List<Column> getIcebergSchema(List<FieldSchema> hmsSchema) {
-        Table icebergTable = Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache().getIcebergTable(this);
-        Schema schema = icebergTable.schema();
-        List<Column> tmpSchema = Lists.newArrayListWithCapacity(hmsSchema.size());
-        for (FieldSchema field : hmsSchema) {
-            tmpSchema.add(new Column(field.getName().toLowerCase(Locale.ROOT),
-                    HiveMetaStoreClientHelper.hiveTypeToDorisType(field.getType(),
-                            IcebergExternalTable.ICEBERG_DATETIME_SCALE_MS),
-                    true, null, true, false, null, field.getComment(), true, null,
-                    schema.caseInsensitiveFindField(field.getName()).fieldId(), null));
-        }
-        return tmpSchema;
-    }
-
     private void initPartitionColumns(List<Column> schema) {
         List<String> partitionKeys = remoteTable.getPartitionKeys().stream().map(FieldSchema::getName)
                 .collect(Collectors.toList());
@@ -598,7 +593,9 @@ public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableI
                 return getHiveColumnStats(colName);
             case ICEBERG:
                 return StatisticsUtil.getIcebergColumnStats(colName,
-                        Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache().getIcebergTable(this));
+                        Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache().getIcebergTable(
+                                catalog, dbName, name
+                        ));
             default:
                 LOG.warn("get column stats for dlaType {} is not supported.", dlaType);
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java
index be99e26de62..be320fc9268 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java
@@ -17,12 +17,10 @@
 
 package org.apache.doris.catalog.external;
 
-import org.apache.doris.catalog.ArrayType;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.HiveMetaStoreClientHelper;
-import org.apache.doris.catalog.ScalarType;
-import org.apache.doris.catalog.Type;
 import org.apache.doris.datasource.iceberg.IcebergExternalCatalog;
+import org.apache.doris.external.iceberg.util.IcebergUtils;
 import org.apache.doris.statistics.AnalysisInfo;
 import org.apache.doris.statistics.BaseAnalysisTask;
 import org.apache.doris.statistics.ColumnStatistic;
@@ -33,21 +31,11 @@ import org.apache.doris.thrift.TIcebergTable;
 import org.apache.doris.thrift.TTableDescriptor;
 import org.apache.doris.thrift.TTableType;
 
-import com.google.common.collect.Lists;
-import org.apache.iceberg.Schema;
-import org.apache.iceberg.types.Types;
-
 import java.util.HashMap;
 import java.util.List;
-import java.util.Locale;
 import java.util.Optional;
 
 public class IcebergExternalTable extends ExternalTable {
-
-    // https://iceberg.apache.org/spec/#schemas-and-data-types
-    // All time and timestamp values are stored with microsecond precision
-    public static final int ICEBERG_DATETIME_SCALE_MS = 6;
-
     public IcebergExternalTable(long id, String name, String dbName, IcebergExternalCatalog catalog) {
         super(id, name, catalog, dbName, TableType.ICEBERG_EXTERNAL_TABLE);
     }
@@ -65,66 +53,8 @@ public class IcebergExternalTable extends ExternalTable {
 
     @Override
     public List<Column> initSchema() {
-        return HiveMetaStoreClientHelper.ugiDoAs(catalog.getConfiguration(), () -> {
-            Schema schema = ((IcebergExternalCatalog) catalog).getIcebergTable(dbName, name).schema();
-            List<Types.NestedField> columns = schema.columns();
-            List<Column> tmpSchema = Lists.newArrayListWithCapacity(columns.size());
-            for (Types.NestedField field : columns) {
-                tmpSchema.add(new Column(field.name().toLowerCase(Locale.ROOT),
-                        icebergTypeToDorisType(field.type()), true, null, true, field.doc(), true,
-                        schema.caseInsensitiveFindField(field.name()).fieldId()));
-            }
-            return tmpSchema;
-        });
-    }
-
-    private Type icebergPrimitiveTypeToDorisType(org.apache.iceberg.types.Type.PrimitiveType primitive) {
-        switch (primitive.typeId()) {
-            case BOOLEAN:
-                return Type.BOOLEAN;
-            case INTEGER:
-                return Type.INT;
-            case LONG:
-                return Type.BIGINT;
-            case FLOAT:
-                return Type.FLOAT;
-            case DOUBLE:
-                return Type.DOUBLE;
-            case STRING:
-            case BINARY:
-            case UUID:
-                return Type.STRING;
-            case FIXED:
-                Types.FixedType fixed = (Types.FixedType) primitive;
-                return ScalarType.createCharType(fixed.length());
-            case DECIMAL:
-                Types.DecimalType decimal = (Types.DecimalType) primitive;
-                return ScalarType.createDecimalV3Type(decimal.precision(), decimal.scale());
-            case DATE:
-                return ScalarType.createDateV2Type();
-            case TIMESTAMP:
-                return ScalarType.createDatetimeV2Type(ICEBERG_DATETIME_SCALE_MS);
-            case TIME:
-                return Type.UNSUPPORTED;
-            default:
-                throw new IllegalArgumentException("Cannot transform unknown type: " + primitive);
-        }
-    }
-
-    protected Type icebergTypeToDorisType(org.apache.iceberg.types.Type type) {
-        if (type.isPrimitiveType()) {
-            return icebergPrimitiveTypeToDorisType((org.apache.iceberg.types.Type.PrimitiveType) type);
-        }
-        switch (type.typeId()) {
-            case LIST:
-                Types.ListType list = (Types.ListType) type;
-                return ArrayType.create(icebergTypeToDorisType(list.elementType()), true);
-            case MAP:
-            case STRUCT:
-                return Type.UNSUPPORTED;
-            default:
-                throw new IllegalArgumentException("Cannot transform unknown type: " + type);
-        }
+        makeSureInitialized();
+        return IcebergUtils.getSchema(catalog, dbName, name);
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java
index 56fffc41ddd..ebe8d692c75 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java
@@ -358,14 +358,14 @@ public class HiveMetaStoreCache {
     }
 
     // Get File Status by using FileSystem API.
-    private FileCacheValue getFileCache(String location, InputFormat<?, ?> inputFormat,
-                                        JobConf jobConf,
-                                        List<String> partitionValues,
-                                        String bindBrokerName) throws UserException {
+    private FileCacheValue getFileCache(String location, String inputFormat,
+            JobConf jobConf,
+            List<String> partitionValues,
+            String bindBrokerName) throws UserException {
         FileCacheValue result = new FileCacheValue();
         RemoteFileSystem fs = Env.getCurrentEnv().getExtMetaCacheMgr().getFsCache().getRemoteFileSystem(
                 new FileSystemCache.FileSystemCacheKey(LocationPath.getFSIdentity(
-                    location, bindBrokerName), jobConf, bindBrokerName));
+                        location, bindBrokerName), jobConf, bindBrokerName));
         result.setSplittable(HiveUtil.isSplittable(fs, inputFormat, location, jobConf));
         try {
             // For Tez engine, it may generate subdirectoies for "union" query.
@@ -425,12 +425,12 @@ public class HiveMetaStoreCache {
             FileInputFormat.setInputPaths(jobConf, finalLocation.get());
             try {
                 FileCacheValue result;
-                InputFormat<?, ?> inputFormat = HiveUtil.getInputFormat(jobConf, key.inputFormat, false);
                 // TODO: This is a temp config, will remove it after the HiveSplitter is stable.
                 if (key.useSelfSplitter) {
-                    result = getFileCache(finalLocation.get(), inputFormat, jobConf,
-                        key.getPartitionValues(), key.bindBrokerName);
+                    result = getFileCache(finalLocation.get(), key.inputFormat, jobConf,
+                            key.getPartitionValues(), key.bindBrokerName);
                 } else {
+                    InputFormat<?, ?> inputFormat = HiveUtil.getInputFormat(jobConf, key.inputFormat, false);
                     InputSplit[] splits;
                     String remoteUser = jobConf.get(HdfsResource.HADOOP_USER_NAME);
                     if (!Strings.isNullOrEmpty(remoteUser)) {
@@ -1082,7 +1082,7 @@ public class HiveMetaStoreCache {
         private static boolean isGeneratedPath(String name) {
             return "_temporary".equals(name) // generated by spark
                     || "_imapala_insert_staging".equals(name) // generated by impala
-                    || name.startsWith(".hive-staging"); // generated by hive
+                    || name.startsWith("."); // generated by hive or hidden file
         }
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java
index c8ff468ab29..fadc60913be 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java
@@ -106,6 +106,6 @@ public abstract class IcebergExternalCatalog extends ExternalCatalog {
         return Env.getCurrentEnv()
                 .getExtMetaCacheMgr()
                 .getIcebergMetadataCache()
-                .getIcebergTable(catalog, id, dbName, tblName, getProperties());
+                .getIcebergTable(this, dbName, tblName);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/hive/util/HiveUtil.java b/fe/fe-core/src/main/java/org/apache/doris/external/hive/util/HiveUtil.java
index deb048b5943..7ed44620a37 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/hive/util/HiveUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/hive/util/HiveUtil.java
@@ -21,16 +21,14 @@ import org.apache.doris.catalog.ArrayType;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.Type;
+import org.apache.doris.catalog.external.HMSExternalTable;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.UserException;
-import org.apache.doris.fs.FileSystemFactory;
 import org.apache.doris.fs.remote.BrokerFileSystem;
 import org.apache.doris.fs.remote.RemoteFileSystem;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat;
 import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
@@ -46,10 +44,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import java.io.IOException;
 import java.io.UnsupportedEncodingException;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
 import java.net.URLDecoder;
 import java.nio.charset.StandardCharsets;
 import java.util.List;
@@ -134,7 +129,6 @@ public final class HiveUtil {
     }
 
     private static Type convertHiveTypeToiveDoris(TypeInfo hiveTypeInfo) {
-
         switch (hiveTypeInfo.getCategory()) {
             case PRIMITIVE: {
                 PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) hiveTypeInfo;
@@ -190,39 +184,14 @@ public final class HiveUtil {
         }
     }
 
-    public static boolean isSplittable(RemoteFileSystem remoteFileSystem, InputFormat<?, ?> inputFormat,
-                                       String location, JobConf jobConf) throws UserException {
+    public static boolean isSplittable(RemoteFileSystem remoteFileSystem, String inputFormat,
+            String location, JobConf jobConf) throws UserException {
         if (remoteFileSystem instanceof BrokerFileSystem) {
-            return ((BrokerFileSystem) remoteFileSystem)
-                    .isSplittable(location, inputFormat.getClass().getCanonicalName());
-        }
-
-        // ORC uses a custom InputFormat but is always splittable
-        if (inputFormat.getClass().getSimpleName().equals("OrcInputFormat")) {
-            return true;
-        }
-        // use reflection to get isSplitable method on FileInputFormat
-        // ATTN: the method name is actually "isSplitable", but the right spell is "isSplittable"
-        Method method = null;
-        for (Class<?> clazz = inputFormat.getClass(); clazz != null; clazz = clazz.getSuperclass()) {
-            try {
-                method = clazz.getDeclaredMethod("isSplitable", FileSystem.class, Path.class);
-                break;
-            } catch (NoSuchMethodException ignored) {
-                LOG.debug("Class {} doesn't contain isSplitable method.", clazz);
-            }
+            return ((BrokerFileSystem) remoteFileSystem).isSplittable(location, inputFormat);
         }
 
-        if (method == null) {
-            return false;
-        }
-        Path path = new Path(location);
-        try {
-            method.setAccessible(true);
-            return (boolean) method.invoke(inputFormat, FileSystemFactory.getNativeByPath(path, jobConf), path);
-        } catch (InvocationTargetException | IllegalAccessException | IOException e) {
-            throw new RuntimeException(e);
-        }
+        // All supported hive input format are splittable
+        return HMSExternalTable.SUPPORTED_HIVE_FILE_FORMATS.contains(inputFormat);
     }
 
     public static String getHivePartitionValue(String part) {
@@ -236,5 +205,4 @@ public final class HiveUtil {
             throw new RuntimeException(e);
         }
     }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/iceberg/util/IcebergUtils.java b/fe/fe-core/src/main/java/org/apache/doris/external/iceberg/util/IcebergUtils.java
index 78a0df2ee6d..8a6864aba3a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/iceberg/util/IcebergUtils.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/iceberg/util/IcebergUtils.java
@@ -33,9 +33,17 @@ import org.apache.doris.analysis.NullLiteral;
 import org.apache.doris.analysis.SlotRef;
 import org.apache.doris.analysis.StringLiteral;
 import org.apache.doris.analysis.Subquery;
+import org.apache.doris.catalog.ArrayType;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.Env;
+import org.apache.doris.catalog.HiveMetaStoreClientHelper;
+import org.apache.doris.catalog.ScalarType;
+import org.apache.doris.catalog.Type;
 import org.apache.doris.common.util.TimeUtils;
+import org.apache.doris.datasource.ExternalCatalog;
 import org.apache.doris.thrift.TExprOpcode;
 
+import com.google.common.collect.Lists;
 import org.apache.iceberg.Schema;
 import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.expressions.Expressions;
@@ -45,19 +53,17 @@ import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Locale;
 
 /**
  * Iceberg utils
  */
 public class IcebergUtils {
     private static final Logger LOG = LogManager.getLogger(IcebergUtils.class);
-    private static ThreadLocal<Integer> columnIdThreadLocal = new ThreadLocal<Integer>() {
-        @Override
-        public Integer initialValue() {
-            return 0;
-        }
-    };
-    static long MILLIS_TO_NANO_TIME = 1000;
+    private static long MILLIS_TO_NANO_TIME = 1000;
+    // https://iceberg.apache.org/spec/#schemas-and-data-types
+    // All time and timestamp values are stored with microsecond precision
+    private static final int ICEBERG_DATETIME_SCALE_MS = 6;
 
     public static Expression convertToIcebergExpr(Expr expr, Schema schema) {
         if (expr == null) {
@@ -238,4 +244,74 @@ public class IcebergUtils {
         }
         return slotRef;
     }
+
+    private static Type icebergPrimitiveTypeToDorisType(org.apache.iceberg.types.Type.PrimitiveType primitive) {
+        switch (primitive.typeId()) {
+            case BOOLEAN:
+                return Type.BOOLEAN;
+            case INTEGER:
+                return Type.INT;
+            case LONG:
+                return Type.BIGINT;
+            case FLOAT:
+                return Type.FLOAT;
+            case DOUBLE:
+                return Type.DOUBLE;
+            case STRING:
+            case BINARY:
+            case UUID:
+                return Type.STRING;
+            case FIXED:
+                Types.FixedType fixed = (Types.FixedType) primitive;
+                return ScalarType.createCharType(fixed.length());
+            case DECIMAL:
+                Types.DecimalType decimal = (Types.DecimalType) primitive;
+                return ScalarType.createDecimalV3Type(decimal.precision(), decimal.scale());
+            case DATE:
+                return ScalarType.createDateV2Type();
+            case TIMESTAMP:
+                return ScalarType.createDatetimeV2Type(ICEBERG_DATETIME_SCALE_MS);
+            case TIME:
+                return Type.UNSUPPORTED;
+            default:
+                throw new IllegalArgumentException("Cannot transform unknown type: " + primitive);
+        }
+    }
+
+    public static Type icebergTypeToDorisType(org.apache.iceberg.types.Type type) {
+        if (type.isPrimitiveType()) {
+            return icebergPrimitiveTypeToDorisType((org.apache.iceberg.types.Type.PrimitiveType) type);
+        }
+        switch (type.typeId()) {
+            case LIST:
+                Types.ListType list = (Types.ListType) type;
+                return ArrayType.create(icebergTypeToDorisType(list.elementType()), true);
+            case MAP:
+            case STRUCT:
+                return Type.UNSUPPORTED;
+            default:
+                throw new IllegalArgumentException("Cannot transform unknown type: " + type);
+        }
+    }
+
+    /**
+     * Get iceberg schema from catalog and convert them to doris schema
+     */
+    public static List<Column> getSchema(ExternalCatalog catalog, String dbName, String name) {
+        return HiveMetaStoreClientHelper.ugiDoAs(catalog.getConfiguration(), () -> {
+            org.apache.iceberg.Table icebergTable = Env.getCurrentEnv()
+                    .getExtMetaCacheMgr()
+                    .getIcebergMetadataCache()
+                    .getIcebergTable(catalog, dbName, name);
+            Schema schema = icebergTable.schema();
+            List<Types.NestedField> columns = schema.columns();
+            List<Column> tmpSchema = Lists.newArrayListWithCapacity(columns.size());
+            for (Types.NestedField field : columns) {
+                tmpSchema.add(new Column(field.name().toLowerCase(Locale.ROOT),
+                        IcebergUtils.icebergTypeToDorisType(field.type()), true, null, true, field.doc(), true,
+                        schema.caseInsensitiveFindField(field.name()).fieldId()));
+            }
+            return tmpSchema;
+        });
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java
index 1a39f729738..9b5added1b9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java
@@ -148,6 +148,10 @@ public class MTMVTask extends AbstractTask {
         LOG.info("mtmv task run, taskId: {}", super.getTaskId());
         try {
             ConnectContext ctx = MTMVPlanUtil.createMTMVContext(mtmv);
+            if (LOG.isDebugEnabled()) {
+                String taskSessionContext = ctx.getSessionVariable().toJson().toJSONString();
+                LOG.debug("mtmv task session variable, taskId: {}, session: {}", super.getTaskId(), taskSessionContext);
+            }
             // Every time a task is run, the relation is regenerated because baseTables and baseViews may change,
             // such as deleting a table and creating a view with the same name
             this.relation = MTMVPlanUtil.generateMTMVRelation(mtmv, ctx);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergApiSource.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergApiSource.java
index 73ac8ed7b39..6eed310927c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergApiSource.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergApiSource.java
@@ -24,7 +24,6 @@ import org.apache.doris.catalog.external.IcebergExternalTable;
 import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.UserException;
 import org.apache.doris.datasource.ExternalCatalog;
-import org.apache.doris.datasource.iceberg.IcebergExternalCatalog;
 import org.apache.doris.planner.ColumnRange;
 import org.apache.doris.thrift.TFileAttributes;
 
@@ -48,11 +47,9 @@ public class IcebergApiSource implements IcebergSource {
         this.icebergExtTable = table;
 
         this.originTable = Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache().getIcebergTable(
-            ((IcebergExternalCatalog) icebergExtTable.getCatalog()).getCatalog(),
-            icebergExtTable.getCatalog().getId(),
-            icebergExtTable.getDbName(),
-            icebergExtTable.getName(),
-            icebergExtTable.getCatalog().getProperties());
+                icebergExtTable.getCatalog(),
+                icebergExtTable.getDbName(),
+                icebergExtTable.getName());
 
         this.desc = desc;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergHMSSource.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergHMSSource.java
index 478e78c0d0e..62c96930f00 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergHMSSource.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergHMSSource.java
@@ -47,7 +47,9 @@ public class IcebergHMSSource implements IcebergSource {
         this.desc = desc;
         this.columnNameToRange = columnNameToRange;
         this.icebergTable =
-            Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache().getIcebergTable(hmsTable);
+                Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache()
+                        .getIcebergTable(hmsTable.getCatalog(),
+                                hmsTable.getDbName(), hmsTable.getName());
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergMetadataCache.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergMetadataCache.java
index 91a208202d0..9f273ca6305 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergMetadataCache.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergMetadataCache.java
@@ -19,9 +19,7 @@ package org.apache.doris.planner.external.iceberg;
 
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.HiveMetaStoreClientHelper;
-import org.apache.doris.catalog.external.HMSExternalTable;
 import org.apache.doris.common.Config;
-import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.UserException;
 import org.apache.doris.datasource.CatalogIf;
 import org.apache.doris.datasource.HMSExternalCatalog;
@@ -72,76 +70,44 @@ public class IcebergMetadataCache {
             return ifPresent;
         }
 
-        Table icebergTable = getIcebergTable(key, catalog, params.getDatabase(), params.getTable());
+        Table icebergTable = getIcebergTable(catalog, params.getDatabase(), params.getTable());
         List<Snapshot> snaps = Lists.newArrayList();
         Iterables.addAll(snaps, icebergTable.snapshots());
         snapshotListCache.put(key, snaps);
         return snaps;
     }
 
-    public Table getIcebergTable(IcebergMetadataCacheKey key, CatalogIf catalog, String dbName, String tbName)
-            throws UserException {
+    public Table getIcebergTable(CatalogIf catalog, String dbName, String tbName) {
+        IcebergMetadataCacheKey key = IcebergMetadataCacheKey.of(catalog.getId(), dbName, tbName);
         Table cacheTable = tableCache.getIfPresent(key);
         if (cacheTable != null) {
             return cacheTable;
         }
 
-        Table icebergTable;
+        Catalog icebergCatalog;
         if (catalog instanceof HMSExternalCatalog) {
             HMSExternalCatalog ctg = (HMSExternalCatalog) catalog;
-            icebergTable = createIcebergTable(
-                ctg.getHiveMetastoreUris(),
-                ctg.getCatalogProperty().getHadoopProperties(),
-                dbName,
-                tbName,
-                ctg.getProperties());
+            icebergCatalog = createIcebergHiveCatalog(
+                    ctg.getHiveMetastoreUris(),
+                    ctg.getCatalogProperty().getHadoopProperties(),
+                    ctg.getProperties());
         } else if (catalog instanceof IcebergExternalCatalog) {
-            IcebergExternalCatalog extCatalog = (IcebergExternalCatalog) catalog;
-            icebergTable = getIcebergTable(
-                extCatalog.getCatalog(), extCatalog.getId(), dbName, tbName, extCatalog.getProperties());
+            icebergCatalog = ((IcebergExternalCatalog) catalog).getCatalog();
         } else {
-            throw new UserException("Only support 'hms' and 'iceberg' type for iceberg table");
-        }
-        tableCache.put(key, icebergTable);
-        return icebergTable;
-    }
-
-    public Table getIcebergTable(IcebergSource icebergSource) throws MetaNotFoundException {
-        return icebergSource.getIcebergTable();
-    }
-
-    public Table getIcebergTable(HMSExternalTable hmsTable) {
-        IcebergMetadataCacheKey key = IcebergMetadataCacheKey.of(
-                hmsTable.getCatalog().getId(),
-                hmsTable.getDbName(),
-                hmsTable.getName());
-        Table table = tableCache.getIfPresent(key);
-        if (table != null) {
-            return table;
+            throw new RuntimeException("Only support 'hms' and 'iceberg' type for iceberg table");
         }
-        Table icebergTable = createIcebergTable(hmsTable);
+        Table icebergTable = HiveMetaStoreClientHelper.ugiDoAs(catalog.getId(),
+                () -> icebergCatalog.loadTable(TableIdentifier.of(dbName, tbName)));
+        initIcebergTableFileIO(icebergTable, catalog.getProperties());
         tableCache.put(key, icebergTable);
-
         return icebergTable;
     }
 
-    public Table getIcebergTable(Catalog catalog, long catalogId, String dbName, String tbName,
+    private Table getIcebergTable(Catalog catalog, long catalogId, String dbName, String tbName,
             Map<String, String> props) {
-        IcebergMetadataCacheKey key = IcebergMetadataCacheKey.of(
-                catalogId,
-                dbName,
-                tbName);
-        Table cacheTable = tableCache.getIfPresent(key);
-        if (cacheTable != null) {
-            return cacheTable;
-        }
-
         Table table = HiveMetaStoreClientHelper.ugiDoAs(catalogId,
                 () -> catalog.loadTable(TableIdentifier.of(dbName, tbName)));
         initIcebergTableFileIO(table, props);
-
-        tableCache.put(key, table);
-
         return table;
     }
 
@@ -190,14 +156,12 @@ public class IcebergMetadataCache {
                 });
     }
 
-    private Table createIcebergTable(String uri, Map<String, String> hdfsConf, String db, String tbl,
-            Map<String, String> props) {
+    private Catalog createIcebergHiveCatalog(String uri, Map<String, String> hdfsConf, Map<String, String> props) {
         // set hdfs configure
         Configuration conf = new HdfsConfiguration();
         for (Map.Entry<String, String> entry : hdfsConf.entrySet()) {
             conf.set(entry.getKey(), entry.getValue());
         }
-
         HiveCatalog hiveCatalog = new HiveCatalog();
         hiveCatalog.setConf(conf);
 
@@ -211,20 +175,10 @@ public class IcebergMetadataCache {
             catalogProperties.put("uri", uri);
             hiveCatalog.initialize("hive", catalogProperties);
         }
-        Table table = HiveMetaStoreClientHelper.ugiDoAs(conf, () -> hiveCatalog.loadTable(TableIdentifier.of(db, tbl)));
-        initIcebergTableFileIO(table, props);
-        return table;
-    }
-
-    private Table createIcebergTable(HMSExternalTable hmsTable) {
-        return createIcebergTable(hmsTable.getMetastoreUri(),
-            hmsTable.getHadoopProperties(),
-            hmsTable.getDbName(),
-            hmsTable.getName(),
-            hmsTable.getCatalogProperties());
+        return hiveCatalog;
     }
 
-    private void initIcebergTableFileIO(Table table, Map<String, String> props) {
+    private static void initIcebergTableFileIO(Table table, Map<String, String> props) {
         Map<String, String> ioConf = new HashMap<>();
         table.properties().forEach((key, value) -> {
             if (key.startsWith("io.")) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanNode.java
index a335ccfa021..18b745e402a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanNode.java
@@ -22,7 +22,6 @@ import org.apache.doris.analysis.FunctionCallExpr;
 import org.apache.doris.analysis.TableSnapshot;
 import org.apache.doris.analysis.TupleDescriptor;
 import org.apache.doris.catalog.Column;
-import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.HdfsResource;
 import org.apache.doris.catalog.HiveMetaStoreClientHelper;
 import org.apache.doris.catalog.TableIf;
@@ -128,7 +127,7 @@ public class IcebergScanNode extends FileQueryScanNode {
 
     @Override
     protected void doInitialize() throws UserException {
-        icebergTable = Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache().getIcebergTable(source);
+        icebergTable = source.getIcebergTable();
         super.doInitialize();
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java
index 40ea594819c..65c4800c585 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java
@@ -609,7 +609,7 @@ public class StatisticsUtil {
             Table icebergTable = Env.getCurrentEnv()
                     .getExtMetaCacheMgr()
                     .getIcebergMetadataCache()
-                    .getIcebergTable(table);
+                    .getIcebergTable(table.getCatalog(), table.getDbName(), table.getName());
             TableScan tableScan = icebergTable.newScan().includeColumnStats();
             for (FileScanTask task : tableScan.planFiles()) {
                 rowCount += task.file().recordCount();


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


(doris) 07/19: [fix](Nereids) div priority is not right (#30575)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit ebb04cd7f4e8f6f5914f739e5c02307033236996
Author: morrySnow <10...@users.noreply.github.com>
AuthorDate: Wed Jan 31 14:58:44 2024 +0800

    [fix](Nereids) div priority is not right (#30575)
---
 .../antlr4/org/apache/doris/nereids/DorisParser.g4 |   4 +-
 regression-test/data/nereids_arith_p0/integer.out  | 414 +++++++++++----------
 .../suites/nereids_arith_p0/integer.groovy         |  12 +
 3 files changed, 236 insertions(+), 194 deletions(-)

diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
index 3f2ae5a87d6..3943a30ee2a 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
@@ -612,8 +612,8 @@ predicate
 valueExpression
     : primaryExpression                                                                      #valueExpressionDefault
     | operator=(SUBTRACT | PLUS | TILDE) valueExpression                                     #arithmeticUnary
-    | left=valueExpression operator=(ASTERISK | SLASH | MOD) right=valueExpression           #arithmeticBinary
-    | left=valueExpression operator=(PLUS | SUBTRACT | DIV | HAT | PIPE | AMPERSAND)
+    | left=valueExpression operator=(ASTERISK | SLASH | MOD | DIV) right=valueExpression     #arithmeticBinary
+    | left=valueExpression operator=(PLUS | SUBTRACT | HAT | PIPE | AMPERSAND)
                            right=valueExpression                                             #arithmeticBinary
     | left=valueExpression comparisonOperator right=valueExpression                          #comparison
     | operator=(BITAND | BITOR | BITXOR) LEFT_PAREN left = valueExpression
diff --git a/regression-test/data/nereids_arith_p0/integer.out b/regression-test/data/nereids_arith_p0/integer.out
index dc9077fbfdc..d2e1904fd65 100644
--- a/regression-test/data/nereids_arith_p0/integer.out
+++ b/regression-test/data/nereids_arith_p0/integer.out
@@ -20724,56 +20724,56 @@
 
 -- !sql_test_LargeInt_DecimalV2_0 --
 \N	\N	\N
-1	1.07090669395E8	1.07090620605E8
-2	2.13965679484E8	2.13965610516E8
-3	4.27715693756E8	4.27715596244E8
-4	8.55215713943E8	8.55215576057E8
-5	1.710215742494E9	1.710215547506E9
-6	3.420215782874E9	3.420215507126E9
-7	6.84021583998E9	6.84021545002E9
-8	1.3680215920741E10	1.3680215369259E10
-9	2.7360216034955E10	2.7360215255045E10
-10	5.4720216196479E10	5.4720215093521E10
-11	1.09440216424908E11	1.09440214865092E11
-12	2.18880216747957E11	2.18880214542043E11
-13	1.07090669395E8	1.07090620605E8
-14	2.13965679484E8	2.13965610516E8
-15	4.27715693756E8	4.27715596244E8
-16	8.55215713943E8	8.55215576057E8
-17	1.710215742494E9	1.710215547506E9
-18	3.420215782874E9	3.420215507126E9
-19	6.84021583998E9	6.84021545002E9
-20	1.3680215920741E10	1.3680215369259E10
-21	2.7360216034955E10	2.7360215255045E10
-22	5.4720216196479E10	5.4720215093521E10
-23	1.09440216424908E11	1.09440214865092E11
-24	2.18880216747957E11	2.18880214542043E11
+1	107090669	107090621
+2	213965679	213965611
+3	427715694	427715596
+4	855215714	855215576
+5	1710215742	1710215548
+6	3420215783	3420215507
+7	6840215840	6840215450
+8	13680215921	13680215369
+9	27360216035	27360215255
+10	54720216196	54720215094
+11	109440216425	109440214865
+12	218880216748	218880214542
+13	107090669	107090621
+14	213965679	213965611
+15	427715694	427715596
+16	855215714	855215576
+17	1710215742	1710215548
+18	3420215783	3420215507
+19	6840215840	6840215450
+20	13680215921	13680215369
+21	27360216035	27360215255
+22	54720216196	54720215094
+23	109440216425	109440214865
+24	218880216748	218880214542
 
 -- !sql_test_LargeInt_DecimalV2_notn_0 --
-1	1.07090669395E8	1.07090620605E8
-2	2.13965679484E8	2.13965610516E8
-3	4.27715693756E8	4.27715596244E8
-4	8.55215713943E8	8.55215576057E8
-5	1.710215742494E9	1.710215547506E9
-6	3.420215782874E9	3.420215507126E9
-7	6.84021583998E9	6.84021545002E9
-8	1.3680215920741E10	1.3680215369259E10
-9	2.7360216034955E10	2.7360215255045E10
-10	5.4720216196479E10	5.4720215093521E10
-11	1.09440216424908E11	1.09440214865092E11
-12	2.18880216747957E11	2.18880214542043E11
-13	1.07090669395E8	1.07090620605E8
-14	2.13965679484E8	2.13965610516E8
-15	4.27715693756E8	4.27715596244E8
-16	8.55215713943E8	8.55215576057E8
-17	1.710215742494E9	1.710215547506E9
-18	3.420215782874E9	3.420215507126E9
-19	6.84021583998E9	6.84021545002E9
-20	1.3680215920741E10	1.3680215369259E10
-21	2.7360216034955E10	2.7360215255045E10
-22	5.4720216196479E10	5.4720215093521E10
-23	1.09440216424908E11	1.09440214865092E11
-24	2.18880216747957E11	2.18880214542043E11
+1	107090669	107090621
+2	213965679	213965611
+3	427715694	427715596
+4	855215714	855215576
+5	1710215742	1710215548
+6	3420215783	3420215507
+7	6840215840	6840215450
+8	13680215921	13680215369
+9	27360216035	27360215255
+10	54720216196	54720215094
+11	109440216425	109440214865
+12	218880216748	218880214542
+13	107090669	107090621
+14	213965679	213965611
+15	427715694	427715596
+16	855215714	855215576
+17	1710215742	1710215548
+18	3420215783	3420215507
+19	6840215840	6840215450
+20	13680215921	13680215369
+21	27360216035	27360215255
+22	54720216196	54720215094
+23	109440216425	109440214865
+24	218880216748	218880214542
 
 -- !sql_test_LargeInt_DecimalV2_1 --
 \N	\N	\N	\N
@@ -20936,56 +20936,56 @@
 
 -- !sql_test_LargeInt_Decimal32V3_0 --
 \N	\N	\N
-1	1.07090657012E8	1.07090632988E8
-2	2.13965668123E8	2.13965621877E8
-3	4.27715679234E8	4.27715610766E8
-4	8.55215690345E8	8.55215599655E8
-5	1.710215701456E9	1.710215588544E9
-6	3.420215712567E9	3.420215577433E9
-7	6.840215723678E9	6.840215566322E9
-8	1.3680215734789E10	1.3680215555211E10
-9	2.73602157459E10	2.73602155441E10
-10	5.4720215757011E10	5.4720215532989E10
-11	1.09440215768122E11	1.09440215521878E11
-12	2.18880215779233E11	2.18880215510767E11
-13	1.07090790344E8	1.07090499656E8
-14	2.13965801455E8	2.13965488545E8
-15	4.27715812566E8	4.27715477434E8
-16	8.55215823677E8	8.55215466323E8
-17	1.710215834788E9	1.710215455212E9
-18	3.420215845899E9	3.420215444101E9
-19	6.84021585701E9	6.84021543299E9
-20	1.3680215868121E10	1.3680215421879E10
-21	2.7360215879232E10	2.7360215410768E10
-22	5.4720215890343E10	5.4720215399657E10
-23	1.09440215901454E11	1.09440215388546E11
-24	2.18880215912565E11	2.18880215377435E11
+1	107090657	107090633
+2	213965668	213965622
+3	427715679	427715611
+4	855215690	855215600
+5	1710215701	1710215589
+6	3420215713	3420215577
+7	6840215724	6840215566
+8	13680215735	13680215555
+9	27360215746	27360215544
+10	54720215757	54720215533
+11	109440215768	109440215522
+12	218880215779	218880215511
+13	107090790	107090500
+14	213965801	213965489
+15	427715813	427715477
+16	855215824	855215466
+17	1710215835	1710215455
+18	3420215846	3420215444
+19	6840215857	6840215433
+20	13680215868	13680215422
+21	27360215879	27360215411
+22	54720215890	54720215400
+23	109440215901	109440215389
+24	218880215913	218880215377
 
 -- !sql_test_LargeInt_Decimal32V3_notn_0 --
-1	1.07090657012E8	1.07090632988E8
-2	2.13965668123E8	2.13965621877E8
-3	4.27715679234E8	4.27715610766E8
-4	8.55215690345E8	8.55215599655E8
-5	1.710215701456E9	1.710215588544E9
-6	3.420215712567E9	3.420215577433E9
-7	6.840215723678E9	6.840215566322E9
-8	1.3680215734789E10	1.3680215555211E10
-9	2.73602157459E10	2.73602155441E10
-10	5.4720215757011E10	5.4720215532989E10
-11	1.09440215768122E11	1.09440215521878E11
-12	2.18880215779233E11	2.18880215510767E11
-13	1.07090790344E8	1.07090499656E8
-14	2.13965801455E8	2.13965488545E8
-15	4.27715812566E8	4.27715477434E8
-16	8.55215823677E8	8.55215466323E8
-17	1.710215834788E9	1.710215455212E9
-18	3.420215845899E9	3.420215444101E9
-19	6.84021585701E9	6.84021543299E9
-20	1.3680215868121E10	1.3680215421879E10
-21	2.7360215879232E10	2.7360215410768E10
-22	5.4720215890343E10	5.4720215399657E10
-23	1.09440215901454E11	1.09440215388546E11
-24	2.18880215912565E11	2.18880215377435E11
+1	107090657	107090633
+2	213965668	213965622
+3	427715679	427715611
+4	855215690	855215600
+5	1710215701	1710215589
+6	3420215713	3420215577
+7	6840215724	6840215566
+8	13680215735	13680215555
+9	27360215746	27360215544
+10	54720215757	54720215533
+11	109440215768	109440215522
+12	218880215779	218880215511
+13	107090790	107090500
+14	213965801	213965489
+15	427715813	427715477
+16	855215824	855215466
+17	1710215835	1710215455
+18	3420215846	3420215444
+19	6840215857	6840215433
+20	13680215868	13680215422
+21	27360215879	27360215411
+22	54720215890	54720215400
+23	109440215901	109440215389
+24	218880215913	218880215377
 
 -- !sql_test_LargeInt_Decimal32V3_2 --
 \N	\N
@@ -21095,56 +21095,56 @@
 
 -- !sql_test_LargeInt_Decimal64V3_0 --
 \N	\N	\N
-1	1.0709187901234E8	1.0708941098766E8
-2	2.1396799012345E8	2.1396329987655E8
-3	4.2771910123456E8	4.2771218876544E8
-4	8.5522021234567E8	8.5521107765433E8
-5	1.71022132345678E9	1.71020996654322E9
-6	3.42022243456789E9	3.42020885543211E9
-7	6.840223545679E9	6.840207744321E9
-8	1.368022465679011E10	1.368020663320989E10
-9	2.736022576790122E10	2.736020552209878E10
-10	5.472022687901233E10	5.472020441098767E10
-11	1.0944022799012344E11	1.0944020329987656E11
-12	2.1888022910123456E11	2.1888020218876544E11
-13	1.0710521234566E8	1.0707607765434E8
-14	2.1398132345677E8	2.1394996654323E8
-15	4.2773243456788E8	4.2769885543212E8
-16	8.5523354567899E8	8.5519774432101E8
-17	1.7102346567901E9	1.7101966332099E9
-18	3.42023576790121E9	3.42019552209879E9
-19	6.84023687901232E9	6.84019441098768E9
-20	1.368023799012343E10	1.368019329987657E10
-21	2.736023910123454E10	2.736019218876546E10
-22	5.472024021234565E10	5.472019107765435E10
-23	1.0944024132345676E11	1.0944018996654324E11
-24	2.1888024243456787E11	2.1888018885543213E11
+1	107091879	107089411
+2	213967990	213963300
+3	427719101	427712189
+4	855220212	855211078
+5	1710221323	1710209967
+6	3420222435	3420208855
+7	6840223546	6840207744
+8	13680224657	13680206633
+9	27360225768	27360205522
+10	54720226879	54720204411
+11	109440227990	109440203300
+12	218880229101	218880202189
+13	107105212	107076078
+14	213981323	213949967
+15	427732435	427698855
+16	855233546	855197744
+17	1710234657	1710196633
+18	3420235768	3420195522
+19	6840236879	6840194411
+20	13680237990	13680193300
+21	27360239101	27360192189
+22	54720240212	54720191078
+23	109440241323	109440189967
+24	218880242435	218880188855
 
 -- !sql_test_LargeInt_Decimal64V3_notn_0 --
-1	1.0709187901234E8	1.0708941098766E8
-2	2.1396799012345E8	2.1396329987655E8
-3	4.2771910123456E8	4.2771218876544E8
-4	8.5522021234567E8	8.5521107765433E8
-5	1.71022132345678E9	1.71020996654322E9
-6	3.42022243456789E9	3.42020885543211E9
-7	6.840223545679E9	6.840207744321E9
-8	1.368022465679011E10	1.368020663320989E10
-9	2.736022576790122E10	2.736020552209878E10
-10	5.472022687901233E10	5.472020441098767E10
-11	1.0944022799012344E11	1.0944020329987656E11
-12	2.1888022910123456E11	2.1888020218876544E11
-13	1.0710521234566E8	1.0707607765434E8
-14	2.1398132345677E8	2.1394996654323E8
-15	4.2773243456788E8	4.2769885543212E8
-16	8.5523354567899E8	8.5519774432101E8
-17	1.7102346567901E9	1.7101966332099E9
-18	3.42023576790121E9	3.42019552209879E9
-19	6.84023687901232E9	6.84019441098768E9
-20	1.368023799012343E10	1.368019329987657E10
-21	2.736023910123454E10	2.736019218876546E10
-22	5.472024021234565E10	5.472019107765435E10
-23	1.0944024132345676E11	1.0944018996654324E11
-24	2.1888024243456787E11	2.1888018885543213E11
+1	107091879	107089411
+2	213967990	213963300
+3	427719101	427712189
+4	855220212	855211078
+5	1710221323	1710209967
+6	3420222435	3420208855
+7	6840223546	6840207744
+8	13680224657	13680206633
+9	27360225768	27360205522
+10	54720226879	54720204411
+11	109440227990	109440203300
+12	218880229101	218880202189
+13	107105212	107076078
+14	213981323	213949967
+15	427732435	427698855
+16	855233546	855197744
+17	1710234657	1710196633
+18	3420235768	3420195522
+19	6840236879	6840194411
+20	13680237990	13680193300
+21	27360239101	27360192189
+22	54720240212	54720191078
+23	109440241323	109440189967
+24	218880242435	218880188855
 
 -- !sql_test_LargeInt_Decimal64V3_2 --
 \N	\N
@@ -21254,56 +21254,56 @@
 
 -- !sql_test_LargeInt_Decimal128V3_0 --
 \N	\N	\N
-1	1.19436323012345E8	9.4744966987655E7
-2	2.37422434123456E8	1.90508855876544E8
-3	4.62283545234567E8	3.93147744765433E8
-4	9.00894656345678E8	8.09536633654322E8
-5	1.767005767456789E9	1.653425522543211E9
-6	3.4881168785679E9	3.3523144114321E9
-7	6.919227989679011E9	6.761203300320989E9
-8	1.3770339100790121E10	1.3590092189209879E10
-9	2.7461450211901234E10	2.7258981078098766E10
-10	5.4832561323012344E10	5.4607869966987656E10
-11	1.0956367243412346E11	1.0931675885587654E11
-12	2.1901478354523456E11	2.1874564774476544E11
-13	2.52769656345677E8	-3.858836634567699E7
-14	3.70755767456788E8	5.7175522543212E7
-15	5.95616878567899E8	2.59814411432101E8
-16	1.03422798967901E9	6.7620330032099E8
-17	1.900339100790121E9	1.520092189209879E9
-18	3.621450211901232E9	3.218981078098768E9
-19	7.052561323012343E9	6.627869966987657E9
-20	1.3903672434123453E10	1.3456758855876547E10
-21	2.7594783545234566E10	2.7125647744765434E10
-22	5.496589465634567E10	5.447453663365433E10
-23	1.0969700576745679E11	1.0918342552254321E11
-24	2.191481168785679E11	2.186123144114321E11
+1	119436323	94744967
+2	237422434	190508856
+3	462283545	393147745
+4	900894656	809536634
+5	1767005767	1653425523
+6	3488116879	3352314411
+7	6919227990	6761203300
+8	13770339101	13590092189
+9	27461450212	27258981078
+10	54832561323	54607869967
+11	109563672434	109316758856
+12	219014783545	218745647745
+13	252769656	-38588366
+14	370755767	57175523
+15	595616879	259814411
+16	1034227990	676203300
+17	1900339101	1520092189
+18	3621450212	3218981078
+19	7052561323	6627869967
+20	13903672434	13456758856
+21	27594783545	27125647745
+22	54965894656	54474536634
+23	109697005767	109183425523
+24	219148116879	218612314411
 
 -- !sql_test_LargeInt_Decimal128V3_notn_0 --
-1	1.19436323012345E8	9.4744966987655E7
-2	2.37422434123456E8	1.90508855876544E8
-3	4.62283545234567E8	3.93147744765433E8
-4	9.00894656345678E8	8.09536633654322E8
-5	1.767005767456789E9	1.653425522543211E9
-6	3.4881168785679E9	3.3523144114321E9
-7	6.919227989679011E9	6.761203300320989E9
-8	1.3770339100790121E10	1.3590092189209879E10
-9	2.7461450211901234E10	2.7258981078098766E10
-10	5.4832561323012344E10	5.4607869966987656E10
-11	1.0956367243412346E11	1.0931675885587654E11
-12	2.1901478354523456E11	2.1874564774476544E11
-13	2.52769656345677E8	-3.858836634567699E7
-14	3.70755767456788E8	5.7175522543212E7
-15	5.95616878567899E8	2.59814411432101E8
-16	1.03422798967901E9	6.7620330032099E8
-17	1.900339100790121E9	1.520092189209879E9
-18	3.621450211901232E9	3.218981078098768E9
-19	7.052561323012343E9	6.627869966987657E9
-20	1.3903672434123453E10	1.3456758855876547E10
-21	2.7594783545234566E10	2.7125647744765434E10
-22	5.496589465634567E10	5.447453663365433E10
-23	1.0969700576745679E11	1.0918342552254321E11
-24	2.191481168785679E11	2.186123144114321E11
+1	119436323	94744967
+2	237422434	190508856
+3	462283545	393147745
+4	900894656	809536634
+5	1767005767	1653425523
+6	3488116879	3352314411
+7	6919227990	6761203300
+8	13770339101	13590092189
+9	27461450212	27258981078
+10	54832561323	54607869967
+11	109563672434	109316758856
+12	219014783545	218745647745
+13	252769656	-38588366
+14	370755767	57175523
+15	595616879	259814411
+16	1034227990	676203300
+17	1900339101	1520092189
+18	3621450212	3218981078
+19	7052561323	6627869967
+20	13903672434	13456758856
+21	27594783545	27125647745
+22	54965894656	54474536634
+23	109697005767	109183425523
+24	219148116879	218612314411
 
 -- !sql_test_LargeInt_Decimal128V3_2 --
 \N	\N
@@ -23213,3 +23213,33 @@
 23	1	109440215645	109440215644
 24	1	218880215645	218880215644
 
+-- !test_div_priority_1 --
+7
+
+-- !test_div_priority_2 --
+3
+
+-- !test_div_priority_3 --
+10
+
+-- !test_div_priority_4 --
+2.5
+
+-- !test_div_priority_5 --
+1
+
+-- !test_div_priority_6 --
+7
+
+-- !test_div_priority_7 --
+-3
+
+-- !test_div_priority_8 --
+10
+
+-- !test_div_priority_9 --
+0
+
+-- !test_div_priority_10 --
+1
+
diff --git a/regression-test/suites/nereids_arith_p0/integer.groovy b/regression-test/suites/nereids_arith_p0/integer.groovy
index 0157613c483..e934705cfca 100644
--- a/regression-test/suites/nereids_arith_p0/integer.groovy
+++ b/regression-test/suites/nereids_arith_p0/integer.groovy
@@ -2299,4 +2299,16 @@ suite('nereids_arith_p0_integer') {
 		select id, BITAND(klint, kbool), BITOR(klint, kbool), BITXOR(klint, kbool) from expr_test order by id"""
 	qt_sql_test_LargeInt_Boolean_notn_4 """
 		select id, BITAND(klint, kbool), BITOR(klint, kbool), BITXOR(klint, kbool) from expr_test_not_nullable order by id"""
+
+	qt_test_div_priority_1 """select 10 div 2 + 2"""
+	qt_test_div_priority_2 """select 10 div 2 - 2"""
+	qt_test_div_priority_3 """select 10 div 2 * 2"""
+	qt_test_div_priority_4 """select 10 div 2 / 2"""
+	qt_test_div_priority_5 """select 10 div 2 % 2"""
+	qt_test_div_priority_6 """select 2 + 10 div 2"""
+	qt_test_div_priority_7 """select 2 - 10 div 2"""
+	qt_test_div_priority_8 """select 2 * 10 div 2"""
+	qt_test_div_priority_9 """select 2 / 10 div 2"""
+	qt_test_div_priority_10 """select 2 % 10 div 2"""
+
 }
\ No newline at end of file


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


(doris) 14/19: [fix](insert into) 'output_tuple_slot_num should be equal to output_expr_num' when insert into unique table with sequence column map (#30635)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit e4fa19bb078f327416d4e57e62ffbba20e66b6b1
Author: meiyi <my...@gmail.com>
AuthorDate: Wed Jan 31 20:18:42 2024 +0800

    [fix](insert into) 'output_tuple_slot_num should be equal to output_expr_num' when insert into unique table with sequence column map (#30635)
---
 .../src/main/java/org/apache/doris/analysis/NativeInsertStmt.java    | 5 +++--
 .../src/main/java/org/apache/doris/datasource/InternalCatalog.java   | 2 +-
 .../suites/correctness_p0/test_sequence_col_default_value.groovy     | 2 +-
 3 files changed, 5 insertions(+), 4 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java
index 96dbb2e0edf..f2a6ee607c0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java
@@ -929,10 +929,11 @@ public class NativeInsertStmt extends InsertStmt {
                         && col.getName().equals(Column.SEQUENCE_COL)
                         && ((OlapTable) targetTable).getSequenceMapCol() != null) {
                     if (resultExprByName.stream().map(Pair::key)
-                            .anyMatch(key -> key.equals(((OlapTable) targetTable).getSequenceMapCol()))) {
+                            .anyMatch(key -> key.equalsIgnoreCase(((OlapTable) targetTable).getSequenceMapCol()))) {
                         resultExprByName.add(Pair.of(Column.SEQUENCE_COL,
                                 resultExprByName.stream()
-                                        .filter(p -> p.key().equals(((OlapTable) targetTable).getSequenceMapCol()))
+                                        .filter(p -> p.key()
+                                                .equalsIgnoreCase(((OlapTable) targetTable).getSequenceMapCol()))
                                         .map(Pair::value).findFirst().orElse(null)));
                     }
                     continue;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
index e0a10d4349e..485b5cdad7b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
@@ -2434,7 +2434,7 @@ public class InternalCatalog implements CatalogIf<Database> {
                 if (!col.getType().isFixedPointType() && !col.getType().isDateType()) {
                     throw new DdlException("Sequence type only support integer types and date types");
                 }
-                olapTable.setSequenceMapCol(sequenceMapCol);
+                olapTable.setSequenceMapCol(col.getName());
                 olapTable.setSequenceInfo(col.getType());
             }
         } catch (Exception e) {
diff --git a/regression-test/suites/correctness_p0/test_sequence_col_default_value.groovy b/regression-test/suites/correctness_p0/test_sequence_col_default_value.groovy
index eb30f3d8f6a..9c3f9828e2d 100644
--- a/regression-test/suites/correctness_p0/test_sequence_col_default_value.groovy
+++ b/regression-test/suites/correctness_p0/test_sequence_col_default_value.groovy
@@ -36,7 +36,7 @@ suite("test_sequence_col_default_value") {
         PROPERTIES (
             "replication_allocation" = "tag.location.default: 1",
             "enable_unique_key_merge_on_write" = "true",
-            "function_column.sequence_col" = 'write_time'
+            "function_column.sequence_col" = 'WRITE_TIME'
         );
     """
 


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


(doris) 12/19: [fix](join) incorrect result of mark join (#30543)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit b911ebc4ad84fa78a2bfe94dd1c2eff52c2af7f4
Author: Jerry Hu <mr...@gmail.com>
AuthorDate: Wed Jan 31 17:59:27 2024 +0800

    [fix](join) incorrect result of mark join (#30543)
    
     incorrect result of mark join
---
 be/src/pipeline/exec/hashjoin_build_sink.cpp       |   8 -
 be/src/pipeline/exec/hashjoin_build_sink.h         |   7 -
 be/src/pipeline/exec/hashjoin_probe_operator.cpp   |   1 -
 be/src/pipeline/exec/hashjoin_probe_operator.h     |   7 -
 be/src/vec/common/hash_table/join_hash_table.h     | 101 ++++++------
 be/src/vec/exec/join/process_hash_table_probe.h    |  19 ++-
 .../vec/exec/join/process_hash_table_probe_impl.h  | 172 +++++++++++++--------
 be/src/vec/exec/join/vhash_join_node.cpp           |   8 -
 be/src/vec/exec/join/vhash_join_node.h             |  16 +-
 .../data/nereids_p0/join/test_mark_join.out        |  43 ++++++
 .../suites/nereids_p0/join/test_mark_join.groovy   | 126 +++++++++++++++
 11 files changed, 352 insertions(+), 156 deletions(-)

diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp
index 5b6689e35a5..5c8995bebe9 100644
--- a/be/src/pipeline/exec/hashjoin_build_sink.cpp
+++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp
@@ -76,12 +76,6 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo
         _shared_hash_table_dependency->block();
         p._shared_hashtable_controller->append_dependency(p.node_id(),
                                                           _shared_hash_table_dependency);
-    } else {
-        if ((p._join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN ||
-             p._join_op == TJoinOp::NULL_AWARE_LEFT_SEMI_JOIN) &&
-            p._have_other_join_conjunct) {
-            _build_indexes_null = std::make_shared<std::vector<uint32_t>>();
-        }
     }
 
     _build_blocks_memory_usage =
@@ -496,7 +490,6 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block*
                 state, local_state._shared_state->build_block.get(), &local_state, use_global_rf));
         RETURN_IF_ERROR(
                 local_state.process_build_block(state, (*local_state._shared_state->build_block)));
-        local_state._shared_state->build_indexes_null = local_state._build_indexes_null;
         if (_shared_hashtable_controller) {
             _shared_hash_table_context->status = Status::OK();
             // arena will be shared with other instances.
@@ -542,7 +535,6 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block*
                         _shared_hash_table_context->hash_table_variants));
 
         local_state._shared_state->build_block = _shared_hash_table_context->block;
-        local_state._build_indexes_null = _shared_hash_table_context->build_indexes_null;
         local_state._shared_state->build_indexes_null =
                 _shared_hash_table_context->build_indexes_null;
         const bool use_global_rf =
diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h
index 2acc25151ab..efc0a46f3ba 100644
--- a/be/src/pipeline/exec/hashjoin_build_sink.h
+++ b/be/src/pipeline/exec/hashjoin_build_sink.h
@@ -120,13 +120,6 @@ protected:
     std::shared_ptr<SharedHashTableDependency> _shared_hash_table_dependency;
     std::vector<int> _build_col_ids;
 
-    /*
-     * For null aware anti/semi join with other join conjuncts, we do need to care about the rows in
-     * build side with null keys,
-     * because the other join conjuncts' result may be changed from null to false(null & false == false).
-     */
-    std::shared_ptr<std::vector<uint32_t>> _build_indexes_null;
-
     RuntimeProfile::Counter* _build_table_timer = nullptr;
     RuntimeProfile::Counter* _build_expr_call_timer = nullptr;
     RuntimeProfile::Counter* _build_table_insert_timer = nullptr;
diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp
index f7a06655b19..a2fb0012ffa 100644
--- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp
+++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp
@@ -301,7 +301,6 @@ Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc
 
     Status st;
     if (local_state._probe_index < local_state._probe_block.rows()) {
-        local_state._build_indexes_null = local_state._shared_state->build_indexes_null;
         DCHECK(local_state._has_set_need_null_map_for_probe);
         RETURN_IF_CATCH_EXCEPTION({
             std::visit(
diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.h b/be/src/pipeline/exec/hashjoin_probe_operator.h
index 1bdb9864c40..4b7f9271920 100644
--- a/be/src/pipeline/exec/hashjoin_probe_operator.h
+++ b/be/src/pipeline/exec/hashjoin_probe_operator.h
@@ -125,13 +125,6 @@ private:
     // For mark join, last probe index of null mark
     int _last_probe_null_mark;
 
-    /*
-     * For null aware anti/semi join with other join conjuncts, we do need to care about the rows in
-     * build side with null keys,
-     * because the other join conjuncts' result may be changed from null to false(null & false == false).
-     */
-    std::shared_ptr<std::vector<uint32_t>> _build_indexes_null;
-
     vectorized::Block _probe_block;
     vectorized::ColumnRawPtrs _probe_columns;
     // other expr
diff --git a/be/src/vec/common/hash_table/join_hash_table.h b/be/src/vec/common/hash_table/join_hash_table.h
index 08311989b5d..85665e76853 100644
--- a/be/src/vec/common/hash_table/join_hash_table.h
+++ b/be/src/vec/common/hash_table/join_hash_table.h
@@ -68,6 +68,7 @@ public:
 
     std::vector<uint8_t>& get_visited() { return visited; }
 
+    template <int JoinOpType, bool with_other_conjuncts>
     void build(const Key* __restrict keys, const uint32_t* __restrict bucket_nums,
                size_t num_elem) {
         build_keys = keys;
@@ -76,19 +77,24 @@ public:
             next[i] = first[bucket_num];
             first[bucket_num] = i;
         }
-        first[bucket_size] = 0; // index = bucket_num means null
+        if constexpr ((JoinOpType != TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN &&
+                       JoinOpType != TJoinOp::NULL_AWARE_LEFT_SEMI_JOIN) ||
+                      !with_other_conjuncts) {
+            /// Only null aware join with other conjuncts need to access the null value in hash table
+            first[bucket_size] = 0; // index = bucket_num means null
+        }
     }
 
     template <int JoinOpType, bool with_other_conjuncts, bool is_mark_join, bool need_judge_null>
     auto find_batch(const Key* __restrict keys, const uint32_t* __restrict build_idx_map,
                     int probe_idx, uint32_t build_idx, int probe_rows,
                     uint32_t* __restrict probe_idxs, bool& probe_visited,
-                    uint32_t* __restrict build_idxs, vectorized::ColumnFilterHelper* mark_column) {
+                    uint32_t* __restrict build_idxs) {
         if constexpr (JoinOpType == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) {
             if (_empty_build_side) {
                 return _process_null_aware_left_anti_join_for_empty_build_side<
-                        JoinOpType, with_other_conjuncts, is_mark_join>(
-                        probe_idx, probe_rows, probe_idxs, build_idxs, mark_column);
+                        JoinOpType, with_other_conjuncts, is_mark_join>(probe_idx, probe_rows,
+                                                                        probe_idxs, build_idxs);
             }
         }
 
@@ -128,51 +134,48 @@ public:
      * select 'a' not in ('b', null) => null => 'a' != 'b' and 'a' != null => true and null => null
      * select 'a' not in ('a', 'b', null) => false
      */
-    auto find_null_aware_with_other_conjuncts(
-            const Key* __restrict keys, const uint32_t* __restrict build_idx_map, int probe_idx,
-            uint32_t build_idx, int probe_rows, uint32_t* __restrict probe_idxs,
-            uint32_t* __restrict build_idxs, std::set<uint32_t>& null_result,
-            const std::vector<uint32_t>& build_indexes_null, const size_t build_block_count) {
+    auto find_null_aware_with_other_conjuncts(const Key* __restrict keys,
+                                              const uint32_t* __restrict build_idx_map,
+                                              int probe_idx, uint32_t build_idx, int probe_rows,
+                                              uint32_t* __restrict probe_idxs,
+                                              uint32_t* __restrict build_idxs,
+                                              uint8_t* __restrict null_flags,
+                                              bool picking_null_keys) {
         auto matched_cnt = 0;
         const auto batch_size = max_batch_size;
 
-        bool has_matched = false;
         auto do_the_probe = [&]() {
+            /// If no any rows match the probe key, here start to handle null keys in build side.
+            /// The result of "Any = null" is null.
+            if (build_idx == 0 && !picking_null_keys) {
+                build_idx = first[bucket_size];
+                picking_null_keys = true; // now pick null from build side
+            }
+
             while (build_idx && matched_cnt < batch_size) {
-                if (build_idx == bucket_size) {
-                    /// All rows in build side should be executed with other join conjuncts.
-                    for (size_t i = 1; i != build_block_count; ++i) {
-                        build_idxs[matched_cnt] = i;
-                        probe_idxs[matched_cnt] = probe_idx;
-                        matched_cnt++;
-                    }
-                    null_result.emplace(probe_idx);
-                    build_idx = 0;
-                    has_matched = true;
-                    break;
-                } else if (keys[probe_idx] == build_keys[build_idx]) {
+                if (picking_null_keys || keys[probe_idx] == build_keys[build_idx]) {
                     build_idxs[matched_cnt] = build_idx;
                     probe_idxs[matched_cnt] = probe_idx;
+                    null_flags[matched_cnt] = picking_null_keys;
                     matched_cnt++;
-                    has_matched = true;
                 }
 
                 build_idx = next[build_idx];
+
+                // If `build_idx` is 0, all matched keys are handled,
+                // now need to handle null keys in build side.
+                if (!build_idx && !picking_null_keys) {
+                    build_idx = first[bucket_size];
+                    picking_null_keys = true; // now pick null keys from build side
+                }
             }
 
             // may over batch_size when emplace 0 into build_idxs
             if (!build_idx) {
-                if (!has_matched) { // has no any row matched
-                    for (auto index : build_indexes_null) {
-                        build_idxs[matched_cnt] = index;
-                        probe_idxs[matched_cnt] = probe_idx;
-                        matched_cnt++;
-                    }
-                }
                 probe_idxs[matched_cnt] = probe_idx;
                 build_idxs[matched_cnt] = 0;
+                picking_null_keys = false;
                 matched_cnt++;
-                has_matched = false;
             }
 
             probe_idx++;
@@ -184,11 +187,20 @@ public:
 
         while (probe_idx < probe_rows && matched_cnt < batch_size) {
             build_idx = build_idx_map[probe_idx];
+
+            /// If the probe key is null
+            if (build_idx == bucket_size) {
+                probe_idx++;
+                break;
+            }
             do_the_probe();
+            if (picking_null_keys) {
+                break;
+            }
         }
 
         probe_idx -= (build_idx != 0);
-        return std::tuple {probe_idx, build_idx, matched_cnt};
+        return std::tuple {probe_idx, build_idx, matched_cnt, picking_null_keys};
     }
 
     template <int JoinOpType>
@@ -215,21 +227,23 @@ public:
 
     bool has_null_key() { return _has_null_key; }
 
-    void pre_build_idxs(std::vector<uint32>& bucksets, const uint8_t* null_map) {
+    void pre_build_idxs(std::vector<uint32>& buckets, const uint8_t* null_map) {
         if (null_map) {
-            first[bucket_size] = bucket_size; // distinguish between not matched and null
-        }
-
-        for (uint32_t i = 0; i < bucksets.size(); i++) {
-            bucksets[i] = first[bucksets[i]];
+            for (unsigned int& bucket : buckets) {
+                bucket = bucket == bucket_size ? bucket_size : first[bucket];
+            }
+        } else {
+            for (unsigned int& bucket : buckets) {
+                bucket = first[bucket];
+            }
         }
     }
 
 private:
     template <int JoinOpType, bool with_other_conjuncts, bool is_mark_join>
-    auto _process_null_aware_left_anti_join_for_empty_build_side(
-            int probe_idx, int probe_rows, uint32_t* __restrict probe_idxs,
-            uint32_t* __restrict build_idxs, vectorized::ColumnFilterHelper* mark_column) {
+    auto _process_null_aware_left_anti_join_for_empty_build_side(int probe_idx, int probe_rows,
+                                                                 uint32_t* __restrict probe_idxs,
+                                                                 uint32_t* __restrict build_idxs) {
         static_assert(JoinOpType == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN);
         auto matched_cnt = 0;
         const auto batch_size = max_batch_size;
@@ -240,11 +254,6 @@ private:
             ++matched_cnt;
         }
 
-        if constexpr (is_mark_join && !with_other_conjuncts) {
-            // we will flip the mark column later for anti join, so here set 0 into mark column.
-            mark_column->resize_fill(matched_cnt, 0);
-        }
-
         return std::tuple {probe_idx, 0U, matched_cnt};
     }
 
diff --git a/be/src/vec/exec/join/process_hash_table_probe.h b/be/src/vec/exec/join/process_hash_table_probe.h
index 02bf242e55a..9f4ddbabdcb 100644
--- a/be/src/vec/exec/join/process_hash_table_probe.h
+++ b/be/src/vec/exec/join/process_hash_table_probe.h
@@ -67,12 +67,11 @@ struct ProcessHashTableProbe {
     // each matching join column need to be processed by other join conjunct. so the struct of mutable block
     // and output block may be different
     // The output result is determined by the other join conjunct result and same_to_prev struct
-    Status do_other_join_conjuncts(Block* output_block, bool is_mark_join,
-                                   std::vector<uint8_t>& visited, bool has_null_in_build_side);
+    Status do_other_join_conjuncts(Block* output_block, std::vector<uint8_t>& visited,
+                                   bool has_null_in_build_side);
 
     template <bool with_other_conjuncts>
-    Status do_mark_join_conjuncts(Block* output_block, size_t hash_table_bucket_size,
-                                  const std::set<uint32_t>& null_result);
+    Status do_mark_join_conjuncts(Block* output_block, size_t hash_table_bucket_size);
 
     template <typename HashTableType>
     typename HashTableType::State _init_probe_side(HashTableType& hash_table_ctx, size_t probe_rows,
@@ -85,6 +84,10 @@ struct ProcessHashTableProbe {
     Status process_data_in_hashtable(HashTableType& hash_table_ctx, MutableBlock& mutable_block,
                                      Block* output_block, bool* eos);
 
+    /// For null aware join with other conjuncts, if the probe key of one row on left side is null,
+    /// we should make this row match with all rows in build side.
+    size_t _process_probe_null_key(uint32_t probe_idx);
+
     Parent* _parent = nullptr;
     const int _batch_size;
     const std::shared_ptr<Block>& _build_block;
@@ -93,7 +96,15 @@ struct ProcessHashTableProbe {
 
     std::vector<uint32_t> _probe_indexs;
     bool _probe_visited = false;
+    bool _picking_null_keys = false;
     std::vector<uint32_t> _build_indexs;
+    std::vector<uint8_t> _null_flags;
+
+    /// If the probe key of one row on left side is null,
+    /// we will make all rows in build side match with this row,
+    /// `_build_index_for_null_probe_key` is used to record the progress if the build block is too big.
+    uint32_t _build_index_for_null_probe_key {0};
+
     std::vector<int> _build_blocks_locs;
     // only need set the tuple is null in RIGHT_OUTER_JOIN and FULL_OUTER_JOIN
     ColumnUInt8::Container* _tuple_is_null_left_flags = nullptr;
diff --git a/be/src/vec/exec/join/process_hash_table_probe_impl.h b/be/src/vec/exec/join/process_hash_table_probe_impl.h
index 1939b702c69..06dfdac9074 100644
--- a/be/src/vec/exec/join/process_hash_table_probe_impl.h
+++ b/be/src/vec/exec/join/process_hash_table_probe_impl.h
@@ -131,6 +131,11 @@ typename HashTableType::State ProcessHashTableProbe<JoinOpType, Parent>::_init_p
     // may over batch size 1 for some outer join case
     _probe_indexs.resize(_batch_size + 1);
     _build_indexs.resize(_batch_size + 1);
+    if constexpr (JoinOpType == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN ||
+                  JoinOpType == TJoinOp::NULL_AWARE_LEFT_SEMI_JOIN) {
+        _null_flags.resize(_batch_size + 1);
+        memset(_null_flags.data(), 0, _batch_size + 1);
+    }
 
     if (!_parent->_ready_probe) {
         _parent->_ready_probe = true;
@@ -175,26 +180,41 @@ Status ProcessHashTableProbe<JoinOpType, Parent>::do_process(HashTableType& hash
     auto& mcol = mutable_block.mutable_columns();
 
     int current_offset = 0;
-
-    std::unique_ptr<ColumnFilterHelper> mark_column;
-    if (is_mark_join) {
-        mark_column = std::make_unique<ColumnFilterHelper>(*mcol[mcol.size() - 1]);
-    }
-
-    /// `null_result` set which contains the probe indexes of null results.
-    std::set<uint32_t> null_result;
     if constexpr ((JoinOpType == doris::TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN ||
                    JoinOpType == doris::TJoinOp::NULL_AWARE_LEFT_SEMI_JOIN) &&
                   with_other_conjuncts) {
         SCOPED_TIMER(_search_hashtable_timer);
-        auto [new_probe_idx, new_build_idx, new_current_offset] =
-                hash_table_ctx.hash_table->find_null_aware_with_other_conjuncts(
-                        hash_table_ctx.keys, hash_table_ctx.bucket_nums.data(), probe_index,
-                        build_index, probe_rows, _probe_indexs.data(), _build_indexs.data(),
-                        null_result, *(_parent->_build_indexes_null), _build_block->rows());
-        probe_index = new_probe_idx;
-        build_index = new_build_idx;
-        current_offset = new_current_offset;
+
+        /// If `_build_index_for_null_probe_key` is not zero, it means we are in progress of handling probe null key.
+        if (_build_index_for_null_probe_key) {
+            DCHECK_EQ(build_index, hash_table_ctx.hash_table->get_bucket_size());
+            current_offset = _process_probe_null_key(probe_index);
+            if (!_build_index_for_null_probe_key) {
+                probe_index++;
+                build_index = 0;
+            }
+        } else {
+            auto [new_probe_idx, new_build_idx, new_current_offset, picking_null_keys] =
+                    hash_table_ctx.hash_table->find_null_aware_with_other_conjuncts(
+                            hash_table_ctx.keys, hash_table_ctx.bucket_nums.data(), probe_index,
+                            build_index, probe_rows, _probe_indexs.data(), _build_indexs.data(),
+                            _null_flags.data(), _picking_null_keys);
+            probe_index = new_probe_idx;
+            build_index = new_build_idx;
+            current_offset = new_current_offset;
+            _picking_null_keys = picking_null_keys;
+
+            if (build_index == hash_table_ctx.hash_table->get_bucket_size()) {
+                _build_index_for_null_probe_key = 1;
+                if (current_offset == 0) {
+                    current_offset = _process_probe_null_key(probe_index);
+                    if (!_build_index_for_null_probe_key) {
+                        probe_index++;
+                        build_index = 0;
+                    }
+                }
+            }
+        }
     } else {
         SCOPED_TIMER(_search_hashtable_timer);
         auto [new_probe_idx, new_build_idx,
@@ -203,7 +223,7 @@ Status ProcessHashTableProbe<JoinOpType, Parent>::do_process(HashTableType& hash
               need_null_map_for_probe &&
                       ignore_null > (hash_table_ctx.keys, hash_table_ctx.bucket_nums.data(),
                                      probe_index, build_index, probe_rows, _probe_indexs.data(),
-                                     _probe_visited, _build_indexs.data(), mark_column.get());
+                                     _probe_visited, _build_indexs.data());
         probe_index = new_probe_idx;
         build_index = new_build_idx;
         current_offset = new_current_offset;
@@ -235,20 +255,76 @@ Status ProcessHashTableProbe<JoinOpType, Parent>::do_process(HashTableType& hash
 
     if constexpr (is_mark_join) {
         return do_mark_join_conjuncts<with_other_conjuncts>(
-                output_block, hash_table_ctx.hash_table->get_bucket_size(), null_result);
+                output_block, hash_table_ctx.hash_table->get_bucket_size());
     } else if constexpr (with_other_conjuncts) {
-        return do_other_join_conjuncts(output_block, is_mark_join,
-                                       hash_table_ctx.hash_table->get_visited(),
+        return do_other_join_conjuncts(output_block, hash_table_ctx.hash_table->get_visited(),
                                        hash_table_ctx.hash_table->has_null_key());
     }
 
     return Status::OK();
 }
 
+template <int JoinOpType, typename Parent>
+size_t ProcessHashTableProbe<JoinOpType, Parent>::_process_probe_null_key(uint32_t probe_index) {
+    const auto rows = _build_block->rows();
+
+    DCHECK_LT(_build_index_for_null_probe_key, rows);
+    DCHECK_LT(0, _build_index_for_null_probe_key);
+    size_t matched_cnt = 0;
+    for (; _build_index_for_null_probe_key < rows && matched_cnt < _batch_size; ++matched_cnt) {
+        _probe_indexs[matched_cnt] = probe_index;
+        _build_indexs[matched_cnt] = _build_index_for_null_probe_key++;
+        _null_flags[matched_cnt] = 1;
+    }
+
+    if (_build_index_for_null_probe_key == rows) {
+        _build_index_for_null_probe_key = 0;
+        _probe_indexs[matched_cnt] = probe_index;
+        _build_indexs[matched_cnt] = 0;
+        _null_flags[matched_cnt] = 0;
+        matched_cnt++;
+    }
+
+    return matched_cnt;
+}
+
+/**
+     * Mark join: there is a column named mark column which stores the result of mark join conjunct.
+     * For example:
+     * ```sql
+     *  select * from t1 where t1.k1 not in (select t2.k1 from t2 where t2.k2 = t1.k2 and t2.k3 > t1.k3) or t1.k4 < 10;
+     * ```
+     * equal join conjuncts: t2.k2 = t1.k2
+     * mark join conjunct: t1.k1 = t2.k1
+     * other join conjuncts: t2.k3 > t1.k3
+     * other predicates: $c$1 or t1.k4 < 10   # `$c$1` means the result of mark join conjunct(mark column)
+     *
+     * Executing flow:
+     *
+     * Equal join conjuncts (probe hash table)
+     *                  ↓↓
+     * Mark join conjuncts (result is nullable, stored in mark column)
+     *                  ↓↓
+     * Other join conjuncts (update the mark column)
+     *                  ↓↓
+     * Other predicates (filter rows)
+     *
+     * ```sql
+     *   select * from t1 where t1.k1 not in (select t2.k1 from t2 where t2.k3 > t1.k3) or t1.k4 < 10;
+     * ```
+     * This sql has no equal join conjuncts:
+     * equal join conjuncts: NAN
+     * mark join conjunct: t1.k1 = t2.k1
+     * other join conjuncts: t2.k3 > t1.k3
+     * other predicates: $c$1 or t1.k4 < 10   # `$c$1` means the result of mark join conjunct(mark column)
+     *
+     * To avoid using nested loop join, we use the mark join conjunct(`t1.k1 = t2.k1`) as the equal join conjunct.
+     * So this query will be a "null aware left anti join", which means the equal conjunct's result should be nullable.
+     */
 template <int JoinOpType, typename Parent>
 template <bool with_other_conjuncts>
 Status ProcessHashTableProbe<JoinOpType, Parent>::do_mark_join_conjuncts(
-        Block* output_block, size_t hash_table_bucket_size, const std::set<uint32_t>& null_result) {
+        Block* output_block, size_t hash_table_bucket_size) {
     DCHECK(JoinOpType == TJoinOp::LEFT_ANTI_JOIN ||
            JoinOpType == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN ||
            JoinOpType == TJoinOp::LEFT_SEMI_JOIN ||
@@ -260,6 +336,10 @@ Status ProcessHashTableProbe<JoinOpType, Parent>::do_mark_join_conjuncts(
                                         JoinOpType == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN;
 
     const auto row_count = output_block->rows();
+    if (!row_count) {
+        return Status::OK();
+    }
+
     auto mark_column_mutable =
             output_block->get_by_position(_parent->_mark_column_id).column->assume_mutable();
     auto& mark_column = assert_cast<ColumnNullable&>(*mark_column_mutable);
@@ -281,8 +361,7 @@ Status ProcessHashTableProbe<JoinOpType, Parent>::do_mark_join_conjuncts(
             filter_data[i] = _build_indexs[i] != 0 && _build_indexs[i] != hash_table_bucket_size;
             if constexpr (is_null_aware_join) {
                 if constexpr (with_other_conjuncts) {
-                    mark_null_map[i] =
-                            null_result.contains(_probe_indexs[i]) && _build_indexs[i] != 0;
+                    mark_null_map[i] = _null_flags[i];
                 } else {
                     if (filter_data[i]) {
                         last_probe_matched = _probe_indexs[i];
@@ -361,7 +440,7 @@ Status ProcessHashTableProbe<JoinOpType, Parent>::do_mark_join_conjuncts(
     if constexpr (is_anti_join) {
         // flip the mark column
         for (size_t i = 0; i != row_count; ++i) {
-            mark_filter_data[i] ^= 1;
+            mark_filter_data[i] ^= 1; // not null/ null
         }
     }
 
@@ -372,8 +451,7 @@ Status ProcessHashTableProbe<JoinOpType, Parent>::do_mark_join_conjuncts(
 
 template <int JoinOpType, typename Parent>
 Status ProcessHashTableProbe<JoinOpType, Parent>::do_other_join_conjuncts(
-        Block* output_block, bool is_mark_join, std::vector<uint8_t>& visited,
-        bool has_null_in_build_side) {
+        Block* output_block, std::vector<uint8_t>& visited, bool has_null_in_build_side) {
     // dispose the other join conjunct exec
     auto row_count = output_block->rows();
     if (!row_count) {
@@ -440,30 +518,18 @@ Status ProcessHashTableProbe<JoinOpType, Parent>::do_other_join_conjuncts(
         for (size_t i = 0; i < row_count; ++i) {
             bool not_matched_before = _parent->_last_probe_match != _probe_indexs[i];
 
-            // _build_indexs[i] == 0 means the end of this probe index
-            // if a probe row not matched with any build row, we need output a false value into mark column
             if constexpr (JoinOpType == TJoinOp::LEFT_SEMI_JOIN) {
                 if (_build_indexs[i] == 0) {
-                    filter_map[i] = is_mark_join && not_matched_before;
-                    filter_column_ptr[i] = false;
+                    filter_map[i] = false;
+                } else if (filter_column_ptr[i]) {
+                    filter_map[i] = not_matched_before;
+                    _parent->_last_probe_match = _probe_indexs[i];
                 } else {
-                    if (filter_column_ptr[i]) {
-                        filter_map[i] = not_matched_before;
-                        _parent->_last_probe_match = _probe_indexs[i];
-                    } else {
-                        filter_map[i] = false;
-                    }
+                    filter_map[i] = false;
                 }
             } else {
                 if (_build_indexs[i] == 0) {
-                    if (not_matched_before) {
-                        filter_map[i] = true;
-                    } else if (is_mark_join) {
-                        filter_map[i] = true;
-                        filter_column_ptr[i] = false;
-                    } else {
-                        filter_map[i] = false;
-                    }
+                    filter_map[i] = not_matched_before;
                 } else {
                     filter_map[i] = false;
                     if (filter_column_ptr[i]) {
@@ -473,21 +539,6 @@ Status ProcessHashTableProbe<JoinOpType, Parent>::do_other_join_conjuncts(
             }
         }
 
-        if (is_mark_join) {
-            auto mark_column =
-                    output_block->get_by_position(orig_columns - 1).column->assume_mutable();
-            ColumnFilterHelper helper(*mark_column);
-            for (size_t i = 0; i < row_count; ++i) {
-                bool mathced = filter_column_ptr[i] &&
-                               (_build_indexs[i] != 0) == (JoinOpType == TJoinOp::LEFT_SEMI_JOIN);
-                if (has_null_in_build_side && !mathced) {
-                    helper.insert_null();
-                } else {
-                    helper.insert_value(mathced);
-                }
-            }
-        }
-
         output_block->get_by_position(result_column_id).column = std::move(new_filter_column);
     } else if constexpr (JoinOpType == TJoinOp::RIGHT_SEMI_JOIN ||
                          JoinOpType == TJoinOp::RIGHT_ANTI_JOIN) {
@@ -512,8 +563,7 @@ Status ProcessHashTableProbe<JoinOpType, Parent>::do_other_join_conjuncts(
                       JoinOpType == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) {
             orig_columns = _right_col_idx;
         }
-        RETURN_IF_ERROR(Block::filter_block(output_block, result_column_id,
-                                            is_mark_join ? output_block->columns() : orig_columns));
+        RETURN_IF_ERROR(Block::filter_block(output_block, result_column_id, orig_columns));
     }
 
     return Status::OK();
diff --git a/be/src/vec/exec/join/vhash_join_node.cpp b/be/src/vec/exec/join/vhash_join_node.cpp
index a813ec565a4..ec630f3fe32 100644
--- a/be/src/vec/exec/join/vhash_join_node.cpp
+++ b/be/src/vec/exec/join/vhash_join_node.cpp
@@ -180,12 +180,6 @@ Status HashJoinNode::init(const TPlanNode& tnode, RuntimeState* state) {
     }
 #endif
 
-    if ((_join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN ||
-         _join_op == TJoinOp::NULL_AWARE_LEFT_SEMI_JOIN) &&
-        _have_other_join_conjunct) {
-        _build_indexes_null = std::make_shared<std::vector<uint32_t>>();
-    }
-
     _runtime_filters.resize(_runtime_filter_descs.size());
     for (size_t i = 0; i < _runtime_filter_descs.size(); i++) {
         RETURN_IF_ERROR(state->runtime_filter_mgr()->register_producer_filter(
@@ -761,7 +755,6 @@ Status HashJoinNode::sink(doris::RuntimeState* state, vectorized::Block* in_bloc
             // arena will be shared with other instances.
             _shared_hash_table_context->arena = _arena;
             _shared_hash_table_context->block = _build_block;
-            _shared_hash_table_context->build_indexes_null = _build_indexes_null;
             _shared_hash_table_context->hash_table_variants = _hash_table_variants;
             _shared_hash_table_context->short_circuit_for_null_in_probe_side =
                     _has_null_in_build_side;
@@ -794,7 +787,6 @@ Status HashJoinNode::sink(doris::RuntimeState* state, vectorized::Block* in_bloc
                 *std::static_pointer_cast<HashTableVariants>(
                         _shared_hash_table_context->hash_table_variants));
         _build_block = _shared_hash_table_context->block;
-        _build_indexes_null = _shared_hash_table_context->build_indexes_null;
 
         if (!_shared_hash_table_context->runtime_filters.empty()) {
             auto ret = std::visit(
diff --git a/be/src/vec/exec/join/vhash_join_node.h b/be/src/vec/exec/join/vhash_join_node.h
index 95c59094ba6..c38f8f563ea 100644
--- a/be/src/vec/exec/join/vhash_join_node.h
+++ b/be/src/vec/exec/join/vhash_join_node.h
@@ -117,11 +117,6 @@ struct ProcessHashTableBuild {
             for (uint32_t i = 1; i < _rows; i++) {
                 if ((*null_map)[i]) {
                     *has_null_key = true;
-                    if constexpr (with_other_conjuncts &&
-                                  (JoinOpType == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN ||
-                                   JoinOpType == TJoinOp::NULL_AWARE_LEFT_SEMI_JOIN)) {
-                        _parent->_build_indexes_null->emplace_back(i);
-                    }
                 }
             }
             if (short_circuit_for_null && *has_null_key) {
@@ -136,8 +131,8 @@ struct ProcessHashTableBuild {
         hash_table_ctx.init_serialized_keys(_build_raw_ptrs, _rows,
                                             null_map ? null_map->data() : nullptr, true, true,
                                             hash_table_ctx.hash_table->get_bucket_size());
-        hash_table_ctx.hash_table->build(hash_table_ctx.keys, hash_table_ctx.bucket_nums.data(),
-                                         _rows);
+        hash_table_ctx.hash_table->template build<JoinOpType, with_other_conjuncts>(
+                hash_table_ctx.keys, hash_table_ctx.bucket_nums.data(), _rows);
         hash_table_ctx.bucket_nums.resize(_batch_size);
         hash_table_ctx.bucket_nums.shrink_to_fit();
 
@@ -303,13 +298,6 @@ private:
     std::vector<uint16_t> _probe_column_disguise_null;
     std::vector<uint16_t> _probe_column_convert_to_null;
 
-    /*
-     * For null aware anti/semi join with other join conjuncts, we do need to care about the rows in
-     * build side with null keys,
-     * because the other join conjuncts' result maybe change null to false(null & false == false).
-     */
-    std::shared_ptr<std::vector<uint32_t>> _build_indexes_null;
-
     DataTypes _right_table_data_types;
     DataTypes _left_table_data_types;
     std::vector<std::string> _right_table_column_names;
diff --git a/regression-test/data/nereids_p0/join/test_mark_join.out b/regression-test/data/nereids_p0/join/test_mark_join.out
new file mode 100644
index 00000000000..4098502b75d
--- /dev/null
+++ b/regression-test/data/nereids_p0/join/test_mark_join.out
@@ -0,0 +1,43 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !mark_join1 --
+1	1	true
+2	2	true
+3	\N	true
+3	\N	true
+4	\N	\N
+
+-- !mark_join2 --
+1	1	\N
+2	2	\N
+3	\N	\N
+3	\N	true
+4	\N	true
+
+-- !mark_join3 --
+1	1	false
+2	2	false
+3	\N	false
+3	\N	false
+4	\N	false
+
+-- !mark_join4 --
+1	1	false
+2	2	false
+3	\N	\N
+3	\N	true
+4	\N	true
+
+-- !mark_join5 --
+1	1	false
+2	2	false
+3	\N	true
+3	\N	true
+4	\N	\N
+
+-- !mark_join6 --
+1	1	true
+2	2	true
+3	\N	false
+3	\N	true
+4	\N	false
+
diff --git a/regression-test/suites/nereids_p0/join/test_mark_join.groovy b/regression-test/suites/nereids_p0/join/test_mark_join.groovy
new file mode 100644
index 00000000000..6008919d831
--- /dev/null
+++ b/regression-test/suites/nereids_p0/join/test_mark_join.groovy
@@ -0,0 +1,126 @@
+// 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_mark_join", "nereids_p0") {
+    sql "SET enable_nereids_planner=true"
+    sql "SET enable_fallback_to_original_planner=false"
+
+    sql "drop table if exists `test_mark_join_t1`;"
+    sql "drop table if exists `test_mark_join_t2`;"
+
+    sql """
+        CREATE TABLE IF NOT EXISTS `test_mark_join_t1` (
+          k1 int not null,
+          k2 int,
+          k3 bigint,
+          v1 varchar(255) not null,
+          v2 varchar(255),
+          v3 varchar(255)
+        ) ENGINE=OLAP
+        DUPLICATE KEY(`k1`, `k2`)
+        COMMENT "OLAP"
+        DISTRIBUTED BY HASH(`k1`) BUCKETS 3
+          PROPERTIES (
+          "replication_allocation" = "tag.location.default: 1",
+          "in_memory" = "false",
+          "storage_format" = "V2"
+        );
+    """
+
+    sql """
+        CREATE TABLE IF NOT EXISTS `test_mark_join_t2` (
+          k1 int not null,
+          k2 int,
+          k3 bigint,
+          v1 varchar(255) not null,
+          v2 varchar(255),
+          v3 varchar(255)
+        ) ENGINE=OLAP
+        DUPLICATE KEY(`k1`, `k2`)
+        COMMENT "OLAP"
+        DISTRIBUTED BY HASH(`k1`) BUCKETS 3
+          PROPERTIES (
+          "replication_allocation" = "tag.location.default: 1",
+          "in_memory" = "false",
+          "storage_format" = "V2"
+        );
+    """
+
+    sql """
+        insert into `test_mark_join_t1` values
+            (1,     1,      1,      'abc',      'efg',      'hjk'),
+            (2,     2,      2,      'aabb',     'eeff',     'ccdd'),
+            (3,     null,   3,      'iii',      null,       null),
+            (3,     null,   null,   'hhhh',     null,       null),
+            (4,     null,   4,      'dddd',     'ooooo',    'kkkkk'
+        );
+    """
+
+    sql """
+        insert into `test_mark_join_t2` values
+            (1,     1,      1,      'abc',      'efg',      'hjk'),
+            (2,     2,      2,      'aabb',     'eeff',     'ccdd'),
+            (3,     null,   null,   'diid',     null,       null),
+            (3,     null,   3,      'ooekd',    null,       null),
+            (4,     4,   null,   'oepeld',   null,       'kkkkk'
+        );
+    """
+
+    qt_mark_join1 """
+        select
+            k1, k2
+            , k1 not in (select test_mark_join_t2.k2 from test_mark_join_t2 where test_mark_join_t2.k3 < test_mark_join_t1.k3) vv
+        from test_mark_join_t1 order by 1, 2, 3;
+    """
+
+    qt_mark_join2 """
+        select
+            k1, k2
+            , k2 not in (select test_mark_join_t2.k3 from test_mark_join_t2 where test_mark_join_t2.k2 > test_mark_join_t1.k3) vv
+        from test_mark_join_t1 order by 1, 2, 3;
+    """
+
+    qt_mark_join3 """
+        select
+            k1, k2
+            , k1 in (select test_mark_join_t2.k1 from test_mark_join_t2 where test_mark_join_t2.k3 < test_mark_join_t1.k3) vv
+        from test_mark_join_t1 order by 1, 2, 3;
+    """
+
+    qt_mark_join4 """
+        select
+            k1, k2
+            , k1 not in (select test_mark_join_t2.k2 from test_mark_join_t2 where test_mark_join_t2.k3 = test_mark_join_t1.k3) vv
+        from test_mark_join_t1 order by 1, 2, 3;
+    """
+
+    qt_mark_join5 """
+        select
+            k1, k2
+            , k2 not in (select test_mark_join_t2.k3 from test_mark_join_t2 where test_mark_join_t2.k2 = test_mark_join_t1.k3) vv
+        from test_mark_join_t1 order by 1, 2, 3;
+    """
+
+    qt_mark_join6 """
+        select
+            k1, k2
+            , k1 in (select test_mark_join_t2.k1 from test_mark_join_t2 where test_mark_join_t2.k3 = test_mark_join_t1.k3) vv
+        from test_mark_join_t1 order by 1, 2, 3;
+    """
+
+
+}


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


(doris) 02/19: [pipelinex](profile) improve memory counter of pipelineX (#30538)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 9012098bcaa334320ea419300d0d7b453b44c79c
Author: TengJianPing <18...@users.noreply.github.com>
AuthorDate: Wed Jan 31 14:08:08 2024 +0800

    [pipelinex](profile) improve memory counter of pipelineX (#30538)
---
 be/src/pipeline/exec/aggregation_sink_operator.cpp   |  7 +++----
 be/src/pipeline/exec/aggregation_sink_operator.h     |  1 -
 be/src/pipeline/exec/analytic_sink_operator.cpp      |  4 ++--
 be/src/pipeline/exec/analytic_sink_operator.h        |  1 -
 be/src/pipeline/exec/analytic_source_operator.cpp    |  3 +--
 be/src/pipeline/exec/analytic_source_operator.h      |  1 -
 be/src/pipeline/exec/exchange_sink_operator.cpp      |  1 -
 be/src/pipeline/exec/exchange_sink_operator.h        |  1 -
 be/src/pipeline/exec/hashjoin_build_sink.cpp         | 20 ++++++++++++--------
 be/src/pipeline/exec/hashjoin_build_sink.h           |  1 -
 be/src/pipeline/exec/hashjoin_probe_operator.cpp     |  4 +++-
 be/src/pipeline/exec/scan_operator.cpp               |  9 +++++----
 be/src/pipeline/exec/sort_sink_operator.cpp          | 11 ++++++++---
 be/src/pipeline/exec/sort_sink_operator.h            |  2 +-
 be/src/pipeline/pipeline_x/operator.cpp              |  8 ++++----
 be/src/util/runtime_profile.cpp                      |  5 +++--
 be/src/util/runtime_profile.h                        |  6 ++++--
 be/src/vec/common/hash_table/hash_map_context.h      | 14 ++++++++++++++
 be/src/vec/exec/join/process_hash_table_probe_impl.h |  2 ++
 be/src/vec/exec/join/vhash_join_node.h               |  6 ++++--
 20 files changed, 66 insertions(+), 41 deletions(-)

diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp
index 5746551a29f..c5822591d4a 100644
--- a/be/src/pipeline/exec/aggregation_sink_operator.cpp
+++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp
@@ -88,11 +88,10 @@ Status AggSinkLocalState<DependencyType, Derived>::init(RuntimeState* state,
         RETURN_IF_ERROR(
                 p._probe_expr_ctxs[i]->clone(state, Base::_shared_state->probe_expr_ctxs[i]));
     }
-    _memory_usage_counter = ADD_LABEL_COUNTER(Base::profile(), "MemoryUsage");
-    _hash_table_memory_usage =
-            ADD_CHILD_COUNTER(Base::profile(), "HashTable", TUnit::BYTES, "MemoryUsage");
+    _hash_table_memory_usage = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "HashTable",
+                                                            TUnit::BYTES, "MemoryUsage", 1);
     _serialize_key_arena_memory_usage = Base::profile()->AddHighWaterMarkCounter(
-            "SerializeKeyArena", TUnit::BYTES, "MemoryUsage");
+            "SerializeKeyArena", TUnit::BYTES, "MemoryUsage", 1);
 
     _build_timer = ADD_TIMER(Base::profile(), "BuildTime");
     _build_table_convert_timer = ADD_TIMER(Base::profile(), "BuildConvertToPartitionedTime");
diff --git a/be/src/pipeline/exec/aggregation_sink_operator.h b/be/src/pipeline/exec/aggregation_sink_operator.h
index a7b30d46117..8bdd624ca48 100644
--- a/be/src/pipeline/exec/aggregation_sink_operator.h
+++ b/be/src/pipeline/exec/aggregation_sink_operator.h
@@ -308,7 +308,6 @@ protected:
     RuntimeProfile::Counter* _serialize_data_timer = nullptr;
     RuntimeProfile::Counter* _deserialize_data_timer = nullptr;
     RuntimeProfile::Counter* _max_row_size_counter = nullptr;
-    RuntimeProfile::Counter* _memory_usage_counter = nullptr;
     RuntimeProfile::Counter* _hash_table_memory_usage = nullptr;
     RuntimeProfile::HighWaterMarkCounter* _serialize_key_arena_memory_usage = nullptr;
 
diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp
index d9923a68f24..e9cc0cb1d06 100644
--- a/be/src/pipeline/exec/analytic_sink_operator.cpp
+++ b/be/src/pipeline/exec/analytic_sink_operator.cpp
@@ -34,8 +34,8 @@ Status AnalyticSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf
     _shared_state->partition_by_column_idxs.resize(p._partition_by_eq_expr_ctxs.size());
     _shared_state->ordey_by_column_idxs.resize(p._order_by_eq_expr_ctxs.size());
 
-    _memory_usage_counter = ADD_LABEL_COUNTER(profile(), "MemoryUsage");
-    _blocks_memory_usage = _profile->AddHighWaterMarkCounter("Blocks", TUnit::BYTES, "MemoryUsage");
+    _blocks_memory_usage =
+            _profile->AddHighWaterMarkCounter("Blocks", TUnit::BYTES, "MemoryUsage", 1);
     _evaluation_timer = ADD_TIMER(profile(), "EvaluationTime");
 
     size_t agg_size = p._agg_expr_ctxs.size();
diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h
index 064d68f189a..c291aa95226 100644
--- a/be/src/pipeline/exec/analytic_sink_operator.h
+++ b/be/src/pipeline/exec/analytic_sink_operator.h
@@ -84,7 +84,6 @@ private:
                                                      bool need_check_first = false);
     bool _whether_need_next_partition(vectorized::BlockRowPos& found_partition_end);
 
-    RuntimeProfile::Counter* _memory_usage_counter = nullptr;
     RuntimeProfile::Counter* _evaluation_timer = nullptr;
     RuntimeProfile::HighWaterMarkCounter* _blocks_memory_usage = nullptr;
 
diff --git a/be/src/pipeline/exec/analytic_source_operator.cpp b/be/src/pipeline/exec/analytic_source_operator.cpp
index a95c2a1225a..0642b4b76bd 100644
--- a/be/src/pipeline/exec/analytic_source_operator.cpp
+++ b/be/src/pipeline/exec/analytic_source_operator.cpp
@@ -167,9 +167,8 @@ Status AnalyticLocalState::init(RuntimeState* state, LocalStateInfo& info) {
     auto& p = _parent->cast<AnalyticSourceOperatorX>();
     _agg_functions_size = p._agg_functions.size();
 
-    _memory_usage_counter = ADD_LABEL_COUNTER(profile(), "MemoryUsage");
     _blocks_memory_usage =
-            profile()->AddHighWaterMarkCounter("Blocks", TUnit::BYTES, "MemoryUsage");
+            profile()->AddHighWaterMarkCounter("Blocks", TUnit::BYTES, "MemoryUsage", 1);
     _evaluation_timer = ADD_TIMER(profile(), "EvaluationTime");
 
     _agg_functions.resize(p._agg_functions.size());
diff --git a/be/src/pipeline/exec/analytic_source_operator.h b/be/src/pipeline/exec/analytic_source_operator.h
index f4e2f10a719..b4acb2fbce5 100644
--- a/be/src/pipeline/exec/analytic_source_operator.h
+++ b/be/src/pipeline/exec/analytic_source_operator.h
@@ -121,7 +121,6 @@ private:
     std::unique_ptr<vectorized::Arena> _agg_arena_pool;
     std::vector<vectorized::AggFnEvaluator*> _agg_functions;
 
-    RuntimeProfile::Counter* _memory_usage_counter = nullptr;
     RuntimeProfile::Counter* _evaluation_timer = nullptr;
     RuntimeProfile::HighWaterMarkCounter* _blocks_memory_usage = nullptr;
 
diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp
index 1f9ba3b4203..9c26e975918 100644
--- a/be/src/pipeline/exec/exchange_sink_operator.cpp
+++ b/be/src/pipeline/exec/exchange_sink_operator.cpp
@@ -124,7 +124,6 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf
             "");
     _merge_block_timer = ADD_TIMER(profile(), "MergeBlockTime");
     _local_bytes_send_counter = ADD_COUNTER(_profile, "LocalBytesSent", TUnit::BYTES);
-    _memory_usage_counter = ADD_LABEL_COUNTER(_profile, "MemoryUsage");
     static const std::string timer_name = "WaitForDependencyTime";
     _wait_for_dependency_timer = ADD_TIMER_WITH_LEVEL(_profile, timer_name, 1);
     _wait_queue_timer =
diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h
index 3e6486e34fd..5ba8228e502 100644
--- a/be/src/pipeline/exec/exchange_sink_operator.h
+++ b/be/src/pipeline/exec/exchange_sink_operator.h
@@ -178,7 +178,6 @@ private:
     // Used to counter send bytes under local data exchange
     RuntimeProfile::Counter* _local_bytes_send_counter = nullptr;
     RuntimeProfile::Counter* _merge_block_timer = nullptr;
-    RuntimeProfile::Counter* _memory_usage_counter = nullptr;
 
     RuntimeProfile::Counter* _wait_queue_timer = nullptr;
     RuntimeProfile::Counter* _wait_broadcast_buffer_timer = nullptr;
diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp
index ccf2f2a5894..5b6689e35a5 100644
--- a/be/src/pipeline/exec/hashjoin_build_sink.cpp
+++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp
@@ -84,14 +84,12 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo
         }
     }
 
-    _memory_usage_counter = ADD_LABEL_COUNTER(profile(), "MemoryUsage");
-
     _build_blocks_memory_usage =
-            ADD_CHILD_COUNTER(profile(), "BuildBlocks", TUnit::BYTES, "MemoryUsage");
+            ADD_CHILD_COUNTER_WITH_LEVEL(profile(), "BuildBlocks", TUnit::BYTES, "MemoryUsage", 1);
     _hash_table_memory_usage =
-            ADD_CHILD_COUNTER(profile(), "HashTable", TUnit::BYTES, "MemoryUsage");
+            ADD_CHILD_COUNTER_WITH_LEVEL(profile(), "HashTable", TUnit::BYTES, "MemoryUsage", 1);
     _build_arena_memory_usage =
-            profile()->AddHighWaterMarkCounter("BuildKeyArena", TUnit::BYTES, "MemoryUsage");
+            profile()->AddHighWaterMarkCounter("BuildKeyArena", TUnit::BYTES, "MemoryUsage", 1);
 
     // Build phase
     auto* record_profile = _should_build_hash_table ? profile() : faker_runtime_profile();
@@ -271,7 +269,9 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state,
                                                             HashJoinBuildSinkLocalState>
                                   hash_table_build_process(rows, raw_ptrs, this,
                                                            state->batch_size(), state);
-                          return hash_table_build_process.template run<
+                          auto old_hash_table_size = arg.hash_table->get_byte_size();
+                          auto old_key_size = arg.serialized_keys_size(true);
+                          auto st = hash_table_build_process.template run<
                                   JoinOpType::value, has_null_value,
                                   short_circuit_for_null_in_build_side, with_other_conjuncts>(
                                   arg,
@@ -279,6 +279,10 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state,
                                           ? &null_map_val->get_data()
                                           : nullptr,
                                   &_shared_state->_has_null_in_build_side);
+                          _mem_tracker->consume(arg.hash_table->get_byte_size() -
+                                                old_hash_table_size);
+                          _mem_tracker->consume(arg.serialized_keys_size(true) - old_key_size);
+                          return st;
                       }},
             *_shared_state->hash_table_variants, _shared_state->join_op_variants,
             vectorized::make_bool_variant(_build_side_ignore_null),
@@ -469,6 +473,8 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block*
 
             SCOPED_TIMER(local_state._build_side_merge_block_timer);
             RETURN_IF_ERROR(local_state._build_side_mutable_block.merge(*in_block));
+            COUNTER_UPDATE(local_state._build_blocks_memory_usage, in_block->bytes());
+            local_state._mem_tracker->consume(in_block->bytes());
             if (local_state._build_side_mutable_block.rows() >
                 std::numeric_limits<uint32_t>::max()) {
                 return Status::NotSupported(
@@ -483,8 +489,6 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block*
         DCHECK(!local_state._build_side_mutable_block.empty());
         local_state._shared_state->build_block = std::make_shared<vectorized::Block>(
                 local_state._build_side_mutable_block.to_block());
-        COUNTER_UPDATE(local_state._build_blocks_memory_usage,
-                       (*local_state._shared_state->build_block).bytes());
 
         const bool use_global_rf =
                 local_state._parent->cast<HashJoinBuildSinkOperatorX>()._use_global_rf;
diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h
index 7521563ecb8..2acc25151ab 100644
--- a/be/src/pipeline/exec/hashjoin_build_sink.h
+++ b/be/src/pipeline/exec/hashjoin_build_sink.h
@@ -135,7 +135,6 @@ protected:
 
     RuntimeProfile::Counter* _allocate_resource_timer = nullptr;
 
-    RuntimeProfile::Counter* _memory_usage_counter = nullptr;
     RuntimeProfile::Counter* _build_blocks_memory_usage = nullptr;
     RuntimeProfile::Counter* _hash_table_memory_usage = nullptr;
     RuntimeProfile::HighWaterMarkCounter* _build_arena_memory_usage = nullptr;
diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp
index aef2e011fa0..f7a06655b19 100644
--- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp
+++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp
@@ -53,7 +53,7 @@ Status HashJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info)
     _construct_mutable_join_block();
     _probe_column_disguise_null.reserve(_probe_expr_ctxs.size());
     _probe_arena_memory_usage =
-            profile()->AddHighWaterMarkCounter("ProbeKeyArena", TUnit::BYTES, "MemoryUsage");
+            profile()->AddHighWaterMarkCounter("ProbeKeyArena", TUnit::BYTES, "MemoryUsage", 1);
     // Probe phase
     _probe_next_timer = ADD_TIMER(profile(), "ProbeFindNextTime");
     _probe_expr_call_timer = ADD_TIMER(profile(), "ProbeExprCallTime");
@@ -320,6 +320,8 @@ Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc
                                         mutable_join_block, &temp_block,
                                         local_state._probe_block.rows(), _is_mark_join,
                                         _have_other_join_conjunct);
+                                local_state._mem_tracker->set_consumption(
+                                        arg.serialized_keys_size(false));
                             } else {
                                 st = Status::InternalError("uninited hash table");
                             }
diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp
index 49f3bd2269c..88e6c880568 100644
--- a/be/src/pipeline/exec/scan_operator.cpp
+++ b/be/src/pipeline/exec/scan_operator.cpp
@@ -28,6 +28,7 @@
 #include "pipeline/exec/meta_scan_operator.h"
 #include "pipeline/exec/olap_scan_operator.h"
 #include "pipeline/exec/operator.h"
+#include "util/runtime_profile.h"
 #include "vec/exec/runtime_filter_consumer.h"
 #include "vec/exec/scan/pip_scanner_context.h"
 #include "vec/exec/scan/scanner_context.h"
@@ -1286,11 +1287,11 @@ Status ScanLocalState<Derived>::_init_profile() {
     _scanner_profile.reset(new RuntimeProfile("VScanner"));
     profile()->add_child(_scanner_profile.get(), true, nullptr);
 
-    _memory_usage_counter = ADD_LABEL_COUNTER(_scanner_profile, "MemoryUsage");
-    _queued_blocks_memory_usage =
-            _scanner_profile->AddHighWaterMarkCounter("QueuedBlocks", TUnit::BYTES, "MemoryUsage");
+    _memory_usage_counter = ADD_LABEL_COUNTER_WITH_LEVEL(_scanner_profile, "MemoryUsage", 1);
+    _queued_blocks_memory_usage = _scanner_profile->AddHighWaterMarkCounter(
+            "QueuedBlocks", TUnit::BYTES, "MemoryUsage", 1);
     _free_blocks_memory_usage =
-            _scanner_profile->AddHighWaterMarkCounter("FreeBlocks", TUnit::BYTES, "MemoryUsage");
+            _scanner_profile->AddHighWaterMarkCounter("FreeBlocks", TUnit::BYTES, "MemoryUsage", 1);
     _newly_create_free_blocks_num =
             ADD_COUNTER(_scanner_profile, "NewlyCreateFreeBlocksNum", TUnit::UNIT);
     // time of transfer thread to wait for block from scan thread
diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp
index 56a81422484..465622be4aa 100644
--- a/be/src/pipeline/exec/sort_sink_operator.cpp
+++ b/be/src/pipeline/exec/sort_sink_operator.cpp
@@ -63,7 +63,8 @@ Status SortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) {
 
     _profile->add_info_string("TOP-N", p._limit == -1 ? "false" : "true");
 
-    _memory_usage_counter = ADD_LABEL_COUNTER(_profile, "MemoryUsage");
+    _sort_blocks_memory_usage =
+            ADD_CHILD_COUNTER_WITH_LEVEL(_profile, "SortBlocks", TUnit::BYTES, "MemoryUsage", 1);
     return Status::OK();
 }
 
@@ -149,16 +150,20 @@ Status SortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in
     COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows());
     if (in_block->rows() > 0) {
         RETURN_IF_ERROR(local_state._shared_state->sorter->append_block(in_block));
+        local_state._mem_tracker->set_consumption(local_state._shared_state->sorter->data_size());
+        COUNTER_SET(local_state._sort_blocks_memory_usage,
+                    (int64_t)local_state._shared_state->sorter->data_size());
         RETURN_IF_CANCELLED(state);
 
         // update runtime predicate
         if (_use_topn_opt) {
             vectorized::Field new_top = local_state._shared_state->sorter->get_top_value();
             if (!new_top.is_null() && new_top != local_state.old_top) {
-                auto& sort_description = local_state._shared_state->sorter->get_sort_description();
+                const auto& sort_description =
+                        local_state._shared_state->sorter->get_sort_description();
                 auto col = in_block->get_by_position(sort_description[0].column_number);
                 bool is_reverse = sort_description[0].direction < 0;
-                auto query_ctx = state->get_query_ctx();
+                auto* query_ctx = state->get_query_ctx();
                 RETURN_IF_ERROR(
                         query_ctx->get_runtime_predicate().update(new_top, col.name, is_reverse));
                 local_state.old_top = std::move(new_top);
diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h
index 7069183f3b2..d3d85a3e5c9 100644
--- a/be/src/pipeline/exec/sort_sink_operator.h
+++ b/be/src/pipeline/exec/sort_sink_operator.h
@@ -72,7 +72,7 @@ private:
     // Expressions and parameters used for build _sort_description
     vectorized::VSortExecExprs _vsort_exec_exprs;
 
-    RuntimeProfile::Counter* _memory_usage_counter = nullptr;
+    RuntimeProfile::Counter* _sort_blocks_memory_usage = nullptr;
 
     // topn top value
     vectorized::Field old_top {vectorized::Field::Types::Null};
diff --git a/be/src/pipeline/pipeline_x/operator.cpp b/be/src/pipeline/pipeline_x/operator.cpp
index ef17ee70ed6..4be062b9e8c 100644
--- a/be/src/pipeline/pipeline_x/operator.cpp
+++ b/be/src/pipeline/pipeline_x/operator.cpp
@@ -403,9 +403,9 @@ Status PipelineXLocalState<DependencyType>::init(RuntimeState* state, LocalState
     _close_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "CloseTime", 1);
     _exec_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "ExecTime", 1);
     _mem_tracker = std::make_unique<MemTracker>("PipelineXLocalState:" + _runtime_profile->name());
-    _memory_used_counter = ADD_LABEL_COUNTER(_runtime_profile, "MemoryUsage");
+    _memory_used_counter = ADD_LABEL_COUNTER_WITH_LEVEL(_runtime_profile, "MemoryUsage", 1);
     _peak_memory_usage_counter = _runtime_profile->AddHighWaterMarkCounter(
-            "PeakMemoryUsage", TUnit::BYTES, "MemoryUsage");
+            "PeakMemoryUsage", TUnit::BYTES, "MemoryUsage", 1);
     return Status::OK();
 }
 
@@ -462,9 +462,9 @@ Status PipelineXSinkLocalState<DependencyType>::init(RuntimeState* state,
     _exec_timer = ADD_TIMER_WITH_LEVEL(_profile, "ExecTime", 1);
     info.parent_profile->add_child(_profile, true, nullptr);
     _mem_tracker = std::make_unique<MemTracker>(_parent->get_name());
-    _memory_used_counter = ADD_LABEL_COUNTER(_profile, "MemoryUsage");
+    _memory_used_counter = ADD_LABEL_COUNTER_WITH_LEVEL(_profile, "MemoryUsage", 1);
     _peak_memory_usage_counter =
-            _profile->AddHighWaterMarkCounter("PeakMemoryUsage", TUnit::BYTES, "MemoryUsage");
+            _profile->AddHighWaterMarkCounter("PeakMemoryUsage", TUnit::BYTES, "MemoryUsage", 1);
     return Status::OK();
 }
 
diff --git a/be/src/util/runtime_profile.cpp b/be/src/util/runtime_profile.cpp
index 38464cda63e..f5ed7250618 100644
--- a/be/src/util/runtime_profile.cpp
+++ b/be/src/util/runtime_profile.cpp
@@ -364,7 +364,8 @@ const std::string* RuntimeProfile::get_info_string(const std::string& key) {
 
 #define ADD_COUNTER_IMPL(NAME, T)                                                                  \
     RuntimeProfile::T* RuntimeProfile::NAME(const std::string& name, TUnit::type unit,             \
-                                            const std::string& parent_counter_name) {              \
+                                            const std::string& parent_counter_name,                \
+                                            int64_t level) {                                       \
         DCHECK_EQ(_is_averaged_profile, false);                                                    \
         std::lock_guard<std::mutex> l(_counter_map_lock);                                          \
         if (_counter_map.find(name) != _counter_map.end()) {                                       \
@@ -372,7 +373,7 @@ const std::string* RuntimeProfile::get_info_string(const std::string& key) {
         }                                                                                          \
         DCHECK(parent_counter_name == ROOT_COUNTER ||                                              \
                _counter_map.find(parent_counter_name) != _counter_map.end());                      \
-        T* counter = _pool->add(new T(unit));                                                      \
+        T* counter = _pool->add(new T(unit, level));                                               \
         _counter_map[name] = counter;                                                              \
         std::set<std::string>* child_counters =                                                    \
                 find_or_insert(&_child_counter_map, parent_counter_name, std::set<std::string>()); \
diff --git a/be/src/util/runtime_profile.h b/be/src/util/runtime_profile.h
index d5233d40f2c..4cc2c2617ec 100644
--- a/be/src/util/runtime_profile.h
+++ b/be/src/util/runtime_profile.h
@@ -132,7 +132,8 @@ public:
     /// as value()) and the current value.
     class HighWaterMarkCounter : public Counter {
     public:
-        HighWaterMarkCounter(TUnit::type unit) : Counter(unit), current_value_(0) {}
+        HighWaterMarkCounter(TUnit::type unit, int64_t level = 2)
+                : Counter(unit, 0, level), current_value_(0) {}
 
         virtual void add(int64_t delta) {
             current_value_.fetch_add(delta, std::memory_order_relaxed);
@@ -413,7 +414,8 @@ public:
     /// Adds a high water mark counter to the runtime profile. Otherwise, same behavior
     /// as AddCounter().
     HighWaterMarkCounter* AddHighWaterMarkCounter(const std::string& name, TUnit::type unit,
-                                                  const std::string& parent_counter_name = "");
+                                                  const std::string& parent_counter_name = "",
+                                                  int64_t level = 2);
 
     // Only for create MemTracker(using profile's counter to calc consumption)
     std::shared_ptr<HighWaterMarkCounter> AddSharedHighWaterMarkCounter(
diff --git a/be/src/vec/common/hash_table/hash_map_context.h b/be/src/vec/common/hash_table/hash_map_context.h
index 41f3bd52efd..1536d48fe7a 100644
--- a/be/src/vec/common/hash_table/hash_map_context.h
+++ b/be/src/vec/common/hash_table/hash_map_context.h
@@ -69,6 +69,8 @@ struct MethodBaseInner {
                                       const uint8_t* null_map = nullptr, bool is_join = false,
                                       bool is_build = false, uint32_t bucket_size = 0) = 0;
 
+    virtual size_t serialized_keys_size(bool is_build) const { return 0; }
+
     void init_join_bucket_num(uint32_t num_rows, uint32_t bucket_size, const uint8_t* null_map) {
         bucket_nums.resize(num_rows);
 
@@ -243,6 +245,10 @@ struct MethodSerialized : public MethodBase<TData> {
         Base::keys = input_keys.data();
     }
 
+    size_t serialized_keys_size(bool is_build) const override {
+        return is_build ? build_arena.size() : Base::arena.size();
+    }
+
     void init_serialized_keys(const ColumnRawPtrs& key_columns, size_t num_rows,
                               const uint8_t* null_map = nullptr, bool is_join = false,
                               bool is_build = false, uint32_t bucket_size = 0) override {
@@ -277,6 +283,10 @@ struct MethodStringNoCache : public MethodBase<TData> {
 
     std::vector<StringRef> stored_keys;
 
+    size_t serialized_keys_size(bool is_build) const override {
+        return stored_keys.size() * sizeof(StringRef);
+    }
+
     void init_serialized_keys(const ColumnRawPtrs& key_columns, size_t num_rows,
                               const uint8_t* null_map = nullptr, bool is_join = false,
                               bool is_build = false, uint32_t bucket_size = 0) override {
@@ -430,6 +440,10 @@ struct MethodKeysFixed : public MethodBase<TData> {
         }
     }
 
+    size_t serialized_keys_size(bool is_build) const override {
+        return (is_build ? build_stored_keys.size() : stored_keys.size()) *
+               sizeof(typename Base::Key);
+    }
     void init_serialized_keys(const ColumnRawPtrs& key_columns, size_t num_rows,
                               const uint8_t* null_map = nullptr, bool is_join = false,
                               bool is_build = false, uint32_t bucket_size = 0) override {
diff --git a/be/src/vec/exec/join/process_hash_table_probe_impl.h b/be/src/vec/exec/join/process_hash_table_probe_impl.h
index 9f5167bb555..1939b702c69 100644
--- a/be/src/vec/exec/join/process_hash_table_probe_impl.h
+++ b/be/src/vec/exec/join/process_hash_table_probe_impl.h
@@ -139,6 +139,8 @@ typename HashTableType::State ProcessHashTableProbe<JoinOpType, Parent>::_init_p
                                             false, hash_table_ctx.hash_table->get_bucket_size());
         hash_table_ctx.hash_table->pre_build_idxs(hash_table_ctx.bucket_nums,
                                                   need_judge_null ? null_map : nullptr);
+        COUNTER_SET(_parent->_probe_arena_memory_usage,
+                    (int64_t)hash_table_ctx.serialized_keys_size(false));
     }
     return typename HashTableType::State(_parent->_probe_columns);
 }
diff --git a/be/src/vec/exec/join/vhash_join_node.h b/be/src/vec/exec/join/vhash_join_node.h
index fb5bf19015d..95c59094ba6 100644
--- a/be/src/vec/exec/join/vhash_join_node.h
+++ b/be/src/vec/exec/join/vhash_join_node.h
@@ -141,8 +141,10 @@ struct ProcessHashTableBuild {
         hash_table_ctx.bucket_nums.resize(_batch_size);
         hash_table_ctx.bucket_nums.shrink_to_fit();
 
-        COUNTER_UPDATE(_parent->_hash_table_memory_usage,
-                       hash_table_ctx.hash_table->get_byte_size());
+        COUNTER_SET(_parent->_hash_table_memory_usage,
+                    (int64_t)hash_table_ctx.hash_table->get_byte_size());
+        COUNTER_SET(_parent->_build_arena_memory_usage,
+                    (int64_t)hash_table_ctx.serialized_keys_size(true));
         return Status::OK();
     }
 


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


(doris) 10/19: [Feature](executor)Insert select limited by WorkloadGroup #30610

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit eacef8bee7c9acb767aa27f36a3a39ab0b85dbdc
Author: wangbo <wa...@apache.org>
AuthorDate: Wed Jan 31 16:29:28 2024 +0800

    [Feature](executor)Insert select limited by WorkloadGroup #30610
---
 be/src/olap/delta_writer.cpp                       |  2 +-
 be/src/olap/delta_writer_v2.cpp                    |  6 +++++
 be/src/olap/memtable_flush_executor.cpp            | 12 +++++++++
 be/src/olap/memtable_flush_executor.h              |  3 +++
 be/src/olap/memtable_writer.cpp                    | 11 +++++---
 be/src/olap/memtable_writer.h                      |  2 +-
 be/src/vec/sink/writer/async_result_writer.cpp     | 30 ++++++++++++++-------
 .../main/java/org/apache/doris/qe/Coordinator.java | 31 +++++++++++++---------
 .../java/org/apache/doris/qe/StmtExecutor.java     |  5 ----
 9 files changed, 71 insertions(+), 31 deletions(-)

diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp
index bc1152fcd1f..c1bf7cd5381 100644
--- a/be/src/olap/delta_writer.cpp
+++ b/be/src/olap/delta_writer.cpp
@@ -105,7 +105,7 @@ Status BaseDeltaWriter::init() {
     RETURN_IF_ERROR(_rowset_builder->init());
     RETURN_IF_ERROR(_memtable_writer->init(
             _rowset_builder->rowset_writer(), _rowset_builder->tablet_schema(),
-            _rowset_builder->get_partial_update_info(),
+            _rowset_builder->get_partial_update_info(), nullptr,
             _rowset_builder->tablet()->enable_unique_key_merge_on_write()));
     ExecEnv::GetInstance()->memtable_memory_limiter()->register_writer(_memtable_writer);
     _is_init = true;
diff --git a/be/src/olap/delta_writer_v2.cpp b/be/src/olap/delta_writer_v2.cpp
index db2c7ef80ce..e97db641a80 100644
--- a/be/src/olap/delta_writer_v2.cpp
+++ b/be/src/olap/delta_writer_v2.cpp
@@ -51,6 +51,7 @@
 #include "olap/tablet_manager.h"
 #include "olap/tablet_schema.h"
 #include "runtime/exec_env.h"
+#include "runtime/query_context.h"
 #include "service/backend_options.h"
 #include "util/brpc_client_cache.h"
 #include "util/debug_points.h"
@@ -134,7 +135,12 @@ Status DeltaWriterV2::init() {
 
     _rowset_writer = std::make_shared<BetaRowsetWriterV2>(_streams);
     RETURN_IF_ERROR(_rowset_writer->init(context));
+    ThreadPool* wg_thread_pool_ptr = nullptr;
+    if (_state->get_query_ctx()) {
+        wg_thread_pool_ptr = _state->get_query_ctx()->get_non_pipe_exec_thread_pool();
+    }
     RETURN_IF_ERROR(_memtable_writer->init(_rowset_writer, _tablet_schema, _partial_update_info,
+                                           wg_thread_pool_ptr,
                                            _streams[0]->enable_unique_mow(_req.index_id)));
     ExecEnv::GetInstance()->memtable_memory_limiter()->register_writer(_memtable_writer);
     _is_init = true;
diff --git a/be/src/olap/memtable_flush_executor.cpp b/be/src/olap/memtable_flush_executor.cpp
index e1fa68a6a53..4d0af7496d9 100644
--- a/be/src/olap/memtable_flush_executor.cpp
+++ b/be/src/olap/memtable_flush_executor.cpp
@@ -238,6 +238,18 @@ Status MemTableFlushExecutor::create_flush_token(std::unique_ptr<FlushToken>& fl
     }
 }
 
+Status MemTableFlushExecutor::create_flush_token(std::unique_ptr<FlushToken>& flush_token,
+                                                 RowsetWriter* rowset_writer,
+                                                 ThreadPool* wg_flush_pool_ptr) {
+    if (rowset_writer->type() == BETA_ROWSET) {
+        flush_token = std::make_unique<FlushToken>(wg_flush_pool_ptr);
+    } else {
+        return Status::InternalError<false>("not support alpha rowset load now.");
+    }
+    flush_token->set_rowset_writer(rowset_writer);
+    return Status::OK();
+}
+
 void MemTableFlushExecutor::_register_metrics() {
     REGISTER_HOOK_METRIC(flush_thread_pool_queue_size,
                          [this]() { return _flush_pool->get_queue_size(); });
diff --git a/be/src/olap/memtable_flush_executor.h b/be/src/olap/memtable_flush_executor.h
index 80983baa66f..9896b8382da 100644
--- a/be/src/olap/memtable_flush_executor.h
+++ b/be/src/olap/memtable_flush_executor.h
@@ -128,6 +128,9 @@ public:
     Status create_flush_token(std::unique_ptr<FlushToken>& flush_token, RowsetWriter* rowset_writer,
                               bool is_high_priority);
 
+    Status create_flush_token(std::unique_ptr<FlushToken>& flush_token, RowsetWriter* rowset_writer,
+                              ThreadPool* wg_flush_pool_ptr);
+
 private:
     void _register_metrics();
     static void _deregister_metrics();
diff --git a/be/src/olap/memtable_writer.cpp b/be/src/olap/memtable_writer.cpp
index d3ff02333a1..4fb24cb01cd 100644
--- a/be/src/olap/memtable_writer.cpp
+++ b/be/src/olap/memtable_writer.cpp
@@ -65,7 +65,7 @@ MemTableWriter::~MemTableWriter() {
 Status MemTableWriter::init(std::shared_ptr<RowsetWriter> rowset_writer,
                             TabletSchemaSPtr tablet_schema,
                             std::shared_ptr<PartialUpdateInfo> partial_update_info,
-                            bool unique_key_mow) {
+                            ThreadPool* wg_flush_pool_ptr, bool unique_key_mow) {
     _rowset_writer = rowset_writer;
     _tablet_schema = tablet_schema;
     _unique_key_mow = unique_key_mow;
@@ -76,8 +76,13 @@ Status MemTableWriter::init(std::shared_ptr<RowsetWriter> rowset_writer,
     // create flush handler
     // by assigning segment_id to memtable before submiting to flush executor,
     // we can make sure same keys sort in the same order in all replicas.
-    RETURN_IF_ERROR(StorageEngine::instance()->memtable_flush_executor()->create_flush_token(
-            _flush_token, _rowset_writer.get(), _req.is_high_priority));
+    if (wg_flush_pool_ptr) {
+        RETURN_IF_ERROR(StorageEngine::instance()->memtable_flush_executor()->create_flush_token(
+                _flush_token, _rowset_writer.get(), wg_flush_pool_ptr));
+    } else {
+        RETURN_IF_ERROR(StorageEngine::instance()->memtable_flush_executor()->create_flush_token(
+                _flush_token, _rowset_writer.get(), _req.is_high_priority));
+    }
 
     _is_init = true;
     return Status::OK();
diff --git a/be/src/olap/memtable_writer.h b/be/src/olap/memtable_writer.h
index a2687d9402c..6b48421cb6e 100644
--- a/be/src/olap/memtable_writer.h
+++ b/be/src/olap/memtable_writer.h
@@ -69,7 +69,7 @@ public:
 
     Status init(std::shared_ptr<RowsetWriter> rowset_writer, TabletSchemaSPtr tablet_schema,
                 std::shared_ptr<PartialUpdateInfo> partial_update_info,
-                bool unique_key_mow = false);
+                ThreadPool* wg_flush_pool_ptr, bool unique_key_mow = false);
 
     Status write(const vectorized::Block* block, const std::vector<uint32_t>& row_idxs,
                  bool is_append = false);
diff --git a/be/src/vec/sink/writer/async_result_writer.cpp b/be/src/vec/sink/writer/async_result_writer.cpp
index bb8e3ea77e4..f7f4c40f010 100644
--- a/be/src/vec/sink/writer/async_result_writer.cpp
+++ b/be/src/vec/sink/writer/async_result_writer.cpp
@@ -92,15 +92,27 @@ void AsyncResultWriter::start_writer(RuntimeState* state, RuntimeProfile* profil
     // This is a async thread, should lock the task ctx, to make sure runtimestate and profile
     // not deconstructed before the thread exit.
     auto task_ctx = state->get_task_execution_context();
-    static_cast<void>(ExecEnv::GetInstance()->fragment_mgr()->get_thread_pool()->submit_func(
-            [this, state, profile, task_ctx]() {
-                auto task_lock = task_ctx.lock();
-                if (task_lock == nullptr) {
-                    _writer_thread_closed = true;
-                    return;
-                }
-                this->process_block(state, profile);
-            }));
+    if (state->get_query_ctx() && state->get_query_ctx()->get_non_pipe_exec_thread_pool()) {
+        ThreadPool* pool_ptr = state->get_query_ctx()->get_non_pipe_exec_thread_pool();
+        static_cast<void>(pool_ptr->submit_func([this, state, profile, task_ctx]() {
+            auto task_lock = task_ctx.lock();
+            if (task_lock == nullptr) {
+                _writer_thread_closed = true;
+                return;
+            }
+            this->process_block(state, profile);
+        }));
+    } else {
+        static_cast<void>(ExecEnv::GetInstance()->fragment_mgr()->get_thread_pool()->submit_func(
+                [this, state, profile, task_ctx]() {
+                    auto task_lock = task_ctx.lock();
+                    if (task_lock == nullptr) {
+                        _writer_thread_closed = true;
+                        return;
+                    }
+                    this->process_block(state, profile);
+                }));
+    }
 }
 
 void AsyncResultWriter::process_block(RuntimeState* state, RuntimeProfile* profile) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
index cca9014a715..57dfde62f6e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
@@ -599,18 +599,25 @@ public class Coordinator implements CoordInterface {
     @Override
     public void exec() throws Exception {
         // LoadTask does not have context, not controlled by queue now
-        if (Config.enable_workload_group && Config.enable_query_queue && context != null) {
-            queryQueue = context.getEnv().getWorkloadGroupMgr().getWorkloadGroupQueryQueue(context);
-            if (queryQueue == null) {
-                // This logic is actually useless, because when could not find query queue, it will
-                // throw exception during workload group manager.
-                throw new UserException("could not find query queue");
-            }
-            queueToken = queryQueue.getToken();
-            if (!queueToken.waitSignal(this.queryOptions.getExecutionTimeout() * 1000)) {
-                LOG.error("query (id=" + DebugUtil.printId(queryId) + ") " + queueToken.getOfferResultDetail());
-                queryQueue.returnToken(queueToken);
-                throw new UserException(queueToken.getOfferResultDetail());
+        if (context != null) {
+            if (Config.enable_workload_group) {
+                this.setTWorkloadGroups(context.getEnv().getWorkloadGroupMgr().getWorkloadGroup(context));
+                if (Config.enable_query_queue) {
+                    queryQueue = context.getEnv().getWorkloadGroupMgr().getWorkloadGroupQueryQueue(context);
+                    if (queryQueue == null) {
+                        // This logic is actually useless, because when could not find query queue, it will
+                        // throw exception during workload group manager.
+                        throw new UserException("could not find query queue");
+                    }
+                    queueToken = queryQueue.getToken();
+                    if (!queueToken.waitSignal(this.queryOptions.getExecutionTimeout() * 1000)) {
+                        LOG.error("query (id=" + DebugUtil.printId(queryId) + ") " + queueToken.getOfferResultDetail());
+                        queryQueue.returnToken(queueToken);
+                        throw new UserException(queueToken.getOfferResultDetail());
+                    }
+                }
+            } else {
+                context.setWorkloadGroupName("");
             }
         }
         execInternal();
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 94a9fa5712c..173a4c1d53f 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
@@ -1525,11 +1525,6 @@ public class StmtExecutor {
             coordBase = new PointQueryExec(planner, analyzer);
         } else {
             coord = new Coordinator(context, analyzer, planner, context.getStatsErrorEstimator());
-            if (Config.enable_workload_group) {
-                coord.setTWorkloadGroups(context.getEnv().getWorkloadGroupMgr().getWorkloadGroup(context));
-            } else {
-                context.setWorkloadGroupName("");
-            }
             QeProcessorImpl.INSTANCE.registerQuery(context.queryId(),
                     new QeProcessorImpl.QueryInfo(context, originStmt.originStmt, coord));
             profile.setExecutionProfile(coord.getExecutionProfile());


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


(doris) 17/19: [fix](compile)Fix Ambiguous regex Namespace Issue on MacOS Compilation (#30652)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit be1d5a315149940be0e53e5ea8b5bad5b37622fc
Author: zy-kkk <zh...@gmail.com>
AuthorDate: Wed Jan 31 20:52:33 2024 +0800

    [fix](compile)Fix Ambiguous regex Namespace Issue on MacOS Compilation (#30652)
---
 be/src/olap/delete_handler.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/be/src/olap/delete_handler.cpp b/be/src/olap/delete_handler.cpp
index 29f84563627..b3096d353a0 100644
--- a/be/src/olap/delete_handler.cpp
+++ b/be/src/olap/delete_handler.cpp
@@ -303,7 +303,7 @@ Status DeleteHandler::parse_condition(const std::string& condition_str, TConditi
         //  group3:  ((?:[\s\S]+)?) matches "1597751948193618247  and length(source)<1;\n;\n"
         const char* const CONDITION_STR_PATTERN =
                 R"(([\w$#%]+)\s*((?:=)|(?:!=)|(?:>>)|(?:<<)|(?:>=)|(?:<=)|(?:\*=)|(?:IS))\s*('((?:[\s\S]+)?)'|(?:[\s\S]+)?))";
-        regex ex(CONDITION_STR_PATTERN);
+        boost::regex ex(CONDITION_STR_PATTERN);
         if (regex_match(condition_str, what, ex)) {
             if (condition_str.size() != what[0].str().size()) {
                 matched = false;
@@ -311,7 +311,7 @@ Status DeleteHandler::parse_condition(const std::string& condition_str, TConditi
         } else {
             matched = false;
         }
-    } catch (regex_error& e) {
+    } catch (boost::regex_error& e) {
         VLOG_NOTICE << "fail to parse expr. [expr=" << condition_str << "; error=" << e.what()
                     << "]";
         matched = false;


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


(doris) 08/19: [docs](update) Update Doris-Streamloader docs (#30552)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit ae3ee4b85c19b2f49f05685caf3e0eef90377351
Author: KassieZ <13...@users.noreply.github.com>
AuthorDate: Wed Jan 31 15:02:40 2024 +0800

    [docs](update) Update Doris-Streamloader docs (#30552)
---
 .../import/import-way/stream-load-manual.md        |  13 ++
 docs/en/docs/ecosystem/doris-streamloader.md       | 250 +++++++++++++++++++++
 docs/sidebars.json                                 |   1 +
 .../import/import-way/stream-load-manual.md        |  12 +
 docs/zh-CN/docs/ecosystem/doris-streamloader.md    | 247 ++++++++++++++++++++
 5 files changed, 523 insertions(+)

diff --git a/docs/en/docs/data-operate/import/import-way/stream-load-manual.md b/docs/en/docs/data-operate/import/import-way/stream-load-manual.md
index bb2cdde4e3d..e7aefde9357 100644
--- a/docs/en/docs/data-operate/import/import-way/stream-load-manual.md
+++ b/docs/en/docs/data-operate/import/import-way/stream-load-manual.md
@@ -30,6 +30,19 @@ Stream load is a synchronous way of importing. Users import local files or data
 
 Stream load is mainly suitable for importing local files or data from data streams through procedures.
 
+:::tip
+
+In comparison to single-threaded load using `curl`, Doris-Streamloader is a client tool designed for loading data into Apache Doris. it reduces the ingestion latency of large datasets by its concurrent loading capabilities. It comes with the following features:
+
+- **Parallel loading**: multi-threaded load for the Stream Load method. You can set the parallelism level using the `workers` parameter.
+- **Multi-file load:** simultaneously load of multiple files and directories with one shot. It supports recursive file fetching and allows you to specify file names with wildcard characters.
+- **Path traversal support:** support path traversal when the source files are in directories
+- **Resilience and continuity:** in case of partial load failures, it can resume data loading from the point of failure.
+- **Automatic retry mechanism:** in case of loading failures, it can automatically retry a default number of times. If the loading remains unsuccessful, it will print the command for manual retry.
+
+See [Doris-Streamloader](../docs/ecosystem/doris-streamloader) for detailed instructions and best practices.
+:::
+
 ## Basic Principles
 
 The following figure shows the main flow of Stream load, omitting some import details.
diff --git a/docs/en/docs/ecosystem/doris-streamloader.md b/docs/en/docs/ecosystem/doris-streamloader.md
new file mode 100644
index 00000000000..4dc46085241
--- /dev/null
+++ b/docs/en/docs/ecosystem/doris-streamloader.md
@@ -0,0 +1,250 @@
+---
+{
+    "title": "Doris-Streamloader",
+    "language": "en"
+}
+---
+
+<!--
+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.
+-->
+
+
+## Overview
+Doris-Streamloader is a client tool designed for loading data into Apache Doris. In comparison to single-threaded load using `curl`, it reduces the load latency of large datasets by its concurrent loading capabilities. It comes with the following features:
+
+- **Parallel loading**: multi-threaded load for the Stream Load method. You can set the parallelism level using the `workers` parameter.
+- **Multi-file load:** simultaneously load of multiple files and directories with one shot. It supports recursive file fetching and allows you to specify file names with wildcard characters.
+- **Path traversal support:** support path traversal when the source files are in directories
+- **Resilience and continuity:** in case of partial load failures, it can resume data loading from the point of failure.
+- **Automatic retry mechanism:** in case of loading failures, it can automatically retry a default number of times. If the loading remains unsuccessful, it will print the command for manual retry.
+
+
+## Installation
+
+**Version 1.0**
+
+Source Code: https://github.com/apache/doris-streamloader/
+
+| Version | Date |  Architecture  |  Link  |
+|---|---|---|---|
+| v1.0   |  20240131 |  x64  | https://apache-doris-releases.oss-accelerate.aliyuncs.com/apache-doris-streamloader-1.0.1-bin-x64.tar.xz |
+| v1.0   | 20240131  | arm64 | https://apache-doris-releases.oss-accelerate.aliyuncs.com/apache-doris-streamloader-1.0.1-bin-arm64.tar.xz |
+
+:::note
+The obtained result is the executable binary.
+:::
+
+
+## How to use
+
+```bash
+
+doris-streamloader --source_file={FILE_LIST} --url={FE_OR_BE_SERVER_URL}:{PORT} --header={STREAMLOAD_HEADER} --db={TARGET_DATABASE} --table={TARGET_TABLE}
+
+
+```
+
+**1. `FILE_LIST` support:**
+
+- Single file
+
+    E.g. Load a single file
+
+
+    ```json
+    
+    doris-streamloader --source_file="dir" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl"
+    
+    ```
+
+- Single directory
+
+    E.g. Load a single directory
+
+    ```json
+    doris-streamloader --source_file="dir" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl"        
+    ```
+
+- File names with wildcard characters (enclosed in quotes)
+
+    E.g. Load file0.csv, file1.csv, file2.csv
+
+    ```json
+    doris-streamloader --source_file="file*" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl"
+    ```
+
+- A list of files separated by commas
+
+    E.g. Load file0.csv, file1.csv, file2.csv
+    
+  ```json
+   doris-streamloader --source_file="file0.csv,file1.csv,file2.csv" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl"
+  ```
+
+- A list of directories separated by commas
+
+  E.g. Load dir1, dir2, dir3
+
+   ```json
+    doris-streamloader --source_file="dir1,dir2,dir3" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl" 
+   ```
+  
+
+**2. `STREAMLOAD_HEADER` supports all streamload headers separated with '?' if there is more than one**
+
+Example:
+
+```bash
+doris-streamloader --source_file="data.csv" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl"
+```
+
+The parameters above are required, and the following parameters are optional: 
+
+| Parameter | Description  |  Default Value  |  Suggestions  |
+|---|---|---|---|
+| --u      | Username of the database |  root    |      |
+| --p      | Password |  empty string  |      |
+| --compress      | Whether to compress data upon HTTP transmission |  false    |   Remain as default. Compression and decompression can increase pressure on Doris-Streamloader side and the CPU resources on Doris BE side, so it is advised to only enable this when network bandwidth is constrained.   |
+|--timeout    | Timeout of the HTTP request sent to Doris (seconds) |  60\*60\*10    | Remain as default |
+| --batch      | Granularity of batch reading and sending of files (rows) |  4096    | Remain as default |
+| --batch_byte      | Granularity of batch reading and sending of files (byte) |  943718400 (900MB)    | Remain as default |
+| --workers   | Concurrency  level of data loading |  0    |   "0" means the auto mode, in which the streamload speed is based on the data size and disk throughput. You can dial this up for a high-performance cluster, but it is advised to keep it below 10. If you observe excessive memory usage (via the memtracker in log), you can dial this down.   |
+| --disk_throughput      | Disk throughput (MB/s) |  800    |  Usually remain as default. This parameter is a basis of the automatic inference of workers. You can adjust this based on your needs to get a more appropriate value of workers.  |
+|--streamload_throughput | Streamload throughput (MB/s) | 100 | Usually remain as default. The default value is derived from the streamload throughput and predicted performance provided by the daily performance testing environment. To get a more appropriate value of workers, you can configure this based on your measured streamload throughput: (LoadBytes*1000)/(LoadTimeMs*1024*1024) |
+| --max_byte_per_task      | Maximum data size for each load task. For a dataset exceeding this size, the remaining part will be split into a new load task. |  107374182400 (100G)    | This is recommended to be large in order to reduce the number of load versions. However, if you encounter a "body exceed max size" and try to avoid adjusting the streaming_load_max_mb parameter (which requires restarting the backend), or if you encounter a "-238 TOO MANY SEGMENT" error, you can temporarily [...]
+| --check_utf8 | <p>Whether to check the encoding of the data that has been loaded: </p>   <p> 1) false, direct load of raw data without checking;  2)  true, replacing non UTF-8 characters with � </p> | true |Remain as default|
+|--debug |Print debug log | false | Remain as default |
+|--auto_retry| The list of failed workers and tasks for auto retry | empty string | This is only used when there is an load failure. The serial numbers of the failed workers and tasks will be shown and all you need is to copy and execute the the entire command. For example, if --auto_retry="1,1;2,1", that means the failed tasks include the first task in the first worker and the first task in the second worker. |
+|--auto_retry_times | Times of auto retries | 3 | Remain as default. If you don't need retries, you can set this to 0. |
+|--auto_retry_interval | Interval of auto retries | 60 | Remain as default. If the load failure is caused by a Doris downtime, it is recommended to set this parameter based on the restart interval of Doris. |
+|--log_filename | Path for log storage | "" | Logs are printed to the console by default. To print them to a log file, you can set the path, such as --log_filename="/var/log". |
+
+
+
+## Result description
+
+A result will be returned no matter the data loading succeeds or fails. 
+
+
+|Parameter | Description |
+|---|---|
+| Status | Loading succeeded or failed |
+| TotalRows | Total number of rows |
+| FailLoadRows | Number of rows failed to be loaded |
+| LoadedRows | Number of rows loaded |
+| FilteredRows | Number of rows filtered |
+| UnselectedRows | Number of rows unselected |
+| LoadBytes | Number of bytes loaded |
+| LoadTimeMs | Actual loading time |
+| LoadFiles | List of loaded files |
+
+
+
+Examples: 
+
+- If the loading succeeds, you will see a result like: 
+  ```Go
+  Load Result: {
+          "Status": "Success",
+          "TotalRows": 120,
+          "FailLoadRows": 0,
+          "LoadedRows": 120,
+          "FilteredRows": 0,
+          "UnselectedRows": 0,
+          "LoadBytes": 40632,
+          "LoadTimeMs": 971,
+          "LoadFiles": [
+                  "basic.csv",
+                  "basic_data1.csv",
+                  "basic_data2.csv",
+                  "dir1/basic_data.csv",
+                  "dir1/basic_data.csv.1",
+                  "dir1/basic_data1.csv"
+          ]
+  }
+  ```
+  
+- If the loading fails (or partially fails), you will see a retry message: 
+  
+  ```Go
+  load has some error and auto retry failed, you can retry by : 
+  ./doris-streamloader --source_file /mnt/disk1/laihui/doris/tools/tpch-tools/bin/tpch-data/lineitem.tbl.1  --url="http://127.0.0.1:8239" --header="column_separator:|?columns: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag,l_linestatus, l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment,temp" --db="db" --table="lineitem1" -u root -p "" --compress=false --timeout=36000 --workers=3 --batch=4096 --batch_byt [...]
+  ```
+  
+
+You can copy and execute the command. The failure message will also be provided:
+
+```Go
+Load Result: {
+      "Status": "Failed",
+      "TotalRows": 1,
+      "FailLoadRows": 1,
+      "LoadedRows": 0,
+      "FilteredRows": 0,
+      "UnselectedRows": 0,
+      "LoadBytes": 0,
+      "LoadTimeMs": 104,
+      "LoadFiles": [
+              "/mnt/disk1/laihui/doris/tools/tpch-tools/bin/tpch-data/lineitem.tbl.1"
+      ]
+}
+
+```
+
+
+## Best practice
+
+### Parameter suggestions
+
+1. Required parameters:  
+```--source_file=FILE_LIST --url=FE_OR_BE_SERVER_URL_WITH_PORT --header=STREAMLOAD_HEADER --db=TARGET_DATABASE --table=TARGET_TABLE``` 
+   If you need to load multiple files, you should configure all of them at a time in `source_file`.
+
+2. The default value of `workers` is the number of CPU cores. When that is large, for example, 96 cores, the value of `workers` should be dialed down. **The recommended value for most cases is 8.**
+
+3. `max_byte_per_task` is recommended to be large in order to reduce the number of load versions. However, if you encounter a "body exceed max size" and try to avoid adjusting the streaming_load_max_mb parameter (which requires restarting the backend), or if you encounter a `-238 TOO MANY SEGMENT` error, you can temporarily dial this down. **For most cases, this can remain as default.**
+
+**Two parameters that impacts the number of versions:**
+
+- `workers`: The more `workers`, the higher concurrency level, and thus the more versions. The recommended value for most cases is 8.
+- `max_byte_per_task`:  The larger `max_byte_per_task` , the larger data size in one single version, and thus the less versions. However, if this is excessively high, it could easily cause an `-238 TOO MANY SEGMENT ` error. For most cases, this can remain as default. 
+
+
+
+### Recommended commands
+
+In most cases, you only need to set the required parameters and `workers`. 
+
+```text
+./doris-streamloader --source_file="demo.csv,demoFile*.csv,demoDir" --url="http://127.0.0.1:8030" --header="column_separator:," --db="demo" --table="test_load" --u="root" --workers=8
+```
+
+
+### FAQ
+
+- Before resumable loading was available, to fix any partial failures in loading would require deleting the current table and starting over. In this case, Doris-Streamloader would retry automatically. If the retry fails, a retry command will be printed so you can copy and execute it.
+- The default maximum data loading size for Doris-Streamloader is limited by BE config `streaming_load_max_mb` (default: 100GB). If you don't want to restart BE, you can also dial down `max_byte_per_task`.
+
+  To show current `streaming_load_max_mb`: 
+
+  ```Go
+  curl "http://127.0.0.1:8040/api/show_config"
+  ```
+  
+- If you encounter an `-238 TOO MANY SEGMENT ` error, you can dial down `max_byte_per_task`.
\ No newline at end of file
diff --git a/docs/sidebars.json b/docs/sidebars.json
index f892da7c345..a366e32d0d3 100644
--- a/docs/sidebars.json
+++ b/docs/sidebars.json
@@ -259,6 +259,7 @@
             "items": [
                 "ecosystem/spark-doris-connector",
                 "ecosystem/flink-doris-connector",
+                "ecosystem/doris-streamloader",
                 "ecosystem/datax",
                 "ecosystem/seatunnel",
                 "ecosystem/kyuubi",
diff --git a/docs/zh-CN/docs/data-operate/import/import-way/stream-load-manual.md b/docs/zh-CN/docs/data-operate/import/import-way/stream-load-manual.md
index 0f4dc28c737..69e47407906 100644
--- a/docs/zh-CN/docs/data-operate/import/import-way/stream-load-manual.md
+++ b/docs/zh-CN/docs/data-operate/import/import-way/stream-load-manual.md
@@ -30,6 +30,18 @@ Stream load 是一个同步的导入方式,用户通过发送 HTTP 协议发
 
 Stream load 主要适用于导入本地文件,或通过程序导入数据流中的数据。
 
+:::tip
+相比于直接使用 `curl` 的单并发导入,更推荐使用 **专用导入工具 Doris-Streamloader** 该工具是一款用于将数据导入 Doris 数据库的专用客户端工具,可以提供 **多并发导入** 的功能,降低大数据量导入的耗时。拥有以下功能:
+
+- 并发导入,实现 Stream Load 的多并发导入。可以通过 `workers` 值设置并发数。
+- 多文件导入,一次导入可以同时导入多个文件及目录,支持设置通配符以及会自动递归获取文件夹下的所有文件。
+- 断点续传,在导入过程中可能出现部分失败的情况,支持在失败点处进行继续传输。
+- 自动重传,在导入出现失败的情况后,无需手动重传,工具会自动重传默认的次数,如果仍然不成功,打印出手动重传的命令。
+
+
+点击 [Doris-Streamloader 文档](../docs/ecosystem/doris-streamloader)了解使用方法与实践详情。
+:::
+
 ## 基本原理
 
 下图展示了 Stream load 的主要流程,省略了一些导入细节。
diff --git a/docs/zh-CN/docs/ecosystem/doris-streamloader.md b/docs/zh-CN/docs/ecosystem/doris-streamloader.md
new file mode 100644
index 00000000000..40e7b1b3fbc
--- /dev/null
+++ b/docs/zh-CN/docs/ecosystem/doris-streamloader.md
@@ -0,0 +1,247 @@
+
+---
+{
+    "title": "Doris-Streamloader",
+    "language": "zh-CN"
+}
+---
+
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+
+## 概述
+[Doris-Streamloader](https://github.com/apache/doris-streamloader) 是一款用于将数据导入 Doris 数据库的专用客户端工具。相比于直接使用 `curl` 的单并发导入,该工具可以提供多并发导入的功能,降低大数据量导入的耗时。拥有以下功能:
+
+- 并发导入,实现 Stream Load 的多并发导入。可以通过 workers 值设置并发数。
+- 多文件导入,一次导入可以同时导入多个文件及目录,支持设置通配符以及会自动递归获取文件夹下的所有文件。
+- 断点续传,在导入过程中可能出现部分失败的情况,支持在失败点处进行继续传输。 
+- 自动重传,在导入出现失败的情况后,无需手动重传,工具会自动重传默认的次数,如果仍然不成功,打印出手动重传的命令。
+
+## 获取与安装
+
+**1.0 版本** 
+
+源代码:  https://github.com/apache/doris-streamloader
+
+| 版本    | 日期      |  平台  |  链接  |
+|---|---|---|---|
+| v1.0   |  20240131 |  x64  | https://apache-doris-releases.oss-accelerate.aliyuncs.com/apache-doris-streamloader-1.0.1-bin-x64.tar.xz|
+| v1.0   | 20240131  | arm64 | https://apache-doris-releases.oss-accelerate.aliyuncs.com/apache-doris-streamloader-1.0.1-bin-arm64.tar.xz|
+
+:::note
+获取结果即为可执行二进制。
+:::
+
+## 使用方法
+
+```bash
+
+doris-streamloader --source_file={FILE_LIST} --url={FE_OR_BE_SERVER_URL}:{PORT} --header={STREAMLOAD_HEADER} --db={TARGET_DATABASE} --table={TARGET_TABLE}
+
+```
+
+
+**1. `FILE_LIST` 支持:**
+
+- 单个文件
+
+    例如:导入单个文件 file.csv
+
+    ```json
+    doris-streamloader --source_file="dir" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl"
+    ```
+
+- 单个目录
+
+    例如:导入单个目录 dir
+
+    ```json
+    doris-streamloader --source_file="dir" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl"        
+    ```
+
+- 带通配符的文件名(需要用引号包围)
+
+    例如:导入 file0.csv, file1.csv, file2.csv
+
+    ```json
+    doris-streamloader --source_file="file*" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl"
+    ```
+
+- 逗号分隔的文件名列表
+
+    例如:导入 file0.csv, file1.csv file2.csv
+
+    ```json
+    doris-streamloader --source_file="file0.csv,file1.csv,file2.csv" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl"
+    ```
+
+- 逗号分隔的目录列表
+
+  例如:导入 dir1, dir2,dir3
+
+   ```json
+  doris-streamloader --source_file="dir1,dir2,dir3" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl" 
+   ```
+
+:::tip 
+当需要多个文件导入时,使用 Doris-Streamloader 也只会产生一个版本号 
+:::
+
+
+
+**2.** `STREAMLOAD_HEADER` **支持 Stream Load 的所有参数,多个参数之间用  '?' 分隔。**
+
+用法举例:
+
+```bash
+doris-streamloader --source_file="data.csv" --url="http://localhost:8330" --header="column_separator:|?columns:col1,col2" --db="testdb" --table="testtbl"
+```
+
+上述参数均为必要参数,下面介绍可选参数:
+
+| 参数名    | 含义             |  默认值  |  建议  |
+|---|---|---|---|
+| --u      | 数据库用户名      |  root    |      |
+| --p      | 数据库用户对应的密码   |  空字符串    |      |
+| --compress      | 导入数据是否在 HTTP 传输时压缩   |  false    |   保持默认,打开后压缩解压会分别增加工具和 Doris BE 的 CPU 压力,所以仅在数据源所在机器网络带宽瓶颈时打开   |
+|--timeout    | 向 Doris 发送 HTTP 请求的超时时间, 单位:秒   |  60\*60\*10    | 保持默认   |
+| --batch      | 文件批量读取和发送的粒度, 单位: 行      |  4096    |  保持默认    |
+| --batch_byte      | 文件批量读取和发送的粒度, 单位: byte      |  943718400 (900MB)    |  保持默认    |
+| --workers   | 导入的并发数   |  0    |   设置成 0 为自动模式,会根据导入数据的大小,磁盘的吞吐量,Stream Load 导入速度计算一个值。 也可以手动设置,性能好的集群可以设置大点,最好不要超过 10。如果观察到导入内存过高(通过观察 Memtracker 或者 Exceed 日志), 则可适当降低 worker 数量   |
+| --disk_throughput      | 磁盘的吞吐量,单位 MB/s   |  800    |  通常保持默认即可。该值参与 --workers 的自动推算过程。 如果希望通过工具能计算出一个适当的 workers 数,可以根据实际磁盘吞吐设置。  |
+|--streamload_throughput | Stream Load 导入实际的吞吐大小,单位 MB/s | 100 | 通常保持默认即可。该值参与 --workers 的自动推算过程。 默认值是通过每日性能测试环境给出的 Stream Load 吞吐量以及性能可预测性得出的。 如果希望通过工具能计算出一个适当的 workers 数,可以设置实测的 Stream Load 的吞吐,即:(LoadBytes\*1000)/(LoadTimeMs\*1024\*1024) 计算出实际的吞吐量 |
+| --max_byte_per_task      | 每个导入任务数据量的最大大小,超过这个值剩下的数据会被拆分到一个新的导入任务中。  |  107374182400 (100G)    |  建议设置一个很大的值来减少导入的版本数。但如果遇到 body exceed max size 错误且不想调整 streaming_load_max_mb 参数(需重启 be),又或是遇到 -238 TOO MANY SEGMENT 错误,可以临时调小这个配置    |
+| --check_utf8 | <p>是否对导入数据的编码进行检查:</p>   <p> 1) false,那么不做检查直接将原始数据导入; 2) true,那么对数据中非 utf-8 编码的字符用 � 进行替代</p> | true |保持默认|
+|--debug |打印 Debug 日志 | false | 保持默认|
+|--auto_retry| 自动重传失败的 worker 序号和 task 序号的列表 | 空字符串 | 仅导入失败时重传使用,正常导入无需关心。失败时会提示具体参数内容,复制执行即可。例:如果 --auto_retry="1,1,2,1" 则表示: 需要重传的task为:第一个 worker 的第一个 task,第二个 worker 的第一个 task。 |
+|--auto_retry_times | 自动重传的次数 | 3 | 保持默认,如果不想重传需要把这个值设置为 0 |
+|--auto_retry_interval | 自动重传的间隔 | 60 | 保持默认,如果 Doris 因宕机导致失败,建议根据实际 Doris 重启的时间间隔来设置该值 |
+|--log_filename | 日志存储的位置 | "" | 默认将日志打印到控制台上,如果要打印到日志文件中,可以设置存储日志文件的路径,如--log_filename = "/var/log" |
+
+
+
+## 结果说明
+
+无论成功与失败,都会显示最终的结果,结果参数说明:
+
+
+|参数名 | 描述 |
+|---|---|
+| Status |  导入成功(Success)与否(Failed)|
+| TotalRows | 想要导入文件中所有的行数 |
+| FailLoadRows | 想要导入文件中没有导入的行数 |
+| LoadedRows | 实际导入 Doris 的行数 |
+| FilteredRows | 实际导入过程中被 Doris 过滤的行数 |
+| UnselectedRows | 实际导入过程中被 Doris 忽略的行数 |
+| LoadBytes | 实际导入的 byte 大小 |
+| LoadTimeMs | 实际导入的时间 |
+| LoadFiles | 实际导入的文件列表|
+
+
+
+具体例子如下:
+
+- 导入成功,成功信息如下: 
+
+  ```Go
+  Load Result: {
+          "Status": "Success",
+          "TotalRows": 120,
+          "FailLoadRows": 0,
+          "LoadedRows": 120,
+          "FilteredRows": 0,
+          "UnselectedRows": 0,
+          "LoadBytes": 40632,
+          "LoadTimeMs": 971,
+          "LoadFiles": [
+                  "basic.csv",
+                  "basic_data1.csv",
+                  "basic_data2.csv",
+                  "dir1/basic_data.csv",
+                  "dir1/basic_data.csv.1",
+                  "dir1/basic_data1.csv"
+          ]
+  }
+  ```
+  
+- 导入失败:如果导入过程中部分数据没有导入失败了,会给出重传信息,如:
+  
+  ```Go
+  load has some error, and auto retry failed, you can retry by : 
+  ./doris-streamloader --source_file /mnt/disk1/laihui/doris/tools/tpch-tools/bin/tpch-data/lineitem.tbl.1  --url="http://127.0.0.1:8239" --header="column_separator:|?columns: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag,l_linestatus, l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment,temp" --db="db" --table="lineitem1" -u root -p "" --compress=false --timeout=36000 --workers=3 --batch=4096 --batch_byt [...]
+  ```
+  
+只需复制运行该命令即可,`auto_retry` 说明可参考, 并给出失败的结果信息:
+
+```Go
+Load Result: {
+      "Status": "Failed",
+      "TotalRows": 1,
+      "FailLoadRows": 1,
+      "LoadedRows": 0,
+      "FilteredRows": 0,
+      "UnselectedRows": 0,
+      "LoadBytes": 0,
+      "LoadTimeMs": 104,
+      "LoadFiles": [
+              "/mnt/disk1/laihui/doris/tools/tpch-tools/bin/tpch-data/lineitem.tbl.1"
+      ]
+}
+
+```
+
+
+## 最佳实践
+
+### 1. 参数推荐
+
+1. 必要参数,一定要配置: ```--source_file=FILE_LIST --url=FE_OR_BE_SERVER_URL_WITH_PORT --header=STREAMLOAD_HEADER --db=TARGET_DATABASE --table=TARGET_TABLE``` ,**如果需要导入多个文件时,推荐使用** `source_file` **方式。**
+
+2. `workers`,默认值为 CPU 核数,在 CPU 核数过多的场景(比如 96 核)会产生太多的并发,需要减少这个值,**推荐一般设置为 8 即可。**
+
+3. `max_byte_per_task`,可以设置一个很大的值来减少导入的 version 数。但如果遇到 `body exceed max size` 错误且不想调整 `streaming_load_max_mb` 参数(需重启 BE),又或是遇到 `-238 TOO MANY SEGMENT` 错误,可以临时调小这个配置,**一般使用默认即可。**
+
+4. 影响 version 数的两个参数:
+- `workers`:worker 数越多,版本号越多,并发越高,一般使用 8 即可。
+- `max_byte_per_task`:`max_byte_per_task` 越大,单个 version 数据量越大,version 数越少,但是这个值过大可能会遇到 `-238 TOO MANY SEGMENT `的问题。一般使用默认值即可。
+
+
+
+### 2. 推荐命令
+
+设置必要参数以及设置 `workers=8` 即可。
+
+```text
+./doris-streamloader --source_file="demo.csv,demoFile*.csv,demoDir" --url="http://127.0.0.1:8030" --header="column_separator:," --db="demo" --table="test_load" --u="root" --workers=8
+```
+
+
+### 3. FAQ
+
+- 在导入过程中,遇到了部分子任务失败的问题,当时没有断点续传续传的功能,导入失败后重新删表导入,如果遇到这个问题,工具会进行自动重传,如果重传失败会打印出重传命令,复制后可以手动重传。
+- 该工具的默认单个导入是 100G,超过了 BE 默认的 `streaming_load_max_mb` 阈值如果不希望重启 BE,可以减少 `max_byte_per_task` 这个参数的大小。
+ 
+  查看 `streaming_load_max_mb` 大小的方法:
+
+  ```Go
+  -curl "http://127.0.0.1:8040/api/show_config"
+  ```
+  
+- 导入过程如果遇到 `-238 TOO MANY SEGMENT` 的问题,可以减少 `max_byte_per_task` 的大小。
\ No newline at end of file


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


(doris) 16/19: [fix](JDK17) The objects stored in `PriorityQueue` must implement the `Comparable` interface (#30050) (#30625)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 39a6cb6e887d43cbb26d0eaf9564772e50a59a62
Author: Tiewei Fang <43...@users.noreply.github.com>
AuthorDate: Wed Jan 31 20:52:18 2024 +0800

    [fix](JDK17) The objects stored in `PriorityQueue` must implement the `Comparable` interface (#30050) (#30625)
    
    Issue Number:  #30484
    
    The objects stored in PriorityQueue must implement the Comparable interface or passed into the customized `Comparator`.
    
    If we don't do this, run the program in the JDK17 environment will report an exception:
    ```java
    Caused by: java.lang.AssertionError: Expect exception msg contains 'query wait timeout', but meet
    'java.sql.SQLException: ClassCastException,
    msg: class org.apache.doris.resource.workloadgroup.QueueToken cannot be cast to class java.lang.Comparable
    (org.apache.doris.resource.workloadgroup.QueueToken is in unnamed module of loader 'app'; java.lang.Comparable is in module java.base of loader 'bootstrap')'
    ```
---
 .../org/apache/doris/resource/workloadgroup/QueueToken.java    | 10 +++++++++-
 .../array_functions/test_array_zip_array_enumerate_uniq.groovy |  2 +-
 2 files changed, 10 insertions(+), 2 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueueToken.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueueToken.java
index 17299d3ea3b..189ba77e8de 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueueToken.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueueToken.java
@@ -28,9 +28,14 @@ import java.util.concurrent.locks.ReentrantLock;
 // used to mark QueryQueue offer result
 // if offer failed, then need to cancel query
 // and return failed reason to user client
-public class QueueToken {
+public class QueueToken implements Comparable<QueueToken> {
     private static final Logger LOG = LogManager.getLogger(QueueToken.class);
 
+    @Override
+    public int compareTo(QueueToken other) {
+        return Long.compare(this.tokenId, other.getTokenId());
+    }
+
     enum TokenState {
         ENQUEUE_SUCCESS,
         READY_TO_RUN
@@ -136,4 +141,7 @@ public class QueueToken {
         return tokenId == other.tokenId;
     }
 
+    public long getTokenId() {
+        return tokenId;
+    }
 }
diff --git a/regression-test/suites/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.groovy b/regression-test/suites/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.groovy
index b8207c1ee76..38772921ece 100644
--- a/regression-test/suites/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.groovy
+++ b/regression-test/suites/datatype_p0/nested_types/query/array_functions/test_array_zip_array_enumerate_uniq.groovy
@@ -25,7 +25,7 @@ suite("test_array_zip_array_enumerate_uniq", "p0") {
                SELECT array_zip();
                 """
     } catch (Exception ex) {
-        assertTrue(ex.getMessage().contains("errCode = 2, detailMessage = Unexpected exception: 0"))
+        assertTrue(ex.getMessage().contains("errCode = 2, detailMessage = Unexpected exception: "))
     }
 
     try {


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


(doris) 09/19: [typo](doc)Update flink Spark connector documentation (#30565)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 8154b8a3f0b489de87d0ba6405a8f99255b33877
Author: caoliang-web <71...@users.noreply.github.com>
AuthorDate: Wed Jan 31 15:14:31 2024 +0800

    [typo](doc)Update flink Spark connector documentation (#30565)
---
 docs/en/docs/ecosystem/flink-doris-connector.md    | 18 +++++++++---------
 docs/en/docs/ecosystem/spark-doris-connector.md    |  4 ++--
 docs/zh-CN/docs/ecosystem/flink-doris-connector.md | 18 +++++++++---------
 docs/zh-CN/docs/ecosystem/spark-doris-connector.md |  4 ++--
 4 files changed, 22 insertions(+), 22 deletions(-)

diff --git a/docs/en/docs/ecosystem/flink-doris-connector.md b/docs/en/docs/ecosystem/flink-doris-connector.md
index 98ec7fdbe6c..92e0166fcc6 100644
--- a/docs/en/docs/ecosystem/flink-doris-connector.md
+++ b/docs/en/docs/ecosystem/flink-doris-connector.md
@@ -38,13 +38,13 @@ under the License.
 ## Version Compatibility
 
 | Connector Version | Flink Version | Doris Version | Java Version | Scala Version |
-| --------- | ----- | ------ | ---- | ----- |
-| 1.0.3     | 1.11+ | 0.15+  | 8    | 2.11,2.12 |
-| 1.1.1    | 1.14  | 1.0+   | 8    | 2.11,2.12 |
-| 1.2.1    | 1.15  | 1.0+   | 8    | -         |
-| 1.3.0     | 1.16  | 1.0+   | 8    | -         |
-| 1.4.0     | 1.15,1.16,1.17  | 1.0+   | 8   |- |
-| 1.5.0 | 1.15,1.16,1.17,1.18 | 1.0+ | 8 |- |
+|-------------------| ----- | ------ | ---- | ----- |
+| 1.0.3             | 1.11+ | 0.15+  | 8    | 2.11,2.12 |
+| 1.1.1             | 1.14  | 1.0+   | 8    | 2.11,2.12 |
+| 1.2.1             | 1.15  | 1.0+   | 8    | -         |
+| 1.3.0             | 1.16  | 1.0+   | 8    | -         |
+| 1.4.0             | 1.15,1.16,1.17  | 1.0+   | 8   |- |
+| 1.5.2             | 1.15,1.16,1.17,1.18 | 1.0+ | 8 |- |
 
 ## USE
 
@@ -311,14 +311,14 @@ ON a.city = c.city
 ### General configuration items
 
 | Key                              | Default Value | Required | Comment                                                      |
-| -------------------------------- | ------------- | -------- | ------------------------------------------------------------ |
+| -------------------------------- |---------------| -------- | ------------------------------------------------------------ |
 | fenodes                          | --            | Y        | Doris FE http address, multiple addresses are supported, separated by commas |
 | benodes                          | --            | N        | Doris BE http address, multiple addresses are supported, separated by commas. refer to [#187](https://github.com/apache/doris-flink-connector/pull/187) |
 | jdbc-url                         | --            | N        | jdbc connection information, such as: jdbc:mysql://127.0.0.1:9030 |
 | table.identifier                 | --            | Y        | Doris table name, such as: db.tbl                            |
 | username                         | --            | Y        | username to access Doris                                     |
 | password                         | --            | Y        | Password to access Doris                                     |
-| auto-redirect                    | false         | N        | Whether to redirect StreamLoad requests. After being turned on, StreamLoad will be written through FE, and BE information will no longer be displayed. At the same time, it can also be written to SelectDB Cloud by turning on this parameter. |
+| auto-redirect                    | true          | N        | Whether to redirect StreamLoad requests. After being turned on, StreamLoad will be written through FE, and BE information will no longer be displayed. |
 | doris.request.retries            | 3             | N        | Number of retries to send requests to Doris                  |
 | doris.request.connect.timeout.ms | 30000         | N        | Connection timeout for sending requests to Doris             |
 | doris.request.read.timeout.ms    | 30000         | N        | Read timeout for sending requests to Doris                   |
diff --git a/docs/en/docs/ecosystem/spark-doris-connector.md b/docs/en/docs/ecosystem/spark-doris-connector.md
index 80874436421..2e325e244b9 100644
--- a/docs/en/docs/ecosystem/spark-doris-connector.md
+++ b/docs/en/docs/ecosystem/spark-doris-connector.md
@@ -39,7 +39,7 @@ Github: https://github.com/apache/doris-spark-connector
 
 | Connector | Spark               | Doris       | Java | Scala      |
 |-----------|---------------------|-------------|------|------------|
-| 1.3.0     | 3.4 ~ 3.1, 2.4, 2.3 | 1.0 +       | 8    | 2.12, 2.11 |
+| 1.3.1     | 3.4 ~ 3.1, 2.4, 2.3 | 1.0 +       | 8    | 2.12, 2.11 |
 | 1.2.0     | 3.2, 3.1, 2.3       | 1.0 +       | 8    | 2.12, 2.11 |
 | 1.1.0     | 3.2, 3.1, 2.3       | 1.0 +       | 8    | 2.12, 2.11 |
 | 1.0.1     | 3.1, 2.3            | 0.12 - 0.15 | 8    | 2.12, 2.11 |
@@ -274,7 +274,7 @@ kafkaSource.selectExpr("CAST(key AS STRING)", "CAST(value as STRING)")
 | doris.sink.task.use.repartition  | false             | Whether to use repartition mode to control the number of partitions written by Doris. The default value is false, and coalesce is used (note: if there is no Spark action before the write, the whole computation will be less parallel). If it is set to true, then repartition is used (note: you can set the final number of partitions at the cost of shuffle).                                                                                 [...]
 | doris.sink.batch.interval.ms     | 50                | The interval time of each batch sink, unit ms.                                                                                                                                                                                                                                                                                                                                                                                                      [...]
 | doris.sink.enable-2pc            | false             | Whether to enable two-stage commit. When enabled, transactions will be committed at the end of the job, and all pre-commit transactions will be rolled back when some tasks fail.                                                                                                                                                                                                                                                                   [...]
-| doris.sink.auto-redirect         | false             | Whether to redirect StreamLoad requests. After being turned on, StreamLoad will write through FE and no longer obtain BE information explicitly.                                                                                                                                                                                                                         |
+| doris.sink.auto-redirect         | true              | Whether to redirect StreamLoad requests. After being turned on, StreamLoad will write through FE and no longer obtain BE information explicitly.                                                                                                                                                                                                                         |
 
 ### SQL & Dataframe Configuration
 
diff --git a/docs/zh-CN/docs/ecosystem/flink-doris-connector.md b/docs/zh-CN/docs/ecosystem/flink-doris-connector.md
index 9c367fbcd3f..439660ceab6 100644
--- a/docs/zh-CN/docs/ecosystem/flink-doris-connector.md
+++ b/docs/zh-CN/docs/ecosystem/flink-doris-connector.md
@@ -40,13 +40,13 @@ under the License.
 ## 版本兼容
 
 | Connector Version | Flink Version | Doris Version | Java Version | Scala Version |
-| --------- | ----- | ------ | ---- | ----- |
-| 1.0.3     | 1.11+ | 0.15+  | 8    | 2.11,2.12 |
-| 1.1.1     | 1.14  | 1.0+   | 8    | 2.11,2.12 |
-| 1.2.1     | 1.15  | 1.0+   | 8    | -         |
-| 1.3.0     | 1.16  | 1.0+   | 8    | -         |
-| 1.4.0     | 1.15,1.16,1.17  | 1.0+   | 8   |- |
-| 1.5.0 | 1.15,1.16,1.17,1.18 | 1.0+ | 8 |- |
+|-------------------| ----- | ------ | ---- | ----- |
+| 1.0.3             | 1.11+ | 0.15+  | 8    | 2.11,2.12 |
+| 1.1.1             | 1.14  | 1.0+   | 8    | 2.11,2.12 |
+| 1.2.1             | 1.15  | 1.0+   | 8    | -         |
+| 1.3.0             | 1.16  | 1.0+   | 8    | -         |
+| 1.4.0             | 1.15,1.16,1.17  | 1.0+   | 8   |- |
+| 1.5.2             | 1.15,1.16,1.17,1.18 | 1.0+ | 8 |- |
 
 ## 使用
 
@@ -314,14 +314,14 @@ ON a.city = c.city
 ### 通用配置项
 
 | Key                              | Default Value | Required | Comment                                                      |
-| -------------------------------- | ------------- | -------- | ------------------------------------------------------------ |
+| -------------------------------- |---------------| -------- | ------------------------------------------------------------ |
 | fenodes                          | --            | Y        | Doris FE http 地址, 支持多个地址,使用逗号分隔              |
 | benodes                          | --            | N        | Doris BE http 地址, 支持多个地址,使用逗号分隔,参考[#187](https://github.com/apache/doris-flink-connector/pull/187) |
 | jdbc-url                         | --            | N        | jdbc连接信息,如: jdbc:mysql://127.0.0.1:9030                |
 | table.identifier                 | --            | Y        | Doris 表名,如:db.tbl                                       |
 | username                         | --            | Y        | 访问 Doris 的用户名                                          |
 | password                         | --            | Y        | 访问 Doris 的密码                                            |
-| auto-redirect                    | false         | N        | 是否重定向StreamLoad请求。开启后StreamLoad将通过FE写入,不再显示获取BE信息,同时也可通过开启该参数写入SelectDB Cloud |
+| auto-redirect                    | true          | N        | 是否重定向StreamLoad请求。开启后StreamLoad将通过FE写入,不再显示获取BE信息 |
 | doris.request.retries            | 3             | N        | 向 Doris 发送请求的重试次数                                  |
 | doris.request.connect.timeout.ms | 30000         | N        | 向 Doris 发送请求的连接超时时间                              |
 | doris.request.read.timeout.ms    | 30000         | N        | 向 Doris 发送请求的读取超时时间                              |
diff --git a/docs/zh-CN/docs/ecosystem/spark-doris-connector.md b/docs/zh-CN/docs/ecosystem/spark-doris-connector.md
index 8c4c84ad663..50442236889 100644
--- a/docs/zh-CN/docs/ecosystem/spark-doris-connector.md
+++ b/docs/zh-CN/docs/ecosystem/spark-doris-connector.md
@@ -39,7 +39,7 @@ Spark Doris Connector 可以支持通过 Spark 读取 Doris 中存储的数据
 
 | Connector | Spark               | Doris       | Java | Scala      |
 |-----------|---------------------|-------------|------|------------|
-| 1.3.0     | 3.4 ~ 3.1, 2.4, 2.3 | 1.0 +       | 8    | 2.12, 2.11 |
+| 1.3.1     | 3.4 ~ 3.1, 2.4, 2.3 | 1.0 +       | 8    | 2.12, 2.11 |
 | 1.2.0     | 3.2, 3.1, 2.3       | 1.0 +       | 8    | 2.12, 2.11 |
 | 1.1.0     | 3.2, 3.1, 2.3       | 1.0 +       | 8    | 2.12, 2.11 |
 | 1.0.1     | 3.1, 2.3            | 0.12 - 0.15 | 8    | 2.12, 2.11 |
@@ -278,7 +278,7 @@ kafkaSource.selectExpr("CAST(value as STRING)")
 | doris.sink.task.use.repartition  | false             | 是否采用 repartition 方式控制 Doris写入 Partition数。默认值为 false,采用 coalesce 方式控制(注意: 如果在写入之前没有 Spark action 算子,可能会导致整个计算并行度降低)。<br/>如果设置为 true,则采用 repartition 方式(注意: 可设置最后 Partition 数,但会额外增加 shuffle 开销)。                                                                         |
 | doris.sink.batch.interval.ms     | 50                | 每个批次sink的间隔时间,单位 ms。                                                                                                                                                                                                                                                    |
 | doris.sink.enable-2pc            | false             | 是否开启两阶段提交。开启后将会在作业结束时提交事务,而部分任务失败时会将所有预提交状态的事务会滚。                                                                                                                                                |
-| doris.sink.auto-redirect         | false             | 是否重定向 StreamLoad 请求。开启后 StreamLoad 将通过 FE 写入, 不再显式获取 BE 信息。                                                                                                                                                                                                                         |
+| doris.sink.auto-redirect         | true              | 是否重定向 StreamLoad 请求。开启后 StreamLoad 将通过 FE 写入, 不再显式获取 BE 信息。                                                                                                                                                                                                                         |
 
 ### SQL 和 Dataframe 专有配置
 


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


(doris) 01/19: [Chore](workflow)delete auto-reply workflow (#30626)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 4418cac69d4a5c9fbd1816b9e078c0791ec40b26
Author: Calvin Kirs <ki...@apache.org>
AuthorDate: Wed Jan 31 14:02:16 2024 +0800

    [Chore](workflow)delete auto-reply workflow (#30626)
    
    auto-reply not working due to permission issue
    Comments on the fork repository failed due to permission issues
    We will instead use Github App to listen for events and then comment.
---
 .github/actions/create-or-update-comment |  1 -
 .github/workflows/auto-pr-reply.yml      | 40 --------------------------------
 .gitmodules                              |  3 ---
 3 files changed, 44 deletions(-)

diff --git a/.github/actions/create-or-update-comment b/.github/actions/create-or-update-comment
deleted file mode 160000
index 23ff15729ef..00000000000
--- a/.github/actions/create-or-update-comment
+++ /dev/null
@@ -1 +0,0 @@
-Subproject commit 23ff15729ef2fc348714a3bb66d2f655ca9066f2
diff --git a/.github/workflows/auto-pr-reply.yml b/.github/workflows/auto-pr-reply.yml
deleted file mode 100644
index b4c15061f2e..00000000000
--- a/.github/workflows/auto-pr-reply.yml
+++ /dev/null
@@ -1,40 +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.
-
-name: Auto Reply to PR
-
-on:
-  pull_request:
-    types: [opened]
-
-jobs:
-  comment:
-    runs-on: ubuntu-latest
-    steps:
-      - name: Checkout
-        uses: actions/checkout@v3
-        with:
-          persist-credentials: false
-          submodules: recursive
-      - name: Comment on PR
-        uses: ./.github/actions/create-or-update-comment
-        with:
-          issue-number: ${{ github.event.pull_request.number }}
-          body: |
-              Thank you for your contribution to Apache Doris.
-  
-              Don't know what should be done next? See [How to process your PR](https://cwiki.apache.org/confluence/display/DORIS/How+to+process+your+PR)
diff --git a/.gitmodules b/.gitmodules
index f092e8b8e5c..a4e579b1794 100644
--- a/.gitmodules
+++ b/.gitmodules
@@ -24,6 +24,3 @@
 	path = be/src/clucene
 	url = https://github.com/apache/doris-thirdparty.git
 	branch = clucene
-[submodule ".github/actions/create-or-update-comment"]
-	path = .github/actions/create-or-update-comment
-	url = https://github.com/peter-evans/create-or-update-comment.git


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


(doris) 11/19: [Refactor][Rf] remove unless code in RF (#30597)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 7234bdce336845fdb5070755e66de2ee9e8775cc
Author: HappenLee <ha...@hotmail.com>
AuthorDate: Wed Jan 31 16:29:52 2024 +0800

    [Refactor][Rf] remove unless code in RF (#30597)
---
 be/src/exprs/bloom_filter_func.h          |  3 +-
 be/src/exprs/runtime_filter.cpp           | 95 +++++++++++--------------------
 be/src/exprs/runtime_filter.h             |  6 +-
 be/src/exprs/runtime_filter_slots.h       | 13 ++---
 be/src/exprs/runtime_filter_slots_cross.h |  2 +-
 be/src/runtime/runtime_filter_mgr.cpp     |  2 +-
 6 files changed, 47 insertions(+), 74 deletions(-)

diff --git a/be/src/exprs/bloom_filter_func.h b/be/src/exprs/bloom_filter_func.h
index ed4205a7e0d..84e6eba1e44 100644
--- a/be/src/exprs/bloom_filter_func.h
+++ b/be/src/exprs/bloom_filter_func.h
@@ -169,10 +169,9 @@ public:
         return _bloom_filter->init(data, data_size);
     }
 
-    Status get_data(char** data, int* len) {
+    void get_data(char** data, int* len) {
         *data = _bloom_filter->data();
         *len = _bloom_filter->size();
-        return Status::OK();
     }
 
     size_t get_size() const { return _bloom_filter ? _bloom_filter->size() : 0; }
diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp
index bf1db5ff867..06d1c452fdd 100644
--- a/be/src/exprs/runtime_filter.cpp
+++ b/be/src/exprs/runtime_filter.cpp
@@ -67,7 +67,6 @@
 namespace doris {
 
 // PrimitiveType-> PColumnType
-// TODO: use constexpr if we use c++14
 PColumnType to_proto(PrimitiveType type) {
     switch (type) {
     case TYPE_BOOLEAN:
@@ -118,7 +117,6 @@ PColumnType to_proto(PrimitiveType type) {
 }
 
 // PColumnType->PrimitiveType
-// TODO: use constexpr if we use c++14
 PrimitiveType to_primitive_type(PColumnType type) {
     switch (type) {
     case PColumnType::COLUMN_TYPE_BOOL:
@@ -281,12 +279,8 @@ class RuntimePredicateWrapper {
 public:
     RuntimePredicateWrapper(RuntimeFilterParamsContext* state, ObjectPool* pool,
                             const RuntimeFilterParams* params)
-            : _state(state),
-              _be_exec_version(_state->be_exec_version),
-              _pool(pool),
-              _column_return_type(params->column_return_type),
-              _filter_type(params->filter_type),
-              _filter_id(params->filter_id) {}
+            : RuntimePredicateWrapper(state, pool, params->column_return_type, params->filter_type,
+                                      params->filter_id) {};
     // for a 'tmp' runtime predicate wrapper
     // only could called assign method or as a param for merge
     RuntimePredicateWrapper(RuntimeFilterParamsContext* state, ObjectPool* pool,
@@ -350,7 +344,7 @@ public:
             insert_to_bloom_filter(bf);
         }
         // release in filter
-        _context.hybrid_set.reset(create_set(_column_return_type));
+        _context.hybrid_set.reset();
     }
 
     Status init_bloom_filter(const size_t build_bf_cardinality) {
@@ -475,12 +469,12 @@ public:
                 break;
             } else if (wrapper->_is_ignored_in_filter) {
                 VLOG_DEBUG << " ignore merge runtime filter(in filter id " << _filter_id
-                           << ") because: " << *(wrapper->get_ignored_in_filter_msg());
+                           << ") because: " << wrapper->get_ignored_in_filter_msg();
 
                 _is_ignored_in_filter = true;
                 _ignored_in_filter_msg = wrapper->_ignored_in_filter_msg;
                 // release in filter
-                _context.hybrid_set.reset(create_set(_column_return_type));
+                _context.hybrid_set.reset();
                 break;
             }
             // try insert set
@@ -491,14 +485,14 @@ public:
                 msg << " ignore merge runtime filter(in filter id " << _filter_id
                     << ") because: in_num(" << _context.hybrid_set->size() << ") >= max_in_num("
                     << _max_in_num << ")";
-                _ignored_in_filter_msg = _pool->add(new std::string(msg.str()));
+                _ignored_in_filter_msg = std::string(msg.str());
 #else
-                _ignored_in_filter_msg = _pool->add(new std::string("ignored"));
+                _ignored_in_filter_msg = std::string("ignored");
 #endif
                 _is_ignored_in_filter = true;
 
                 // release in filter
-                _context.hybrid_set.reset(create_set(_column_return_type));
+                _context.hybrid_set.reset();
             }
             break;
         }
@@ -529,7 +523,7 @@ public:
                     CHECK(!wrapper->_is_ignored_in_filter)
                             << " can not ignore merge runtime filter(in filter id "
                             << wrapper->_filter_id << ") when used IN_OR_BLOOM_FILTER, ignore msg: "
-                            << *(wrapper->get_ignored_in_filter_msg());
+                            << wrapper->get_ignored_in_filter_msg();
                     _context.hybrid_set->insert(wrapper->_context.hybrid_set.get());
                     if (_max_in_num >= 0 && _context.hybrid_set->size() >= _max_in_num) {
                         VLOG_DEBUG << " change runtime filter to bloom filter(id=" << _filter_id
@@ -549,7 +543,7 @@ public:
                     CHECK(!wrapper->_is_ignored_in_filter)
                             << " can not ignore merge runtime filter(in filter id "
                             << wrapper->_filter_id << ") when used IN_OR_BLOOM_FILTER, ignore msg: "
-                            << *(wrapper->get_ignored_in_filter_msg());
+                            << wrapper->get_ignored_in_filter_msg();
                     wrapper->insert_to_bloom_filter(_context.bloom_filter_func.get());
                     // bloom filter merge bloom filter
                 } else {
@@ -566,14 +560,15 @@ public:
     }
 
     Status assign(const PInFilter* in_filter) {
-        PrimitiveType type = to_primitive_type(in_filter->column_type());
         if (in_filter->has_ignored_msg()) {
             VLOG_DEBUG << "Ignore in filter(id=" << _filter_id
                        << ") because: " << in_filter->ignored_msg();
             _is_ignored_in_filter = true;
-            _ignored_in_filter_msg = _pool->add(new std::string(in_filter->ignored_msg()));
+            _ignored_in_filter_msg = in_filter->ignored_msg();
             return Status::OK();
         }
+
+        PrimitiveType type = to_primitive_type(in_filter->column_type());
         _context.hybrid_set.reset(create_set(type));
         switch (type) {
         case TYPE_BOOLEAN: {
@@ -883,19 +878,15 @@ public:
         return Status::InvalidArgument("not support!");
     }
 
-    Status get_in_filter_iterator(HybridSetBase::IteratorBase** it) {
-        *it = _context.hybrid_set->begin();
-        return Status::OK();
-    }
+    HybridSetBase::IteratorBase* get_in_filter_iterator() { return _context.hybrid_set->begin(); }
 
-    Status get_bloom_filter_desc(char** data, int* filter_length) {
-        return _context.bloom_filter_func->get_data(data, filter_length);
+    void get_bloom_filter_desc(char** data, int* filter_length) {
+        _context.bloom_filter_func->get_data(data, filter_length);
     }
 
-    Status get_minmax_filter_desc(void** min_data, void** max_data) {
+    void get_minmax_filter_desc(void** min_data, void** max_data) {
         *min_data = _context.minmax_func->get_min();
         *max_data = _context.minmax_func->get_max();
-        return Status::OK();
     }
 
     PrimitiveType column_type() { return _column_return_type; }
@@ -904,7 +895,7 @@ public:
 
     bool is_ignored_in_filter() const { return _is_ignored_in_filter; }
 
-    std::string* get_ignored_in_filter_msg() const { return _ignored_in_filter_msg; }
+    const std::string& get_ignored_in_filter_msg() const { return _ignored_in_filter_msg; }
 
     void batch_assign(const PInFilter* filter,
                       void (*assign_func)(std::shared_ptr<HybridSetBase>& _hybrid_set,
@@ -948,7 +939,7 @@ private:
     vectorized::SharedRuntimeFilterContext _context;
     bool _is_bloomfilter = false;
     bool _is_ignored_in_filter = false;
-    std::string* _ignored_in_filter_msg = nullptr;
+    std::string _ignored_in_filter_msg;
     uint32_t _filter_id;
 };
 
@@ -962,13 +953,8 @@ Status IRuntimeFilter::create(RuntimeFilterParamsContext* state, ObjectPool* poo
                                   is_global ? false : build_bf_exactly);
 }
 
-void IRuntimeFilter::copy_to_shared_context(vectorized::SharedRuntimeFilterContext& context) {
-    context = _wrapper->_context;
-}
-
-Status IRuntimeFilter::copy_from_shared_context(vectorized::SharedRuntimeFilterContext& context) {
-    _wrapper->_context = context;
-    return Status::OK();
+vectorized::SharedRuntimeFilterContext& IRuntimeFilter::get_shared_context_ref() {
+    return _wrapper->_context;
 }
 
 void IRuntimeFilter::copy_from_other(IRuntimeFilter* other) {
@@ -1170,7 +1156,7 @@ void IRuntimeFilter::set_ignored(const std::string& msg) {
     _is_ignored = true;
     if (_wrapper->_filter_type == RuntimeFilterType::IN_FILTER) {
         _wrapper->_is_ignored_in_filter = true;
-        _wrapper->_ignored_in_filter_msg = _pool->add(new std::string(msg));
+        _wrapper->_ignored_in_filter_msg = msg;
     }
 }
 
@@ -1363,14 +1349,12 @@ void IRuntimeFilter::update_runtime_filter_type_to_profile() {
 
 Status IRuntimeFilter::merge_from(const RuntimePredicateWrapper* wrapper) {
     if (!_is_ignored && wrapper->is_ignored_in_filter()) {
-        std::string* msg = wrapper->get_ignored_in_filter_msg();
-        set_ignored(msg ? *msg : "");
+        set_ignored(wrapper->get_ignored_in_filter_msg());
     }
     auto origin_type = _wrapper->get_real_type();
     Status status = _wrapper->merge(wrapper);
     if (!_is_ignored && _wrapper->is_ignored_in_filter()) {
-        std::string* msg = _wrapper->get_ignored_in_filter_msg();
-        set_ignored(msg ? *msg : "");
+        set_ignored(_wrapper->get_ignored_in_filter_msg());
     }
     if (origin_type != _wrapper->get_real_type()) {
         update_runtime_filter_type_to_profile();
@@ -1403,7 +1387,7 @@ Status IRuntimeFilter::serialize_impl(T* request, void** data, int* len) {
         auto in_filter = request->mutable_in_filter();
         to_protobuf(in_filter);
     } else if (real_runtime_filter_type == RuntimeFilterType::BLOOM_FILTER) {
-        RETURN_IF_ERROR(_wrapper->get_bloom_filter_desc((char**)data, len));
+        _wrapper->get_bloom_filter_desc((char**)data, len);
         DCHECK(data != nullptr);
         request->mutable_bloom_filter()->set_filter_length(*len);
         request->mutable_bloom_filter()->set_always_true(false);
@@ -1427,8 +1411,7 @@ void IRuntimeFilter::to_protobuf(PInFilter* filter) {
         return;
     }
 
-    HybridSetBase::IteratorBase* it;
-    static_cast<void>(_wrapper->get_in_filter_iterator(&it));
+    auto it = _wrapper->get_in_filter_iterator();
     DCHECK(it != nullptr);
 
     switch (column_type) {
@@ -1554,7 +1537,7 @@ void IRuntimeFilter::to_protobuf(PInFilter* filter) {
 void IRuntimeFilter::to_protobuf(PMinMaxFilter* filter) {
     void* min_data = nullptr;
     void* max_data = nullptr;
-    static_cast<void>(_wrapper->get_minmax_filter_desc(&min_data, &max_data));
+    _wrapper->get_minmax_filter_desc(&min_data, &max_data);
     DCHECK(min_data != nullptr && max_data != nullptr);
     filter->set_column_type(to_proto(_wrapper->column_type()));
 
@@ -1673,7 +1656,8 @@ bool IRuntimeFilter::is_bloomfilter() {
     return _wrapper->is_bloomfilter();
 }
 
-Status IRuntimeFilter::update_filter(const UpdateRuntimeFilterParams* param) {
+template <typename T>
+Status IRuntimeFilter::_update_filter(const T* param) {
     if (param->request->has_in_filter() && param->request->in_filter().has_ignored_msg()) {
         const PInFilter in_filter = param->request->in_filter();
         set_ignored(in_filter.ignored_msg());
@@ -1691,26 +1675,15 @@ Status IRuntimeFilter::update_filter(const UpdateRuntimeFilterParams* param) {
     return Status::OK();
 }
 
+Status IRuntimeFilter::update_filter(const UpdateRuntimeFilterParams* param) {
+    return _update_filter(param);
+}
+
 Status IRuntimeFilter::update_filter(const UpdateRuntimeFilterParamsV2* param,
                                      int64_t start_apply) {
-    if (param->request->has_in_filter() && param->request->in_filter().has_ignored_msg()) {
-        const PInFilter in_filter = param->request->in_filter();
-        set_ignored(in_filter.ignored_msg());
-    }
-
-    std::unique_ptr<RuntimePredicateWrapper> tmp_wrapper;
-    RETURN_IF_ERROR(IRuntimeFilter::create_wrapper(_state, param, _pool, &tmp_wrapper));
-    auto origin_type = _wrapper->get_real_type();
-    RETURN_IF_ERROR(_wrapper->merge(tmp_wrapper.get()));
-    if (origin_type != _wrapper->get_real_type()) {
-        update_runtime_filter_type_to_profile();
-    }
-    this->signal();
-
-    _profile->add_info_string("MergeTime", std::to_string(param->request->merge_time()) + " ms");
     _profile->add_info_string("UpdateTime",
                               std::to_string(MonotonicMillis() - start_apply) + " ms");
-    return Status::OK();
+    return _update_filter(param);
 }
 
 Status RuntimePredicateWrapper::get_push_exprs(std::list<vectorized::VExprContextSPtr>& probe_ctxs,
diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h
index fc324c1c1be..bc487bfe9c9 100644
--- a/be/src/exprs/runtime_filter.h
+++ b/be/src/exprs/runtime_filter.h
@@ -218,8 +218,7 @@ public:
                          bool build_bf_exactly = false, bool is_global = false,
                          int parallel_tasks = 0);
 
-    void copy_to_shared_context(vectorized::SharedRuntimeFilterContext& context);
-    Status copy_from_shared_context(vectorized::SharedRuntimeFilterContext& context);
+    vectorized::SharedRuntimeFilterContext& get_shared_context_ref();
 
     void copy_from_other(IRuntimeFilter* other);
 
@@ -367,6 +366,9 @@ protected:
     void to_protobuf(PInFilter* filter);
     void to_protobuf(PMinMaxFilter* filter);
 
+    template <class T>
+    Status _update_filter(const T* param);
+
     template <class T>
     Status serialize_impl(T* request, void** data, int* len);
 
diff --git a/be/src/exprs/runtime_filter_slots.h b/be/src/exprs/runtime_filter_slots.h
index 4859734a6a4..8f5dab22f8c 100644
--- a/be/src/exprs/runtime_filter_slots.h
+++ b/be/src/exprs/runtime_filter_slots.h
@@ -91,6 +91,10 @@ public:
         std::sort(sorted_runtime_filter_descs.begin(), sorted_runtime_filter_descs.end(),
                   compare_desc);
 
+        // do not create 'in filter' when hash_table size over limit
+        const auto max_in_num = state->runtime_filter_max_in_num();
+        const bool over_max_in_num = (hash_table_size >= max_in_num);
+
         for (auto& filter_desc : sorted_runtime_filter_descs) {
             IRuntimeFilter* runtime_filter = nullptr;
             RETURN_IF_ERROR(state->runtime_filter_mgr()->get_producer_filter(filter_desc.filter_id,
@@ -103,10 +107,6 @@ public:
                         runtime_filter->expr_order(), _build_expr_context.size());
             }
 
-            // do not create 'in filter' when hash_table size over limit
-            auto max_in_num = state->runtime_filter_max_in_num();
-            bool over_max_in_num = (hash_table_size >= max_in_num);
-
             bool is_in_filter = (runtime_filter->type() == RuntimeFilterType::IN_FILTER);
 
             if (over_max_in_num &&
@@ -213,8 +213,7 @@ public:
     void copy_to_shared_context(vectorized::SharedHashTableContextPtr& context) {
         for (auto& it : _runtime_filters) {
             for (auto& filter : it.second) {
-                auto& target = context->runtime_filters[filter->filter_id()];
-                filter->copy_to_shared_context(target);
+                context->runtime_filters[filter->filter_id()] = filter->get_shared_context_ref();
             }
         }
     }
@@ -227,7 +226,7 @@ public:
                 if (ret == context->runtime_filters.end()) {
                     return Status::Aborted("invalid runtime filter id: {}", filter_id);
                 }
-                RETURN_IF_ERROR(filter->copy_from_shared_context(ret->second));
+                filter->get_shared_context_ref() = ret->second;
             }
         }
         return Status::OK();
diff --git a/be/src/exprs/runtime_filter_slots_cross.h b/be/src/exprs/runtime_filter_slots_cross.h
index 76b6085bab9..7b1a2063d15 100644
--- a/be/src/exprs/runtime_filter_slots_cross.h
+++ b/be/src/exprs/runtime_filter_slots_cross.h
@@ -82,7 +82,7 @@ public:
         return Status::OK();
     }
 
-    bool empty() { return _runtime_filters.empty(); }
+    bool empty() const { return _runtime_filters.empty(); }
 
 private:
     const std::vector<TRuntimeFilterDesc>& _runtime_filter_descs;
diff --git a/be/src/runtime/runtime_filter_mgr.cpp b/be/src/runtime/runtime_filter_mgr.cpp
index 29bf22535ed..3a01368b583 100644
--- a/be/src/runtime/runtime_filter_mgr.cpp
+++ b/be/src/runtime/runtime_filter_mgr.cpp
@@ -116,7 +116,7 @@ Status RuntimeFilterMgr::register_consumer_filter(const TRuntimeFilterDesc& desc
         }
     }
 
-    // TODO: make the two case as one case to judge
+    // TODO: union the remote opt and global two case as one case to one judge
     bool remote_opt_or_global =
             (desc.__isset.opt_remote_rf && desc.opt_remote_rf && desc.has_remote_targets &&
              desc.type == TRuntimeFilterType::BLOOM) ||


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


(doris) 06/19: [typo](doc) Modify the alter table column document (#28645)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 47e86d77919699a519803232f1a596f0aef33d8a
Author: yuanyuan8983 <99...@users.noreply.github.com>
AuthorDate: Wed Jan 31 14:47:27 2024 +0800

    [typo](doc) Modify the alter table column document (#28645)
---
 .../Alter/ALTER-TABLE-COLUMN.md                    | 398 ++++++++++++--------
 .../Alter/ALTER-TABLE-COLUMN.md                    | 414 ++++++++++++---------
 2 files changed, 478 insertions(+), 334 deletions(-)

diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-COLUMN.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-COLUMN.md
index 3e477cead22..1276ccddbfa 100644
--- a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-COLUMN.md
+++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-COLUMN.md
@@ -24,9 +24,7 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-## ALTER-TABLE-COLUMN
-
-### Name
+## Name
 
 ALTER TABLE COLUMN
 
@@ -34,7 +32,13 @@ ALTER TABLE COLUMN
 
 This statement is used to perform a schema change operation on an existing table. The schema change is asynchronous, and the task is returned when the task is submitted successfully. After that, you can use the [SHOW ALTER TABLE COLUMN](../../Show-Statements/SHOW-ALTER.md) command to view the progress.
 
-grammar:
+Doris has the concept of materialized index after table construction. After successful table construction, it is the base table and the materialized index is the base index. rollup index can be created based on the base table. Both base index and rollup index are materialized indexes. If rollup_index_name is not specified during the schema change operation, the operation is based on the base table by default.
+
+:::note
+Doris 1.2.0 supports light schema change for light scale structure changes, and addition and subtraction operations for value columns can be completed more quickly and synchronously. You can manually specify "light_schema_change" = 'true' when creating a table. This parameter is enabled by default for versions 2.0.0 and later.
+:::
+
+### Grammar:
 
 ```sql
 ALTER TABLE [database.]table alter_clause;
@@ -42,186 +46,256 @@ ALTER TABLE [database.]table alter_clause;
 
 The alter_clause of schema change supports the following modification methods:
 
-1. Add a column to the specified position at the specified index
+**1. Add a column to the specified position at the specified index**
 
-grammar:
+**Grammar**
 
 ```sql
-ADD COLUMN column_name column_type [KEY | agg_type] [DEFAULT "default_value"]
+ALTER TABLE [database.]table table_name ADD COLUMN column_name column_type [KEY | agg_type] [DEFAULT "default_value"]
 [AFTER column_name|FIRST]
 [TO rollup_index_name]
 [PROPERTIES ("key"="value", ...)]
 ```
-
- Notice:
-
+  
+**Example**
+  
+1. Add a key column new_col to example_db.my_table after key_1 (non-aggregated model)
+  
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN new_col INT KEY DEFAULT "0" AFTER key_1;
+  ```
+  
+2. Add a value column new_col to example_db.my_table after value_1 (non-aggregate model)
+  
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN new_col INT DEFAULT "0" AFTER value_1;
+  ```
+
+3. Add a key column new_col (aggregate model) to example_db.my_table after key_1
+
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN new_col INT KEY DEFAULT "0" AFTER key_1;
+  ```
+
+4. Add a value column to example_db.my_table after value_1 new_col SUM Aggregation type (aggregation model)
+
+  ```sql
+  ALTER TABLE example_db.my_table   
+  ADD COLUMN new_col INT SUM DEFAULT "0" AFTER value_1; 
+  ```
+
+5. Add new_col to the first column position of the example_db.my_table table (non-aggregated model)
+
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN new_col INT KEY DEFAULT "0" FIRST;
+  ```
+
+:::note 
 - If you add a value column to the aggregation model, you need to specify agg_type
 - For non-aggregated models (such as DUPLICATE KEY), if you add a key column, you need to specify the KEY keyword
-- You cannot add columns that already exist in the base index to the rollup index (you can recreate a rollup index if necessary)
+- You cannot add columns that already exist in the base index to the rollup index (you can recreate a rollup index if necessary) :::
 
-2. Add multiple columns to the specified index
 
-grammar:
+**2. Add multiple columns to the specified index**
 
+**Grammar**
+  
 ```sql
-ADD COLUMN (column_name1 column_type [KEY | agg_type] DEFAULT "default_value", ...)
+ALTER TABLE [database.]table table_name ADD COLUMN (column_name1 column_type [KEY | agg_type] DEFAULT "default_value", ...)
 [TO rollup_index_name]
 [PROPERTIES ("key"="value", ...)]
 ```
 
-Notice:
-
-- If you add a value column to the aggregation model, you need to specify agg_type
-- If you add a key column to the aggregation model, you need to specify the KEY keyword
-- You cannot add columns that already exist in the base index to the rollup index (you can recreate a rollup index if necessary)
-
-3. Delete a column from the specified index
-
-grammar:
-
+  
+**Example**
+  
+1. Add multiple columns to example_db.my_table, where new_col and new_col2 are SUM aggregate types (aggregate model)
+  
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN (new_col1 INT SUM DEFAULT "0" ,new_col2 INT SUM DEFAULT "0");
+  ```
+  
+2. Add multiple columns to example_db.my_table (non-aggregated model), where new_col1 is the KEY column and new_col2 is the value column
+  
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN (new_col1 INT key DEFAULT "0" , new_col2 INT DEFAULT "0");
+  ```
+
+:::note
+  - If you add a value column to the aggregation model, you need to specify agg_type
+  - If you add a key column to the aggregation model, you need to specify the KEY keyword
+  - You cannot add columns that already exist in the base index to the rollup index (you can recreate a rollup index if necessary) :::
+
+**3. Delete a column from the specified index**
+
+**Grammar***
+  
+  ```sql
+  ALTER TABLE [database.]table table_name DROP COLUMN column_name
+  [FROM rollup_index_name]
+  ```
+
+**Example**
+   
+1. Delete column col1 from example_db.my_table
+  
+  ```sql
+  ALTER TABLE example_db.my_table DROP COLUMN col1;
+  ```
+
+:::note
+  - Cannot drop partition column
+  - The aggregate model cannot delete KEY columns
+  - If the column is removed from the base index, it will also be removed if it is included in the rollup index :::
+
+**4. Modify the column type and column position of the specified index**
+
+**Grammar**
+  
 ```sql
-DROP COLUMN column_name
-[FROM rollup_index_name]
-```
-
-Notice:
-
-- Cannot drop partition column
-- If the column is removed from the base index, it will also be removed if it is included in the rollup index
-
-4. Modify the column type and column position of the specified index
-
- grammar:
-
-```sql
-MODIFY COLUMN column_name column_type [KEY | agg_type] [NULL | NOT NULL] [DEFAULT "default_value"]
+ALTER TABLE [database.]table table_name MODIFY COLUMN column_name column_type [KEY | agg_type] [NULL | NOT NULL] [DEFAULT "default_value"]
 [AFTER column_name|FIRST]
 [FROM rollup_index_name]
 [PROPERTIES ("key"="value", ...)]
 ```
-
-Notice:
-
-- If you modify the value column in the aggregation model, you need to specify agg_type
-- If you modify the key column for non-aggregate types, you need to specify the KEY keyword
-- Only the type of the column can be modified, and other attributes of the column remain as they are (that is, other attributes need to be explicitly written in the statement according to the original attributes, see example 8)
-- Partitioning and bucketing columns cannot be modified in any way
-- The following types of conversions are currently supported (loss of precision is guaranteed by the user)
-  - Conversion of TINYINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE types to larger numeric types
-  - Convert TINTINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE/DECIMAL to VARCHAR
-  - VARCHAR supports modifying the maximum length
-  - VARCHAR/CHAR converted to TINTINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE
-  - Convert VARCHAR/CHAR to DATE (currently supports "%Y-%m-%d", "%y-%m-%d", "%Y%m%d", "%y%m%d", "%Y/%m/%d, "%y/%m/%d" six formats)
-  - Convert DATETIME to DATE (only keep year-month-day information, for example: `2019-12-09 21:47:05` <--> `2019-12-09`)
-  - DATE is converted to DATETIME (hours, minutes and seconds are automatically filled with zeros, for example: `2019-12-09` <--> `2019-12-09 00:00:00`)
-  - Convert FLOAT to DOUBLE
-  - INT is converted to DATE (if the INT type data is illegal, the conversion fails, and the original data remains unchanged)
-  - All can be converted to STRING except DATE and DATETIME, but STRING cannot be converted to any other type
-
-5. Reorder the column at the specified index
-
-grammar:
-
-```sql
-ORDER BY (column_name1, column_name2, ...)
-[FROM rollup_index_name]
-[PROPERTIES ("key"="value", ...)]
-```
-
-Notice:
-
-- All columns in index are written out
-- the value column comes after the key column
-
-### Example
-
-1. Add a key column new_col after col1 of example_rollup_index (non-aggregated model)
-
-```sql
-ALTER TABLE example_db.my_table
-ADD COLUMN new_col INT KEY DEFAULT "0" AFTER col1
-TO example_rollup_index;
-```
-
-2. Add a value column new_col after col1 of example_rollup_index (non-aggregation model)
-
-```sql
-ALTER TABLE example_db.my_table
-ADD COLUMN new_col INT DEFAULT "0" AFTER col1
-TO example_rollup_index;
-```
-
-3. Add a key column new_col (aggregation model) after col1 of example_rollup_index
-
-```sql
-ALTER TABLE example_db.my_table
-ADD COLUMN new_col INT DEFAULT "0" AFTER col1
-TO example_rollup_index;
-```
-
-4. Add a value column new_col SUM aggregation type (aggregation model) after col1 of example_rollup_index
-
-```sql
-ALTER TABLE example_db.my_table
-ADD COLUMN new_col INT SUM DEFAULT "0" AFTER col1
-TO example_rollup_index;
-```
-
-5. Add multiple columns to example_rollup_index (aggregation model)
-
-```sql
-ALTER TABLE example_db.my_table
-ADD COLUMN (col1 INT DEFAULT "1", col2 FLOAT SUM DEFAULT "2.3")
-TO example_rollup_index;
-```
-
-6. Remove a column from example_rollup_index
-
-```sql
-ALTER TABLE example_db.my_table
-DROP COLUMN col2
-FROM example_rollup_index;
-```
-
-7. Modify the type of the key column col1 of the base index to BIGINT and move it to the back of the col2 column.
-
-```sql
-ALTER TABLE example_db.my_table
-MODIFY COLUMN col1 BIGINT KEY DEFAULT "1" AFTER col2;
-```
-
-Note: Whether you modify the key column or the value column, you need to declare complete column information
-
-8. Modify the maximum length of the val1 column of base index. The original val1 is (val1 VARCHAR(32) REPLACE DEFAULT "abc")
-
-```sql
-ALTER TABLE example_db.my_table
-MODIFY COLUMN val1 VARCHAR(64) REPLACE DEFAULT "abc";
-```
-Note: You can only modify the column's data type; other attributes of the column must remain unchanged.
-
-9. Reorder the columns in example_rollup_index (set the original column order as: k1,k2,k3,v1,v2)
-
-```sql
-ALTER TABLE example_db.my_table
-ORDER BY (k3,k1,k2,v2,v1)
-FROM example_rollup_index;
-```
-
-10. Do Two Actions Simultaneously
-
-```sql
-ALTER TABLE example_db.my_table
-ADD COLUMN v2 INT MAX DEFAULT "0" AFTER k2 TO example_rollup_index,
-ORDER BY (k3,k1,k2,v2,v1) FROM example_rollup_index;
-```
-
-11. Modify the length of a field in the Key column of the Duplicate key table
-
-```sql
-alter table example_tbl modify column k3 varchar(50) key null comment 'to 50'
-````
-
+  
+**Example**
+  
+1. Modify the type of the key column col1 of the base index to BIGINT and move it to the back of the col2 column
+
+  ```sql
+  ALTER TABLE example_db.my_table 
+  MODIFY COLUMN col1 BIGINT KEY DEFAULT "1" AFTER col2;
+  ```
+
+  :::note
+  Whether you modify the key column or the value column, you need to declare complete column information
+  :::
+
+2. Modify the maximum length of the val1 column of base index. The original val1 is (val1 VARCHAR(32) REPLACE DEFAULT "abc")
+
+  ```sql
+  ALTER TABLE example_db.my_table 
+  MODIFY COLUMN val1 VARCHAR(64) REPLACE DEFAULT "abc";
+  ```
+
+  :::note
+  You can only modify the column's data type; other attributes of the column must remain unchanged.
+  :::
+
+3. Modify the length of a field in the Key column of the Duplicate key table
+
+  ```sql
+  ALTER TABLE example_db.my_table 
+  MODIFY COLUMN k3 VARCHAR(50) KEY NULL COMMENT 'to 50';
+  ```
+
+:::note
+  - If you modify the value column in the aggregation model, you need to specify agg_type
+  - If you modify the key column for non-aggregate types, you need to specify the KEY keyword
+  - Only the type of the column can be modified, and other attributes of the column remain as they are (that is, other attributes need to be explicitly written in the statement according to the   original attributes, see example 8)
+  - Partitioning and bucketing columns cannot be modified in any way
+  - The following types of conversions are currently supported (loss of precision is guaranteed by the user)
+    - Conversion of TINYINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE types to larger numeric types
+    - Convert TINTINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE/DECIMAL to VARCHAR
+    - VARCHAR supports modifying the maximum length
+    - VARCHAR/CHAR converted to TINTINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE
+    - Convert VARCHAR/CHAR to DATE (currently supports "%Y-%m-%d", "%y-%m-%d", "%Y%m%d", "%y%m%d", "%Y/%m/%d, "%y/%m/%d" six formats)
+    - Convert DATETIME to DATE (only keep year-month-day information, for example: `2019-12-09 21:47:05` <--> `2019-12-09`)
+    - DATE is converted to DATETIME (hours, minutes and seconds are automatically filled with zeros, for example: `2019-12-09` <--> `2019-12-09 00:00:00`)
+    - Convert FLOAT to DOUBLE
+    - INT is converted to DATE (if the INT type data is illegal, the conversion fails, and the original data remains unchanged)
+    - All can be converted to STRING except DATE and DATETIME, but STRING cannot be converted to any other type :::
+
+**5. Reorder the column at the specified index**
+
+**Grammar**
+  
+  ```sql
+  ALTER TABLE [database.]table table_name ORDER BY (column_name1, column_name2, ...)
+  [FROM rollup_index_name]
+  [PROPERTIES ("key"="value", ...)]
+  ```
+  
+**Example**
+  
+1. Adjust the order of the key and value columns of example_db.my_table (non-aggregate model)
+  
+  ```sql
+  CREATE TABLE `my_table`(
+  `k_1` INT NULL,
+  `k_2` INT NULL,
+  `v_1` INT NULL,
+  `v_2` varchar NULL,
+  `v_3` varchar NULL
+  ) ENGINE=OLAP
+  DUPLICATE KEY(`k_1`, `k_2`)
+  COMMENT 'OLAP'
+  DISTRIBUTED BY HASH(`k_1`) BUCKETS 5
+  PROPERTIES (
+  "replication_allocation" = "tag.location.default: 1"
+  );
+
+  ALTER TABLE example_db.my_table ORDER BY (k_2,k_1,v_3,v_2,v_1);
+
+  mysql> desc my_table;
+  +-------+------------+------+-------+---------+-------+
+  | Field | Type       | Null | Key   | Default | Extra |
+  +-------+------------+------+-------+---------+-------+
+  | k_2   | INT        | Yes  | true  | NULL    |       |
+  | k_1   | INT        | Yes  | true  | NULL    |       |
+  | v_3   | VARCHAR(*) | Yes  | false | NULL    | NONE  |
+  | v_2   | VARCHAR(*) | Yes  | false | NULL    | NONE  |
+  | v_1   | INT        | Yes  | false | NULL    | NONE  |
+  +-------+------------+------+-------+---------+-------+
+  ```
+  
+2. Do two actions simultaneously
+  
+  ```sql
+  CREATE TABLE `my_table` (
+  `k_1` INT NULL,
+  `k_2` INT NULL,
+  `v_1` INT NULL,
+  `v_2` varchar NULL,
+  `v_3` varchar NULL
+  ) ENGINE=OLAP
+  DUPLICATE KEY(`k_1`, `k_2`)
+  COMMENT 'OLAP'
+  DISTRIBUTED BY HASH(`k_1`) BUCKETS 5
+  PROPERTIES (
+  "replication_allocation" = "tag.location.default: 1"
+  );
+
+  ALTER TABLE example_db.my_table
+  ADD COLUMN col INT DEFAULT "0" AFTER v_1,
+  ORDER BY (k_2,k_1,v_3,v_2,v_1,col);
+
+  mysql> desc my_table;
+  +-------+------------+------+-------+---------+-------+
+  | Field | Type       | Null | Key   | Default | Extra |
+  +-------+------------+------+-------+---------+-------+
+  | k_2   | INT        | Yes  | true  | NULL    |       |
+  | k_1   | INT        | Yes  | true  | NULL    |       |
+  | v_3   | VARCHAR(*) | Yes  | false | NULL    | NONE  |
+  | v_2   | VARCHAR(*) | Yes  | false | NULL    | NONE  |
+  | v_1   | INT        | Yes  | false | NULL    | NONE  |
+  | col   | INT        | Yes  | false | 0       | NONE  |
+  +-------+------------+------+-------+---------+-------+
+  ```
+
+:::note
+  - All columns in index are written out
+  - the value column comes after the key column
+  - You can adjust the key column only within the range of the key column. The same applies to the value column
+:::
+  
 ### Keywords
 
 ```text
diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-COLUMN.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-COLUMN.md
index 6394ba62947..db43a0ac8fb 100644
--- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-COLUMN.md
+++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-COLUMN.md
@@ -24,9 +24,7 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-## ALTER-TABLE-COLUMN
-
-### Name
+## Name
 
 ALTER TABLE COLUMN
 
@@ -34,7 +32,12 @@ ALTER TABLE COLUMN
 
 该语句用于对已有 table 进行 Schema change 操作。schema change 是异步的,任务提交成功则返回,之后可使用[SHOW ALTER TABLE COLUMN](../../Show-Statements/SHOW-ALTER.md) 命令查看进度。
 
-语法:
+:::note
+Doris 在建表之后有物化索引的概念,在建表成功后为 base 表,物化索引为 base index,基于 base 表可以创建 rollup index。其中 base index 和 rollup index 都是物化索引,在进行 schema change 操作时如果不指定 rollup_index_name 默认基于 base 表进行操作。
+Doris 在 1.2.0 支持了 light schema change 轻量表结构变更,对于值列的加减操作,可以更快地,同步地完成。可以在建表时手动指定 "light_schema_change" = 'true',2.0.0 及之后版本该参数默认开启。
+:::
+
+### 语法:
 
 ```sql
 ALTER TABLE [database.]table alter_clause;
@@ -42,187 +45,254 @@ ALTER TABLE [database.]table alter_clause;
 
 schema change 的 alter_clause 支持如下几种修改方式:
 
-1. 向指定 index 的指定位置添加一列
+**1. 添加列,向指定的 index 位置进行列添加**
 
-语法:
+**语法**
 
 ```sql
-ADD COLUMN column_name column_type [KEY | agg_type] [DEFAULT "default_value"]
+ALTER TABLE [database.]table table_name ADD COLUMN column_name column_type [KEY | agg_type] [DEFAULT "default_value"]
 [AFTER column_name|FIRST]
 [TO rollup_index_name]
 [PROPERTIES ("key"="value", ...)]
 ```
-
- 注意:
-
-- 聚合模型如果增加 value 列,需要指定 agg_type
-- 非聚合模型(如 DUPLICATE KEY)如果增加key列,需要指定KEY关键字
--  不能在 rollup index 中增加 base index 中已经存在的列(如有需要,可以重新创建一个 rollup index)
-
-2. 向指定 index 添加多列
-
-语法:
-
+  
+**Example**
+  
+1. 向 example_db.my_table 的 key_1 后添加一个 key 列 new_col (非聚合模型)
+
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN new_col INT KEY DEFAULT "0" AFTER key_1;
+  ```
+
+2. 向 example_db.my_table 的 value_1 后添加一个 value 列 new_col (非聚合模型)
+
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN new_col INT DEFAULT "0" AFTER value_1;
+  ```
+
+3. 向 example_db.my_table 的 key_1 后添加一个 key 列 new_col (聚合模型)
+
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN new_col INT KEY DEFAULT "0" AFTER key_1;
+  ```
+
+4. 向 example_db.my_table 的 value_1 后添加一个 value 列 new_col SUM 聚合类型 (聚合模型)
+
+  ```sql
+  ALTER TABLE example_db.my_table   
+  ADD COLUMN new_col INT SUM DEFAULT "0" AFTER value_1; 
+  ```
+
+5. 将 new_col 添加到 example_db.my_table 表的首列位置 (非聚合模型)
+
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN new_col INT KEY DEFAULT "0" FIRST;
+  ```
+  
+  :::note
+  - 聚合模型如果增加 value 列,需要指定 agg_type
+  - 非聚合模型(如 DUPLICATE KEY)如果增加key列,需要指定KEY关键字
+  - 不能在 rollup index 中增加 base index 中已经存在的列(如有需要,可以重新创建一个 rollup index):::
+
+**2. 添加多列,向指定的 index 位置进行多列添加**
+
+**语法**
+  
+  ```sql
+  ALTER TABLE [database.]table table_name ADD COLUMN (column_name1 column_type [KEY | agg_type] DEFAULT "default_value", ...)
+  [TO rollup_index_name]
+  [PROPERTIES ("key"="value", ...)]
+  ```
+  
+**Example**
+  
+1. 向 example_db.my_table 中添加多列,new_col 和 new_col2 都是 SUM 聚合类型(聚合模型)
+
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN (new_col1 INT SUM DEFAULT "0" ,new_col2 INT SUM DEFAULT "0");
+  ```
+
+2. 向 example_db.my_table 中添加多列(非聚合模型),其中 new_col1 为 KEY 列,new_col2 为 value 列
+
+  ```sql
+  ALTER TABLE example_db.my_table
+  ADD COLUMN (new_col1 INT key DEFAULT "0" , new_col2 INT DEFAULT "0");
+  ```
+  
+:::note
+  
+  - 聚合模型如果增加 value 列,需要指定agg_type
+  - 聚合模型如果增加key列,需要指定KEY关键字
+  - 不能在 rollup index 中增加 base index 中已经存在的列(如有需要,可以重新创建一个 rollup index):::
+
+**3. 删除列,从指定 index 中删除一列**
+
+**语法**
+  
+  ```sql
+  ALTER TABLE [database.]table table_name DROP COLUMN column_name
+  [FROM rollup_index_name]
+  ```
+  
+**Example**
+   
+1. 从 example_db.my_table 中删除 col1 列
+  
+    ```sql
+    ALTER TABLE example_db.my_table DROP COLUMN col1;
+    ```
+
+:::note
+  - 不能删除分区列
+  - 聚合模型不能删除KEY列
+  - 如果是从 base index 中删除列,则如果 rollup index 中包含该列,也会被删除 :::
+
+**4. 修改指定列类型以及列位置**
+
+**语法**
+  
 ```sql
-ADD COLUMN (column_name1 column_type [KEY | agg_type] DEFAULT "default_value", ...)
-[TO rollup_index_name]
-[PROPERTIES ("key"="value", ...)]
-```
-
-注意:
-
-- 聚合模型如果增加 value 列,需要指定agg_type
-- 聚合模型如果增加key列,需要指定KEY关键字
-- 不能在 rollup index 中增加 base index 中已经存在的列(如有需要,可以重新创建一个 rollup index)
-
-3. 从指定 index 中删除一列
-
-语法:
-
-```sql
-DROP COLUMN column_name
-[FROM rollup_index_name]
-```
-
-注意:
-
-- 不能删除分区列
-- 如果是从 base index 中删除列,则如果 rollup index 中包含该列,也会被删除
-
-4. 修改指定 index 的列类型以及列位置
-
- 语法:
-
-```sql
-MODIFY COLUMN column_name column_type [KEY | agg_type] [NULL | NOT NULL] [DEFAULT "default_value"]
+ALTER TABLE [database.]table table_name MODIFY COLUMN column_name column_type [KEY | agg_type] [NULL | NOT NULL] [DEFAULT "default_value"]
 [AFTER column_name|FIRST]
 [FROM rollup_index_name]
 [PROPERTIES ("key"="value", ...)]
 ```
 
-注意:
-
-- 聚合模型如果修改 value 列,需要指定 agg_type
-- 非聚合类型如果修改key列,需要指定KEY关键字
-- 只能修改列的类型,列的其他属性维持原样(即其他属性需在语句中按照原属性显式的写出,参见 example 8)
-- 分区列和分桶列不能做任何修改
-- 目前支持以下类型的转换(精度损失由用户保证)
-  - TINYINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE 类型向范围更大的数字类型转换
-  - TINTINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE/DECIMAL 转换成 VARCHAR
-  - VARCHAR 支持修改最大长度
-  - VARCHAR/CHAR 转换成 TINTINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE
-  - VARCHAR/CHAR 转换成 DATE (目前支持"%Y-%m-%d", "%y-%m-%d", "%Y%m%d", "%y%m%d", "%Y/%m/%d, "%y/%m/%d"六种格式化格式)
-  - DATETIME 转换成 DATE(仅保留年-月-日信息, 例如: `2019-12-09 21:47:05` <--> `2019-12-09`)
-  - DATE 转换成 DATETIME(时分秒自动补零, 例如: `2019-12-09` <--> `2019-12-09 00:00:00`)
-  - FLOAT 转换成 DOUBLE
-  - INT 转换成 DATE (如果INT类型数据不合法则转换失败,原始数据不变)
-  - 除DATE与DATETIME以外都可以转换成STRING,但是STRING不能转换任何其他类型
-
-5. 对指定 index 的列进行重新排序
-
-语法:
-
-```sql
-ORDER BY (column_name1, column_name2, ...)
-[FROM rollup_index_name]
-[PROPERTIES ("key"="value", ...)]
-```
-
-注意:
-
-- index 中的所有列都要写出来
-- value 列在 key 列之后
-
-### Example
-
-1. 向 example_rollup_index 的 col1 后添加一个key列 new_col(非聚合模型)
-
-```sql
-ALTER TABLE example_db.my_table
-ADD COLUMN new_col INT KEY DEFAULT "0" AFTER col1
-TO example_rollup_index;
-```
-
-2. 向example_rollup_index的col1后添加一个value列new_col(非聚合模型)
-
-```sql
-ALTER TABLE example_db.my_table   
-ADD COLUMN new_col INT DEFAULT "0" AFTER col1    
-TO example_rollup_index;
-```
-
-3. 向example_rollup_index的col1后添加一个key列new_col(聚合模型)
-
-```sql
-ALTER TABLE example_db.my_table   
-ADD COLUMN new_col INT DEFAULT "0" AFTER col1    
-TO example_rollup_index;
-```
-
-4. 向example_rollup_index的col1后添加一个value列new_col SUM聚合类型(聚合模型)
-
-```sql
-ALTER TABLE example_db.my_table   
-ADD COLUMN new_col INT SUM DEFAULT "0" AFTER col1    
-TO example_rollup_index;
-```
-
-5. 向 example_rollup_index 添加多列(聚合模型)
-
-```sql
-ALTER TABLE example_db.my_table
-ADD COLUMN (col1 INT DEFAULT "1", col2 FLOAT SUM DEFAULT "2.3")
-TO example_rollup_index;
-```
-
-6. 从 example_rollup_index 删除一列
-
-```sql
-ALTER TABLE example_db.my_table
-DROP COLUMN col2
-FROM example_rollup_index;
-```
-
-7. 修改 base index 的 key 列 col1 的类型为 BIGINT,并移动到 col2 列后面。
-
-```sql
-ALTER TABLE example_db.my_table 
-MODIFY COLUMN col1 BIGINT KEY DEFAULT "1" AFTER col2;
-```
-
-注意:无论是修改 key 列还是 value 列都需要声明完整的 column 信息
-
-8. 修改 base index 的 val1 列最大长度。原 val1 为 (val1 VARCHAR(32) REPLACE DEFAULT "abc")
-
-```sql
-ALTER TABLE example_db.my_table 
-MODIFY COLUMN val1 VARCHAR(64) REPLACE DEFAULT "abc";
-```
-注意:只能修改列的类型,列的其他属性维持原样
-
-9. 重新排序 example_rollup_index 中的列(设原列顺序为:k1,k2,k3,v1,v2)
-
-```sql
-ALTER TABLE example_db.my_table
-ORDER BY (k3,k1,k2,v2,v1)
-FROM example_rollup_index;
-```
-
-10. 同时执行两种操作
-
-```sql
-ALTER TABLE example_db.my_table
-ADD COLUMN v2 INT MAX DEFAULT "0" AFTER k2 TO example_rollup_index,
-ORDER BY (k3,k1,k2,v2,v1) FROM example_rollup_index;
-```
-
-11. 修改Duplicate key 表 Key 列的某个字段的长度
-
-```sql
-alter table example_tbl modify column k3 varchar(50) key null comment 'to 50'
-```
-
-
+**Example**
+  
+1. 修改 example_db.my_table 的 key 列 col1 的类型为 BIGINT,并移动到 col2 列后面。
+
+  ```sql
+  ALTER TABLE example_db.my_table 
+  MODIFY COLUMN col1 BIGINT KEY DEFAULT "1" AFTER col2;
+  ```
+
+  :::note
+  无论是修改 key 列还是 value 列都需要声明完整的 column 信息
+  :::
+
+2. 修改 example_db.my_table 的 val1 列最大长度。原 val1 为 (val1 VARCHAR(32) REPLACE DEFAULT "abc")
+
+  ```sql
+  ALTER TABLE example_db.my_table 
+  MODIFY COLUMN val1 VARCHAR(64) REPLACE DEFAULT "abc";
+  ```
+
+  :::note
+  只能修改列的类型,列的其他属性维持原样
+  :::
+
+3. 修改 Duplicate key 表 Key 列的某个字段的长度
+
+  ```sql
+  ALTER TABLE example_db.my_table 
+  MODIFY COLUMN k3 VARCHAR(50) KEY NULL COMMENT 'to 50';
+  ```
+  
+  :::note
+  - 聚合模型如果修改 value 列,需要指定 agg_type
+  - 非聚合类型如果修改key列,需要指定KEY关键字
+  - 只能修改列的类型,列的其他属性维持原样(即其他属性需在语句中按照原属性显式的写出,参见 example 8)
+  - 分区列和分桶列不能做任何修改
+  - 目前支持以下类型的转换(精度损失由用户保证)
+    - TINYINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE 类型向范围更大的数字类型转换
+    - TINTINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE/DECIMAL 转换成 VARCHAR
+    - VARCHAR 支持修改最大长度
+    - VARCHAR/CHAR 转换成 TINTINT/SMALLINT/INT/BIGINT/LARGEINT/FLOAT/DOUBLE
+    - VARCHAR/CHAR 转换成 DATE (目前支持"%Y-%m-%d", "%y-%m-%d", "%Y%m%d", "%y%m%d", "%Y/%m/%d, "%y/%m/%d"六种格式化格式)
+    - DATETIME 转换成 DATE(仅保留年-月-日信息, 例如: `2019-12-09 21:47:05` <--> `2019-12-09`)
+    - DATE 转换成 DATETIME(时分秒自动补零, 例如: `2019-12-09` <--> `2019-12-09 00:00:00`)
+    - FLOAT 转换成 DOUBLE
+    - INT 转换成 DATE (如果INT类型数据不合法则转换失败,原始数据不变)
+    - 除DATE与DATETIME以外都可以转换成STRING,但是STRING不能转换任何其他类型 :::
+
+**5. 对指定表的列进行重新排序**
+
+**语法**
+  
+  ```sql
+  ALTER TABLE [database.]table table_name ORDER BY (column_name1, column_name2, ...)
+  [FROM rollup_index_name]
+  [PROPERTIES ("key"="value", ...)]
+  ```
+
+**Example**
+  
+1. 调整 example_db.my_table 的 key 列 和 value 列的顺序(非聚合模型)
+
+  ```sql
+  CREATE TABLE `my_table`(
+  `k_1` INT NULL,
+  `k_2` INT NULL,
+  `v_1` INT NULL,
+  `v_2` varchar NULL,
+  `v_3` varchar NULL
+  ) ENGINE=OLAP
+  DUPLICATE KEY(`k_1`, `k_2`)
+  COMMENT 'OLAP'
+  DISTRIBUTED BY HASH(`k_1`) BUCKETS 5
+  PROPERTIES (
+  "replication_allocation" = "tag.location.default: 1"
+  );
+
+  ALTER TABLE example_db.my_table ORDER BY (k_2,k_1,v_3,v_2,v_1);
+
+  mysql> desc my_table;
+  +-------+------------+------+-------+---------+-------+
+  | Field | Type       | Null | Key   | Default | Extra |
+  +-------+------------+------+-------+---------+-------+
+  | k_2   | INT        | Yes  | true  | NULL    |       |
+  | k_1   | INT        | Yes  | true  | NULL    |       |
+  | v_3   | VARCHAR(*) | Yes  | false | NULL    | NONE  |
+  | v_2   | VARCHAR(*) | Yes  | false | NULL    | NONE  |
+  | v_1   | INT        | Yes  | false | NULL    | NONE  |
+  +-------+------------+------+-------+---------+-------+
+  ```
+
+2. 同时执行添加列和列排序操作
+
+  ```sql
+  CREATE TABLE `my_table` (
+  `k_1` INT NULL,
+  `k_2` INT NULL,
+  `v_1` INT NULL,
+  `v_2` varchar NULL,
+  `v_3` varchar NULL
+  ) ENGINE=OLAP
+  DUPLICATE KEY(`k_1`, `k_2`)
+  COMMENT 'OLAP'
+  DISTRIBUTED BY HASH(`k_1`) BUCKETS 5
+  PROPERTIES (
+  "replication_allocation" = "tag.location.default: 1"
+  );
+
+  ALTER TABLE example_db.my_table
+  ADD COLUMN col INT DEFAULT "0" AFTER v_1,
+  ORDER BY (k_2,k_1,v_3,v_2,v_1,col);
+
+  mysql> desc my_table;
+  +-------+------------+------+-------+---------+-------+
+  | Field | Type       | Null | Key   | Default | Extra |
+  +-------+------------+------+-------+---------+-------+
+  | k_2   | INT        | Yes  | true  | NULL    |       |
+  | k_1   | INT        | Yes  | true  | NULL    |       |
+  | v_3   | VARCHAR(*) | Yes  | false | NULL    | NONE  |
+  | v_2   | VARCHAR(*) | Yes  | false | NULL    | NONE  |
+  | v_1   | INT        | Yes  | false | NULL    | NONE  |
+  | col   | INT        | Yes  | false | 0       | NONE  |
+  +-------+------------+------+-------+---------+-------+
+  ```
+  
+:::note
+  - index 中的所有列都要写出来
+  - value 列在 key 列之后
+  - key 列只能调整 key 列的范围内进行调整,value 列同理
+:::
 
 ### Keywords
 


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


(doris) 03/19: [tools] update tpcds test tools (#30607)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit b92cae51af503af893f3bc6d3aa66be3e0d338ab
Author: xzj7019 <13...@users.noreply.github.com>
AuthorDate: Wed Jan 31 14:22:29 2024 +0800

    [tools] update tpcds test tools (#30607)
    
    Co-authored-by: zhongjian.xzj <zh...@zhongjianxzjdeMacBook-Pro.local>
---
 tools/tpcds-tools/conf/opt/opt_sf1.sql     | 2 --
 tools/tpcds-tools/conf/opt/opt_sf100.sql   | 4 +---
 tools/tpcds-tools/conf/opt/opt_sf1000.sql  | 5 -----
 tools/tpcds-tools/conf/opt/opt_sf10000.sql | 5 -----
 4 files changed, 1 insertion(+), 15 deletions(-)

diff --git a/tools/tpcds-tools/conf/opt/opt_sf1.sql b/tools/tpcds-tools/conf/opt/opt_sf1.sql
index e4a4d26dec5..ea43723c30d 100644
--- a/tools/tpcds-tools/conf/opt/opt_sf1.sql
+++ b/tools/tpcds-tools/conf/opt/opt_sf1.sql
@@ -1,3 +1 @@
-set global experimental_enable_nereids_planner=true;
-set global experimental_enable_pipeline_engine=true;
 set global runtime_filter_wait_time_ms=1000;
diff --git a/tools/tpcds-tools/conf/opt/opt_sf100.sql b/tools/tpcds-tools/conf/opt/opt_sf100.sql
index 231906fe506..ea43723c30d 100644
--- a/tools/tpcds-tools/conf/opt/opt_sf100.sql
+++ b/tools/tpcds-tools/conf/opt/opt_sf100.sql
@@ -1,3 +1 @@
-set global experimental_enable_nereids_planner=true;
-set global experimental_enable_pipeline_engine=true;
-set global runtime_filter_wait_time_ms=10000;
+set global runtime_filter_wait_time_ms=1000;
diff --git a/tools/tpcds-tools/conf/opt/opt_sf1000.sql b/tools/tpcds-tools/conf/opt/opt_sf1000.sql
index 6a42b7071b1..17d8fa190f3 100644
--- a/tools/tpcds-tools/conf/opt/opt_sf1000.sql
+++ b/tools/tpcds-tools/conf/opt/opt_sf1000.sql
@@ -1,6 +1 @@
-set global experimental_enable_nereids_planner=true;
-set global experimental_enable_pipeline_engine=true;
 set global runtime_filter_wait_time_ms=10000;
-set global broadcast_right_table_scale_factor=0.0;
-set global max_join_number_bushy_tree=10;
-set global memo_max_group_expression_size=15000;
diff --git a/tools/tpcds-tools/conf/opt/opt_sf10000.sql b/tools/tpcds-tools/conf/opt/opt_sf10000.sql
index ce361d4e3f3..ef11bad93de 100644
--- a/tools/tpcds-tools/conf/opt/opt_sf10000.sql
+++ b/tools/tpcds-tools/conf/opt/opt_sf10000.sql
@@ -1,6 +1 @@
-set global experimental_enable_nereids_planner=true;
-set global experimental_enable_pipeline_engine=true;
 set global runtime_filter_wait_time_ms=100000;
-set global broadcast_right_table_scale_factor=5.0;
-set global max_join_number_bushy_tree=10;
-set global memo_max_group_expression_size=15000;


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


(doris) 18/19: Sync stats cache while task finished, doesn't need to query column_statistics table. (#30609)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 21d31c6dd9f248748c345b69ab0acb0c5e2aa404
Author: Jibing-Li <64...@users.noreply.github.com>
AuthorDate: Wed Jan 31 21:28:30 2024 +0800

    Sync stats cache while task finished, doesn't need to query column_statistics table. (#30609)
---
 .../apache/doris/service/FrontendServiceImpl.java  | 12 ++--
 .../org/apache/doris/statistics/AnalysisJob.java   | 14 -----
 .../apache/doris/statistics/AnalysisManager.java   |  2 +-
 .../apache/doris/statistics/BaseAnalysisTask.java  | 12 +---
 .../org/apache/doris/statistics/ColStatsData.java  | 71 ++++++++++++++++++++--
 .../apache/doris/statistics/ColumnStatistic.java   |  1 +
 .../doris/statistics/ExternalAnalysisTask.java     | 10 ---
 .../apache/doris/statistics/JdbcAnalysisTask.java  |  9 ---
 .../apache/doris/statistics/StatisticsCache.java   | 39 ++++++------
 .../java/org/apache/doris/statistics/StatsId.java  | 11 +++-
 .../apache/doris/statistics/AnalysisJobTest.java   |  7 ---
 .../org/apache/doris/statistics/CacheTest.java     |  2 -
 .../apache/doris/statistics/ColStatsDataTest.java  | 71 ++++++++++++++++++++++
 gensrc/thrift/FrontendService.thrift               |  3 +-
 14 files changed, 176 insertions(+), 88 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
index 86e88bb2b20..9e0b9ed9fd5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
@@ -96,9 +96,9 @@ import org.apache.doris.qe.StmtExecutor;
 import org.apache.doris.qe.VariableMgr;
 import org.apache.doris.service.arrowflight.FlightSqlConnectProcessor;
 import org.apache.doris.statistics.AnalysisManager;
+import org.apache.doris.statistics.ColStatsData;
 import org.apache.doris.statistics.ColumnStatistic;
 import org.apache.doris.statistics.InvalidateStatsTarget;
-import org.apache.doris.statistics.ResultRow;
 import org.apache.doris.statistics.StatisticsCacheKey;
 import org.apache.doris.statistics.TableStatsMeta;
 import org.apache.doris.statistics.query.QueryStats;
@@ -3045,11 +3045,11 @@ public class FrontendServiceImpl implements FrontendService.Iface {
     @Override
     public TStatus updateStatsCache(TUpdateFollowerStatsCacheRequest request) throws TException {
         StatisticsCacheKey k = GsonUtils.GSON.fromJson(request.key, StatisticsCacheKey.class);
-        List<ResultRow> rows = request.statsRows.stream()
-                .map(s -> GsonUtils.GSON.fromJson(s, ResultRow.class))
-                .collect(Collectors.toList());
-        ColumnStatistic c = ColumnStatistic.fromResultRow(rows);
-        if (c != ColumnStatistic.UNKNOWN) {
+        ColStatsData data = GsonUtils.GSON.fromJson(request.colStatsData, ColStatsData.class);
+        ColumnStatistic c = data.toColumnStatistic();
+        if (c == ColumnStatistic.UNKNOWN) {
+            Env.getCurrentEnv().getStatisticsCache().invalidate(k.tableId, k.idxId, k.colName);
+        } else {
             Env.getCurrentEnv().getStatisticsCache().updateColStatsCache(k.tableId, k.idxId, k.colName, c);
         }
         // Return Ok anyway
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java
index 193e0c4900e..19dda4934d4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java
@@ -136,7 +136,6 @@ public class AnalysisJob {
             }
         }
         updateTaskState(AnalysisState.FINISHED, "");
-        syncLoadStats();
         queryFinished.clear();
         buf.clear();
     }
@@ -192,17 +191,4 @@ public class AnalysisJob {
         }
     }
 
-    protected void syncLoadStats() {
-        long tblId = jobInfo.tblId;
-        for (BaseAnalysisTask task : queryFinished) {
-            if (task.info.externalTableLevelTask) {
-                continue;
-            }
-            String colName = task.col.getName();
-            if (!Env.getCurrentEnv().getStatisticsCache().syncLoadColStats(tblId, -1, colName)) {
-                analysisManager.removeColStatsStatus(tblId, colName);
-            }
-        }
-    }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java
index fe64fb14142..a1a6dd962eb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java
@@ -705,7 +705,7 @@ public class AnalysisManager implements Writable {
         boolean success = true;
         for (Frontend frontend : Env.getCurrentEnv().getFrontends(null)) {
             // Skip master
-            if (selfNode.equals(frontend.getHost())) {
+            if (selfNode.getHost().equals(frontend.getHost())) {
                 continue;
             }
             success = success && statisticsCache.invalidateStats(frontend, request);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java
index a67b2b9bee0..6a8bc2602e4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java
@@ -205,16 +205,7 @@ public abstract class BaseAnalysisTask {
 
     public abstract void doExecute() throws Exception;
 
-    protected void afterExecution() {
-        if (killed) {
-            return;
-        }
-        long tblId = tbl.getId();
-        String colName = col.getName();
-        if (!Env.getCurrentEnv().getStatisticsCache().syncLoadColStats(tblId, -1, colName)) {
-            Env.getCurrentEnv().getAnalysisManager().removeColStatsStatus(tblId, colName);
-        }
-    }
+    protected void afterExecution() {}
 
     protected void setTaskStateToRunning() {
         Env.getCurrentEnv().getAnalysisManager()
@@ -318,6 +309,7 @@ public abstract class BaseAnalysisTask {
         try (AutoCloseConnectContext a  = StatisticsUtil.buildConnectContext()) {
             stmtExecutor = new StmtExecutor(a.connectContext, sql);
             ColStatsData colStatsData = new ColStatsData(stmtExecutor.executeInternalQuery().get(0));
+            Env.getCurrentEnv().getStatisticsCache().syncColStats(colStatsData);
             queryId = DebugUtil.printId(stmtExecutor.getContext().queryId());
             job.appendBuf(this, Collections.singletonList(colStatsData));
         } finally {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsData.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsData.java
index ab551e2d4cf..bdc600987f4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsData.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsData.java
@@ -17,9 +17,14 @@
 
 package org.apache.doris.statistics;
 
+import org.apache.doris.catalog.Column;
+import org.apache.doris.common.AnalysisException;
 import org.apache.doris.statistics.util.StatisticsUtil;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.annotations.SerializedName;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 import java.time.LocalDateTime;
 import java.time.format.DateTimeFormatter;
@@ -43,17 +48,23 @@ import java.util.StringJoiner;
  * 13: update_time
  */
 public class ColStatsData {
+    private static final Logger LOG = LogManager.getLogger(ColStatsData.class);
+
+    @SerializedName("statsId")
     public final StatsId statsId;
+    @SerializedName("count")
     public final long count;
+    @SerializedName("ndv")
     public final long ndv;
-
+    @SerializedName("nullCount")
     public final long nullCount;
-
+    @SerializedName("minLit")
     public final String minLit;
+    @SerializedName("maxLit")
     public final String maxLit;
-
+    @SerializedName("dataSizeInBytes")
     public final long dataSizeInBytes;
-
+    @SerializedName("updateTime")
     public final String updateTime;
 
     @VisibleForTesting
@@ -106,4 +117,56 @@ public class ColStatsData {
         sj.add(StatisticsUtil.quote(updateTime));
         return sj.toString();
     }
+
+    public ColumnStatistic toColumnStatistic() {
+        try {
+            ColumnStatisticBuilder columnStatisticBuilder = new ColumnStatisticBuilder();
+            columnStatisticBuilder.setCount(count);
+            columnStatisticBuilder.setNdv(ndv);
+            columnStatisticBuilder.setNumNulls(nullCount);
+            columnStatisticBuilder.setDataSize(dataSizeInBytes);
+            columnStatisticBuilder.setAvgSizeByte(count == 0 ? 0 : dataSizeInBytes / count);
+            if (statsId == null) {
+                return ColumnStatistic.UNKNOWN;
+            }
+            long catalogId = statsId.catalogId;
+            long idxId = statsId.idxId;
+            long dbID = statsId.dbId;
+            long tblId = statsId.tblId;
+            String colName = statsId.colId;
+            Column col = StatisticsUtil.findColumn(catalogId, dbID, tblId, idxId, colName);
+            if (col == null) {
+                return ColumnStatistic.UNKNOWN;
+            }
+            String min = minLit;
+            String max = maxLit;
+            if (min != null && !min.equalsIgnoreCase("NULL")) {
+                try {
+                    columnStatisticBuilder.setMinValue(StatisticsUtil.convertToDouble(col.getType(), min));
+                    columnStatisticBuilder.setMinExpr(StatisticsUtil.readableValue(col.getType(), min));
+                } catch (AnalysisException e) {
+                    LOG.warn("Failed to process column {} min value {}.", col, min, e);
+                    columnStatisticBuilder.setMinValue(Double.NEGATIVE_INFINITY);
+                }
+            } else {
+                columnStatisticBuilder.setMinValue(Double.NEGATIVE_INFINITY);
+            }
+            if (max != null && !max.equalsIgnoreCase("NULL")) {
+                try {
+                    columnStatisticBuilder.setMaxValue(StatisticsUtil.convertToDouble(col.getType(), max));
+                    columnStatisticBuilder.setMaxExpr(StatisticsUtil.readableValue(col.getType(), max));
+                } catch (AnalysisException e) {
+                    LOG.warn("Failed to process column {} max value {}.", col, max, e);
+                    columnStatisticBuilder.setMaxValue(Double.POSITIVE_INFINITY);
+                }
+            } else {
+                columnStatisticBuilder.setMaxValue(Double.POSITIVE_INFINITY);
+            }
+            columnStatisticBuilder.setUpdatedTime(updateTime);
+            return columnStatisticBuilder.build();
+        } catch (Exception e) {
+            LOG.warn("Failed to convert column statistics.", e);
+            return ColumnStatistic.UNKNOWN;
+        }
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java
index 77796e04eb4..b01e47ef346 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java
@@ -88,6 +88,7 @@ public class ColumnStatistic {
     public final LiteralExpr minExpr;
     public final LiteralExpr maxExpr;
 
+    @SerializedName("updatedTime")
     public final String updatedTime;
 
     public ColumnStatistic(double count, double ndv, ColumnStatistic original, double avgSizeByte,
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java
index 15848c013d6..b9ce541d55f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ExternalAnalysisTask.java
@@ -225,16 +225,6 @@ public class ExternalAnalysisTask extends BaseAnalysisTask {
         return Pair.of(Math.max(((double) total) / cumulate, 1), cumulate);
     }
 
-    @Override
-    protected void afterExecution() {
-        // Table level task doesn't need to sync any value to sync stats, it stores the value in metadata.
-        // Partition only task doesn't need to refresh cached.
-        if (isTableLevelTask || isPartitionOnly) {
-            return;
-        }
-        Env.getCurrentEnv().getStatisticsCache().syncLoadColStats(tbl.getId(), -1, col.getName());
-    }
-
     /**
      * If the size to sample is larger than LIMIT_SIZE (1GB)
      * and is much larger (1.2*) than the size user want to sample,
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/JdbcAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/JdbcAnalysisTask.java
index 2bf72843a71..38ee648cad4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/JdbcAnalysisTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/JdbcAnalysisTask.java
@@ -137,13 +137,4 @@ public class JdbcAnalysisTask extends BaseAnalysisTask {
         commonParams.put("lastAnalyzeTimeInMs", String.valueOf(System.currentTimeMillis()));
         return commonParams;
     }
-
-    @Override
-    protected void afterExecution() {
-        // Table level task doesn't need to sync any value to sync stats, it stores the value in metadata.
-        if (isTableLevelTask) {
-            return;
-        }
-        Env.getCurrentEnv().getStatisticsCache().syncLoadColStats(tbl.getId(), -1, col.getName());
-    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java
index 0cf2808222e..73eaaaff1c8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java
@@ -21,11 +21,11 @@ import org.apache.doris.catalog.Env;
 import org.apache.doris.common.ClientPool;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.ThreadPoolManager;
-import org.apache.doris.ha.FrontendNodeType;
 import org.apache.doris.persist.gson.GsonUtils;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.statistics.util.StatisticsUtil;
 import org.apache.doris.system.Frontend;
+import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.thrift.FrontendService;
 import org.apache.doris.thrift.TInvalidateFollowerStatsCacheRequest;
 import org.apache.doris.thrift.TNetworkAddress;
@@ -39,12 +39,12 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import java.time.Duration;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
 
 public class StatisticsCache {
 
@@ -203,37 +203,34 @@ public class StatisticsCache {
     }
 
     /**
-     * Return false if the log of corresponding stats load is failed.
+     * Refresh stats cache, invalidate cache if the new data is unknown.
      */
-    public boolean syncLoadColStats(long tableId, long idxId, String colName) {
-        List<ResultRow> columnResults = StatisticsRepository.loadColStats(tableId, idxId, colName);
-        final StatisticsCacheKey k =
-                new StatisticsCacheKey(tableId, idxId, colName);
-        final ColumnStatistic c = ColumnStatistic.fromResultRow(columnResults);
-        if (c == ColumnStatistic.UNKNOWN) {
-            return false;
-        }
-        putCache(k, c);
-        if (ColumnStatistic.UNKNOWN == c) {
-            return false;
+    public void syncColStats(ColStatsData data) {
+        StatsId statsId = data.statsId;
+        final StatisticsCacheKey k = new StatisticsCacheKey(statsId.tblId, statsId.idxId, statsId.colId);
+        ColumnStatistic columnStatistic = data.toColumnStatistic();
+        if (columnStatistic == ColumnStatistic.UNKNOWN) {
+            invalidate(k.tableId, k.idxId, k.colName);
+        } else {
+            putCache(k, columnStatistic);
         }
         TUpdateFollowerStatsCacheRequest updateFollowerStatsCacheRequest = new TUpdateFollowerStatsCacheRequest();
         updateFollowerStatsCacheRequest.key = GsonUtils.GSON.toJson(k);
-        updateFollowerStatsCacheRequest.statsRows = columnResults.stream().map(GsonUtils.GSON::toJson).collect(
-                Collectors.toList());
-        for (Frontend frontend : Env.getCurrentEnv().getFrontends(FrontendNodeType.FOLLOWER)) {
-            if (StatisticsUtil.isMaster(frontend)) {
+        updateFollowerStatsCacheRequest.colStatsData = GsonUtils.GSON.toJson(data);
+        // For compatible only, to be deprecated.
+        updateFollowerStatsCacheRequest.statsRows = new ArrayList<>();
+        SystemInfoService.HostInfo selfNode = Env.getCurrentEnv().getSelfNode();
+        for (Frontend frontend : Env.getCurrentEnv().getFrontends(null)) {
+            if (selfNode.getHost().equals(frontend.getHost())) {
                 continue;
             }
             sendStats(frontend, updateFollowerStatsCacheRequest);
         }
-        return true;
     }
 
     @VisibleForTesting
     public void sendStats(Frontend frontend, TUpdateFollowerStatsCacheRequest updateFollowerStatsCacheRequest) {
-        TNetworkAddress address = new TNetworkAddress(frontend.getHost(),
-                frontend.getRpcPort());
+        TNetworkAddress address = new TNetworkAddress(frontend.getHost(), frontend.getRpcPort());
         FrontendService.Client client = null;
         try {
             client = ClientPool.frontendPool.borrowObject(address);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatsId.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatsId.java
index 21395638cd6..22f2f73ac27 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatsId.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatsId.java
@@ -20,20 +20,25 @@ package org.apache.doris.statistics;
 import org.apache.doris.statistics.util.StatisticsUtil;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.annotations.SerializedName;
 
 import java.util.StringJoiner;
 
 public class StatsId {
 
+    @SerializedName("id")
     public final String id;
+    @SerializedName("catalogId")
     public final long catalogId;
+    @SerializedName("dbId")
     public final long dbId;
+    @SerializedName("tblId")
     public final long tblId;
+    @SerializedName("idxId")
     public final long idxId;
-
+    @SerializedName("colId")
     public final String colId;
-
-    // nullable
+    @SerializedName("partId")
     public final String partId;
 
     @VisibleForTesting
diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java
index 255ab7106aa..1bf2041bb4f 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java
@@ -21,7 +21,6 @@ import org.apache.doris.catalog.Env;
 import org.apache.doris.qe.StmtExecutor;
 import org.apache.doris.statistics.util.StatisticsUtil;
 
-import mockit.Expectations;
 import mockit.Mock;
 import mockit.MockUp;
 import mockit.Mocked;
@@ -185,12 +184,6 @@ public class AnalysisJobTest {
             protected void syncLoadStats() {
             }
         };
-        new Expectations() {
-            {
-                job.syncLoadStats();
-                times = 1;
-            }
-        };
         job.writeBuf();
 
         Assertions.assertEquals(0, job.queryFinished.size());
diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/CacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/CacheTest.java
index 0d968167b73..6b3b09c4968 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/statistics/CacheTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/CacheTest.java
@@ -301,7 +301,6 @@ public class CacheTest extends TestWithFeService {
             }
         };
         StatisticsCache statisticsCache = new StatisticsCache();
-        statisticsCache.syncLoadColStats(1L, 1L, "any");
         new Expectations() {
             {
                 statisticsCache.sendStats((Frontend) any, (TUpdateFollowerStatsCacheRequest) any);
@@ -346,7 +345,6 @@ public class CacheTest extends TestWithFeService {
             }
         };
         StatisticsCache statisticsCache = new StatisticsCache();
-        statisticsCache.syncLoadColStats(1L, 1L, "any");
         new Expectations() {
             {
                 statisticsCache.sendStats((Frontend) any, (TUpdateFollowerStatsCacheRequest) any);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/ColStatsDataTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/ColStatsDataTest.java
index dcbbe6e2f35..8743105a644 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/statistics/ColStatsDataTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/ColStatsDataTest.java
@@ -17,7 +17,13 @@
 
 package org.apache.doris.statistics;
 
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.statistics.util.StatisticsUtil;
+
 import com.google.common.collect.Lists;
+import mockit.Expectations;
+import mockit.Mocked;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
@@ -93,4 +99,69 @@ public class ColStatsDataTest {
         Assertions.assertEquals(0, data.dataSizeInBytes);
         Assertions.assertEquals(null, data.updateTime);
     }
+
+    @Test
+    public void testToColumnStatisticUnknown(@Mocked StatisticsUtil mockedClass) {
+        // Test column is null
+        new Expectations() {
+            {
+                mockedClass.findColumn(anyLong, anyLong, anyLong, anyLong, anyString);
+                result = null;
+            }
+        };
+        List<String> values = Lists.newArrayList();
+        values.add("id");
+        values.add("10000");
+        values.add("20000");
+        values.add("30000");
+        values.add("0");
+        values.add("col");
+        values.add(null);
+        values.add("100");
+        values.add("200");
+        values.add("300");
+        values.add("min");
+        values.add("max");
+        values.add("400");
+        values.add("500");
+        ResultRow row = new ResultRow(values);
+        ColStatsData data = new ColStatsData(row);
+        ColumnStatistic columnStatistic = data.toColumnStatistic();
+        Assertions.assertEquals(ColumnStatistic.UNKNOWN, columnStatistic);
+    }
+
+    @Test
+    public void testToColumnStatisticNormal(@Mocked StatisticsUtil mockedClass) {
+        new Expectations() {
+            {
+                mockedClass.findColumn(anyLong, anyLong, anyLong, anyLong, anyString);
+                result = new Column("colName", PrimitiveType.STRING);
+            }
+        };
+        List<String> values = Lists.newArrayList();
+        values.add("id");
+        values.add("10000");
+        values.add("20000");
+        values.add("30000");
+        values.add("0");
+        values.add("col");
+        values.add(null);
+        values.add("100");
+        values.add("200");
+        values.add("300");
+        values.add("null");
+        values.add("null");
+        values.add("400");
+        values.add("500");
+        ResultRow row = new ResultRow(values);
+        ColStatsData data = new ColStatsData(row);
+        ColumnStatistic columnStatistic = data.toColumnStatistic();
+        Assertions.assertEquals(100, columnStatistic.count);
+        Assertions.assertEquals(200, columnStatistic.ndv);
+        Assertions.assertEquals(300, columnStatistic.numNulls);
+        Assertions.assertEquals(Double.NEGATIVE_INFINITY, columnStatistic.minValue);
+        Assertions.assertEquals(Double.POSITIVE_INFINITY, columnStatistic.maxValue);
+        Assertions.assertEquals(400, columnStatistic.dataSize);
+        Assertions.assertEquals("500", columnStatistic.updatedTime);
+    }
 }
diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift
index f73325c69bd..4a317bc5f2b 100644
--- a/gensrc/thrift/FrontendService.thrift
+++ b/gensrc/thrift/FrontendService.thrift
@@ -1161,7 +1161,8 @@ struct TGetBinlogLagResult {
 
 struct TUpdateFollowerStatsCacheRequest {
     1: optional string key;
-    2: list<string> statsRows;
+    2: optional list<string> statsRows;
+    3: optional string colStatsData;
 }
 
 struct TInvalidateFollowerStatsCacheRequest {


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


(doris) 04/19: [Chore](case) reset all variables at start on set_and_unset_variable case (#30580)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 6634d9f2a98ae48ea62b656ea639da8889e2c4b9
Author: Pxl <px...@qq.com>
AuthorDate: Wed Jan 31 14:27:31 2024 +0800

    [Chore](case) reset all variables at start on set_and_unset_variable case (#30580)
    
    reset all variables at start on set_and_unset_variable case
---
 regression-test/data/variable_p0/set_and_unset_variable.out      | 6 ++++++
 regression-test/suites/variable_p0/set_and_unset_variable.groovy | 2 ++
 2 files changed, 8 insertions(+)

diff --git a/regression-test/data/variable_p0/set_and_unset_variable.out b/regression-test/data/variable_p0/set_and_unset_variable.out
index 89689297e61..7f6aeed2efe 100644
--- a/regression-test/data/variable_p0/set_and_unset_variable.out
+++ b/regression-test/data/variable_p0/set_and_unset_variable.out
@@ -2,6 +2,12 @@
 -- !cmd --
 0
 
+-- !cmd --
+0
+
+-- !cmd --
+0
+
 -- !cmd --
 wait_timeout	1000	28800	1
 
diff --git a/regression-test/suites/variable_p0/set_and_unset_variable.groovy b/regression-test/suites/variable_p0/set_and_unset_variable.groovy
index 18fae2a8c2f..47749774336 100644
--- a/regression-test/suites/variable_p0/set_and_unset_variable.groovy
+++ b/regression-test/suites/variable_p0/set_and_unset_variable.groovy
@@ -16,6 +16,8 @@
 // under the License.
 
 suite("set_and_unset_variable") {
+    qt_cmd """UNSET VARIABLE ALL"""
+    qt_cmd """UNSET global VARIABLE ALL"""
 
     qt_cmd """set wait_timeout = 1000"""
     qt_cmd """show variables like 'wait_timeout'"""


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