This is an automated email from the ASF dual-hosted git repository.
yiguolei pushed a change to branch auto-pick-58426-branch-4.0
in repository https://gitbox.apache.org/repos/asf/doris.git
discard 18007d9d9b9 [fix](mv) Fix stats unknown when calc sync mv plan
statistics (#58426)
add 6244527a8b1 branch-4.0: [opt](arm) Improve count_zero_num performance
with NEON intrinsics (#58803)
add 66de1e460c6 branch-4.0: [fix](filecache) using invalid memory crashes
clear_need_update_lru_blocks #58768 (#58820)
add 458826fddcc branch-4.0: [fix](multi_distinct_count) fix wrong result
of multi_distinct_count of datetime #58775 (#58786)
add 7db6b204b01 branch-4.0: [fix](case) test_alter_table_property #58769
(#58826)
add 3d169778d51 branch-4.0: [security](cloud) Support to set internal port
for builtin brpc services on MS #58810 (#58824)
add f66172bec51 branch-4.0: log print flight sql ticket error #58811
(#58842)
add dcecaf0c371 branch-4.0: Revert "[Fix](mow) seg compaction should wait
for calc delete bitmap tasks on input segments before converting delete bitmaps
on them (#53198)" (#58855)
add 5cd3734089c branch-4.0: [Feature] doris cross-cluster query #57898
(#58571)
add 119a5dafb36 branch-4.0: [Feature](serd)Support read column from json
arrow. #58429 (#58757)
add c5c63542aaa branch-4.0: [fix](case) support retryIfHttpError in
test_array_with_large_dataset… #58771 (#58825)
add d4caf9e73c2 branch-4.0: [fix](hive) fix potential inconsistent hive
partition info cache #58707 (#58798)
add f841fb6a9e5 branch-4.0: [Fix](AI_Func) Fix the error prompt-building
and response-parsing process of Local_Adapter #58492 (#58796)
add 7770090235b branch-4.0: [Chore](scan) make scanner wait worker timer
more more accurate #58833 (#58862)
add f6442a9e529 branch-4.0: [fix](compaction) fix vertical compaction IO
stats accumulation #58756 (#58866)
add 2b34a393312 branch-4.0: [fix](unique function) fix push unique
function through join #58516 (#58854)
add e47082dcc51 branch-4.0: [fix](inverted index) fix pinyin bug #57756
(#58852)
add 12bd9ae9b9f branch-4.0: [Feature](function) Support function
TO_SECONDS #58635 (#58868)
add 0b5a73262f9 branch-4.0: fix flight sql when contains & #58830 (#58884)
add d44b65324eb branch-4.0: [fix](hint)should pull up subquery alias with
hint for all kind of root nodes #58767 (#58853)
add 30d2df04594 4.0.2-rc02 (#58600)
add 4ca9fabd122 branch-4.0: [chore](be) remove cpu limit since it is
useless #58860 (#58911)
add 468b3f6dbb5 [test](mtls)test framwork support tls (#5874) (#58368)
(#58838)
add e40beaa4000 branch-4.0: [fix](test) fix unstable export external table
#58799 (#58887)
add 1362c00c7ef branch-4.0: [fix](regr) Use Youngs-Cramer for
REGR_SLOPE/INTERCEPT to align with PG #55940 (#58920)
add 07f248b7737 branch-4.0: [fix](test) fix unstable es catalog test
#58800 (#58888)
add bf22d3d3e7b branch-4.0: [feat](load) introduce load job system table
(#57421) (#58850)
add 62a5a1545c8 branch-4.0: [fix](test) fix regression case
limit_push_down #58873 (#58933)
add da806a9543f branch-4.0: [Improvement](join) lazy calculate
all_match_one #58729 (#58901)
add 43b4b98f193 branch-4.0: [test](doris catalog)Add test cases for doris
catalog #58831 (#58934)
add ffb11fc3cca branch-4.0: [fix](jdbc) Fix JNI global reference leak in
JdbcConnector close #58574 (#58753)
add 5d30bfdb684 branch-4.0: [fix](mv) Fix sync mv could not be chosen by
cbo stable because stats upload from be not in time #58720 (#58772)
add a0726e8f275 branch-4.0: [opt](partition pruner) add
cache_sorted_partition_interval_second #58766 (#58809)
add d5fd1b68aad branch-4.0: [Refactor](cast) Make all nullable for cast
from string to simple types #57949 (#58924)
add 9543f0fd967 [fix](mv) Fix stats unknown when calc sync mv plan
statistics (#58426)
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 (18007d9d9b9)
\
N -- N -- N refs/heads/auto-pick-58426-branch-4.0 (9543f0fd967)
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/common/config.cpp | 2 -
be/src/common/config.h | 2 -
be/src/exec/schema_scanner.cpp | 3 +
be/src/exec/schema_scanner/schema_helper.cpp | 9 +
be/src/exec/schema_scanner/schema_helper.h | 5 +
.../schema_scanner/schema_load_job_scanner.cpp | 189 +++++++++
...oad_job_scanner.h => schema_load_job_scanner.h} | 12 +-
be/src/io/cache/block_file_cache.cpp | 10 +-
be/src/olap/base_tablet.cpp | 24 +-
be/src/olap/base_tablet.h | 15 +-
be/src/olap/calc_delete_bitmap_executor.cpp | 16 +-
be/src/olap/calc_delete_bitmap_executor.h | 3 +-
be/src/olap/merger.cpp | 46 ++-
be/src/olap/olap_common.h | 36 --
be/src/olap/rowset/beta_rowset_writer.cpp | 22 +-
be/src/olap/rowset/beta_rowset_writer.h | 8 +-
be/src/olap/rowset/segcompaction.cpp | 40 +-
be/src/olap/rowset/segcompaction.h | 11 +-
.../inverted_index/token_filter/pinyin_filter.cpp | 76 +++-
.../token_filter/pinyin_filter_factory.cpp | 8 -
.../tokenizer/pinyin/pinyin_formatter.cpp | 37 +-
.../tokenizer/pinyin/pinyin_tokenizer.cpp | 58 +--
.../tokenizer/pinyin/pinyin_tokenizer.h | 2 +-
.../pipeline/exec/join/process_hash_table_probe.h | 3 +-
.../exec/join/process_hash_table_probe_impl.h | 17 +-
be/src/pipeline/exec/olap_scan_operator.cpp | 4 +-
be/src/pipeline/exec/scan_operator.cpp | 29 +-
be/src/service/arrow_flight/flight_sql_service.cpp | 11 +-
be/src/util/simd/bits.h | 62 ++-
.../aggregate_function_regr_union.h | 250 +++++++++---
.../aggregate_function_uniq.cpp | 12 +-
.../vec/data_types/serde/data_type_jsonb_serde.cpp | 70 ++++
.../vec/data_types/serde/data_type_jsonb_serde.h | 2 +
.../data_types/serde/data_type_nullable_serde.cpp | 10 +-
be/src/vec/exec/scan/scanner_scheduler.cpp | 20 +-
be/src/vec/exec/vjdbc_connector.cpp | 50 ++-
be/src/vec/functions/ai/ai_adapter.h | 194 ++++++---
be/src/vec/functions/ai/ai_functions.h | 14 +
be/src/vec/functions/cast/cast_base.cpp | 18 +-
.../functions/cast/cast_to_basic_number_common.h | 26 +-
be/src/vec/functions/cast/cast_to_boolean.h | 26 +-
be/src/vec/functions/cast/cast_to_date.h | 67 ++--
be/src/vec/functions/cast/cast_to_decimal.h | 80 ++--
be/src/vec/functions/cast/cast_to_float.h | 2 +-
be/src/vec/functions/cast/cast_to_int.h | 37 +-
be/src/vec/functions/cast/cast_to_ip.h | 25 +-
be/src/vec/functions/cast/cast_to_jsonb.h | 9 +-
be/src/vec/functions/date_time_transforms.h | 2 +
be/src/vec/functions/to_time_function.cpp | 3 +
be/test/ai/ai_adapter_test.cpp | 164 ++++++--
be/test/ai/ai_function_test.cpp | 24 ++
be/test/ai/embed_test.cpp | 11 +
.../token_filter/pinyin_filter_test.cpp | 355 +++++++++++++++++
.../tokenizer/pinyin_analysis_test.cpp | 437 +++++++++++++++++++++
.../inverted_index/tokenizer/pinyin_util_test.cpp | 122 ++++++
be/test/olap/segcompaction_mow_test.cpp | 20 -
.../serde/data_type_jsonb_serde_test.cpp | 49 +++
be/test/vec/function/cast/cast_to_integer.cpp | 7 +-
cloud/src/common/config.h | 1 +
cloud/src/main.cpp | 8 +-
.../org/apache/doris/jdbc/BaseJdbcExecutor.java | 19 +-
.../org/apache/doris/analysis/SchemaTableType.java | 2 +
.../doris/catalog/BuiltinScalarFunctions.java | 2 +
.../java/org/apache/doris/catalog/OlapTable.java | 49 ++-
.../java/org/apache/doris/catalog/SchemaTable.java | 24 ++
.../java/org/apache/doris/catalog/TableIf.java | 2 +
.../cache/NereidsSortedPartitionsCacheManager.java | 15 +-
.../doris/datasource/ExternalMetaCacheMgr.java | 9 +
.../doris/DorisExternalMetaCacheMgr.java | 83 ++++
.../doris/datasource/doris/FeServiceClient.java | 278 +++++++++++++
.../doris/RemoteDorisExternalCatalog.java | 37 +-
.../datasource/doris/RemoteDorisExternalTable.java | 79 +++-
.../doris/datasource/doris/RemoteOlapTable.java | 130 ++++++
.../doris/datasource/hive/HMSExternalTable.java | 5 +-
.../datasource/iceberg/IcebergMetadataOps.java | 2 +
.../property/constants/RemoteDorisProperties.java | 4 +
.../doris/httpv2/restv2/ESCatalogAction.java | 4 -
.../indexpolicy/PinyinTokenFilterValidator.java | 35 +-
.../indexpolicy/PinyinTokenizerValidator.java | 35 +-
.../processor/pre/PullUpSubqueryAliasToCTE.java | 69 ++--
.../doris/nereids/rules/analysis/BindRelation.java | 18 +
.../LogicalOlapScanToPhysicalOlapScan.java | 3 +-
.../rules/rewrite/PushDownFilterThroughJoin.java | 7 +
.../rules/rewrite/PushFilterInsideJoin.java | 23 +-
.../doris/nereids/rules/rewrite/ReorderJoin.java | 42 +-
.../doris/nereids/stats/StatsCalculator.java | 6 +
.../doris/nereids/trees/expressions/Cast.java | 13 -
.../functions/agg/MultiDistinctCount.java | 8 +-
.../executable/DateTimeExtractAndTransform.java | 46 ++-
.../trees/expressions/functions/scalar/Date.java | 1 +
.../scalar/{ToDays.java => ToSeconds.java} | 22 +-
.../expressions/visitor/ScalarFunctionVisitor.java | 5 +
.../trees/plans/distribute/DistributePlanner.java | 14 +
.../trees/plans/distribute/SelectedWorkers.java | 20 +-
.../BackendDistributedPlanWorkerManager.java | 57 ++-
.../plans/distribute/worker/BackendWorker.java | 14 +-
.../distribute/worker/DistributedPlanWorker.java | 2 +
.../worker/DistributedPlanWorkerManager.java | 12 +-
.../trees/plans/distribute/worker/DummyWorker.java | 7 +
.../worker/LoadBalanceScanWorkerSelector.java | 11 +-
.../worker/job/AbstractUnassignedScanJob.java | 7 +-
.../distribute/worker/job/UnassignedAllBEJob.java | 22 +-
.../worker/job/UnassignedGroupCommitJob.java | 4 +-
.../job/UnassignedScanBucketOlapTableJob.java | 6 +-
.../org/apache/doris/planner/OlapScanNode.java | 13 +-
.../java/org/apache/doris/planner/ScanNode.java | 5 +
.../org/apache/doris/qe/AbstractJobProcessor.java | 6 +-
.../java/org/apache/doris/qe/SessionVariable.java | 27 +-
.../doris/qe/runtime/PipelineExecutionTask.java | 11 +-
.../qe/runtime/PipelineExecutionTaskBuilder.java | 6 +-
.../apache/doris/service/FrontendServiceImpl.java | 186 +++++++++
.../arrowflight/DorisFlightSqlProducer.java | 5 +-
.../main/java/org/apache/doris/system/Backend.java | 12 +
gensrc/script/gen_build_version.sh | 2 +-
gensrc/thrift/Descriptors.thrift | 1 +
gensrc/thrift/FrontendService.thrift | 57 +++
regression-test/conf/regression-conf.groovy | 13 +
.../date/test_agg_functions.out} | 15 +-
.../datatype_p0/datetimev2/test_agg_functions.out | 23 ++
.../sql-functions/doc_date_functions_test.out | 48 +++
...est_query_remote_doris_as_olap_table_select.out | 48 +++
.../test_remote_doris_agg_table_select.out | 54 +++
.../test_remote_doris_all_types_select.out | 8 +-
.../test_remote_doris_all_types_show.out | 3 +-
.../test_remote_doris_unique_table_select.out | 32 ++
.../test_remote_doris_variant_select.out | 81 ++++
.../test_mow_seq_seg_compaction.out | 23 --
.../analyzer/test_custom_analyzer.out | 38 +-
.../limit_push_down/limit_push_down.out | 32 +-
...wn_filter_through_join_with_unique_function.out | 67 ++++
.../support_type/regr_intercept/regr_intercept.out | 8 +-
.../support_type/regr_slope/regr_slope.out | 8 +-
.../org/apache/doris/regression/Config.groovy | 66 +++-
.../doris/regression/action/HttpCliAction.groovy | 223 ++++++-----
.../doris/regression/action/ProfileAction.groovy | 66 ++++
.../regression/action/StreamLoadAction.groovy | 211 ++++++++--
.../org/apache/doris/regression/suite/Suite.groovy | 14 +-
.../doris/regression/suite/SuiteContext.groovy | 19 +
.../org/apache/doris/regression/util/Http.groovy | 78 ++++
.../pipeline/external/conf/regression-conf.groovy | 1 +
.../plugins/cloud_compaction_plugin.groovy | 16 +-
.../plugins/cloud_show_data_plugin.groovy | 3 +
.../plugins/plugin_curl_requester.groovy | 66 +++-
.../plugins/plugin_query_timeout_debugger.groovy | 103 ++++-
.../alter_p2/test_alter_table_property.groovy | 2 +-
.../auth_call/test_dml_stream_load_auth.groovy | 8 +-
...est_bloom_filter_hit_with_renamed_column.groovy | 69 +---
.../cloud_p0/balance/test_balance_warm_up.groovy | 3 +
.../test_balance_warm_up_use_peer_cache.groovy | 3 +
...e_warm_up_with_compaction_use_peer_cache.groovy | 3 +
.../balance/test_peer_read_async_warmup.groovy | 3 +
.../cache/compaction/test_stale_rowset.groovy | 9 +-
.../cluster/test_immediate_warmup_basic.groovy | 3 +
.../test_immediate_warmup_multi_segments.groovy | 12 +-
.../cluster/test_warm_up_cluster_event.groovy | 3 +
.../test_warm_up_cluster_event_add_new_be.groovy | 3 +
...test_warm_up_cluster_event_cancel_active.groovy | 3 +
...est_warm_up_cluster_event_cancel_passive.groovy | 3 +
.../test_warm_up_cluster_event_compaction.groovy | 3 +
...rm_up_cluster_event_compaction_sync_wait.groovy | 12 +-
...uster_event_compaction_sync_wait_timeout.groovy | 12 +-
.../test_warm_up_cluster_event_rename.groovy | 3 +
...est_warm_up_cluster_event_restart_all_be.groovy | 3 +
..._warm_up_cluster_event_restart_master_fe.groovy | 3 +
...test_warm_up_cluster_event_schema_change.groovy | 3 +
.../cluster/test_warm_up_cluster_periodic.groovy | 3 +
...test_warm_up_cluster_periodic_add_new_be.groovy | 3 +
.../test_warm_up_cluster_periodic_and_event.groovy | 3 +
.../test_warm_up_cluster_periodic_rename.groovy | 3 +
...rm_up_cluster_periodic_restart_master_fe.groovy | 3 +
.../test_warm_up_cluster_periodic_slow_job.groovy | 3 +
.../test_mow_agg_delete_bitmap.groovy | 36 +-
.../test_enable_prefer_cached_rowset.groovy | 3 +
.../test_query_freshness_tolerance.groovy | 3 +
..._warmup_delay_compaction_query_tolerance.groovy | 12 +-
.../test_warmup_delay_idx_query_tolerance.groovy | 12 +-
.../test_warmup_delay_sc_query_tolerance.groovy | 12 +-
...delay_timeout_compaction_query_tolerance.groovy | 12 +-
.../warmup/test_warmup_download_fail.groovy | 12 +-
.../cloud_p0/test_read_cluster_var_property.groovy | 3 +
.../test_compaction_fail_release_lock.groovy | 22 +-
...test_compaction_update_big_delete_bitmap.groovy | 22 +-
...e_compaction_and_read_stale_cloud_docker.groovy | 22 +-
...ultisegments_and_read_stale_cloud_docker.groovy | 13 +-
.../test_filecache_with_base_compaction.groovy | 22 +-
...ilecache_with_base_compaction_thresthold.groovy | 22 +-
.../test_mow_compact_multi_segments.groovy | 13 +-
...paction_agg_and_remove_pre_delete_bitmap.groovy | 22 +-
.../test_mow_compaction_and_read_stale.groovy | 22 +-
...w_compaction_and_read_stale_cloud_docker.groovy | 22 +-
...test_mow_compaction_and_rowset_not_exist.groovy | 22 +-
.../test_mow_compaction_and_schema_change.groovy | 22 +-
.../test_mow_delete_unused_rowset_dm_docker.groovy | 22 +-
.../test_mow_stale_rowset_delete_bitmap.groovy | 22 +-
.../test_single_compaction_fault_injection.groovy | 22 +-
...e_compaction_with_variant_inverted_index.groovy | 22 +-
.../test_single_replica_compaction.groovy | 22 +-
.../date/test_agg_functions.groovy} | 38 +-
.../datetimev2/test_agg_functions.groovy | 54 +++
.../import-way/group-commit-manual.md.groovy | 15 +-
.../sql-functions/doc_date_functions_test.groovy | 35 ++
.../doc/table-design/auto-increment.md.groovy | 11 +-
.../dynamic-partitioning.md.groovy | 10 +-
.../index/ngram-bloomfilter-index.md.groovy | 8 +-
.../suites/export/test_array_export.groovy | 7 +-
.../suites/export/test_map_export.groovy | 7 +-
.../suites/export/test_struct_export.groovy | 7 +-
.../outfile/parquet/test_outfile_parquet.groovy | 8 +-
.../suites/export_p0/test_export_basic.groovy | 8 +-
.../suites/export_p0/test_export_csv.groovy | 7 +-
.../export_p0/test_export_data_consistency.groovy | 7 +-
.../export_p0/test_export_delete_disallow.groovy | 7 +-
.../export_p0/test_export_empty_table.groovy | 7 +-
.../test_export_table_with_label_retry.groovy | 7 +-
.../suites/export_p0/test_export_view.groovy | 7 +-
.../suites/export_p0/test_outfile.groovy | 8 +-
.../export_p0/test_outfile_csv_with_names.groovy | 8 +-
.../test_outfile_csv_with_names_and_types.groovy | 7 +-
.../test_outfile_csv_with_success_prefix.groovy | 7 +-
.../suites/export_p0/test_outfile_expr.groovy | 7 +-
.../suites/export_p0/test_outfile_separator.groovy | 7 +-
.../suites/export_p2/test_export_big_data.groovy | 7 +-
.../export/test_export_external_table.groovy | 190 +++++----
..._query_remote_doris_as_olap_table_select.groovy | 350 +++++++++++++++++
.../test_remote_doris_agg_table_select.groovy | 307 +++++++++++++++
.../test_remote_doris_all_types_select.groovy | 17 +-
.../test_remote_doris_all_types_show.groovy | 9 +-
.../remote_doris/test_remote_doris_catalog.groovy | 8 +-
.../remote_doris/test_remote_doris_predict.groovy | 8 +-
.../remote_doris/test_remote_doris_refresh.groovy | 8 +-
.../test_remote_doris_statistics.groovy | 8 +-
.../test_remote_doris_table_stats.groovy | 8 +-
.../test_remote_doris_unique_table_select.groovy | 238 +++++++++++
.../test_remote_doris_variant_select.groovy | 148 +++++++
.../test_cloud_mow_delete_bitmap_lock_case.groovy | 22 +-
.../test_delete_bitmap_lock_with_restart.groovy | 9 +-
.../test_mow_seq_seg_compaction.groovy | 103 -----
.../flink_connector_response.groovy | 11 +-
.../cast/test_cast_to_complex_types_strict.groovy | 6 +
.../http_rest_api/get/test_schema_api.groovy | 14 +
.../http_rest_api/post/test_query_stmt.groovy | 54 ++-
.../analyzer/test_custom_analyzer.groovy | 93 ++++-
..._index_change_with_cumulative_compaction.groovy | 12 +-
.../test_index_change_with_full_compaction.groovy | 12 +-
...est_cumulative_compaction_with_format_v2.groovy | 12 +-
.../test_mor_table_with_format_v2.groovy | 13 +-
.../test_mow_table_with_format_v2.groovy | 53 ++-
...single_replica_compaction_with_format_v2.groovy | 12 +-
.../test_arrayInvertedIdx_profile.groovy | 14 +-
.../test_array_with_large_dataset.groovy | 1 +
.../test_load_job_info_system_table.groovy | 74 ++++
.../stream_load/test_get_stream_load_state.groovy | 8 +-
...t_stream_load_with_nonexist_db_and_table.groovy | 3 +
.../stream_load/test_group_commit_wal_limit.groovy | 6 +
.../load_p0/stream_load/test_json_load.groovy | 2 +-
.../test_load_with_transfer_encoding.groovy | 3 +
.../stream_load/test_publish_timeout.groovy | 9 +-
.../test_s3_bytes_written_metrics.groovy | 3 +
.../load_p0/stream_load/test_stream_load.groovy | 175 +++++----
.../stream_load/test_stream_load_2pc.groovy | 16 +-
.../test_stream_load_2pc_with_schema_change.groovy | 3 +
.../test_stream_load_err_log_limit.groovy | 9 +-
.../stream_load/test_stream_load_properties.groovy | 2 +-
.../suites/manager/test_manager_interface_1.groovy | 10 +-
.../metrics_p0/test_delete_bitmap_metrics.groovy | 3 +
.../ssb/multiple_no_where/multiple_no_where.groovy | 2 +
.../mv_p0/ssb/multiple_ssb/multiple_ssb.groovy | 4 +
.../multiple_ssb_between.groovy | 6 +-
.../suites/mv_p0/ssb/q_1_1/q_1_1.groovy | 1 +
.../suites/mv_p0/ssb/q_2_1/q_2_1.groovy | 1 +
.../suites/mv_p0/ssb/q_3_1/q_3_1.groovy | 1 +
.../suites/mv_p0/ssb/q_4_1/q_4_1.groovy | 1 +
.../suites/mv_p0/ssb/q_4_1_r1/q_4_1_r1.groovy | 1 +
.../mysql_ssl_p0/test_mysql_connection.groovy | 40 +-
.../nereids_p0/aggregate/window_funnel.groovy | 3 +
.../authorization/column_authorization.groovy | 7 +-
.../authorization/view_authorization.groovy | 7 +-
.../suites/nereids_p0/hint/test_hint.groovy | 163 ++++++++
.../suites/nereids_p0/outfile/test_outfile.groovy | 7 +-
.../outfile/test_outfile_csv_with_names.groovy | 10 +-
.../nereids_p0/outfile/test_outfile_expr.groovy | 7 +-
.../nereids_p0/outfile/test_outfile_parquet.groovy | 10 +-
.../outfile/test_outfile_separator.groovy | 7 +-
...filter_through_join_with_unique_function.groovy | 84 ++++
.../suites/prepared_stmt_p0/prepared_stmt.groovy | 31 +-
.../query_p0/aggregate/test_regr_intercept.groovy | 18 +-
.../query_p0/aggregate/test_regr_slope.groovy | 18 +-
...ptive_pipeline_task_serial_read_on_limit.groovy | 39 +-
.../query_profile/s3_load_profile_test.groovy | 25 +-
.../suites/query_profile/scanner_profile.groovy | 70 +---
.../query_profile/test_execute_by_frontend.groovy | 31 +-
.../suites/query_profile/test_profile.groovy | 28 +-
292 files changed, 7725 insertions(+), 1735 deletions(-)
create mode 100644 be/src/exec/schema_scanner/schema_load_job_scanner.cpp
copy be/src/exec/schema_scanner/{schema_routine_load_job_scanner.h =>
schema_load_job_scanner.h} (83%)
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/datasource/doris/DorisExternalMetaCacheMgr.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/datasource/doris/FeServiceClient.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/datasource/doris/RemoteOlapTable.java
copy
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/{ToDays.java
=> ToSeconds.java} (77%)
copy regression-test/data/{data_model_p0/unique/test_sequence_column.out =>
datatype_p0/date/test_agg_functions.out} (52%)
create mode 100644
regression-test/data/datatype_p0/datetimev2/test_agg_functions.out
create mode 100644
regression-test/data/external_table_p0/remote_doris/test_query_remote_doris_as_olap_table_select.out
create mode 100644
regression-test/data/external_table_p0/remote_doris/test_remote_doris_agg_table_select.out
create mode 100644
regression-test/data/external_table_p0/remote_doris/test_remote_doris_unique_table_select.out
create mode 100644
regression-test/data/external_table_p0/remote_doris/test_remote_doris_variant_select.out
delete mode 100644
regression-test/data/fault_injection_p0/test_mow_seq_seg_compaction.out
create mode 100644
regression-test/data/nereids_rules_p0/unique_function/push_down_filter_through_join_with_unique_function.out
copy
regression-test/suites/{nereids_p0/create_table/test_create_table_like.groovy
=> datatype_p0/date/test_agg_functions.groovy} (58%)
create mode 100644
regression-test/suites/datatype_p0/datetimev2/test_agg_functions.groovy
create mode 100644
regression-test/suites/external_table_p0/remote_doris/test_query_remote_doris_as_olap_table_select.groovy
create mode 100644
regression-test/suites/external_table_p0/remote_doris/test_remote_doris_agg_table_select.groovy
create mode 100644
regression-test/suites/external_table_p0/remote_doris/test_remote_doris_unique_table_select.groovy
create mode 100644
regression-test/suites/external_table_p0/remote_doris/test_remote_doris_variant_select.groovy
delete mode 100644
regression-test/suites/fault_injection_p0/test_mow_seq_seg_compaction.groovy
create mode 100644
regression-test/suites/job_p0/job_system_table/test_load_job_info_system_table.groovy
create mode 100644
regression-test/suites/nereids_rules_p0/unique_function/push_down_filter_through_join_with_unique_function.groovy
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]