This is an automated email from the ASF dual-hosted git repository. dataroaring pushed a change to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/doris.git
from 7259dd0ee5d [fix](nereids) fix partitionTopN choosing under multi winexprs (#39233) new 21c946f61f9 [improvement](iceberg)add some description for `show create` (#39179) new edf190307da [case](iceberg)append iceberg schema change case. (#38766) new 95ea213351e [Fix](broker)fix change afs and bos properties not working cause of hadoop filesystem cache. (#39117) new d2c37907361 [Improment]Add workload group resource usage (#39177) new f3e28031b10 [enhancement](regression-test) agg schema key modify case (#39198) new ba2397f4e03 [enhancement](regression-test) agg schema key/value drop case (#39098) new 219694cb129 [enhancement](regression-test) agg schema key add case (#38914) new b8e4811706d [fix](tools) tpcds-tools: report benchmark result before cleanup (#38864) new 5fdd144dcb2 [opt](nereids) get table row count according to BE reported num and analyze result (#38880) new ec7619d9431 [fix](nereids)update regression case for eager-agg (#38464) new 7298b416f03 [Chore](materialized-view) add key column limit of uniq table's create mv stmt (#39139) new 3ad98f2d60d [fix](jdbc scan) Remove the `conjuncts.remove` call in JdbcScan (#39180) new a01518cdd1d [enhance](mtmv)hive cache add partitionId to partitionName Map (#38525) new 3c5a2a26c49 [enhancement](regression-test) agg schema value add case (#38968) new 25c8f45ae68 [opt](bytebuffer) allocate ByteBuffer memory by Allocator and make it exception safe (#38960) new 467c2fa3dc2 [enhance](auth)modify priv of refresh catalog/db/table (#39008) new fc572bc0a75 [fix](variable) support all type functions (#39144) new 26b494f3faf [fix](window_funnel) fix wrong result of window_funnel (#38954) new fc1a4763f83 [fix](Nereids) fix insert into table with null literal default value (#39122) new d115a9f96d4 [chore](table) Add batch method to get visible version of the olap table (#38949) new 21213baa8e5 [opt](log) avoid lots of json parse error logs (#39190) new a99926f8f85 [Fix]Fix wg test failed in cloud mode (#39034) new 83aed35acd3 [refactor](minor) Delete non-pipeline code (#39156) new 6df5fd57ba1 [Refactor](config) remove unless config in doris (#39218) new 6a5e65fe6ea [Fix](JsonReader) Return correct status when parse failed (#39206) new 6266743d4c8 [enhance](Hdfs) Add bvar latencyrecorder for HDFS operation in Recycler (#39236) new 3de1c942e34 [fix](ES Catalog)Check isArray before parse json to array (#39104) new b759df27659 [fix](suites) fix backup cancelled cases (#39204) new 09d61526f1b [opt](nereids) Join related cost calculation formula works well at least 3be num (#39118) new 36fc3d56d06 [fix](inverted index)Add exception check when write bkd index (#39248) new 78d4e688ec8 [chore](cloud) Add show_meta_ranges API (#39208) new 507ea3af702 [refactor](S3) Refactor S3 file writer's duplicate code into common function (#38771) new 2e843a6be6c [chore](ci) add branch-3.0 required checks (#39250) new a1219d0f270 [opt](assert_cast) Make assert cast do type check in release build by default (#39030) new 16aa2bd84cb [Improvement](runtime-filter) do not use bloom to replace in_or_bloom when rf need merge (#39147) new 4bc3932c8fd [Fix](function) fix coredump because short of check on randoms arguments (#39255) new ec88399d3d3 Revert "[chore](ci) add branch-3.0 required checks" (#39294) new fe7f195f9d4 [refactor](opt) optimize BE code of array_distinct function (#38560) new f30529a9903 [fix](inverted index) Writing to the inverted index also writes to the file cache. (#39076) new 75f9594c4a8 [Bug](brpc) fix sync_filter_size/apply_filterv2 has wrong closure (#39155) new 8e3c030af31 [fix](expr) Enhance SQL Expression Handling by Introducing printSqlInParens to CompoundPredicate (#39064) new 1237e327151 [Fix](local merge) Fix local exchange dependencies acquired by local merge (#39238) new 16ae1eba908 [refactor](minor) Init counter in prepare phase (#39287) new a2bcbe13350 [fix](routine load) add read lock to fix some concurrent bugs (#39242) new 61a30d6dfa1 [opt](set operation) INTERSECT should evaluated before others (#39095) new 9edd639fcbb [enhance](mtmv) mtmv query sql expand star (#36543) new f1083ef1475 [fix](Nereids) polish function signature search algorithm (#38497) new fa91fbaef9c [opt](inverted index) Optimize the usage of the `multi_match` function (#39193) new 6fe8417bb60 [deps](hadoop) update hadoop libs to 3.3.6.2 (#39282) new 1af389464ca [Enhancement] add information_schema.table_properties #38745 (#38746) new b4a8c6f5fe1 [improve](ip)improve data type ip for rowstore and trycatch in segment_flusher (#39258) new 3de7a76cdd6 [fix](compile) failed on MacOS because size_t is not uint64_t on MacOS (#39296) new 3da9d7d07e6 [Fix](nereids) fix condition function partition prune (#39298) new e51a2e2378c [fix](function) fix error return type in corr(float32,float32) (#39251) new f37e314f341 [fix](function) Results for stddev with nan and inf are unstable. (#38866) new b8c9cacc32f [fix](function)timediff with now function causes a error signature (#39322) new 4fe2885494a [chore](test) rename conflict name under same database in test cases (#39243) new 7e7ee67ce6e [Improvement](runtime-filter) adjust always true judge rule of runtime filter apply (#38300) new b01a52ba4c4 [fix](Nereids) support implicit cast ip types to string (#39318) new a6adb47b270 [opt](mtmv) partition rollup support week and quarter (#39286) new 07dbee40208 [fix](cloud) fix routine load job progress cache incorrect in cloud mode (#39313) new 649922a97a5 [Fix](cloud) CloudStorageEngine should wait compaction task done when stoping it (#39302) new 67961b006ef [improve](move-memtable) reuse connection in load_stream_stub (#39231) new 30fdb4cfe4a [fix](routine load) should update progress before handle transaction state transform (#39311) new da074c7da83 [opt](Nereids) add truncate statement and update set statement (#39312) new a6b96c70c9c [opt](memory) BE memory info compatible with CgroupV2 (#39256) new cbec39fe945 [enhance](mtmv)Improve the performance of obtaining partition/table versions (#39301) new b03dbbb7560 [Chore](aggregation) remove useless code for aggregation function Covar (#39309) new 4c8235d304e [fix](protocol) CLIENT_MULTI_STATEMENTS not used actually (#39308) new 29988c4665b [improve](nestedtype) support schema change for nested type (#39210) new ccff40e8dfc [testcase](auto-partition) test auto partition with altering property of table (#39260) new 1ba5e9257cd Revert "[fix] (scan) push down expr in agg and mor" (#39212) new 805a0d587a9 [test](inverted index) do not run test_index_lowercase_fault_injection in cloud mode (#39235) new e80374e5119 [improvement](jdbc catalog) Compatible with ojdbc6 by adding version check (#39341) new 662cd1a2144 [test](jdbc catalog) add oceanbase ce jdbc catalog test (#34972) new 51ab1cbad50 [Bug](runtime-filter) set inited to true on BloomFilterFuncBase::assign (#39335) new 1575053d204 [improvement](external catalog)Optimize the process of refreshing catalog (#39205) new 47391207794 [fix](nullif) revert nullif signature change by #38497 (#39342) new 0107e95fe40 [Enhancement](alter) support add partitions in batch (#37114) new 2e20d94d822 [chore](recycler) log num deleted objects (#39353) new c2c5182a9a7 [opt](mtmv) support creating views based on Hive tables with the LIMIT (#38812) new 0cf14cd7b38 [fix](file cache) Fix data race of rowset meta when download segment data (#39361) new a73fb9f9d1b [fix](group commit) fix group commit core if be inject FragmentMgr.exec_plan_fragment.failed (#39339) new 2801b67754f [fix](nereids)fix unstable test case (#39244) new 3293c762ece [fix](regress-test) fix presto dialect regression-test case (#39326) new a81c8e69940 [fix])(function) add function regexp_extract_or_null (#38296) new 25cf8cdd6c9 [Fix](regression) Fix flaky case in test_stream_load_2pc (#39257) new 6ad17912370 [chore](ci) only add doing label on master pr (#39297) new 9cab50040f6 [ci](opt) support skil buildall (#38616) new de838c4d129 [fix](statistics)Fix update rows bug. (#39368) new b1cb62c183a [improve](ip) update ip for bloom_filter index (#39253) new db853add41a [deps](hadoop) update hadoop to 3.3.6.3 (#39376) new 37eee1cf8a4 [fix](cgroup) Fix cgroup compatibility with MacOS (#39398) new 1011a2b2053 [fix](http) fix http url with incorrect character notation (#38420) new 907a4f60f12 [fix](stream load) do not throw exception but skip record when can not find database (#39360) new f1ba7622a06 [regression](test) adjust max_failure_num to 50 (#39420) new 43d2c258584 [opt](Nereids) add switch / unset / kill / desc statement syntax (#39359) new ea059be8310 [improve](array) forbid create inverted index with parser for array and add cases for write correctness check (#39062) new a1d4b8d2691 [chore](cloud) Add recycle tablet key log (#39405) new 3ca955cbb5c [fix](function) avoid calculating sqrt of negative in agg function CORR (#39324) new 729025a99ff [chore](cloud) Log rowset value when txn value size exceeds the limit (#39388) new 888ab3bb244 [fix](nereids) fix distribute expr list (#39148) new 3be198fcb8e [Fix](Row store) fix row store with invalid json string in variant type (#39394) new ce34258859c [test](inverted index) add test for need read data opt (#38261) new 02303b935df [test](statistics)Add delete test case for statistics. (#39143) new 238fe535b47 [fix](group commit) fix group commit insert rpc may stuck (#39391) new 55519d3e7b5 [Fix](regression) fix unstable case `test_schema_change_unique_mow` (#39315) new e5eca21ce4e [enhancement](schema-change) Record detailed fail reason for schema change tasks (#39351) new e9f15876acb [opt](Nereids) support parse sql with semicolon at beginning (#39399) new 02e93f41a9d [Fix](tools) Fix wrong tpcds tools instructions (#39001) new e11c7ec3eb0 [opt](Nereids) polish aggregate function signature matching (#39352) new da3b0bd042e [fix](Nereids) npe when delete with cte and without using (#39379) new bbf97723bbd [fix](delete) Incorrect status handling in CloudDeleteTask::execute() (#39428) new c9ab71a784c [improvement](statistics)Support drop cached stats. (#39367) new fa89b742b13 [fix] (inverted index) remove tmp columns in block (#39369) new c07f97252fc [Fix](load) Fix the incorrect src value printed in the error log when strict mode is true (#39447) new 0131b2a24f8 [fix](catalog) set timeout for split fetch (#39346) new 245880455b9 [Improvement](multi-catalog)support max compute complex type parse (#39259) new 1ac05ad5a4f [opt](binlog) Log table rename when reply (#38034) new 8fe4ba275fa [Improvement]Support cgroup v2 for workload group (#39374) new 8ac98512694 [enhancement][regression] remove binary data in regression test output file (#39364) new a456f8a83c8 [fix](java udf) fix clean_udf_cache_callback without enable_java_support (#39340) new 05a6b86635e [improvement](nereids) support extract from disjunction in join on condition (#38479) new 9160e5a1541 [fix](Nereids) fix explain plan with sql cache (#39431) new 085553c226c [Fix](Txn) Fix wrong columns sequence in txn model (#39295) new 37b94434321 [chore](conf)remove unused doris_max_scan_key_num and max_send_batch_parallelism_per_job conf (#39219) new 0c89e994353 [Improvement](sort) add session variable force_sort_algorithm and adjust some parameter about sort (#39334) new eb96213f92b [improve](explode-json-array)change explode-json-array-xx func signature from string to json type (#37278) new 0fe77c12dcf [Fix](core) Fix wal mgr heap use after free when stop doris (#33131) new 88df6b14cad [fix](mtmv)Fix cancelled tasks with running status (#39424) new 3138e99a7cb [fix](nereids) fix unstable mv regression cases (#39265) new d281487aa22 [enhance](mtmv)refresh hms table before run mtmv task (#38212) new 9f189524c11 [opt](Nereids) support create statement syntax in Nereids (#39427) new 1535ae58292 [opt] (inverted index) add inverted index file size for open file (#37482) new 5bfd7115b1b [Chore](exec) remove some thread pool and rename fragmentMgrThreadPool (#39344) new 1daf1d7051c [Fix](build) fix cloud compilation on arm without AZURE (#39507) new e790a85fe00 [opt](query cancel) cancel query if it has pipeline task leakage (#39223) new c37a1d7cd3a [Chore] remove unused code (#38903) The 138 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/workflows/comment-to-trigger-teamcity.yml | 33 +- .github/workflows/labeler/scope-label-conf.yml | 4 +- be/src/agent/be_exec_version_manager.h | 2 +- be/src/agent/cgroup_cpu_ctl.cpp | 354 ++- be/src/agent/cgroup_cpu_ctl.h | 87 +- be/src/agent/task_worker_pool.cpp | 10 +- be/src/cloud/cloud_delete_task.cpp | 2 +- be/src/cloud/cloud_meta_mgr.cpp | 3 + be/src/cloud/cloud_rowset_writer.cpp | 8 + be/src/cloud/cloud_storage_engine.cpp | 8 + be/src/cloud/cloud_warm_up_manager.cpp | 12 +- be/src/cloud/pb_convert.cpp | 8 + be/src/common/cgroup_memory_ctl.cpp | 196 ++ be/src/common/cgroup_memory_ctl.h | 45 + be/src/common/config.cpp | 28 +- be/src/common/config.h | 25 +- be/src/common/status.h | 4 +- be/src/exec/es/es_scroll_parser.cpp | 25 +- be/src/exec/schema_scanner.cpp | 6 + .../schema_scanner/schema_backend_active_tasks.cpp | 2 +- .../schema_table_options_scanner.cpp | 57 +- .../schema_scanner/schema_table_options_scanner.h | 7 +- ...ner.cpp => schema_table_properties_scanner.cpp} | 93 +- ...scanner.h => schema_table_properties_scanner.h} | 17 +- ...hema_workload_group_resource_usage_scanner.cpp} | 56 +- ...schema_workload_group_resource_usage_scanner.h} | 15 +- be/src/exprs/bloom_filter_func.h | 1 + be/src/exprs/json_functions.cpp | 9 +- be/src/exprs/runtime_filter.cpp | 25 +- be/src/http/action/http_stream.cpp | 51 +- be/src/http/action/stream_load.cpp | 32 +- be/src/http/http_client.cpp | 60 +- be/src/http/http_client.h | 9 + be/src/io/cache/block_file_cache_downloader.cpp | 12 +- be/src/io/fs/local_file_reader.cpp | 3 + be/src/io/fs/multi_table_pipe.cpp | 2 +- be/src/io/fs/s3_file_writer.cpp | 116 +- be/src/io/fs/s3_file_writer.h | 2 + be/src/io/fs/stream_load_pipe.cpp | 7 +- be/src/olap/base_tablet.cpp | 4 +- be/src/olap/bitmap_filter_predicate.h | 2 + be/src/olap/bloom_filter_predicate.h | 3 + be/src/olap/column_predicate.h | 28 +- be/src/olap/compaction.cpp | 8 +- be/src/olap/comparison_predicate.h | 45 +- be/src/olap/in_list_predicate.h | 4 +- be/src/olap/olap_common.h | 1 + be/src/olap/rowset/beta_rowset.cpp | 6 +- be/src/olap/rowset/beta_rowset_writer.cpp | 22 +- be/src/olap/rowset/beta_rowset_writer.h | 60 + be/src/olap/rowset/beta_rowset_writer_v2.cpp | 2 +- be/src/olap/rowset/beta_rowset_writer_v2.h | 2 + be/src/olap/rowset/rowset_meta.cpp | 30 + be/src/olap/rowset/rowset_meta.h | 10 + be/src/olap/rowset/rowset_writer_context.h | 10 + be/src/olap/rowset/segment_creator.cpp | 25 +- be/src/olap/rowset/segment_creator.h | 8 +- .../segment_v2/bloom_filter_index_writer.cpp | 4 +- .../segment_v2/inverted_index_compaction.cpp | 1 - .../segment_v2/inverted_index_compound_reader.cpp | 117 +- .../segment_v2/inverted_index_compound_reader.h | 29 +- .../segment_v2/inverted_index_file_reader.cpp | 93 +- .../rowset/segment_v2/inverted_index_file_reader.h | 11 +- .../segment_v2/inverted_index_file_writer.cpp | 24 +- .../rowset/segment_v2/inverted_index_file_writer.h | 19 +- .../segment_v2/inverted_index_fs_directory.cpp | 35 +- .../segment_v2/inverted_index_fs_directory.h | 11 +- .../rowset/segment_v2/inverted_index_writer.cpp | 57 +- be/src/olap/rowset/segment_v2/segment.cpp | 15 +- be/src/olap/rowset/segment_v2/segment.h | 7 +- be/src/olap/rowset/segment_v2/segment_iterator.cpp | 127 +- be/src/olap/rowset/segment_v2/segment_iterator.h | 10 +- be/src/olap/rowset/segment_v2/segment_writer.cpp | 18 +- be/src/olap/rowset/segment_v2/segment_writer.h | 7 +- .../rowset/segment_v2/vertical_segment_writer.cpp | 21 +- .../rowset/segment_v2/vertical_segment_writer.h | 9 +- be/src/olap/rowset/vertical_beta_rowset_writer.cpp | 15 +- be/src/olap/single_replica_compaction.cpp | 15 +- be/src/olap/tablet_meta.h | 12 - be/src/olap/task/engine_clone_task.cpp | 20 +- be/src/olap/task/index_builder.cpp | 4 +- be/src/olap/wal/wal_manager.cpp | 5 + be/src/pipeline/dependency.h | 17 +- be/src/pipeline/exec/exchange_sink_buffer.h | 13 - be/src/pipeline/exec/exchange_sink_operator.cpp | 8 +- be/src/pipeline/exec/file_scan_operator.cpp | 6 +- be/src/pipeline/exec/olap_scan_operator.cpp | 35 +- be/src/pipeline/exec/olap_scan_operator.h | 2 +- be/src/pipeline/exec/operator.cpp | 7 +- .../exec/partitioned_aggregation_sink_operator.cpp | 2 + be/src/pipeline/exec/scan_operator.cpp | 14 +- be/src/pipeline/exec/scan_operator.h | 6 +- .../local_exchange_source_operator.cpp | 17 +- be/src/pipeline/pipeline_fragment_context.cpp | 8 +- be/src/pipeline/pipeline_task.cpp | 1 + be/src/runtime/exec_env.h | 13 +- be/src/runtime/exec_env_init.cpp | 24 +- be/src/runtime/fragment_mgr.cpp | 205 +- be/src/runtime/fragment_mgr.h | 14 +- be/src/runtime/frontend_info.h | 6 + be/src/runtime/group_commit_mgr.cpp | 9 +- be/src/runtime/query_context.cpp | 25 +- be/src/runtime/query_context.h | 25 +- be/src/runtime/runtime_filter_mgr.cpp | 10 +- be/src/runtime/stream_load/stream_load_context.h | 8 +- .../runtime/stream_load/stream_load_executor.cpp | 9 +- be/src/runtime/thread_context.h | 36 +- be/src/runtime/workload_group/workload_group.cpp | 53 +- be/src/runtime/workload_group/workload_group.h | 19 + .../workload_group/workload_group_manager.cpp | 74 +- .../workload_group/workload_group_manager.h | 10 +- be/src/runtime/workload_management/io_throttle.cpp | 40 +- be/src/runtime/workload_management/io_throttle.h | 18 +- be/src/service/backend_service.cpp | 3 +- be/src/service/internal_service.cpp | 42 +- be/src/util/brpc_client_cache.cpp | 22 +- be/src/util/brpc_client_cache.h | 24 +- be/src/util/byte_buffer.h | 19 +- be/src/util/cgroup_util.cpp | 232 +- be/src/util/cgroup_util.h | 87 +- be/src/util/doris_metrics.h | 1 + be/src/util/mem_info.cpp | 61 +- be/src/util/simd/bits.h | 16 + .../vec/aggregate_functions/aggregate_function.h | 109 +- .../aggregate_function_approx_count_distinct.h | 7 +- .../aggregate_functions/aggregate_function_avg.h | 3 +- .../aggregate_function_avg_weighted.h | 6 +- .../aggregate_function_binary.h | 3 +- .../aggregate_functions/aggregate_function_bit.h | 4 +- .../aggregate_function_bitmap.h | 19 +- .../aggregate_function_bitmap_agg.h | 9 +- .../aggregate_function_collect.h | 42 +- .../aggregate_function_corr.cpp | 3 +- .../aggregate_functions/aggregate_function_count.h | 4 +- .../aggregate_function_covar.cpp | 9 - .../aggregate_functions/aggregate_function_covar.h | 120 +- .../aggregate_function_distinct.h | 4 +- .../aggregate_function_foreach.h | 9 +- .../aggregate_function_group_array_intersect.h | 23 +- .../aggregate_function_group_concat.h | 10 +- .../aggregate_function_histogram.h | 7 +- .../aggregate_function_hll_union_agg.h | 2 +- .../aggregate_functions/aggregate_function_map.h | 22 +- .../aggregate_function_min_max.h | 45 +- .../aggregate_function_min_max_by.h | 3 +- .../aggregate_functions/aggregate_function_null.h | 7 +- .../aggregate_function_orthogonal_bitmap.h | 21 +- .../aggregate_function_percentile.h | 131 +- .../aggregate_function_percentile_approx.h | 20 +- .../aggregate_function_product.h | 3 +- .../aggregate_function_quantile_state.h | 10 +- .../aggregate_function_reader_first_last.h | 25 +- .../aggregate_function_retention.h | 4 +- .../aggregate_function_sequence_match.h | 14 +- .../aggregate_function_simple_factory.cpp | 2 + .../aggregate_function_stddev.h | 17 +- .../aggregate_functions/aggregate_function_sum.h | 3 +- .../aggregate_functions/aggregate_function_topn.h | 57 +- .../aggregate_functions/aggregate_function_uniq.h | 7 +- .../aggregate_function_window.h | 6 +- .../aggregate_function_window_funnel.cpp | 35 +- .../aggregate_function_window_funnel.h | 423 +++- be/src/vec/columns/column_array.cpp | 34 +- be/src/vec/columns/column_array.h | 4 +- be/src/vec/columns/column_complex.h | 4 +- be/src/vec/columns/column_const.h | 2 +- be/src/vec/columns/column_decimal.cpp | 2 +- be/src/vec/columns/column_decimal.h | 4 +- be/src/vec/columns/column_map.cpp | 2 +- be/src/vec/columns/column_map.h | 5 +- be/src/vec/columns/column_nullable.cpp | 10 +- be/src/vec/columns/column_nullable.h | 22 +- be/src/vec/columns/column_object.cpp | 6 +- be/src/vec/columns/column_string.cpp | 21 +- be/src/vec/columns/column_struct.cpp | 10 +- be/src/vec/columns/column_vector.cpp | 3 +- be/src/vec/columns/column_vector.h | 9 +- be/src/vec/common/assert_cast.h | 35 +- be/src/vec/common/sort/sorter.cpp | 2 + be/src/vec/core/block.cpp | 9 + be/src/vec/core/block.h | 5 + be/src/vec/data_types/data_type_number_base.cpp | 29 +- .../serde/data_type_datetimev2_serde.cpp | 6 +- .../vec/data_types/serde/data_type_ipv6_serde.cpp | 22 + be/src/vec/data_types/serde/data_type_ipv6_serde.h | 3 + .../data_types/serde/data_type_object_serde.cpp | 30 +- .../data_types/serde/data_type_struct_serde.cpp | 10 +- be/src/vec/exec/format/orc/vorc_reader.cpp | 8 +- .../exec/format/parquet/vparquet_group_reader.cpp | 9 +- be/src/vec/exec/scan/new_es_scanner.cpp | 1 - be/src/vec/exec/scan/new_es_scanner.h | 8 +- be/src/vec/exec/scan/new_jdbc_scanner.h | 3 +- be/src/vec/exec/scan/new_olap_scanner.cpp | 6 +- be/src/vec/exec/scan/new_olap_scanner.h | 3 - be/src/vec/exec/scan/scanner_context.cpp | 11 +- be/src/vec/exec/scan/scanner_context.h | 2 - be/src/vec/exec/scan/split_source_connector.cpp | 5 + be/src/vec/exec/scan/vfile_scanner.cpp | 30 +- be/src/vec/exec/scan/vfile_scanner.h | 10 +- be/src/vec/exec/scan/vmeta_scanner.cpp | 1 - be/src/vec/exec/scan/vmeta_scanner.h | 5 +- be/src/vec/exec/scan/vscanner.cpp | 20 +- be/src/vec/exec/scan/vscanner.h | 12 +- .../exprs/table_function/vexplode_json_array.cpp | 22 +- .../vec/exprs/table_function/vexplode_json_array.h | 151 ++ be/src/vec/exprs/vectorized_fn_call.cpp | 5 +- be/src/vec/exprs/vectorized_fn_call.h | 2 +- be/src/vec/exprs/vexpr.cpp | 2 +- be/src/vec/exprs/vexpr.h | 13 +- be/src/vec/exprs/vexpr_context.cpp | 56 +- be/src/vec/exprs/vexpr_context.h | 4 +- be/src/vec/exprs/vruntimefilter_wrapper.cpp | 50 +- be/src/vec/exprs/vruntimefilter_wrapper.h | 47 +- .../vec/functions/array/function_array_distinct.h | 106 +- .../vec/functions/array/function_array_element.h | 1 + be/src/vec/functions/array/function_array_index.h | 15 + be/src/vec/functions/comparison_equal_for_null.cpp | 8 +- be/src/vec/functions/function.h | 8 +- be/src/vec/functions/function_binary_arithmetic.h | 24 +- be/src/vec/functions/function_case.h | 8 +- be/src/vec/functions/function_cast.h | 17 +- be/src/vec/functions/function_coalesce.cpp | 4 +- be/src/vec/functions/function_fake.cpp | 3 +- be/src/vec/functions/function_helpers.cpp | 2 +- be/src/vec/functions/function_helpers.h | 2 +- be/src/vec/functions/function_ip.h | 4 +- be/src/vec/functions/function_jsonb.cpp | 2 - be/src/vec/functions/function_multi_match.cpp | 121 +- be/src/vec/functions/function_multi_match.h | 17 +- be/src/vec/functions/function_regexp.cpp | 18 +- be/src/vec/functions/function_string.cpp | 1 + be/src/vec/functions/function_string.h | 17 +- be/src/vec/functions/function_variant_element.cpp | 4 +- be/src/vec/functions/random.cpp | 11 +- be/src/vec/olap/vcollect_iterator.cpp | 19 +- be/src/vec/sink/load_stream_stub.cpp | 3 +- be/src/vec/sink/writer/async_result_writer.cpp | 7 + be/src/vec/sink/writer/vtablet_writer.cpp | 4 +- be/src/vec/sink/writer/vtablet_writer_v2.cpp | 2 +- be/test/http/http_client_test.cpp | 42 + .../segment_v2/inverted_index_array_test.cpp | 9 +- be/test/util/cgroup_util_test.cpp | 49 +- be/test/util/test_data/memory.limit_in_bytes | 1 + be/test/util/test_data/memory.stat | 36 + be/test/vec/exec/vwal_scanner_test.cpp | 10 +- be/test/vec/function/function_like_test.cpp | 39 + cloud/CMakeLists.txt | 4 +- cloud/cmake/thirdparty.cmake | 10 +- cloud/src/meta-service/meta_service.cpp | 23 +- cloud/src/meta-service/meta_service_helper.h | 8 +- cloud/src/meta-service/meta_service_http.cpp | 71 + cloud/src/meta-service/meta_service_txn.cpp | 41 +- cloud/src/meta-service/txn_kv.cpp | 69 + cloud/src/meta-service/txn_kv.h | 11 + cloud/src/recycler/azure_obj_client.cpp | 2 + cloud/src/recycler/hdfs_accessor.cpp | 33 +- cloud/src/recycler/obj_storage_client.cpp | 14 +- cloud/src/recycler/recycler.cpp | 6 +- cloud/src/recycler/s3_accessor.h | 9 +- cloud/src/recycler/s3_obj_client.cpp | 2 + cloud/src/recycler/sync_executor.h | 6 +- cloud/src/recycler/util.h | 6 + .../scripts/data/composite_type_array_bulk.json | 10 + .../elasticsearch/scripts/es_init.sh | 58 + .../index/array_meta_composite_type_array.json | 9 + .../scripts/index/es6_composite_type_array.json | 14 + .../scripts/index/es7_composite_type_array.json | 12 + .../docker-compose/iceberg/entrypoint.sh.tpl | 23 +- .../docker-compose/iceberg/iceberg.yaml.tpl | 25 +- .../scripts/create_preinstalled_scripts/run01.sql | 77 + .../scripts/create_preinstalled_scripts/run02.sql | 75 + .../scripts/create_preinstalled_scripts/run03.sql | 75 + .../scripts/create_preinstalled_scripts/run04.sql | 80 + .../create_preinstalled_scripts/run05.sql} | 0 .../docker-compose/iceberg/spark-init-paimon.sql | 1 - .../docker-compose/mysql/init/03-create-table.sql | 11 +- .../docker-compose/mysql/init/04-insert.sql | 4 + .../{mariadb => oceanbase}/init/01-drop-db.sql | 0 .../{mariadb => oceanbase}/init/02-create-db.sql | 0 .../oceanbase/init/03-create-table.sql | 51 + .../docker-compose/oceanbase/init/04-insert.sql | 30 + .../docker-compose/oceanbase/oceanbase.env | 13 +- .../docker-compose/oceanbase/oceanbase.yaml.tpl | 51 + docker/thirdparties/run-thirdparties-docker.sh | 22 +- fe/be-java-extensions/jdbc-scanner/pom.xml | 4 + .../org/apache/doris/jdbc/BaseJdbcExecutor.java | 30 +- .../org/apache/doris/jdbc/OracleJdbcExecutor.java | 115 +- .../doris/maxcompute/MaxComputeColumnValue.java | 41 +- .../doris/maxcompute/MaxComputeJniScanner.java | 24 +- .../org/apache/doris/catalog/PrimitiveType.java | 8 + .../main/java/org/apache/doris/common/Config.java | 2 +- .../antlr4/org/apache/doris/nereids/DorisLexer.g4 | 7 + .../antlr4/org/apache/doris/nereids/DorisParser.g4 | 197 +- fe/fe-core/src/main/cup/sql_parser.cup | 12 + .../main/java/org/apache/doris/alter/Alter.java | 10 +- .../doris/alter/MaterializedViewHandler.java | 20 +- .../org/apache/doris/alter/SchemaChangeJobV2.java | 6 +- .../doris/analysis/AlterMultiPartitionClause.java | 73 + .../apache/doris/analysis/CompoundPredicate.java | 2 + .../apache/doris/analysis/CreateCatalogStmt.java | 2 +- .../doris/analysis/CreateDataSyncJobStmt.java | 2 +- .../org/apache/doris/analysis/CreateDbStmt.java | 2 +- .../doris/analysis/CreateEncryptKeyStmt.java | 2 +- .../org/apache/doris/analysis/CreateFileStmt.java | 2 +- .../apache/doris/analysis/CreateFunctionStmt.java | 2 +- .../org/apache/doris/analysis/CreateMTMVStmt.java | 9 +- .../doris/analysis/CreateMaterializedViewStmt.java | 2 +- .../apache/doris/analysis/CreatePolicyStmt.java | 2 +- .../doris/analysis/CreateRepositoryStmt.java | 2 +- .../apache/doris/analysis/CreateResourceStmt.java | 2 +- .../doris/analysis/CreateSqlBlockRuleStmt.java | 2 +- .../org/apache/doris/analysis/CreateStageStmt.java | 2 +- .../doris/analysis/CreateStorageVaultStmt.java | 2 +- .../org/apache/doris/analysis/CreateUserStmt.java | 2 +- .../doris/analysis/CreateWorkloadGroupStmt.java | 2 +- .../analysis/CreateWorkloadSchedPolicyStmt.java | 2 +- .../org/apache/doris/analysis/DescribeStmt.java | 2 +- .../apache/doris/analysis/DropCachedStatsStmt.java | 121 + .../java/org/apache/doris/analysis/EmptyStmt.java | 2 +- .../java/org/apache/doris/analysis/IndexDef.java | 6 +- .../apache/doris/analysis/InvertedIndexUtil.java | 6 + .../java/org/apache/doris/analysis/KillStmt.java | 2 +- .../apache/doris/analysis/NativeInsertStmt.java | 8 +- .../apache/doris/analysis/RefreshCatalogStmt.java | 6 +- .../org/apache/doris/analysis/RefreshDbStmt.java | 11 +- .../apache/doris/analysis/RefreshTableStmt.java | 11 +- .../org/apache/doris/analysis/SchemaTableType.java | 6 +- .../java/org/apache/doris/analysis/SetStmt.java | 2 +- .../java/org/apache/doris/analysis/SwitchStmt.java | 2 +- .../apache/doris/analysis/TruncateTableStmt.java | 2 +- .../analysis/UnsetDefaultStorageVaultStmt.java | 2 +- .../apache/doris/analysis/UnsetVariableStmt.java | 2 +- .../java/org/apache/doris/analysis/UseStmt.java | 2 +- .../doris/catalog/BuiltinScalarFunctions.java | 2 + .../main/java/org/apache/doris/catalog/Column.java | 8 + .../java/org/apache/doris/catalog/ColumnType.java | 6 +- .../main/java/org/apache/doris/catalog/Env.java | 24 +- .../main/java/org/apache/doris/catalog/MTMV.java | 15 - .../java/org/apache/doris/catalog/OlapTable.java | 139 +- .../org/apache/doris/catalog/OlapTableFactory.java | 13 +- .../org/apache/doris/catalog/RefreshManager.java | 2 +- .../java/org/apache/doris/catalog/SchemaTable.java | 23 +- .../java/org/apache/doris/catalog/TableIf.java | 3 +- .../apache/doris/cloud/catalog/CloudPartition.java | 17 +- .../org/apache/doris/cloud/rpc/VersionHelper.java | 38 +- .../java/org/apache/doris/common/ErrorCode.java | 2 + .../apache/doris/datasource/ExternalCatalog.java | 9 + .../apache/doris/datasource/InternalCatalog.java | 20 + .../org/apache/doris/datasource/SplitSource.java | 9 +- .../doris/datasource/hive/HMSExternalTable.java | 26 +- .../doris/datasource/hive/HiveMetaStoreCache.java | 10 +- .../doris/datasource/hive/HiveMetadataOps.java | 4 +- .../iceberg/IcebergExternalDatabase.java | 11 + .../datasource/iceberg/IcebergMetadataOps.java | 4 +- .../doris/datasource/jdbc/JdbcExternalCatalog.java | 5 + .../doris/datasource/jdbc/source/JdbcScanNode.java | 39 +- .../doris/datasource/odbc/source/OdbcScanNode.java | 5 +- .../apache/doris/job/extensions/mtmv/MTMVTask.java | 34 +- .../org/apache/doris/job/task/AbstractTask.java | 2 +- .../org/apache/doris/load/StreamLoadRecordMgr.java | 4 +- .../doris/load/routineload/KafkaProgress.java | 23 +- .../load/routineload/KafkaRoutineLoadJob.java | 2 +- .../doris/load/routineload/RoutineLoadJob.java | 53 +- .../mtmv/{EnvInfo.java => MTMVBaseVersions.java} | 35 +- .../doris/mtmv/MTMVPartitionExprDateTrunc.java | 8 +- .../org/apache/doris/mtmv/MTMVPartitionUtil.java | 151 +- .../java/org/apache/doris/mtmv/MTMVPlanUtil.java | 35 - .../org/apache/doris/mtmv/MTMVRefreshContext.java | 54 + .../org/apache/doris/mtmv/MTMVRelatedTableIf.java | 13 +- .../org/apache/doris/mtmv/MTMVRewriteUtil.java | 12 +- .../java/org/apache/doris/mysql/MysqlChannel.java | 11 + .../java/org/apache/doris/mysql/MysqlProto.java | 6 + .../org/apache/doris/nereids/NereidsPlanner.java | 28 +- .../org/apache/doris/nereids/cost/CostModelV1.java | 23 +- .../glue/translator/PhysicalPlanTranslator.java | 9 +- .../apache/doris/nereids/memo/GroupExpression.java | 2 +- .../doris/nereids/parser/LogicalPlanBuilder.java | 18 +- .../post/RuntimeFilterPushDownVisitor.java | 6 - .../properties/ChildOutputPropertyDeriver.java | 50 +- .../doris/nereids/rules/analysis/BindSink.java | 2 +- .../nereids/rules/analysis/ExpressionAnalyzer.java | 4 +- .../exploration/mv/MaterializationContext.java | 15 +- .../exploration/mv/MaterializedViewUtils.java | 2 + .../rules/OneRangePartitionEvaluator.java | 16 +- .../rules/implementation/AggregateStrategies.java | 3 +- ...xtractSingleTableExpressionFromDisjunction.java | 54 +- .../doris/nereids/stats/StatsCalculator.java | 181 +- .../expressions/functions/agg/AvgWeighted.java | 10 +- .../trees/expressions/functions/agg/BitmapAgg.java | 8 +- .../expressions/functions/agg/CollectList.java | 6 - .../trees/expressions/functions/agg/Corr.java | 10 +- .../trees/expressions/functions/agg/Covar.java | 10 +- .../trees/expressions/functions/agg/CovarSamp.java | 10 +- .../expressions/functions/agg/GroupBitAnd.java | 8 +- .../expressions/functions/agg/GroupBitOr.java | 8 +- .../expressions/functions/agg/GroupBitXor.java | 8 +- .../functions/agg/MultiDistinctSum.java | 17 +- .../functions/agg/MultiDistinctSum0.java | 17 +- .../trees/expressions/functions/agg/Stddev.java | 6 +- .../expressions/functions/agg/StddevSamp.java | 6 +- .../expressions/functions/agg/TopNWeighted.java | 73 +- .../trees/expressions/functions/agg/Variance.java | 6 +- .../expressions/functions/agg/VarianceSamp.java | 6 +- .../generator/ExplodeJsonArrayDouble.java | 2 + .../generator/ExplodeJsonArrayDoubleOuter.java | 2 + .../functions/generator/ExplodeJsonArrayInt.java | 2 + .../generator/ExplodeJsonArrayIntOuter.java | 2 + .../functions/generator/ExplodeJsonArrayJson.java | 4 +- .../generator/ExplodeJsonArrayJsonOuter.java | 4 +- .../generator/ExplodeJsonArrayString.java | 2 + .../generator/ExplodeJsonArrayStringOuter.java | 2 + .../expressions/functions/scalar/ArrayAvg.java | 4 +- .../expressions/functions/scalar/ArrayCumSum.java | 4 +- .../functions/scalar/ArrayDifference.java | 2 +- .../functions/scalar/ArrayEnumerateUniq.java | 3 +- .../expressions/functions/scalar/ArrayFilter.java | 2 +- .../expressions/functions/scalar/ArrayFirst.java | 5 - .../expressions/functions/scalar/ArrayLast.java | 5 - .../expressions/functions/scalar/ArrayProduct.java | 4 +- .../functions/scalar/ArrayPushBack.java | 4 +- .../functions/scalar/ArrayPushFront.java | 4 +- .../functions/scalar/ArrayReverseSplit.java | 14 +- .../expressions/functions/scalar/ArraySortBy.java | 2 +- .../expressions/functions/scalar/ArraySplit.java | 6 +- .../functions/scalar/ArrayWithConstant.java | 4 +- .../functions/scalar/BitmapFromArray.java | 6 +- .../expressions/functions/scalar/Coalesce.java | 6 +- .../trees/expressions/functions/scalar/Fmod.java | 4 +- .../expressions/functions/scalar/Greatest.java | 47 +- .../trees/expressions/functions/scalar/If.java | 11 +- .../expressions/functions/scalar/LastDay.java | 6 +- .../trees/expressions/functions/scalar/Least.java | 47 +- .../expressions/functions/scalar/MinutesDiff.java | 2 +- .../expressions/functions/scalar/MultiMatch.java | 22 +- .../trees/expressions/functions/scalar/Pmod.java | 42 +- .../trees/expressions/functions/scalar/Random.java | 11 +- .../scalar/{Pmod.java => RegexpExtractOrNull.java} | 31 +- .../expressions/functions/scalar/TimeDiff.java | 18 - .../expressions/visitor/ScalarFunctionVisitor.java | 5 + .../trees/plans/commands/info/AlterViewInfo.java | 2 +- .../trees/plans/commands/info/BaseViewInfo.java | 17 +- .../trees/plans/commands/info/CreateMTMVInfo.java | 34 +- .../trees/plans/commands/info/CreateViewInfo.java | 2 +- .../trees/plans/commands/info/IndexDefinition.java | 6 +- .../insert/BatchInsertIntoTableCommand.java | 51 +- .../trees/plans/commands/insert/InsertUtils.java | 4 +- .../trees/plans/physical/PhysicalOlapScan.java | 12 +- .../nereids/trees/plans/physical/PhysicalSink.java | 4 + .../doris/nereids/util/TypeCoercionUtils.java | 32 +- .../java/org/apache/doris/persist/EditLog.java | 3 + .../planner/BackendPartitionedSchemaScanNode.java | 1 + .../org/apache/doris/planner/OlapScanNode.java | 4 +- .../java/org/apache/doris/planner/SortNode.java | 28 +- .../java/org/apache/doris/qe/ConnectContext.java | 4 +- .../java/org/apache/doris/qe/ConnectProcessor.java | 3 +- .../main/java/org/apache/doris/qe/DdlExecutor.java | 3 + .../java/org/apache/doris/qe/SessionVariable.java | 23 +- .../java/org/apache/doris/qe/ShowExecutor.java | 8 + .../java/org/apache/doris/qe/StmtExecutor.java | 5 + .../resource/workloadgroup/WorkloadGroup.java | 2 +- .../apache/doris/service/FrontendServiceImpl.java | 27 +- .../apache/doris/statistics/AnalysisManager.java | 29 + .../doris/tablefunction/MetadataGenerator.java | 253 +- .../tablefunction/MvInfosTableValuedFunction.java | 1 - .../doris/transaction/PublishVersionDaemon.java | 15 +- .../doris/analysis/CancelExportStmtTest.java | 4 +- .../apache/doris/analysis/CancelLoadStmtTest.java | 4 +- .../org/apache/doris/analysis/SelectStmtTest.java | 28 +- .../doris/analysis/ShowBuildIndexStmtTest.java | 8 +- .../org/apache/doris/analysis/SqlModeTest.java | 2 +- .../org/apache/doris/catalog/RefreshDbTest.java | 8 +- .../org/apache/doris/catalog/RefreshTableTest.java | 6 +- .../apache/doris/datasource/CatalogMgrTest.java | 5 +- .../doris/datasource/RefreshCatalogTest.java | 11 - .../apache/doris/mtmv/MTMVPartitionUtilTest.java | 35 +- .../org/apache/doris/mtmv/MTMVRewriteUtilTest.java | 10 +- .../java/org/apache/doris/mtmv/MTMVTaskTest.java | 24 +- .../test/java/org/apache/doris/mtmv/MTMVTest.java | 2 - .../doris/nereids/parser/NereidsParserTest.java | 205 ++ .../properties/ChildOutputPropertyDeriverTest.java | 36 +- .../exploration/mv/MaterializedViewUtilsTest.java | 20 +- ...ctSingleTableExpressionFromDisjunctionTest.java | 36 + .../rules/rewrite/mv/SelectMvIndexTest.java | 2 +- .../org/apache/doris/planner/QueryPlanTest.java | 23 +- .../java/org/apache/doris/policy/PolicyTest.java | 10 +- .../org/apache/doris/qe/OlapQueryCacheTest.java | 18 +- .../ExtractCommonFactorsRuleFunctionTest.java | 6 +- fe/pom.xml | 6 + .../doris/broker/hdfs/FileSystemManager.java | 4 + gensrc/proto/olap_common.proto | 12 +- gensrc/proto/olap_file.proto | 6 + gensrc/script/doris_builtins_functions.py | 2 +- gensrc/thrift/Descriptors.thrift | 4 +- gensrc/thrift/FrontendService.thrift | 15 + regression-test/conf/regression-conf.groovy | 1 + .../test_bloom_filter_hit.out} | 10 +- .../correctness/test_time_diff_microseconds.out | 3 + .../ddl/create_nestedtypes_with_schemachange.out | 46 + .../get_assignment_compatible_type.out | 2 +- .../sql/presto/scalar/TestJsonFunctions.out | 37 +- .../iceberg/iceberg_schema_change.out | 1016 ++++++-- .../jdbc/test_mariadb_jdbc_catalog.out | 2 + .../jdbc/test_mysql_jdbc_catalog.out | 30 +- .../jdbc/test_mysql_jdbc_catalog_nereids.out | 2 + .../jdbc/test_mysql_jdbc_driver5_catalog.out | 2 + .../jdbc/test_oceanbase_jdbc_catalog.out | 7 + .../jdbc/test_oracle_jdbc_catalog.out | 4 + .../mc/test_max_compute_complex_type.out | 17 + .../insert_p0/txn_insert_with_specify_columns.out | 21 + .../array_contains/test_index_change_2_arr.out | 1 - .../test_arrayInvertedIdx_profile.out | 55 + ...ut => test_compound_reader_fault_injection.out} | 9 +- .../test_inverted_index_file_size.out | 49 + .../test_inverted_index_v2_file_size.out | 85 + .../need_read_data/documents-1000.json | 1000 ++++++++ .../load_p0/insert/test_insert_default_value.out | 15 +- .../test_json_extract_path_invalid_type.json | 13 + .../data/load_p0/stream_load/test_json_load.out | 6 +- .../data/mtmv_p0/test_expand_star_mtmv.out | 7 + .../data/mtmv_p0/test_rollup_partition_mtmv.out | 10 + regression-test/data/mv_p0/k1s2m3/k1s2m3.out | 27 +- regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out | 383 ++++ .../data/mv_p0/test_mv_mor/test_mv_mor.out | 4 +- .../test_row_store/test_row_store.out} | 9 +- .../nereids_function_p0/agg_function/test_corr.out | 21 + .../data/nereids_function_p0/type_coercion.out | 64 + .../aggregate/aggregate_stddev_over_range.out} | 7 +- .../data/nereids_p0/aggregate/window_funnel.out | 99 + .../fold_constant/fold_constant_by_be.out | 15 - .../data/nereids_p0/hint/test_leading.out | 4 +- .../window_functions/window_funnel.out | 5 + regression-test/data/nereids_p0/test_user_var.out | 3 + .../push_down_count_through_join.out | 8 +- .../agg_optimize_when_uniform.out | 4 +- .../extract_from_disjunction_in_join.out | 94 + .../push_down_filter_through_window.out | 0 .../shape/query24.out | 4 +- .../nereids_tpcds_shape_sf100_p0/shape/query24.out | 4 +- .../new_shapes_p0/hint_tpcds/shape/query64.out | 28 +- .../new_shapes_p0/tpcds_sf100/shape/query24.out | 4 +- .../new_shapes_p0/tpcds_sf1000/shape/query24.out | 4 +- .../test_string_function_regexp.out | 12 + .../data/query_p0/system/test_table_options.out | 34 +- .../data/query_p0/system/test_table_properties.out | 206 ++ .../schema_change_p0/test_agg_schema_value_add.out | 11 + regression-test/data/statistics/delete | 2 + regression-test/data/statistics/load | 5 + .../test_partial_update_insert_seq_col.out | Bin 1416 -> 1123 bytes .../partial_update/test_partial_update_seq_col.out | Bin 1415 -> 1123 bytes .../test_partial_update_seq_col_delete.out | Bin 1530 -> 1047 bytes .../test_partial_update_seq_type.out | Bin 3363 -> 2185 bytes .../test_partial_update_seq_type_delete.out | Bin 2967 -> 1789 bytes .../data/variant_p0/variant_with_rowstore.out | 3 + .../data/workload_manager_p0/test_curd_wlg.out | 8 +- .../doris/regression/action/ExplainAction.groovy | 3 - .../org/apache/doris/regression/suite/Suite.groovy | 61 +- .../cloud_p0/conf/regression-conf-custom.groovy | 2 +- .../pipeline/external/conf/regression-conf.groovy | 4 + .../pipeline/p0/conf/regression-conf.groovy | 2 +- .../pipeline/p1/conf/regression-conf.groovy | 2 +- .../alter_p0/test_alter_add_multi_partition.groovy | 110 + .../backup_restore/test_backup_cancelled.groovy | 4 +- .../bloom_filter_p0/test_bloom_filter_hit.groovy | 78 + .../correctness/test_time_diff_microseconds.groovy | 8 + .../unique/test_unique_table_new_sequence.groovy | 2 + .../unique/test_unique_table_sequence.groovy | 3 + .../create_nestedtypes_with_schemachange.groovy | 65 + .../sql/presto/scalar/TestJsonFunctions.sql | 2 +- .../external_table_p0/es/test_es_query.groovy | 25 + .../iceberg/iceberg_schema_change.groovy | 59 +- .../iceberg/test_iceberg_show_create.groovy | 73 + .../jdbc/test_clickhouse_jdbc_catalog.groovy | 2 +- .../jdbc/test_mysql_jdbc_catalog.groovy | 237 +- .../jdbc/test_oceanbase_jdbc_catalog.groovy | 45 + .../jdbc/test_oracle_jdbc_catalog.groovy | 16 + .../mc/test_max_compute_complex_type.groovy | 48 + .../test_index_bkd_writer_fault_injection.groovy | 51 + .../test_index_writer_file_cache.groovy | 116 + .../group_commit/test_group_commit_error.groovy | 79 + .../insert_p0/insert_group_commit_into.groovy | 3 + .../txn_insert_with_specify_columns.groovy | 122 + .../test_add_drop_index_with_data_arr.groovy | 4 +- ...test_add_drop_index_with_delete_data_arr.groovy | 4 +- .../array_contains/test_char_replace_arr.groovy | 6 +- .../test_count_on_index_httplogs_arr.groovy | 4 +- .../test_equal_on_fulltext_arr.groovy | 2 +- .../array_contains/test_index_change_2_arr.groovy | 4 +- .../array_contains/test_index_change_4_arr.groovy | 4 +- .../array_contains/test_index_change_6_arr.groovy | 4 +- .../test_index_compaction_dup_keys_arr.groovy | 2 +- .../test_index_compaction_null_arr.groovy | 4 +- ...compaction_with_multi_index_segments_arr.groovy | 4 +- .../array_contains/test_index_delete_arr.groovy | 2 +- .../test_index_empty_string_arr.groovy | 2 +- .../test_index_equal_select_arr.groovy | 8 +- .../test_index_match_phrase_edge_arr.groovy | 4 +- .../test_index_match_regexp_arr.groovy | 2 +- .../array_contains/test_index_null_arr.groovy | 2 +- .../array_contains/test_invalid_param.groovy | 4 +- .../test_inverted_index_null_arr.groovy | 2 +- .../array_contains/test_lowercase_arr.groovy | 18 +- .../test_parser_with_none_stopwords.groovy | 4 +- .../test_agg_table_with_inverted_index.groovy | 45 - .../test_arrayInvertedIdx_profile.groovy | 128 ++ .../test_array_index_parser.groovy | 55 + .../test_array_index_write.groovy | 84 + .../test_array_with_inverted_index_all_type.groovy | 18 +- .../test_compound_reader_fault_injection.groovy | 62 + .../inverted_index_p0/test_count_on_index.groovy | 7 +- .../test_index_key_match_select.groovy | 2 +- .../test_index_lowercase_fault_injection.groovy | 4 + .../test_index_multi_match.groovy | 16 +- .../inverted_index_p0/test_index_null.groovy | 2 +- .../test_inverted_index_file_size.groovy | 145 ++ ...ovy => test_inverted_index_v2_file_size.groovy} | 150 +- .../test_mor_table_with_inverted_index.groovy | 45 - .../inverted_index_p0/topn_clear_block.groovy | 10 + .../test_dup_table_inverted_index.groovy | 904 ++++++++ .../insert/test_insert_default_value.groovy | 48 +- .../load_p0/stream_load/test_json_load.groovy | 27 + .../stream_load/test_stream_load_2pc.groovy | 2 +- .../suites/load_p2/test_single_replica_load.groovy | 67 + .../suites/mtmv_p0/test_env_db_dropped_mtmv.groovy | 2 +- .../suites/mtmv_p0/test_expand_star_mtmv.groovy | 82 + .../suites/mtmv_p0/test_hive_refresh_mtmv.groovy | 13 +- .../mtmv_p0/test_rollup_partition_mtmv.groovy | 96 + .../suites/mtmv_p0/test_show_create_mtmv.groovy | 2 +- regression-test/suites/mv_p0/k1s2m3/k1s2m3.groovy | 88 +- .../mv_p0/partition_prune/partition_prune.groovy | 448 ++-- .../ssb/multiple_no_where/multiple_no_where.groovy | 178 +- .../mv_p0/ssb/multiple_ssb/multiple_ssb.groovy | 11 +- .../multiple_ssb_between.groovy | 104 +- .../suites/mv_p0/ssb/q_1_1/q_1_1.groovy | 37 +- .../suites/mv_p0/ssb/q_2_1/q_2_1.groovy | 22 +- .../suites/mv_p0/ssb/q_3_1/q_3_1.groovy | 40 +- .../suites/mv_p0/ssb/q_4_1/q_4_1.groovy | 24 +- .../suites/mv_p0/ssb/q_4_1_r1/q_4_1_r1.groovy | 8 - .../suites/mv_p0/sum_count/sum_count.groovy | 68 +- .../suites/mv_p0/test_mv_mor/test_mv_mor.groovy | 2 +- .../mv_p0/test_row_store/test_row_store.groovy | 19 + regression-test/suites/mv_p0/unique/unique.groovy | 8 +- .../testAggQueryOnAggMV1.groovy | 77 +- .../testSelectMVWithTableAlias.groovy | 24 +- .../mv_p0/where/k123_nereids/k123_nereids.groovy | 142 +- .../ddl/articles_dk_array.sql | 4 +- .../ddl/articles_uk_array.sql | 4 +- .../ddl/fulltext_t1_dk_array.sql | 2 +- .../ddl/fulltext_t1_uk_array.sql | 2 +- .../ddl/join_t2_dk_array.sql | 2 +- .../ddl/join_t2_uk_array.sql | 2 +- .../ddl/large_records_t1_dk_array.sql | 2 +- .../ddl/large_records_t1_uk_array.sql | 2 +- .../ddl/large_records_t2_dk_array.sql | 2 +- .../ddl/large_records_t2_uk_array.sql | 2 +- .../ddl/large_records_t3_dk_array.sql | 2 +- .../ddl/large_records_t3_uk_array.sql | 2 +- .../ddl/large_records_t4_dk_array.sql | 2 +- .../ddl/large_records_t4_uk_array.sql | 2 +- .../agg_function/test_corr.groovy | 11 +- .../nereids_function_p0/gen_function/gen.groovy | 2 +- .../nereids_function_p0/type_coercion.groovy | 59 + .../aggregate/aggregate_stddev_over_range.groovy | 90 + .../nereids_p0/aggregate/window_funnel.groovy | 361 +++ .../suites/nereids_p0/delete/delete_cte.groovy | 3 + .../fold_constant/fold_constant_by_be.groovy | 8 +- .../suites/nereids_p0/join/test_join.groovy | 1 - .../suites/nereids_p0/join/test_join_14.groovy | 6 +- .../suites/nereids_p0/join/test_join_15.groovy | 7 +- .../window_functions/window_funnel.sql | 71 +- .../suites/nereids_p0/system/test_query_sys.groovy | 21 + .../suites/nereids_p0/test_user_var.groovy | 2 + .../push_down_count_through_join.groovy | 11 +- .../push_down_count_through_join_one_side.groovy | 4 +- .../push_down_max_through_join.groovy | 2 + .../push_down_min_through_join.groovy | 3 +- .../push_down_sum_through_join.groovy | 3 +- .../push_down_sum_through_join_one_side.groovy | 3 +- .../infer_set_operator_distinct.groovy | 6 +- .../mv/agg_on_none_agg/agg_on_none_agg.groovy | 26 +- .../agg_optimize_when_uniform.groovy | 23 +- .../mv/agg_variety/agg_variety.groovy | 20 +- .../agg_with_roll_up/aggregate_with_roll_up.groovy | 78 +- .../aggregate_without_roll_up.groovy | 66 +- .../availability/materialized_view_switch.groovy | 19 +- .../mv/grouping_sets/grouping_sets.groovy | 28 +- .../mv/join/dphyp_inner/inner_join_dphyp.groovy | 40 +- .../mv/join/dphyp_outer/outer_join_dphyp.groovy | 38 +- .../mv/join/inner/inner_join.groovy | 40 +- .../mv/join/left_outer/outer_join.groovy | 42 +- .../mv/nested/nested_materialized_view.groovy | 4 +- .../mv/partition_mv_rewrite.groovy | 109 +- .../nereids_rules_p0/mv/scan/scan_table.groovy | 8 +- .../nereids_rules_p0/mv/ssb/mv_ssb_test.groovy | 26 +- .../nereids_rules_p0/mv/tpch/mv_tpch_test.groovy | 44 +- .../nereids_rules_p0/mv/variant/variant_mv.groovy | 34 +- .../partition_prune/test_condition_function.groovy | 51 + .../extract_from_disjunction_in_join.groovy | 83 + .../push_down_filter_through_window.groovy | 0 .../suites/nereids_syntax_p0/cast.groovy | 15 + .../distribute/distribution_expr.groovy | 130 ++ .../mv/newMv/aggHaveDupBase.groovy | 52 +- .../nereids_syntax_p0/mv/ut/aggOnAggMV5.groovy | 26 +- .../nereids_syntax_p0/mv/ut/distinctQuery.groovy | 26 +- .../suites/nereids_syntax_p0/rollup/date.groovy | 53 +- .../test_semicolon_at_begin.groovy | 17 +- .../test_auto_partition_behavior.groovy | 20 + .../suites/query_p0/cache/sql_cache.groovy | 7 + .../explain/test_compoundpredicate_explain.groovy | 72 + .../suites/query_p0/join/test_join.groovy | 1 - .../test_string_function_regexp.groovy | 5 + .../table_function/explode_json_array.groovy | 2 +- .../query_p0/system/test_table_options.groovy | 50 +- .../query_p0/system/test_table_properties.groovy | 124 + .../test_agg_schema_key_add.groovy | 518 +++++ .../test_agg_schema_key_change_modify.groovy | 1111 +++++++++ .../test_agg_schema_key_change_modify1.groovy | 2422 ++++++++++++++++++++ .../test_agg_schema_key_drop.groovy | 608 +++++ .../test_agg_schema_value_add.groovy | 655 ++++++ .../test_agg_schema_value_drop.groovy | 1069 +++++++++ .../test_dup_schema_value_modify3.groovy | 170 +- .../test_dup_schema_value_modify4.groovy | 178 +- .../test_schema_change_unique_mow.groovy | 8 +- .../test_uniq_mv_schema_change.groovy | 4 +- .../suites/statistics/test_analyze_mv.groovy | 11 +- .../statistics/test_delete_data_analyze.groovy | 132 ++ .../statistics/test_drop_cached_stats.groovy | 58 + .../suites/tpcds_sf1_unique_p1/spill/q32.groovy | 1 - .../suites/tpcds_sf1_unique_p1/sql/q24_1.sql | 92 +- .../suites/tpcds_sf1_unique_p1/sql/q24_2.sql | 93 +- .../suites/tpcds_sf1_unique_p1/sql/q32.sql | 1 - .../suites/tpcds_sf1_unique_p1/sql/q92.sql | 1 - .../suites/tpch_sf0.1_unique_p1/sql/q02.sql | 1 - .../tpch_unique_sql_zstd_bucket1_p0/sql/q02.sql | 2 +- .../test_partial_update_insert_seq_col.groovy | 7 +- .../test_partial_update_seq_col.groovy | 5 +- .../test_partial_update_seq_col_delete.groovy | 10 +- .../test_partial_update_seq_type.groovy | 20 +- .../test_partial_update_seq_type_delete.groovy | 20 +- .../suites/variant_p0/variant_with_rowstore.groovy | 18 + .../workload_manager_p0/test_curd_wlg.groovy | 7 + thirdparty/CHANGELOG.md | 8 + thirdparty/vars.sh | 8 +- tools/tpcds-tools/README.md | 10 +- tools/tpcds-tools/bin/run-tpcds-queries.sh | 8 +- .../tpcds-tools/ddl/create-tpcds-tables-sf1000.sql | 24 +- .../ddl/create-tpcds-tables-sf10000.sql | 24 +- 746 files changed, 22726 insertions(+), 5789 deletions(-) create mode 100644 be/src/common/cgroup_memory_ctl.cpp create mode 100644 be/src/common/cgroup_memory_ctl.h copy be/src/exec/schema_scanner/{schema_table_options_scanner.cpp => schema_table_properties_scanner.cpp} (58%) copy be/src/exec/schema_scanner/{schema_table_options_scanner.h => schema_table_properties_scanner.h} (74%) copy be/src/exec/schema_scanner/{schema_backend_active_tasks.cpp => schema_workload_group_resource_usage_scanner.cpp} (50%) copy be/src/exec/schema_scanner/{schema_table_options_scanner.h => schema_workload_group_resource_usage_scanner.h} (78%) create mode 100644 be/test/util/test_data/memory.limit_in_bytes create mode 100644 be/test/util/test_data/memory.stat create mode 100755 docker/thirdparties/docker-compose/elasticsearch/scripts/data/composite_type_array_bulk.json create mode 100644 docker/thirdparties/docker-compose/elasticsearch/scripts/index/array_meta_composite_type_array.json create mode 100755 docker/thirdparties/docker-compose/elasticsearch/scripts/index/es6_composite_type_array.json create mode 100644 docker/thirdparties/docker-compose/elasticsearch/scripts/index/es7_composite_type_array.json mode change 100755 => 100644 docker/thirdparties/docker-compose/iceberg/entrypoint.sh.tpl create mode 100644 docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/run01.sql create mode 100644 docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/run02.sql create mode 100644 docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/run03.sql create mode 100644 docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/run04.sql rename docker/thirdparties/docker-compose/iceberg/{spark-init-iceberg.sql => scripts/create_preinstalled_scripts/run05.sql} (100%) delete mode 100644 docker/thirdparties/docker-compose/iceberg/spark-init-paimon.sql copy docker/thirdparties/docker-compose/{mariadb => oceanbase}/init/01-drop-db.sql (100%) copy docker/thirdparties/docker-compose/{mariadb => oceanbase}/init/02-create-db.sql (100%) create mode 100644 docker/thirdparties/docker-compose/oceanbase/init/03-create-table.sql create mode 100644 docker/thirdparties/docker-compose/oceanbase/init/04-insert.sql copy .github/workflows/labeler/scope-label-conf.yml => docker/thirdparties/docker-compose/oceanbase/oceanbase.env (68%) create mode 100644 docker/thirdparties/docker-compose/oceanbase/oceanbase.yaml.tpl create mode 100644 fe/fe-core/src/main/java/org/apache/doris/analysis/AlterMultiPartitionClause.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/analysis/DropCachedStatsStmt.java rename fe/fe-core/src/main/java/org/apache/doris/mtmv/{EnvInfo.java => MTMVBaseVersions.java} (59%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshContext.java copy fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/{Pmod.java => RegexpExtractOrNull.java} (59%) copy regression-test/data/{nereids_function_p0/agg_function/test_corr.out => bloom_filter_p0/test_bloom_filter_hit.out} (51%) create mode 100644 regression-test/data/datatype_p0/nested_types/ddl/create_nestedtypes_with_schemachange.out create mode 100644 regression-test/data/external_table_p0/jdbc/test_oceanbase_jdbc_catalog.out create mode 100644 regression-test/data/external_table_p0/mc/test_max_compute_complex_type.out create mode 100644 regression-test/data/insert_p0/txn_insert_with_specify_columns.out create mode 100644 regression-test/data/inverted_index_p0/test_arrayInvertedIdx_profile.out rename regression-test/data/inverted_index_p0/{test_mor_table_with_inverted_index.out => test_compound_reader_fault_injection.out} (65%) create mode 100644 regression-test/data/inverted_index_p0/test_inverted_index_file_size.out create mode 100644 regression-test/data/inverted_index_p0/test_inverted_index_v2_file_size.out create mode 100644 regression-test/data/inverted_index_p1/need_read_data/documents-1000.json create mode 100644 regression-test/data/load_p0/stream_load/test_json_extract_path_invalid_type.json create mode 100644 regression-test/data/mtmv_p0/test_expand_star_mtmv.out copy regression-test/data/{inverted_index_p0/test_agg_table_with_inverted_index.out => mv_p0/test_row_store/test_row_store.out} (65%) create mode 100644 regression-test/data/nereids_function_p0/type_coercion.out rename regression-test/data/{inverted_index_p0/test_agg_table_with_inverted_index.out => nereids_p0/aggregate/aggregate_stddev_over_range.out} (80%) create mode 100644 regression-test/data/nereids_rules_p0/push_down_filter/extract_from_disjunction_in_join.out rename regression-test/data/nereids_rules_p0/{push_down_filter_through_window => push_down_filter}/push_down_filter_through_window.out (100%) create mode 100644 regression-test/data/query_p0/system/test_table_properties.out create mode 100644 regression-test/data/schema_change_p0/test_agg_schema_value_add.out create mode 100644 regression-test/data/statistics/delete create mode 100644 regression-test/data/statistics/load create mode 100644 regression-test/suites/alter_p0/test_alter_add_multi_partition.groovy create mode 100644 regression-test/suites/datatype_p0/nested_types/ddl/create_nestedtypes_with_schemachange.groovy create mode 100644 regression-test/suites/external_table_p0/iceberg/test_iceberg_show_create.groovy create mode 100644 regression-test/suites/external_table_p0/jdbc/test_oceanbase_jdbc_catalog.groovy create mode 100644 regression-test/suites/external_table_p0/mc/test_max_compute_complex_type.groovy create mode 100644 regression-test/suites/fault_injection_p0/test_index_bkd_writer_fault_injection.groovy create mode 100644 regression-test/suites/fault_injection_p0/test_index_writer_file_cache.groovy create mode 100644 regression-test/suites/insert_p0/group_commit/test_group_commit_error.groovy create mode 100644 regression-test/suites/insert_p0/txn_insert_with_specify_columns.groovy delete mode 100644 regression-test/suites/inverted_index_p0/test_agg_table_with_inverted_index.groovy create mode 100644 regression-test/suites/inverted_index_p0/test_arrayInvertedIdx_profile.groovy create mode 100644 regression-test/suites/inverted_index_p0/test_array_inverted_index/test_array_index_parser.groovy create mode 100644 regression-test/suites/inverted_index_p0/test_array_inverted_index/test_array_index_write.groovy create mode 100644 regression-test/suites/inverted_index_p0/test_compound_reader_fault_injection.groovy create mode 100644 regression-test/suites/inverted_index_p0/test_inverted_index_file_size.groovy copy regression-test/suites/inverted_index_p0/{array_contains/test_index_compaction_dup_keys_arr.groovy => test_inverted_index_v2_file_size.groovy} (53%) delete mode 100644 regression-test/suites/inverted_index_p0/test_mor_table_with_inverted_index.groovy create mode 100644 regression-test/suites/inverted_index_p1/need_read_data/test_dup_table_inverted_index.groovy create mode 100644 regression-test/suites/load_p2/test_single_replica_load.groovy create mode 100644 regression-test/suites/mtmv_p0/test_expand_star_mtmv.groovy create mode 100644 regression-test/suites/nereids_function_p0/type_coercion.groovy create mode 100644 regression-test/suites/nereids_p0/aggregate/aggregate_stddev_over_range.groovy create mode 100644 regression-test/suites/nereids_rules_p0/partition_prune/test_condition_function.groovy create mode 100644 regression-test/suites/nereids_rules_p0/push_down_filter/extract_from_disjunction_in_join.groovy rename regression-test/suites/nereids_rules_p0/{push_down_filter_through_window => push_down_filter}/push_down_filter_through_window.groovy (100%) create mode 100644 regression-test/suites/nereids_syntax_p0/distribute/distribution_expr.groovy copy be/src/runtime/frontend_info.h => regression-test/suites/nereids_syntax_p0/test_semicolon_at_begin.groovy (76%) create mode 100644 regression-test/suites/query_p0/explain/test_compoundpredicate_explain.groovy create mode 100644 regression-test/suites/query_p0/system/test_table_properties.groovy create mode 100644 regression-test/suites/schema_change_p0/test_agg_schema_key_add.groovy create mode 100644 regression-test/suites/schema_change_p0/test_agg_schema_key_change_modify.groovy create mode 100644 regression-test/suites/schema_change_p0/test_agg_schema_key_change_modify1.groovy create mode 100644 regression-test/suites/schema_change_p0/test_agg_schema_key_drop.groovy create mode 100644 regression-test/suites/schema_change_p0/test_agg_schema_value_add.groovy create mode 100644 regression-test/suites/schema_change_p0/test_agg_schema_value_drop.groovy create mode 100644 regression-test/suites/statistics/test_delete_data_analyze.groovy create mode 100644 regression-test/suites/statistics/test_drop_cached_stats.groovy --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org