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

mrhhsg pushed a change to branch deduplicate_keys
in repository https://gitbox.apache.org/repos/asf/doris.git


    omit fffe0aae38f fix UT
    omit bff79ab3fd6 remove duplicated keys in MapLiteral.
    omit d786bfea883 [feat](map) remove duplicated keys in ColumnMap
     add c334a01575a [fix](external) check duplicate column names for external 
table schema (#52315)
     add 969f5feff78 [fix](chore) fix cache release core (#52756)
     add 9920e9678f7 [improve](hive row count)Log stack info when fetch hive 
row count fail. (#52795)
     add d4d556851da [improve](analytic) support window function execute with 
incremental mode (#52138)
     add 723bcbd6835 [fix](session variables) Limit query_timeout to within 
LoadTimeout (#51222)
     add 285b1e45df2 [Opt](cloud-mow) Do fast retry when commit compaction job 
for mow tablet (#52476)
     add 522cefb0b34 [chore](cloud) Add async wrap for `bthread_fork_join` with 
promise-future (#52816)
     add 95a01acec32 [feat](txnkv) support atomic_set_ver_key with offset 
(#52671)
     add 7efcf534b18 [Opt](cloud-mow) Retry to commit txn when encounter stale 
calc delete bitmap response regardless of status code (#52547)
     add ea6202576c7 [opt](auditlog) Use varchar(1024) for column frontend_ip 
of audit log table (#52762)
     add 4c6f12fb2ba [opt](nereids) optimize stats derive when using delta rows 
(#52788)
     add 78ff9e56489 [feat](nereids)support stats derive in RBO phase (#52385)
     add 8254daed946 [fix](be) fix macos report wrong cpu cores (#52726)
     add fe0fb6a55aa [fix](txnkv) avoid skip keys during reverse full range 
iteration (#52800)
     add cc7a1f93c4b [chore](profile) Modify profile structure (#48184)
     add c10bbe95103 [Enhancement] (nereids) implement mysqlLoadCommand in 
nereids (#49109)
     add 581872aba8d [fix](test) fix audit_log regression test (#52587)
     add 29421edd7f2 [fix](nereids)fix multi_leading regression test case 
(#51165)
     add d4290b684c1 [fix](nereids)scalar subquery should not show error 
message when there are multiple agg functions in top-level agg node (#52667)
     add a241e668f79 [Fix](UBSan) fix mem alignment in rapidjson ralated code 
(#52809)
     add c3608472624 [fix](Nereids) not process must shuffle when regulate can 
not be banned agg (#52827)
     add bcb06ac3b55 [chore](build) suppress BE CMake warning about unused 
variable (#52891)
     add 390e18283e7 [Fix](ub) Fix undefined behaviour on visiting vector 
(#52901)
     add 64c7f80ac64 [chore] Add GCC 15 and Clang 20 Support (#52835)
     add 38e82f76e37 [feat](map) remove duplicated keys in ColumnMap
     add 34c84b06c35 remove duplicated keys in MapLiteral.
     add 77ed9b8fbd0 fix UT

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (fffe0aae38f)
            \
             N -- N -- N   refs/heads/deduplicate_keys (77ed9b8fbd0)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

No new revisions were added by this update.

Summary of changes:
 be/CMakeLists.txt                                  |   1 +
 be/src/cloud/cloud_meta_mgr.cpp                    |  32 +-
 be/src/cloud/cloud_meta_mgr.h                      |   9 +
 be/src/exec/table_connector.cpp                    |   9 +-
 be/src/exprs/json_functions.cpp                    |   9 +-
 be/src/olap/bitmap_filter_predicate.h              |   2 +
 be/src/olap/bloom_filter_predicate.h               |   2 +
 be/src/olap/tablet_meta.cpp                        |   4 +-
 be/src/pipeline/exec/aggregation_sink_operator.cpp |  31 +-
 .../pipeline/exec/aggregation_source_operator.cpp  |  28 +-
 be/src/pipeline/exec/analytic_sink_operator.cpp    |  74 ++--
 be/src/pipeline/exec/analytic_sink_operator.h      |   5 +-
 be/src/pipeline/exec/analytic_source_operator.cpp  |   4 +-
 be/src/pipeline/exec/cache_source_operator.cpp     |   6 +-
 be/src/pipeline/exec/datagen_operator.cpp          |   4 +-
 .../distinct_streaming_aggregation_operator.cpp    |  13 +-
 be/src/pipeline/exec/es_scan_operator.cpp          |   6 +-
 be/src/pipeline/exec/exchange_sink_operator.cpp    |  66 ++--
 be/src/pipeline/exec/exchange_sink_operator.h      |   8 +-
 be/src/pipeline/exec/exchange_source_operator.cpp  |  14 +-
 be/src/pipeline/exec/file_scan_operator.cpp        |   2 +-
 .../exec/group_commit_block_sink_operator.cpp      |   8 +-
 be/src/pipeline/exec/hashjoin_build_sink.cpp       |  22 +-
 be/src/pipeline/exec/hashjoin_probe_operator.cpp   |  17 +-
 be/src/pipeline/exec/jdbc_scan_operator.cpp        |   7 +-
 be/src/pipeline/exec/join_build_sink_operator.cpp  |   5 +-
 be/src/pipeline/exec/join_probe_operator.cpp       |  11 +-
 .../exec/local_merge_sort_source_operator.cpp      |   2 +-
 .../exec/materialization_source_operator.h         |   4 +-
 .../pipeline/exec/memory_scratch_sink_operator.cpp |   7 +-
 be/src/pipeline/exec/meta_scan_operator.cpp        |   3 +-
 .../pipeline/exec/multi_cast_data_stream_sink.cpp  |  15 +-
 be/src/pipeline/exec/multi_cast_data_stream_sink.h |   6 +-
 .../exec/multi_cast_data_stream_source.cpp         |  18 +-
 .../pipeline/exec/multi_cast_data_stream_source.h  |   4 +-
 be/src/pipeline/exec/multi_cast_data_streamer.cpp  |  13 +-
 be/src/pipeline/exec/multi_cast_data_streamer.h    |  11 +-
 .../exec/nested_loop_join_build_operator.cpp       |   3 +-
 .../exec/nested_loop_join_probe_operator.cpp       |  10 +-
 be/src/pipeline/exec/olap_scan_operator.cpp        |  18 +-
 be/src/pipeline/exec/olap_scan_operator.h          |   6 +-
 be/src/pipeline/exec/operator.cpp                  |  96 ++---
 be/src/pipeline/exec/operator.h                    | 148 +++++---
 .../pipeline/exec/partition_sort_sink_operator.cpp |  34 +-
 .../exec/partition_sort_source_operator.cpp        |   4 +-
 .../exec/partitioned_aggregation_sink_operator.cpp |  20 +-
 .../exec/partitioned_aggregation_sink_operator.h   |   2 +-
 .../partitioned_aggregation_source_operator.cpp    |  10 +-
 .../exec/partitioned_hash_join_probe_operator.cpp  | 125 ++++---
 .../exec/partitioned_hash_join_probe_operator.h    |  10 +-
 .../exec/partitioned_hash_join_sink_operator.cpp   |  29 +-
 be/src/pipeline/exec/repeat_operator.cpp           |   6 +-
 be/src/pipeline/exec/result_sink_operator.cpp      |  12 +-
 be/src/pipeline/exec/scan_operator.cpp             |  26 +-
 be/src/pipeline/exec/schema_scan_operator.cpp      |   2 +-
 be/src/pipeline/exec/set_probe_sink_operator.cpp   |   4 +-
 be/src/pipeline/exec/set_sink_operator.cpp         |   7 +-
 be/src/pipeline/exec/set_source_operator.cpp       |   4 +-
 be/src/pipeline/exec/sort_sink_operator.cpp        |  16 +-
 be/src/pipeline/exec/spill_sort_sink_operator.cpp  |  23 +-
 .../pipeline/exec/spill_sort_source_operator.cpp   |  10 +-
 be/src/pipeline/exec/spill_utils.h                 |  52 +--
 .../exec/streaming_aggregation_operator.cpp        |  35 +-
 be/src/pipeline/exec/table_function_operator.cpp   |   6 +-
 be/src/pipeline/exec/union_sink_operator.cpp       |   2 +-
 be/src/pipeline/exec/union_source_operator.cpp     |   2 +-
 .../local_exchange_sink_operator.cpp               |  10 +-
 .../local_exchange_source_operator.cpp             |   4 +-
 .../local_exchange_source_operator.h               |   2 +-
 be/src/pipeline/pipeline_fragment_context.cpp      |   6 +-
 be/src/pipeline/pipeline_task.cpp                  |   7 +-
 be/src/service/point_query_executor.h              |   7 +-
 be/src/util/cpu_info.cpp                           |   6 +-
 be/src/util/hash_util.hpp                          |   4 +-
 be/src/util/jsonb_document.h                       |  14 -
 be/src/util/runtime_profile.cpp                    |  11 +
 be/src/util/runtime_profile.h                      |   2 +
 .../vec/aggregate_functions/aggregate_function.h   |  52 ++-
 .../aggregate_functions/aggregate_function_avg.h   |  77 +++-
 .../aggregate_functions/aggregate_function_count.h |  46 +++
 .../aggregate_function_java_udaf.h                 |  11 +-
 .../aggregate_function_min_max.h                   |  98 +++++
 .../aggregate_functions/aggregate_function_null.h  | 106 +++++-
 .../aggregate_function_reader_first_last.h         |   2 +-
 .../aggregate_functions/aggregate_function_sum.h   |  58 +++
 .../aggregate_function_window.h                    |  14 +-
 .../data_types/data_type_date_or_datetime_v2.cpp   |  13 +
 be/src/vec/exec/scan/olap_scanner.cpp              |   2 +-
 be/src/vec/exec/vjdbc_connector.h                  |   4 +-
 be/src/vec/functions/date_time_transforms.h        |  13 +
 .../vec/functions/function_datetime_floor_ceil.cpp |  13 +
 be/src/vec/functions/function_json.cpp             |   6 +-
 be/src/vec/functions/function_string.h             |   1 -
 be/src/vec/sink/vrow_distribution.cpp              |   2 +-
 be/src/vec/sink/writer/async_result_writer.cpp     |  22 +-
 be/src/vec/sink/writer/async_result_writer.h       |   8 +-
 .../sink/writer/iceberg/viceberg_table_writer.cpp  |  24 +-
 be/src/vec/sink/writer/vhive_partition_writer.cpp  |   2 +-
 be/src/vec/sink/writer/vhive_table_writer.cpp      |  32 +-
 be/src/vec/sink/writer/vjdbc_table_writer.h        |   4 +-
 be/src/vec/sink/writer/vtablet_writer.cpp          |  10 +-
 be/src/vec/sink/writer/vtablet_writer_v2.cpp       |  38 +-
 be/src/vec/spill/spill_reader.h                    |  18 +-
 be/src/vec/spill/spill_stream.cpp                  |  13 +-
 be/src/vec/spill/spill_stream.h                    |   4 +-
 be/src/vec/spill/spill_stream_manager.cpp          |   5 +-
 be/src/vec/spill/spill_stream_manager.h            |   2 +-
 be/src/vec/spill/spill_writer.h                    |  22 +-
 be/test/cloud/cloud_meta_mgr_test.cpp              |  89 +++++
 .../exec/multi_cast_data_streamer_test.cpp         |  92 +++--
 .../partitioned_aggregation_sink_operator_test.cpp |  18 +-
 ...artitioned_aggregation_source_operator_test.cpp |  18 +-
 .../partitioned_aggregation_test_helper.cpp        |  10 +-
 .../operator/partitioned_aggregation_test_helper.h |  16 +-
 .../partitioned_hash_join_probe_operator_test.cpp  |  63 ++--
 .../partitioned_hash_join_sink_operator_test.cpp   |  12 +-
 .../operator/partitioned_hash_join_test_helper.cpp |  10 +-
 .../operator/partitioned_hash_join_test_helper.h   |  69 ++--
 .../operator/spill_sort_sink_operator_test.cpp     |  10 +-
 .../operator/spill_sort_source_operator_test.cpp   |  33 +-
 .../operator/spillable_operator_test_helper.cpp    |  34 +-
 .../operator/spillable_operator_test_helper.h      |   5 +-
 be/test/util/profile_spec_test.cpp                 | 180 ++++++++++
 be/test/util/runtime_profile_test.cpp              |  30 ++
 cloud/src/meta-service/meta_service_job.cpp        | 141 ++++----
 cloud/src/meta-store/mem_txn_kv.cpp                | 253 +++++++++++--
 cloud/src/meta-store/mem_txn_kv.h                  |  30 +-
 cloud/src/meta-store/txn_kv.cpp                    |  64 +++-
 cloud/src/meta-store/txn_kv.h                      |  48 ++-
 cloud/test/http_encode_key_test.cpp                |   7 +-
 cloud/test/mem_txn_kv_test.cpp                     | 400 +++++++++++++++++++++
 cloud/test/recycler_test.cpp                       |   4 +
 cloud/test/txn_kv_test.cpp                         | 159 +++++++-
 ...-72d4d52a-eca9-4542-a2af-cc17499731e6-0.parquet | Bin 0 -> 504 bytes
 ...anifest-5cae1365-d123-4172-9a89-4fbc02bee658-0} | Bin 1946 -> 1942 bytes
 ...st-list-391058a9-952c-4aa9-892f-df3334e4109b-0} | Bin 884 -> 884 bytes
 ...st-list-391058a9-952c-4aa9-892f-df3334e4109b-1} | Bin 989 -> 989 bytes
 .../schema/schema-0                                |  15 +-
 .../snapshot/EARLIEST                              |   0
 .../snapshot/LATEST                                |   0
 .../snapshot/snapshot-1                            |  12 +-
 .../antlr4/org/apache/doris/nereids/DorisParser.g4 |   8 +-
 .../java/org/apache/doris/analysis/Separator.java  |  44 +++
 .../org/apache/doris/catalog/InternalSchema.java   |   2 +-
 .../transaction/CloudGlobalTransactionMgr.java     |   5 +-
 .../doris/common/profile/ExecutionProfile.java     |  33 +-
 .../org/apache/doris/common/profile/Profile.java   |  65 ++--
 .../doris/common/profile/RuntimeProfile.java       |  37 +-
 .../doris/datasource/ExternalRowCountCache.java    |  10 +-
 .../doris/datasource/ExternalSchemaCache.java      |   3 +
 .../apache/doris/datasource/SchemaCacheValue.java  |  12 +
 .../apache/doris/load/loadv2/MysqlLoadManager.java | 226 ++++++++++++
 .../java/org/apache/doris/master/MasterImpl.java   |  13 +-
 .../doris/nereids/jobs/executor/Rewriter.java      |   3 +
 .../doris/nereids/parser/LogicalPlanBuilder.java   |  96 +++++
 .../properties/ChildrenPropertiesRegulator.java    |   6 +-
 .../org/apache/doris/nereids/rules/RuleType.java   |   2 +
 .../nereids/rules/analysis/SubqueryToApply.java    | 190 ++++++----
 .../doris/nereids/rules/rewrite/StatsDerive.java   | 353 ++++++++++++++++++
 .../rewrite/UnCorrelatedApplyAggregateFilter.java  |   3 +-
 .../rules/rewrite/UnCorrelatedApplyFilter.java     |   2 +-
 .../rewrite/UnCorrelatedApplyProjectFilter.java    |   2 +-
 .../doris/nereids/stats/FilterEstimation.java      |  47 ++-
 .../doris/nereids/stats/HboStatsCalculator.java    |  13 +-
 .../apache/doris/nereids/stats/JoinEstimation.java |  51 ++-
 .../doris/nereids/stats/StatsCalculator.java       | 356 ++++++++----------
 .../trees/copier/LogicalPlanDeepCopier.java        |   2 +-
 .../nereids/trees/expressions/ScalarSubquery.java  |  20 +-
 .../doris/nereids/trees/plans/AbstractPlan.java    |   6 +-
 .../apache/doris/nereids/trees/plans/PlanType.java |   1 +
 .../plans/commands/load/MysqlDataDescription.java  | 277 ++++++++++++++
 .../plans/commands/load/MysqlLoadCommand.java      | 339 +++++++++++++++++
 .../trees/plans/logical/LogicalAggregate.java      |   5 +-
 .../nereids/trees/plans/logical/LogicalApply.java  |  22 +-
 .../trees/plans/logical/LogicalCTEConsumer.java    |   5 +-
 .../trees/plans/logical/LogicalCTEProducer.java    |   5 +-
 .../logical/LogicalDeferMaterializeOlapScan.java   |   5 +-
 .../logical/LogicalDeferMaterializeResultSink.java |   5 +-
 .../plans/logical/LogicalDeferMaterializeTopN.java |   5 +-
 .../nereids/trees/plans/logical/LogicalEsScan.java |   4 +-
 .../nereids/trees/plans/logical/LogicalExcept.java |   5 +-
 .../trees/plans/logical/LogicalFileScan.java       |   5 +-
 .../nereids/trees/plans/logical/LogicalFilter.java |   5 +-
 .../trees/plans/logical/LogicalGenerate.java       |   5 +-
 .../nereids/trees/plans/logical/LogicalHaving.java |   3 +-
 .../trees/plans/logical/LogicalHudiScan.java       |   5 +-
 .../trees/plans/logical/LogicalIntersect.java      |   5 +-
 .../nereids/trees/plans/logical/LogicalJoin.java   |   5 +-
 .../nereids/trees/plans/logical/LogicalLimit.java  |   5 +-
 .../trees/plans/logical/LogicalOlapScan.java       |   5 +-
 .../trees/plans/logical/LogicalPartitionTopN.java  |   4 +-
 .../trees/plans/logical/LogicalProject.java        |   5 +-
 .../nereids/trees/plans/logical/LogicalUnion.java  |   5 +-
 .../trees/plans/logical/LogicalUsingJoin.java      |   5 +-
 .../nereids/trees/plans/logical/LogicalWindow.java |   5 +-
 .../trees/plans/visitor/CommandVisitor.java        |   5 +
 .../java/org/apache/doris/nereids/util/Utils.java  |  34 ++
 .../apache/doris/statistics/ColumnStatistic.java   |   4 +
 .../org/apache/doris/statistics/Statistics.java    |   5 +-
 .../doris/transaction/GlobalTransactionMgr.java    |   6 +-
 .../transaction/GlobalTransactionMgrIface.java     |  22 +-
 .../doris/common/profile/ProfileStructureTest.java | 130 +++++++
 .../common/profile/RuntimeProfileMergeTest.java    |   4 +-
 .../rules/analysis/SubqueryToApplyTest.java        |  67 ++++
 .../rules/rewrite/ExistsApplyToJoinTest.java       |   8 +-
 .../doris/nereids/stats/FilterEstimationTest.java  |  45 ++-
 .../doris/nereids/stats/StatsCalculatorTest.java   |  10 +-
 .../nereids/trees/plans/PlanToStringTest.java      |  12 +-
 .../data/audit/test_audit_log_behavior.out         | Bin 1231 -> 1837 bytes
 .../data/nereids_p0/datatype/test_map.out          | Bin 1251 -> 1322 bytes
 .../data/nereids_p0/hint/multi_leading.out         | Bin 9186 -> 9171 bytes
 .../subquery/correlated_scalar_subquery.out        | Bin 736 -> 866 bytes
 .../query_p0/cast/test_complextype_to_json.out     | Bin 1980 -> 1963 bytes
 .../query_p0/group_concat/test_group_concat.out    | Bin 1132 -> 1158 bytes
 .../window_functions/test_window_fn.out            | Bin 7822 -> 16313 bytes
 .../paimon/test_paimon_catalog.groovy              |   6 +
 .../suites/insert_p0/test_insert_timeout.groovy    |  59 +++
 .../suites/manager/test_manager_interface_1.groovy |   2 +-
 .../suites/nereids_p0/datatype/test_map.groovy     |  11 +-
 .../test_outerjoin_isnull_estimation.groovy        |  52 ++-
 .../subquery/correlated_scalar_subquery.groovy     |   5 +
 .../query_p0/group_concat/test_group_concat.groovy |  10 +
 .../window_functions/test_window_fn.groovy         | 273 +++++++++++++-
 thirdparty/build-thirdparty.sh                     |  20 +-
 .../patches/thrift-0.16-fix_mutex_include.patch    |   9 +
 225 files changed, 5464 insertions(+), 1511 deletions(-)
 create mode 100644 be/test/cloud/cloud_meta_mgr_test.cpp
 create mode 100644 be/test/util/profile_spec_test.cpp
 create mode 100644 
docker/thirdparties/docker-compose/hive/scripts/paimon1/db1.db/dup_columns_table/bucket-0/data-72d4d52a-eca9-4542-a2af-cc17499731e6-0.parquet
 copy 
docker/thirdparties/docker-compose/hive/scripts/paimon1/db1.db/{merge_on_read_table/manifest/manifest-20e5db07-f633-4e29-8b06-93cdc5ff787f-0
 => dup_columns_table/manifest/manifest-5cae1365-d123-4172-9a89-4fbc02bee658-0} 
(88%)
 copy 
docker/thirdparties/docker-compose/hive/scripts/paimon1/db1.db/{merge_on_read_table/manifest/manifest-list-69d9eec6-cd34-45f7-9f89-435b76fa8b03-0
 => 
dup_columns_table/manifest/manifest-list-391058a9-952c-4aa9-892f-df3334e4109b-0}
 (94%)
 copy 
docker/thirdparties/docker-compose/hive/scripts/paimon1/db1.db/{append_table/manifest/manifest-list-bfad6e57-668c-4bf4-ac98-64c7e14cc8aa-1
 => 
dup_columns_table/manifest/manifest-list-391058a9-952c-4aa9-892f-df3334e4109b-1}
 (84%)
 copy 
docker/thirdparties/docker-compose/hive/scripts/paimon1/db1.db/{tb_with_upper_case
 => dup_columns_table}/schema/schema-0 (57%)
 copy 
docker/thirdparties/docker-compose/hive/scripts/paimon1/db1.db/{append_table => 
dup_columns_table}/snapshot/EARLIEST (100%)
 copy 
docker/thirdparties/docker-compose/hive/scripts/paimon1/db1.db/{all_table_with_parquet
 => dup_columns_table}/snapshot/LATEST (100%)
 copy 
docker/thirdparties/docker-compose/hive/scripts/paimon1/db1.db/{append_table => 
dup_columns_table}/snapshot/snapshot-1 (50%)
 create mode 100644 
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/StatsDerive.java
 create mode 100644 
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/MysqlDataDescription.java
 create mode 100644 
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/load/MysqlLoadCommand.java
 create mode 100644 
fe/fe-core/src/test/java/org/apache/doris/common/profile/ProfileStructureTest.java
 create mode 100644 
fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/SubqueryToApplyTest.java
 create mode 100644 regression-test/suites/insert_p0/test_insert_timeout.groovy
 create mode 100644 thirdparty/patches/thrift-0.16-fix_mutex_include.patch


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to