This is an automated email from the ASF dual-hosted git repository. liaoxin pushed a change to branch auto-pick-42639-branch-3.0 in repository https://gitbox.apache.org/repos/asf/doris.git
from 2a5de38e3fe [test](move memtable) fix move memtable injection case fail (#42639) add 52a0484350d [opt](Nereids) use 1 instead narrowest column when do column pruning (#41548) (#42376) add e13a3e9e8cf [Impl](Nereids) add nereids gc cost time and be fold const cost time to profile (#42007) (#42518) add 25b7d349e36 [fix](Nereids) simplify range result wrong when reference is nullable (#41356) (#42373) add 128533950d4 [Test](bloom filter) add retry and retry timeout for bloom filter test case #42609 (#42673) add 3dd0b03fa7e [fix](Nereids) offset do more than once when have shuffle after limit (#42576) (#42682) add 12ea20ae5bf [fix](coalesce) fix 'heap-use-after-free' of function coalesce (#42666) (#42668) add 1ea434f6e88 [fix](cases) fix alias conflict case problem of driver url error (#42676) (#42691) add 189e94f4eab [opt](Nereids) use 1 as narrowest column when do column pruning on union (#41719) (#42860) add 0a888a09cb6 [fix](schema-change) fix the bug of alter column nullable when double writing (#41737) (#42656) add 6bf4f0b68bd [feature](doris compose) start node wait for ready service #42266 (#42540) add caa0ff80fed [fix](create table) Fix `test_partition_create_tablet_rr` docker regression case #41012 (#42388) add f625bbb06ea [enhancement](cloud) fix TTL cache size negative overflow (#42869) add ece718c9b37 [enhancement](cloud) make active warmup job number configurable (#42868) add 3b183a6fae8 [fix](regression) fix lru ttl evict regression (#42870) add ab0a7c89600 [enhance](paimon)paimon scanner code optimization #42606 (#42874) add f43d9d92706 [Test](count on index) enhance count on index case retrying for statistics #42699 (#42887) add da9845bc411 [enhance](hive)hive event code optimization #42637 (#42877) add 7490df85b60 [feat](nereids)disable join reorder if column stats is invalid (#41790) (branch-3.0) (#42917) add 77edc59eca1 [enhance](hive) support reading hive table with OpenCSVSerde #42257 (#42942) add 79b6a99f12d [fix](cloud) Fix tablet_id is zero when txn lazy commit (#42043) (#42905) add 60deaf68f99 [fix](Nerieds) fix FoldConstantRuleOnFE too slow when CaseWhen clause nesting multiple layers (#42897) (#42910) add ca35ea385fa [Test](bloom filter) add retry query for bloom filter test case #42937 (#42974) add 3082951c917 [fix](metrics) Fix doris compaction deltas total #42071 (#42960) add e9e0c38f60a [fix](jdbc catalog) fix be crash when create jdbc catalog without driver jar #42948 (#42971) add 86bb03435e1 [fix](task) Abort creating replica task if sending RPC failed #42276 (#42961) add 0c27eda872c [improvement](statistics)Add log for estimating hive table row count logic. (#42921) (#42980) add d60268d30c8 [fix](cloud) fix job status is empty when label reused in cloud mode (#42878) add b4ac202787c [fix](regression) fix txn_insert (#42885) (#42957) add 48bb6ad0c5e [improve](restore) Log create replica task progress #42350 (#42964) add 7769c68d682 [branch-3.0](pick) Pick #41789 #42482 #41210 #42460 (#42914) add 4da628deb77 [improve](task) Support splitting agent batch tasks automatically #42703 (#42989) add 01c19648979 [Test](count on index) add statistics check for unique table #42982 (#43021) add 78f31d72468 [3.0][opt](Catalog) Remove unnecessary conjuncts handling on External Scan (#43018) add f4f82299ee2 [fix](cloud) Fix incorrect DCHECK in txn lazy commit `convert_tmp_rowset` (#42903) (#42988) add 340334659c1 [improvement](statistics)Improve statisitcs insert into audit log. (#42983) (#43053) add 2abe20b020c [branch-3.0](pick) pick #42857 #42898 #42909 #42992 #43002 (#43039) add d27c1bb0472 [enhance](auth) Optimize the authentication logic of Ranger Doris (#41207) (#41840) add 8034ab187d4 [enhance](mtmv)Mtmv support audit log (#41499) (#41841) add 959bd31183d [fix](mtmv) cannot refresh mtmv when base table recreating (#41762) (#42340) add db0986e512b [fix](case)Fix multiple case tables with the same name causing case failure (#42203) (#43041) add 594865c34f5 [improvement](statistics)External table getRowCount return -1 when row count is not available or row count is 0. (#43009) (#43081) add 757b0ea23c5 [fix](regression-test) fix test_hive_serde_prop #42886 (#43099) add e46149b6fd4 [cherry-pick](branch-3.0) Pick "[Feature](schema change) Support add column bitmap with default value bitmap_empty (#42331)" (#42701) add 502d9e5bdcd [fix](exit) stop storage engine in exec_env before exit (#42015) (#43141) add d3c02efe38c [feature](statistics)Support get row count for pg and sql server. (#42674) (#43150) add 48ea74b050a [enhancement](Nereids) boost characterLiteralTypeCoercion (#42941) (#43094) add 14a1edc6ec9 [fix](spill) disable fuzzy spill for now (#42892) (#42894) add 84e5de52505 [test](p0) fix load stream leak in injection cases (#42681) (#43160) add 4c634d9cb17 Merge branch 'branch-3.0' into auto-pick-42639-branch-3.0 No new revisions were added by this update. Summary of changes: be/src/cloud/injection_point_action.cpp | 9 + be/src/exec/tablet_info.cpp | 32 +- be/src/io/cache/block_file_cache.cpp | 29 +- be/src/olap/olap_common.h | 26 +- be/src/olap/rowset/rowset_reader_context.h | 2 - be/src/olap/rowset/segment_v2/segment_iterator.cpp | 75 +- be/src/olap/rowset/segment_v2/segment_iterator.h | 4 +- be/src/olap/storage_engine.cpp | 8 +- be/src/olap/storage_engine.h | 10 +- be/src/pipeline/dependency.h | 8 +- be/src/pipeline/exec/aggregation_sink_operator.cpp | 9 +- be/src/pipeline/exec/aggregation_sink_operator.h | 11 +- .../pipeline/exec/aggregation_source_operator.cpp | 51 +- be/src/pipeline/exec/aggregation_source_operator.h | 11 +- be/src/pipeline/exec/analytic_sink_operator.cpp | 61 +- be/src/pipeline/exec/analytic_sink_operator.h | 7 +- be/src/pipeline/exec/analytic_source_operator.cpp | 39 +- be/src/pipeline/exec/analytic_source_operator.h | 8 +- be/src/pipeline/exec/assert_num_rows_operator.cpp | 1 + be/src/pipeline/exec/cache_source_operator.cpp | 20 +- be/src/pipeline/exec/datagen_operator.cpp | 24 +- be/src/pipeline/exec/datagen_operator.h | 2 + .../distinct_streaming_aggregation_operator.cpp | 5 +- .../exec/distinct_streaming_aggregation_operator.h | 7 +- be/src/pipeline/exec/es_scan_operator.cpp | 8 +- be/src/pipeline/exec/es_scan_operator.h | 3 +- be/src/pipeline/exec/exchange_sink_operator.cpp | 75 +- be/src/pipeline/exec/exchange_sink_operator.h | 23 +- be/src/pipeline/exec/exchange_source_operator.cpp | 23 +- be/src/pipeline/exec/exchange_source_operator.h | 5 +- .../exec/group_commit_block_sink_operator.cpp | 52 +- .../exec/group_commit_block_sink_operator.h | 7 +- .../pipeline/exec/group_commit_scan_operator.cpp | 1 + be/src/pipeline/exec/hashjoin_build_sink.cpp | 17 +- be/src/pipeline/exec/hashjoin_build_sink.h | 13 +- be/src/pipeline/exec/hashjoin_probe_operator.cpp | 7 +- be/src/pipeline/exec/hashjoin_probe_operator.h | 7 +- be/src/pipeline/exec/jdbc_table_sink_operator.cpp | 1 + .../pipeline/exec/join/process_hash_table_probe.h | 7 +- .../exec/join/process_hash_table_probe_impl.h | 18 +- be/src/pipeline/exec/join_build_sink_operator.cpp | 10 +- be/src/pipeline/exec/join_build_sink_operator.h | 2 - be/src/pipeline/exec/join_probe_operator.cpp | 4 +- be/src/pipeline/exec/join_probe_operator.h | 2 +- .../pipeline/exec/memory_scratch_sink_operator.cpp | 24 +- .../pipeline/exec/memory_scratch_sink_operator.h | 3 + .../exec/multi_cast_data_stream_source.cpp | 13 +- .../pipeline/exec/multi_cast_data_stream_source.h | 3 + .../exec/nested_loop_join_build_operator.cpp | 1 - .../exec/nested_loop_join_build_operator.h | 4 +- .../exec/nested_loop_join_probe_operator.cpp | 41 +- .../exec/nested_loop_join_probe_operator.h | 80 +- be/src/pipeline/exec/olap_scan_operator.cpp | 80 +- be/src/pipeline/exec/olap_scan_operator.h | 24 +- be/src/pipeline/exec/operator.cpp | 19 +- be/src/pipeline/exec/operator.h | 31 +- .../exec/partitioned_aggregation_sink_operator.h | 3 - .../partitioned_aggregation_source_operator.cpp | 4 + .../exec/partitioned_aggregation_source_operator.h | 2 + .../exec/partitioned_hash_join_probe_operator.h | 3 - .../exec/partitioned_hash_join_sink_operator.h | 3 - be/src/pipeline/exec/repeat_operator.cpp | 75 +- be/src/pipeline/exec/repeat_operator.h | 5 + be/src/pipeline/exec/result_file_sink_operator.cpp | 6 - be/src/pipeline/exec/result_file_sink_operator.h | 14 - be/src/pipeline/exec/result_sink_operator.cpp | 14 +- be/src/pipeline/exec/result_sink_operator.h | 7 +- be/src/pipeline/exec/scan_operator.cpp | 9 +- be/src/pipeline/exec/scan_operator.h | 8 +- be/src/pipeline/exec/set_probe_sink_operator.cpp | 11 +- be/src/pipeline/exec/set_probe_sink_operator.h | 5 +- be/src/pipeline/exec/set_sink_operator.cpp | 8 +- be/src/pipeline/exec/set_sink_operator.h | 7 +- be/src/pipeline/exec/set_source_operator.cpp | 37 +- be/src/pipeline/exec/set_source_operator.h | 3 + be/src/pipeline/exec/sort_sink_operator.cpp | 12 +- be/src/pipeline/exec/sort_sink_operator.h | 6 +- be/src/pipeline/exec/sort_source_operator.cpp | 4 +- .../exec/streaming_aggregation_operator.cpp | 28 +- .../pipeline/exec/streaming_aggregation_operator.h | 18 +- be/src/pipeline/exec/table_function_operator.cpp | 25 +- be/src/pipeline/exec/table_function_operator.h | 8 + be/src/pipeline/exec/union_sink_operator.cpp | 1 + be/src/pipeline/exec/union_sink_operator.h | 2 + be/src/pipeline/exec/union_source_operator.h | 4 +- .../local_exchange_sink_operator.cpp | 6 +- .../local_exchange/local_exchange_sink_operator.h | 2 +- .../local_exchange_source_operator.h | 3 - be/src/pipeline/local_exchange/local_exchanger.cpp | 2 +- be/src/pipeline/local_exchange/local_exchanger.h | 11 +- be/src/pipeline/pipeline.cpp | 44 +- be/src/pipeline/pipeline.h | 46 +- be/src/pipeline/pipeline_fragment_context.cpp | 322 ++++---- be/src/pipeline/pipeline_fragment_context.h | 38 +- be/src/runtime/exec_env_init.cpp | 2 +- be/src/util/arrow/row_batch.cpp | 17 +- be/src/util/arrow/row_batch.h | 7 +- be/src/util/doris_metrics.cpp | 4 +- .../vec/aggregate_functions/aggregate_function.h | 43 +- .../aggregate_functions/aggregate_function_avg.h | 22 +- .../aggregate_function_bitmap.h | 24 +- .../aggregate_function_bitmap_agg.h | 22 +- .../aggregate_functions/aggregate_function_count.h | 44 +- .../aggregate_function_count_by_enum.h | 4 +- .../aggregate_function_histogram.h | 4 +- .../aggregate_function_hll_union_agg.h | 2 +- .../aggregate_function_java_udaf.h | 6 +- .../aggregate_functions/aggregate_function_map.h | 18 +- .../aggregate_function_min_max.h | 158 ++-- .../aggregate_function_min_max_by.h | 44 +- .../aggregate_function_percentile.h | 2 +- .../aggregate_function_reader_first_last.h | 4 +- .../aggregate_functions/aggregate_function_rpc.h | 2 +- .../aggregate_functions/aggregate_function_sort.h | 6 +- .../aggregate_functions/aggregate_function_sum.h | 22 +- .../aggregate_functions/aggregate_function_uniq.h | 8 +- .../aggregate_function_uniq_distribute_key.h | 26 +- .../aggregate_function_window.h | 23 +- be/src/vec/exec/scan/new_es_scanner.cpp | 11 +- be/src/vec/exec/scan/new_olap_scanner.cpp | 191 +++-- be/src/vec/exec/scan/new_olap_scanner.h | 2 - be/src/vec/exec/scan/scanner_context.cpp | 38 +- be/src/vec/exec/scan/scanner_context.h | 23 +- be/src/vec/exec/scan/scanner_scheduler.cpp | 9 +- be/src/vec/exec/scan/vfile_scanner.cpp | 6 +- be/src/vec/exec/scan/vfile_scanner.h | 3 - be/src/vec/exec/scan/vscanner.cpp | 6 +- be/src/vec/exec/vjdbc_connector.cpp | 3 + be/src/vec/functions/function_coalesce.cpp | 20 +- be/src/vec/olap/vertical_block_reader.cpp | 1 - be/src/vec/sink/vdata_stream_sender.cpp | 17 +- be/src/vec/sink/vdata_stream_sender.h | 4 +- .../serde/data_type_serde_arrow_test.cpp | 654 ---------------- cloud/src/meta-service/meta_service_txn.cpp | 5 +- cloud/src/meta-service/txn_lazy_committer.cpp | 63 +- docker/runtime/doris-compose/command.py | 102 ++- docker/runtime/doris-compose/utils.py | 12 +- .../regression/serde_prop/some_serde_table.hql | 64 ++ .../docker-compose/mysql/init/04-insert.sql | 1 + .../docker-compose/postgresql/init/04-insert.sql | 1 + .../docker-compose/sqlserver/init/04-insert.sql | 1 + .../org/apache/doris/paimon/PaimonJniScanner.java | 6 +- .../{PaimonScannerUtils.java => PaimonUtils.java} | 25 +- .../main/java/org/apache/doris/common/Config.java | 10 + fe/fe-core/src/main/cup/sql_parser.cup | 7 + .../org/apache/doris/alter/SchemaChangeJobV2.java | 2 +- .../java/org/apache/doris/analysis/ColumnDef.java | 3 +- .../java/org/apache/doris/backup/BackupJob.java | 9 +- .../java/org/apache/doris/backup/RestoreJob.java | 43 +- .../main/java/org/apache/doris/catalog/Column.java | 5 + .../java/org/apache/doris/catalog/OlapTable.java | 10 +- .../main/java/org/apache/doris/catalog/Table.java | 2 +- .../java/org/apache/doris/catalog/TableIf.java | 2 + .../authorizer/ranger/RangerAccessController.java | 7 + .../ranger/cache/CatalogCacheAccessController.java | 91 --- .../authorizer/ranger/cache/DatamaskCacheKey.java | 89 --- .../authorizer/ranger/cache/RangerCache.java | 107 --- .../authorizer/ranger/cache/RowFilterCacheKey.java | 82 -- .../authorizer/ranger/doris/DorisAccessType.java | 79 +- .../authorizer/ranger/doris/DorisObjectType.java | 2 +- .../doris/RangerCacheDorisAccessController.java | 44 -- .../ranger/doris/RangerDorisAccessController.java | 177 +++-- .../ranger/doris/RangerDorisResource.java | 6 +- .../hive/RangerCacheHiveAccessController.java | 47 -- .../hive/RangerHiveAccessControllerFactory.java | 2 +- .../org/apache/doris/clone/TabletScheduler.java | 9 +- .../apache/doris/cloud/CacheHotspotManager.java | 5 +- .../transaction/CloudGlobalTransactionMgr.java | 3 +- .../doris/common/LabelAlreadyUsedException.java | 21 + .../apache/doris/common/MarkedCountDownLatch.java | 14 + .../doris/common/profile/SummaryProfile.java | 24 + .../doris/datasource/ExternalRowCountCache.java | 7 +- .../org/apache/doris/datasource/ExternalTable.java | 4 +- .../apache/doris/datasource/FileQueryScanNode.java | 11 +- .../doris/datasource/es/source/EsScanNode.java | 2 + .../doris/datasource/hive/HMSExternalTable.java | 56 +- .../doris/datasource/hive/HiveProperties.java | 50 +- .../datasource/hive/event/AddPartitionEvent.java | 8 +- .../datasource/hive/event/AlterDatabaseEvent.java | 8 +- .../datasource/hive/event/AlterPartitionEvent.java | 6 +- .../datasource/hive/event/AlterTableEvent.java | 10 +- .../datasource/hive/event/CreateDatabaseEvent.java | 4 +- .../datasource/hive/event/CreateTableEvent.java | 8 +- .../datasource/hive/event/DropDatabaseEvent.java | 4 +- .../datasource/hive/event/DropPartitionEvent.java | 8 +- .../datasource/hive/event/DropTableEvent.java | 6 +- .../doris/datasource/hive/event/IgnoredEvent.java | 2 +- .../doris/datasource/hive/event/InsertEvent.java | 6 +- .../datasource/hive/event/MetastoreEvent.java | 167 ++--- .../hive/event/MetastoreEventsProcessor.java | 2 +- .../doris/datasource/hive/source/HiveScanNode.java | 83 +-- .../datasource/iceberg/IcebergExternalTable.java | 3 +- .../doris/datasource/iceberg/IcebergUtils.java | 8 +- .../doris/datasource/jdbc/JdbcExternalTable.java | 96 ++- .../maxcompute/source/MaxComputeScanNode.java | 14 +- .../datasource/paimon/PaimonExternalTable.java | 8 +- .../datasource/paimon/source/PaimonScanNode.java | 4 + .../source/TrinoConnectorScanNode.java | 4 +- .../apache/doris/job/extensions/mtmv/MTMVTask.java | 43 +- .../apache/doris/journal/bdbje/BDBJEJournal.java | 4 +- .../java/org/apache/doris/mtmv/MTMVPlanUtil.java | 1 + .../doris/mtmv/MTMVRefreshPartitionSnapshot.java | 34 + .../mysql/privilege/AccessControllerManager.java | 10 +- .../org/apache/doris/nereids/NereidsPlanner.java | 20 +- .../org/apache/doris/nereids/StatementContext.java | 10 + .../glue/translator/PhysicalPlanTranslator.java | 5 - .../doris/nereids/jobs/executor/Rewriter.java | 8 +- .../doris/nereids/rules/analysis/BindRelation.java | 4 +- .../rules/expression/rules/FoldConstantRule.java | 2 +- .../expression/rules/FoldConstantRuleOnBE.java | 8 +- .../expression/rules/FoldConstantRuleOnFE.java | 11 +- .../rules/expression/rules/SimplifyRange.java | 248 ++++--- .../LogicalEsScanToPhysicalEsScan.java | 3 +- .../LogicalFileScanToPhysicalFileScan.java | 1 - .../LogicalHudiScanToPhysicalHudiScan.java | 1 - .../LogicalJdbcScanToPhysicalJdbcScan.java | 3 +- .../LogicalOdbcScanToPhysicalOdbcScan.java | 3 +- .../nereids/rules/rewrite/AdjustNullable.java | 12 - .../doris/nereids/rules/rewrite/ColumnPruning.java | 50 +- .../rules/rewrite/PruneFileScanPartition.java | 12 +- .../rules/rewrite/PushConjunctsIntoJdbcScan.java | 39 - .../rules/rewrite/PushConjunctsIntoOdbcScan.java | 39 - .../doris/nereids/stats/StatsCalculator.java | 46 +- .../trees/copier/LogicalPlanDeepCopier.java | 17 - .../expressions/functions/SearchSignature.java | 9 +- .../trees/expressions/literal/DateLiteral.java | 73 +- .../trees/expressions/literal/DateTimeLiteral.java | 75 +- .../expressions/literal/DateTimeV2Literal.java | 2 +- .../expressions/literal/DecimalV3Literal.java | 13 +- .../nereids/trees/expressions/literal/Result.java | 66 ++ .../expressions/literal/format/AndChecker.java} | 38 +- .../expressions/literal/format/AtLeastChecker.java | 49 ++ .../expressions/literal/format/CharChecker.java} | 20 +- .../expressions/literal/format/CheckResult.java | 49 ++ .../literal/format/CustomCharChecker.java} | 23 +- .../literal/format/DateTimeChecker.java | 137 ++++ .../expressions/literal/format/DebugChecker.java} | 22 +- .../expressions/literal/format/DigitChecker.java} | 40 +- .../expressions/literal/format/FloatChecker.java | 55 ++ .../expressions/literal/format/FormatChecker.java | 170 +++++ .../literal/format/IntegerChecker.java} | 32 +- .../expressions/literal/format/LetterChecker.java} | 40 +- .../expressions/literal/format/OptionChecker.java} | 20 +- .../expressions/literal/format/OrChecker.java | 55 ++ .../expressions/literal/format/StringChecker.java} | 28 +- .../expressions/literal/format/StringInspect.java | 64 ++ .../nereids/trees/plans/logical/LogicalEsScan.java | 26 +- .../plans/logical/LogicalExternalRelation.java | 68 -- .../trees/plans/logical/LogicalFileScan.java | 32 +- .../trees/plans/logical/LogicalHudiScan.java | 24 +- .../trees/plans/logical/LogicalJdbcScan.java | 24 +- .../trees/plans/logical/LogicalOdbcScan.java | 24 +- .../trees/plans/logical/LogicalProject.java | 3 +- .../trees/plans/physical/PhysicalEsScan.java | 23 +- .../trees/plans/physical/PhysicalFileScan.java | 29 +- .../trees/plans/physical/PhysicalHudiScan.java | 17 +- .../trees/plans/physical/PhysicalJdbcScan.java | 25 +- .../trees/plans/physical/PhysicalOdbcScan.java | 26 +- .../trees/plans/visitor/RelationVisitor.java | 13 +- .../apache/doris/nereids/types/DecimalV3Type.java | 25 +- .../apache/doris/nereids/util/ExpressionUtils.java | 12 + .../doris/nereids/util/TypeCoercionUtils.java | 54 +- .../org/apache/doris/planner/AggregationNode.java | 6 + .../org/apache/doris/planner/AnalyticEvalNode.java | 11 + .../apache/doris/planner/AssertNumRowsNode.java | 5 + .../org/apache/doris/planner/DataPartition.java | 4 + .../org/apache/doris/planner/EmptySetNode.java | 1 - .../org/apache/doris/planner/ExchangeNode.java | 29 + .../org/apache/doris/planner/JoinNodeBase.java | 1 - .../apache/doris/planner/NestedLoopJoinNode.java | 15 + .../org/apache/doris/planner/PlanFragment.java | 20 + .../java/org/apache/doris/planner/PlanNode.java | 14 + .../java/org/apache/doris/planner/RepeatNode.java | 6 + .../java/org/apache/doris/planner/ScanNode.java | 7 + .../java/org/apache/doris/planner/SelectNode.java | 6 + .../java/org/apache/doris/planner/SortNode.java | 6 + .../java/org/apache/doris/planner/UnionNode.java | 7 + .../main/java/org/apache/doris/qe/Coordinator.java | 37 +- .../java/org/apache/doris/qe/SessionVariable.java | 7 +- .../org/apache/doris/statistics/AnalysisJob.java | 2 +- .../doris/statistics/StatisticsAutoCollector.java | 2 +- .../doris/statistics/util/StatisticsUtil.java | 15 +- .../java/org/apache/doris/task/AgentBatchTask.java | 48 +- .../main/java/org/apache/doris/task/AgentTask.java | 4 + .../org/apache/doris/task/CreateReplicaTask.java | 17 + fe/fe-core/src/main/jflex/sql_scanner.flex | 1 + .../datasource/ExternalRowCountCacheTest.java | 32 +- .../apache/doris/external/hms/HmsCatalogTest.java | 30 + .../doris/nereids/jobs/joinorder/TPCHTest.java | 4 +- .../rules/expression/SimplifyRangeTest.java | 358 +++++---- .../nereids/rules/rewrite/AdjustNullableTest.java | 74 -- .../nereids/rules/rewrite/ColumnPruningTest.java | 31 +- .../trees/expressions/literal/DateLiteralTest.java | 16 +- .../expressions/literal/DateTimeLiteralTest.java | 496 +++++++------ .../expressions/literal/FloatLiteralTest.java | 80 ++ .../expressions/literal/IntegerLiteralTest.java | 64 ++ .../org/apache/doris/qe/HmsQueryCacheTest.java | 6 + gensrc/proto/cloud.proto | 1 + gensrc/thrift/PlanNodes.thrift | 1 + .../test_alter_add_column_default_value.out | 14 + .../hive/test_hive_serde_prop.out | 32 +- .../data/mtmv_p0/test_recreate_table_mtmv.out | 13 + .../eliminate_aggregate_constant.out | 96 +-- .../eliminate_join_condition.out | 8 +- .../push_down_filter_other_condition.out | 2 +- .../infer_predicate/infer_intersect_except.out | 11 - .../limit_push_down/limit_push_down.out | 24 +- .../limit_push_down/order_push_down.out | 8 +- .../push_down_top_n_distinct_through_join.out | 2 +- .../nereids_syntax_p0/sub_query_correlated.out | 9 + .../data/nereids_syntax_p0/test_limit.out | 6 - .../shape/query90.out | 36 +- .../rf_prune/query90.out | 36 +- .../nereids_tpcds_shape_sf100_p0/shape/query90.out | 36 +- .../runtime_filter/test_pushdown_setop.out | 2 +- .../new_shapes_p0/tpcds_sf100/rf_prune/query90.out | 36 +- .../new_shapes_p0/tpcds_sf100/shape/query90.out | 36 +- .../new_shapes_p0/tpcds_sf1000/shape/query90.out | 36 +- .../runtime_filter/test_pushdown_setop.out | 2 +- .../doris/regression/suite/SuiteCluster.groovy | 34 +- .../test_alter_add_column_default_value.groovy | 66 ++ ...est_bloom_filter_hit_with_renamed_column.groovy | 7 +- .../cloud_p0/cache/ttl/test_ttl_lru_evict.groovy | 53 +- .../test_fe_tablet_same_backend.groovy | 8 +- .../hive/test_hive_serde_prop.groovy | 4 + .../jdbc/test_jdbc_row_count.groovy | 107 +++ .../jdbc/test_mysql_jdbc_statistics.groovy | 11 - .../test_writer_v2_fault_injection.groovy | 4 +- .../suites/insert_p0/transaction/txn_insert.groovy | 2 +- .../inverted_index_p0/test_count_on_index.groovy | 34 +- .../stream_load/test_stream_load_job_status.groovy | 90 +++ .../suites/mtmv_p0/test_recreate_table_mtmv.groovy | 91 +++ .../suites/nereids_hint_tpcds_p0/load.groovy | 2 +- .../insert_into_table/complex_insert.groovy | 6 +- .../suites/nereids_p0/stats/invalid_stats.groovy | 111 +++ .../suites/nereids_p0/union/test_union.groovy | 12 +- .../union_const_expr_column_pruning.groovy | 7 +- .../column_pruning/window_column_pruning.groovy | 5 + .../nereids_rules_p0/eager_aggregate/basic.groovy | 4 +- .../eager_aggregate/basic_one_side.groovy | 3 +- .../infer_predicate/infer_intersect_except.groovy | 26 +- .../suites/nereids_syntax_p0/alias_conflict.groovy | 5 +- .../distribute/local_shuffle.groovy | 14 +- .../slow_fold_constant_case_when.groovy | 824 +++++++++++++++++++++ .../suites/nereids_syntax_p0/test_limit.groovy | 23 +- .../nereids_tpcds_shape_sf1000_p0/load.groovy | 2 +- .../constraints/load.groovy | 2 +- .../nereids_tpcds_shape_sf100_p0/load.groovy | 6 +- .../tpcds_sf100/constraints/load.groovy | 2 +- .../suites/new_shapes_p0/tpcds_sf100/load.groovy | 6 +- .../suites/new_shapes_p0/tpcds_sf1000/load.groovy | 2 +- .../test_partition_create_tablet_rr.groovy | 3 +- .../suites/query_p0/union/test_union.groovy | 12 +- .../suites/query_profile/test_profile.groovy | 16 + .../schema_change_p0/test_alter_uniq_null.groovy | 71 ++ 355 files changed, 6373 insertions(+), 4496 deletions(-) delete mode 100644 be/test/vec/data_types/serde/data_type_serde_arrow_test.cpp rename fe/be-java-extensions/paimon-scanner/src/main/java/org/apache/doris/paimon/{PaimonScannerUtils.java => PaimonUtils.java} (72%) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/authorizer/ranger/cache/CatalogCacheAccessController.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/authorizer/ranger/cache/DatamaskCacheKey.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/authorizer/ranger/cache/RangerCache.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/authorizer/ranger/cache/RowFilterCacheKey.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/authorizer/ranger/doris/RangerCacheDorisAccessController.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/authorizer/ranger/hive/RangerCacheHiveAccessController.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushConjunctsIntoJdbcScan.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushConjunctsIntoOdbcScan.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Result.java rename fe/fe-core/src/main/java/org/apache/doris/nereids/{rules/rewrite/PushConjunctsIntoEsScan.java => trees/expressions/literal/format/AndChecker.java} (51%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/format/AtLeastChecker.java copy fe/fe-core/src/main/java/org/apache/doris/{catalog/authorizer/ranger/hive/RangerHiveAccessControllerFactory.java => nereids/trees/expressions/literal/format/CharChecker.java} (65%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/format/CheckResult.java copy fe/fe-core/src/main/java/org/apache/doris/{catalog/authorizer/ranger/hive/RangerHiveAccessControllerFactory.java => nereids/trees/expressions/literal/format/CustomCharChecker.java} (56%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/format/DateTimeChecker.java copy fe/fe-core/src/main/java/org/apache/doris/{catalog/authorizer/ranger/hive/RangerHiveAccessControllerFactory.java => nereids/trees/expressions/literal/format/DebugChecker.java} (53%) copy fe/fe-core/src/main/java/org/apache/doris/{datasource/hive/event/IgnoredEvent.java => nereids/trees/expressions/literal/format/DigitChecker.java} (50%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/format/FloatChecker.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/format/FormatChecker.java rename fe/fe-core/src/main/java/org/apache/doris/{catalog/authorizer/ranger/cache/RangerCacheInvalidateListener.java => nereids/trees/expressions/literal/format/IntegerChecker.java} (52%) copy fe/fe-core/src/main/java/org/apache/doris/{datasource/hive/event/IgnoredEvent.java => nereids/trees/expressions/literal/format/LetterChecker.java} (50%) copy fe/fe-core/src/main/java/org/apache/doris/{catalog/authorizer/ranger/hive/RangerHiveAccessControllerFactory.java => nereids/trees/expressions/literal/format/OptionChecker.java} (62%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/format/OrChecker.java copy fe/fe-core/src/main/java/org/apache/doris/{catalog/authorizer/ranger/hive/RangerHiveAccessControllerFactory.java => nereids/trees/expressions/literal/format/StringChecker.java} (54%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/format/StringInspect.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalExternalRelation.java delete mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AdjustNullableTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/literal/FloatLiteralTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/literal/IntegerLiteralTest.java create mode 100644 regression-test/data/alter_p0/test_alter_add_column_default_value.out create mode 100644 regression-test/data/mtmv_p0/test_recreate_table_mtmv.out delete mode 100644 regression-test/data/nereids_syntax_p0/test_limit.out create mode 100644 regression-test/suites/alter_p0/test_alter_add_column_default_value.groovy create mode 100644 regression-test/suites/external_table_p0/jdbc/test_jdbc_row_count.groovy create mode 100644 regression-test/suites/load_p0/stream_load/test_stream_load_job_status.groovy create mode 100644 regression-test/suites/mtmv_p0/test_recreate_table_mtmv.groovy create mode 100644 regression-test/suites/nereids_p0/stats/invalid_stats.groovy create mode 100644 regression-test/suites/nereids_syntax_p0/slow_fold_constant_case_when.groovy create mode 100644 regression-test/suites/schema_change_p0/test_alter_uniq_null.groovy --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org