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

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


    from feaf30d5203 [feature](shuffle) enable strict consistency dml by 
default (#32958)
     add 25193198afa [fix](spill) incorrect result of hash join (#34450)
     add 87a9b7aac67 [bug](ipv6) the ipv6 type should be uint128_t (#34121)
     add 836709efc70 Create partition statistics table. (#34305)
     add 5202e8ad55f  [fix](tracing) Fix wrong global variable init order of 
tracing log dir (#34454)
     add 42bec050f90 (cloud-merge) Enables concurrent file downloads (#34495)
     add 9310b8391c7 [pipeline](fix) Prevent re-cancel pipeline tasks (#34462)
     add bed3d005d85 [test](ES Catalog) Add test cases for ES 5.x (#34441)
     add 07267faac0d [refactor](Nereids)refactor runtime filter generator 
(#34275)
     add 4303f57cb25 [opt](inverted index) opt for log output when matching 
without an index (#34024)
     add 6fde37e19f2 [fix](mtmv)fix refresh failed when not use db before 
create MTMV (#34431)
     add 77f187d9b47 [Refactor](exec) refactor the pipline code (#34402)
     add e8cd5e1a3e5 [bugfix](compile error) unused variables compile error 
(#34509)
     add c1666291cb8 [chore](log) add try lock db failed log (#34503)
     add 12a548f2fbe [fix](tablet schedule) disable schedule need clear running 
tasks (#34461)
     add ebf2b88f039 [optimization](agg) add more type in agg percentile 
(#34423)
     add f1b6bc4da99 [fix](merge-on-write) remove some CHECKs in 
Tablet::revise_tablet_meta (#31268)
     add adad523d7b3 [enhancement](regression-test) modify a key type from 
BIGINT/LARGEINT to other type (#34436)
     add a22b4f4d379 [WIP](test) remove enable_nereids_planner in regression 
cases (part 2) (#34490)
     add d20b18f222e [test](fix) Fix test check fail when test nested mv hit 
(#34293)
     add 32db91cf784 [fix](compile) fix macOS compile (#34478)
     add 65e4f08ae51 [Fix](nereids) fix NormalizeRepeat, change the 
outputExpression rewrite logic (#34196)
     add 3713bb9f16f [opt](stats) Remove collect external table row count task 
(#34487)
     add ba460b1cbf1 [Fix](job)Reclaim resources held by finished tasks (#34506)
     add 6f4738c955e [improve](restore) Release useless info for the finished 
job to reduce mem usage (#34513)
     add 6849cd521a6 [pipeline](fix) Logging blocking dependency if task is not 
cancelled (#34502)
     add 99a6dc00a0b [Improve](cache) Estimated column reader memory to control 
segment cache (#34526)
     add b78830d5732 [config](move-memtable) support HTTP header 
load_stream_per_node in stream load (#34444)
     add 28dbbe605a8 [fix](local exchange) fix bug oflocal exchange free blocks 
(#34530)
     add e7a58ee97bc [fix](txn insert) Txn insert stuck if execute on observer 
fe (#34547)
     add a16dbdfa392 [bugfix](gccompile) fix gcc compile error (#34546)
     add e4c7abf720c [localexchange](conf) Add a variable to limit free blocks 
memory (#34536)
     add 04390fb1317 [fix](Nereids) array_intersect should be a variadic 
function (#34543)
     add 3fc12915249 [opt](tools) update bucket number in schema file (#34561)
     add c4722069a06 [Fix](planner) fix 
ScalarType.getAssignmentCompatibleType() when deal boolean and decimal (#34435)
     add f452e95d165 [Feature](PreparedStatement) implement general server side 
prepared (#33807)
     add 84623fbbcf3 [refactor](type) AggStateType should not extends 
ScalarType (#34463)
     add 6802932720f [fix](catalog) refactor the schema cache for external 
table (#34517)
     add cf889571ee8 [WIP](test) remove enable_nereids_planner in regression 
cases (part 3) (#34558)
     add 365afc754c6 Collect partition stats (#34552)
     add b313ee04343 [improve](shcema change)fix alter table faild when  modify 
multiple columns with column changed positions (#34244)
     add 66de110fe46 [Bug](Variant) fix rapidjson::Allocator may cause mem 
allocate issue when build with `DENABLE_CLANG_COVERAGE`  (#34150)
     add 57eab889b19 [bug](config) Fix modifying label_num_threshold does not 
take effect (#34575)

No new revisions were added by this update.

Summary of changes:
 be/src/cloud/cloud_warm_up_manager.cpp             |  16 +-
 be/src/cloud/cloud_warm_up_manager.h               |   1 +
 be/src/common/config.cpp                           |   4 +
 be/src/common/config.h                             |   3 +
 be/src/common/status.h                             |   9 +-
 be/src/http/action/stream_load.cpp                 |   4 +
 be/src/http/http_common.h                          |   1 +
 be/src/io/cache/block_file_cache_downloader.cpp    |   6 +-
 be/src/olap/base_tablet.cpp                        |  31 +-
 be/src/olap/base_tablet.h                          |   5 +-
 be/src/olap/rowset/segment_v2/segment.cpp          |   1 +
 be/src/olap/rowset/segment_v2/segment.h            |   6 +-
 be/src/olap/segment_loader.cpp                     |   5 +-
 be/src/olap/segment_loader.h                       |   2 +-
 be/src/olap/tablet.cpp                             |  85 ++-
 be/src/olap/types.h                                |   2 +-
 be/src/pipeline/exec/operator.cpp                  |   6 +-
 be/src/pipeline/exec/operator.h                    |  10 +-
 .../exec/partitioned_hash_join_probe_operator.cpp  |  63 +-
 .../exec/partitioned_hash_join_sink_operator.cpp   |   6 +-
 be/src/pipeline/exec/result_sink_operator.cpp      |   9 +-
 be/src/pipeline/local_exchange/local_exchanger.cpp |  22 +-
 be/src/pipeline/local_exchange/local_exchanger.h   |  49 +-
 be/src/pipeline/pipeline_fragment_context.cpp      |  48 +-
 be/src/pipeline/pipeline_task.cpp                  |  22 +-
 be/src/pipeline/pipeline_task.h                    |  14 +-
 be/src/pipeline/pipeline_tracing.cpp               |   6 +-
 be/src/pipeline/pipeline_tracing.h                 |   5 +-
 be/src/pipeline/task_queue.cpp                     |   9 +-
 be/src/pipeline/task_queue.h                       |   1 -
 be/src/pipeline/task_scheduler.cpp                 |  45 +-
 be/src/pipeline/task_scheduler.h                   |   6 +-
 be/src/runtime/exec_env_init.cpp                   |  13 +-
 be/src/runtime/runtime_state.h                     |   5 +
 be/src/service/doris_main.cpp                      |   2 +-
 be/src/service/point_query_executor.cpp            |   1 -
 be/src/util/counts.h                               |  19 +-
 .../aggregate_function_percentile.cpp              |   8 +-
 .../aggregate_function_percentile.h                |  35 +-
 be/src/vec/columns/column_object.cpp               |  19 +-
 be/src/vec/core/block.cpp                          |   1 +
 .../vec/data_types/serde/data_type_array_serde.cpp |   5 +-
 .../vec/data_types/serde/data_type_array_serde.h   |   2 +-
 .../vec/data_types/serde/data_type_jsonb_serde.cpp |   2 +-
 .../vec/data_types/serde/data_type_jsonb_serde.h   |   2 +-
 .../data_types/serde/data_type_nullable_serde.cpp  |   6 +-
 .../data_types/serde/data_type_nullable_serde.h    |   2 +-
 .../vec/data_types/serde/data_type_number_serde.h  |   4 +-
 be/src/vec/data_types/serde/data_type_serde.cpp    |   2 +-
 be/src/vec/data_types/serde/data_type_serde.h      |   2 +-
 .../vec/data_types/serde/data_type_string_serde.h  |   2 +-
 be/src/vec/functions/match.cpp                     |   8 +-
 be/src/vec/sink/vresult_sink.cpp                   |  12 +-
 be/test/util/counts_test.cpp                       |  12 +-
 .../docker-compose/elasticsearch/es.env            |   1 +
 .../docker-compose/elasticsearch/es.yaml.tpl       |  26 +
 .../elasticsearch/scripts/data/data3_es5.json      |  28 +
 .../elasticsearch/scripts/es_init.sh               |  21 +
 .../org/apache/doris/catalog/AggStateType.java     |  49 +-
 .../java/org/apache/doris/catalog/ArrayType.java   |  12 +-
 .../java/org/apache/doris/catalog/MapType.java     |   5 +
 .../java/org/apache/doris/catalog/ScalarType.java  |  30 +-
 .../java/org/apache/doris/catalog/StructType.java  |  13 +-
 .../main/java/org/apache/doris/catalog/Type.java   |  43 +-
 .../java/org/apache/doris/catalog/VariantType.java |  46 +-
 .../main/java/org/apache/doris/common/Config.java  |   6 +-
 fe/fe-core/src/main/cup/sql_parser.cup             |   8 +-
 .../apache/doris/alter/SchemaChangeHandler.java    |   4 +-
 .../java/org/apache/doris/analysis/Analyzer.java   |  16 +-
 .../org/apache/doris/analysis/BinaryPredicate.java |   8 +-
 .../main/java/org/apache/doris/analysis/Expr.java  |   6 +-
 .../org/apache/doris/analysis/LiteralExpr.java     |  16 +-
 .../apache/doris/analysis/NativeInsertStmt.java    |  13 +-
 .../org/apache/doris/analysis/PlaceHolderExpr.java |  16 +-
 .../org/apache/doris/analysis/PrepareStmt.java     | 124 ++--
 .../org/apache/doris/analysis/StatementBase.java   |  15 +-
 .../org/apache/doris/analysis/StringLiteral.java   |   1 +
 .../org/apache/doris/backup/BackupJobInfo.java     |  14 +
 .../apache/doris/backup/RestoreFileMapping.java    |   5 +
 .../java/org/apache/doris/backup/RestoreJob.java   |   5 +
 .../java/org/apache/doris/catalog/Database.java    |  17 +-
 .../org/apache/doris/catalog/InternalSchema.java   |  74 ++-
 .../doris/catalog/InternalSchemaInitializer.java   |  66 +-
 .../java/org/apache/doris/catalog/OlapTable.java   |  11 +-
 .../main/java/org/apache/doris/catalog/Table.java  |   6 -
 .../java/org/apache/doris/catalog/TableIf.java     |   4 +
 .../org/apache/doris/clone/TabletScheduler.java    |  26 +-
 .../apache/doris/datasource/ExternalCatalog.java   |   9 +-
 .../apache/doris/datasource/ExternalDatabase.java  |   2 +-
 .../doris/datasource/ExternalSchemaCache.java      |  12 +-
 .../org/apache/doris/datasource/ExternalTable.java |  13 +-
 .../apache/doris/datasource/SchemaCacheValue.java  |  30 +-
 .../doris/datasource/es/EsExternalTable.java       |   8 +-
 .../doris/datasource/hive/HMSExternalTable.java    |  37 +-
 .../doris/datasource/hive/HMSSchemaCacheValue.java |  33 +-
 .../datasource/iceberg/IcebergExternalTable.java   |   6 +-
 .../infoschema/ExternalInfoSchemaTable.java        |   9 +-
 .../datasource/infoschema/ExternalMysqlTable.java  |   8 +-
 .../doris/datasource/jdbc/JdbcExternalTable.java   |   7 +-
 .../maxcompute/MaxComputeExternalTable.java        |  58 +-
 .../maxcompute/MaxComputeSchemaCacheValue.java     |  57 ++
 .../doris/datasource/metacache/MetaCache.java      |  18 +-
 .../datasource/paimon/PaimonExternalTable.java     |  29 +-
 .../datasource/paimon/PaimonSchemaCacheValue.java  |  31 +-
 .../datasource/paimon/source/PaimonSource.java     |   2 +-
 .../doris/datasource/test/TestExternalTable.java   |   8 +-
 .../TrinoConnectorExternalTable.java               |  33 +-
 .../trinoconnector/TrinoSchemaCacheValue.java      |  48 ++
 .../doris/job/extensions/insert/InsertTask.java    |  15 +-
 .../apache/doris/job/extensions/mtmv/MTMVTask.java |  40 +-
 .../org/apache/doris/job/task/AbstractTask.java    |  40 +-
 .../java/org/apache/doris/mtmv/MTMVPlanUtil.java   |  43 +-
 .../org/apache/doris/mysql/AcceptListener.java     |   4 +-
 .../glue/translator/PhysicalPlanTranslator.java    |   2 +-
 .../processor/post/RuntimeFilterContext.java       |  21 +-
 .../processor/post/RuntimeFilterGenerator.java     |  81 +--
 .../processor/post/RuntimeFilterPruner.java        |  16 +-
 .../post/RuntimeFilterPushDownVisitor.java         | 402 ++++++++++++
 .../nereids/rules/analysis/NormalizeRepeat.java    | 204 ++++--
 .../nereids/rules/rewrite/NormalizeToSlot.java     |  11 +
 .../functions/BuiltinFunctionBuilder.java          |   1 +
 .../expressions/functions/agg/Percentile.java      |  11 +-
 .../expressions/functions/agg/PercentileArray.java |  15 +-
 .../functions/scalar/ArrayIntersect.java           |  18 +-
 .../trees/plans/physical/AbstractPhysicalPlan.java |  79 ---
 .../trees/plans/physical/PhysicalCTEConsumer.java  |  16 -
 .../trees/plans/physical/PhysicalDistribute.java   |  36 -
 .../plans/physical/PhysicalHashAggregate.java      |  30 -
 .../trees/plans/physical/PhysicalHashJoin.java     |  91 ---
 .../plans/physical/PhysicalNestedLoopJoin.java     |  20 -
 .../trees/plans/physical/PhysicalProject.java      |  75 ---
 .../trees/plans/physical/PhysicalSetOperation.java |  41 --
 .../nereids/trees/plans/visitor/PlanVisitor.java   |   5 -
 .../trees/plans/visitor/RelationVisitor.java       |   5 +
 .../trees/plans/visitor/TableCollector.java        |  14 +-
 .../apache/doris/nereids/types/AggStateType.java   |   2 +-
 .../org/apache/doris/planner/OlapScanNode.java     |  13 +-
 .../main/java/org/apache/doris/qe/Coordinator.java |   3 +
 .../org/apache/doris/qe/MysqlConnectProcessor.java |  42 +-
 .../java/org/apache/doris/qe/PointQueryExec.java   |   2 +-
 .../org/apache/doris/qe/PrepareStmtContext.java    |   3 +
 .../java/org/apache/doris/qe/SessionVariable.java  |  18 +
 .../java/org/apache/doris/qe/StmtExecutor.java     |  62 +-
 .../org/apache/doris/statistics/AnalysisInfo.java  |   8 +-
 .../doris/statistics/AnalysisInfoBuilder.java      |   9 +-
 .../org/apache/doris/statistics/AnalysisJob.java   |   6 -
 .../apache/doris/statistics/AnalysisManager.java   |  37 +-
 .../apache/doris/statistics/BaseAnalysisTask.java  |  63 +-
 .../doris/statistics/ExternalAnalysisTask.java     |  31 +-
 .../apache/doris/statistics/JdbcAnalysisTask.java  |  28 +-
 .../apache/doris/statistics/OlapAnalysisTask.java  |  56 +-
 .../doris/statistics/StatisticConstants.java       |   8 +-
 .../doris/statistics/StatisticsAutoCollector.java  |   7 +-
 .../apache/doris/statistics/StatisticsCleaner.java |  10 +-
 .../doris/statistics/StatisticsRepository.java     |  17 +-
 .../apache/doris/statistics/TableStatsMeta.java    |   4 +-
 .../doris/statistics/util/StatisticsUtil.java      |  12 +-
 .../doris/transaction/DatabaseTransactionMgr.java  |   7 +-
 fe/fe-core/src/main/jflex/sql_scanner.flex         |   1 -
 .../doris/alter/InternalSchemaAlterTest.java       |   6 +-
 .../doris/cluster/DecommissionBackendTest.java     |   4 +-
 .../doris/statistics/JdbcAnalysisTaskTest.java     |  68 --
 gensrc/thrift/PaloInternalService.thrift           |   3 +-
 regression-test/conf/regression-conf.groovy        |   1 +
 .../test_bloom_filter_is_not_null.out              |  10 -
 .../data/correctness/test_cast_as_decimalv3.out    |   6 -
 .../test_from_millisecond_microsecond.out          |   9 -
 regression-test/data/correctness/test_in_null.out  |  14 -
 .../data/correctness/test_str_to_date.out          |  34 +-
 .../correctness/test_time_diff_microseconds.out    |   2 +-
 .../data/correctness/test_time_function.out        |  48 --
 regression-test/data/correctness_p0/test_avg.out   |   5 +-
 .../data/correctness_p0/test_cast_null.out         |   3 -
 .../data/correctness_p0/test_constant_having.out   |   2 -
 .../correctness_p0/test_div_decimal_overflow.out   |   5 +-
 .../data/correctness_p0/test_function_conjunct.out |   2 -
 .../agg_state/max/test_agg_state_max.out           |  27 -
 .../data/datatype_p0/datetimev2/test_exprs.out     |   3 -
 .../test_predicate_with_implicit_cast.out          |   9 -
 .../data/datatype_p0/ip/test_ip_basic.out          |   7 +
 .../three_level_nestedtypes_with_s3data.out        | 124 ++--
 .../two_level_nestedtypes_with_s3data.out          | 128 ++--
 .../test_complextype_nested_version_schema.out     |   7 -
 .../test_array_push_pop_backfront.out              |   6 +-
 .../get_assignment_compatible_type.out             |   8 +
 regression-test/data/ddl_p0/test_create_view.out   |   2 +-
 .../data/external_table_p0/es/test_es_query.out    |  62 ++
 regression-test/data/mv_p0/k1s2m3/k1s2m3.out       |  28 -
 .../data/mv_p0/mv_percentile/mv_percentile.out     |   6 +-
 .../test_aggregate_percentile_no_cast.out          |  36 +
 .../grouping_sets/grouping_normalize_test.out}     |  10 +-
 .../runtime_filter/test_pushdown_setop.out         |   2 +-
 .../data/performance_p0/redundant_conjuncts.out    |  37 +-
 .../data/point_query_p0/test_point_query.out       |  36 -
 .../test_point_query_cluster_key.out               | 111 ----
 .../data/prepared_stmt_p0/prepared_stmt.out        |  55 ++
 .../data/query_p0/having/having_between.out        |  15 -
 .../data/query_p0/lateral_view/lateral_view.out    |   8 -
 .../bitmap_functions/test_bitmap_function.out      | 117 ----
 .../test_unique_schema_key_change_add.out          | 111 ++++
 .../test_unique_schema_key_change_modify.out       | 141 ++++
 .../org/apache/doris/regression/suite/Suite.groovy |   5 +
 .../apache/doris/regression/util/JdbcUtils.groovy  |   9 +
 .../pipeline/external/conf/regression-conf.groovy  |   1 +
 .../alter_p0/test_alter_muti_modify_column.groovy  |  82 +++
 .../test_bloom_filter_is_not_null.groovy           |  12 +-
 .../correctness/test_cast_as_decimalv3.groovy      |  14 +-
 .../correctness/test_date_function_const.groovy    |   5 +-
 .../test_from_millisecond_microsecond.groovy       |  21 +-
 .../suites/correctness/test_in_null.groovy         |  15 +-
 .../suites/correctness/test_str_to_date.groovy     |  28 +-
 .../correctness/test_time_diff_microseconds.groovy |   5 +-
 .../suites/correctness/test_time_function.groovy   |  68 +-
 .../suites/correctness/test_timev2_fold.groovy     |   6 +-
 .../correctness_p0/test_agg_materialize.groovy     |   1 -
 .../suites/correctness_p0/test_avg.groovy          |   6 -
 .../suites/correctness_p0/test_bit_function.groovy |   2 -
 .../correctness_p0/test_bit_shift_lagency.groovy   |   4 +-
 .../correctness_p0/test_bucket_shuffle_join.groovy |  11 +-
 .../correctness_p0/test_cast_date_decimal.groovy   |   4 -
 .../suites/correctness_p0/test_cast_in.groovy      |   3 -
 .../suites/correctness_p0/test_cast_null.groovy    |  17 -
 .../correctness_p0/test_constant_having.groovy     |  21 -
 .../suites/correctness_p0/test_ctas_mv.groovy      |   1 -
 .../test_div_decimal_overflow.groovy               |   9 -
 .../correctness_p0/test_function_conjunct.groovy   |   5 -
 .../suites/correctness_p0/test_implict_cast.groovy |   3 +-
 .../test_inlineview_error_msg.groovy               |   1 -
 .../test_inlineview_with_lateralview.groovy        |   1 -
 .../test_inlineview_with_project.groovy            |   1 -
 .../test_inlineview_with_window_function.groovy    |   1 -
 .../test_join_with_projection.groovy               |   1 -
 .../suites/correctness_p0/test_mv_case.groovy      |   1 -
 .../test_outer_join_with_cross_join.groovy         |   3 +-
 .../test_outer_join_with_inline_view.groovy        |   1 -
 .../test_outer_join_with_null_value.groovy         |   1 -
 .../correctness_p0/test_outerjoin_jsonb.groovy     |   1 -
 .../test_push_conjunct_external_table.groovy       |   1 -
 .../correctness_p0/test_select_constant.groovy     |   2 -
 .../correctness_p0/test_table_function.groovy      |   1 -
 .../suites/correctness_p0/test_time_round.groovy   |   4 +-
 .../agg_state/max/test_agg_state_max.groovy        |  19 +-
 .../datatype_p0/complex_types/test_map.groovy      |   1 -
 .../datatype_p0/datetimev2/test_exprs.groovy       |   3 +-
 .../decimalv3/test_decimalv3_where.groovy          |   1 -
 .../test_predicate_with_implicit_cast.groovy       |   7 +-
 .../suites/datatype_p0/ip/test_ip_basic.groovy     |   2 +
 .../one_level_nestedtypes_with_s3data.groovy       |   5 -
 .../three_level_nestedtypes_with_s3data.groovy     |   6 -
 .../two_level_nestedtypes_with_s3data.groovy       |   5 -
 .../suites/datatype_p0/nested_types/load.groovy    |   1 -
 .../test_complextype_information_schema.groovy     |   1 -
 .../test_complextype_nested_version_schema.groovy  |   8 -
 ..._nested_types_insert_into_with_agg_table.groovy |   3 +-
 ..._nested_types_insert_into_with_dup_table.groovy |   1 -
 ...sted_types_insert_into_with_unique_table.groovy |   3 +-
 .../test_array_element_at_and_slice.groovy         |   2 -
 .../test_array_functions_array_join.groovy         |   2 -
 .../test_array_functions_array_with_const.groovy   |   3 -
 ...rray_functions_concat_compact_difference.groovy |   2 -
 .../test_array_push_pop_backfront.groovy           |   3 -
 .../test_array_zip_array_enumerate_uniq.groovy     |   4 +-
 .../map_functions/test_basic_map_function.groovy   |   5 -
 .../query/test_nested_type_with_count.groovy       |   3 -
 .../query/test_nested_type_with_rowstore.groovy    |   2 -
 .../test_nested_types_insert_into_with_s3.groovy   |   1 -
 ...test_nestedtypes_csv_insert_into_with_s3.groovy |   1 -
 .../test_nestedtypes_insert_into_select.groovy     |   2 -
 ...est_nestedtypes_json_insert_into_with_s3.groovy |   1 -
 .../get_assignment_compatible_type.groovy          |  52 ++
 .../suites/ddl_p0/test_create_view.groovy          |   1 -
 regression-test/suites/ddl_p0/test_ctas.groovy     |   6 -
 ...urrent_timestamp_as_column_default_value.groovy |   4 -
 .../suites/delete_p0/test_delete_on_value.groovy   |   3 -
 .../external_table_p0/es/test_es_query.groovy      |  30 +
 .../jdbc/test_clickhouse_jdbc_catalog.groovy       |   1 -
 .../jdbc/test_mysql_jdbc_catalog.groovy            |   1 -
 .../jdbc/test_mysql_jdbc_driver5_catalog.groovy    |   1 -
 .../jdbc/test_oracle_jdbc_catalog.groovy           |   8 -
 .../jdbc/test_pg_jdbc_catalog.groovy               |   1 -
 .../jdbc/test_sqlserver_jdbc_catalog.groovy        |   1 -
 .../insert_p0/insert_group_commit_into.groovy      |   2 +-
 .../suites/load_p0/insert/test_insert.groovy       |   7 +-
 .../insert/test_insert_move_memtable.groovy        |   7 +-
 .../suites/mtmv_p0/test_env_db_dropped_mtmv.groovy |  93 +++
 regression-test/suites/mv_p0/k1s2m3/k1s2m3.groovy  |  30 -
 .../mv_p0/mv_percentile/mv_percentile.groovy       |   2 +-
 .../cache/parse_sql_from_sql_cache.groovy          |  10 -
 .../nereids_p0/create_table/test_ctas.groovy       |   2 -
 .../suites/nereids_p0/datatype/test_cast.groovy    |   7 +-
 regression-test/suites/nereids_p0/load.groovy      |   3 -
 .../test_aggregate_percentile_no_cast.groovy       |  97 +++
 .../test_string_function_like.groovy               |  11 -
 .../table_function/explode_map.groovy              |   7 -
 .../suites/nereids_p0/tablesample.groovy           |   4 +-
 .../nereids_p0/test_filter_pushdown_set.groovy     |   3 +-
 .../grouping_sets/grouping_normalize_test.groovy}  |  39 +-
 .../mv/nested_mtmv/nested_mtmv.groovy              |   6 +-
 .../mv/newMv/dup_mv_bm_hash.groovy                 |   8 -
 .../nereids_syntax_p0/mv/newMv/dup_mv_year.groovy  |   8 -
 .../mv/newMv/sum_devide_count.groovy               |   6 -
 .../test_multi_column_partition.groovy             |  19 +-
 .../performance_p0/redundant_conjuncts.groovy      |   6 +-
 regression-test/suites/point_query_p0/load.groovy  |   2 -
 .../suites/point_query_p0/test_point_query.groovy  |  12 +-
 .../test_point_query_cluster_key.groovy            |  12 +-
 .../suites/prepared_stmt_p0/prepared_stmt.groovy   | 170 +++++
 .../aggregate_group_by_metric_type.groovy          |   2 -
 .../suites/query_p0/aggregate/window_funnel.groovy |   1 -
 .../join_with_column_casesensetive.groovy          |  10 -
 .../query_p0/cast/test_cast_decimalv3.groovy       |   1 -
 .../grouping_sets/test_grouping_sets.groovy        |  37 --
 .../suites/query_p0/having/having.groovy           |   1 -
 .../suites/query_p0/having/having_between.groovy   |   8 -
 .../suites/query_p0/join/test_join.groovy          | 729 ++++++++++-----------
 .../query_p0/lateral_view/lateral_view.groovy      |   4 -
 .../query_p0/literal_view/lietral_test.groovy      |   8 -
 .../bitmap_functions/test_bitmap_function.groovy   |  95 +--
 .../conditional_functions/test_nvl.groovy          |   1 -
 .../sql/dateTimeOperatorsAccessible.sql            |   2 +-
 ...st_array_with_single_row_max_than_uint16.groovy |   2 +-
 .../test_unique_schema_key_change_add.groovy       | 285 ++++++++
 ...=> test_unique_schema_key_change_modify.groovy} | 696 ++++++++++++--------
 .../test_partial_update_delete.groovy              |   4 -
 .../test_partial_update_delete_sign.groovy         |   4 -
 .../test_partial_update_insert_seq_col.groovy      |   6 +-
 .../test_partial_update_native_insert_stmt.groovy  |   6 +-
 ...artial_update_native_insert_stmt_complex.groovy |   6 +-
 .../tpcds-tools/ddl/create-tpcds-tables-sf1000.sql |  16 +-
 329 files changed, 4506 insertions(+), 3537 deletions(-)
 create mode 100755 
docker/thirdparties/docker-compose/elasticsearch/scripts/data/data3_es5.json
 copy 
regression-test/suites/datatype_p0/nested_types/meta/test_complextype_information_schema.groovy
 => fe/fe-core/src/main/java/org/apache/doris/datasource/SchemaCacheValue.java 
(57%)
 copy 
regression-test/suites/datatype_p0/nested_types/meta/test_complextype_information_schema.groovy
 => 
fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSSchemaCacheValue.java
 (52%)
 create mode 100644 
fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeSchemaCacheValue.java
 copy 
regression-test/suites/query_p0/sql_functions/conditional_functions/test_nvl.groovy
 => 
fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheValue.java
 (61%)
 create mode 100644 
fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoSchemaCacheValue.java
 create mode 100644 
fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPushDownVisitor.java
 create mode 100644 
regression-test/data/datatype_p0/scalar_types/get_assignment_compatible_type.out
 create mode 100644 
regression-test/data/nereids_p0/sql_functions/aggregate_functions/test_aggregate_percentile_no_cast.out
 copy regression-test/data/{correctness_p0/test_function_conjunct.out => 
nereids_rules_p0/grouping_sets/grouping_normalize_test.out} (62%)
 create mode 100644 regression-test/data/prepared_stmt_p0/prepared_stmt.out
 create mode 100644 
regression-test/data/schema_change_p0/test_unique_schema_key_change_add.out
 create mode 100644 
regression-test/data/schema_change_p0/test_unique_schema_key_change_modify.out
 create mode 100644 
regression-test/suites/alter_p0/test_alter_muti_modify_column.groovy
 create mode 100644 
regression-test/suites/datatype_p0/scalar_types/get_assignment_compatible_type.groovy
 create mode 100644 
regression-test/suites/mtmv_p0/test_env_db_dropped_mtmv.groovy
 create mode 100644 
regression-test/suites/nereids_p0/sql_functions/aggregate_functions/test_aggregate_percentile_no_cast.groovy
 copy regression-test/suites/{nereids_p0/tablesample.groovy => 
nereids_rules_p0/grouping_sets/grouping_normalize_test.groovy} (50%)
 create mode 100644 regression-test/suites/prepared_stmt_p0/prepared_stmt.groovy
 create mode 100644 
regression-test/suites/schema_change_p0/test_unique_schema_key_change_add.groovy
 rename 
regression-test/suites/schema_change_p0/{test_unique_model_schema_key_change.groovy
 => test_unique_schema_key_change_modify.groovy} (66%)


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

Reply via email to