This is an automated email from the ASF dual-hosted git repository.
panxiaolei pushed a change to branch new_join2
in repository https://gitbox.apache.org/repos/asf/doris.git
discard d40a449c943 fix
discard 57160b4346f fix
discard 65258c607e0 update fix
discard f67d0b1635e [performance] opt the join and other join conjuncts
(#27604)
discard 75e9798f9d3 fix
discard 9f1db34e9c9 fix
discard 8b487c0734d fix
discard dac204d6e53 update
discard f7f982d7898 fix
discard c8d72228e63 update fix
discard c24fcd13cd1 fix compile
discard 7e6fe0b58dc [Rf](bf) fix bug of in change bf (#27355)
discard c2a3a36ed14 fix outer join and other join conjuncts (#27319)
discard e5fb35abb35 Join rewrite (#26140)
add dc1a31715ba [doc](flink) Update doc index title (#27410)
add 612347f650e [fix](planner)sort node should materialized required slots
for itself (#27605)
add cbdb886b6e8 [fix](Nereids): fill up miss slot of order having project
(#27480)
add 3d0dc94b180 [fix](ci) fix bug that "run build\n" not trigger pipeline
(#27617)
add fde4bab048d [fix](Nereids) non-deterministic expression should not be
constant (#27606)
add 66eeafcd484 [refactor](Nereids): unify one DateLiteral init() (#27618)
add d5a56dc7f4c [information_schema](tables)modify
information_schema.tables rows column use cache rows. (#27028)
add 50c442fc6c2 [DOC](sparkload)add spark load faq (#27455)
add 36a528b6bca [fix](judge-partition) Fix incorrect logic in determining
whether it is a partitioned table (#27515)
add 3d7d166355a [feature](cmd) add UNSET_VARIABLE statement to set back
variables (#27552)
add c7b9a32e3ea [improvement](show snapshot) show iceberg snapshot print
summary info (#27448)
add 646f1ea087a [performance](Nereids): avoid use `getStringValue()` in
getTimeFormatter() (#27625)
add bb68900bedb [fix](bdbje) Fix bdbje logging level not work (#27597)
add 4ea69ed3902 [regression test](broker load) add case for num_as_string
(#27588)
add 6a1c98af82b [regression case](broker laod) add case for without seq
(#27586)
add 7ac97c16500 [fix](bdbje) add free disk config (#27578)
add 2076d2b3907 [Fix](statistics)Fix bug and improve auto analyze. (#27626)
add 5bdfaf64479 [improve](metrics)Display garbage collector type (#27408)
add ea7eca9345c [pipelineX](bug) Add some logs (#27596)
add b48c40ed31b Make blockschduler first stop then delete (#27645)
add 65126459bd6 [deps](compression) Opt gzip decompress by libdeflate on
X86 and X86_64 platforms: 1. Add libdeflate lib. (#27542)
add c83e3318a87 (session) fix NereidsTracer shouldLog always true after
set enable_nereids_trace from true to false (#27420)
add 98c6885ae27 [opt](plan) only lock olap table when query plan (#27639)
add d1e163126c8 [regression] remove useless case (#27590)
add 9903c305919 [opt](nereids)adjust distribution cost for better choice
of broadcast join and shuffle join (#27113)
add fe7ff6f113d [Opt](functions) Opt tvf number for performance regression
framework (#27582)
add 4cfb9b73b8d [regression](partial update) Fix unstable p0 case
test_primary_key_partial_update_parallel due to conflicting table name (#27633)
add f329b906964 [fix](show_variables) fix default value for special
variables (#27651)
add fc2129a09f4 [fix](stats) skip collect agg_state type (#27640)
add f565f60bc3e [refactor](standard)BE:Initialize pointer variables in the
class to nullptr by default (#27587)
add ff0c0e82ba2 Join rewrite (#26140)
add 1bd2c009ed0 fix outer join and other join conjuncts (#27319)
add 0c82d151bd8 [Rf](bf) fix bug of in change bf (#27355)
add c016309b968 fix compile
add eb4a8bbd802 update fix
add c4aac825e6e fix
add b49f240eb25 update
add 3ca420eac8f fix
add 56f042562bc fix
add b4aca9e9bbc fix
add 3d680a9e2df [performance] opt the join and other join conjuncts
(#27604)
add b13c06f0736 update fix
add 3d135de1489 fix
add e57eb6e2787 fix
add 6c5ea6ed5d0 fix compile
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 (d40a449c943)
\
N -- N -- N refs/heads/new_join2 (6c5ea6ed5d0)
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:
.github/workflows/comment-to-trigger-teamcity.yml | 29 +-
be/src/agent/heartbeat_server.h | 4 +-
be/src/agent/task_worker_pool.h | 4 +-
be/src/agent/workload_group_listener.h | 2 +-
be/src/common/dwarf.h | 14 +-
be/src/common/elf.h | 8 +-
be/src/common/object_pool.h | 2 +-
be/src/common/signal_handler.h | 6 +-
be/src/common/symbol_index.h | 10 +-
be/src/exec/decompressor.h | 2 +-
be/src/exec/exec_node.h | 18 +-
be/src/exec/odbc_connector.h | 2 +-
be/src/exec/scan_node.h | 8 +-
be/src/exec/schema_scanner.h | 22 +-
.../exec/schema_scanner/schema_charsets_scanner.h | 6 +-
.../schema_scanner/schema_collations_scanner.h | 8 +-
.../exec/schema_scanner/schema_variables_scanner.h | 4 +-
be/src/exec/table_connector.h | 2 +-
be/src/exprs/function_filter.h | 2 +-
be/src/exprs/hybrid_map.h | 2 +-
be/src/exprs/runtime_filter.h | 16 +-
be/src/geo/ByteOrderDataInStream.h | 4 +-
be/src/geo/geo_tobinary_type.h | 2 +-
be/src/gutil/gscoped_ptr.h | 4 +-
be/src/gutil/move.h | 2 +-
be/src/gutil/port.h | 2 +-
be/src/gutil/ref_counted.h | 2 +-
be/src/gutil/stl_util.h | 14 +-
be/src/gutil/strings/numbers.h | 4 +-
be/src/gutil/strings/stringpiece.h | 2 +-
be/src/gutil/strings/substitute.h | 2 +-
be/src/gutil/threading/thread_collision_warner.h | 8 +-
be/src/http/action/download_action.h | 2 +-
be/src/http/action/download_binlog_action.h | 2 +-
be/src/io/cache/block/block_file_segment.h | 2 +-
be/src/io/cache/block/block_lru_file_cache.h | 2 +-
be/src/io/fs/broker_file_writer.h | 2 +-
be/src/io/fs/buffered_reader.h | 22 +-
be/src/io/fs/hdfs_file_reader.h | 18 +-
be/src/io/fs/hdfs_file_system.h | 4 +-
be/src/io/fs/hdfs_file_writer.h | 2 +-
be/src/io/fs/multi_table_pipe.h | 2 +-
be/src/io/fs/s3_file_writer.h | 2 +-
be/src/olap/base_tablet.h | 10 +-
be/src/olap/block_column_predicate.h | 2 +-
be/src/olap/column_mapping.h | 4 +-
be/src/olap/data_dir.h | 20 +-
be/src/olap/delta_writer_context.h | 2 +-
be/src/olap/hll.h | 10 +-
be/src/olap/inverted_index_parser.h | 2 +-
be/src/olap/iterators.h | 4 +-
be/src/olap/like_column_predicate.h | 2 +-
be/src/olap/lru_cache.h | 10 +-
be/src/olap/memtable.h | 10 +-
be/src/olap/memtable_flush_executor.h | 2 +-
be/src/olap/page_cache.h | 8 +-
be/src/olap/push_handler.h | 6 +-
be/src/olap/reader.h | 2 +-
be/src/olap/row_cursor.h | 2 +-
be/src/olap/row_cursor_cell.h | 2 +-
be/src/olap/rowset/beta_rowset_reader.h | 4 +-
be/src/olap/rowset/pending_rowset_helper.h | 2 +-
be/src/olap/rowset/segcompaction.h | 2 +-
be/src/olap/rowset/segment_creator.h | 6 +-
be/src/olap/rowset/segment_v2/binary_dict_page.h | 3 +-
.../olap/rowset/segment_v2/bitmap_index_reader.h | 4 +-
be/src/olap/rowset/segment_v2/bloom_filter.h | 4 +-
.../rowset/segment_v2/bloom_filter_index_reader.h | 6 +-
.../rowset/segment_v2/bloom_filter_index_writer.h | 2 +-
be/src/olap/rowset/segment_v2/column_reader.h | 10 +-
be/src/olap/rowset/segment_v2/column_writer.h | 2 +-
be/src/olap/rowset/segment_v2/encoding_info.h | 2 +-
be/src/olap/rowset/segment_v2/index_page.h | 2 +-
.../olap/rowset/segment_v2/indexed_column_reader.h | 2 +-
.../olap/rowset/segment_v2/indexed_column_writer.h | 8 +-
.../olap/rowset/segment_v2/inverted_index_cache.h | 2 +-
.../segment_v2/inverted_index_compound_directory.h | 6 +-
.../segment_v2/inverted_index_compound_reader.h | 8 +-
.../olap/rowset/segment_v2/inverted_index_reader.h | 4 +-
be/src/olap/rowset/segment_v2/ordinal_page_index.h | 2 +-
be/src/olap/rowset/segment_v2/parsed_page.h | 2 +-
be/src/olap/rowset/segment_v2/rle_page.h | 2 +-
be/src/olap/rowset/segment_v2/segment_iterator.h | 2 +-
be/src/olap/rowset/segment_v2/segment_writer.h | 4 +-
.../rowset/segment_v2/vertical_segment_writer.h | 4 +-
be/src/olap/rowset/segment_v2/zone_map_index.h | 2 +-
be/src/olap/segment_loader.h | 2 +-
be/src/olap/selection_vector.h | 2 +-
be/src/olap/short_key_index.h | 2 +-
be/src/olap/skiplist.h | 6 +-
be/src/olap/storage_engine.h | 2 +-
be/src/olap/tablet.h | 2 +-
be/src/olap/task/engine_clone_task.h | 2 +-
be/src/olap/task/engine_publish_version_task.h | 8 +-
be/src/olap/task/engine_storage_migration_task.h | 2 +-
be/src/olap/txn_manager.h | 14 +-
be/src/olap/wal_manager.h | 2 +-
be/src/pipeline/exec/aggregation_sink_operator.h | 40 +-
be/src/pipeline/exec/aggregation_source_operator.h | 14 +-
be/src/pipeline/exec/analytic_sink_operator.h | 6 +-
be/src/pipeline/exec/analytic_source_operator.h | 10 +-
be/src/pipeline/exec/datagen_operator.h | 2 +-
.../distinct_streaming_aggregation_sink_operator.h | 2 +-
be/src/pipeline/exec/es_scan_operator.h | 8 +-
be/src/pipeline/exec/exchange_sink_buffer.h | 19 +-
be/src/pipeline/exec/exchange_sink_operator.cpp | 9 +
be/src/pipeline/exec/exchange_sink_operator.h | 7 +-
be/src/pipeline/exec/exchange_source_operator.cpp | 24 +
be/src/pipeline/exec/exchange_source_operator.h | 3 +
be/src/pipeline/exec/hashjoin_build_sink.h | 24 +-
be/src/pipeline/exec/hashjoin_probe_operator.h | 16 +-
be/src/pipeline/exec/join_build_sink_operator.h | 6 +-
be/src/pipeline/exec/join_probe_operator.h | 8 +-
be/src/pipeline/exec/multi_cast_data_streamer.h | 6 +-
.../exec/nested_loop_join_probe_operator.h | 2 +-
be/src/pipeline/exec/olap_table_sink_operator.h | 2 +-
be/src/pipeline/exec/olap_table_sink_v2_operator.h | 2 +-
be/src/pipeline/exec/operator.h | 10 +-
.../pipeline/exec/partition_sort_sink_operator.h | 10 +-
be/src/pipeline/exec/repeat_operator.h | 4 +-
be/src/pipeline/exec/result_file_sink_operator.h | 12 +-
be/src/pipeline/exec/result_sink_operator.h | 2 +-
be/src/pipeline/exec/scan_operator.h | 14 +-
be/src/pipeline/exec/schema_scan_operator.h | 6 +-
be/src/pipeline/exec/set_sink_operator.h | 2 +-
be/src/pipeline/exec/sort_sink_operator.h | 4 +-
.../exec/streaming_aggregation_sink_operator.h | 10 +-
be/src/pipeline/pipeline_fragment_context.h | 6 +-
be/src/pipeline/pipeline_task.h | 60 +-
be/src/pipeline/pipeline_x/dependency.h | 19 +-
be/src/pipeline/pipeline_x/operator.h | 44 +-
be/src/pipeline/pipeline_x/pipeline_x_task.cpp | 10 +-
be/src/pipeline/pipeline_x/pipeline_x_task.h | 4 +-
be/src/runtime/broker_mgr.h | 2 +-
be/src/runtime/buffer_control_block.h | 2 +-
be/src/runtime/cache/result_cache.h | 4 +-
be/src/runtime/cache/result_node.h | 6 +-
be/src/runtime/client_cache.h | 6 +-
be/src/runtime/collection_value.h | 4 +-
be/src/runtime/descriptors.h | 2 +-
be/src/runtime/exec_env.h | 12 +-
be/src/runtime/exec_env_init.cpp | 26 +-
be/src/runtime/external_scan_context_mgr.h | 2 +-
be/src/runtime/fragment_mgr.h | 7 +-
be/src/runtime/group_commit_mgr.h | 6 +-
be/src/runtime/load_channel.h | 2 +-
be/src/runtime/load_path_mgr.h | 2 +-
be/src/runtime/load_stream.h | 4 +-
be/src/runtime/load_stream_mgr.h | 4 +-
be/src/runtime/load_stream_writer.h | 2 +-
be/src/runtime/map_value.h | 4 +-
be/src/runtime/memory/mem_tracker.h | 2 +-
be/src/runtime/plan_fragment_executor.h | 12 +-
be/src/runtime/query_context.h | 8 +-
.../routine_load/routine_load_task_executor.h | 2 +-
be/src/runtime/runtime_filter_mgr.h | 10 +-
be/src/runtime/runtime_predicate.h | 6 +-
be/src/runtime/runtime_state.h | 4 +-
be/src/runtime/small_file_mgr.h | 2 +-
be/src/runtime/snapshot_loader.h | 2 +-
be/src/runtime/stream_load/stream_load_context.h | 2 +-
be/src/runtime/stream_load/stream_load_executor.h | 2 +-
be/src/runtime/stream_load/stream_load_recorder.h | 2 +-
be/src/runtime/struct_value.h | 2 +-
be/src/runtime/tablets_channel.h | 2 +-
be/src/runtime/task_group/task_group.h | 4 +-
be/src/runtime/thread_context.h | 8 +-
.../service/arrow_flight/auth_server_middleware.h | 2 +-
be/src/service/backend_service.h | 2 +-
be/src/service/http_service.h | 4 +-
be/src/service/internal_service.h | 2 +-
be/src/service/point_query_executor.h | 8 +-
be/src/udf/udf.h | 2 +-
be/src/util/bfd_parser.h | 4 +-
be/src/util/bit_stream_utils.h | 4 +-
be/src/util/bitmap.h | 2 +-
be/src/util/countdown_latch.h | 2 +-
be/src/util/doris_metrics.h | 334 ++--
be/src/util/easy_json.h | 2 +-
be/src/util/faststring.h | 2 +-
be/src/util/frame_of_reference_coding.h | 2 +-
be/src/util/interval_tree-inl.h | 4 +-
be/src/util/interval_tree.h | 2 +-
be/src/util/jni-util.h | 8 +-
be/src/util/jsonb_document.h | 6 +-
be/src/util/jsonb_stream.h | 2 +-
be/src/util/jsonb_updater.h | 4 +-
be/src/util/jsonb_writer.h | 2 +-
be/src/util/lru_multi_cache.h | 2 +-
be/src/util/mem_range.h | 2 +-
be/src/util/mysql_row_buffer.h | 4 +-
be/src/util/obj_lru_cache.h | 2 +-
be/src/util/runtime_profile.h | 16 +-
be/src/util/slice.h | 2 +-
be/src/util/sm3.h | 4 +-
be/src/util/system_metrics.h | 8 +-
be/src/util/thread.h | 2 +-
be/src/util/threadpool.h | 2 +-
be/src/util/thrift_server.h | 6 +-
.../aggregate_function_collect.h | 8 +-
.../aggregate_function_percentile_approx.h | 2 +-
.../aggregate_functions/aggregate_function_sort.h | 2 +-
be/src/vec/columns/column_vector.h | 12 +-
be/src/vec/common/arena.h | 10 +-
be/src/vec/common/columns_hashing_impl.h | 2 +-
be/src/vec/common/cow.h | 2 +-
be/src/vec/common/hash_table/hash_map_context.h | 2 +-
be/src/vec/common/hash_table/hash_table.h | 8 +-
.../vec/common/hash_table/hash_table_set_build.h | 4 +-
.../vec/common/hash_table/hash_table_set_probe.h | 2 +-
be/src/vec/common/hash_table/string_hash_table.h | 4 +-
be/src/vec/common/sort/partition_sorter.h | 4 +-
be/src/vec/common/sort/sorter.h | 10 +-
be/src/vec/core/block_spill_reader.h | 8 +-
be/src/vec/core/block_spill_writer.h | 8 +-
be/src/vec/core/sort_cursor.h | 2 +-
.../data_gen_functions/vdata_gen_function_inf.h | 2 +-
.../vec/exec/data_gen_functions/vnumbers_tvf.cpp | 67 +-
be/src/vec/exec/data_gen_functions/vnumbers_tvf.h | 2 +-
be/src/vec/exec/format/avro/avro_jni_reader.h | 6 +-
be/src/vec/exec/format/csv/csv_reader.h | 8 +-
.../file_reader/new_plain_text_line_reader.h | 16 +-
be/src/vec/exec/format/format_common.h | 2 +-
be/src/vec/exec/format/jni_reader.h | 4 +-
be/src/vec/exec/format/json/new_json_reader.h | 18 +-
be/src/vec/exec/format/orc/vorc_reader.h | 40 +-
be/src/vec/exec/format/parquet/decoder.h | 4 +-
.../exec/format/parquet/parquet_column_convert.h | 2 +-
.../format/parquet/vparquet_column_chunk_reader.h | 8 +-
.../exec/format/parquet/vparquet_column_reader.h | 12 +-
.../exec/format/parquet/vparquet_group_reader.h | 16 +-
.../vec/exec/format/parquet/vparquet_page_reader.h | 4 +-
be/src/vec/exec/format/parquet/vparquet_reader.h | 82 +-
.../vec/exec/format/table/max_compute_jni_reader.h | 10 +-
be/src/vec/exec/format/table/paimon_reader.h | 4 +-
.../exec/format/table/transactional_hive_reader.h | 14 +-
be/src/vec/exec/format/wal/wal_reader.h | 4 +-
be/src/vec/exec/jni_connector.h | 12 +-
be/src/vec/exec/join/join_op.h | 6 +-
be/src/vec/exec/join/process_hash_table_probe.h | 22 +-
be/src/vec/exec/join/vhash_join_node.h | 50 +-
be/src/vec/exec/join/vjoin_node_base.h | 24 +-
be/src/vec/exec/join/vnested_loop_join_node.h | 4 +-
be/src/vec/exec/runtime_filter_consumer.h | 4 +-
be/src/vec/exec/scan/mysql_scanner.h | 4 +-
be/src/vec/exec/scan/new_es_scan_node.h | 10 +-
be/src/vec/exec/scan/new_es_scanner.h | 2 +-
be/src/vec/exec/scan/new_jdbc_scanner.h | 2 +-
be/src/vec/exec/scan/new_odbc_scanner.h | 2 +-
be/src/vec/exec/scan/new_olap_scanner.h | 4 +-
be/src/vec/exec/scan/scan_task_queue.h | 2 +-
be/src/vec/exec/scan/scanner_context.h | 12 +-
be/src/vec/exec/scan/scanner_scheduler.h | 6 +-
be/src/vec/exec/scan/vfile_scanner.h | 10 +-
be/src/vec/exec/scan/vmeta_scanner.h | 2 +-
be/src/vec/exec/scan/vscan_node.h | 14 +-
be/src/vec/exec/scan/vscanner.h | 8 +-
be/src/vec/exec/vaggregation_node.h | 50 +-
be/src/vec/exec/vanalytic_eval_node.h | 2 +-
be/src/vec/exec/vdata_gen_scan_node.h | 2 +-
be/src/vec/exec/vjdbc_connector.h | 2 +-
be/src/vec/exec/vmysql_scan_node.h | 2 +-
be/src/vec/exec/vpartition_sort_node.h | 14 +-
be/src/vec/exec/vschema_scan_node.h | 2 +-
be/src/vec/exec/vset_operation_node.h | 6 +-
be/src/vec/exec/vsort_node.h | 4 +-
be/src/vec/exprs/table_function/vexplode_bitmap.h | 2 +-
be/src/vec/exprs/vectorized_agg_fn.h | 10 +-
be/src/vec/exprs/vslot_ref.h | 2 +-
be/src/vec/functions/array/function_array_remove.h | 4 +-
be/src/vec/functions/function_cast.h | 2 +-
be/src/vec/functions/regexps.h | 2 +-
be/src/vec/io/reader_buffer.h | 4 +-
be/src/vec/olap/vcollect_iterator.h | 2 +-
be/src/vec/runtime/vcsv_transformer.h | 2 +-
be/src/vec/runtime/vdata_stream_recvr.h | 40 +-
be/src/vec/runtime/vfile_format_transformer.h | 2 +-
be/src/vec/runtime/vorc_transformer.h | 6 +-
be/src/vec/runtime/vparquet_transformer.h | 4 +-
be/src/vec/runtime/vsorted_run_merger.h | 4 +-
be/src/vec/sink/delta_writer_v2_pool.h | 2 +-
be/src/vec/sink/load_stream_stub.h | 2 +-
be/src/vec/sink/load_stream_stub_pool.h | 2 +-
be/src/vec/sink/varrow_flight_result_writer.h | 4 +-
be/src/vec/sink/vdata_stream_sender.h | 18 +-
be/src/vec/sink/vmysql_result_writer.h | 2 +-
be/src/vec/sink/vresult_file_sink.h | 2 +-
be/src/vec/sink/vrow_distribution.h | 24 +-
be/src/vec/sink/vtablet_block_convertor.h | 2 +-
be/src/vec/sink/vtablet_finder.h | 2 +-
be/src/vec/sink/vtablet_sink.h | 2 +-
be/src/vec/sink/vtablet_sink_v2.h | 2 +-
be/src/vec/sink/writer/vfile_result_writer.h | 2 +-
be/src/vec/sink/writer/vmysql_table_writer.h | 2 +-
be/src/vec/sink/writer/vtablet_writer.h | 10 +-
be/src/vec/sink/writer/vtablet_writer_v2.h | 2 +-
be/src/vec/sink/writer/vwal_writer.h | 2 +-
be/test/testutil/desc_tbl_builder.h | 2 +-
.../admin-manual/maint-monitor/monitor-alert.md | 12 +-
.../import/import-way/spark-load-manual.md | 64 +-
docs/en/docs/ecosystem/flink-doris-connector.md | 3 +-
.../sql-functions/table-functions/iceberg-meta.md | 24 +-
.../admin-manual/maint-monitor/monitor-alert.md | 12 +-
.../maint-monitor/monitor-metrics/metrics.md | 513 +++--
.../import/import-way/spark-load-manual.md | 28 +-
docs/zh-CN/docs/ecosystem/flink-doris-connector.md | 10 +-
.../sql-functions/table-functions/iceberg-meta.md | 24 +-
.../main/java/org/apache/doris/common/Config.java | 10 +-
fe/fe-core/src/main/cup/sql_parser.cup | 22 +-
.../org/apache/doris/analysis/AnalyzeTblStmt.java | 3 +-
.../org/apache/doris/analysis/DateLiteral.java | 201 +-
.../apache/doris/analysis/UnsetVariableStmt.java | 97 +
.../java/org/apache/doris/catalog/OlapTable.java | 27 +-
.../main/java/org/apache/doris/catalog/Table.java | 4 +
.../java/org/apache/doris/catalog/TableIf.java | 13 +
.../doris/catalog/external/ExternalTable.java | 19 +-
.../doris/catalog/external/HMSExternalTable.java | 17 +
.../doris/catalog/external/JdbcExternalTable.java | 5 +
.../doris/datasource/hive/HiveMetaStoreCache.java | 5 +-
.../apache/doris/journal/bdbje/BDBEnvironment.java | 5 +
.../org/apache/doris/metric/JsonMetricVisitor.java | 64 +-
.../doris/metric/PrometheusMetricVisitor.java | 29 +-
.../java/org/apache/doris/monitor/jvm/GcNames.java | 12 -
.../org/apache/doris/monitor/jvm/JvmStats.java | 2 +-
.../org/apache/doris/nereids/CascadesContext.java | 3 +
.../org/apache/doris/nereids/NereidsPlanner.java | 2 +
.../org/apache/doris/nereids/cost/CostModelV1.java | 11 +-
.../doris/nereids/minidump/NereidsTracer.java | 4 +
.../org/apache/doris/nereids/rules/RuleType.java | 1 +
.../nereids/rules/analysis/FillUpMissingSlots.java | 20 +
.../nereids/trees/expressions/Expression.java | 3 +-
.../org/apache/doris/planner/JoinNodeBase.java | 3 -
.../java/org/apache/doris/planner/SortNode.java | 7 +
.../java/org/apache/doris/qe/SessionVariable.java | 9 +
.../java/org/apache/doris/qe/ShowExecutor.java | 79 +-
.../java/org/apache/doris/qe/StmtExecutor.java | 28 +
.../main/java/org/apache/doris/qe/VariableMgr.java | 45 +-
.../doris/rewrite/RewriteDateLiteralRule.java | 3 +-
.../apache/doris/service/FrontendServiceImpl.java | 2 +-
.../apache/doris/statistics/AnalysisManager.java | 23 +
.../doris/statistics/StatisticConstants.java | 2 +
.../org/apache/doris/statistics/Statistics.java | 4 +-
.../doris/statistics/StatisticsAutoCollector.java | 19 +-
.../doris/statistics/StatisticsCollector.java | 2 +-
.../doris/statistics/util/StatisticsUtil.java | 14 +-
.../tablefunction/IcebergTableValuedFunction.java | 3 +-
.../doris/tablefunction/MetadataGenerator.java | 1 +
fe/fe-core/src/main/jflex/sql_scanner.flex | 2 +
.../java/org/apache/doris/metric/MetricsTest.java | 44 +
.../rules/analysis/FillUpMissingSlotsTest.java | 7 +
pytest/qe/palo2/src/test_query_sql_mode.py | 235 ---
pytest/qe/palo2/src/test_query_window_slow.py | 2084 --------------------
pytest/qe/query_regression/result/sample.result | 20 -
pytest/qe/query_regression/sql/sample.sql | 18 -
.../broker_load/broker_load_with_properties.json | 6 +
..._with_where.csv => broker_load_without_seq.csv} | 0
.../test_broker_load_with_properties.out | 10 +
.../data/nereids_hint_tpch_p0/shape/q16.out | 15 +-
.../data/nereids_hint_tpch_p0/shape/q5.out | 55 +-
.../data/nereids_hint_tpch_p0/shape/q9.out | 5 +-
regression-test/data/nereids_shape_check/load.out | 37 +
.../data/nereids_ssb_shape_sf100_p0/shape/flat.out | 17 +
.../data/nereids_ssb_shape_sf100_p0/shape/q2.2.out | 24 +-
.../data/nereids_ssb_shape_sf100_p0/shape/q2.3.out | 24 +-
.../data/nereids_ssb_shape_sf100_p0/shape/q3.3.out | 24 +-
.../data/nereids_ssb_shape_sf100_p0/shape/q3.4.out | 29 +-
.../shape/query10.out | 3 +-
.../shape/query13.out | 47 +-
.../shape/query14.out | 18 +-
.../shape/query20.out | 13 +-
.../shape/query33.out | 15 +-
.../shape/query48.out | 41 +-
.../nereids_tpcds_shape_sf1000_p0/shape/query5.out | 23 +-
.../shape/query53.out | 27 +-
.../shape/query54.out | 54 +-
.../shape/query56.out | 15 +-
.../shape/query59.out | 35 +-
.../nereids_tpcds_shape_sf1000_p0/shape/query6.out | 35 +-
.../shape/query60.out | 15 +-
.../shape/query63.out | 27 +-
.../shape/query64.out | 32 +-
.../shape/query70.out | 5 +-
.../shape/query71.out | 61 +-
.../shape/query77.out | 29 +-
.../shape/query84.out | 20 +-
.../shape/query85.out | 64 +-
.../shape/query86.out | 15 +-
.../nereids_tpcds_shape_sf1000_p0/shape/query9.out | 41 +-
.../noStatsRfPrune/query1.out | 31 +-
.../noStatsRfPrune/query10.out | 20 +-
.../noStatsRfPrune/query12.out | 5 +-
.../noStatsRfPrune/query13.out | 41 +-
.../noStatsRfPrune/query14.out | 84 +-
.../noStatsRfPrune/query16.out | 49 +-
.../noStatsRfPrune/query17.out | 59 +-
.../noStatsRfPrune/query18.out | 51 +-
.../noStatsRfPrune/query19.out | 24 +-
.../noStatsRfPrune/query2.out | 19 +-
.../noStatsRfPrune/query20.out | 5 +-
.../noStatsRfPrune/query22.out | 18 +-
.../noStatsRfPrune/query23.out | 85 +-
.../noStatsRfPrune/query24.out | 23 +-
.../noStatsRfPrune/query25.out | 59 +-
.../noStatsRfPrune/query26.out | 25 +-
.../noStatsRfPrune/query29.out | 51 +-
.../noStatsRfPrune/query30.out | 60 +-
.../noStatsRfPrune/query31.out | 5 +-
.../noStatsRfPrune/query32.out | 26 +-
.../noStatsRfPrune/query33.out | 63 +-
.../noStatsRfPrune/query34.out | 5 +-
.../noStatsRfPrune/query35.out | 24 +-
.../noStatsRfPrune/query36.out | 18 +-
.../noStatsRfPrune/query38.out | 22 +-
.../noStatsRfPrune/query39.out | 33 +-
.../noStatsRfPrune/query41.out | 20 +-
.../noStatsRfPrune/query45.out | 38 +-
.../noStatsRfPrune/query46.out | 47 +-
.../noStatsRfPrune/query47.out | 5 +-
.../noStatsRfPrune/query48.out | 43 +-
.../noStatsRfPrune/query49.out | 34 +-
.../noStatsRfPrune/query54.out | 113 +-
.../noStatsRfPrune/query55.out | 20 +-
.../noStatsRfPrune/query56.out | 78 +-
.../noStatsRfPrune/query57.out | 5 +-
.../noStatsRfPrune/query58.out | 37 +-
.../noStatsRfPrune/query6.out | 29 +-
.../noStatsRfPrune/query60.out | 63 +-
.../noStatsRfPrune/query61.out | 107 +-
.../noStatsRfPrune/query64.out | 166 +-
.../noStatsRfPrune/query65.out | 51 +-
.../noStatsRfPrune/query66.out | 12 +-
.../noStatsRfPrune/query67.out | 21 +-
.../noStatsRfPrune/query68.out | 47 +-
.../noStatsRfPrune/query69.out | 65 +-
.../noStatsRfPrune/query72.out | 43 +-
.../noStatsRfPrune/query73.out | 5 +-
.../noStatsRfPrune/query75.out | 58 +-
.../noStatsRfPrune/query76.out | 60 +-
.../noStatsRfPrune/query77.out | 15 +-
.../noStatsRfPrune/query78.out | 67 +-
.../noStatsRfPrune/query80.out | 21 +-
.../noStatsRfPrune/query81.out | 46 +-
.../noStatsRfPrune/query83.out | 63 +-
.../noStatsRfPrune/query84.out | 50 +-
.../noStatsRfPrune/query85.out | 63 +-
.../noStatsRfPrune/query86.out | 18 +-
.../noStatsRfPrune/query87.out | 22 +-
.../noStatsRfPrune/query90.out | 40 +-
.../noStatsRfPrune/query91.out | 39 +-
.../noStatsRfPrune/query92.out | 26 +-
.../noStatsRfPrune/query94.out | 49 +-
.../noStatsRfPrune/query95.out | 69 +-
.../no_stats_shape/query1.out | 31 +-
.../no_stats_shape/query10.out | 20 +-
.../no_stats_shape/query12.out | 5 +-
.../no_stats_shape/query13.out | 41 +-
.../no_stats_shape/query14.out | 84 +-
.../no_stats_shape/query16.out | 49 +-
.../no_stats_shape/query17.out | 59 +-
.../no_stats_shape/query18.out | 51 +-
.../no_stats_shape/query19.out | 28 +-
.../no_stats_shape/query2.out | 21 +-
.../no_stats_shape/query20.out | 5 +-
.../no_stats_shape/query22.out | 18 +-
.../no_stats_shape/query23.out | 85 +-
.../no_stats_shape/query24.out | 23 +-
.../no_stats_shape/query25.out | 59 +-
.../no_stats_shape/query26.out | 25 +-
.../no_stats_shape/query29.out | 51 +-
.../no_stats_shape/query30.out | 60 +-
.../no_stats_shape/query31.out | 5 +-
.../no_stats_shape/query32.out | 26 +-
.../no_stats_shape/query33.out | 63 +-
.../no_stats_shape/query34.out | 5 +-
.../no_stats_shape/query35.out | 24 +-
.../no_stats_shape/query36.out | 18 +-
.../no_stats_shape/query38.out | 22 +-
.../no_stats_shape/query39.out | 33 +-
.../no_stats_shape/query41.out | 20 +-
.../no_stats_shape/query45.out | 38 +-
.../no_stats_shape/query46.out | 47 +-
.../no_stats_shape/query47.out | 5 +-
.../no_stats_shape/query48.out | 43 +-
.../no_stats_shape/query49.out | 34 +-
.../no_stats_shape/query54.out | 113 +-
.../no_stats_shape/query55.out | 20 +-
.../no_stats_shape/query56.out | 78 +-
.../no_stats_shape/query57.out | 5 +-
.../no_stats_shape/query58.out | 37 +-
.../no_stats_shape/query6.out | 29 +-
.../no_stats_shape/query60.out | 63 +-
.../no_stats_shape/query61.out | 107 +-
.../no_stats_shape/query64.out | 166 +-
.../no_stats_shape/query65.out | 51 +-
.../no_stats_shape/query66.out | 10 +-
.../no_stats_shape/query67.out | 21 +-
.../no_stats_shape/query68.out | 47 +-
.../no_stats_shape/query69.out | 65 +-
.../no_stats_shape/query72.out | 43 +-
.../no_stats_shape/query73.out | 5 +-
.../no_stats_shape/query75.out | 58 +-
.../no_stats_shape/query76.out | 60 +-
.../no_stats_shape/query77.out | 15 +-
.../no_stats_shape/query78.out | 67 +-
.../no_stats_shape/query80.out | 21 +-
.../no_stats_shape/query81.out | 46 +-
.../no_stats_shape/query83.out | 63 +-
.../no_stats_shape/query84.out | 50 +-
.../no_stats_shape/query85.out | 63 +-
.../no_stats_shape/query86.out | 18 +-
.../no_stats_shape/query87.out | 22 +-
.../no_stats_shape/query90.out | 40 +-
.../no_stats_shape/query91.out | 39 +-
.../no_stats_shape/query92.out | 26 +-
.../no_stats_shape/query94.out | 49 +-
.../no_stats_shape/query95.out | 67 +-
.../rf_prune/query13.out | 32 +-
.../rf_prune/query14.out | 22 +-
.../rf_prune/query3.out | 13 +-
.../rf_prune/query33.out | 61 +-
.../rf_prune/query36.out | 15 +-
.../rf_prune/query48.out | 40 +-
.../rf_prune/query5.out | 23 +-
.../rf_prune/query50.out | 19 +-
.../rf_prune/query53.out | 28 +-
.../rf_prune/query54.out | 57 +-
.../rf_prune/query56.out | 60 +-
.../rf_prune/query59.out | 35 +-
.../rf_prune/query60.out | 102 +-
.../rf_prune/query61.out | 28 +-
.../rf_prune/query63.out | 28 +-
.../rf_prune/query64.out | 151 +-
.../rf_prune/query66.out | 46 +-
.../rf_prune/query69.out | 5 +-
.../rf_prune/query70.out | 5 +-
.../rf_prune/query72.out | 70 +-
.../rf_prune/query76.out | 22 +-
.../rf_prune/query77.out | 29 +-
.../rf_prune/query80.out | 134 +-
.../rf_prune/query81.out | 5 +-
.../rf_prune/query84.out | 15 +-
.../rf_prune/query85.out | 62 +-
.../rf_prune/query9.out | 41 +-
.../rf_prune/query90.out | 46 +-
.../nereids_tpcds_shape_sf100_p0/shape/query13.out | 32 +-
.../nereids_tpcds_shape_sf100_p0/shape/query14.out | 22 +-
.../nereids_tpcds_shape_sf100_p0/shape/query3.out | 13 +-
.../nereids_tpcds_shape_sf100_p0/shape/query33.out | 61 +-
.../nereids_tpcds_shape_sf100_p0/shape/query36.out | 15 +-
.../nereids_tpcds_shape_sf100_p0/shape/query48.out | 40 +-
.../nereids_tpcds_shape_sf100_p0/shape/query5.out | 23 +-
.../nereids_tpcds_shape_sf100_p0/shape/query50.out | 19 +-
.../nereids_tpcds_shape_sf100_p0/shape/query53.out | 28 +-
.../nereids_tpcds_shape_sf100_p0/shape/query54.out | 57 +-
.../nereids_tpcds_shape_sf100_p0/shape/query56.out | 60 +-
.../nereids_tpcds_shape_sf100_p0/shape/query59.out | 35 +-
.../nereids_tpcds_shape_sf100_p0/shape/query60.out | 102 +-
.../nereids_tpcds_shape_sf100_p0/shape/query61.out | 28 +-
.../nereids_tpcds_shape_sf100_p0/shape/query63.out | 28 +-
.../nereids_tpcds_shape_sf100_p0/shape/query64.out | 151 +-
.../nereids_tpcds_shape_sf100_p0/shape/query66.out | 46 +-
.../nereids_tpcds_shape_sf100_p0/shape/query69.out | 5 +-
.../nereids_tpcds_shape_sf100_p0/shape/query70.out | 5 +-
.../nereids_tpcds_shape_sf100_p0/shape/query72.out | 70 +-
.../nereids_tpcds_shape_sf100_p0/shape/query76.out | 22 +-
.../nereids_tpcds_shape_sf100_p0/shape/query77.out | 29 +-
.../nereids_tpcds_shape_sf100_p0/shape/query80.out | 134 +-
.../nereids_tpcds_shape_sf100_p0/shape/query81.out | 5 +-
.../nereids_tpcds_shape_sf100_p0/shape/query84.out | 15 +-
.../nereids_tpcds_shape_sf100_p0/shape/query85.out | 62 +-
.../nereids_tpcds_shape_sf100_p0/shape/query9.out | 41 +-
.../nereids_tpcds_shape_sf100_p0/shape/query90.out | 46 +-
.../nostats_rf_prune/q16.out | 21 +-
.../nostats_rf_prune/q20-rewrite.out | 47 +-
.../nostats_rf_prune/q20.out | 45 +-
.../nostats_rf_prune/q21.out | 42 +-
.../nostats_rf_prune/q5.out | 27 +-
.../nostats_rf_prune/q8.out | 45 +-
.../nostats_rf_prune/q9.out | 33 +-
.../nereids_tpch_shape_sf1000_p0/rf_prune/q2.out | 5 +-
.../nereids_tpch_shape_sf1000_p0/rf_prune/q5.out | 5 +-
.../nereids_tpch_shape_sf1000_p0/rf_prune/q8.out | 5 +-
.../data/nereids_tpch_shape_sf1000_p0/shape/q2.out | 5 +-
.../data/nereids_tpch_shape_sf1000_p0/shape/q5.out | 5 +-
.../data/nereids_tpch_shape_sf1000_p0/shape/q8.out | 5 +-
.../shape_no_stats/q16.out | 21 +-
.../shape_no_stats/q20-rewrite.out | 47 +-
.../shape_no_stats/q20.out | 45 +-
.../shape_no_stats/q21.out | 42 +-
.../shape_no_stats/q5.out | 27 +-
.../shape_no_stats/q8.out | 45 +-
.../shape_no_stats/q9.out | 33 +-
.../data/variable_p0/set_and_unset_variable.out | 157 ++
.../test_broker_load_with_properties.groovy | 118 ++
.../test_broker_load_without_seq.groovy | 101 +
.../suites/metrics_p0/test_fe_metrics.groovy | 15 +-
.../suites/nereids_hint_tpch_p0/shape/q5.groovy | 2 +
.../suites/nereids_shape_check/load.groovy | 82 +
.../nereids_ssb_shape_sf100_p0/shape/flat.groovy | 102 +
.../suites/statistics/test_agg_complex_type.groovy | 8 +-
.../test_partial_update_parallel.groovy | 10 +-
.../variable_p0/set_and_unset_variable.groovy | 84 +
samples/connect/cpp/doris_client.h | 2 +-
thirdparty/build-thirdparty.sh | 15 +
thirdparty/vars.sh | 7 +
605 files changed, 7225 insertions(+), 8971 deletions(-)
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/analysis/UnsetVariableStmt.java
delete mode 100644 pytest/qe/palo2/src/test_query_sql_mode.py
delete mode 100644 pytest/qe/palo2/src/test_query_window_slow.py
delete mode 100644 pytest/qe/query_regression/result/sample.result
delete mode 100644 pytest/qe/query_regression/sql/sample.sql
create mode 100644
regression-test/data/load_p0/broker_load/broker_load_with_properties.json
copy regression-test/data/load_p0/broker_load/{broker_load_with_where.csv =>
broker_load_without_seq.csv} (100%)
create mode 100644
regression-test/data/load_p0/broker_load/test_broker_load_with_properties.out
create mode 100644 regression-test/data/nereids_shape_check/load.out
create mode 100644
regression-test/data/nereids_ssb_shape_sf100_p0/shape/flat.out
create mode 100644 regression-test/data/variable_p0/set_and_unset_variable.out
create mode 100644
regression-test/suites/load_p0/broker_load/test_broker_load_with_properties.groovy
create mode 100644
regression-test/suites/load_p0/broker_load/test_broker_load_without_seq.groovy
create mode 100644 regression-test/suites/nereids_shape_check/load.groovy
create mode 100644
regression-test/suites/nereids_ssb_shape_sf100_p0/shape/flat.groovy
create mode 100644
regression-test/suites/variable_p0/set_and_unset_variable.groovy
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]