This is an automated email from the ASF dual-hosted git repository.
mrhhsg pushed a change to branch deduplicate_keys
in repository https://gitbox.apache.org/repos/asf/doris.git
discard 83d19cc8ed0 [feat](map) remove duplicated keys in ColumnMap
add 4757c6ed671 [fix](inverted index) create empty idx file when creating
a index on variant-type column (#50937)
add d6e624c7033 [fix](community) remove the collaborators for readding
(#51049)
add aba01d6719c [hotfix](jdbc catalog) Fix jdbcclient repeated
initialization (#51036)
add be10f9f1827 [improvement](nereids)abstract order by for show command
(#51019)
add f4641a9d1e7 [improve](nereids)Remove use of session variable
deprecated_group_by_and_having_use_alias_first (#51015)
add df1c5521126 [fix](pipelinex) fix null aware left anti join instance
num (#51053)
add d9d67511b6e [Feat](nereids) support ShowQueryStatsCommand (#50998)
add 7f933f50462 [Enhancement] (nereids)implement revokeRoleCommand in
nereids (#50328)
add 427b9174683 [fix](group commit) reduce cpu cost for group_commit
get_block (#49822)
add c27b721dd7d [fix](community) re-add the collaborators (#51051)
add 986f46b5500 [enhance](mtmv)cache table snapshot in refresh context
(#50855)
add b0612edab39 [fix](metadata) replace view will break view's metadata
(#51058)
add 09789a40c67 [Fix](dictionary) Skip auto refresh dictionary if we know
source data is invalid (#50863)
add bc6f05d557e [feat](sql-convertor) support setting sql convertor's
config by session variable (#50959)
add 81d77fb05c9 [fix](join) Should not use the build block's size to
resize mark_join_flags (#50993)
add 3bca44ca09e [opt](docker) add a script flag to control load data or
not (#51065)
add 85f9fbfd7e5 [Enhancement] (nereids)implement showOpenTablesCommand in
nereids (#50204)
add 7e7d5c56031 [Enhancement] (nereids)implement
CreateWorkloadPolicyCommand in nereids (#49370)
add 98b428437ac [Test](cloud-mow) check delete bitmap key's existence
before put KV (#51013)
add d4f4b97f362 [Enhancement] implement createDatabase command in nereids
(#51059)
add a6b603d4be9 [Feat](nereids) support ShowCatalogRecycleBinCommand
(#49818)
add 044b497bd7b [Enhancement] (nereids) implement
CancelBackup/Restore/Command in nereids (#49653)
add 3b42e9cb8d4 [Enhancement] (nereids)implement
alterDatabasePropertiesCommand in nereids (#49365)
add eaa2cbfa0c2 [check](pipeline) Check the shared state is not set in
the pipeline. (#51031)
add b3d922f4efc [Refactor](workload group)reset workload group check
defaut interval to 2s (#51029)
add c7a3c49326f [fix](memory) Fix `PODArray::add_num_element` step2
(#50784)
add 42fcf8cdb12 [chore](load) delete StreamLoadPlanner and
LoadingTaskPlanner (#51055)
add ae3a8804422 [fix](parquet/orc) Disable string dictionary filtering
when predicate express is not binary pred and in pred (#50749)
add e1321a632a9 [fix](planner)comparing partition column name should be
case insensitive when pruning partition in DeleteJob (#50674)
add 1a51bc3f28f [Enhancement] (nereids)implement installPluginCommand in
nereids (#48589)
add e1b572e5a88 [improve](nereids)Remove use of session variable
deprecated_group_by_and_having_use_alias_first 2 (#51100)
add e9adc01de17 [log](mow) optimize some log for mow table (#51093)
add 60ee1a84ae3 [refine](DataTypeSerDe) Remove the level variable from
FormatOptions and use _nesting_level. (#50977)
add 31adcf69c4f [Feature](agg-state) support import/export agg-state data
(#50769)
add dfd2fc27422 [fix](schema-cache) make catalog level schema cache config
work for "get_schema_from_table" mode (#51057)
add 7b0828c7f98 [fix](nereids) fix bug in "Push down agg through join one
side" rule (#51076)
add a8707a06d6d [Enhancement] (nereids) implement CancelAlterTableCommand
in nereids (#49488)
add bfa9588f46e [Fix](MTMV) mtmv not refresh cache of external table
before run task (#50979)
add cf5aa68c9a7 [refactor](type) Use PrimitiveType to replace CppType
(#50935)
add 8a12288134e [Enhancement] (nereids) implement ShowCopyCommand in
nereids (#51040)
add 98d87b21655 [fix](load) fix the error msg of task submission failure
for memory back pressure (#51078)
add a6fc1a2b7e2 [opt](inverted index) Optimize prefix query in
match_regexp (#50968)
add 4dfafd46a4e [fix](case)fix mark_join and right_semi_mark_join has same
table name (#51124)
add b289297b945 [fix](nereids) fix subquery unnest can not found aggregate
slot (#51086)
add e137ae943b3 [fix](nereids) Not use rule
FOUR_PHASE_AGGREGATE_WITH_DISTINCT_WITH_FULL_DISTRIBUTE when
mustUseMultiDistinctAgg (#51099)
add 8fa8c4a4aff [fix](meta) fix timeout for syncing image when FE scales
out (#51120)
add a62e651275b [opt](cloud) load data no call partition.getVisibleVersion
in cloud mode (#51111)
add 589dd2fb54f [Bug](agg-state) fix missing meta field set with agg_state
type on VerticalSegmentWriter (#51033)
add 7665c078e12 [opt](Nereids) ignore distinct for some aggregate
functions (#51088)
add c07e893d87e [improve](udf)add some check for udf when result is null
(#51084)
add 45dd031c05b [Fix](function) Support encrypt empty string (#51126)
add 3970183fec3 [clean](planner) row policy removes dependency on the old
optimizer (#51122)
add 29cb2917c6a [fix](docker case) Fix some docker case due to `defined
global variables in script are not allowed` (#50976)
add 5ea8025defb [feat](refactor-param)Integrate New Storage System Support
for BACKUP/RESTORE/LOAD/TVF (#50849)
add 839c277878b [Enhancement] (nereids)implement showColumnsCommand in
nereids (#45832)
add e6771f8566e [feature](function) support nth_value window function
(#50559)
add 41ba1743d1a [refactor](type) remove TypeToPrimitiveType (#51142)
add aff662bf06d [Enhancement] (nereids) implement
showWorkloadGroupsCommand in nereids (#50777)
add bf26920aaf4 [clean](planner)create table like removes dependency on
the old optimizer (#51135)
add 06748cbcd8c [feature](Nereids) support standard sql query organization
(#49465)
add ea62d6be461 [fix](statistics)Fix sample analyze agg/mor table doesn't
remove new partition flag bug. (#51116)
add c7ec98f2b6f [Fix](Catalog)Ensure preExecutionAuthenticator is properly
initialized (#50839)
add 8292ce83e09 [fix](core) fix avg rate field always showing 0 in
cluster_balance show proc (#51101) (#51104)
add 516d27bf6bf [Improvement](function) optimize case_when<then_null>
branch (#51132)
add fdb8409d466 [feat](doris compose) Add extra hosts option for up
command (#51098)
add 8fe3b735904 [Enhancement] (nereids)implement
revokeResourcePrivilegeCommand in ne… (#50419)
add c737662c09b [clean](planner)delete original planner (#51056)
add f4d78f5ef56 [enhancement](cloud-schema-change) Enable new tablet to do
compactionby default (#51070)
add 51f89f1beb4 [Opt](multi-catalog)Disable dict filter in parquet/orc
reader if have non-single conjuncts. (#44777)
add 33e0ec7d3c8 [opt](Nereids) optimize view related capabilities (#51028)
add 5fdaf7b495a [enhancement](cloud) make file cache version upgrade
faster (#50726)
add 4a8d2cc1e18 [fix](cloud-schema-change) Make SC tablet job abort logic
really work (#50908)
add af382359274 [chore](cataglog) Unlimit db data size quota (#51108)
add 8488e9c0c83 [Refactor] Refactor ConnectContext.getCluster (#50783)
add 80c4a0b2c68 [fix](jdbc test) Add more connections to mysql docker
(#50970)
add de1fcd37ce7 [fix](agent) cancel agent task when it is rejected by
agent-task-pool (#51138)
add ef003cde1aa [Fix](orc-reader) Fix merge range not sorted in new merge
io facility of orc-reader. (#51102)
add 22d293269db [Optimize](orc-reader) Optimize stripe footer multiple
reads in orc reader. (#51117)
add f94e0b41ee4 [feat](desc) add comment column in desc statement (#51047)
add 2c257f2a8e4 [fix](cloud) Cloud tablet cache should not be pruned
(#51193)
add f2432586297 [Enhancement] (nereids)implement showStorageVaultCommand
in nereids (#44805)
add 5dd341edddc [bugfix](nerids) complete the implementation of the concat
method. (#51141)
add 2f60e2d7bb9 [Fix](function) Add lost check of function SHA2 in nereids
(#51163)
add 2f795bc21dc [fix](export) remove task from map finally (#51171)
add 8081c331726 [fix](auto bucket) Configurable parameters for partition
size estimation and number of buckets in auto bucket (#50566)
add af2cda43dba [Performance](clould) reduce tablet cache read in topn
query (#50803)
add bd25d471bed [chore](load) reduce log for data load (#51172)
add 4f1aa7db923 [fix/feature](kill) fix kill operation and support kill by
trace id (#50791)
add bf50441c9c0 [Chore](test) add beut about count (#51176)
add 61034f4706d [fix](nereids)update operative slots in
LogicalPlanDeepCopier (#51022)
add c2e867ee667 [Fix](multicatelog) Fix insert iceberg/hive table when use
broker (#51187)
add d35485a17f5 [test](mtmv)fix master-slave consistency after schema
change (#51191)
add 7eaed2e7e3d [fix](arrow) Rollback Arrow version 19.0.1 -> 17.0.0, Fix
MacOS compile error and decimal type error when convert to Parquet (#51217)
add 1ff0bc8c747 [config](defaut) string_type_length_soft_limit_bytes to
10MB (#51121)
add 67126a329c4 [refactor](type) Use PrimitiveType as template arg of data
type (#51201)
add ebc26bcfcbd [fix](coordinator) fix query cache throw
CacheSourceOperator only support one scan range (#51202)
add ae9453e5e09 [feat](udf) support "prefer_udf_over_builtin" session
variable (#51195)
add a9377e2c8b3 [Bug](set) fix missing null when intersect operator's
hashmap shrink (#51063)
add 08b62047d1d [fix](cloud) don't log sensitive info for copy into
(#51264)
add 2d6c1204ce9 [regression-test](improvement) show line number when
global var defined (#51254)
add a74649a7da8 [Fix](nereids) Fix incorrect hive references in
LogicalIcebergTableSink and update PlanType (#49306)
add 1a0a9ae2cf5 [Enhancement] implement createRepositoryCommand in nereids
(#51161)
add 3c27146c7e9 correct the profile count prompt on the FE webui
/QueryProfile (#51000)
add 0fad959ec4d [fix](group commit) fix group commit with schema change
(#51144)
add edcff62dd7f [chore](json)Remove some unnecessary parameters in
JsonbParserTSIMD and JsonbWriterT. (#51239)
add 4b3c2b393fe [Enhancement] (nereids)implement alterResourceCommand in
nereids (#49173)
add 20cd1e1c2bb [Enhancement] (nereids) implement showWarmUpCommand in
nereids (#51081)
add e3b0cfa5b02 [Enhancement] (nereids)implement uninstallPluginCommand in
nereids (#49136)
add 5545558181a [opt](nereids) extract common factor apply recursively for
the common part (#49774)
add 79a97db04a8 [Chore](case) adjust map_agg mv case (#51262)
add 6f15d7ce715 [Improvement](function) optimize for case when have many
conditions (#51205)
add 5fe8a891266 [opt](Compaction Profiling) refactor Compaction Profiling
JSON construct (#51146)
add a59168417ab [chore](feut) add maven.test.failure.ignore (#51134)
add ae2dcaf6757 [fix](iceberg) fix the iceberg eq-delete filter
resize-fill bug. (#51253)
add 144ddbfa82c [improvement](jdbc catalog) Optimize the acquisition of
indentity type in SQLServer (#51285)
add 753fddb2a29 [chore](json) remove jsonb_parser.h file (#51284)
add 9745befcad2 [Enhancement](multi-catalog) Add PredicateFilterTime,
DictFilterRewriteTime, LazyReadFilteredRows profile metrics in parquet orc
profiles. (#51248)
add 8e165d69def [improvement](statistics)Improve fetch hive table row
count. (#51192)
add 2528d3347e8 [fix](Nereids) cbo rewrite not work when not cte in query
(#46088)
add eae82c1191e [Fix](ShortCircuit) need to shrink char type when read
column store (#50975)
add 08dd1f7af83 [Fix](Iceberg-hadoop-catalog)Fix Kerberos-authenticated
HadoopCatalog insert failures due to missing kerberos credentials (#51245)
add cf3098e7b65 [enhancement](txn lazy commit) Add more ut and log for
recycler about txn lazy commit (#51310)
add bd061fa39f6 [fix](UT) Fix BE UT failure (#51324)
add 9cf359a3dc8 [opt](query) accelerate query information_schema.tables
from follower node in cloud mode (#51240)
add 21f9e872508 [Enhancement] (nereids) implement ShowReplicaStatusCommand
in nereids (#50787)
add 05696052cc4 [fix](cloud) Change the executor of the warm up job from
root to admin (#51297)
add 159ced2df05 [Fix](cloud-mow) Fix invalid specified version.
spec_version=0--1 (#51255)
add ca843544f56 [improve](cloud-mow) batch get tablet stats when
get_delete_bitmap_update_lock (#47281)
add 0b95a4d6682 [feat](refactor-param) refactor brokerLoad's code about
fileformat (#50882)
add 41afc2955d6 [opt](profile) optimize timer in profile (#51327)
add 08ada7be74f [Enhancement] (nereids)implement
revokeTablePrivilegeCommand in nereids (#50502)
add 3121a8705dd [improve](function) support any_value function with
complex type (#49419)
add d3a247eb1aa [fix](arrow-flight) Support SerDe of Doris string larger
than 2GB to Arrow large string (#51265)
add a4b50087bce [Exec](performance) Topn lazy materialize
add c8c3cb8dddf [Refactor]refactor workload group cpu hard limit (#51278)
add 3da42579731 [Enhancement](nereids)suppot show partitions (#51328)
add 63c4e4cc9a9 [fix](nereids) fix bugs in PushDownAggThroughJoin rule
(#50852)
add 6cf39e82c04 [enhance](nereids) expand support for eliminate agg by
unique (#48317)
add 3c670fef2f2 [regression-test](case) move
test_group_commit_and_wal_back_pressure to p2 (#51334)
add 56eee0799bc [enhancement](memoryfailed) throw exception if
block.serialize meet error status (#51335)
add a8f16add35c [fix](cloud) Prevent S3 error log name collisions in
multiple instances (#51356)
add bc396e1b7b4 [feat](fs-v2): add S3 IAM Role support (#51229)
add 056fffbc6b6 [opt](nereids) add cost penalty for pattern
Aggregation-nestedLoopJoin (#51127)
add de405ee7035 [fix](nereids) fix equals function of FuncDepsItem (#43390)
add 18ce7bc6693 [Enhancement] (nereids)implement CreateStorageVaultCommand
in nereids (#50607)
add 32e83fec3ae [Enhancement] (nereids) implement showTransactionCommand
in nereids (#50695)
add 8d915760d02 [Enhancement] (nereids)implement lockTablesCommand in
nereids (#49375)
add 383f45c12d3 [fix](mow) reduce mow delete bitmap count: agg and remove
delete bitmap of pre rowsets when cu compaction (#49383)
add 7f0397e9813 [fix](group commit) fix group_commit get_block too slow
(#51358)
add 1eb79343868 [fix](column_string) remove inaccurate error message of
string overflow (#51290)
add 4e20a3ba24d [feat](map) remove duplicated keys in ColumnMap
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 (83d19cc8ed0)
\
N -- N -- N refs/heads/deduplicate_keys (4e20a3ba24d)
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/agent/task_worker_pool.cpp | 7 +-
be/src/agent/workload_group_listener.cpp | 26 +-
be/src/apache-orc | 2 +-
be/src/cloud/cloud_cumulative_compaction.cpp | 46 +-
be/src/cloud/cloud_meta_mgr.cpp | 35 +-
be/src/cloud/cloud_meta_mgr.h | 6 +-
be/src/cloud/cloud_schema_change_job.cpp | 7 +
be/src/cloud/cloud_storage_engine.cpp | 9 +-
be/src/cloud/cloud_storage_engine.h | 4 +-
be/src/cloud/cloud_tablet.cpp | 85 +-
be/src/cloud/cloud_tablet.h | 13 +
be/src/cloud/cloud_tablet_mgr.cpp | 75 +-
be/src/cloud/cloud_tablet_mgr.h | 3 +-
be/src/cloud/config.cpp | 2 +-
be/src/common/config.cpp | 17 +-
be/src/common/config.h | 5 +
be/src/common/consts.h | 1 +
be/src/common/exception.h | 1 +
be/src/common/status.h | 2 +
be/src/exec/es/es_scroll_parser.cpp | 204 +-
be/src/exec/olap_common.h | 6 +-
be/src/exec/rowid_fetcher.cpp | 357 ++-
be/src/exec/rowid_fetcher.h | 49 +
be/src/http/action/delete_bitmap_action.cpp | 134 +-
be/src/io/cache/fs_file_cache_storage.cpp | 267 +-
be/src/io/cache/fs_file_cache_storage.h | 8 +
be/src/io/fs/err_utils.cpp | 2 +
be/src/olap/base_tablet.cpp | 158 ++
be/src/olap/base_tablet.h | 6 +
be/src/olap/compaction.cpp | 3 +-
be/src/olap/id_manager.h | 264 ++
be/src/olap/in_list_predicate.h | 8 +-
be/src/olap/like_column_predicate.cpp | 8 +-
be/src/olap/like_column_predicate.h | 8 +-
be/src/olap/olap_server.cpp | 5 +-
be/src/olap/partial_update_info.cpp | 6 +-
be/src/olap/push_handler.cpp | 4 +-
be/src/olap/rowset/segment_v2/column_reader.cpp | 27 +
be/src/olap/rowset/segment_v2/column_reader.h | 35 +
.../inverted_index/query/regexp_query.cpp | 130 +-
.../segment_v2/inverted_index/query/regexp_query.h | 13 +
.../segment_v2/inverted_index_file_writer.cpp | 4 +
be/src/olap/rowset/segment_v2/segment_iterator.cpp | 11 +
.../rowset/segment_v2/vertical_segment_writer.cpp | 3 +
be/src/olap/rowset_builder.cpp | 8 -
be/src/olap/schema.cpp | 4 +-
be/src/olap/schema.h | 6 +-
be/src/olap/storage_engine.cpp | 61 +-
be/src/olap/storage_engine.h | 18 +-
be/src/olap/tablet.cpp | 115 +-
be/src/olap/tablet_manager.cpp | 51 +-
be/src/olap/tablet_meta.cpp | 37 +-
be/src/olap/tablet_meta.h | 7 +-
be/src/olap/tablet_schema.cpp | 19 +
be/src/olap/tablet_schema.h | 10 +
be/src/olap/task/engine_clone_task.h | 3 +
be/src/olap/utils.h | 11 +
be/src/olap/wal/wal_table.cpp | 1 +
be/src/pipeline/dependency.cpp | 199 +-
be/src/pipeline/dependency.h | 119 +-
.../pipeline/exec/aggregation_source_operator.cpp | 4 +-
be/src/pipeline/exec/analytic_sink_operator.cpp | 3 +-
be/src/pipeline/exec/cache_sink_operator.cpp | 1 -
be/src/pipeline/exec/cache_sink_operator.h | 6 +-
be/src/pipeline/exec/cache_source_operator.cpp | 2 +-
be/src/pipeline/exec/cache_source_operator.h | 4 +-
.../exec/group_commit_block_sink_operator.cpp | 8 +-
.../pipeline/exec/group_commit_scan_operator.cpp | 18 +-
be/src/pipeline/exec/group_commit_scan_operator.h | 3 +-
.../exec/join/process_hash_table_probe_impl.h | 22 +-
.../exec/materialization_sink_operator.cpp | 157 ++
.../pipeline/exec/materialization_sink_operator.h | 71 +
.../exec/materialization_source_operator.cpp | 59 +
.../exec/materialization_source_operator.h | 72 +
be/src/pipeline/exec/operator.cpp | 16 +-
be/src/pipeline/exec/operator.h | 8 +
be/src/pipeline/exec/scan_operator.cpp | 30 +-
be/src/pipeline/exec/set_probe_sink_operator.cpp | 17 +-
be/src/pipeline/exec/set_source_operator.cpp | 18 +-
.../exec/streaming_aggregation_operator.cpp | 3 +-
be/src/pipeline/exec/union_source_operator.cpp | 6 +
be/src/pipeline/exec/union_source_operator.h | 2 +
be/src/pipeline/pipeline_fragment_context.cpp | 21 +
be/src/runtime/descriptors.cpp | 8 +
be/src/runtime/descriptors.h | 4 +-
be/src/runtime/exec_env.cpp | 3 +-
be/src/runtime/exec_env.h | 6 +-
be/src/runtime/exec_env_init.cpp | 8 +-
be/src/runtime/group_commit_mgr.cpp | 135 +-
be/src/runtime/group_commit_mgr.h | 18 +-
be/src/runtime/jsonb_value.h | 4 -
be/src/runtime/load_stream_writer.cpp | 3 +-
be/src/runtime/primitive_type.h | 507 +++-
be/src/runtime/query_context.h | 5 +-
.../routine_load/routine_load_task_executor.cpp | 29 +-
.../routine_load/routine_load_task_executor.h | 2 +-
be/src/runtime/runtime_state.cpp | 6 +-
be/src/runtime/runtime_state.h | 8 +
be/src/runtime/tablets_channel.cpp | 9 +-
be/src/runtime/workload_group/workload_group.cpp | 28 +-
be/src/runtime/workload_group/workload_group.h | 3 +-
.../workload_group/workload_group_manager.cpp | 8 +-
.../workload_group/workload_group_manager.h | 6 -
be/src/runtime_filter/runtime_filter.cpp | 1 +
.../arrow_flight/arrow_flight_batch_reader.cpp | 12 +-
.../arrow_flight/arrow_flight_batch_reader.h | 3 +
be/src/service/backend_options.h | 1 +
be/src/service/brpc.h | 5 +-
be/src/service/internal_service.cpp | 44 +
be/src/service/internal_service.h | 4 +
be/src/service/point_query_executor.cpp | 11 +-
be/src/util/arrow/block_convertor.cpp | 14 +-
be/src/util/arrow/row_batch.cpp | 4 +-
be/src/util/arrow/row_batch.h | 2 +
be/src/util/arrow/utils.h | 10 +
be/src/util/brpc_closure.h | 7 +-
be/src/util/jsonb_parser.h | 1060 -------
be/src/util/jsonb_parser_simd.h | 52 +-
be/src/util/jsonb_writer.h | 41 +-
.../aggregate_function_approx_count_distinct.cpp | 58 +-
.../aggregate_function_approx_count_distinct.h | 10 +-
.../aggregate_function_approx_top_sum.h | 12 +-
.../aggregate_function_array_agg.cpp | 46 +-
.../aggregate_function_array_agg.h | 32 +-
.../aggregate_functions/aggregate_function_avg.cpp | 18 +-
.../aggregate_functions/aggregate_function_avg.h | 46 +-
.../aggregate_function_avg_weighted.h | 27 +-
.../aggregate_function_binary.h | 51 +-
.../aggregate_functions/aggregate_function_bit.h | 48 +-
.../aggregate_function_bitmap.h | 9 +-
.../aggregate_function_bitmap_agg.cpp | 9 +-
.../aggregate_function_bitmap_agg.h | 10 +-
.../aggregate_function_collect.cpp | 91 +-
.../aggregate_function_collect.h | 54 +-
.../aggregate_function_corr.cpp | 2 +-
.../aggregate_functions/aggregate_function_corr.h | 8 +-
.../aggregate_functions/aggregate_function_count.h | 7 +-
.../aggregate_function_covar.cpp | 34 +-
.../aggregate_functions/aggregate_function_covar.h | 14 +-
.../aggregate_function_distinct.cpp | 6 +-
.../aggregate_function_distinct.h | 20 +-
.../aggregate_function_group_array_intersect.cpp | 21 +-
.../aggregate_function_group_array_intersect.h | 64 +-
.../aggregate_function_histogram.cpp | 39 +-
.../aggregate_function_histogram.h | 27 +-
.../aggregate_function_kurtosis.cpp | 30 +-
.../aggregate_function_linear_histogram.cpp | 41 +-
.../aggregate_function_linear_histogram.h | 12 +-
.../aggregate_functions/aggregate_function_map.cpp | 39 +-
.../aggregate_functions/aggregate_function_map.h | 24 +-
.../aggregate_function_min_max.cpp | 48 +-
.../aggregate_function_min_max.h | 173 +-
.../aggregate_function_min_max_by.h | 70 +-
.../aggregate_function_orthogonal_bitmap.cpp | 6 +-
.../aggregate_function_orthogonal_bitmap.h | 39 +-
.../aggregate_function_percentile.h | 23 +-
.../aggregate_function_product.h | 63 +-
.../aggregate_function_regr_union.cpp | 20 +-
.../aggregate_function_regr_union.h | 11 +-
.../aggregate_function_sequence_match.cpp | 16 +-
.../aggregate_function_sequence_match.h | 67 +-
.../aggregate_function_skew.cpp | 30 +-
.../aggregate_functions/aggregate_function_sort.h | 8 +-
.../aggregate_function_statistic.h | 4 +-
.../aggregate_function_stddev.cpp | 18 +-
.../aggregate_function_stddev.h | 22 +-
.../aggregate_functions/aggregate_function_sum.h | 64 +-
.../aggregate_function_topn.cpp | 69 +-
.../aggregate_functions/aggregate_function_topn.h | 51 +-
.../aggregate_function_uniq.cpp | 25 +-
.../aggregate_functions/aggregate_function_uniq.h | 47 +-
.../aggregate_function_uniq_distribute_key.cpp | 12 +-
.../aggregate_function_uniq_distribute_key.h | 33 +-
.../aggregate_function_window.cpp | 6 +-
.../aggregate_function_window.h | 20 +
be/src/vec/aggregate_functions/helpers.h | 69 +-
be/src/vec/columns/column_array.cpp | 6 +-
be/src/vec/columns/column_complex.h | 23 +-
be/src/vec/columns/column_decimal.cpp | 16 +
be/src/vec/columns/column_decimal.h | 12 +-
be/src/vec/columns/column_dictionary.h | 61 +-
be/src/vec/columns/column_fixed_length_object.h | 5 +-
be/src/vec/columns/column_map.cpp | 10 +-
be/src/vec/columns/column_map.h | 7 +-
be/src/vec/columns/column_nullable.cpp | 4 +-
be/src/vec/columns/column_nullable.h | 5 +-
be/src/vec/columns/column_object.cpp | 231 +-
be/src/vec/columns/column_string.cpp | 23 +
be/src/vec/columns/column_string.h | 27 +-
be/src/vec/columns/column_struct.cpp | 3 +-
be/src/vec/columns/column_vector.cpp | 44 +-
be/src/vec/columns/column_vector.h | 2 +-
be/src/vec/common/field_visitors.h | 63 +-
be/src/vec/common/hash_table/hash_map_context.h | 81 +-
be/src/vec/common/hash_table/ph_hash_map.h | 4 +
be/src/vec/common/pod_array.h | 26 +-
be/src/vec/common/schema_util.cpp | 5 +-
be/src/vec/common/string_ref.cpp | 5 +-
be/src/vec/core/call_on_type_index.h | 34 +-
be/src/vec/core/decimal_comparison.h | 2 +-
be/src/vec/core/field.cpp | 631 +++--
be/src/vec/core/field.h | 213 +-
be/src/vec/core/types.h | 15 +-
be/src/vec/data_types/convert_field_to_type.cpp | 88 +-
be/src/vec/data_types/data_type_agg_state.h | 6 +
be/src/vec/data_types/data_type_array.cpp | 2 +-
be/src/vec/data_types/data_type_array.h | 1 -
be/src/vec/data_types/data_type_bitmap.h | 11 +-
be/src/vec/data_types/data_type_date.cpp | 2 +-
be/src/vec/data_types/data_type_date.h | 9 +-
.../data_types/data_type_date_or_datetime_v2.cpp | 4 +-
.../vec/data_types/data_type_date_or_datetime_v2.h | 16 +-
be/src/vec/data_types/data_type_date_time.cpp | 2 +-
be/src/vec/data_types/data_type_date_time.h | 8 +-
be/src/vec/data_types/data_type_decimal.cpp | 15 +-
be/src/vec/data_types/data_type_decimal.h | 15 +-
be/src/vec/data_types/data_type_factory.cpp | 12 -
.../vec/data_types/data_type_fixed_length_object.h | 9 +-
be/src/vec/data_types/data_type_hll.h | 5 +-
be/src/vec/data_types/data_type_ipv4.h | 9 +-
be/src/vec/data_types/data_type_ipv6.h | 7 +-
be/src/vec/data_types/data_type_jsonb.h | 6 +-
be/src/vec/data_types/data_type_map.cpp | 6 +-
be/src/vec/data_types/data_type_map.h | 1 -
be/src/vec/data_types/data_type_nothing.h | 10 +-
be/src/vec/data_types/data_type_nullable.cpp | 2 +-
be/src/vec/data_types/data_type_nullable.h | 2 +-
be/src/vec/data_types/data_type_number.cpp | 39 +-
be/src/vec/data_types/data_type_number.h | 55 +-
be/src/vec/data_types/data_type_number_base.cpp | 198 +-
be/src/vec/data_types/data_type_number_base.h | 84 +-
be/src/vec/data_types/data_type_object.cpp | 13 +
be/src/vec/data_types/data_type_object.h | 15 +-
be/src/vec/data_types/data_type_quantilestate.h | 11 +-
be/src/vec/data_types/data_type_string.cpp | 2 +-
be/src/vec/data_types/data_type_string.h | 2 +-
be/src/vec/data_types/data_type_struct.cpp | 2 +-
be/src/vec/data_types/data_type_struct.h | 1 -
be/src/vec/data_types/data_type_time.cpp | 2 +-
be/src/vec/data_types/data_type_time.h | 6 +-
be/src/vec/data_types/get_least_supertype.cpp | 12 +-
be/src/vec/data_types/number_traits.h | 108 +-
.../vec/data_types/serde/data_type_array_serde.cpp | 27 +-
.../vec/data_types/serde/data_type_array_serde.h | 10 +-
.../data_types/serde/data_type_bitmap_serde.cpp | 15 +-
.../vec/data_types/serde/data_type_bitmap_serde.h | 16 +-
.../data_types/serde/data_type_date64_serde.cpp | 130 +-
.../vec/data_types/serde/data_type_date64_serde.h | 75 +-
.../serde/data_type_datetimev2_serde.cpp | 32 +-
.../data_types/serde/data_type_datetimev2_serde.h | 14 +-
.../data_types/serde/data_type_datev2_serde.cpp | 23 +-
.../vec/data_types/serde/data_type_datev2_serde.h | 15 +-
.../data_types/serde/data_type_decimal_serde.cpp | 66 +-
.../vec/data_types/serde/data_type_decimal_serde.h | 10 +-
.../vec/data_types/serde/data_type_hll_serde.cpp | 15 +-
be/src/vec/data_types/serde/data_type_hll_serde.h | 14 +-
.../vec/data_types/serde/data_type_ipv4_serde.cpp | 23 +-
be/src/vec/data_types/serde/data_type_ipv4_serde.h | 21 +-
.../vec/data_types/serde/data_type_ipv6_serde.cpp | 37 +-
be/src/vec/data_types/serde/data_type_ipv6_serde.h | 15 +-
.../vec/data_types/serde/data_type_jsonb_serde.cpp | 23 +-
.../vec/data_types/serde/data_type_jsonb_serde.h | 6 +-
.../vec/data_types/serde/data_type_map_serde.cpp | 61 +-
be/src/vec/data_types/serde/data_type_map_serde.h | 10 +-
.../vec/data_types/serde/data_type_nothing_serde.h | 18 +-
.../data_types/serde/data_type_nullable_serde.cpp | 20 +-
.../data_types/serde/data_type_nullable_serde.h | 10 +-
.../data_types/serde/data_type_number_serde.cpp | 187 +-
.../vec/data_types/serde/data_type_number_serde.h | 139 +-
.../data_types/serde/data_type_object_serde.cpp | 35 +-
.../vec/data_types/serde/data_type_object_serde.h | 14 +-
.../serde/data_type_quantilestate_serde.h | 26 +-
be/src/vec/data_types/serde/data_type_serde.h | 27 +-
.../vec/data_types/serde/data_type_string_serde.h | 54 +-
.../data_types/serde/data_type_struct_serde.cpp | 32 +-
.../vec/data_types/serde/data_type_struct_serde.h | 10 +-
be/src/vec/data_types/serde/data_type_time_serde.h | 4 +-
.../vec/exec/format/arrow/arrow_stream_reader.cpp | 5 +-
be/src/vec/exec/format/generic_reader.h | 13 +
be/src/vec/exec/format/orc/vorc_reader.cpp | 220 +-
be/src/vec/exec/format/orc/vorc_reader.h | 40 +-
be/src/vec/exec/format/parquet/parquet_pred_cmp.h | 6 +-
.../exec/format/parquet/vparquet_group_reader.cpp | 367 +--
.../exec/format/parquet/vparquet_group_reader.h | 25 +-
be/src/vec/exec/format/parquet/vparquet_reader.cpp | 60 +-
be/src/vec/exec/format/parquet/vparquet_reader.h | 17 +-
be/src/vec/exec/format/table/equality_delete.cpp | 6 +-
be/src/vec/exec/jni_connector.h | 3 +-
be/src/vec/exec/scan/file_scanner.cpp | 416 ++-
be/src/vec/exec/scan/file_scanner.h | 44 +-
be/src/vec/exec/scan/olap_scanner.cpp | 10 +
be/src/vec/exec/scan/scanner.h | 6 +
be/src/vec/exec/vjdbc_connector.cpp | 3 +-
be/src/vec/exprs/varray_literal.cpp | 2 +-
be/src/vec/exprs/vinfo_func.cpp | 4 +-
be/src/vec/exprs/vmap_literal.cpp | 6 +-
be/src/vec/exprs/vstruct_literal.cpp | 2 +-
.../functions/array/function_array_aggregation.cpp | 159 +-
be/src/vec/functions/array/function_array_index.h | 36 +-
.../vec/functions/complex_hash_map_dictionary.cpp | 6 +-
be/src/vec/functions/complex_hash_map_dictionary.h | 4 +-
be/src/vec/functions/dictionary.h | 2 +
be/src/vec/functions/divide.cpp | 7 +-
be/src/vec/functions/function.cpp | 2 +-
be/src/vec/functions/function_binary_arithmetic.h | 7 +-
be/src/vec/functions/function_bit.cpp | 18 +-
be/src/vec/functions/function_bit_count.cpp | 6 +-
be/src/vec/functions/function_bit_shift.cpp | 8 +-
be/src/vec/functions/function_bitmap.cpp | 3 +-
be/src/vec/functions/function_bitmap_min_or_max.h | 2 +-
be/src/vec/functions/function_case.cpp | 6 +-
be/src/vec/functions/function_case.h | 188 +-
be/src/vec/functions/function_cast.h | 32 +-
be/src/vec/functions/function_const.h | 3 +-
be/src/vec/functions/function_convert_tz.h | 28 +-
.../vec/functions/function_datetime_floor_ceil.cpp | 2 +-
be/src/vec/functions/function_encode_varchar.cpp | 32 +-
be/src/vec/functions/function_encryption.cpp | 4 -
be/src/vec/functions/function_helpers.h | 16 +-
be/src/vec/functions/function_hll.cpp | 2 +-
be/src/vec/functions/function_ip.h | 8 +-
be/src/vec/functions/function_json.cpp | 8 +-
be/src/vec/functions/function_jsonb.cpp | 4 -
be/src/vec/functions/function_map.cpp | 6 +-
be/src/vec/functions/function_string.h | 4 +-
be/src/vec/functions/function_unary_arithmetic.h | 25 +-
be/src/vec/functions/function_utility.cpp | 10 +-
be/src/vec/functions/functions_comparison.h | 13 +-
.../functions/functions_multi_string_search.cpp | 12 +-
be/src/vec/functions/int_div.h | 6 +-
be/src/vec/functions/ip_address_dictionary.cpp | 21 +-
be/src/vec/functions/ip_address_dictionary.h | 4 +-
be/src/vec/functions/is_not_null.h | 4 +-
be/src/vec/functions/is_null.h | 4 +-
be/src/vec/functions/math.cpp | 125 +-
be/src/vec/functions/minus.cpp | 6 +-
be/src/vec/functions/modulo.cpp | 16 +-
be/src/vec/functions/multiply.cpp | 7 +-
be/src/vec/functions/plus.cpp | 7 +-
be/src/vec/json/json_parser.cpp | 13 +-
be/src/vec/json/json_parser.h | 10 +-
be/src/vec/json/parse2column.cpp | 10 +-
be/src/vec/olap/olap_data_convertor.h | 1 -
be/src/vec/olap/vcollect_iterator.cpp | 3 +-
be/src/vec/sink/vtablet_block_convertor.cpp | 2 +-
.../sink/writer/iceberg/partition_transformers.h | 9 +-
.../writer/iceberg/viceberg_partition_writer.cpp | 3 +
.../writer/iceberg/viceberg_partition_writer.h | 1 +
.../sink/writer/iceberg/viceberg_table_writer.cpp | 12 +-
be/src/vec/sink/writer/vhive_partition_writer.cpp | 3 +
be/src/vec/sink/writer/vhive_partition_writer.h | 1 +
be/src/vec/sink/writer/vhive_table_writer.cpp | 46 +-
be/src/vec/sink/writer/vtablet_writer.cpp | 4 +-
be/src/vec/utils/arrow_column_to_doris_column.cpp | 7 +-
be/test/exec/hash_map/hash_table_method_test.cpp | 5 +-
be/test/io/cache/block_file_cache_test.cpp | 414 ++-
be/test/olap/id_manager_test.cpp | 154 ++
be/test/olap/index_builder_test.cpp | 4 +-
.../common/inverted_index_gc_binlogs_test.cpp | 4 +-
.../compaction/util/index_compaction_utils.cpp | 25 +-
.../inverted_index/empty_index_file_test.cpp | 92 +
.../inverted_index/query/regexp_query_test.cpp | 189 ++
.../segment_v2/inverted_index_array_test.cpp | 112 +-
be/test/olap/storage_types_test.cpp | 38 +
be/test/pipeline/common/agg_utils_test.cpp | 12 +-
.../pipeline/common/distinct_agg_utils_test.cpp | 28 +-
be/test/pipeline/common/set_utils_test.cpp | 24 +-
.../pipeline/operator/agg_shared_state_test.cpp | 12 +-
.../operator/hashjoin_probe_operator_test.cpp | 343 ++-
.../operator/materialization_shared_state_test.cpp | 343 +++
.../operator/partition_sort_sink_operator_test.cpp | 12 +-
be/test/pipeline/operator/set_operator_test.cpp | 13 +-
.../runtime_filter_producer_helper_cross_test.cpp | 4 +-
.../runtime_filter_producer_helper_set_test.cpp | 4 +-
.../runtime_filter_producer_helper_test.cpp | 16 +-
be/test/testutil/mock/mock_in_expr.cpp | 1 +
be/test/testutil/run_all_tests.cpp | 3 +
.../vec/aggregate_functions/agg_bitmap_test.cpp | 23 +-
.../vec/aggregate_functions/agg_collect_test.cpp | 8 +-
be/test/vec/aggregate_functions/agg_corr_test.cpp | 4 +-
.../vec/aggregate_functions/agg_count_test.cpp} | 18 +-
.../agg_group_array_intersect_test.cpp | 128 +-
.../aggregate_functions/agg_min_max_by_test.cpp | 8 +-
.../vec/aggregate_functions/agg_min_max_test.cpp | 6 +-
.../vec/aggregate_functions/agg_replace_test.cpp | 10 +-
be/test/vec/aggregate_functions/agg_test.cpp | 4 +-
.../aggregate_functions/vec_count_by_enum_test.cpp | 42 +-
.../vec/aggregate_functions/vec_retention_test.cpp | 72 +-
.../vec_sequence_match_test.cpp | 80 +-
.../aggregate_functions/vec_window_funnel_test.cpp | 180 +-
be/test/vec/columns/column_array_test.cpp | 2 +-
be/test/vec/columns/column_dictionary_test.cpp | 37 +-
be/test/vec/columns/column_hash_func_test.cpp | 56 +-
be/test/vec/columns/column_ip_test.cpp | 3 +-
.../column_nullable_seriazlization_test.cpp | 40 +-
be/test/vec/columns/column_nullable_test.cpp | 8 +-
be/test/vec/columns/column_nullable_test.h | 33 +-
be/test/vec/columns/column_object_test.cpp | 30 +-
be/test/vec/columns/column_resize_test.cpp | 10 +-
be/test/vec/columns/column_string_test.cpp | 4 +-
be/test/vec/columns/common_column_test.h | 47 +-
be/test/vec/common/pod_array_test.cpp | 51 +-
be/test/vec/core/block_test.cpp | 6 +-
be/test/vec/core/column_map_test.cpp | 73 +-
be/test/vec/core/column_struct_test.cpp | 13 +-
be/test/vec/core/column_test.cpp | 56 +-
be/test/vec/core/field_test.cpp | 19 +-
be/test/vec/core/get_common_type_test.cpp | 6 -
be/test/vec/core/number_traits_test.cpp | 102 +-
.../vec/data_types/common_data_type_serder_test.h | 11 +-
be/test/vec/data_types/common_data_type_test.h | 11 +-
.../vec/data_types/data_type_agg_state_test.cpp | 4 +-
be/test/vec/data_types/data_type_array_test.cpp | 24 +-
be/test/vec/data_types/data_type_bitmap_test.cpp | 4 +-
be/test/vec/data_types/data_type_decimal_test.cpp | 6 +-
.../data_type_fixed_length_object_test.cpp | 4 +-
be/test/vec/data_types/data_type_hll_test.cpp | 4 +-
be/test/vec/data_types/data_type_ip_test.cpp | 42 +-
be/test/vec/data_types/data_type_map_test.cpp | 122 +-
be/test/vec/data_types/data_type_number_test.cpp | 80 +-
.../data_types/data_type_quantile_state_test.cpp | 4 +-
be/test/vec/data_types/data_type_string_test.cpp | 6 +-
be/test/vec/data_types/data_type_struct_test.cpp | 60 +-
be/test/vec/data_types/data_type_time_v2_test.cpp | 10 +-
be/test/vec/data_types/datetime_round_test.cpp | 2 +-
be/test/vec/data_types/from_string_test.cpp | 2 +-
.../serde/data_type_serde_arrow_test.cpp | 146 +-
.../data_types/serde/data_type_serde_csv_test.cpp | 2 +-
.../serde/data_type_serde_datetime_v1_test.cpp | 2 +-
.../serde/data_type_serde_mysql_test.cpp | 2 +-
.../serde/data_type_serde_number_test.cpp | 27 +-
.../data_types/serde/data_type_serde_pb_test.cpp | 112 +-
.../vec/data_types/serde/data_type_serde_test.cpp | 4 +-
.../data_types/serde/data_type_serde_text_test.cpp | 2 +-
.../data_types/serde/data_type_to_string_test.cpp | 41 +-
.../parquet/byte_array_dict_decoder_test.cpp | 8 +-
.../parquet/fix_length_dict_decoder_test.cpp | 8 +-
.../vec/exec/format/parquet/parquet_read_lines.cpp | 350 +++
be/test/vec/exec/orc/orc_convert_dict_test.cpp | 10 +-
be/test/vec/exec/orc/orc_read_lines.cpp | 391 +++
be/test/vec/exec/orc_reader_test.cpp | 1 -
be/test/vec/exec/sort/partition_sorter_test.cpp | 2 +
be/test/vec/function/cast/cast_to_decimal.cpp | 128 +-
be/test/vec/function/cast/cast_to_float_double.cpp | 139 +-
be/test/vec/function/cast/cast_to_integer.cpp | 257 +-
.../function_compressed_materialization_test.cpp | 8 +-
be/test/vec/function/function_eq_for_null_test.cpp | 40 +-
be/test/vec/function/function_is_null_test.cpp | 6 +-
be/test/vec/function/function_round_test.cpp | 53 +-
be/test/vec/function/function_string_test.cpp | 28 +-
be/test/vec/function/function_test_util.cpp | 2 +-
be/test/vec/function/function_test_util.h | 2 +-
be/test/vec/jsonb/serialize_test.cpp | 59 +-
.../writer/iceberg/partition_transformers_test.cpp | 8 +-
cloud/src/common/bvars.cpp | 1 +
cloud/src/common/bvars.h | 1 +
cloud/src/common/config.h | 8 +
cloud/src/meta-service/meta_service.cpp | 559 ++--
cloud/src/meta-service/meta_service.h | 7 +
cloud/src/meta-service/meta_service_job.cpp | 4 +-
cloud/src/recycler/checker.cpp | 251 +-
cloud/src/recycler/checker.h | 8 +-
cloud/src/recycler/recycler.cpp | 62 +-
cloud/test/meta_service_test.cpp | 313 ++-
cloud/test/recycler_test.cpp | 209 +-
cloud/test/txn_lazy_commit_test.cpp | 697 ++++-
docker/runtime/doris-compose/Dockerfile | 9 +-
docker/runtime/doris-compose/Readme.md | 31 +
docker/runtime/doris-compose/cluster.py | 33 +-
docker/runtime/doris-compose/command.py | 12 +-
.../docker-compose/hive/hadoop-hive.env.tpl | 3 +
.../scripts/create_preinstalled_scripts/run80.hql | 23 +
.../docker-compose/hive/scripts/hive-metastore.sh | 8 +
.../file_id=1/example_1.orc | Bin 0 -> 7203 bytes
.../file_id=2/example_2.orc | Bin 0 -> 7307 bytes
.../file_id=2/example_3.orc | Bin 0 -> 7331 bytes
.../file_id=1/example_1.parquet | Bin 0 -> 6593 bytes
.../file_id=1/example_3.parquet | Bin 0 -> 6620 bytes
.../file_id=2/example_2.parquet | Bin 0 -> 6619 bytes
docker/thirdparties/docker-compose/mysql/my.cnf | 3 +-
.../sqlserver/init/03-create-table.sql | 7 +-
.../docker-compose/sqlserver/init/04-insert.sql | 2 +
docker/thirdparties/run-thirdparties-docker.sh | 7 +
.../apache/doris/common/jni/vec/VectorColumn.java | 47 +
.../org/apache/doris/catalog/AggStateType.java | 2 +
.../main/java/org/apache/doris/common/Config.java | 15 +-
.../authentication/PreExecutionAuthenticator.java | 15 +
fe/fe-core/pom.xml | 15 +
.../antlr4/org/apache/doris/nereids/DorisParser.g4 | 122 +-
fe/fe-core/src/main/cup/sql_parser.cup | 16 -
.../main/java/org/apache/doris/alter/Alter.java | 9 +-
.../java/org/apache/doris/alter/AlterJobV2.java | 11 +
.../org/apache/doris/alter/CloudRollupJobV2.java | 2 +-
.../apache/doris/alter/CloudSchemaChangeJobV2.java | 4 +-
.../doris/alter/MaterializedViewHandler.java | 55 +
.../apache/doris/alter/SchemaChangeHandler.java | 43 +
.../org/apache/doris/alter/SchemaChangeJobV2.java | 23 +-
.../java/org/apache/doris/analysis/BrokerDesc.java | 63 +-
.../java/org/apache/doris/analysis/CopyStmt.java | 3 +-
.../apache/doris/analysis/CreatePolicyStmt.java | 161 --
.../doris/analysis/CreateTableAsSelectStmt.java | 54 +-
.../org/apache/doris/analysis/DataDescription.java | 362 +--
.../org/apache/doris/analysis/DescribeStmt.java | 5 +
.../main/java/org/apache/doris/analysis/Expr.java | 14 +
.../java/org/apache/doris/analysis/LoadStmt.java | 113 +-
.../org/apache/doris/analysis/OutFileClause.java | 116 +-
.../java/org/apache/doris/analysis/QueryStmt.java | 7 +-
.../java/org/apache/doris/analysis/SelectStmt.java | 55 +-
.../java/org/apache/doris/analysis/SlotRef.java | 2 +-
.../org/apache/doris/analysis/StmtRewriter.java | 67 +-
.../org/apache/doris/analysis/StorageBackend.java | 99 +-
.../org/apache/doris/analysis/StorageDesc.java | 26 +
.../org/apache/doris/backup/BackupHandler.java | 213 +-
.../java/org/apache/doris/backup/BackupJob.java | 5 +-
.../java/org/apache/doris/backup/Repository.java | 156 +-
.../org/apache/doris/backup/RepositoryMgr.java | 7 +-
.../java/org/apache/doris/backup/RestoreJob.java | 5 +-
.../java/org/apache/doris/catalog/BrokerMgr.java | 15 +
.../doris/catalog/BuiltinWindowFunctions.java | 2 +
.../main/java/org/apache/doris/catalog/Column.java | 2 +-
.../main/java/org/apache/doris/catalog/Env.java | 53 +-
.../org/apache/doris/catalog/FunctionRegistry.java | 66 +-
.../doris/catalog/InternalSchemaInitializer.java | 6 +-
.../org/apache/doris/catalog/MetadataViewer.java | 34 +-
.../java/org/apache/doris/catalog/OlapTable.java | 26 +-
.../java/org/apache/doris/catalog/Resource.java | 4 +-
.../java/org/apache/doris/catalog/ResourceMgr.java | 14 +-
.../org/apache/doris/catalog/S3StorageVault.java | 7 +
.../org/apache/doris/catalog/StorageVault.java | 42 +
.../org/apache/doris/catalog/StorageVaultMgr.java | 17 +
.../java/org/apache/doris/catalog/TableIf.java | 11 +
.../doris/cloud/CacheHotspotManagerUtils.java | 14 +-
.../org/apache/doris/cloud/catalog/CloudEnv.java | 5 +-
.../apache/doris/cloud/catalog/CloudReplica.java | 54 +-
.../cloud/datasource/CloudInternalCatalog.java | 3 +-
.../doris/cloud/load/CloudBrokerLoadJob.java | 17 +-
.../doris/cloud/load/CloudLoadLoadingTask.java | 10 +-
.../cloud/planner/CloudStreamLoadPlanner.java | 62 -
.../org/apache/doris/cloud/storage/RemoteBase.java | 2 +-
.../doris/cloud/system/CloudSystemInfoService.java | 2 +-
.../transaction/CloudGlobalTransactionMgr.java | 10 +-
.../java/org/apache/doris/common/ErrorCode.java | 2 +-
.../java/org/apache/doris/common/FeConstants.java | 2 +
.../doris/common/proc/IndexSchemaProcNode.java | 49 +-
.../doris/common/proc/PartitionsProcDir.java | 140 +
.../common/proc/RemoteIndexSchemaProcDir.java | 2 +-
.../common/proc/RemoteIndexSchemaProcNode.java | 2 +-
.../doris/common/profile/RuntimeProfile.java | 4 +-
.../doris/common/profile/SummaryProfile.java | 93 +-
.../apache/doris/common/util/AutoBucketUtils.java | 17 +-
.../org/apache/doris/common/util/BrokerReader.java | 4 +-
.../org/apache/doris/common/util/BrokerUtil.java | 24 +-
.../java/org/apache/doris/common/util/S3Util.java | 35 +
.../org/apache/doris/datasource/CatalogIf.java | 3 +
.../apache/doris/datasource/ExternalCatalog.java | 37 +
.../doris/datasource/ExternalRowCountCache.java | 11 +-
.../doris/datasource/FederationBackendPolicy.java | 8 +-
.../org/apache/doris/datasource/FileGroupInfo.java | 18 +-
.../apache/doris/datasource/FileQueryScanNode.java | 7 +
.../apache/doris/datasource/InternalCatalog.java | 119 +-
.../apache/doris/datasource/LoadScanProvider.java | 56 +-
.../apache/doris/datasource/hive/HMSDlaTable.java | 3 -
.../doris/datasource/hive/HMSExternalCatalog.java | 18 +-
.../doris/datasource/hive/HMSExternalTable.java | 136 +-
.../datasource/hive/HiveMetaStoreClientHelper.java | 5 +-
.../doris/datasource/hive/HiveMetadataOps.java | 33 +-
.../iceberg/IcebergDLFExternalCatalog.java | 6 +
.../datasource/iceberg/IcebergExternalCatalog.java | 16 +-
.../iceberg/IcebergHadoopExternalCatalog.java | 20 +-
.../datasource/iceberg/IcebergMetadataOps.java | 36 +
.../datasource/iceberg/IcebergTransaction.java | 16 +-
.../doris/datasource/jdbc/JdbcExternalCatalog.java | 50 +-
.../jdbc/client/JdbcSQLServerClient.java | 9 +-
.../datasource/operations/ExternalMetadataOps.java | 13 +
.../datasource/property/ConnectionProperties.java | 10 +-
.../fileformat/ArrowFileFormatProperties.java | 49 +
.../fileformat/CsvFileFormatProperties.java | 20 +-
.../property/fileformat/FileFormatProperties.java | 10 +-
.../fileformat/JsonFileFormatProperties.java | 61 +-
.../property/metastore/AliyunDLFProperties.java | 3 +-
.../property/metastore/HMSProperties.java | 3 +-
.../storage/AbstractS3CompatibleProperties.java | 30 +-
.../property/storage/AzureProperties.java | 2 +-
.../property/storage/BrokerProperties.java} | 30 +-
.../datasource/property/storage/COSProperties.java | 14 +-
.../property/storage/HdfsProperties.java | 27 +-
.../property/storage/HdfsPropertiesUtils.java | 135 +-
.../property/storage/MinioProperties.java | 78 +
.../datasource/property/storage/OBSProperties.java | 17 +-
.../property/storage/OSSHdfsProperties.java | 2 +-
.../datasource/property/storage/OSSProperties.java | 14 +-
.../datasource/property/storage/S3Properties.java | 89 +-
.../property/storage/S3PropertyUtils.java | 63 +-
.../property/storage/StorageProperties.java | 12 +-
.../exception/StoragePropertiesException.java | 47 +
.../doris/datasource/tvf/source/TVFScanNode.java | 2 +-
.../org/apache/doris/dictionary/Dictionary.java | 46 +-
.../apache/doris/dictionary/DictionaryManager.java | 11 +-
.../org/apache/doris/fsv2/FileSystemFactory.java | 29 +
.../org/apache/doris/fsv2/StorageTypeMapper.java | 2 +
.../java/org/apache/doris/fsv2/obj/ObjStorage.java | 4 +
.../org/apache/doris/fsv2/obj/S3ObjStorage.java | 298 +-
.../apache/doris/fsv2/remote/BrokerFileSystem.java | 3 +
.../apache/doris/fsv2/remote/RemoteFileSystem.java | 4 +-
.../org/apache/doris/fsv2/remote/S3FileSystem.java | 23 +-
.../doris/fsv2/remote/dfs/DFSFileSystem.java | 3 +
.../apache/doris/httpv2/rest/CopyIntoAction.java | 1 -
.../org/apache/doris/httpv2/rest/LoadAction.java | 10 +-
.../doris/httpv2/rest/TableQueryPlanAction.java | 2 +-
.../org/apache/doris/load/BrokerFileGroup.java | 122 +-
.../main/java/org/apache/doris/load/DeleteJob.java | 3 +-
.../main/java/org/apache/doris/load/ExportJob.java | 2 +-
.../apache/doris/load/loadv2/BrokerLoadJob.java | 2 +-
.../apache/doris/load/loadv2/LoadLoadingTask.java | 13 +-
.../org/apache/doris/load/loadv2/LoadManager.java | 26 +-
.../doris/load/loadv2/LoadingTaskPlanner.java | 287 --
.../apache/doris/load/loadv2/MysqlLoadManager.java | 57 +-
.../org/apache/doris/load/loadv2/SparkLoadJob.java | 2 +-
.../doris/load/loadv2/SparkLoadPendingTask.java | 14 +-
.../doris/load/routineload/RoutineLoadJob.java | 6 +-
.../org/apache/doris/master/ReportHandler.java | 10 +-
.../org/apache/doris/mtmv/MTMVPartitionUtil.java | 28 +-
.../java/org/apache/doris/mtmv/MTMVPlanUtil.java | 4 +-
.../org/apache/doris/mtmv/MTMVRefreshContext.java | 10 +
.../org/apache/doris/mysql/privilege/Auth.java | 33 +
.../org/apache/doris/nereids/NereidsPlanner.java | 5 +-
.../apache/doris/nereids/analyzer/MappingSlot.java | 5 -
.../nereids/analyzer/UnboundIcebergTableSink.java | 4 +-
.../apache/doris/nereids/analyzer/UnboundSlot.java | 5 -
.../org/apache/doris/nereids/cost/CostModelV1.java | 25 +-
.../glue/translator/ExpressionTranslator.java | 4 +-
.../glue/translator/PhysicalPlanTranslator.java | 118 +-
.../glue/translator/PlanTranslatorContext.java | 15 +-
.../nereids/jobs/rewrite/CostBasedRewriteJob.java | 21 +-
.../doris/nereids/load/NereidsBrokerFileGroup.java | 137 +-
.../doris/nereids/load/NereidsDataDescription.java | 363 +--
.../doris/nereids/load/NereidsFileGroupInfo.java | 18 +-
.../nereids/load/NereidsLoadPlanInfoCollector.java | 24 +-
.../nereids/load/NereidsLoadScanProvider.java | 4 +-
.../doris/nereids/parser/LogicalPlanBuilder.java | 596 +++-
.../apache/doris/nereids/parser/NereidsParser.java | 3 +-
.../nereids/processor/post/PlanPostProcessors.java | 5 +
.../processor/post/RuntimeFilterGenerator.java | 13 +
.../processor/post/RuntimeFilterPruner.java | 4 +-
.../post/RuntimeFilterPushDownVisitor.java | 6 +
.../nereids/processor/post/TopnFilterContext.java | 4 +
.../processor/post/TopnFilterPushDownVisitor.java | 9 +-
.../post/materialize/LazyMaterializeTopN.java | 204 ++
.../post/materialize/LazySlotPruning.java | 245 ++
.../post/materialize/MaterializeProbeVisitor.java | 166 ++
.../post/materialize/MaterializeSource.java} | 40 +-
.../apache/doris/nereids/properties/FuncDeps.java | 7 +-
.../doris/nereids/rules/analysis/AnalyzeCTE.java | 2 -
.../nereids/rules/analysis/BindExpression.java | 25 +-
.../doris/nereids/rules/analysis/BindRelation.java | 10 +-
.../nereids/rules/analysis/SubqueryToApply.java | 2 +-
.../rules/analysis/WindowFunctionChecker.java | 7 +
.../mv/AbstractMaterializedViewAggregateRule.java | 4 +-
.../exploration/mv/MaterializedViewUtils.java | 13 +-
.../rules/expression/QueryColumnCollector.java | 4 +-
.../nereids/rules/expression/check/CheckCast.java | 4 +
.../nereids/rules/expression/rules/AddMinMax.java | 2 +-
.../expression/rules/DistinctPredicatesRule.java | 3 +-
.../expression/rules/ExtractCommonFactorRule.java | 74 +-
.../rules/PredicateRewriteForPartitionPrune.java | 7 +-
.../rules/implementation/AggregateStrategies.java | 38 +-
.../LogicalFileScanToPhysicalFileScan.java | 3 +-
.../LogicalHudiScanToPhysicalHudiScan.java | 3 +-
.../LogicalOlapScanToPhysicalOlapScan.java | 8 +-
...ogicalPartitionTopNToPhysicalPartitionTopN.java | 2 +-
.../nereids/rules/rewrite/CheckPrivileges.java | 4 +-
.../rules/rewrite/DeferMaterializeTopNResult.java | 3 +
.../nereids/rules/rewrite/EliminateGroupBy.java | 145 +-
.../nereids/rules/rewrite/ForeignKeyContext.java | 8 +-
.../rules/rewrite/OperativeColumnDerive.java | 22 +
.../rules/rewrite/PushDownAggThroughJoin.java | 34 +-
.../rewrite/PushDownAggThroughJoinOneSide.java | 18 +-
.../rewrite/PushDownFilterIntoSchemaScan.java | 2 +-
.../nereids/rules/rewrite/SetPreAggStatus.java | 18 +-
.../rules/rewrite/UnequalPredicateInfer.java | 6 +-
.../rules/rewrite/VariantSubPathPruning.java | 10 +-
.../doris/nereids/stats/FilterEstimation.java | 8 +-
.../doris/nereids/stats/StatsCalculator.java | 8 +-
.../trees/copier/LogicalPlanDeepCopier.java | 34 +
.../doris/nereids/trees/expressions/Alias.java | 26 +-
.../trees/expressions/ArrayItemReference.java | 3 +-
.../nereids/trees/expressions/Expression.java | 6 +-
.../doris/nereids/trees/expressions/Slot.java | 8 +-
.../nereids/trees/expressions/SlotReference.java | 152 +-
.../nereids/trees/expressions/functions/Udf.java | 9 +
.../trees/expressions/functions/agg/AnyValue.java | 6 +-
.../trees/expressions/functions/agg/BitmapAgg.java | 2 +-
.../expressions/functions/agg/CollectSet.java | 4 +-
.../expressions/functions/agg/GroupBitAnd.java | 2 +-
.../expressions/functions/agg/GroupBitOr.java | 2 +-
.../functions/executable/StringArithmetic.java | 9 +-
.../trees/expressions/functions/scalar/Sha2.java | 21 +
.../expressions/functions/window/NthValue.java | 92 +
.../trees/expressions/literal/DateLiteral.java | 2 +-
.../nereids/trees/expressions/literal/Literal.java | 22 +
.../expressions/visitor/WindowFunctionVisitor.java | 5 +
.../apache/doris/nereids/trees/plans/PlanType.java | 27 +-
.../trees/plans/algebra/CatalogRelation.java | 2 +
.../trees/plans/commands/AddConstraintCommand.java | 4 +-
.../commands/AlterDatabasePropertiesCommand.java | 84 +
.../trees/plans/commands/AlterResourceCommand.java | 89 +
.../plans/commands/CancelAlterTableCommand.java | 121 +
.../trees/plans/commands/CancelBackupCommand.java | 87 +
.../trees/plans/commands/CopyIntoCommand.java | 1 +
.../plans/commands/CreateDatabaseCommand.java | 119 +
.../commands/CreateMaterializedViewCommand.java | 4 +-
.../trees/plans/commands/CreatePolicyCommand.java | 2 +-
.../plans/commands/CreateRepositoryCommand.java | 106 +
.../plans/commands/CreateStorageVaultCommand.java | 180 ++
.../trees/plans/commands/CreateTableCommand.java | 4 +-
.../commands/CreateWorkloadPolicyCommand.java | 93 +
.../trees/plans/commands/DeleteFromCommand.java | 4 +-
.../trees/plans/commands/DescribeCommand.java | 17 +-
.../plans/commands/DropWorkloadGroupCommand.java | 5 +-
.../plans/commands/GrantTablePrivilegeCommand.java | 7 +-
.../trees/plans/commands/InstallPluginCommand.java | 100 +
.../plans/commands/KillConnectionCommand.java | 27 +-
.../trees/plans/commands/KillQueryCommand.java | 63 +-
.../nereids/trees/plans/commands/LoadCommand.java | 5 +-
...nnectionCommand.java => LockTablesCommand.java} | 53 +-
.../commands/RevokeResourcePrivilegeCommand.java | 155 ++
.../trees/plans/commands/RevokeRoleCommand.java | 93 +
.../commands/RevokeTablePrivilegeCommand.java | 136 +
.../commands/ShowCatalogRecycleBinCommand.java | 167 ++
.../trees/plans/commands/ShowColumnsCommand.java | 254 ++
.../nereids/trees/plans/commands/ShowCommand.java | 42 +
.../trees/plans/commands/ShowCopyCommand.java | 233 ++
.../plans/commands/ShowCreateTableCommand.java | 2 +-
.../trees/plans/commands/ShowLoadCommand.java | 33 +-
.../plans/commands/ShowOpenTablesCommand.java | 72 +
.../plans/commands/ShowPartitionsCommand.java | 472 ++++
.../plans/commands/ShowQueryStatsCommand.java | 207 ++
...sCommand.java => ShowReplicaStatusCommand.java} | 160 +-
.../trees/plans/commands/ShowResourcesCommand.java | 33 +-
.../plans/commands/ShowStorageVaultCommand.java | 96 +
.../plans/commands/ShowTransactionCommand.java | 186 ++
.../trees/plans/commands/ShowWarmUpCommand.java | 131 +
.../plans/commands/ShowWorkloadGroupsCommand.java | 90 +
...ionCommand.java => UninstallPluginCommand.java} | 53 +-
.../commands/alter/AlterRepositoryCommand.java | 3 +-
.../trees/plans/commands/info/AlterViewInfo.java | 10 +-
.../trees/plans/commands/info/CopyIntoInfo.java | 3 +-
.../trees/plans/commands/info/CreateViewInfo.java | 3 +-
.../trees/plans/commands/info/LockTableInfo.java} | 42 +-
.../commands/insert/DictionaryInsertExecutor.java | 12 +-
.../plans/commands/insert/OlapInsertExecutor.java | 5 +
.../trees/plans/commands/utils/KillUtils.java | 191 ++
.../BackendDistributedPlanWorkerManager.java | 70 +-
.../trees/plans/logical/LogicalCTEConsumer.java | 7 +-
.../plans/logical/LogicalCatalogRelation.java | 3 +-
.../trees/plans/logical/LogicalFileScan.java | 40 +-
.../trees/plans/logical/LogicalHudiScan.java | 20 +-
.../plans/logical/LogicalIcebergTableSink.java | 10 +-
.../trees/plans/logical/LogicalOlapScan.java | 2 +-
.../nereids/trees/plans/logical/LogicalView.java | 16 +-
.../plans/physical/PhysicalCatalogRelation.java | 15 +-
.../physical/PhysicalDeferMaterializeOlapScan.java | 5 +-
.../trees/plans/physical/PhysicalEsScan.java | 6 +-
.../trees/plans/physical/PhysicalFileScan.java | 42 +-
.../trees/plans/physical/PhysicalHudiScan.java | 18 +-
.../trees/plans/physical/PhysicalJdbcScan.java | 13 +-
.../plans/physical/PhysicalLazyMaterialize.java | 248 ++
.../physical/PhysicalLazyMaterializeFileScan.java | 77 +
.../physical/PhysicalLazyMaterializeOlapScan.java | 121 +
.../trees/plans/physical/PhysicalOdbcScan.java | 13 +-
.../trees/plans/physical/PhysicalOlapScan.java | 7 +-
.../trees/plans/physical/PhysicalSchemaScan.java | 7 +-
.../physical/PhysicalStorageLayerAggregate.java | 5 +-
.../trees/plans/visitor/CommandVisitor.java | 137 +-
.../trees/plans/visitor/DefaultPlanRewriter.java | 7 +-
.../nereids/trees/plans/visitor/PlanVisitor.java | 15 +
.../apache/doris/nereids/util/ExpressionUtils.java | 74 +-
.../org/apache/doris/nereids/util/PlanUtils.java | 4 +-
.../org/apache/doris/planner/AnalyticPlanner.java | 931 -------
.../doris/planner/BaseExternalTableDataSink.java | 21 +
.../apache/doris/planner/DistributedPlanner.java | 1343 ---------
.../java/org/apache/doris/planner/ExportSink.java | 2 +-
.../apache/doris/planner/GroupCommitPlanner.java | 7 +-
.../org/apache/doris/planner/HashJoinNode.java | 8 -
.../org/apache/doris/planner/HiveTableSink.java | 3 +
.../org/apache/doris/planner/IcebergTableSink.java | 7 +-
.../apache/doris/planner/MaterializationNode.java | 210 ++
.../org/apache/doris/planner/OlapScanNode.java | 64 +-
.../org/apache/doris/planner/OlapTableSink.java | 44 +-
.../org/apache/doris/planner/OriginalPlanner.java | 628 -----
.../org/apache/doris/planner/PlanFragment.java | 4 +-
.../java/org/apache/doris/planner/PlanNode.java | 4 -
.../org/apache/doris/planner/ProjectPlanner.java | 83 -
.../org/apache/doris/planner/RuntimeFilter.java | 48 +-
.../java/org/apache/doris/planner/ScanNode.java | 6 +-
.../org/apache/doris/planner/SchemaScanNode.java | 20 +-
.../apache/doris/planner/SingleNodePlanner.java | 2921 --------------------
.../java/org/apache/doris/planner/SortNode.java | 15 +-
.../apache/doris/planner/StreamLoadPlanner.java | 454 ---
.../java/org/apache/doris/plugin/PluginMgr.java | 16 +-
.../plugin/dialect/HttpDialectConverterPlugin.java | 2 +-
.../doris/plugin/dialect/HttpDialectUtils.java | 8 +-
.../main/java/org/apache/doris/policy/Policy.java | 28 -
.../java/org/apache/doris/policy/PolicyMgr.java | 55 -
.../java/org/apache/doris/policy/RowPolicy.java | 41 +-
.../java/org/apache/doris/qe/ConnectContext.java | 80 +-
.../java/org/apache/doris/qe/ConnectPoolMgr.java | 2 +-
.../main/java/org/apache/doris/qe/Coordinator.java | 3 +
.../main/java/org/apache/doris/qe/DdlExecutor.java | 10 +-
.../java/org/apache/doris/qe/GlobalVariable.java | 15 +
.../apache/doris/qe/InsertStreamTxnExecutor.java | 9 +-
.../java/org/apache/doris/qe/MultiLoadMgr.java | 18 +-
.../org/apache/doris/qe/PointQueryExecutor.java | 2 +-
.../java/org/apache/doris/qe/SessionVariable.java | 57 +-
.../java/org/apache/doris/qe/ShowExecutor.java | 2 +-
.../java/org/apache/doris/qe/StmtExecutor.java | 89 +-
.../main/java/org/apache/doris/qe/VariableMgr.java | 5 +
.../resource/computegroup/ComputeGroupMgr.java | 2 +-
.../resource/computegroup/MergedComputeGroup.java | 16 +-
.../resource/workloadgroup/WorkloadGroup.java | 12 -
.../workloadgroup/WorkloadGroupChecker.java | 4 +-
.../resource/workloadgroup/WorkloadGroupMgr.java | 104 +-
.../WorkloadSchedPolicyMgr.java | 23 +-
.../scheduler/manager/TransientTaskManager.java | 2 +-
.../apache/doris/service/FrontendServiceImpl.java | 4 +
.../apache/doris/statistics/AnalysisManager.java | 4 +-
.../org/apache/doris/statistics/Statistics.java | 2 +-
.../apache/doris/statistics/TableStatsMeta.java | 22 +-
.../doris/statistics/hbo/ScanPlanStatistics.java | 4 +-
.../doris/statistics/util/StatisticsUtil.java | 38 +-
.../ExternalFileTableValuedFunction.java | 22 +-
.../GroupCommitTableValuedFunction.java | 2 +-
.../tablefunction/HdfsTableValuedFunction.java | 38 +-
.../HttpStreamTableValuedFunction.java | 2 +-
.../tablefunction/LocalTableValuedFunction.java | 2 +-
.../doris/tablefunction/S3TableValuedFunction.java | 128 +-
.../main/java/org/apache/doris/task/AgentTask.java | 11 +
.../org/apache/doris/task/AgentTaskExecutor.java | 19 +-
.../java/org/apache/doris/task/AgentTaskQueue.java | 13 +
.../java/org/apache/doris/alter/AlterTest.java | 9 +-
.../analysis/CreateTableAsSelectStmtTest.java | 692 -----
.../apache/doris/analysis/DataDescriptionTest.java | 26 +-
.../doris/analysis/InstallPluginCommandTest.java | 63 +
.../org/apache/doris/backup/BackupJobTest.java | 9 +-
.../org/apache/doris/backup/RepositoryTest.java | 49 +-
.../org/apache/doris/backup/RestoreJobTest.java | 11 +-
.../apache/doris/catalog/CreateFunctionTest.java | 4 +-
.../apache/doris/catalog/CreateTableLikeTest.java | 10 +-
.../apache/doris/common/proc/DbsProcDirTest.java | 4 +-
.../doris/common/util/AutoBucketUtilsTest.java | 20 +-
.../datasource/ExternalRowCountCacheTest.java | 6 +-
.../dlf/client/IcebergDLFExternalCatalogTest.java | 5 +-
.../datasource/property/PropertyConverterTest.java | 16 +-
.../property/PropertyPassThroughTest.java | 6 +-
.../fileformat/JsonFileFormatPropertiesTest.java | 2 +
.../property/storage/AzurePropertiesTest.java | 7 +-
.../property/storage/COSPropertiesTest.java | 3 +-
.../property/storage/HdfsPropertiesTest.java | 9 +-
.../property/storage/HdfsPropertiesUtilsTest.java | 23 +-
.../property/storage/MinioPropertiesTest.java | 99 +
.../property/storage/OBSPropertyTest.java | 4 +-
.../property/storage/OSSHdfsPropertiesTest.java | 2 +-
.../property/storage/OSSPropertiesTest.java | 6 +-
.../property/storage/S3PropertiesTest.java | 76 +-
.../property/storage/S3PropertyUtilsTest.java | 17 +-
.../doris/{fs => fsv2}/obj/S3FileSystemTest.java | 22 +-
.../load/loadv2/SparkLoadPendingTaskTest.java | 4 +
.../load/routineload/RoutineLoadSchedulerTest.java | 3 +-
.../doris/load/sync/canal/CanalSyncDataTest.java | 4 +-
.../apache/doris/mtmv/MTMVPartitionUtilTest.java | 21 +
.../org/apache/doris/mtmv/MTMVPlanUtilTest.java | 4 +-
.../apache/doris/nereids/UnsupportedTypeTest.java | 24 -
.../nereids/jobs/cascades/DeriveStatsJobTest.java | 5 +-
.../doris/nereids/parser/EncryptSQLTest.java | 8 +-
.../doris/nereids/parser/NereidsParserTest.java | 61 +-
.../PushDownFilterThroughProjectTest.java | 2 +-
.../nereids/postprocess/TopNRuntimeFilterTest.java | 16 +-
.../postprocess/TopnLazyMaterializeTest.java | 54 +
.../doris/nereids/properties/EqualSetTest.java | 2 +-
.../apache/doris/nereids/properties/FdTest.java | 2 +-
.../doris/nereids/properties/FuncDepsTest.java | 12 +
.../doris/nereids/properties/UniformTest.java | 4 +-
.../doris/nereids/properties/UniqueTest.java | 4 +-
.../rules/analysis/AnalyzeWhereSubqueryTest.java | 4 +-
.../rules/expression/ExpressionRewriteTest.java | 11 +
.../expression/ExpressionRewriteTestHelper.java | 2 +-
.../rules/rewrite/EliminateGroupByTest.java | 49 +-
.../nereids/rules/rewrite/InferPredicatesTest.java | 2 +-
.../rewrite/PushDownCountThroughJoinTest.java | 57 +-
.../nereids/rules/rewrite/PushDownLimitTest.java | 3 +-
.../doris/nereids/stats/StatsCalculatorTest.java | 10 +-
.../trees/copier/LogicalPlanDeepCopierTest.java} | 41 +-
.../doris/nereids/trees/plans/ConstraintTest.java | 4 +-
.../commands/CancelAlterTableCommandTest.java | 88 +
...mmandTest.java => CancelBackupCommandTest.java} | 56 +-
.../plans/commands/CreateDatabaseCommandTest.java | 68 +
.../commands/CreateRepositoryCommandTest.java | 125 +
.../commands/CreateStorageVaultCommandTest.java | 87 +
.../trees/plans/commands/KillCommandTest.java | 315 +++
.../plans/commands/KillConnectionCommandTest.java | 71 -
.../RevokeResourcePrivilegeCommandTest.java | 133 +
.../plans/commands/RevokeRoleCommandTest.java | 74 +
.../commands/RevokeTablePrivilegeCommandTest.java | 98 +
.../commands/ShowCatalogRecycleBinCommandTest.java | 105 +
...adCommandTest.java => ShowCopyCommandTest.java} | 79 +-
.../trees/plans/commands/ShowLoadCommandTest.java | 20 +-
.../plans/commands/ShowPartitionsCommandTest.java | 239 ++
.../plans/commands/ShowQueryStatsCommandTest.java | 116 +
.../commands/ShowReplicaStatusCommandTest.java | 118 +
.../plans/commands/ShowResourcesCommandTest.java | 9 +-
.../plans/commands/ShowTransactionCommandTest.java | 94 +
.../plans/commands/ShowWarmUpCommandTest.java | 51 +
.../trees/plans/commands/utils/KillUtilsTest.java | 355 +++
.../apache/doris/nereids/util/PlanConstructor.java | 20 +
.../doris/planner/SingleNodePlannerTest.java | 1085 --------
.../apache/doris/plugin/HttpDialectUtilsTest.java | 10 +-
.../java/org/apache/doris/policy/PolicyTest.java | 4 +-
.../java/org/apache/doris/qe/CoordinatorTest.java | 877 ------
.../org/apache/doris/qe/OlapQueryCacheTest.java | 16 +
.../apache/doris/resource/ComputeGroupTest.java | 36 +-
.../workloadgroup/WorkloadGroupMgrTest.java | 5 +-
.../doris/statistics/AnalysisManagerTest.java | 7 +-
.../apache/doris/utframe/TestWithFeService.java | 8 +-
fe/pom.xml | 21 +-
gensrc/proto/cloud.proto | 9 +-
gensrc/proto/internal_service.proto | 37 +
gensrc/thrift/DataSinks.thrift | 2 +
gensrc/thrift/PlanNodes.thrift | 25 +-
gensrc/thrift/Status.thrift | 1 +
regression-test/conf/regression-conf.groovy | 1 +
.../diffrent_serialize/diffrent_serialize.out | Bin 280 -> 512 bytes
.../multi_cluster/test_mow_agg_delete_bitmap.out | Bin 0 -> 296 bytes
...compaction_agg_and_remove_pre_delete_bitmap.out | Bin 0 -> 257 bytes
.../test_mow_compaction_and_read_stale.out | Bin 0 -> 287 bytes
..._mow_compaction_and_read_stale_cloud_docker.out | Bin 0 -> 287 bytes
.../test_mow_compaction_and_rowset_not_exist.out | Bin 0 -> 300 bytes
.../test_mow_compaction_and_schema_change.out | Bin 0 -> 720 bytes
.../agg_state/array/array.out} | Bin 123 -> 123 bytes
.../agg_state/bitmap/bitmap.out} | Bin 123 -> 118 bytes
.../agg_state/hll/hll.out} | Bin 123 -> 116 bytes
.../agg_state/map/map.out} | Bin 123 -> 137 bytes
.../agg_state/nereids/test_agg_state_nereids.out | Bin 456 -> 465 bytes
.../data/datatype_p0/agg_state/test_agg_state.out | Bin 502 -> 511 bytes
.../data/ddl_p0/test_create_or_replace_view.out | Bin 135 -> 605 bytes
.../test_invalid_skip_load.out} | Bin 123 -> 115 bytes
.../hive/test_hive_meta_cache.out | Bin 1649 -> 2098 bytes
.../hive/test_hive_rename_column_orc_parquet.out | Bin 6269 -> 9919 bytes
.../hive/test_hive_topn_lazy_mat.out | Bin 0 -> 19465 bytes
.../hive/test_string_dict_filter.out | Bin 11747 -> 20131 bytes
.../hive/test_transactional_hive.out | Bin 925 -> 1104 bytes
.../jdbc/test_sqlserver_jdbc_catalog.out | Bin 5638 -> 5667 bytes
.../test_outfile_s3_storage.out | Bin 0 -> 1005 bytes
.../test_external_and_internal_describe.out | Bin 0 -> 2972 bytes
.../test_group_commit_schema_change.out} | Bin 123 -> 115 bytes
.../inverted_index_p0/test_index_match_regexp.out | Bin 234 -> 822 bytes
.../test_variant_empty_index_file.out} | Bin 123 -> 116 bytes
.../data/javaudf_p0/test_javaudf_override.out | Bin 0 -> 299 bytes
.../data/mtmv_p0/test_hive_refresh_mtmv.out | Bin 363 -> 411 bytes
.../diffrent_serialize/diffrent_serialize.out | Bin 363 -> 626 bytes
.../test_nereids_alter_database_set_quota.out | Bin 725 -> 731 bytes
.../outfile/agg_state/test_outfile_agg_state.out} | Bin 123 -> 155 bytes
.../agg_state_array/test_outfile_agg_array.out} | Bin 123 -> 136 bytes
.../test_outfile_agg_state_bitmap.out} | Bin 123 -> 141 bytes
.../outfile/hll/test_outfile_hll.out} | Bin 123 -> 137 bytes
.../test_outfile_quantile_state.out} | Bin 123 -> 145 bytes
.../nereids_p0/show/test_show_columns_command.out | Bin 0 -> 340 bytes
.../encryption_digest/test_encryption_function.out | Bin 1175 -> 1403 bytes
.../nereids_p0/subquery/subquery_unnesting.out | Bin 7209 -> 7253 bytes
.../nereids_rules_p0/eager_aggregate/basic.out | Bin 3241 -> 3079 bytes
.../eager_aggregate/basic_one_side.out | Bin 3261 -> 3099 bytes
.../push_down_count_through_join_one_side.out | Bin 38441 -> 34895 bytes
.../push_down_sum_through_join_one_side.out | Bin 20501 -> 18663 bytes
.../eliminate_gby_key/eliminate_group_by.out | Bin 0 -> 3494 bytes
.../data/nereids_syntax_p0/any_value.out | Bin 145 -> 1452 bytes
.../data}/plugin_test/auditdemo.zip | Bin
.../data/point_query_p0/test_point_query.out | Bin 9920 -> 9934 bytes
.../data/query_p0/cache/query_cache.out | Bin 365 -> 635 bytes
.../join/mark_join/right_semi_mark_join.out | Bin 0 -> 568 bytes
.../set_operations/set_with_null/set_with_null.out | Bin 123 -> 248 bytes
.../window_functions/test_nthvalue_function.out | Bin 0 -> 1555 bytes
.../data/query_p0/topn_lazy/topn_lazy.out | Bin 0 -> 3437 bytes
.../query_p0/topn_lazy/topn_lazy_on_data_model.out | Bin 0 -> 1272 bytes
.../data/shape_check/clickbench/query24.out | Bin 355 -> 4021 bytes
regression-test/data/shape_check/others/nlj.out | Bin 0 -> 481 bytes
.../tpcds_sf100/noStatsRfPrune/query13.out | Bin 3603 -> 3603 bytes
.../tpcds_sf100/noStatsRfPrune/query24.out | Bin 3152 -> 3122 bytes
.../tpcds_sf100/noStatsRfPrune/query30.out | Bin 2420 -> 3113 bytes
.../tpcds_sf100/noStatsRfPrune/query41.out | Bin 2404 -> 2362 bytes
.../tpcds_sf100/noStatsRfPrune/query44.out | Bin 4406 -> 4881 bytes
.../tpcds_sf100/noStatsRfPrune/query6.out | Bin 3007 -> 2984 bytes
.../tpcds_sf100/noStatsRfPrune/query65.out | Bin 2503 -> 2916 bytes
.../tpcds_sf100/noStatsRfPrune/query68.out | Bin 2605 -> 3016 bytes
.../tpcds_sf100/noStatsRfPrune/query8.out | Bin 9787 -> 10013 bytes
.../tpcds_sf100/noStatsRfPrune/query81.out | Bin 2440 -> 3069 bytes
.../tpcds_sf100/no_stats_shape/query13.out | Bin 3647 -> 3647 bytes
.../tpcds_sf100/no_stats_shape/query24.out | Bin 3308 -> 3278 bytes
.../tpcds_sf100/no_stats_shape/query30.out | Bin 2517 -> 3272 bytes
.../tpcds_sf100/no_stats_shape/query41.out | Bin 2404 -> 2362 bytes
.../tpcds_sf100/no_stats_shape/query44.out | Bin 4406 -> 4881 bytes
.../tpcds_sf100/no_stats_shape/query6.out | Bin 3217 -> 3194 bytes
.../tpcds_sf100/no_stats_shape/query65.out | Bin 2589 -> 3002 bytes
.../tpcds_sf100/no_stats_shape/query68.out | Bin 2765 -> 3176 bytes
.../tpcds_sf100/no_stats_shape/query8.out | Bin 9831 -> 10057 bytes
.../tpcds_sf100/no_stats_shape/query81.out | Bin 2537 -> 3228 bytes
.../shape_check/tpcds_sf100/rf_prune/query13.out | Bin 3630 -> 3630 bytes
.../shape_check/tpcds_sf100/rf_prune/query24.out | Bin 3152 -> 3122 bytes
.../shape_check/tpcds_sf100/rf_prune/query30.out | Bin 2484 -> 3239 bytes
.../shape_check/tpcds_sf100/rf_prune/query41.out | Bin 2404 -> 2362 bytes
.../shape_check/tpcds_sf100/rf_prune/query44.out | Bin 4206 -> 4681 bytes
.../shape_check/tpcds_sf100/rf_prune/query6.out | Bin 3256 -> 3233 bytes
.../shape_check/tpcds_sf100/rf_prune/query65.out | Bin 2499 -> 2912 bytes
.../shape_check/tpcds_sf100/rf_prune/query68.out | Bin 2797 -> 3208 bytes
.../shape_check/tpcds_sf100/rf_prune/query8.out | Bin 9785 -> 10011 bytes
.../shape_check/tpcds_sf100/rf_prune/query81.out | Bin 2432 -> 3061 bytes
.../data/shape_check/tpcds_sf100/shape/query13.out | Bin 3674 -> 3674 bytes
.../data/shape_check/tpcds_sf100/shape/query24.out | Bin 3307 -> 3277 bytes
.../data/shape_check/tpcds_sf100/shape/query30.out | Bin 2536 -> 3291 bytes
.../data/shape_check/tpcds_sf100/shape/query41.out | Bin 2404 -> 2362 bytes
.../data/shape_check/tpcds_sf100/shape/query44.out | Bin 4206 -> 4681 bytes
.../data/shape_check/tpcds_sf100/shape/query6.out | Bin 3310 -> 3287 bytes
.../data/shape_check/tpcds_sf100/shape/query65.out | Bin 2585 -> 2998 bytes
.../data/shape_check/tpcds_sf100/shape/query68.out | Bin 2797 -> 3208 bytes
.../data/shape_check/tpcds_sf100/shape/query8.out | Bin 9829 -> 10055 bytes
.../data/shape_check/tpcds_sf100/shape/query81.out | Bin 2529 -> 3220 bytes
.../tpcds_sf1000/bs_downgrade_shape/query13.out | Bin 3678 -> 3678 bytes
.../tpcds_sf1000/bs_downgrade_shape/query44.out | Bin 4200 -> 4675 bytes
.../tpcds_sf1000/bs_downgrade_shape/query6.out | Bin 3310 -> 3287 bytes
.../tpcds_sf1000/bs_downgrade_shape/query68.out | Bin 2786 -> 3197 bytes
.../tpcds_sf1000/bs_downgrade_shape/query8.out | Bin 9829 -> 10055 bytes
.../data/shape_check/tpcds_sf1000/hint/query13.out | Bin 3814 -> 3814 bytes
.../data/shape_check/tpcds_sf1000/hint/query24.out | Bin 3465 -> 3435 bytes
.../data/shape_check/tpcds_sf1000/hint/query30.out | Bin 2657 -> 3412 bytes
.../data/shape_check/tpcds_sf1000/hint/query41.out | Bin 2379 -> 2338 bytes
.../data/shape_check/tpcds_sf1000/hint/query44.out | Bin 4279 -> 4754 bytes
.../data/shape_check/tpcds_sf1000/hint/query6.out | Bin 3310 -> 3287 bytes
.../data/shape_check/tpcds_sf1000/hint/query65.out | Bin 2820 -> 3237 bytes
.../data/shape_check/tpcds_sf1000/hint/query68.out | Bin 2945 -> 3356 bytes
.../data/shape_check/tpcds_sf1000/hint/query8.out | Bin 10032 -> 10258 bytes
.../data/shape_check/tpcds_sf1000/hint/query81.out | Bin 2669 -> 3360 bytes
.../shape_check/tpcds_sf1000/shape/query13.out | Bin 3678 -> 3678 bytes
.../shape_check/tpcds_sf1000/shape/query24.out | Bin 3312 -> 3282 bytes
.../shape_check/tpcds_sf1000/shape/query30.out | Bin 2517 -> 3272 bytes
.../shape_check/tpcds_sf1000/shape/query41.out | Bin 2379 -> 2338 bytes
.../shape_check/tpcds_sf1000/shape/query44.out | Bin 4200 -> 4675 bytes
.../data/shape_check/tpcds_sf1000/shape/query6.out | Bin 3310 -> 3287 bytes
.../shape_check/tpcds_sf1000/shape/query65.out | Bin 2585 -> 2998 bytes
.../shape_check/tpcds_sf1000/shape/query68.out | Bin 2786 -> 3197 bytes
.../data/shape_check/tpcds_sf1000/shape/query8.out | Bin 9829 -> 10055 bytes
.../shape_check/tpcds_sf1000/shape/query81.out | Bin 2529 -> 3220 bytes
.../shape_check/tpcds_sf10t_orc/shape/query13.out | Bin 3641 -> 3641 bytes
.../shape_check/tpcds_sf10t_orc/shape/query24.out | Bin 3302 -> 3272 bytes
.../shape_check/tpcds_sf10t_orc/shape/query41.out | Bin 2388 -> 2353 bytes
.../shape_check/tpcds_sf10t_orc/shape/query44.out | Bin 4358 -> 4833 bytes
.../shape_check/tpcds_sf10t_orc/shape/query6.out | Bin 3215 -> 3192 bytes
.../shape_check/tpcds_sf10t_orc/shape/query65.out | Bin 2589 -> 3002 bytes
.../shape_check/tpcds_sf10t_orc/shape/query68.out | Bin 2755 -> 3166 bytes
.../shape_check/tpcds_sf10t_orc/shape/query8.out | Bin 9829 -> 10055 bytes
.../data/shape_check/tpch_sf1000/hint/q11.out | Bin 2104 -> 2074 bytes
.../tpch_sf1000/nostats_rf_prune/q11.out | Bin 2197 -> 2167 bytes
.../tpch_sf1000/nostats_rf_prune/q2.out | Bin 1897 -> 2290 bytes
.../tpch_sf1000/nostats_rf_prune/q22.out | Bin 1376 -> 1350 bytes
.../data/shape_check/tpch_sf1000/rf_prune/q11.out | Bin 2205 -> 2175 bytes
.../data/shape_check/tpch_sf1000/rf_prune/q2.out | Bin 1968 -> 2407 bytes
.../data/shape_check/tpch_sf1000/rf_prune/q22.out | Bin 1313 -> 1287 bytes
.../runtime_filter/test_pushdown_setop.out | Bin 1662 -> 1884 bytes
.../data/shape_check/tpch_sf1000/shape/q11.out | Bin 2205 -> 2175 bytes
.../data/shape_check/tpch_sf1000/shape/q2.out | Bin 1968 -> 2407 bytes
.../data/shape_check/tpch_sf1000/shape/q22.out | Bin 1313 -> 1287 bytes
.../shape_check/tpch_sf1000/shape_no_stats/q11.out | Bin 2197 -> 2167 bytes
.../shape_check/tpch_sf1000/shape_no_stats/q2.out | Bin 1897 -> 2332 bytes
.../shape_check/tpch_sf1000/shape_no_stats/q22.out | Bin 1376 -> 1350 bytes
.../regression/action/StreamLoadAction.groovy | 20 +-
.../org/apache/doris/regression/suite/Suite.groovy | 35 -
.../doris/regression/suite/SuiteCluster.groovy | 17 +-
.../doris/regression/util/LoggerUtils.groovy | 15 +-
.../main/java/org/apache/doris/udf/StringTest.java | 3 +
.../pipeline/cloud_p0/conf/fe_custom.conf | 1 +
.../pipeline/external/conf/regression-conf.groovy | 2 +
.../cloud_p0/auth/test_set_default_cluster.groovy | 6 +-
.../diffrent_serialize/diffrent_serialize.groovy | 20 +-
.../test_mow_agg_delete_bitmap.groovy | 327 +++
.../multi_cluster/test_no_cluster_hits.groovy | 29 +-
.../node_mgr/test_rename_compute_group.groovy | 4 +-
.../test_base_compaction_after_sc_fail.groovy | 100 +
...paction_agg_and_remove_pre_delete_bitmap.groovy | 258 ++
.../test_mow_compaction_and_read_stale.groovy | 300 ++
...w_compaction_and_read_stale_cloud_docker.groovy | 328 +++
...test_mow_compaction_and_rowset_not_exist.groovy | 311 +++
.../test_mow_compaction_and_schema_change.groovy | 343 +++
.../agg_state/array/array.groovy} | 48 +-
.../agg_state/bitmap/bitmap.groovy} | 48 +-
.../agg_state/hll/hll.groovy} | 48 +-
.../agg_state/map/map.groovy} | 49 +-
.../ddl_p0/test_create_or_replace_view.groovy | 23 +
.../test_dict_load_and_get_hash_map.groovy | 75 +-
.../test_dict_load_and_get_ip_trie.groovy | 1 +
.../test_dict_nullable_key.groovy | 1 +
.../dictionary_p0/test_invalid_skip_load.groovy | 80 +
.../hive/test_external_sql_block_rule.groovy | 16 +-
.../hive/test_hive_meta_cache.groovy | 60 +
.../test_hive_rename_column_orc_parquet.groovy | 63 +-
.../hive/test_hive_statistics_p0.groovy | 15 +-
.../hive/test_hive_topn_lazy_mat.groovy | 209 ++
.../hive/test_string_dict_filter.groovy | 36 +
.../hive/test_transactional_hive.groovy | 20 +
.../jdbc/test_jdbc_query_pg.groovy | 2 +-
.../jdbc/test_sqlserver_jdbc_catalog.groovy | 2 +
.../test_iceberg_hadoop_catalog_kerberos.groovy | 98 +
.../kerberos/test_two_hive_kerberos.groovy | 45 +-
.../backup_restore_azure.groovy | 182 ++
.../backup_restore_object_storage.groovy | 300 ++
.../refactor_storage_param/hdfs_all_test.groovy | 319 +++
.../refactor_storage_param/s3_load.groovy | 285 ++
.../test_outfile_s3_storage.groovy | 218 ++
.../test_s3_tvf_s3_storage.groovy | 265 ++
.../test_external_and_internal_describe.groovy | 163 ++
.../external_table_p0/tvf/test_hdfs_tvf.groovy | 4 +-
.../group_commit/test_group_commit_error.groovy | 31 +
.../test_group_commit_replay_wal.groovy | 15 +-
.../test_group_commit_schema_change.groovy | 135 +
.../test_index_match_regexp.groovy | 47 +
.../test_variant_empty_index_file.groovy | 57 +
.../suites/javaudf_p0/test_javaudf_array.groovy | 12 +-
.../suites/javaudf_p0/test_javaudf_int.groovy | 26 +
.../suites/javaudf_p0/test_javaudf_override.groovy | 82 +
.../suites/javaudf_p0/test_javaudf_string.groovy | 13 +
...t_domain_connection_and_ak_sk_correction.groovy | 2 +-
.../test_routine_load_error_info.groovy | 174 +-
.../test_group_commit_and_wal_back_pressure.groovy | 2 +-
.../test_base_add_col_multi_level_mtmv.groovy | 30 +-
..._base_alter_col_comment_multi_level_mtmv.groovy | 24 +-
...est_base_alter_col_type_multi_level_mtmv.groovy | 23 +-
.../test_base_comment_multi_level_mtmv.groovy | 23 +-
.../test_base_drop_col_multi_level_mtmv.groovy | 25 +-
.../mtmv_p0/test_base_drop_multi_level_mtmv.groovy | 30 +-
.../test_base_recreate_on_commit_mtmv.groovy | 30 +-
.../test_base_rename_col_multi_level_mtmv.groovy | 24 +-
.../test_base_rename_multi_level_mtmv.groovy | 24 +-
.../test_base_rename_mv_multi_level_mtmv.groovy | 36 +-
.../mtmv_p0/test_base_rename_on_commit_mtmv.groovy | 55 +-
.../test_base_rename_p_mv_multi_level_mtmv.groovy | 26 +-
.../test_base_replace_multi_level_mtmv.groovy | 63 +-
.../test_base_replace_mv_multi_level_mtmv.groovy | 84 +-
.../test_base_replace_on_commit_mtmv.groovy | 45 +-
.../suites/mtmv_p0/test_column_type_mtmv.groovy | 30 +-
.../suites/mtmv_p0/test_hive_refresh_mtmv.groovy | 10 +-
.../suites/mtmv_p0/test_recreate_table_mtmv.groovy | 37 +-
.../diffrent_serialize/diffrent_serialize.groovy | 38 +-
.../suites/mysql_compatibility_p0/metadata.groovy | 59 +-
.../cte/test_cte_with_duplicate_consumer.groovy | 8 +-
.../ddl/grant/test_revoke_role_nereids.groovy | 47 +
.../fold_constant/fold_constant_by_fe.groovy | 4 +-
.../agg_state/test_outfile_agg_state.groovy | 68 +
.../agg_state_array/test_outfile_agg_array.groovy | 67 +
.../test_outfile_agg_state_bitmap.groovy | 66 +
.../nereids_p0/outfile/hll/test_outfile_hll.groovy | 67 +
.../test_outfile_quantile_state.groovy | 68 +
.../show/test_nereids_show_partitions.groovy | 197 ++
.../show/test_show_columns_command.groovy | 57 +
.../show/test_show_open_tables_command.groovy} | 17 +-
.../test_show_storage_vault_command.groovy} | 29 +-
.../test_encryption_function.groovy | 7 +
.../nereids_p0/subquery/subquery_unnesting.groovy | 9 +
.../subquery/test_duplicate_name_in_view.groovy | 175 --
.../test_alter_database_properties_command.groovy} | 20 +-
.../nereids_p0/test_alter_resource_nereids.groovy | 50 +
.../test_create_workload_policy_command.groovy} | 17 +-
...r.groovy => test_install_plugin_command.groovy} | 27 +-
...umer.groovy => test_lock_tables_command.groovy} | 36 +-
.../union/push_limit_with_eliminate_union.groovy | 2 +-
.../adjust_nullable/set_operation.groovy | 3 +-
.../eliminate_gby_key/eliminate_group_by.groovy | 54 +
.../test_convert_median_to_percentile.groovy | 4 +-
.../pull_up_predicate_set_op.groovy | 4 +-
.../infer_set_operator_distinct.groovy | 6 +-
.../limit_push_down/order_push_down.groovy | 2 +-
.../push_down_top_n_distinct_through_union.groovy | 2 +-
.../push_down_top_n_through_union.groovy | 2 +-
.../suites/nereids_syntax_p0/agg_4_phase.groovy | 10 +
.../suites/nereids_syntax_p0/any_value.groovy | 179 ++
.../suites/nereids_syntax_p0/cast.groovy | 180 +-
.../suites/nereids_syntax_p0/cte.groovy | 2 +-
.../suites/nereids_syntax_p0/explain.groovy | 5 +-
.../suites/nereids_syntax_p0/set_operation.groovy | 8 +-
.../suites/point_query_p0/test_point_query.groovy | 21 +-
.../suites/query_p0/cache/query_cache.groovy | 48 +-
.../suites/query_p0/cache/sql_cache.groovy | 12 +-
.../query_p0/cte/query_with_dup_column.groovy | 112 -
.../query_p0/join/mark_join/mark_join.groovy | 181 +-
.../join/mark_join/right_semi_mark_join.groovy | 109 +
.../set_with_null/set_with_null.groovy | 179 ++
regression-test/suites/query_p0/sort/sort.groovy | 2 +
.../suites/query_p0/sort/topn_2pr_rule.groovy | 2 +
.../encryption_digest/test_digest.groovy | 25 +-
.../window_functions/test_nthvalue_function.groovy | 82 +
.../query_p0/topn_lazy/ddl/customer_create.sql | 15 +
.../suites/query_p0/topn_lazy/ddl/dates_create.sql | 24 +
.../query_p0/topn_lazy/ddl/lineorder_create.sql | 24 +
.../suites/query_p0/topn_lazy/ddl/part_create.sql | 16 +
.../suites/query_p0/topn_lazy/ddl/ssb.tables.sql | 89 +
.../query_p0/topn_lazy/ddl/supplier_create.sql | 14 +
.../suites/query_p0/topn_lazy/load.groovy | 83 +
.../suites/query_p0/topn_lazy/topn_lazy.groovy | 208 ++
.../topn_lazy/topn_lazy_on_data_model.groovy | 80 +
.../suites/shape_check/others/nlj.groovy | 73 +
.../suites/show_p0/test_show_data_warehouse.groovy | 10 +-
.../test_skip_agg_table_value_column.groovy | 78 +-
.../workload_manager_p0/test_curd_wlg.groovy | 31 -
run-fe-ut.sh | 2 +-
thirdparty/vars.sh | 10 +-
ui/src/pages/query-profile/index.tsx | 2 +-
1211 files changed, 34660 insertions(+), 22109 deletions(-)
create mode 100644 be/src/olap/id_manager.h
create mode 100644 be/src/pipeline/exec/materialization_sink_operator.cpp
create mode 100644 be/src/pipeline/exec/materialization_sink_operator.h
create mode 100644 be/src/pipeline/exec/materialization_source_operator.cpp
create mode 100644 be/src/pipeline/exec/materialization_source_operator.h
delete mode 100644 be/src/util/jsonb_parser.h
create mode 100644 be/test/olap/id_manager_test.cpp
create mode 100644
be/test/olap/rowset/segment_v2/inverted_index/empty_index_file_test.cpp
create mode 100644
be/test/olap/rowset/segment_v2/inverted_index/query/regexp_query_test.cpp
create mode 100644
be/test/pipeline/operator/materialization_shared_state_test.cpp
copy be/{src/vec/functions/function_case.cpp =>
test/vec/aggregate_functions/agg_count_test.cpp} (66%)
create mode 100644 be/test/vec/exec/format/parquet/parquet_read_lines.cpp
create mode 100644 be/test/vec/exec/orc/orc_read_lines.cpp
create mode 100644
docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run80.hql
create mode 100644
docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/orc_table/orc_global_lazy_mat_table/file_id=1/example_1.orc
create mode 100644
docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/orc_table/orc_global_lazy_mat_table/file_id=2/example_2.orc
create mode 100644
docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/orc_table/orc_global_lazy_mat_table/file_id=2/example_3.orc
create mode 100644
docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/parquet_global_lazy_mat_table/file_id=1/example_1.parquet
create mode 100644
docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/parquet_global_lazy_mat_table/file_id=1/example_3.parquet
create mode 100644
docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/parquet_table/parquet_global_lazy_mat_table/file_id=2/example_2.parquet
delete mode 100644
fe/fe-core/src/main/java/org/apache/doris/analysis/CreatePolicyStmt.java
delete mode 100644
fe/fe-core/src/main/java/org/apache/doris/cloud/planner/CloudStreamLoadPlanner.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/datasource/property/fileformat/ArrowFileFormatProperties.java
copy fe/fe-core/src/main/java/org/apache/doris/{fsv2/FileSystemFactory.java =>
datasource/property/storage/BrokerProperties.java} (57%)
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/MinioProperties.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/exception/StoragePropertiesException.java
delete mode 100644
fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/materialize/LazyMaterializeTopN.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/materialize/LazySlotPruning.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/materialize/MaterializeProbeVisitor.java
copy fe/fe-core/src/main/java/org/apache/doris/{task/AgentTaskExecutor.java =>
nereids/processor/post/materialize/MaterializeSource.java} (59%)
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/NthValue.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterDatabasePropertiesCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterResourceCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelAlterTableCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelBackupCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateDatabaseCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateRepositoryCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateStorageVaultCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateWorkloadPolicyCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InstallPluginCommand.java
copy
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/{KillConnectionCommand.java
=> LockTablesCommand.java} (51%)
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/RevokeResourcePrivilegeCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/RevokeRoleCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/RevokeTablePrivilegeCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCatalogRecycleBinCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowColumnsCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCopyCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowOpenTablesCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowPartitionsCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowQueryStatsCommand.java
rename
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/{AdminShowReplicaStatusCommand.java
=> ShowReplicaStatusCommand.java} (62%)
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowStorageVaultCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTransactionCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowWarmUpCommand.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowWorkloadGroupsCommand.java
copy
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/{KillConnectionCommand.java
=> UninstallPluginCommand.java} (51%)
copy fe/fe-core/src/main/java/org/apache/doris/{task/AgentTaskExecutor.java =>
nereids/trees/plans/commands/info/LockTableInfo.java} (51%)
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/utils/KillUtils.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalLazyMaterialize.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalLazyMaterializeFileScan.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalLazyMaterializeOlapScan.java
delete mode 100644
fe/fe-core/src/main/java/org/apache/doris/planner/AnalyticPlanner.java
delete mode 100644
fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java
create mode 100644
fe/fe-core/src/main/java/org/apache/doris/planner/MaterializationNode.java
delete mode 100644
fe/fe-core/src/main/java/org/apache/doris/planner/OriginalPlanner.java
delete mode 100644
fe/fe-core/src/main/java/org/apache/doris/planner/ProjectPlanner.java
delete mode 100644
fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java
delete mode 100644
fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java
delete mode 100644
fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableAsSelectStmtTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/analysis/InstallPluginCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/MinioPropertiesTest.java
rename fe/fe-core/src/test/java/org/apache/doris/{fs =>
fsv2}/obj/S3FileSystemTest.java (92%)
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/TopnLazyMaterializeTest.java
copy
fe/fe-core/src/{main/java/org/apache/doris/nereids/trees/plans/algebra/CatalogRelation.java
=>
test/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopierTest.java}
(52%)
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/CancelAlterTableCommandTest.java
rename
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/{KillQueryCommandTest.java
=> CancelBackupCommandTest.java} (50%)
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/CreateDatabaseCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/CreateRepositoryCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/CreateStorageVaultCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/KillCommandTest.java
delete mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/KillConnectionCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/RevokeResourcePrivilegeCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/RevokeRoleCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/RevokeTablePrivilegeCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowCatalogRecycleBinCommandTest.java
copy
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/{ShowLoadCommandTest.java
=> ShowCopyCommandTest.java} (63%)
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowPartitionsCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowQueryStatsCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowReplicaStatusCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowTransactionCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowWarmUpCommandTest.java
create mode 100644
fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/utils/KillUtilsTest.java
delete mode 100644
fe/fe-core/src/test/java/org/apache/doris/planner/SingleNodePlannerTest.java
delete mode 100644
fe/fe-core/src/test/java/org/apache/doris/qe/CoordinatorTest.java
create mode 100644
regression-test/data/cloud_p0/multi_cluster/test_mow_agg_delete_bitmap.out
create mode 100644
regression-test/data/compaction/test_mow_compaction_agg_and_remove_pre_delete_bitmap.out
create mode 100644
regression-test/data/compaction/test_mow_compaction_and_read_stale.out
create mode 100644
regression-test/data/compaction/test_mow_compaction_and_read_stale_cloud_docker.out
create mode 100644
regression-test/data/compaction/test_mow_compaction_and_rowset_not_exist.out
create mode 100644
regression-test/data/compaction/test_mow_compaction_and_schema_change.out
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> datatype_p0/agg_state/array/array.out} (87%)
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> datatype_p0/agg_state/bitmap/bitmap.out} (87%)
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> datatype_p0/agg_state/hll/hll.out} (87%)
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> datatype_p0/agg_state/map/map.out} (78%)
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> dictionary_p0/test_invalid_skip_load.out} (78%)
create mode 100644
regression-test/data/external_table_p0/hive/test_hive_topn_lazy_mat.out
create mode 100644
regression-test/data/external_table_p0/refactor_storage_param/test_outfile_s3_storage.out
create mode 100644
regression-test/data/external_table_p0/test_external_and_internal_describe.out
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> insert_p0/group_commit/test_group_commit_schema_change.out} (78%)
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> inverted_index_p0/test_variant_empty_index_file.out} (78%)
create mode 100644 regression-test/data/javaudf_p0/test_javaudf_override.out
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> nereids_p0/outfile/agg_state/test_outfile_agg_state.out} (77%)
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> nereids_p0/outfile/agg_state_array/test_outfile_agg_array.out} (88%)
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> nereids_p0/outfile/agg_state_bitmap/test_outfile_agg_state_bitmap.out} (85%)
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> nereids_p0/outfile/hll/test_outfile_hll.out} (87%)
copy
regression-test/data/{query_p0/set_operations/set_with_null/set_with_null.out
=> nereids_p0/outfile/quantile_state/test_outfile_quantile_state.out} (82%)
create mode 100644
regression-test/data/nereids_p0/show/test_show_columns_command.out
create mode 100644
regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by.out
copy {fe/fe-core/src/test/resources =>
regression-test/data}/plugin_test/auditdemo.zip (100%)
create mode 100644
regression-test/data/query_p0/join/mark_join/right_semi_mark_join.out
create mode 100644
regression-test/data/query_p0/sql_functions/window_functions/test_nthvalue_function.out
create mode 100644 regression-test/data/query_p0/topn_lazy/topn_lazy.out
create mode 100644
regression-test/data/query_p0/topn_lazy/topn_lazy_on_data_model.out
create mode 100644 regression-test/data/shape_check/others/nlj.out
create mode 100644
regression-test/suites/cloud_p0/multi_cluster/test_mow_agg_delete_bitmap.groovy
create mode 100644
regression-test/suites/compaction/test_base_compaction_after_sc_fail.groovy
create mode 100644
regression-test/suites/compaction/test_mow_compaction_agg_and_remove_pre_delete_bitmap.groovy
create mode 100644
regression-test/suites/compaction/test_mow_compaction_and_read_stale.groovy
create mode 100644
regression-test/suites/compaction/test_mow_compaction_and_read_stale_cloud_docker.groovy
create mode 100644
regression-test/suites/compaction/test_mow_compaction_and_rowset_not_exist.groovy
create mode 100644
regression-test/suites/compaction/test_mow_compaction_and_schema_change.groovy
copy
regression-test/suites/{query_p0/set_operations/set_with_null/set_with_null.groovy
=> datatype_p0/agg_state/array/array.groovy} (59%)
copy
regression-test/suites/{query_p0/set_operations/set_with_null/set_with_null.groovy
=> datatype_p0/agg_state/bitmap/bitmap.groovy} (56%)
copy
regression-test/suites/{query_p0/set_operations/set_with_null/set_with_null.groovy
=> datatype_p0/agg_state/hll/hll.groovy} (57%)
copy
regression-test/suites/{query_p0/set_operations/set_with_null/set_with_null.groovy
=> datatype_p0/agg_state/map/map.groovy} (56%)
create mode 100644
regression-test/suites/dictionary_p0/test_invalid_skip_load.groovy
create mode 100644
regression-test/suites/external_table_p0/hive/test_hive_topn_lazy_mat.groovy
create mode 100644
regression-test/suites/external_table_p0/kerberos/test_iceberg_hadoop_catalog_kerberos.groovy
create mode 100644
regression-test/suites/external_table_p0/refactor_storage_param/backup_restore_azure.groovy
create mode 100644
regression-test/suites/external_table_p0/refactor_storage_param/backup_restore_object_storage.groovy
create mode 100644
regression-test/suites/external_table_p0/refactor_storage_param/hdfs_all_test.groovy
create mode 100644
regression-test/suites/external_table_p0/refactor_storage_param/s3_load.groovy
create mode 100644
regression-test/suites/external_table_p0/refactor_storage_param/test_outfile_s3_storage.groovy
create mode 100644
regression-test/suites/external_table_p0/refactor_storage_param/test_s3_tvf_s3_storage.groovy
create mode 100644
regression-test/suites/external_table_p0/test_external_and_internal_describe.groovy
create mode 100644
regression-test/suites/insert_p0/group_commit/test_group_commit_schema_change.groovy
create mode 100644
regression-test/suites/inverted_index_p0/test_variant_empty_index_file.groovy
create mode 100644
regression-test/suites/javaudf_p0/test_javaudf_override.groovy
create mode 100644
regression-test/suites/nereids_p0/ddl/grant/test_revoke_role_nereids.groovy
create mode 100644
regression-test/suites/nereids_p0/outfile/agg_state/test_outfile_agg_state.groovy
create mode 100644
regression-test/suites/nereids_p0/outfile/agg_state_array/test_outfile_agg_array.groovy
create mode 100644
regression-test/suites/nereids_p0/outfile/agg_state_bitmap/test_outfile_agg_state_bitmap.groovy
create mode 100644
regression-test/suites/nereids_p0/outfile/hll/test_outfile_hll.groovy
create mode 100644
regression-test/suites/nereids_p0/outfile/quantile_state/test_outfile_quantile_state.groovy
create mode 100644
regression-test/suites/nereids_p0/show/test_nereids_show_partitions.groovy
create mode 100644
regression-test/suites/nereids_p0/show/test_show_columns_command.groovy
copy
regression-test/{java-udf-src/src/main/java/org/apache/doris/udf/StringTest.java
=> suites/nereids_p0/show/test_show_open_tables_command.groovy} (61%)
copy
regression-test/suites/nereids_p0/{cte/test_cte_with_duplicate_consumer.groovy
=> show/test_show_storage_vault_command.groovy} (57%)
delete mode 100644
regression-test/suites/nereids_p0/subquery/test_duplicate_name_in_view.groovy
copy
regression-test/{java-udf-src/src/main/java/org/apache/doris/udf/StringTest.java
=> suites/nereids_p0/test_alter_database_properties_command.groovy} (63%)
create mode 100644
regression-test/suites/nereids_p0/test_alter_resource_nereids.groovy
copy
regression-test/{java-udf-src/src/main/java/org/apache/doris/udf/StringTest.java
=> suites/nereids_p0/test_create_workload_policy_command.groovy} (62%)
copy
regression-test/suites/nereids_p0/{cte/test_cte_with_duplicate_consumer.groovy
=> test_install_plugin_command.groovy} (59%)
copy
regression-test/suites/nereids_p0/{cte/test_cte_with_duplicate_consumer.groovy
=> test_lock_tables_command.groovy} (52%)
create mode 100644
regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by.groovy
delete mode 100644
regression-test/suites/query_p0/cte/query_with_dup_column.groovy
create mode 100644
regression-test/suites/query_p0/join/mark_join/right_semi_mark_join.groovy
create mode 100644
regression-test/suites/query_p0/sql_functions/window_functions/test_nthvalue_function.groovy
create mode 100644
regression-test/suites/query_p0/topn_lazy/ddl/customer_create.sql
create mode 100644
regression-test/suites/query_p0/topn_lazy/ddl/dates_create.sql
create mode 100644
regression-test/suites/query_p0/topn_lazy/ddl/lineorder_create.sql
create mode 100644
regression-test/suites/query_p0/topn_lazy/ddl/part_create.sql
create mode 100644 regression-test/suites/query_p0/topn_lazy/ddl/ssb.tables.sql
create mode 100644
regression-test/suites/query_p0/topn_lazy/ddl/supplier_create.sql
create mode 100644 regression-test/suites/query_p0/topn_lazy/load.groovy
create mode 100644 regression-test/suites/query_p0/topn_lazy/topn_lazy.groovy
create mode 100644
regression-test/suites/query_p0/topn_lazy/topn_lazy_on_data_model.groovy
create mode 100644 regression-test/suites/shape_check/others/nlj.groovy
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]