This is an automated email from the ASF dual-hosted git repository. airborne pushed a change to branch auto-pick-47093-branch-3.0 in repository https://gitbox.apache.org/repos/asf/doris.git
discard 44abd6b38a9 [fix](Nereids) let anonymous alias same as user input (#47093) add ef37a2813ef branch-3.0: [fix](nereids) fix push down non-foldable filter through project #47989 (#48083) add 7d944b89c8d branch-3.0: [opt](nereids) optimize not found function error message #47919 (#48081) add 7f62e9c6190 branch-3.0: [fix](mv) Fix use sync mv wrongly when use rbo materialized view rewrite rule #47650 (#47834) add 0311659bcc5 branch-3.0: [fix](nereids) fix bug in PhysicalTopN.equals() #46547 (#46632) add 4524a8c5ff0 branch-3.0: [fix](cases) fix unstable leading cases #46441 (#47898) add 197869e5f64 branch-3.0: [opt](nereids) optimize limit on distinct aggregate #47570 (#47816) add 56b712908d9 branch-3.0-pick: [test](case) add a case for cloud mow (#47868) (#48230) add 6c70e034eac [fix](load) fix metrics memtable_flush_duration_us (#47607) (#48211) add e84a55488e4 branch-3.0: [Fix](agg) Remove useless DCHECK for streaming agg node #47842 (#48258) add 2b8a4418d06 branch-3.0: [fix](Outfile) Set `retryTime` to one while executing `outfile` statement. #48095 (#48216) add b54cacfa0ff branch-3.0: [fix](parquet)Fix data column and null map column not equal when reading Parquet complex type cross-page data #47734 (#48038) add b8e907717c7 branch-3.0: [fix](test) fix the regression test `test_trino_hive_serde_prop` #48012 (#48044) add 2a9243c2d9d branch-3.0: [fix](regression) Fix test_show_data_warehouse groovy failed occasionally in `ASAN` mode #47802 (#48268) add 2bd0ed6f0d1 [Chore](parallel-result-sink) disable fuzzy ENABLE_PARALLEL_RESULT_SINK on branch-3.0 (#48068) add d9815ae98c9 [branch-3.0](pick) pick #47501 #47535 #47744 (#47867) add 704047c410c branch-3.0: [Fix](catalog)Fixes query failures for Paimon tables stored in Kerberized HDFS #47192 (#47693) add b718d915b02 branch-3.0:[opt](jdbc catalog) Change jdbc Driver loading to Java code (#47538) add 656d53a5e8f branch-3.0: [test](jdbc catalog) add more jdbc catalog extreme test #47525 (#47886) add c3fc9eab5ae branch-3.0: [opt](nereids)prune unused column after push down common column from agg #46627 (#47029) add a215a63b50b branch-3.0: [fix](external catalog) Persisting the External Catalog comment field #46946 (#47270) add fedbf161cd0 [Bug](join) return eof when join build sink awakend by downstream source #47380 #48247 (#47791) add ed50d7f344f branch-3.0: [BugFix](Variant) limit cast elimination to one level cast #47778 (#47954) add ee534dfaf10 branch-3.0: [opt](identifier) let column's name format more flexable #47840 (#47902) add 1b658a51fb3 [cherry-pick](branch-30) add more signatures for lag/lead fucntion (#47940) (#48077) add bc2bd9a8850 branch-3.0: [bug](restore) Fix wrong storage policy in show create stmt after restore #48158 (#48226) add d9926daaead branch-3.0: [fix](array index) Fix index build failure when all array column data is null #48231 (#48264) add f00c10317e4 branch-3.0: [fix](group commit) fix wal reader handle empty block #48290 (#48333) add 8f1f7dec7a3 branch-3.0: [Opt](cloud) Add inject points for cloud mow #48190 (#48330) add 8cb0470fe9d branch-3.0: [fix](group commit) group commit print profile #48256 (#48331) add 6d568f352b2 branch-3.0: [cloud](ut) fix unstable schema_kv_test.cpp #48299 (#48320) add 3cf8ad212e5 branch-3.0: [fix](partition) Incorrectly add partition to non-partitioned table #48121 (#48322) add e8c8709c4a7 branch-3.0: [improve](meta-service) improve error message "Transaction exceeds byte limit" #47507 (#48324) add d7b45324de9 branch-3.0: [fix](regression-test) Fix unstable regression test case due to trigger compaction failed #48228 (#48245) add 5618f137967 branch-3.0: [Fix](case) Fix show data case #48090 (#48260) add 267935936fa branch-3.0: [fix](group commit) group commit failed if enable global enable_unique_key_partial_update #48120 (#48221) add 56d1e9cefce branch-3.0: [test](hudi) add more hudi jni test cases #47431 (#47692) add 14c34ed9502 branch-3.0: [regression-test](suite) add sleep to waitingMVTaskFinishedByMvName #47549 (#47754) add 5df051daabf branch-3.0: [fix](nereids) analyzing job should not be blocked by enable_stats=false #46630 (#46690) add fae68ae0998 branch-3.0: [Fix](Variant) fix variant serialize to string #47121 (#47142) add 27f4500352f branch-3.0: [fix](Export) Clone `SessionVariable` for each ExportTaskExecutor #48239 (#48254) add c0764915df4 branch-3.0: [fix](Nereids) change numeric arithmatic function boundary behavior to match with be execution behavior (#47966) (#48281) add 5edfa824b16 [fix](Nereids) let anonymous alias same as user input (#47093) 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 (44abd6b38a9) \ N -- N -- N refs/heads/auto-pick-47093-branch-3.0 (5edfa824b16) 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/src/cloud/cloud_meta_mgr.cpp | 11 ++ be/src/cloud/cloud_schema_change_job.cpp | 16 +++ be/src/common/config.cpp | 3 + be/src/common/config.h | 3 + be/src/exprs/runtime_filter_slots.h | 3 + be/src/olap/memtable_flush_executor.cpp | 12 +- be/src/olap/parallel_scanner_builder.cpp | 11 +- be/src/olap/parallel_scanner_builder.h | 5 +- be/src/olap/task/index_builder.cpp | 13 +-- be/src/olap/wal/wal_reader.cpp | 16 ++- be/src/olap/wal/wal_reader.h | 1 + .../distinct_streaming_aggregation_operator.cpp | 1 - be/src/pipeline/exec/hashjoin_build_sink.cpp | 11 +- be/src/pipeline/exec/olap_scan_operator.cpp | 123 ++++++++++++++------ be/src/pipeline/exec/olap_scan_operator.h | 5 + be/src/pipeline/exec/operator.h | 2 + be/src/pipeline/exec/scan_operator.cpp | 6 +- be/src/pipeline/local_exchange/local_exchanger.cpp | 4 +- be/src/pipeline/pipeline.cpp | 9 ++ be/src/pipeline/pipeline_fragment_context.cpp | 3 +- be/src/pipeline/pipeline_task.cpp | 28 +++++ be/src/runtime/group_commit_mgr.cpp | 1 + be/src/runtime/user_function_cache.cpp | 20 +--- be/src/runtime/user_function_cache.h | 1 - be/src/vec/columns/column_object.cpp | 35 +++++- .../exec/format/parquet/vparquet_column_reader.cpp | 23 +++- .../exec/format/parquet/vparquet_column_reader.h | 21 ++-- be/src/vec/exec/vjdbc_connector.cpp | 29 ++--- be/src/vec/exec/vjdbc_connector.h | 2 + be/src/vec/runtime/shared_hash_table_controller.h | 1 - .../segment_v2/inverted_index_array_test.cpp | 74 ++++++++++++ cloud/src/meta-service/meta_service_txn.cpp | 4 + cloud/test/schema_kv_test.cpp | 16 ++- .../clickhouse/init/03-create-table.sql | 104 +++++++++++++++++ .../docker-compose/clickhouse/init/04-insert.sql | 18 +++ .../docker-compose/oracle/init/03-create-table.sql | 72 ++++++++++++ .../docker-compose/oracle/init/04-insert.sql | 50 ++++++++ .../postgresql/init/02-create-table.sql | 75 ++++++++++++ .../docker-compose/postgresql/init/04-insert.sql | 77 ++++++++++++ .../sqlserver/init/03-create-table.sql | 129 +++++++++++++++++++++ .../docker-compose/sqlserver/init/04-insert.sql | 18 +++ .../apache/doris/hudi/HadoopHudiJniScanner.java | 55 +++++---- .../org/apache/doris/jdbc/BaseJdbcExecutor.java | 70 ++++++++++- .../apache/doris/jdbc/JdbcDataSourceConfig.java | 10 ++ fe/be-java-extensions/paimon-scanner/pom.xml | 6 - .../org/apache/doris/paimon/PaimonJniScanner.java | 30 +++-- .../authentication/AuthenticationConfig.java | 17 ++- .../PreExecutionAuthenticatorCache.java | 87 ++++++++++++++ .../org/apache/doris/analysis/AnalyzeTblStmt.java | 5 - .../org/apache/doris/catalog/FunctionRegistry.java | 24 +++- .../org/apache/doris/catalog/JdbcResource.java | 7 +- .../org/apache/doris/catalog/PartitionInfo.java | 2 +- .../transaction/CloudGlobalTransactionMgr.java | 13 +++ .../apache/doris/common/ColumnAliasGenerator.java | 2 +- .../java/org/apache/doris/common/FeNameFormat.java | 4 +- .../apache/doris/datasource/ExternalCatalog.java | 4 +- .../apache/doris/datasource/InternalCatalog.java | 5 + .../org/apache/doris/load/ExportTaskExecutor.java | 3 +- .../glue/translator/PhysicalPlanTranslator.java | 28 ++++- .../post/ProjectAggregateExpressionsForCse.java | 33 +++++- .../post/PushDownFilterThroughProject.java | 13 ++- .../doris/nereids/processor/post/Validator.java | 8 -- .../nereids/rules/rewrite/LimitAggToTopNAgg.java | 12 +- .../rewrite/PushDownFilterThroughProject.java | 44 ++++--- .../mv/SelectMaterializedIndexWithAggregate.java | 16 ++- .../functions/AggCombinerFunctionBuilder.java | 5 + .../functions/BuiltinFunctionBuilder.java | 16 +++ .../expressions/functions/ExpressionTrait.java | 9 +- .../expressions/functions/FunctionBuilder.java | 5 + .../functions/executable/NumericArithmetic.java | 69 +++++++---- .../expressions/functions/udf/AliasUdfBuilder.java | 7 ++ .../expressions/functions/udf/JavaUdafBuilder.java | 15 +++ .../expressions/functions/udf/JavaUdfBuilder.java | 15 +++ .../expressions/functions/udf/JavaUdtfBuilder.java | 15 +++ .../trees/expressions/functions/window/Lag.java | 34 ++++-- .../trees/expressions/functions/window/Lead.java | 34 ++++-- .../nereids/trees/plans/algebra/Aggregate.java | 6 + .../trees/plans/logical/LogicalAggregate.java | 7 +- .../plans/physical/PhysicalHashAggregate.java | 2 + .../nereids/trees/plans/physical/PhysicalTopN.java | 4 +- .../java/org/apache/doris/qe/SessionVariable.java | 3 +- .../java/org/apache/doris/qe/StmtExecutor.java | 5 + .../apache/doris/alter/AddExistsPartitionTest.java | 7 +- .../org/apache/doris/catalog/JdbcResourceTest.java | 51 ++++++++ .../org/apache/doris/common/FeNameFormatTest.java | 5 +- .../doris/datasource/ExternalCatalogTest.java | 33 ++++++ .../PushDownFilterThroughProjectTest.java | 41 +++++++ .../nereids/rules/expression/FoldConstantTest.java | 87 ++++++-------- .../rewrite/PushDowFilterThroughProjectTest.java | 48 +++++++- .../PushDownLimitDistinctThroughJoinTest.java | 2 +- .../trees/plans/physical/PhysicalTopNTest.java | 55 +++++++++ gensrc/thrift/Types.thrift | 1 + .../data/correctness_p0/test_lag_lead_window.out | Bin 844 -> 1402 bytes .../data/datatype_p0/double/test_double_nan.out | Bin 110 -> 133 bytes .../data/datatype_p0/float/test_float_nan.out | Bin 110 -> 133 bytes .../select/test_clickhouse_all_types_select.out | Bin 0 -> 2936 bytes .../select/test_oracle_all_types_select.out | Bin 0 -> 10265 bytes .../type_test/select/test_pg_all_types_select.out | Bin 0 -> 1959 bytes .../select/test_sqlserver_all_types_select.out | Bin 0 -> 3508 bytes .../tvf/{test_tvf_p2.out => test_tvf_p0.out} | Bin 5562 -> 5674 bytes .../hive/test_parquet_complex_cross_page.out} | Bin 127 -> 137 bytes .../hudi/test_hudi_incremental.out | Bin 5687 -> 7551 bytes .../hudi/test_hudi_schema_evolution.out | Bin 7521 -> 11234 bytes .../external_table_p2/hudi/test_hudi_snapshot.out | Bin 696105 -> 1044838 bytes .../hudi/test_hudi_timetravel.out | Bin 2111 -> 3119 bytes .../cloud/test_cloud_mow_correctness_inject.out | Bin 185 -> 368 bytes .../cloud/test_cloud_mow_lock_expired.out} | Bin 365 -> 388 bytes ...h_regexp_arr.out => test_add_index_for_arr.out} | Bin 179 -> 187 bytes .../agg_use_key_direct/agg_use_key_direct.out} | Bin 136 -> 210 bytes .../data/nereids_p0/hint/test_leading.out | Bin 27327 -> 33391 bytes .../window_functions/test_window_function.out | Bin 10367 -> 11741 bytes .../filter_push_down/push_filter_through.out | Bin 10087 -> 10023 bytes .../push_down_limit_distinct_through_join.out | Bin 543 -> 442 bytes .../data/nereids_tpch_p0/tpch/push_topn_to_agg.out | Bin 158 -> 725 bytes .../join/test_slow_close/test_slow_close.out} | Bin 123 -> 133 bytes .../window_functions/test_window_function.out | Bin 10367 -> 11741 bytes regression-test/data/variant_p0/agg.out | Bin 1444 -> 1478 bytes .../data/variant_p0/compaction/test_compaction.out | Bin 5827 -> 5835 bytes regression-test/data/variant_p0/rqg/rqg4.out | Bin 184 -> 199 bytes .../org/apache/doris/regression/suite/Suite.groovy | 3 + .../plugins/cloud_show_data_plugin.groovy | 59 ++++++---- .../correctness_p0/test_lag_lead_window.groovy | 9 ++ .../datatype_p0/double/test_double_nan.groovy | 10 +- .../suites/datatype_p0/float/test_float_nan.groovy | 10 +- .../test_clickhouse_all_types_select.groovy} | 33 +++--- .../select/test_oracle_all_types_select.groovy} | 34 +++--- .../select/test_pg_all_types_select.groovy | 76 ++++++++++++ .../select/test_sqlserver_all_types_select.groovy} | 34 +++--- .../hive/test_trino_hive_serde_prop.groovy | 1 + .../tvf/{test_tvf_p2.groovy => test_tvf_p0.groovy} | 24 +++- .../hive/test_parquet_complex_cross_page.groovy | 52 +++++++++ .../hudi/test_hudi_incremental.groovy | 18 ++- .../hudi/test_hudi_schema_evolution.groovy | 16 ++- .../hudi/test_hudi_snapshot.groovy | 10 +- .../hudi/test_hudi_timetravel.groovy | 11 +- .../cloud/test_cloud_mow_correctness_inject.groovy | 71 +++++++++++- .../cloud/test_cloud_mow_lock_expired.groovy | 126 ++++++++++++++++++++ .../not_found_function.groovy} | 12 +- .../test_group_commit_replay_wal.groovy | 4 +- ...nsert_group_commit_into_unique_sync_mode.groovy | 5 + .../array_contains/test_add_index_for_arr.groovy | 43 +++++-- .../agg_use_key_direct/agg_use_key_direct.groovy | 63 ++++++++++ .../nereids_function_p0/scalar_function/A.groovy | 5 +- .../fold_constant_numeric_arithmatic.groovy | 60 +++++----- .../suites/nereids_p0/hint/test_leading.groovy | 17 +++ .../window_functions/test_window_function.groovy | 33 +++--- .../nereids_tpch_p0/tpch/push_topn_to_agg.groovy | 82 +++++++++++++ .../test_add_partition_exception.groovy} | 26 ++--- .../join/test_slow_close/test_slow_close.groovy | 88 ++++++++++++++ .../window_functions/test_window_function.groovy | 34 +++--- .../test_dup_keys_schema_change.groovy | 2 +- .../test_uniq_keys_schema_change.groovy | 2 +- .../suites/show_p0/test_show_data_warehouse.groovy | 29 +++-- .../suites/variant_log_data_p2/load.groovy | 6 +- regression-test/suites/variant_p0/agg.groovy | 8 +- .../suites/variant_p0/delete_update.groovy | 2 +- regression-test/suites/variant_p0/desc.groovy | 2 +- regression-test/suites/variant_p0/load.groovy | 4 +- regression-test/suites/variant_p0/rqg/rqg4.sql | 3 +- regression-test/suites/variant_p2/load.groovy | 2 +- 160 files changed, 2710 insertions(+), 554 deletions(-) create mode 100644 fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/PreExecutionAuthenticatorCache.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTopNTest.java create mode 100644 regression-test/data/external_table_p0/jdbc/type_test/select/test_clickhouse_all_types_select.out create mode 100644 regression-test/data/external_table_p0/jdbc/type_test/select/test_oracle_all_types_select.out create mode 100644 regression-test/data/external_table_p0/jdbc/type_test/select/test_pg_all_types_select.out create mode 100644 regression-test/data/external_table_p0/jdbc/type_test/select/test_sqlserver_all_types_select.out rename regression-test/data/external_table_p0/tvf/{test_tvf_p2.out => test_tvf_p0.out} (98%) copy regression-test/data/{nereids_rules_p0/defer_materialize_topn/one_phase.out => external_table_p2/hive/test_parquet_complex_cross_page.out} (83%) copy regression-test/data/{delete_p0/test_segment_iterator_delete.out => fault_injection_p0/cloud/test_cloud_mow_lock_expired.out} (63%) copy regression-test/data/inverted_index_p0/array_contains/{test_index_match_regexp_arr.out => test_add_index_for_arr.out} (89%) copy regression-test/data/{query_p0/show/test_struct_show_create.out => mv_p0/agg_use_key_direct/agg_use_key_direct.out} (55%) copy regression-test/data/{correctness_p0/test_group_by_constant.out => query_p0/join/test_slow_close/test_slow_close.out} (89%) copy regression-test/suites/external_table_p0/jdbc/{test_oceanbase_jdbc_catalog.groovy => type_test/select/test_clickhouse_all_types_select.groovy} (54%) copy regression-test/suites/external_table_p0/jdbc/{test_oceanbase_jdbc_catalog.groovy => type_test/select/test_oracle_all_types_select.groovy} (55%) create mode 100644 regression-test/suites/external_table_p0/jdbc/type_test/select/test_pg_all_types_select.groovy copy regression-test/suites/external_table_p0/jdbc/{test_oceanbase_jdbc_catalog.groovy => type_test/select/test_sqlserver_all_types_select.groovy} (54%) rename regression-test/suites/external_table_p0/tvf/{test_tvf_p2.groovy => test_tvf_p0.groovy} (78%) create mode 100644 regression-test/suites/external_table_p2/hive/test_parquet_complex_cross_page.groovy create mode 100644 regression-test/suites/fault_injection_p0/cloud/test_cloud_mow_lock_expired.groovy copy regression-test/suites/{nereids_rules_p0/expression/test_simplify_decimal_comparison.groovy => function_p0/not_found_function.groovy} (70%) create mode 100644 regression-test/suites/mv_p0/agg_use_key_direct/agg_use_key_direct.groovy copy regression-test/suites/{datatype_p0/date/test_weekofyear.groovy => partition_p0/no_partition/test_add_partition_exception.groovy} (62%) create mode 100644 regression-test/suites/query_p0/join/test_slow_close/test_slow_close.groovy --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org