You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by kx...@apache.org on 2023/06/06 15:15:02 UTC

[doris] branch branch-2.0-beta updated (fcc73625a5 -> 5c33dd7a2c)

This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a change to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git


    from fcc73625a5 [build](scripts) modify build-for-release.sh (#20398)
     new eb044d8b75 [Fix](Planner)fix cast date/datev2/datetime to float/double return null. (#20008)
     new 8edb297534 [fix](Nereids) give clean error message when there are subquery in the on clause (#20211)
     new 219332211d [opt](Nereids) perfer use datev2 / datetimev2 in date related functions (#20224)
     new b79fd59c7d [fix](dynamic partition) partition create failed after alter distributed column (#20239)
     new 4ebf48b437 [fix](memory) Fix query memory tracking #20253
     new 5c4c653aac [fix](nereids) select with specified partition name is not work as expected (#20269)
     new 9e44e20be5 [Optimize](function) Optimize locate function by compare across strings (#20290)
     new a01fb11972 [Enchancement](Agg State)  storage function name and result is nullable in agg state type  (#20298)
     new 31bd9ba03c [fix](Nereids): fix filter can't be pushdown unionAll (#20310)
     new d8c6dfbf09 [Feature](Nereids) support update unique table statement (#20313)
     new a368255baa [feature](profile)Add the filtering info of the in filter in profile #20321
     new 1cc61fbf1d [feature](planner)(nereids) support user defined variable (#20334)
     new a92b268810 [improve](nereids)derive analytics node stats (#20340)
     new 2d762f6da9 [fix](nereids) avg size of column stats always be 0 (#20341)
     new 98be895a4c [fix](stats) skip forbid_unknown_col_stats check for invisible column and internal db (#20362)
     new e64ab13b45 [Fix](Nereids) Fix duplicated name in view does not throw exception (#20374)
     new b31747a37d [fix](load) in strict mode, return error for insert if datatype convert fails (#20378)
     new b5027ef1ba [fix](nereids) change defaultConcreteType function's return value for decimal (#20380)
     new 2e0ff4f788 [performance](load) improve memtable sort performance (#20392)
     new b6a1092d87 [fix][refactor](backend-policy)(compute) refactor the hierarchy of external scan node and fix compute node bug #20402
     new 0b16cde515 [fix](sequence) value predicates shouldn't be push down when has sequence column (#20408)
     new 4889512689 [Fix] (tablet) fix tablet queryable set (#20413) (#20414)
     new 8c79108d8e [fix](conf) fix fe host in doris-cluster.conf #20422
     new 7b0de61f0f [fix](workload-group)  fix workload group non-existence error (#20428)
     new 1f273fd856 Fix query hang when using queue (#20434)
     new 34eca4d692 [fix](execution) result_filter_data should be filled by 0 when can_filter_all is true (#20438)
     new e30e986aa5 [fix](Nereids) throw NPE when sql cannot be parsed by all planner (#20440)
     new 5146a71c4f [bug](jdbc) fix trino date/datetime filter (#20443)
     new 6fefe31e48 [fix](load) fix generate delete bitmap in memtable flush (#20446)
     new c3ef8e0a29 [fix](executor)Fix duplicate timer and add open timer #20448
     new f4d0089888 [improvement](column reader) lazy load indices (#20456)
     new 43f6dbab24 [enhancement](profile) add build get child next time (#20460)
     new 704934c733 [fix](regression) fix export file test cases (#20463)
     new 8e2ae30b20 [Fix](WorkloadGroup)Fix query queue nereids bug #20484
     new 193cf917d0 [fix](Nereids) join condition not extract as conjunctions (#20498)
     new 5c33dd7a2c [fix](log) publish version log is printed too frequently (#20507)

The 36 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/CMakeLists.txt                                  |    2 +-
 be/src/agent/task_worker_pool.cpp                  |    4 +-
 be/src/exprs/hybrid_set.h                          |    3 +
 be/src/exprs/runtime_filter.cpp                    |    3 +
 be/src/olap/in_list_predicate.h                    |   15 +-
 be/src/olap/memtable.cpp                           |   68 +-
 be/src/olap/memtable.h                             |   66 +-
 be/src/olap/page_cache.h                           |    2 -
 be/src/olap/rowset/beta_rowset_reader.cpp          |    6 +-
 be/src/olap/rowset/beta_rowset_writer.cpp          |    2 +
 be/src/olap/rowset/beta_rowset_writer.h            |    2 -
 be/src/olap/rowset/rowset_writer.h                 |    6 +-
 be/src/olap/rowset/segment_v2/column_reader.cpp    |   42 +-
 be/src/olap/rowset/segment_v2/column_reader.h      |   34 +-
 be/src/olap/rowset/segment_v2/page_handle.h        |    5 +-
 be/src/olap/rowset/segment_v2/segment_iterator.cpp |    6 +-
 be/src/olap/tablet_meta.cpp                        |    5 +
 be/src/olap/tablet_schema.cpp                      |   19 +-
 be/src/olap/tablet_schema.h                        |    4 +
 be/src/pipeline/pipeline_fragment_context.cpp      |    9 +-
 be/src/pipeline/pipeline_task.cpp                  |    4 +-
 be/src/runtime/plan_fragment_executor.cpp          |   10 +-
 be/src/runtime/runtime_state.h                     |    4 +
 be/src/runtime/types.h                             |    8 +
 be/src/vec/core/block.h                            |    8 +
 be/src/vec/data_types/data_type_agg_state.h        |   42 +-
 be/src/vec/data_types/data_type_factory.cpp        |   28 +-
 be/src/vec/exec/join/vhash_join_node.cpp           |   29 +-
 be/src/vec/exec/join/vhash_join_node.h             |    3 +
 be/src/vec/exec/join/vjoin_node_base.h             |    1 +
 be/src/vec/exec/scan/new_olap_scan_node.cpp        |    5 +-
 be/src/vec/exec/scan/vscan_node.cpp                |    6 +-
 be/src/vec/exec/scan/vscan_node.h                  |    2 +
 be/src/vec/exec/vaggregation_node.cpp              |   13 +-
 be/src/vec/exprs/vectorized_agg_fn.cpp             |   27 +-
 be/src/vec/exprs/vectorized_fn_call.cpp            |    4 +-
 be/src/vec/exprs/vexpr_context.cpp                 |    1 +
 be/src/vec/functions/function_agg_state.h          |   23 +-
 be/src/vec/functions/function_cast.h               |  101 +-
 be/src/vec/functions/function_string.cpp           |   44 +-
 be/test/CMakeLists.txt                             |    1 +
 be/test/olap/memtable_sort_test.cpp                |   83 +
 .../org/apache/doris/catalog/AggStateType.java     |  134 +
 .../java/org/apache/doris/catalog/ScalarType.java  |   62 +-
 .../main/java/org/apache/doris/catalog/Type.java   |    2 +-
 .../antlr4/org/apache/doris/nereids/DorisLexer.g4  |    1 +
 .../antlr4/org/apache/doris/nereids/DorisParser.g4 |   15 +-
 fe/fe-core/src/main/cup/sql_parser.cup             |   18 +-
 .../java/org/apache/doris/analysis/Analyzer.java   |   11 -
 .../org/apache/doris/analysis/BinaryPredicate.java |    4 -
 .../java/org/apache/doris/analysis/ColumnDef.java  |   13 +-
 .../main/java/org/apache/doris/analysis/Expr.java  |   79 +-
 .../apache/doris/analysis/ExpressionFunctions.java |    6 +-
 .../doris/analysis/HashDistributionDesc.java       |    4 +-
 .../java/org/apache/doris/analysis/SetType.java    |    3 +-
 .../{SetType.java => SetUserDefinedVar.java}       |   49 +-
 .../java/org/apache/doris/analysis/SetVar.java     |   37 +-
 .../apache/doris/analysis/ShowColumnStatsStmt.java |    2 +-
 .../org/apache/doris/analysis/StringLiteral.java   |    9 +-
 .../{SysVariableDesc.java => VariableExpr.java}    |   66 +-
 .../main/java/org/apache/doris/catalog/Column.java |   28 +-
 .../main/java/org/apache/doris/catalog/Env.java    |   25 +-
 .../java/org/apache/doris/catalog/Function.java    |   41 +-
 .../apache/doris/catalog/HashDistributionInfo.java |   14 +-
 .../main/java/org/apache/doris/catalog/Tablet.java |    9 +
 .../doris/catalog/external/HMSExternalTable.java   |    2 +-
 .../org/apache/doris/clone/TabletScheduler.java    |    4 +-
 .../apache/doris/datasource/InternalCatalog.java   |   13 +-
 .../doris/nereids/parser/LogicalPlanBuilder.java   |   77 +-
 .../nereids/rules/analysis/BindExpression.java     |   20 +
 .../nereids/rules/analysis/CheckAnalysis.java      |   16 +-
 .../rules/expression/ExpressionRewrite.java        |   14 +-
 .../rewrite/logical/PruneOlapScanPartition.java    |   15 +-
 .../logical/PushdownFilterThroughSetOperation.java |   21 +-
 .../doris/nereids/stats/ExpressionEstimation.java  |    4 +-
 .../doris/nereids/stats/FilterEstimation.java      |   11 +-
 .../apache/doris/nereids/stats/JoinEstimation.java |    4 +-
 .../doris/nereids/stats/StatsCalculator.java       |   55 +-
 .../expressions/functions/SearchSignature.java     |   30 +-
 .../expressions/functions/scalar/ConvertTz.java    |    6 +-
 .../trees/expressions/functions/scalar/Date.java   |    4 +-
 .../expressions/functions/scalar/DateDiff.java     |    8 +-
 .../expressions/functions/scalar/DateFormat.java   |    6 +-
 .../expressions/functions/scalar/DateTrunc.java    |    4 +-
 .../expressions/functions/scalar/DayCeil.java      |   18 +-
 .../expressions/functions/scalar/DayFloor.java     |   18 +-
 .../expressions/functions/scalar/DayName.java      |    4 +-
 .../expressions/functions/scalar/DayOfMonth.java   |    6 +-
 .../expressions/functions/scalar/DayOfWeek.java    |    4 +-
 .../expressions/functions/scalar/DayOfYear.java    |    4 +-
 .../expressions/functions/scalar/DaysAdd.java      |    4 +-
 .../expressions/functions/scalar/DaysDiff.java     |   10 +-
 .../expressions/functions/scalar/DaysSub.java      |    4 +-
 .../trees/expressions/functions/scalar/Hour.java   |    4 +-
 .../expressions/functions/scalar/HourCeil.java     |   13 +-
 .../expressions/functions/scalar/HourFloor.java    |   13 +-
 .../expressions/functions/scalar/HoursAdd.java     |    7 +-
 .../expressions/functions/scalar/HoursDiff.java    |    4 +-
 .../expressions/functions/scalar/HoursSub.java     |    7 +-
 .../trees/expressions/functions/scalar/Minute.java |    4 +-
 .../expressions/functions/scalar/MinuteCeil.java   |   14 +-
 .../expressions/functions/scalar/MinuteFloor.java  |   14 +-
 .../expressions/functions/scalar/MinutesAdd.java   |    9 +-
 .../expressions/functions/scalar/MinutesDiff.java  |    4 +-
 .../expressions/functions/scalar/MinutesSub.java   |    9 +-
 .../trees/expressions/functions/scalar/Month.java  |    6 +-
 .../expressions/functions/scalar/MonthCeil.java    |   20 +-
 .../expressions/functions/scalar/MonthFloor.java   |   20 +-
 .../expressions/functions/scalar/MonthName.java    |    6 +-
 .../expressions/functions/scalar/MonthsAdd.java    |    9 +-
 .../expressions/functions/scalar/MonthsDiff.java   |   10 +-
 .../expressions/functions/scalar/MonthsSub.java    |    9 +-
 .../trees/expressions/functions/scalar/Nvl.java    |    2 +-
 .../trees/expressions/functions/scalar/Second.java |    6 +-
 .../expressions/functions/scalar/SecondCeil.java   |    2 +-
 .../expressions/functions/scalar/SecondFloor.java  |   15 +-
 .../expressions/functions/scalar/SecondsAdd.java   |    9 +-
 .../expressions/functions/scalar/SecondsDiff.java  |    8 +-
 .../expressions/functions/scalar/SecondsSub.java   |    9 +-
 .../expressions/functions/scalar/TimeDiff.java     |    4 +-
 .../trees/expressions/functions/scalar/ToDate.java |    6 +-
 .../trees/expressions/functions/scalar/ToDays.java |    6 +-
 .../expressions/functions/scalar/ToMonday.java     |    8 +-
 .../functions/scalar/UnixTimestamp.java            |    6 +-
 .../trees/expressions/functions/scalar/Week.java   |   10 +-
 .../expressions/functions/scalar/WeekCeil.java     |   20 +-
 .../expressions/functions/scalar/WeekFloor.java    |   20 +-
 .../expressions/functions/scalar/WeekOfYear.java   |    6 +-
 .../expressions/functions/scalar/Weekday.java      |    6 +-
 .../expressions/functions/scalar/WeeksAdd.java     |    5 +-
 .../expressions/functions/scalar/WeeksDiff.java    |   10 +-
 .../expressions/functions/scalar/WeeksSub.java     |    5 +-
 .../trees/expressions/functions/scalar/Year.java   |    6 +-
 .../expressions/functions/scalar/YearCeil.java     |   20 +-
 .../expressions/functions/scalar/YearFloor.java    |   20 +-
 .../expressions/functions/scalar/YearWeek.java     |   10 +-
 .../expressions/functions/scalar/YearsAdd.java     |    9 +-
 .../expressions/functions/scalar/YearsDiff.java    |   10 +-
 .../expressions/functions/scalar/YearsSub.java     |    9 +-
 .../nereids/trees/expressions/literal/Literal.java |    7 +
 .../doris/nereids/trees/plans/Explainable.java     |   27 +
 .../trees/plans/commands/ExplainCommand.java       |    9 +-
 .../plans/commands/InsertIntoTableCommand.java     |   19 +-
 .../trees/plans/commands/UpdateCommand.java        |  156 +
 .../trees/plans/logical/AbstractLogicalPlan.java   |    9 +-
 .../nereids/trees/plans/logical/LogicalExcept.java |    5 -
 .../trees/plans/logical/LogicalIntersect.java      |    5 -
 .../trees/plans/logical/LogicalSetOperation.java   |    3 -
 .../nereids/trees/plans/logical/LogicalUnion.java  |    5 -
 .../trees/plans/physical/AbstractPhysicalPlan.java |    9 +-
 .../trees/plans/visitor/CommandVisitor.java        |   46 +
 .../nereids/trees/plans/visitor/PlanVisitor.java   |   18 +-
 .../apache/doris/nereids/types/AggStateType.java   |   14 +-
 .../org/apache/doris/nereids/types/DataType.java   |    5 +-
 .../apache/doris/nereids/types/DecimalV2Type.java  |    3 +-
 .../apache/doris/nereids/types/DecimalV3Type.java  |    3 +-
 .../org/apache/doris/persist/gson/GsonUtils.java   |    4 +-
 .../planner/BackendPartitionedSchemaScanNode.java  |   18 +-
 .../org/apache/doris/planner/DataGenScanNode.java  |   37 +-
 .../java/org/apache/doris/planner/EsScanNode.java  |   48 +-
 .../org/apache/doris/planner/FileLoadScanNode.java |   29 +-
 .../org/apache/doris/planner/JdbcScanNode.java     |   49 +-
 .../org/apache/doris/planner/MysqlScanNode.java    |   27 +-
 .../org/apache/doris/planner/OdbcScanNode.java     |   51 +-
 .../org/apache/doris/planner/OlapScanNode.java     |   52 +-
 .../java/org/apache/doris/planner/ScanNode.java    |   25 +-
 .../org/apache/doris/planner/SchemaScanNode.java   |   15 +-
 .../doris/planner/TestExternalTableScanNode.java   |   31 +-
 .../doris/planner/external/ExternalScanNode.java   |   30 +-
 .../planner/external/FederationBackendPolicy.java  |    4 +
 .../doris/planner/external/FileQueryScanNode.java  |   18 +-
 .../doris/planner/external/FileScanNode.java       |   12 +-
 .../doris/planner/external/MetadataScanNode.java   |   32 +-
 .../apache/doris/planner/external/TVFScanNode.java |    9 -
 .../doris/planner/external/hudi/HudiScanNode.java  |    2 +-
 .../planner/external/iceberg/IcebergScanNode.java  |   20 +-
 .../java/org/apache/doris/qe/ConnectProcessor.java |    2 +-
 .../main/java/org/apache/doris/qe/Coordinator.java |   53 +-
 .../java/org/apache/doris/qe/SessionVariable.java  |    2 +
 .../main/java/org/apache/doris/qe/SetExecutor.java |    3 +
 .../java/org/apache/doris/qe/StmtExecutor.java     |   77 +-
 .../main/java/org/apache/doris/qe/VariableMgr.java |   80 +-
 .../resource/workloadgroup/WorkloadGroupMgr.java   |   19 +-
 .../apache/doris/rewrite/FoldConstantsRule.java    |   14 +-
 .../apache/doris/statistics/ColumnStatistic.java   |   23 +-
 .../doris/statistics/ColumnStatisticBuilder.java   |   14 +-
 .../doris/statistics/StatisticConstants.java       |   14 +
 .../apache/doris/statistics/StatisticsCache.java   |    2 +-
 .../doris/statistics/StatisticsRepository.java     |    2 +-
 .../java/org/apache/doris/analysis/ExprTest.java   |   10 +
 .../java/org/apache/doris/analysis/SetVarTest.java |    2 +-
 ...VariableDescTest.java => VariableExprTest.java} |    6 +-
 .../doris/nereids/stats/FilterEstimationTest.java  |    6 +-
 .../trees/plans/ExplainInsertCommandTest.java      |    6 -
 .../nereids/trees/plans/UpdateCommandTest.java     |  126 +
 .../doris/nereids/util/HyperGraphBuilder.java      |    2 +-
 .../java/org/apache/doris/qe/CoordinatorTest.java  |    8 +-
 .../java/org/apache/doris/qe/VariableMgrTest.java  |    4 +-
 .../org/apache/doris/statistics/CacheTest.java     |    6 +-
 .../doris/statistics/StatsDeriveResultTest.java    |    2 +-
 gensrc/proto/data.proto                            |    2 +
 gensrc/proto/olap_file.proto                       |    1 +
 gensrc/thrift/Descriptors.thrift                   |    1 +
 gensrc/thrift/PaloInternalService.thrift           |    2 +
 gensrc/thrift/Types.thrift                         |    4 +-
 .../data/correctness_p0/test_function_conjunct.out |    5 +
 regression-test/data/cte_reuse/q11.out             |   65 -
 regression-test/data/cte_reuse/q14.out             |  161 -
 regression-test/data/cte_reuse/q23.out             |   98 -
 regression-test/data/cte_reuse/q24.out             |   54 -
 regression-test/data/cte_reuse/q31.out             |   74 -
 regression-test/data/cte_reuse/q4.out              |   97 -
 regression-test/data/cte_reuse/q47.out             |   49 -
 regression-test/data/cte_reuse/q57.out             |   49 -
 regression-test/data/cte_reuse/q59.out             |   45 -
 regression-test/data/cte_reuse/q64.out             |  115 -
 regression-test/data/cte_reuse/q74.out             |   64 -
 .../unique/test_unique_value_predicate.out         |   45 +
 regression-test/data/es_p0/test_es_query.out       |   46 +-
 .../data/es_p0/test_es_query_no_http_url.out       |   10 +-
 regression-test/data/insert_p0/insert_invalid.out  |   11 +
 .../conditional_functions/test_coalesce.out        |   24 +-
 .../conditional_functions/test_nullif.out          |    8 +-
 .../datetime_functions/test_date_function.out      |    6 +-
 regression-test/data/nereids_p0/test_user_var.out  |   16 +
 .../data/nereids_p0/update/update_unique_table.out |    6 +-
 .../data/nereids_syntax_p0/select_partition.out    |    3 +
 .../nereids_tpcds_shape_sf100_p0/shape/query11.out |   30 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query4.out  |   30 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query44.out |   62 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query70.out |   58 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query74.out |   29 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query75.out |  104 +-
 .../data/query_p0/set/test_user_var.out            |   16 +
 .../test_conditional_function.out                  |    3 +-
 .../datetime_functions/test_date_function.out      |    6 +-
 .../sql/dateTimeOperatorsAccessible.out            |    2 +-
 .../string_functions/test_string_function.out      |   24 +-
 .../org/apache/doris/regression/suite/Suite.groovy |   24 +
 regression-test/pipeline/p0/conf/fe.conf           |    2 +
 regression-test/pipeline/p1/conf/fe.conf           |    2 +
 .../correctness_p0/test_function_conjunct.groovy   |   45 +
 .../suites/cte_reuse/ddl/call_center.sql           |   38 -
 .../suites/cte_reuse/ddl/catalog_page.sql          |   17 -
 .../suites/cte_reuse/ddl/catalog_returns.sql       |   34 -
 .../suites/cte_reuse/ddl/catalog_sales.sql         |   42 -
 regression-test/suites/cte_reuse/ddl/customer.sql  |   26 -
 .../suites/cte_reuse/ddl/customer_address.sql      |   21 -
 .../suites/cte_reuse/ddl/customer_demographics.sql |   16 -
 regression-test/suites/cte_reuse/ddl/date_dim.sql  |   35 -
 .../cte_reuse/ddl/household_demographics.sql       |   13 -
 .../suites/cte_reuse/ddl/income_band.sql           |   11 -
 regression-test/suites/cte_reuse/ddl/inventory.sql |   12 -
 regression-test/suites/cte_reuse/ddl/item.sql      |   29 -
 regression-test/suites/cte_reuse/ddl/promotion.sql |   27 -
 regression-test/suites/cte_reuse/ddl/reason.sql    |   11 -
 regression-test/suites/cte_reuse/ddl/ship_mode.sql |   14 -
 regression-test/suites/cte_reuse/ddl/store.sql     |   36 -
 .../suites/cte_reuse/ddl/store_returns.sql         |   28 -
 .../suites/cte_reuse/ddl/store_sales.sql           |   32 -
 regression-test/suites/cte_reuse/ddl/time_dim.sql  |   17 -
 regression-test/suites/cte_reuse/ddl/warehouse.sql |   22 -
 regression-test/suites/cte_reuse/ddl/web_page.sql  |   21 -
 .../suites/cte_reuse/ddl/web_returns.sql           |   31 -
 regression-test/suites/cte_reuse/ddl/web_sales.sql |   42 -
 regression-test/suites/cte_reuse/ddl/web_site.sql  |   36 -
 regression-test/suites/cte_reuse/load.groovy       | 4228 --------------------
 regression-test/suites/cte_reuse/q11.groovy        |  108 -
 regression-test/suites/cte_reuse/q14.groovy        |  207 -
 regression-test/suites/cte_reuse/q23.groovy        |  129 -
 regression-test/suites/cte_reuse/q24.groovy        |   88 -
 regression-test/suites/cte_reuse/q31.groovy        |  104 -
 regression-test/suites/cte_reuse/q4.groovy         |  134 -
 regression-test/suites/cte_reuse/q47.groovy        |  103 -
 regression-test/suites/cte_reuse/q57.groovy        |   99 -
 regression-test/suites/cte_reuse/q59.groovy        |  115 -
 regression-test/suites/cte_reuse/q64.groovy        |  149 -
 regression-test/suites/cte_reuse/q74.groovy        |  100 -
 .../storage/test_dup_tab_date_nullable.groovy      |    2 +-
 .../unique/test_unique_value_predicate.groovy      |  111 +
 regression-test/suites/es_p0/test_es_query.groovy  |    4 -
 .../suites/es_p0/test_es_query_nereids.groovy      |    5 -
 .../suites/es_p0/test_es_query_no_http_url.groovy  |    4 -
 .../suites/export/test_array_export.groovy         |   48 +-
 .../suites/export/test_map_export.groovy           |   15 +-
 .../suites/export/test_struct_export.groovy        |   15 +-
 .../suites/export_p2/test_export_with_s3.groovy    |    2 +-
 .../hive/test_external_yandex_nereids.groovy       |    1 +
 .../suites/insert_p0/insert_invalid.groovy         |  215 +
 .../datetime_functions/test_date_function.groovy   |    6 +-
 .../subquery/test_duplicate_name_in_view.groovy    |   69 +
 .../suites/nereids_p0/test_user_var.groovy         |   32 +
 .../suites/nereids_p0/update/load.groovy           |   81 +
 .../nereids_p0/update/update_unique_table.groovy   |   39 +
 .../nereids_syntax_p0/select_partition.groovy      |   28 +
 .../suites/nereids_syntax_p0/type_cast.groovy      |    2 +-
 .../test_dynamic_partition_with_alter.groovy       |   56 +
 .../test_dynamic_partition_with_rename.groovy      |   60 +
 .../suites/query_p0/cast/test_cast.groovy          |   54 +-
 .../suites/query_p0/set/test_user_var.groovy       |   30 +
 .../datetime_functions/test_date_function.groovy   |    6 +-
 .../string_functions/test_string_function.groovy   |   16 +-
 thirdparty/build-thirdparty.sh                     |   10 +-
 thirdparty/vars.sh                                 |    8 +-
 tools/tpch-tools/conf/doris-cluster.conf           |    2 +-
 305 files changed, 3597 insertions(+), 8533 deletions(-)
 create mode 100644 be/test/olap/memtable_sort_test.cpp
 create mode 100644 fe/fe-common/src/main/java/org/apache/doris/catalog/AggStateType.java
 copy fe/fe-core/src/main/java/org/apache/doris/analysis/{SetType.java => SetUserDefinedVar.java} (53%)
 rename fe/fe-core/src/main/java/org/apache/doris/analysis/{SysVariableDesc.java => VariableExpr.java} (74%)
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Explainable.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java
 rename fe/fe-core/src/test/java/org/apache/doris/analysis/{SysVariableDescTest.java => VariableExprTest.java} (92%)
 create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/UpdateCommandTest.java
 create mode 100644 regression-test/data/correctness_p0/test_function_conjunct.out
 delete mode 100644 regression-test/data/cte_reuse/q11.out
 delete mode 100644 regression-test/data/cte_reuse/q14.out
 delete mode 100644 regression-test/data/cte_reuse/q23.out
 delete mode 100644 regression-test/data/cte_reuse/q24.out
 delete mode 100644 regression-test/data/cte_reuse/q31.out
 delete mode 100644 regression-test/data/cte_reuse/q4.out
 delete mode 100644 regression-test/data/cte_reuse/q47.out
 delete mode 100644 regression-test/data/cte_reuse/q57.out
 delete mode 100644 regression-test/data/cte_reuse/q59.out
 delete mode 100644 regression-test/data/cte_reuse/q64.out
 delete mode 100644 regression-test/data/cte_reuse/q74.out
 create mode 100644 regression-test/data/data_model_p0/unique/test_unique_value_predicate.out
 create mode 100644 regression-test/data/insert_p0/insert_invalid.out
 create mode 100644 regression-test/data/nereids_p0/test_user_var.out
 create mode 100644 regression-test/data/query_p0/set/test_user_var.out
 create mode 100644 regression-test/suites/correctness_p0/test_function_conjunct.groovy
 delete mode 100644 regression-test/suites/cte_reuse/ddl/call_center.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/catalog_page.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/catalog_returns.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/catalog_sales.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/customer.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/customer_address.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/customer_demographics.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/date_dim.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/household_demographics.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/income_band.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/inventory.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/item.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/promotion.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/reason.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/ship_mode.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/store.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/store_returns.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/store_sales.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/time_dim.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/warehouse.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/web_page.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/web_returns.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/web_sales.sql
 delete mode 100644 regression-test/suites/cte_reuse/ddl/web_site.sql
 delete mode 100644 regression-test/suites/cte_reuse/load.groovy
 delete mode 100644 regression-test/suites/cte_reuse/q11.groovy
 delete mode 100644 regression-test/suites/cte_reuse/q14.groovy
 delete mode 100644 regression-test/suites/cte_reuse/q23.groovy
 delete mode 100644 regression-test/suites/cte_reuse/q24.groovy
 delete mode 100644 regression-test/suites/cte_reuse/q31.groovy
 delete mode 100644 regression-test/suites/cte_reuse/q4.groovy
 delete mode 100644 regression-test/suites/cte_reuse/q47.groovy
 delete mode 100644 regression-test/suites/cte_reuse/q57.groovy
 delete mode 100644 regression-test/suites/cte_reuse/q59.groovy
 delete mode 100644 regression-test/suites/cte_reuse/q64.groovy
 delete mode 100644 regression-test/suites/cte_reuse/q74.groovy
 create mode 100644 regression-test/suites/data_model_p0/unique/test_unique_value_predicate.groovy
 create mode 100644 regression-test/suites/insert_p0/insert_invalid.groovy
 create mode 100644 regression-test/suites/nereids_p0/subquery/test_duplicate_name_in_view.groovy
 create mode 100644 regression-test/suites/nereids_p0/test_user_var.groovy
 create mode 100644 regression-test/suites/nereids_p0/update/load.groovy
 create mode 100644 regression-test/suites/nereids_p0/update/update_unique_table.groovy
 create mode 100644 regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_alter.groovy
 create mode 100644 regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_rename.groovy
 copy fe/fe-core/src/main/java/org/apache/doris/analysis/SetType.java => regression-test/suites/query_p0/cast/test_cast.groovy (50%)
 create mode 100644 regression-test/suites/query_p0/set/test_user_var.groovy


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 29/36: [fix](load) fix generate delete bitmap in memtable flush (#20446)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 6fefe31e48ba0538ff57961f9831e66916f979e7
Author: Kaijie Chen <ck...@apache.org>
AuthorDate: Tue Jun 6 09:48:30 2023 +0800

    [fix](load) fix generate delete bitmap in memtable flush (#20446)
    
    1. Generate delete bitmap for one segment at a time.
    2. Generate delete bitmap before segment compaction.
    Fix #20445
---
 be/src/olap/memtable.cpp                  | 20 +++++++-------------
 be/src/olap/memtable.h                    |  3 +--
 be/src/olap/rowset/beta_rowset_writer.cpp |  2 ++
 be/src/olap/rowset/beta_rowset_writer.h   |  2 --
 be/src/olap/rowset/rowset_writer.h        |  6 ++++--
 5 files changed, 14 insertions(+), 19 deletions(-)

diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp
index d61a25c8be..9a6e5687d2 100644
--- a/be/src/olap/memtable.cpp
+++ b/be/src/olap/memtable.cpp
@@ -443,21 +443,16 @@ bool MemTable::need_agg() const {
     return false;
 }
 
-Status MemTable::_generate_delete_bitmap(int64_t atomic_num_segments_before_flush,
-                                         int64_t atomic_num_segments_after_flush) {
+Status MemTable::_generate_delete_bitmap(int32_t segment_id) {
     SCOPED_RAW_TIMER(&_stat.delete_bitmap_ns);
     // generate delete bitmap, build a tmp rowset and load recent segment
     if (!_tablet->enable_unique_key_merge_on_write()) {
         return Status::OK();
     }
-    if (atomic_num_segments_before_flush >= atomic_num_segments_after_flush) {
-        return Status::OK();
-    }
     auto rowset = _rowset_writer->build_tmp();
     auto beta_rowset = reinterpret_cast<BetaRowset*>(rowset.get());
     std::vector<segment_v2::SegmentSharedPtr> segments;
-    RETURN_IF_ERROR(beta_rowset->load_segments(atomic_num_segments_before_flush,
-                                               atomic_num_segments_after_flush, &segments));
+    RETURN_IF_ERROR(beta_rowset->load_segments(segment_id, segment_id + 1, &segments));
     std::shared_lock meta_rlock(_tablet->get_header_lock());
     // tablet is under alter process. The delete bitmap will be calculated after conversion.
     if (_tablet->tablet_state() == TABLET_NOTREADY &&
@@ -477,15 +472,9 @@ Status MemTable::flush() {
     // The id of new segment is set by the _num_segment of beta_rowset_writer,
     // and new segment ids is between [atomic_num_segments_before_flush, atomic_num_segments_after_flush),
     // and use the ids to load segment data file for calc delete bitmap.
-    int64_t atomic_num_segments_before_flush = _rowset_writer->get_atomic_num_segment();
     int64_t duration_ns;
     SCOPED_RAW_TIMER(&duration_ns);
     SKIP_MEMORY_CHECK(RETURN_IF_ERROR(_do_flush()));
-    int64_t atomic_num_segments_after_flush = _rowset_writer->get_atomic_num_segment();
-    if (!_tablet_schema->is_partial_update()) {
-        RETURN_IF_ERROR(_generate_delete_bitmap(atomic_num_segments_before_flush,
-                                                atomic_num_segments_after_flush));
-    }
     _delta_writer_callback(_stat);
     DorisMetrics::instance()->memtable_flush_total->increment(1);
     DorisMetrics::instance()->memtable_flush_duration_us->increment(duration_ns / 1000);
@@ -515,6 +504,11 @@ Status MemTable::_do_flush() {
         // Unfold variant column
         RETURN_IF_ERROR(unfold_variant_column(block, &ctx));
     }
+    if (!_tablet_schema->is_partial_update()) {
+        ctx.generate_delete_bitmap = [this](size_t segment_id) {
+            return _generate_delete_bitmap(segment_id);
+        };
+    }
     ctx.segment_id = _segment_id;
     SCOPED_RAW_TIMER(&_stat.segment_writer_ns);
     RETURN_IF_ERROR(_rowset_writer->flush_single_memtable(&block, &_flush_size, &ctx));
diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h
index e7844f5d93..912f2f4211 100644
--- a/be/src/olap/memtable.h
+++ b/be/src/olap/memtable.h
@@ -216,8 +216,7 @@ private:
     void _aggregate_two_row_in_block(vectorized::MutableBlock& mutable_block, RowInBlock* new_row,
                                      RowInBlock* row_in_skiplist);
 
-    Status _generate_delete_bitmap(int64_t atomic_num_segments_before_flush,
-                                   int64_t atomic_num_segments_after_flush);
+    Status _generate_delete_bitmap(int32_t segment_id);
 
     // serialize block to row store format and append serialized data into row store column
     // in block
diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp
index e75075d27f..c729a65077 100644
--- a/be/src/olap/rowset/beta_rowset_writer.cpp
+++ b/be/src/olap/rowset/beta_rowset_writer.cpp
@@ -491,8 +491,10 @@ Status BetaRowsetWriter::flush_single_memtable(const vectorized::Block* block, i
 
     std::unique_ptr<segment_v2::SegmentWriter> writer;
     RETURN_IF_ERROR(_create_segment_writer(&writer, ctx));
+    int32_t segment_id = writer->get_segment_id();
     RETURN_IF_ERROR(_add_block(block, &writer));
     RETURN_IF_ERROR(_flush_segment_writer(&writer, flush_size));
+    RETURN_IF_ERROR(ctx->generate_delete_bitmap(segment_id));
     RETURN_IF_ERROR(_segcompaction_if_necessary());
     return Status::OK();
 }
diff --git a/be/src/olap/rowset/beta_rowset_writer.h b/be/src/olap/rowset/beta_rowset_writer.h
index 88219e5941..b646f2a681 100644
--- a/be/src/olap/rowset/beta_rowset_writer.h
+++ b/be/src/olap/rowset/beta_rowset_writer.h
@@ -106,8 +106,6 @@ public:
         return Status::OK();
     }
 
-    int32_t get_atomic_num_segment() const override { return _num_segment.load(); }
-
     int32_t allocate_segment_id() override { return _next_segment_id.fetch_add(1); };
 
     // Maybe modified by local schema change
diff --git a/be/src/olap/rowset/rowset_writer.h b/be/src/olap/rowset/rowset_writer.h
index 0ae5fd46c5..3ccd481ec7 100644
--- a/be/src/olap/rowset/rowset_writer.h
+++ b/be/src/olap/rowset/rowset_writer.h
@@ -20,6 +20,7 @@
 #include <gen_cpp/olap_file.pb.h>
 #include <gen_cpp/types.pb.h>
 
+#include <functional>
 #include <optional>
 
 #include "common/factory_creator.h"
@@ -40,6 +41,9 @@ struct FlushContext {
     TabletSchemaSPtr flush_schema = nullptr;
     const vectorized::Block* block = nullptr;
     std::optional<int32_t> segment_id = std::nullopt;
+    std::function<Status(int32_t)> generate_delete_bitmap = [](int32_t segment_id) {
+        return Status::OK();
+    };
 };
 
 class RowsetWriter {
@@ -100,8 +104,6 @@ public:
         return Status::NotSupported("to be implemented");
     }
 
-    virtual int32_t get_atomic_num_segment() const = 0;
-
     virtual int32_t allocate_segment_id() = 0;
 
     virtual bool is_doing_segcompaction() const = 0;


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 01/36: [Fix](Planner)fix cast date/datev2/datetime to float/double return null. (#20008)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit eb044d8b752e830c5b823f8b46a08736321017ae
Author: mch_ucchi <41...@users.noreply.github.com>
AuthorDate: Mon Jun 5 19:06:50 2023 +0800

    [Fix](Planner)fix cast date/datev2/datetime to float/double return null. (#20008)
---
 fe/fe-core/src/main/cup/sql_parser.cup             |  6 ++--
 .../java/org/apache/doris/analysis/ExprTest.java   | 10 +++++++
 .../sql/dateTimeOperatorsAccessible.out            |  2 +-
 .../suites/query_p0/cast/test_cast.groovy          | 34 ++++++++++++++++++++++
 4 files changed, 48 insertions(+), 4 deletions(-)

diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index 6542ad40e4..3a85b99dd4 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -6421,11 +6421,11 @@ non_pred_expr ::=
   | function_call_expr:e
   {: RESULT = e; :}
   | KW_DATE STRING_LITERAL:l
-  {: RESULT = new StringLiteral(l); :}
+  {: RESULT = new CastExpr(Type.DATE, new StringLiteral(l)); :}
   | KW_DATEV2 STRING_LITERAL:l
-  {: RESULT = new StringLiteral(l); :}
+  {: RESULT = new CastExpr(Type.DATEV2, new StringLiteral(l)); :}
   | KW_TIMESTAMP STRING_LITERAL:l
-  {: RESULT = new StringLiteral(l); :}
+  {: RESULT = new CastExpr(Type.DATETIME, new StringLiteral(l)); :}
   | KW_EXTRACT LPAREN function_name:fn_name KW_FROM func_arg_list:exprs RPAREN
   {: RESULT = new FunctionCallExpr(fn_name, exprs); :}
   //| function_name:fn_name LPAREN RPAREN
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ExprTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ExprTest.java
index 475d9dc8fb..5262a29034 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ExprTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ExprTest.java
@@ -127,6 +127,16 @@ public class ExprTest {
         Assert.assertEquals(0, castLiteral.getMinute());
         Assert.assertEquals(0, castLiteral.getSecond());
 
+        DateLiteral srcDate = new DateLiteral("2020-01-01", Type.DATE);
+        DateLiteral srcDateV2 = new DateLiteral("2020-01-01", Type.DATEV2);
+        DateLiteral srcDateTime = new DateLiteral("2020-01-01 12:34:45", Type.DATETIME);
+        Assert.assertEquals(20200101L, ((FloatLiteral) (new CastExpr(Type.FLOAT, srcDate)
+                .castTo(Type.FLOAT)).getResultValue(false)).getLongValue());
+        Assert.assertEquals(20200101L, ((FloatLiteral) new CastExpr(Type.FLOAT, srcDateV2)
+                .castTo(Type.FLOAT).getResultValue(false)).getLongValue());
+        Assert.assertEquals(20200101123445L, ((FloatLiteral) new CastExpr(Type.FLOAT, srcDateTime)
+                .castTo(Type.FLOAT).getResultValue(false)).getLongValue());
+
         // float
         FloatLiteral floatLiteral = new FloatLiteral(0.1, Type.FLOAT);
         Assert.assertEquals(floatLiteral.getType(), Type.FLOAT);
diff --git a/regression-test/data/query_p0/sql_functions/horology_functions/sql/dateTimeOperatorsAccessible.out b/regression-test/data/query_p0/sql_functions/horology_functions/sql/dateTimeOperatorsAccessible.out
index bfac26aba9..a6071694b7 100644
--- a/regression-test/data/query_p0/sql_functions/horology_functions/sql/dateTimeOperatorsAccessible.out
+++ b/regression-test/data/query_p0/sql_functions/horology_functions/sql/dateTimeOperatorsAccessible.out
@@ -1,4 +1,4 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
 -- !dateTimeOperatorsAccessible --
-2012-08-10T00:00	2012-08-09T06:00	2012-11-30T01:00	2012-08-06T00:00	2012-08-06T20:00	2012-09-30T01:00
+2012-08-10	2012-08-09T06:00	2012-11-30T01:00	2012-08-06	2012-08-06T20:00	2012-09-30T01:00
 
diff --git a/regression-test/suites/query_p0/cast/test_cast.groovy b/regression-test/suites/query_p0/cast/test_cast.groovy
new file mode 100644
index 0000000000..bfe4a87989
--- /dev/null
+++ b/regression-test/suites/query_p0/cast/test_cast.groovy
@@ -0,0 +1,34 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite('test_cast') {
+    def date = "date '2020-01-01'"
+    def datev2 = "datev2 '2020-01-01'"
+    def datetime = "timestamp '2020-01-01 12:34:45'"
+    test {
+        sql "select cast(${date} as int), cast(${date} as bigint), cast(${date} as float), cast(${date} as double)"
+        result([[20200101, 20200101l, ((float) 20200101), ((double) 20200101)]])
+    }
+    test {
+        sql "select cast(${datev2} as int), cast(${datev2} as bigint), cast(${datev2} as float), cast(${datev2} as double)"
+        result([[20200101, 20200101l, ((float) 20200101), ((double) 20200101)]])
+    }
+    test {
+        sql "select cast(${datetime} as int), cast(${datetime} as bigint), cast(${datetime} as float), cast(${datetime} as double)"
+        result([[869930357, 20200101123445l, ((float) 20200101123445l), ((double) 20200101123445l)]])
+    }
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 33/36: [fix](regression) fix export file test cases (#20463)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 704934c733564db63bf2c86fb867d9c333942a8c
Author: zgxme <me...@zgx.io>
AuthorDate: Tue Jun 6 20:07:31 2023 +0800

    [fix](regression) fix export file test cases (#20463)
---
 .../org/apache/doris/regression/suite/Suite.groovy | 24 +++++++++++
 .../suites/export/test_array_export.groovy         | 48 +++++++++++++---------
 .../suites/export/test_map_export.groovy           | 15 ++++++-
 .../suites/export/test_struct_export.groovy        | 15 ++++++-
 .../suites/export_p2/test_export_with_s3.groovy    |  2 +-
 5 files changed, 80 insertions(+), 24 deletions(-)

diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy
index 934a18b8dd..7106965929 100644
--- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy
+++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy
@@ -381,6 +381,30 @@ class Suite implements GroovyInterceptable {
         String s3Url = "http://${s3BucketName}.${s3Endpoint}"
         return s3Url
     }
+    
+    void scpFiles(String username, String host, String files, String filePath, boolean fromDst=true) {
+        String cmd = "scp -r ${username}@${host}:${files} ${filePath}"
+        if (!fromDst) {
+            cmd = "scp -r ${files} ${username}@${host}:${filePath}"
+        }
+        logger.info("Execute: ${cmd}".toString())
+        Process process = cmd.execute()
+        def code = process.waitFor()
+        Assert.assertEquals(0, code)
+    }
+    
+    void sshExec(String username, String host, String cmd) {
+        String command = "ssh ${username}@${host} '${cmd}'"
+        def cmds = ["/bin/bash", "-c", command]
+        logger.info("Execute: ${cmds}".toString())
+        Process p = cmds.execute()
+        def errMsg = new StringBuilder()
+        def msg = new StringBuilder()
+        p.waitForProcessOutput(msg, errMsg)
+        assert errMsg.length() == 0: "error occurred!" + errMsg
+        assert p.exitValue() == 0
+    }
+    
 
     void getBackendIpHttpPort(Map<String, String> backendId_to_backendIP, Map<String, String> backendId_to_backendHttpPort) {
         List<List<Object>> backends = sql("show backends");
diff --git a/regression-test/suites/export/test_array_export.groovy b/regression-test/suites/export/test_array_export.groovy
index e59d53f1cb..af17ab32fb 100644
--- a/regression-test/suites/export/test_array_export.groovy
+++ b/regression-test/suites/export/test_array_export.groovy
@@ -26,7 +26,6 @@ suite("test_array_export", "export") {
     StringBuilder strBuilder = new StringBuilder()
     strBuilder.append("curl --location-trusted -u " + context.config.jdbcUser + ":" + context.config.jdbcPassword)
     strBuilder.append(" http://" + context.config.feHttpAddress + "/rest/v1/config/fe")
-
     String command = strBuilder.toString()
     def process = command.toString().execute()
     def code = process.waitFor()
@@ -49,12 +48,15 @@ suite("test_array_export", "export") {
         logger.warn("Please set enable_outfile_to_local to true to run test_outfile")
         return
     }
-    
+
     // define the table and out file path
     def tableName = "array_outfile_test"
     def outFilePath = """${context.file.parent}/test_array_export"""
+    def outFile = "/tmp"
+    def urlHost = ""
+    def csvFiles = ""
     logger.warn("test_array_export the outFilePath=" + outFilePath)
-    
+
     def create_test_table = {testTablex ->
         sql """ DROP TABLE IF EXISTS ${tableName} """
 
@@ -66,7 +68,7 @@ suite("test_array_export", "export") {
               `k4` ARRAY<BIGINT> NOT NULL COMMENT "",
               `k5` ARRAY<CHAR> NOT NULL COMMENT "",
               `k6` ARRAY<VARCHAR(20)> NULL COMMENT "",
-              `k7` ARRAY<DATE> NOT NULL COMMENT "", 
+              `k7` ARRAY<DATE> NOT NULL COMMENT "",
               `k8` ARRAY<DATETIME> NOT NULL COMMENT "",
               `k9` ARRAY<FLOAT> NOT NULL COMMENT "",
               `k10` ARRAY<DOUBLE> NOT NULL COMMENT "",
@@ -84,26 +86,25 @@ suite("test_array_export", "export") {
         assertTrue(result1.size() == 1)
         assertTrue(result1[0].size() == 1)
         assertTrue(result1[0][0] == 0, "Create table should update 0 rows")
-
         sql """ INSERT INTO ${tableName} VALUES
-                        (1, [1, 2, 3], [32767, 32768, 32769], [65534, 65535, 65536], ['a', 'b', 'c'], ["hello", "world"], 
+                        (1, [1, 2, 3], [32767, 32768, 32769], [65534, 65535, 65536], ['a', 'b', 'c'], ["hello", "world"],
                         ['2022-07-13'], ['2022-08-15 12:30:00'], [0.331111, 0.672222], [3.141592, 0.878787], [4.2222, 5.5555, 6.67])
                         """
-        
+
         sql """ INSERT INTO ${tableName} VALUES
-                        (2, [4, 5, 6], [32767, 32768, 32769], [65534, 65535, 65536], ['d', 'e', 'f'], ["good", "luck"], 
+                        (2, [4, 5, 6], [32767, 32768, 32769], [65534, 65535, 65536], ['d', 'e', 'f'], ["good", "luck"],
                         ['2022-07-13'], ['2022-08-15 15:59:59'], [0.333336, 0.666677], [3.141592, 0.878787], [4.22222, 5.5555555, 6.6666777])
                         """
     }
-    
+
     def export_to_hdfs = {exportTable, exportLable, hdfsPath, exportFormat, BrokerName, HdfsUserName, HdfsPasswd->
-        sql """ EXPORT TABLE ${exportTable} 
-                TO "${hdfsPath}" 
+        sql """ EXPORT TABLE ${exportTable}
+                TO "${hdfsPath}"
                 PROPERTIES (
                     "label" = "${exportLable}",
                     "column_separator"=",",
                     "format"="${exportFormat}"
-                ) 
+                )
                 WITH BROKER "${BrokerName}" (
                     "username"="${HdfsUserName}",
                     "password"="${HdfsPasswd}"
@@ -126,7 +127,7 @@ suite("test_array_export", "export") {
             )
         """
     }
-    
+
     def check_export_result = {checklabel->
         max_try_milli_secs = 15000
         while(max_try_milli_secs) {
@@ -155,7 +156,7 @@ suite("test_array_export", "export") {
     // case1: test "select ...into outfile ...."
     try {
         create_test_table.call(tableName)
-        
+
         qt_select_default """ SELECT * FROM ${tableName} t ORDER BY k1; """
 
         // check outfile
@@ -165,9 +166,15 @@ suite("test_array_export", "export") {
         } else {
             throw new IllegalStateException("""${outFilePath} already exists! """)
         }
-        sql """
-            SELECT * FROM ${tableName} t ORDER BY k1 INTO OUTFILE "file://${outFilePath}/";
+        result = sql """
+            SELECT * FROM ${tableName} t ORDER BY k1 INTO OUTFILE "file://${outFile}/";
         """
+        url = result[0][3]
+        urlHost = url.substring(8, url.indexOf("${outFile}"))
+        def filePrifix = url.split("${outFile}")[1]
+        csvFiles = "${outFile}${filePrifix}*.csv"
+        scpFiles ("root", urlHost, csvFiles, outFilePath);
+
         File[] files = path.listFiles()
         assert files.length == 1
         List<String> outLines = Files.readAllLines(Paths.get(files[0].getAbsolutePath()), StandardCharsets.UTF_8);
@@ -186,14 +193,17 @@ suite("test_array_export", "export") {
             }
             path.delete();
         }
+        cmd = "rm -rf ${csvFiles}"
+        sshExec ("root", urlHost, cmd)
     }
-
+    
+    
     if (enableHdfs()) {
         brokerName = getBrokerName()
         hdfsUser = getHdfsUser()
         hdfsPasswd = getHdfsPasswd()
         hdfsDataDir = getHdfsDataDir()
-        
+
         // case2: test "select ...into outfile 'hdfs_path'"
         try {
             create_test_table.call(tableName)
@@ -225,4 +235,4 @@ suite("test_array_export", "export") {
             try_sql("DROP TABLE IF EXISTS ${tableName}")
         }
     }
-}
+}
\ No newline at end of file
diff --git a/regression-test/suites/export/test_map_export.groovy b/regression-test/suites/export/test_map_export.groovy
index 362e512491..d6e3a2e197 100644
--- a/regression-test/suites/export/test_map_export.groovy
+++ b/regression-test/suites/export/test_map_export.groovy
@@ -78,6 +78,9 @@ suite("test_map_export", "export") {
     qt_select_count """SELECT COUNT(m) FROM ${testTable}"""
 
     def outFilePath = """${context.file.parent}/test_map_export"""
+    def outFile = "/tmp"
+    def urlHost = ""
+    def csvFiles = ""
     logger.info("test_map_export the outFilePath=" + outFilePath)
     // map select into outfile
     try {
@@ -87,9 +90,15 @@ suite("test_map_export", "export") {
         } else {
             throw new IllegalStateException("""${outFilePath} already exists! """)
         }
-        sql """
-                    SELECT * FROM ${testTable} ORDER BY id INTO OUTFILE "file://${outFilePath}/";
+        result = sql """
+                    SELECT * FROM ${testTable} ORDER BY id INTO OUTFILE "file://${outFile}/";
         """
+        url = result[0][3]
+        urlHost = url.substring(8, url.indexOf("${outFile}"))
+        def filePrifix = url.split("${outFile}")[1]
+        csvFiles = "${outFile}${filePrifix}*.csv"
+        scpFiles ("root", urlHost, csvFiles, outFilePath);
+
         File[] files = path.listFiles()
         assert files.length == 1
 
@@ -128,5 +137,7 @@ suite("test_map_export", "export") {
             }
             path.delete();
         }
+        cmd = "rm -rf ${csvFiles}"
+        sshExec ("root", urlHost, cmd)
     }
 }
diff --git a/regression-test/suites/export/test_struct_export.groovy b/regression-test/suites/export/test_struct_export.groovy
index a1b6ccbe06..485270ccbf 100644
--- a/regression-test/suites/export/test_struct_export.groovy
+++ b/regression-test/suites/export/test_struct_export.groovy
@@ -80,6 +80,9 @@ suite("test_struct_export", "export") {
     qt_select_count """SELECT COUNT(k2), COUNT(k4) FROM ${testTable}"""
 
     def outFilePath = """${context.file.parent}/test_struct_export"""
+    def outFile = "/tmp"
+    def urlHost = ""
+    def csvFiles = ""
     logger.info("test_struct_export the outFilePath=" + outFilePath)
     // struct select into outfile
     try {
@@ -89,9 +92,15 @@ suite("test_struct_export", "export") {
         } else {
             throw new IllegalStateException("""${outFilePath} already exists! """)
         }
-        sql """
-                    SELECT * FROM ${testTable} ORDER BY k1 INTO OUTFILE "file://${outFilePath}/";
+        result = sql """
+                    SELECT * FROM ${testTable} ORDER BY k1 INTO OUTFILE "file://${outFile}/";
         """
+        url = result[0][3]
+        urlHost = url.substring(8, url.indexOf("${outFile}"))
+        def filePrifix = url.split("${outFile}")[1]
+        csvFiles = "${outFile}${filePrifix}*.csv"
+        scpFiles ("root", urlHost, csvFiles, outFilePath);
+
         File[] files = path.listFiles()
         assert files.length == 1
 
@@ -131,5 +140,7 @@ suite("test_struct_export", "export") {
             }
             path.delete();
         }
+        cmd = "rm -rf ${csvFiles}"
+        sshExec ("root", urlHost, cmd)
     }
 }
diff --git a/regression-test/suites/export_p2/test_export_with_s3.groovy b/regression-test/suites/export_p2/test_export_with_s3.groovy
index 9dc5100054..a26dde3238 100644
--- a/regression-test/suites/export_p2/test_export_with_s3.groovy
+++ b/regression-test/suites/export_p2/test_export_with_s3.groovy
@@ -21,7 +21,7 @@ suite("test_export_with_s3", "p2") {
     String sk = getS3SK()
     String s3_endpoint = getS3Endpoint()
     String region = getS3Region()
-    String bucket = context.config.otherConfigs.get("s3ExportBucketName");
+    String bucket = context.config.otherConfigs.get("s3BucketName");
 
     def table_export_name = "test_export_with_s3"
     // create table and insert


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 03/36: [opt](Nereids) perfer use datev2 / datetimev2 in date related functions (#20224)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 219332211d4fc29fb1a6ddb6b7a906a97dc6e4cc
Author: morrySnow <10...@users.noreply.github.com>
AuthorDate: Tue Jun 6 11:42:29 2023 +0800

    [opt](Nereids) perfer use datev2 / datetimev2 in date related functions (#20224)
    
    1. update all date related functions' signatures order.
    1.1. if return value need to be compute with time info, args with datetimev2 at the top of the list, followed by datev2, datetime and date
    1.2. if return value need to be compute with only date info, args with datev2 at the top of list, followed by datetimev2, date and datetime
    2. Priority for use datev2, if we must cast date to datev2 or datetime/datetimev2
---
 .../expressions/functions/SearchSignature.java     | 30 ++++++++++++++++++----
 .../expressions/functions/scalar/ConvertTz.java    |  6 ++---
 .../trees/expressions/functions/scalar/Date.java   |  4 +--
 .../expressions/functions/scalar/DateDiff.java     |  8 +++---
 .../expressions/functions/scalar/DateFormat.java   |  6 ++---
 .../expressions/functions/scalar/DateTrunc.java    |  4 +--
 .../expressions/functions/scalar/DayCeil.java      | 18 ++++++-------
 .../expressions/functions/scalar/DayFloor.java     | 18 ++++++-------
 .../expressions/functions/scalar/DayName.java      |  4 +--
 .../expressions/functions/scalar/DayOfMonth.java   |  6 ++---
 .../expressions/functions/scalar/DayOfWeek.java    |  4 +--
 .../expressions/functions/scalar/DayOfYear.java    |  4 +--
 .../expressions/functions/scalar/DaysAdd.java      |  4 +--
 .../expressions/functions/scalar/DaysDiff.java     | 10 ++++----
 .../expressions/functions/scalar/DaysSub.java      |  4 +--
 .../trees/expressions/functions/scalar/Hour.java   |  4 +--
 .../expressions/functions/scalar/HourCeil.java     | 13 +++++-----
 .../expressions/functions/scalar/HourFloor.java    | 13 +++++-----
 .../expressions/functions/scalar/HoursAdd.java     |  7 ++---
 .../expressions/functions/scalar/HoursDiff.java    |  4 +--
 .../expressions/functions/scalar/HoursSub.java     |  7 ++---
 .../trees/expressions/functions/scalar/Minute.java |  4 +--
 .../expressions/functions/scalar/MinuteCeil.java   | 14 +++++-----
 .../expressions/functions/scalar/MinuteFloor.java  | 14 +++++-----
 .../expressions/functions/scalar/MinutesAdd.java   |  9 ++++---
 .../expressions/functions/scalar/MinutesDiff.java  |  4 +--
 .../expressions/functions/scalar/MinutesSub.java   |  9 ++++---
 .../trees/expressions/functions/scalar/Month.java  |  6 ++---
 .../expressions/functions/scalar/MonthCeil.java    | 20 +++++++--------
 .../expressions/functions/scalar/MonthFloor.java   | 20 +++++++--------
 .../expressions/functions/scalar/MonthName.java    |  6 ++---
 .../expressions/functions/scalar/MonthsAdd.java    |  9 ++++---
 .../expressions/functions/scalar/MonthsDiff.java   | 10 ++++----
 .../expressions/functions/scalar/MonthsSub.java    |  9 ++++---
 .../trees/expressions/functions/scalar/Second.java |  6 ++---
 .../expressions/functions/scalar/SecondCeil.java   |  2 +-
 .../expressions/functions/scalar/SecondFloor.java  | 15 ++++++-----
 .../expressions/functions/scalar/SecondsAdd.java   |  9 ++++---
 .../expressions/functions/scalar/SecondsDiff.java  |  8 +++---
 .../expressions/functions/scalar/SecondsSub.java   |  9 ++++---
 .../expressions/functions/scalar/TimeDiff.java     |  4 +--
 .../trees/expressions/functions/scalar/ToDate.java |  6 ++---
 .../trees/expressions/functions/scalar/ToDays.java |  6 ++---
 .../expressions/functions/scalar/ToMonday.java     |  8 +++---
 .../functions/scalar/UnixTimestamp.java            |  6 ++---
 .../trees/expressions/functions/scalar/Week.java   | 10 ++++----
 .../expressions/functions/scalar/WeekCeil.java     | 20 +++++++--------
 .../expressions/functions/scalar/WeekFloor.java    | 20 +++++++--------
 .../expressions/functions/scalar/WeekOfYear.java   |  6 ++---
 .../expressions/functions/scalar/Weekday.java      |  6 ++---
 .../expressions/functions/scalar/WeeksAdd.java     |  5 ++--
 .../expressions/functions/scalar/WeeksDiff.java    | 10 ++++----
 .../expressions/functions/scalar/WeeksSub.java     |  5 ++--
 .../trees/expressions/functions/scalar/Year.java   |  6 ++---
 .../expressions/functions/scalar/YearCeil.java     | 20 +++++++--------
 .../expressions/functions/scalar/YearFloor.java    | 20 +++++++--------
 .../expressions/functions/scalar/YearWeek.java     | 10 ++++----
 .../expressions/functions/scalar/YearsAdd.java     |  9 ++++---
 .../expressions/functions/scalar/YearsDiff.java    | 10 ++++----
 .../expressions/functions/scalar/YearsSub.java     |  9 ++++---
 .../test_conditional_function.out                  |  3 ++-
 .../suites/nereids_syntax_p0/type_cast.groovy      |  2 +-
 62 files changed, 299 insertions(+), 263 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/SearchSignature.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/SearchSignature.java
index 9e2fa281ed..496b51d9ca 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/SearchSignature.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/SearchSignature.java
@@ -18,10 +18,13 @@
 package org.apache.doris.nereids.trees.expressions.functions;
 
 import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.common.Pair;
 import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.types.DataType;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DateV2Type;
 import org.apache.doris.nereids.types.DecimalV3Type;
 import org.apache.doris.nereids.types.coercion.AbstractDataType;
 import org.apache.doris.nereids.util.TypeCoercionUtils;
@@ -75,6 +78,7 @@ public class SearchSignature {
         // search every round
         for (BiFunction<AbstractDataType, AbstractDataType, Boolean> typePredicate : typePredicatePerRound) {
             int candidateNonStrictMatched = Integer.MAX_VALUE;
+            int candidateDateToDateV2Count = Integer.MIN_VALUE;
             FunctionSignature candidate = null;
             for (FunctionSignature signature : signatures) {
                 if (doMatchArity(signature, arguments) && doMatchTypes(signature, arguments, typePredicate)) {
@@ -90,10 +94,17 @@ public class SearchSignature {
                         }
                     }
                     // has most identical matched signature has the highest priority
-                    int currentNonStrictMatched = nonStrictMatchedCount(signature, arguments);
-                    if (currentNonStrictMatched < candidateNonStrictMatched) {
-                        candidateNonStrictMatched = currentNonStrictMatched;
+                    Pair<Integer, Integer> currentNonStrictMatched = nonStrictMatchedCount(signature, arguments);
+                    if (currentNonStrictMatched.first < candidateNonStrictMatched) {
+                        candidateNonStrictMatched = currentNonStrictMatched.first;
+                        candidateDateToDateV2Count = currentNonStrictMatched.second;
                         candidate = signature;
+                    } else if (currentNonStrictMatched.first == candidateNonStrictMatched) {
+                        // if we need to do same count cast, then we choose the signature need to do more v1 to v2 cast
+                        if (candidateDateToDateV2Count < currentNonStrictMatched.second) {
+                            candidateDateToDateV2Count = currentNonStrictMatched.second;
+                            candidate = signature;
+                        }
                     }
                 }
             }
@@ -168,17 +179,26 @@ public class SearchSignature {
         return true;
     }
 
-    private int nonStrictMatchedCount(FunctionSignature sig, List<Expression> arguments) {
+    /**
+     * return non-strict matched count and convert v1 to v2 count.
+     *
+     * @return the first value indic non-strict matched count, the second value indic date to datev2 count
+     */
+    private Pair<Integer, Integer> nonStrictMatchedCount(FunctionSignature sig, List<Expression> arguments) {
         int nonStrictMatched = 0;
+        int dateToDateV2Count = 0;
         int arity = arguments.size();
         for (int i = 0; i < arity; i++) {
             AbstractDataType sigArgType = sig.getArgType(i);
             AbstractDataType realType = arguments.get(i).getDataType();
             if (!IdenticalSignature.isIdentical(sigArgType, realType)) {
                 nonStrictMatched++;
+                if (sigArgType instanceof DateV2Type && realType instanceof DateType) {
+                    dateToDateV2Count++;
+                }
             }
         }
-        return nonStrictMatched;
+        return Pair.of(nonStrictMatched, dateToDateV2Count);
     }
 
     private boolean doMatchTypes(FunctionSignature sig, List<Expression> arguments,
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java
index 3d86002ecb..17a0ccfb33 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java
@@ -39,10 +39,10 @@ public class ConvertTz extends ScalarFunction
         implements TernaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
-                    .args(DateTimeV2Type.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT)
+                    .args(DateTimeV2Type.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Date.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Date.java
index 01317ab745..01fa2acd98 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Date.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Date.java
@@ -40,8 +40,8 @@ public class Date extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateType.INSTANCE).args(DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT)
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateDiff.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateDiff.java
index af52b494e4..0595f70ea0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateDiff.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateDiff.java
@@ -40,12 +40,12 @@ public class DateDiff extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateV2Type.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, DateV2Type.INSTANCE),
             FunctionSignature.ret(IntegerType.INSTANCE)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, DateV2Type.INSTANCE),
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateV2Type.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE)
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateFormat.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateFormat.java
index 014e765c1f..43f85542b0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateFormat.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateFormat.java
@@ -41,11 +41,11 @@ public class DateFormat extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateTimeType.INSTANCE, VarcharType.SYSTEM_DEFAULT),
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateType.INSTANCE, VarcharType.SYSTEM_DEFAULT),
             FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT),
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, VarcharType.SYSTEM_DEFAULT)
+            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, VarcharType.SYSTEM_DEFAULT),
+            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateTimeType.INSTANCE, VarcharType.SYSTEM_DEFAULT),
+            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateType.INSTANCE, VarcharType.SYSTEM_DEFAULT)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java
index 7da24c0710..cf492ffc5a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java
@@ -39,9 +39,9 @@ public class DateTrunc extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, VarcharType.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
-                    .args(DateTimeV2Type.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT)
+                    .args(DateTimeV2Type.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, VarcharType.SYSTEM_DEFAULT)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayCeil.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayCeil.java
index e30a2a8ca0..7f559389f6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayCeil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayCeil.java
@@ -39,23 +39,23 @@ public class DayCeil extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE)
-                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE)
+                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayFloor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayFloor.java
index c3ec1b8e6d..f42371f5f0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayFloor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayFloor.java
@@ -39,23 +39,23 @@ public class DayFloor extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE)
-                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE)
+                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayName.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayName.java
index 023e718c42..02457668ca 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayName.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayName.java
@@ -40,9 +40,9 @@ public class DayName extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE),
             FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE)
+            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfMonth.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfMonth.java
index 4740126394..a64bff850b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfMonth.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfMonth.java
@@ -40,10 +40,10 @@ public class DayOfMonth extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE),
             FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE)
-    );
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE)
+            );
 
     /**
      * constructor with 1 argument.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfWeek.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfWeek.java
index aede638b01..e98d250811 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfWeek.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfWeek.java
@@ -40,9 +40,9 @@ public class DayOfWeek extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE),
             FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE)
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfYear.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfYear.java
index 2deb7357d8..a32f64325e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfYear.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DayOfYear.java
@@ -40,9 +40,9 @@ public class DayOfYear extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(SmallIntType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(SmallIntType.INSTANCE).args(DateV2Type.INSTANCE),
             FunctionSignature.ret(SmallIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(SmallIntType.INSTANCE).args(DateV2Type.INSTANCE)
+            FunctionSignature.ret(SmallIntType.INSTANCE).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysAdd.java
index 3cbb1e999c..a5eb3e53f2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysAdd.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysAdd.java
@@ -40,9 +40,9 @@ import java.util.List;
  */
 public class DaysAdd extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    //When enable_date_conversion is true, we prefer to V2 signature.
+    // When enable_date_conversion is true, we prefer to V2 signature.
     // This preference follows original planner. refer to ScalarType.getDefaultDateType()
-    public static final List<FunctionSignature> SIGNATURES = Config.enable_date_conversion ? ImmutableList.of(
+    private static final List<FunctionSignature> SIGNATURES = Config.enable_date_conversion ? ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysDiff.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysDiff.java
index 7cba92b10b..3f12ba2d74 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysDiff.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysDiff.java
@@ -39,13 +39,13 @@ import java.util.List;
 public class DaysDiff extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(BigIntType.INSTANCE)
-                    .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, DateV2Type.INSTANCE),
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE)
+            FunctionSignature.ret(BigIntType.INSTANCE)
+                    .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysSub.java
index 9dbe0cb7d9..09e79fb072 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysSub.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaysSub.java
@@ -40,9 +40,9 @@ import java.util.List;
  */
 public class DaysSub extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    //When enable_date_conversion is true, we prefer to V2 signature.
+    // When enable_date_conversion is true, we prefer to V2 signature.
     // This preference follows original planner. refer to ScalarType.getDefaultDateType()
-    public static final List<FunctionSignature> SIGNATURES = Config.enable_date_conversion ? ImmutableList.of(
+    private static final List<FunctionSignature> SIGNATURES = Config.enable_date_conversion ? ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Hour.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Hour.java
index 9be1d31e32..4d2248b940 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Hour.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Hour.java
@@ -40,9 +40,9 @@ public class Hour extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE),
             FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE)
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HourCeil.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HourCeil.java
index 44416df744..08cef59ce2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HourCeil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HourCeil.java
@@ -38,18 +38,19 @@ public class HourCeil extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
-                    .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT));
+                    .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
+    );
 
     /**
      * constructor with 1 argument.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HourFloor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HourFloor.java
index 8a42ccdeab..2918b28896 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HourFloor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HourFloor.java
@@ -38,18 +38,19 @@ public class HourFloor extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
-                    .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT));
+                    .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
+    );
 
     /**
      * constructor with 1 argument.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursAdd.java
index 5ba319da4d..cb4e601b14 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursAdd.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursAdd.java
@@ -39,12 +39,13 @@ import java.util.List;
  */
 public class HoursAdd extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
+
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE)
     );
 
     public HoursAdd(Expression arg0, Expression arg1) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursDiff.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursDiff.java
index 354b163821..452e3110ff 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursDiff.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursDiff.java
@@ -40,12 +40,12 @@ public class HoursDiff extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(BigIntType.INSTANCE)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, DateV2Type.INSTANCE),
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE)
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursSub.java
index 53f49edb89..e53f8ecb90 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursSub.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HoursSub.java
@@ -39,12 +39,13 @@ import java.util.List;
  */
 public class HoursSub extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
+
     public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE)
     );
 
     public HoursSub(Expression arg0, Expression arg1) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Minute.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Minute.java
index 4bd8608771..642fcd8b14 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Minute.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Minute.java
@@ -39,9 +39,9 @@ import java.util.List;
 public class Minute extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE),
             FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE)
     );
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinuteCeil.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinuteCeil.java
index 38dac88d30..b00eaff07c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinuteCeil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinuteCeil.java
@@ -37,19 +37,19 @@ import java.util.List;
 public class MinuteCeil extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
-                    .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT)
+                    .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinuteFloor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinuteFloor.java
index 6c0e1dcb4b..683acc3a48 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinuteFloor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinuteFloor.java
@@ -37,19 +37,19 @@ import java.util.List;
 public class MinuteFloor extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
-                    .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT)
+                    .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesAdd.java
index cb9c6f1c51..f4c02fb84c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesAdd.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesAdd.java
@@ -39,12 +39,13 @@ import java.util.List;
  */
 public class MinutesAdd extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE)
     );
 
     public MinutesAdd(Expression arg0, Expression arg1) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesDiff.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesDiff.java
index b4f86a1352..178b6a4933 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesDiff.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesDiff.java
@@ -39,10 +39,10 @@ import java.util.List;
 public class MinutesDiff extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(BigIntType.INSTANCE)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, DateV2Type.INSTANCE),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE)
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesSub.java
index 9aa6dee984..4fb6169578 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesSub.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinutesSub.java
@@ -39,12 +39,13 @@ import java.util.List;
  */
 public class MinutesSub extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE)
     );
 
     public MinutesSub(Expression arg0, Expression arg1) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Month.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Month.java
index 29556e705c..9f8d38dc75 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Month.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Month.java
@@ -39,10 +39,10 @@ import java.util.List;
 public class Month extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE),
             FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE)
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthCeil.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthCeil.java
index b2b0560cc6..e952f56a48 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthCeil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthCeil.java
@@ -38,24 +38,24 @@ import java.util.List;
 public class MonthCeil extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE)
-                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE)
+                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthFloor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthFloor.java
index 83ee10f66c..8703f4c1b8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthFloor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthFloor.java
@@ -38,24 +38,24 @@ import java.util.List;
 public class MonthFloor extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE)
-                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE)
+                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthName.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthName.java
index a0444c0fee..e12f39aaef 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthName.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthName.java
@@ -39,10 +39,10 @@ import java.util.List;
 public class MonthName extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE),
             FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE)
+            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsAdd.java
index 106afd7aba..c11534e931 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsAdd.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsAdd.java
@@ -39,12 +39,13 @@ import java.util.List;
  */
 public class MonthsAdd extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE)
     );
 
     public MonthsAdd(Expression arg0, Expression arg1) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsDiff.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsDiff.java
index b42017b3d0..373265b1e3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsDiff.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsDiff.java
@@ -39,13 +39,13 @@ import java.util.List;
 public class MonthsDiff extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(BigIntType.INSTANCE)
-                    .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, DateV2Type.INSTANCE),
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE)
+            FunctionSignature.ret(BigIntType.INSTANCE)
+                    .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsSub.java
index be237f90ff..279cd3af6d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsSub.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MonthsSub.java
@@ -39,12 +39,13 @@ import java.util.List;
  */
 public class MonthsSub extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE)
     );
 
     public MonthsSub(Expression arg0, Expression arg1) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Second.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Second.java
index 6b36147154..91167cfc9a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Second.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Second.java
@@ -39,10 +39,10 @@ import java.util.List;
 public class Second extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE)
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondCeil.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondCeil.java
index e51966258a..3dea172f78 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondCeil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondCeil.java
@@ -37,7 +37,7 @@ import java.util.List;
 public class SecondCeil extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondFloor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondFloor.java
index 811ce9f994..fef3e16e78 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondFloor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondFloor.java
@@ -37,19 +37,20 @@ import java.util.List;
 public class SecondFloor extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
-                    .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT)
+                    .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
+
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsAdd.java
index d85931d39c..a6e131f526 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsAdd.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsAdd.java
@@ -39,12 +39,13 @@ import java.util.List;
  */
 public class SecondsAdd extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE)
     );
 
     public SecondsAdd(Expression arg0, Expression arg1) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsDiff.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsDiff.java
index cd07e6fff0..4dd7e12b9e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsDiff.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsDiff.java
@@ -39,14 +39,14 @@ import java.util.List;
 public class SecondsDiff extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(BigIntType.INSTANCE)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, DateV2Type.INSTANCE),
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE)
-    );
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE)
+            );
 
     /**
      * constructor with 2 arguments.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsSub.java
index f52bb5fbe4..37c59b2168 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsSub.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondsSub.java
@@ -39,12 +39,13 @@ import java.util.List;
  */
 public class SecondsSub extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE)
     );
 
     public SecondsSub(Expression arg0, Expression arg1) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/TimeDiff.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/TimeDiff.java
index 8e6652686e..8004e254a2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/TimeDiff.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/TimeDiff.java
@@ -40,10 +40,10 @@ import java.util.List;
 public class TimeDiff extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(TimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(TimeV2Type.INSTANCE)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(TimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(TimeV2Type.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, DateV2Type.INSTANCE),
             FunctionSignature.ret(TimeV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(TimeV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE)
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToDate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToDate.java
index 5dfbf843ed..a1310d3aa1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToDate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToDate.java
@@ -39,9 +39,9 @@ import java.util.List;
 public class ToDate extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateType.INSTANCE).args(DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT)
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToDays.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToDays.java
index bde66c832b..5e267fbd51 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToDays.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToDays.java
@@ -38,9 +38,9 @@ import java.util.List;
 public class ToDays extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateType.INSTANCE),
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateV2Type.INSTANCE)
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToMonday.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToMonday.java
index 7c2d1f3fe7..216b3d3ba7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToMonday.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ToMonday.java
@@ -39,11 +39,11 @@ import java.util.List;
 public class ToMonday extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateType.INSTANCE).args(DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE)
+            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/UnixTimestamp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/UnixTimestamp.java
index 3762308fc8..2c33115d5b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/UnixTimestamp.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/UnixTimestamp.java
@@ -41,12 +41,12 @@ import java.util.List;
 public class UnixTimestamp extends ScalarFunction
         implements ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(IntegerType.INSTANCE).args(),
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeType.INSTANCE),
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateType.INSTANCE),
             FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(IntegerType.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateType.INSTANCE),
             FunctionSignature.ret(IntegerType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT),
             FunctionSignature.ret(IntegerType.INSTANCE).args(StringType.INSTANCE, StringType.INSTANCE)
     );
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Week.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Week.java
index 51bdbdb07f..b2246d8488 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Week.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Week.java
@@ -39,13 +39,13 @@ import java.util.List;
 public class Week extends ScalarFunction
         implements ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE),
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekCeil.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekCeil.java
index d830e9f15a..968073f690 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekCeil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekCeil.java
@@ -38,24 +38,24 @@ import java.util.List;
 public class WeekCeil extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE)
-                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE)
+                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekFloor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekFloor.java
index 9879d4a036..511aff78ac 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekFloor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekFloor.java
@@ -38,24 +38,24 @@ import java.util.List;
 public class WeekFloor extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE)
-                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE)
+                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekOfYear.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekOfYear.java
index 666f8be866..1d2ab07a25 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekOfYear.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeekOfYear.java
@@ -39,10 +39,10 @@ import java.util.List;
 public class WeekOfYear extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE),
             FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE)
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Weekday.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Weekday.java
index c8700c0119..ffb1cbf6a4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Weekday.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Weekday.java
@@ -39,10 +39,10 @@ import java.util.List;
 public class Weekday extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE),
             FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateV2Type.INSTANCE)
+            FunctionSignature.ret(TinyIntType.INSTANCE).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksAdd.java
index 819c537d13..3cfd6e8c6f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksAdd.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksAdd.java
@@ -40,9 +40,10 @@ import java.util.List;
  */
 public class WeeksAdd extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    //When enable_date_conversion is true, we prefer to V2 signature.
+
+    // When enable_date_conversion is true, we prefer to V2 signature.
     // This preference follows original planner. refer to ScalarType.getDefaultDateType()
-    public static final List<FunctionSignature> SIGNATURES = Config.enable_date_conversion ? ImmutableList.of(
+    private static final List<FunctionSignature> SIGNATURES = Config.enable_date_conversion ? ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksDiff.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksDiff.java
index b4bd09e042..dfacc8137b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksDiff.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksDiff.java
@@ -39,13 +39,13 @@ import java.util.List;
 public class WeeksDiff extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(BigIntType.INSTANCE)
-                    .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, DateV2Type.INSTANCE),
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE)
+            FunctionSignature.ret(BigIntType.INSTANCE)
+            .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksSub.java
index 89ed404a6a..14bfc93683 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksSub.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/WeeksSub.java
@@ -40,9 +40,10 @@ import java.util.List;
  */
 public class WeeksSub extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    //When enable_date_conversion is true, we prefer to V2 signature.
+
+    // When enable_date_conversion is true, we prefer to V2 signature.
     // This preference follows original planner. refer to ScalarType.getDefaultDateType()
-    public static final List<FunctionSignature> SIGNATURES = Config.enable_date_conversion ? ImmutableList.of(
+    private static final List<FunctionSignature> SIGNATURES = Config.enable_date_conversion ? ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Year.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Year.java
index a51a925498..7268e1a79b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Year.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Year.java
@@ -39,10 +39,10 @@ import java.util.List;
 public class Year extends ScalarFunction
         implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(SmallIntType.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(SmallIntType.INSTANCE).args(DateTimeType.INSTANCE),
-            FunctionSignature.ret(SmallIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT)
+            FunctionSignature.ret(SmallIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(SmallIntType.INSTANCE).args(DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearCeil.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearCeil.java
index b5fe27a6fb..bc294638be 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearCeil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearCeil.java
@@ -38,24 +38,24 @@ import java.util.List;
 public class YearCeil extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE)
-                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE)
+                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearFloor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearFloor.java
index 9f046bf5e1..5415502a76 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearFloor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearFloor.java
@@ -38,24 +38,24 @@ import java.util.List;
 public class YearFloor extends ScalarFunction
         implements ExplicitlyCastableSignature, AlwaysNullable {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateTimeType.INSTANCE)
-                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(DateV2Type.INSTANCE)
-                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE)
+                    .args(DateV2Type.INSTANCE, IntegerType.INSTANCE, DateV2Type.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE)
+                    .args(DateTimeType.INSTANCE, IntegerType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearWeek.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearWeek.java
index b4acd75606..5afc52b2f6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearWeek.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearWeek.java
@@ -38,13 +38,13 @@ import java.util.List;
 public class YearWeek extends ScalarFunction
         implements ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeType.INSTANCE),
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(IntegerType.INSTANCE).args(DateV2Type.INSTANCE),
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(IntegerType.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeType.INSTANCE),
+            FunctionSignature.ret(IntegerType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsAdd.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsAdd.java
index 10f39c52c5..36c140ffed 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsAdd.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsAdd.java
@@ -39,12 +39,13 @@ import java.util.List;
  */
 public class YearsAdd extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE)
     );
 
     public YearsAdd(Expression arg0, Expression arg1) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsDiff.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsDiff.java
index f52cdaa105..e217d8da72 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsDiff.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsDiff.java
@@ -39,13 +39,13 @@ import java.util.List;
 public class YearsDiff extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
 
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
-            FunctionSignature.ret(BigIntType.INSTANCE)
-                    .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT),
             FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeV2Type.SYSTEM_DEFAULT, DateV2Type.INSTANCE),
-            FunctionSignature.ret(BigIntType.INSTANCE).args(DateV2Type.INSTANCE, DateV2Type.INSTANCE)
+            FunctionSignature.ret(BigIntType.INSTANCE)
+                    .args(DateTimeV2Type.SYSTEM_DEFAULT, DateTimeV2Type.SYSTEM_DEFAULT),
+            FunctionSignature.ret(BigIntType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE)
     );
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsSub.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsSub.java
index 10ef1c15d0..3011260331 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsSub.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/YearsSub.java
@@ -39,12 +39,13 @@ import java.util.List;
  */
 public class YearsSub extends ScalarFunction
         implements BinaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args {
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+
+    private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
             FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT)
                     .args(DateTimeV2Type.SYSTEM_DEFAULT, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE),
-            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE)
+            FunctionSignature.ret(DateV2Type.INSTANCE).args(DateV2Type.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, IntegerType.INSTANCE),
+            FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, IntegerType.INSTANCE)
     );
 
     public YearsSub(Expression arg0, Expression arg1) {
diff --git a/regression-test/data/query_p0/sql_functions/conditional_functions/test_conditional_function.out b/regression-test/data/query_p0/sql_functions/conditional_functions/test_conditional_function.out
index c657c48dcd..5fa35d4a5b 100644
--- a/regression-test/data/query_p0/sql_functions/conditional_functions/test_conditional_function.out
+++ b/regression-test/data/query_p0/sql_functions/conditional_functions/test_conditional_function.out
@@ -396,4 +396,5 @@ true
 4
 
 -- !sql --
-9999-08-02 00:00
+9999-08-02T00:00
+
diff --git a/regression-test/suites/nereids_syntax_p0/type_cast.groovy b/regression-test/suites/nereids_syntax_p0/type_cast.groovy
index 87d381ad94..1fa087a196 100644
--- a/regression-test/suites/nereids_syntax_p0/type_cast.groovy
+++ b/regression-test/suites/nereids_syntax_p0/type_cast.groovy
@@ -34,7 +34,7 @@ suite("type_cast") {
     sql """insert into test_table2 values('2020-05-25');"""
 
     def ret = sql"""explain verbose select * from test_table2 where day > CONVERT_tz('2020-05-25 00:00:00', 'Asia/Shanghai', 'Asia/Shanghai');"""
-    assertTrue(ret.toString().contains("CAST(day[#0] AS DATETIMEV2(0))"))
+    assertTrue(ret.toString().contains("CAST(day[#0] AS DATETIMEV2(6))"))
 
     qt_sql """select count(*) from test_table2 where 'a' = 'a';"""
     qt_sql """select count(*) from test_table2 where cast('2020-01-01' as date) = cast('2020-01-01' as date);"""


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 13/36: [improve](nereids)derive analytics node stats (#20340)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit a92b26881082f9c096e0c915d96e602ee3b97d4c
Author: minghong <en...@gmail.com>
AuthorDate: Mon Jun 5 18:56:20 2023 +0800

    [improve](nereids)derive analytics node stats (#20340)
    
    1. derive analytic node stats, add support for rank()
    2. filter estimation stats derive updated. update row count of filter column.
    3. use ColumnStatistics.orginal to replace ColumnStatistics.orginalNdv, where ColumnStatistics.orginal is the column statisics get from TableScan.
    TPCDS 70 on tpcds_sf100 improved from 23sec to 2 sec
    This pr has no performance downgrade on other tpcds queries and tpch queries.
---
 .../doris/nereids/stats/FilterEstimation.java      |  9 ++--
 .../apache/doris/nereids/stats/JoinEstimation.java |  4 +-
 .../doris/nereids/stats/StatsCalculator.java       | 14 +++++
 .../apache/doris/statistics/ColumnStatistic.java   | 23 +++++---
 .../doris/statistics/ColumnStatisticBuilder.java   | 14 +++--
 .../doris/statistics/StatisticsRepository.java     |  2 +-
 .../doris/nereids/util/HyperGraphBuilder.java      |  2 +-
 .../doris/statistics/StatsDeriveResultTest.java    |  2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query11.out | 30 +++++------
 .../nereids_tpcds_shape_sf100_p0/shape/query4.out  | 30 +++++------
 .../nereids_tpcds_shape_sf100_p0/shape/query44.out | 62 +++++++++++-----------
 .../nereids_tpcds_shape_sf100_p0/shape/query70.out | 58 ++++++++++----------
 .../nereids_tpcds_shape_sf100_p0/shape/query74.out | 29 +++++-----
 13 files changed, 155 insertions(+), 124 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java
index 90ab646146..d7ae653798 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java
@@ -370,6 +370,7 @@ public class FilterEstimation extends ExpressionVisitor<Statistics, EstimationCo
                 .setNdv(intersectRange.getDistinctValues());
         double sel = leftRange.overlapPercentWith(rightRange);
         Statistics updatedStatistics = context.statistics.withSel(sel);
+        leftColumnStatisticBuilder.setCount(updatedStatistics.getRowCount());
         updatedStatistics.addColumnStats(leftExpr, leftColumnStatisticBuilder.build());
         leftExpr.accept(new ColumnStatsAdjustVisitor(), updatedStatistics);
         return updatedStatistics;
@@ -392,15 +393,17 @@ public class FilterEstimation extends ExpressionVisitor<Statistics, EstimationCo
         double sel;
         double reduceRatio = 0.25;
         double bothSideReducedRatio = 0.9;
-        if (leftStats.ndv < leftStats.originalNdv * bothSideReducedRatio
-                && rightStats.ndv < rightStats.originalNdv * bothSideReducedRatio) {
+        if (!leftStats.rangeChanged() && !rightStats.rangeChanged()
+                && leftStats.ndv < leftStats.getOriginalNdv() * bothSideReducedRatio
+                && rightStats.ndv < rightStats.getOriginalNdv() * bothSideReducedRatio) {
             double sel1;
             if (leftStats.ndv > rightStats.ndv) {
                 sel1 = 1 / StatsMathUtil.nonZeroDivisor(leftStats.ndv);
             } else {
                 sel1 = 1 / StatsMathUtil.nonZeroDivisor(rightStats.ndv);
             }
-            double sel2 = Math.min(rightStats.ndv / rightStats.originalNdv, leftStats.ndv / leftStats.originalNdv);
+            double sel2 = Math.min(rightStats.ndv / rightStats.getOriginalNdv(),
+                    leftStats.ndv / leftStats.getOriginalNdv());
             sel = sel1 * Math.pow(sel2, reduceRatio);
         } else {
             sel = 1 / StatsMathUtil.nonZeroDivisor(Math.max(leftStats.ndv, rightStats.ndv));
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/JoinEstimation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/JoinEstimation.java
index efc5a5a4bc..df36d84a4a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/JoinEstimation.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/JoinEstimation.java
@@ -193,7 +193,7 @@ public class JoinEstimation {
         double rowCount;
         if (join.getJoinType().isLeftSemiOrAntiJoin()) {
             double semiRowCount = StatsMathUtil.divide(leftStats.getRowCount() * buildColStats.ndv,
-                    buildColStats.originalNdv);
+                    buildColStats.getOriginalNdv());
             if (join.getJoinType().isSemiJoin()) {
                 rowCount = semiRowCount;
             } else {
@@ -202,7 +202,7 @@ public class JoinEstimation {
         } else {
             //right semi or anti
             double semiRowCount = StatsMathUtil.divide(rightStats.getRowCount() * probColStats.ndv,
-                    probColStats.originalNdv);
+                    probColStats.getOriginalNdv());
             if (join.getJoinType().isSemiJoin()) {
                 rowCount = semiRowCount;
             } else {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
index 03eee83362..9341371646 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
@@ -30,7 +30,9 @@ import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.NamedExpression;
 import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.expressions.SlotReference;
+import org.apache.doris.nereids.trees.expressions.WindowExpression;
 import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
+import org.apache.doris.nereids.trees.expressions.functions.window.Rank;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.algebra.Aggregate;
 import org.apache.doris.nereids.trees.plans.algebra.EmptyRelation;
@@ -830,6 +832,18 @@ public class StatsCalculator extends DefaultPlanVisitor<Statistics, Void> {
         Map<Expression, ColumnStatistic> childColumnStats = stats.columnStatistics();
         Map<Expression, ColumnStatistic> columnStatisticMap = windowOperator.getWindowExpressions().stream()
                 .map(expr -> {
+                    //estimate rank()
+                    if (expr instanceof Alias && expr.child(0) instanceof WindowExpression
+                            && ((WindowExpression) expr.child(0)).getFunction() instanceof Rank) {
+                        ColumnStatisticBuilder colBuilder = new ColumnStatisticBuilder();
+                        colBuilder.setNdv(stats.getRowCount())
+                                .setOriginal(null)
+                                .setCount(stats.getRowCount())
+                                .setMinValue(0)
+                                .setMaxValue(stats.getRowCount());
+                        return Pair.of(expr.toSlot(), colBuilder.build());
+                    }
+                    //estimate other expressions
                     ColumnStatistic value = null;
                     Set<Slot> slots = expr.getInputSlots();
                     if (slots.isEmpty()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java
index 46dd8fa4a5..9485ecf662 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java
@@ -96,7 +96,7 @@ public class ColumnStatistic {
     but originalNdv is not. It is used to trace the change of a column's ndv through serials
     of sql operators.
      */
-    public final double originalNdv;
+    public final ColumnStatistic original;
 
     // For display only.
     public final LiteralExpr minExpr;
@@ -105,12 +105,12 @@ public class ColumnStatistic {
     // assign value when do stats estimation.
     public final Histogram histogram;
 
-    public ColumnStatistic(double count, double ndv, double originalNdv, double avgSizeByte,
+    public ColumnStatistic(double count, double ndv, ColumnStatistic original, double avgSizeByte,
             double numNulls, double dataSize, double minValue, double maxValue,
             double selectivity, LiteralExpr minExpr, LiteralExpr maxExpr, boolean isUnKnown, Histogram histogram) {
         this.count = count;
         this.ndv = ndv;
-        this.originalNdv = originalNdv;
+        this.original = original;
         this.avgSizeByte = avgSizeByte;
         this.numNulls = numNulls;
         this.dataSize = dataSize;
@@ -165,7 +165,6 @@ public class ColumnStatistic {
                 columnStatisticBuilder.setMaxValue(Double.MAX_VALUE);
             }
             columnStatisticBuilder.setSelectivity(1.0);
-            columnStatisticBuilder.setOriginalNdv(ndv);
             Histogram histogram = Env.getCurrentEnv().getStatisticsCache().getHistogram(tblId, idxId, colName)
                     .orElse(null);
             columnStatisticBuilder.setHistogram(histogram);
@@ -308,7 +307,7 @@ public class ColumnStatistic {
         statistic.put("MaxExpr", maxExpr);
         statistic.put("IsUnKnown", isUnKnown);
         statistic.put("Histogram", Histogram.serializeToJson(histogram));
-        statistic.put("OriginalNdv", originalNdv);
+        statistic.put("Original", original);
         return statistic;
     }
 
@@ -347,7 +346,7 @@ public class ColumnStatistic {
         return new ColumnStatistic(
             stat.getDouble("Count"),
             stat.getDouble("Ndv"),
-            stat.getDouble("OriginalNdv"),
+            null,
             stat.getDouble("AvgSizeByte"),
             stat.getDouble("NumNulls"),
             stat.getDouble("DataSize"),
@@ -368,4 +367,16 @@ public class ColumnStatistic {
     public boolean hasHistogram() {
         return histogram != null && histogram != Histogram.UNKNOWN;
     }
+
+    public double getOriginalNdv() {
+        if (original != null) {
+            return original.ndv;
+        }
+        return ndv;
+    }
+
+    // TODO expanded this function to support more cases, help to compute the change of ndv density
+    public boolean rangeChanged() {
+        return original != null && (minValue != original.minValue || maxValue != original.maxValue);
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatisticBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatisticBuilder.java
index 9c34924b26..60e0bdab85 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatisticBuilder.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatisticBuilder.java
@@ -35,7 +35,7 @@ public class ColumnStatisticBuilder {
 
     private Histogram histogram;
 
-    private double originalNdv;
+    private ColumnStatistic original;
 
     public ColumnStatisticBuilder() {
     }
@@ -53,7 +53,7 @@ public class ColumnStatisticBuilder {
         this.maxExpr = columnStatistic.maxExpr;
         this.isUnknown = columnStatistic.isUnKnown;
         this.histogram = columnStatistic.histogram;
-        this.originalNdv = columnStatistic.originalNdv;
+        this.original = columnStatistic.original;
     }
 
     public ColumnStatisticBuilder setCount(double count) {
@@ -66,8 +66,8 @@ public class ColumnStatisticBuilder {
         return this;
     }
 
-    public ColumnStatisticBuilder setOriginalNdv(double originalNdv) {
-        this.originalNdv = originalNdv;
+    public ColumnStatisticBuilder setOriginal(ColumnStatistic original) {
+        this.original = original;
         return this;
     }
 
@@ -171,7 +171,11 @@ public class ColumnStatisticBuilder {
 
     public ColumnStatistic build() {
         dataSize = Math.max((count - numNulls + 1) * avgSizeByte, 0);
-        return new ColumnStatistic(count, ndv, originalNdv, avgSizeByte, numNulls,
+        if (original == null) {
+            original = new ColumnStatistic(count, ndv, null, avgSizeByte, numNulls,
+                    dataSize, minValue, maxValue, selectivity, minExpr, maxExpr, isUnknown, histogram);
+        }
+        return new ColumnStatistic(count, ndv, original, avgSizeByte, numNulls,
             dataSize, minValue, maxValue, selectivity, minExpr, maxExpr, isUnknown, histogram);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java
index 63e5d1b634..ced8e1e6a6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java
@@ -275,7 +275,7 @@ public class StatisticsRepository {
         if (ndv != null) {
             double dNdv = Double.parseDouble(ndv);
             builder.setNdv(dNdv);
-            builder.setOriginalNdv(dNdv);
+            builder.setOriginal(null);
         }
         if (nullCount != null) {
             builder.setNumNulls(Double.parseDouble(nullCount));
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/HyperGraphBuilder.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/HyperGraphBuilder.java
index a3223f8281..0458f27613 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/HyperGraphBuilder.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/HyperGraphBuilder.java
@@ -190,7 +190,7 @@ public class HyperGraphBuilder {
             int count = rowCounts.get(Integer.parseInt(scanPlan.getTable().getName()));
             for (Slot slot : scanPlan.getOutput()) {
                 slotIdToColumnStats.put(slot,
-                        new ColumnStatistic(count, count, 0, 0, 0, 0, 0,
+                        new ColumnStatistic(count, count, null, 0, 0, 0, 0,
                                 0, 0, null, null, true, null));
             }
             Statistics stats = new Statistics(count, slotIdToColumnStats);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatsDeriveResultTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatsDeriveResultTest.java
index beda41a5b9..6c157bece6 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatsDeriveResultTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatsDeriveResultTest.java
@@ -26,7 +26,7 @@ public class StatsDeriveResultTest {
     @Test
     public void testUpdateRowCountByLimit() {
         StatsDeriveResult stats = new StatsDeriveResult(100);
-        ColumnStatistic a = new ColumnStatistic(100, 10,  10, 1, 5, 10,
+        ColumnStatistic a = new ColumnStatistic(100, 10,  null, 1, 5, 10,
                 1, 100, 0.5, null, null, false, null);
         Id id = new Id(1);
         stats.addColumnStats(id, a);
diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out
index cb3a7412d1..b17960f808 100644
--- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out
+++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out
@@ -45,21 +45,21 @@ CteAnchor[cteId= ( CTEId#4=] )
 ------PhysicalTopN
 --------PhysicalProject
 ----------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END > CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END)
-------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)
---------------PhysicalProject
-----------------filter((t_w_secyear.dyear = 2002)(t_w_secyear.sale_type = 'w'))
-------------------CteConsumer[cteId= ( CTEId#4=] )
---------------PhysicalDistribute
-----------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id)
-------------------PhysicalProject
---------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2002))
-----------------------CteConsumer[cteId= ( CTEId#4=] )
-------------------PhysicalDistribute
+------------PhysicalProject
+--------------filter((t_w_firstyear.year_total > 0.00)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.dyear = 2001))
+----------------CteConsumer[cteId= ( CTEId#4=] )
+------------PhysicalDistribute
+--------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)
+----------------PhysicalProject
+------------------filter((t_w_secyear.dyear = 2002)(t_w_secyear.sale_type = 'w'))
+--------------------CteConsumer[cteId= ( CTEId#4=] )
+----------------PhysicalDistribute
+------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id)
 --------------------PhysicalProject
-----------------------filter((t_s_firstyear.dyear = 2001)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.00))
+----------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2002))
 ------------------------CteConsumer[cteId= ( CTEId#4=] )
-------------PhysicalDistribute
---------------PhysicalProject
-----------------filter((t_w_firstyear.year_total > 0.00)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.dyear = 2001))
-------------------CteConsumer[cteId= ( CTEId#4=] )
+--------------------PhysicalDistribute
+----------------------PhysicalProject
+------------------------filter((t_s_firstyear.dyear = 2001)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.00))
+--------------------------CteConsumer[cteId= ( CTEId#4=] )
 
diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out
index c3e8c94135..4c8dd7e768 100644
--- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out
+++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out
@@ -77,21 +77,21 @@ CteAnchor[cteId= ( CTEId#6=] )
 ----------------PhysicalDistribute
 ------------------PhysicalProject
 --------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_firstyear.customer_id)(CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END)
-----------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id)
-------------------------PhysicalProject
---------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2000))
-----------------------------CteConsumer[cteId= ( CTEId#6=] )
-------------------------PhysicalDistribute
---------------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_secyear.customer_id)
-----------------------------PhysicalProject
-------------------------------filter((t_c_secyear.sale_type = 'c')(t_c_secyear.dyear = 2000))
---------------------------------CteConsumer[cteId= ( CTEId#6=] )
-----------------------------PhysicalDistribute
+----------------------PhysicalProject
+------------------------filter((t_c_firstyear.year_total > 0.000000)(t_c_firstyear.dyear = 1999)(t_c_firstyear.sale_type = 'c'))
+--------------------------CteConsumer[cteId= ( CTEId#6=] )
+----------------------PhysicalDistribute
+------------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id)
+--------------------------PhysicalProject
+----------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2000))
+------------------------------CteConsumer[cteId= ( CTEId#6=] )
+--------------------------PhysicalDistribute
+----------------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_secyear.customer_id)
 ------------------------------PhysicalProject
---------------------------------filter((t_s_firstyear.year_total > 0.000000)(t_s_firstyear.dyear = 1999)(t_s_firstyear.sale_type = 's'))
+--------------------------------filter((t_c_secyear.sale_type = 'c')(t_c_secyear.dyear = 2000))
 ----------------------------------CteConsumer[cteId= ( CTEId#6=] )
-----------------------PhysicalDistribute
-------------------------PhysicalProject
---------------------------filter((t_c_firstyear.year_total > 0.000000)(t_c_firstyear.dyear = 1999)(t_c_firstyear.sale_type = 'c'))
-----------------------------CteConsumer[cteId= ( CTEId#6=] )
+------------------------------PhysicalDistribute
+--------------------------------PhysicalProject
+----------------------------------filter((t_s_firstyear.year_total > 0.000000)(t_s_firstyear.dyear = 1999)(t_s_firstyear.sale_type = 's'))
+------------------------------------CteConsumer[cteId= ( CTEId#6=] )
 
diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out
index bebc49746e..d4bfa8d3ae 100644
--- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out
+++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out
@@ -4,11 +4,11 @@ PhysicalTopN
 --PhysicalDistribute
 ----PhysicalTopN
 ------PhysicalProject
---------hashJoin[INNER_JOIN](asceding.rnk = descending.rnk)
-----------hashJoin[INNER_JOIN](i2.i_item_sk = descending.item_sk)
-------------PhysicalProject
---------------PhysicalOlapScan[item]
-------------PhysicalDistribute
+--------hashJoin[INNER_JOIN](i1.i_item_sk = asceding.item_sk)
+----------PhysicalProject
+------------PhysicalOlapScan[item]
+----------PhysicalDistribute
+------------hashJoin[INNER_JOIN](asceding.rnk = descending.rnk)
 --------------PhysicalProject
 ----------------filter((rnk < 11))
 ------------------PhysicalWindow
@@ -33,33 +33,33 @@ PhysicalTopN
 --------------------------------------------PhysicalProject
 ----------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146))
 ------------------------------------------------PhysicalOlapScan[store_sales]
-----------PhysicalDistribute
-------------hashJoin[INNER_JOIN](i1.i_item_sk = asceding.item_sk)
---------------PhysicalProject
-----------------PhysicalOlapScan[item]
 --------------PhysicalDistribute
-----------------PhysicalProject
-------------------filter((rnk < 11))
---------------------PhysicalWindow
-----------------------PhysicalQuickSort
-------------------------PhysicalDistribute
+----------------hashJoin[INNER_JOIN](i2.i_item_sk = descending.item_sk)
+------------------PhysicalProject
+--------------------PhysicalOlapScan[item]
+------------------PhysicalDistribute
+--------------------PhysicalProject
+----------------------filter((rnk < 11))
+------------------------PhysicalWindow
 --------------------------PhysicalQuickSort
-----------------------------PhysicalProject
-------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE))
---------------------------------hashAgg[GLOBAL]
-----------------------------------PhysicalDistribute
-------------------------------------hashAgg[LOCAL]
---------------------------------------PhysicalProject
-----------------------------------------filter((ss1.ss_store_sk = 146))
-------------------------------------------PhysicalOlapScan[store_sales]
---------------------------------PhysicalDistribute
-----------------------------------PhysicalAssertNumRows
+----------------------------PhysicalDistribute
+------------------------------PhysicalQuickSort
+--------------------------------PhysicalProject
+----------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE))
+------------------------------------hashAgg[GLOBAL]
+--------------------------------------PhysicalDistribute
+----------------------------------------hashAgg[LOCAL]
+------------------------------------------PhysicalProject
+--------------------------------------------filter((ss1.ss_store_sk = 146))
+----------------------------------------------PhysicalOlapScan[store_sales]
 ------------------------------------PhysicalDistribute
---------------------------------------PhysicalProject
-----------------------------------------hashAgg[GLOBAL]
-------------------------------------------PhysicalDistribute
---------------------------------------------hashAgg[LOCAL]
-----------------------------------------------PhysicalProject
-------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146))
---------------------------------------------------PhysicalOlapScan[store_sales]
+--------------------------------------PhysicalAssertNumRows
+----------------------------------------PhysicalDistribute
+------------------------------------------PhysicalProject
+--------------------------------------------hashAgg[GLOBAL]
+----------------------------------------------PhysicalDistribute
+------------------------------------------------hashAgg[LOCAL]
+--------------------------------------------------PhysicalProject
+----------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146))
+------------------------------------------------------PhysicalOlapScan[store_sales]
 
diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out
index b5e0900210..b026346094 100644
--- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out
+++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out
@@ -14,36 +14,36 @@ PhysicalProject
 ----------------------hashAgg[LOCAL]
 ------------------------PhysicalRepeat
 --------------------------PhysicalProject
-----------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk)
-------------------------------PhysicalProject
---------------------------------filter((d1.d_month_seq <= 1224)(d1.d_month_seq >= 1213))
-----------------------------------PhysicalOlapScan[date_dim]
+----------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk)
+------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk)
+--------------------------------PhysicalProject
+----------------------------------PhysicalOlapScan[store_sales]
+--------------------------------PhysicalDistribute
+----------------------------------PhysicalProject
+------------------------------------filter((d1.d_month_seq <= 1224)(d1.d_month_seq >= 1213))
+--------------------------------------PhysicalOlapScan[date_dim]
 ------------------------------PhysicalDistribute
---------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk)
+--------------------------------hashJoin[LEFT_SEMI_JOIN](store.s_state = tmp1.s_state)
 ----------------------------------PhysicalProject
-------------------------------------PhysicalOlapScan[store_sales]
+------------------------------------PhysicalOlapScan[store]
 ----------------------------------PhysicalDistribute
-------------------------------------hashJoin[LEFT_SEMI_JOIN](store.s_state = tmp1.s_state)
---------------------------------------PhysicalProject
-----------------------------------------PhysicalOlapScan[store]
---------------------------------------PhysicalDistribute
-----------------------------------------PhysicalProject
-------------------------------------------filter((ranking <= 5))
---------------------------------------------PhysicalWindow
-----------------------------------------------PhysicalQuickSort
-------------------------------------------------hashAgg[GLOBAL]
---------------------------------------------------PhysicalDistribute
-----------------------------------------------------hashAgg[LOCAL]
-------------------------------------------------------PhysicalProject
---------------------------------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk)
-----------------------------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk)
-------------------------------------------------------------PhysicalProject
---------------------------------------------------------------PhysicalOlapScan[store_sales]
-------------------------------------------------------------PhysicalDistribute
---------------------------------------------------------------PhysicalProject
-----------------------------------------------------------------filter((date_dim.d_month_seq >= 1213)(date_dim.d_month_seq <= 1224))
-------------------------------------------------------------------PhysicalOlapScan[date_dim]
-----------------------------------------------------------PhysicalDistribute
-------------------------------------------------------------PhysicalProject
---------------------------------------------------------------PhysicalOlapScan[store]
+------------------------------------PhysicalProject
+--------------------------------------filter((ranking <= 5))
+----------------------------------------PhysicalWindow
+------------------------------------------PhysicalQuickSort
+--------------------------------------------hashAgg[GLOBAL]
+----------------------------------------------PhysicalDistribute
+------------------------------------------------hashAgg[LOCAL]
+--------------------------------------------------PhysicalProject
+----------------------------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk)
+------------------------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk)
+--------------------------------------------------------PhysicalProject
+----------------------------------------------------------PhysicalOlapScan[store_sales]
+--------------------------------------------------------PhysicalDistribute
+----------------------------------------------------------PhysicalProject
+------------------------------------------------------------filter((date_dim.d_month_seq >= 1213)(date_dim.d_month_seq <= 1224))
+--------------------------------------------------------------PhysicalOlapScan[date_dim]
+------------------------------------------------------PhysicalDistribute
+--------------------------------------------------------PhysicalProject
+----------------------------------------------------------PhysicalOlapScan[store]
 
diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out
index 45d668c27f..633c98d867 100644
--- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out
+++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out
@@ -44,22 +44,21 @@ CteAnchor[cteId= ( CTEId#4=] )
 ------PhysicalTopN
 --------PhysicalProject
 ----------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END > CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END)
-------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)
---------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id)
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.0))
-----------------------CteConsumer[cteId= ( CTEId#4=] )
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.year = 2000))
-----------------------CteConsumer[cteId= ( CTEId#4=] )
---------------PhysicalDistribute
+------------PhysicalProject
+--------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0.0)(t_w_firstyear.sale_type = 'w'))
+----------------CteConsumer[cteId= ( CTEId#4=] )
+------------PhysicalDistribute
+--------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)
 ----------------PhysicalProject
 ------------------filter((t_w_secyear.year = 2000)(t_w_secyear.sale_type = 'w'))
 --------------------CteConsumer[cteId= ( CTEId#4=] )
-------------PhysicalDistribute
---------------PhysicalProject
-----------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0.0)(t_w_firstyear.sale_type = 'w'))
-------------------CteConsumer[cteId= ( CTEId#4=] )
+----------------PhysicalDistribute
+------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id)
+--------------------PhysicalProject
+----------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.0))
+------------------------CteConsumer[cteId= ( CTEId#4=] )
+--------------------PhysicalDistribute
+----------------------PhysicalProject
+------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.year = 2000))
+--------------------------CteConsumer[cteId= ( CTEId#4=] )
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 31/36: [improvement](column reader) lazy load indices (#20456)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit f4d0089888fc21901725b0dbe2ed933620a84d6b
Author: TengJianPing <18...@users.noreply.github.com>
AuthorDate: Tue Jun 6 16:36:06 2023 +0800

    [improvement](column reader) lazy load indices (#20456)
    
    Currently when reading column data, all types of indice are read even if they are not actually used, this PR implements lazy load of indices.
---
 be/src/olap/rowset/segment_v2/column_reader.cpp | 42 ++++++++++++++++---------
 be/src/olap/rowset/segment_v2/column_reader.h   | 34 ++++++++------------
 2 files changed, 40 insertions(+), 36 deletions(-)

diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp
index 2469fa5924..609c9fdc28 100644
--- a/be/src/olap/rowset/segment_v2/column_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/column_reader.cpp
@@ -179,7 +179,8 @@ ColumnReader::ColumnReader(const ColumnReaderOptions& opts, const ColumnMetaPB&
           _opts(opts),
           _num_rows(num_rows),
           _file_reader(std::move(file_reader)),
-          _dict_encoding_type(UNKNOWN_DICT_ENCODING) {}
+          _dict_encoding_type(UNKNOWN_DICT_ENCODING),
+          _use_index_page_cache(!config::disable_storage_page_cache) {}
 
 ColumnReader::~ColumnReader() = default;
 
@@ -195,15 +196,20 @@ Status ColumnReader::init() {
         switch (index_meta.type()) {
         case ORDINAL_INDEX:
             _ordinal_index_meta = &index_meta.ordinal_index();
+            _ordinal_index.reset(
+                    new OrdinalIndexReader(_file_reader, _ordinal_index_meta, _num_rows));
             break;
         case ZONE_MAP_INDEX:
             _zone_map_index_meta = &index_meta.zone_map_index();
+            _zone_map_index.reset(new ZoneMapIndexReader(_file_reader, _zone_map_index_meta));
             break;
         case BITMAP_INDEX:
             _bitmap_index_meta = &index_meta.bitmap_index();
+            _bitmap_index.reset(new BitmapIndexReader(_file_reader, _bitmap_index_meta));
             break;
         case BLOOM_FILTER_INDEX:
             _bf_index_meta = &index_meta.bloom_filter_index();
+            _bloom_filter_index.reset(new BloomFilterIndexReader(_file_reader, _bf_index_meta));
             break;
         default:
             return Status::Corruption("Bad file {}: invalid column index type {}",
@@ -220,7 +226,7 @@ Status ColumnReader::init() {
 }
 
 Status ColumnReader::new_bitmap_index_iterator(BitmapIndexIterator** iterator) {
-    RETURN_IF_ERROR(_ensure_index_loaded());
+    RETURN_IF_ERROR(_load_bitmap_index(_use_index_page_cache, _opts.kept_in_memory));
     RETURN_IF_ERROR(_bitmap_index->new_iterator(iterator));
     return Status::OK();
 }
@@ -261,8 +267,6 @@ Status ColumnReader::read_page(const ColumnIteratorOptions& iter_opts, const Pag
 Status ColumnReader::get_row_ranges_by_zone_map(
         const AndBlockColumnPredicate* col_predicates,
         const std::vector<const ColumnPredicate*>* delete_predicates, RowRanges* row_ranges) {
-    RETURN_IF_ERROR(_ensure_index_loaded());
-
     std::vector<uint32_t> page_indexes;
     RETURN_IF_ERROR(_get_filtered_pages(col_predicates, delete_predicates, &page_indexes));
     RETURN_IF_ERROR(_calculate_row_ranges(page_indexes, row_ranges));
@@ -374,6 +378,8 @@ Status ColumnReader::_get_filtered_pages(
         const AndBlockColumnPredicate* col_predicates,
         const std::vector<const ColumnPredicate*>* delete_predicates,
         std::vector<uint32_t>* page_indexes) {
+    RETURN_IF_ERROR(_load_zone_map_index(_use_index_page_cache, _opts.kept_in_memory));
+
     FieldType type = _type_info->type();
     const std::vector<ZoneMapPB>& zone_maps = _zone_map_index->page_zone_maps();
     int32_t page_size = _zone_map_index->num_pages();
@@ -412,6 +418,7 @@ Status ColumnReader::_get_filtered_pages(
 Status ColumnReader::_calculate_row_ranges(const std::vector<uint32_t>& page_indexes,
                                            RowRanges* row_ranges) {
     row_ranges->clear();
+    RETURN_IF_ERROR(_load_ordinal_index(_use_index_page_cache, _opts.kept_in_memory));
     for (auto i : page_indexes) {
         ordinal_t page_first_id = _ordinal_index->get_first_ordinal(i);
         ordinal_t page_last_id = _ordinal_index->get_last_ordinal(i);
@@ -423,7 +430,8 @@ Status ColumnReader::_calculate_row_ranges(const std::vector<uint32_t>& page_ind
 
 Status ColumnReader::get_row_ranges_by_bloom_filter(const AndBlockColumnPredicate* col_predicates,
                                                     RowRanges* row_ranges) {
-    RETURN_IF_ERROR(_ensure_index_loaded());
+    RETURN_IF_ERROR(_load_ordinal_index(_use_index_page_cache, _opts.kept_in_memory));
+    RETURN_IF_ERROR(_load_bloom_filter_index(_use_index_page_cache, _opts.kept_in_memory));
     RowRanges bf_row_ranges;
     std::unique_ptr<BloomFilterIndexIterator> bf_iter;
     RETURN_IF_ERROR(_bloom_filter_index->new_iterator(&bf_iter));
@@ -455,22 +463,25 @@ Status ColumnReader::get_row_ranges_by_bloom_filter(const AndBlockColumnPredicat
 
 Status ColumnReader::_load_ordinal_index(bool use_page_cache, bool kept_in_memory) {
     DCHECK(_ordinal_index_meta != nullptr);
-    _ordinal_index.reset(new OrdinalIndexReader(_file_reader, _ordinal_index_meta, _num_rows));
-    return _ordinal_index->load(use_page_cache, kept_in_memory);
+    return _load_ordinal_index_once.call([this, use_page_cache, kept_in_memory] {
+        return _ordinal_index->load(use_page_cache, kept_in_memory);
+    });
 }
 
 Status ColumnReader::_load_zone_map_index(bool use_page_cache, bool kept_in_memory) {
     if (_zone_map_index_meta != nullptr) {
-        _zone_map_index.reset(new ZoneMapIndexReader(_file_reader, _zone_map_index_meta));
-        return _zone_map_index->load(use_page_cache, kept_in_memory);
+        return _load_zone_map_index_once.call([this, use_page_cache, kept_in_memory] {
+            return _zone_map_index->load(use_page_cache, kept_in_memory);
+        });
     }
     return Status::OK();
 }
 
 Status ColumnReader::_load_bitmap_index(bool use_page_cache, bool kept_in_memory) {
     if (_bitmap_index_meta != nullptr) {
-        _bitmap_index.reset(new BitmapIndexReader(_file_reader, _bitmap_index_meta));
-        return _bitmap_index->load(use_page_cache, kept_in_memory);
+        return _load_bitmap_index_once.call([this, use_page_cache, kept_in_memory] {
+            return _bitmap_index->load(use_page_cache, kept_in_memory);
+        });
     }
     return Status::OK();
 }
@@ -513,14 +524,15 @@ Status ColumnReader::_load_inverted_index_index(const TabletIndex* index_meta) {
 
 Status ColumnReader::_load_bloom_filter_index(bool use_page_cache, bool kept_in_memory) {
     if (_bf_index_meta != nullptr) {
-        _bloom_filter_index.reset(new BloomFilterIndexReader(_file_reader, _bf_index_meta));
-        return _bloom_filter_index->load(use_page_cache, kept_in_memory);
+        return _load_bloom_filter_index_once.call([this, use_page_cache, kept_in_memory] {
+            return _bloom_filter_index->load(use_page_cache, kept_in_memory);
+        });
     }
     return Status::OK();
 }
 
 Status ColumnReader::seek_to_first(OrdinalPageIndexIterator* iter) {
-    RETURN_IF_ERROR(_ensure_index_loaded());
+    RETURN_IF_ERROR(_load_ordinal_index(_use_index_page_cache, _opts.kept_in_memory));
     *iter = _ordinal_index->begin();
     if (!iter->valid()) {
         return Status::NotFound("Failed to seek to first rowid");
@@ -529,7 +541,7 @@ Status ColumnReader::seek_to_first(OrdinalPageIndexIterator* iter) {
 }
 
 Status ColumnReader::seek_at_or_before(ordinal_t ordinal, OrdinalPageIndexIterator* iter) {
-    RETURN_IF_ERROR(_ensure_index_loaded());
+    RETURN_IF_ERROR(_load_ordinal_index(_use_index_page_cache, _opts.kept_in_memory));
     *iter = _ordinal_index->seek_at_or_before(ordinal);
     if (!iter->valid()) {
         return Status::NotFound("Failed to seek to ordinal {}, ", ordinal);
diff --git a/be/src/olap/rowset/segment_v2/column_reader.h b/be/src/olap/rowset/segment_v2/column_reader.h
index 1fe87acb16..18f5aad760 100644
--- a/be/src/olap/rowset/segment_v2/column_reader.h
+++ b/be/src/olap/rowset/segment_v2/column_reader.h
@@ -173,26 +173,13 @@ public:
 
     DictEncodingType get_dict_encoding_type() { return _dict_encoding_type; }
 
-    void disable_index_meta_cache() { _index_meta_use_page_cache = false; }
+    void disable_index_meta_cache() { _use_index_page_cache = false; }
 
 private:
     ColumnReader(const ColumnReaderOptions& opts, const ColumnMetaPB& meta, uint64_t num_rows,
                  io::FileReaderSPtr file_reader);
     Status init();
 
-    // Read and load necessary column indexes into memory if it hasn't been loaded.
-    // May be called multiple times, subsequent calls will no op.
-    Status _ensure_index_loaded() {
-        return _load_index_once.call([this] {
-            bool use_page_cache = !config::disable_storage_page_cache && _index_meta_use_page_cache;
-            RETURN_IF_ERROR(_load_zone_map_index(use_page_cache, _opts.kept_in_memory));
-            RETURN_IF_ERROR(_load_ordinal_index(use_page_cache, _opts.kept_in_memory));
-            RETURN_IF_ERROR(_load_bitmap_index(use_page_cache, _opts.kept_in_memory));
-            RETURN_IF_ERROR(_load_bloom_filter_index(use_page_cache, _opts.kept_in_memory));
-            return Status::OK();
-        });
-    }
-
     // Read column inverted indexes into memory
     // May be called multiple times, subsequent calls will no op.
     Status _ensure_inverted_index_loaded(const TabletIndex* index_meta) {
@@ -201,11 +188,11 @@ private:
         return Status::OK();
     }
 
-    Status _load_zone_map_index(bool use_page_cache, bool kept_in_memory);
-    Status _load_ordinal_index(bool use_page_cache, bool kept_in_memory);
-    Status _load_bitmap_index(bool use_page_cache, bool kept_in_memory);
-    Status _load_inverted_index_index(const TabletIndex* index_meta);
-    Status _load_bloom_filter_index(bool use_page_cache, bool kept_in_memory);
+    [[nodiscard]] Status _load_zone_map_index(bool use_page_cache, bool kept_in_memory);
+    [[nodiscard]] Status _load_ordinal_index(bool use_page_cache, bool kept_in_memory);
+    [[nodiscard]] Status _load_bitmap_index(bool use_page_cache, bool kept_in_memory);
+    [[nodiscard]] Status _load_inverted_index_index(const TabletIndex* index_meta);
+    [[nodiscard]] Status _load_bloom_filter_index(bool use_page_cache, bool kept_in_memory);
 
     bool _zone_map_match_condition(const ZoneMapPB& zone_map, WrapperField* min_value_container,
                                    WrapperField* max_value_container,
@@ -237,20 +224,25 @@ private:
     const EncodingInfo* _encoding_info =
             nullptr; // initialized in init(), used for create PageDecoder
 
+    bool _use_index_page_cache;
+
     // meta for various column indexes (null if the index is absent)
-    bool _index_meta_use_page_cache = true;
     const ZoneMapIndexPB* _zone_map_index_meta = nullptr;
     const OrdinalIndexPB* _ordinal_index_meta = nullptr;
     const BitmapIndexPB* _bitmap_index_meta = nullptr;
     const BloomFilterIndexPB* _bf_index_meta = nullptr;
 
-    DorisCallOnce<Status> _load_index_once;
     mutable std::mutex _load_index_lock;
     std::unique_ptr<ZoneMapIndexReader> _zone_map_index;
     std::unique_ptr<OrdinalIndexReader> _ordinal_index;
     std::unique_ptr<BitmapIndexReader> _bitmap_index;
     std::unique_ptr<InvertedIndexReader> _inverted_index;
     std::unique_ptr<BloomFilterIndexReader> _bloom_filter_index;
+    DorisCallOnce<Status> _load_zone_map_index_once;
+    DorisCallOnce<Status> _load_ordinal_index_once;
+    DorisCallOnce<Status> _load_bitmap_index_once;
+    DorisCallOnce<Status> _load_bloom_filter_index_once;
+    DorisCallOnce<Status> _load_inverted_index_once;
 
     std::vector<std::unique_ptr<ColumnReader>> _sub_readers;
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 07/36: [Optimize](function) Optimize locate function by compare across strings (#20290)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 9e44e20be570bd9780f900bf739841b9a57804ba
Author: ZhangYu0123 <67...@users.noreply.github.com>
AuthorDate: Mon Jun 5 12:43:14 2023 +0800

    [Optimize](function) Optimize locate function by compare across strings (#20290)
    
    Optimize locate function by compare across strings. about 90% speed up test by sum()
---
 be/src/vec/functions/function_string.cpp           | 44 +++++++++++++++-------
 .../string_functions/test_string_function.out      | 24 +++++++++---
 .../string_functions/test_string_function.groovy   | 16 +++++---
 3 files changed, 59 insertions(+), 25 deletions(-)

diff --git a/be/src/vec/functions/function_string.cpp b/be/src/vec/functions/function_string.cpp
index 670644bba4..dbfbff8800 100644
--- a/be/src/vec/functions/function_string.cpp
+++ b/be/src/vec/functions/function_string.cpp
@@ -205,28 +205,46 @@ struct StringInStrImpl {
         res.resize(size);
 
         if (rdata.size == 0) {
-            for (int i = 0; i < size; ++i) {
-                res[i] = 1;
-            }
+            std::fill(res.begin(), res.end(), 1);
             return Status::OK();
         }
 
+        const UInt8* begin = ldata.data();
+        const UInt8* end = begin + ldata.size();
+        const UInt8* pos = begin;
+
+        /// Current index in the array of strings.
+        size_t i = 0;
+        std::fill(res.begin(), res.end(), 0);
+
         StringRef rstr_ref(rdata.data, rdata.size);
         StringSearch search(&rstr_ref);
 
-        for (int i = 0; i < size; ++i) {
-            const char* l_raw_str = reinterpret_cast<const char*>(&ldata[loffsets[i - 1]]);
-            int l_str_size = loffsets[i] - loffsets[i - 1];
+        while (pos < end) {
+            // search return matched substring start offset
+            pos = (UInt8*)search.search((char*)pos, end - pos);
+            if (pos >= end) {
+                break;
+            }
 
-            StringRef lstr_ref(l_raw_str, l_str_size);
+            /// Determine which index it refers to.
+            /// begin + value_offsets[i] is the start offset of string at i+1
+            while (begin + loffsets[i] < pos) {
+                ++i;
+            }
 
-            // Hive returns positions starting from 1.
-            int loc = search.search(&lstr_ref);
-            if (loc > 0) {
-                size_t len = std::min(lstr_ref.size, (size_t)loc);
-                loc = simd::VStringFunctions::get_char_len(lstr_ref.data, len);
+            /// We check that the entry does not pass through the boundaries of strings.
+            if (pos + rdata.size <= begin + loffsets[i]) {
+                int loc = pos - begin - loffsets[i - 1];
+                int l_str_size = loffsets[i] - loffsets[i - 1];
+                size_t len = std::min(l_str_size, loc);
+                loc = simd::VStringFunctions::get_char_len((char*)(begin + loffsets[i - 1]), len);
+                res[i] = loc + 1;
             }
-            res[i] = loc + 1;
+
+            // move to next string offset
+            pos = begin + loffsets[i];
+            ++i;
         }
 
         return Status::OK();
diff --git a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out
index 587319531e..b51fb32d61 100644
--- a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out
+++ b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out
@@ -122,18 +122,24 @@ A
 -- !sql --
 AB
 
--- !sql --
+-- !sql_instr --
 2
 
--- !sql --
+-- !sql_instr --
 0
 
--- !sql --
+-- !sql_instr --
 \N
 
--- !sql --
+-- !sql_instr --
 \N
 
+-- !sql_instr --
+1
+
+-- !sql_instr --
+5
+
 -- !sql --
 abc123
 
@@ -152,12 +158,18 @@ doris
 -- !sql --
 3
 
--- !sql --
+-- !sql_locate --
 4
 
--- !sql --
+-- !sql_locate --
 0
 
+-- !sql_locate --
+1
+
+-- !sql_locate --
+5
+
 -- !sql --
 xyxhi
 
diff --git a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy
index ae33e448e7..6a06992322 100644
--- a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy
+++ b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy
@@ -69,10 +69,12 @@ suite("test_string_function") {
     qt_sql "select unhex('41');"
     qt_sql "select unhex('4142');"
 
-    qt_sql "select instr(\"abc\", \"b\");"
-    qt_sql "select instr(\"abc\", \"d\");"
-    qt_sql "select instr(\"abc\", null);"
-    qt_sql "select instr(null, \"a\");"
+    qt_sql_instr "select instr(\"abc\", \"b\");"
+    qt_sql_instr "select instr(\"abc\", \"d\");"
+    qt_sql_instr "select instr(\"abc\", null);"
+    qt_sql_instr "select instr(null, \"a\");"
+    qt_sql_instr "SELECT instr('foobar', '');"
+    qt_sql_instr "SELECT instr('上海天津北京杭州', '北京');"
 
     qt_sql "SELECT lcase(\"AbC123\");"
     qt_sql "SELECT lower(\"AbC123\");"
@@ -84,8 +86,10 @@ suite("test_string_function") {
 
     qt_sql "select length(\"abc\");"
 
-    qt_sql "SELECT LOCATE('bar', 'foobarbar');"
-    qt_sql "SELECT LOCATE('xbar', 'foobar');"
+    qt_sql_locate "SELECT LOCATE('bar', 'foobarbar');"
+    qt_sql_locate "SELECT LOCATE('xbar', 'foobar');"
+    qt_sql_locate "SELECT LOCATE('', 'foobar');"
+    qt_sql_locate "SELECT LOCATE('北京', '上海天津北京杭州');"
 
     qt_sql "SELECT lpad(\"hi\", 5, \"xy\");"
     qt_sql "SELECT lpad(\"hi\", 1, \"xy\");"


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 36/36: [fix](log) publish version log is printed too frequently (#20507)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 5c33dd7a2ccfcc537bc0d1e64b3e826e9bfdc103
Author: zhannngchen <48...@users.noreply.github.com>
AuthorDate: Tue Jun 6 20:34:38 2023 +0800

    [fix](log) publish version log is printed too frequently (#20507)
---
 be/src/agent/task_worker_pool.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp
index c3ec02f08d..f596603c35 100644
--- a/be/src/agent/task_worker_pool.cpp
+++ b/be/src/agent/task_worker_pool.cpp
@@ -1455,8 +1455,8 @@ void PublishVersionTaskPool::_publish_version_worker_thread_callback() {
                     _tasks.push_back(agent_task_req);
                     _worker_thread_condition_variable.notify_one();
                 }
-                LOG(INFO) << "wait for previous publish version task to be done"
-                          << "transaction_id: " << publish_version_req.transaction_id;
+                LOG_EVERY_SECOND(INFO) << "wait for previous publish version task to be done"
+                                       << "transaction_id: " << publish_version_req.transaction_id;
                 break;
             } else {
                 LOG_WARNING("failed to publish version")


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 15/36: [fix](stats) skip forbid_unknown_col_stats check for invisible column and internal db (#20362)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 98be895a4c655dc5e50825e701e35cd9241b49f1
Author: minghong <en...@gmail.com>
AuthorDate: Tue Jun 6 19:07:33 2023 +0800

    [fix](stats) skip forbid_unknown_col_stats check for invisible column and internal db (#20362)
    
    1. skip forbidUnknownColStats check for in-visible columns
    2. use columsStatistics.isUnknown to tell if this stats is unknown
    3. skip unknown stats check for internal schema
---
 .../apache/doris/analysis/ShowColumnStatsStmt.java |  2 +-
 .../doris/catalog/external/HMSExternalTable.java   |  2 +-
 .../doris/nereids/stats/ExpressionEstimation.java  |  4 +--
 .../doris/nereids/stats/FilterEstimation.java      |  2 +-
 .../doris/nereids/stats/StatsCalculator.java       | 36 +++++++++++++++++++---
 .../doris/statistics/StatisticConstants.java       | 14 +++++++++
 .../apache/doris/statistics/StatisticsCache.java   |  2 +-
 .../doris/nereids/stats/FilterEstimationTest.java  |  6 ++--
 .../org/apache/doris/statistics/CacheTest.java     |  6 ++--
 9 files changed, 57 insertions(+), 17 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java
index fd67316df4..8e783c1961 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java
@@ -136,7 +136,7 @@ public class ShowColumnStatsStmt extends ShowStmt {
     public ShowResultSet constructResultSet(List<Pair<String, ColumnStatistic>> columnStatistics) {
         List<List<String>> result = Lists.newArrayList();
         columnStatistics.forEach(p -> {
-            if (p.second == ColumnStatistic.UNKNOWN) {
+            if (p.second.isUnKnown) {
                 return;
             }
             List<String> row = Lists.newArrayList();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java
index af032d4ae2..7a8fdaf6fb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java
@@ -386,7 +386,7 @@ public class HMSExternalTable extends ExternalTable {
         ColumnStatistic cache = Config.enable_stats
                 ? Env.getCurrentEnv().getStatisticsCache().getColumnStatistics(id, "")
                 : ColumnStatistic.UNKNOWN;
-        if (cache == ColumnStatistic.UNKNOWN) {
+        if (cache.isUnKnown) {
             return 1;
         } else {
             return (long) cache.count;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java
index d5449987cf..eca7511684 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/ExpressionEstimation.java
@@ -252,7 +252,7 @@ public class ExpressionEstimation extends ExpressionVisitor<ColumnStatistic, Sta
     public ColumnStatistic visitMin(Min min, Statistics context) {
         Expression child = min.child();
         ColumnStatistic columnStat = child.accept(this, context);
-        if (columnStat == ColumnStatistic.UNKNOWN) {
+        if (columnStat.isUnKnown) {
             return ColumnStatistic.UNKNOWN;
         }
         /*
@@ -270,7 +270,7 @@ public class ExpressionEstimation extends ExpressionVisitor<ColumnStatistic, Sta
     public ColumnStatistic visitMax(Max max, Statistics context) {
         Expression child = max.child();
         ColumnStatistic columnStat = child.accept(this, context);
-        if (columnStat == ColumnStatistic.UNKNOWN) {
+        if (columnStat.isUnKnown) {
             return ColumnStatistic.UNKNOWN;
         }
         /*
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java
index d7ae653798..acf072fb82 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java
@@ -197,7 +197,7 @@ public class FilterEstimation extends ExpressionVisitor<Statistics, EstimationCo
 
     private Statistics calculateWhenLiteralRight(ComparisonPredicate cp,
             ColumnStatistic statsForLeft, ColumnStatistic statsForRight, EstimationContext context) {
-        if (statsForLeft == ColumnStatistic.UNKNOWN) {
+        if (statsForLeft.isUnKnown) {
             return context.statistics.withSel(DEFAULT_INEQUALITY_COEFFICIENT);
         }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
index c166bd8d0b..77499198f2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
@@ -18,6 +18,8 @@
 package org.apache.doris.nereids.stats;
 
 import org.apache.doris.catalog.Env;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.SchemaTable;
 import org.apache.doris.catalog.TableIf;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.Pair;
@@ -109,6 +111,7 @@ import org.apache.doris.nereids.types.DataType;
 import org.apache.doris.statistics.ColumnStatistic;
 import org.apache.doris.statistics.ColumnStatisticBuilder;
 import org.apache.doris.statistics.Histogram;
+import org.apache.doris.statistics.StatisticConstants;
 import org.apache.doris.statistics.StatisticRange;
 import org.apache.doris.statistics.Statistics;
 import org.apache.doris.statistics.StatisticsBuilder;
@@ -558,12 +561,21 @@ public class StatsCalculator extends DefaultPlanVisitor<Statistics, Void> {
                         .setAvgSizeByte(slotReference.getColumn().get().getType().getSlotSize())
                         .build();
             }
-            if (cache == ColumnStatistic.UNKNOWN && !colName.equals("__DORIS_DELETE_SIGN__")) {
-                if (forbidUnknownColStats) {
+            if (cache.isUnKnown) {
+                if (forbidUnknownColStats && !ignoreUnknownColStatsCheck(table, slotReference)) {
                     if (StatisticsUtil.statsTblAvailable()) {
-                        throw new AnalysisException("column stats for " + colName
-                                + " is unknown,"
-                                + " `set forbid_unknown_col_stats = false` to execute sql with unknown stats");
+                        throw new AnalysisException(String.format("Found unknown stats for column:%s.%s.\n"
+                                + "It may caused by:\n"
+                                + "\n"
+                                + "1. This column never got analyzed\n"
+                                + "2. This table is empty\n"
+                                + "3. Stats load failed caused by unstable of backends,"
+                                + "and FE cached the unknown stats by default in this scenario\n"
+                                + "4. There is a bug, please report it to Doris community\n"
+                                + "\n"
+                                + "If an unknown stats for this column is tolerable,"
+                                + "you could set session variable `forbid_unknown_col_stats` to false to make planner"
+                                + " ignore this error and keep planning.", table.getName(), colName));
                     } else {
                         throw new AnalysisException("BE is not available!");
                     }
@@ -967,4 +979,18 @@ public class StatsCalculator extends DefaultPlanVisitor<Statistics, Void> {
             PhysicalCTEAnchor<? extends Plan, ? extends Plan> cteAnchor, Void context) {
         return groupExpression.childStatistics(1);
     }
+
+    private boolean ignoreUnknownColStatsCheck(TableIf tableIf, SlotReference slot) {
+        if (tableIf instanceof SchemaTable) {
+            return true;
+        }
+        if (tableIf instanceof OlapTable) {
+            OlapTable olapTable = (OlapTable) tableIf;
+            return StatisticConstants.STATISTICS_DB_BLACK_LIST.contains(olapTable.getQualifiedDbName());
+        }
+        if (slot.getColumn().isPresent() && slot.getColumn().get().isVisible()) {
+            return true;
+        }
+        return false;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java
index 93072c750f..89167d64f2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java
@@ -17,6 +17,12 @@
 
 package org.apache.doris.statistics;
 
+import org.apache.doris.cluster.ClusterNamespace;
+import org.apache.doris.common.FeConstants;
+import org.apache.doris.system.SystemInfoService;
+
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 public class StatisticConstants {
@@ -74,4 +80,12 @@ public class StatisticConstants {
 
     public static final int ANALYZE_MANAGER_INTERVAL_IN_SECS = 60;
 
+    public static List<String> STATISTICS_DB_BLACK_LIST = new ArrayList<>();
+
+    static {
+        STATISTICS_DB_BLACK_LIST.add(SystemInfoService.DEFAULT_CLUSTER
+                + ClusterNamespace.CLUSTER_DELIMITER + FeConstants.INTERNAL_DB_NAME);
+        STATISTICS_DB_BLACK_LIST.add(SystemInfoService.DEFAULT_CLUSTER
+                + ClusterNamespace.CLUSTER_DELIMITER + "information_schema");
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java
index f92ec87841..1cf6e876ef 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCache.java
@@ -204,7 +204,7 @@ public class StatisticsCache {
                         return Optional.of(c);
                     }
                 };
-                if (c == ColumnStatistic.UNKNOWN) {
+                if (c.isUnKnown) {
                     continue;
                 }
                 columnStatisticsCache.put(k, f);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java
index 9452eb6ff8..1fe5e5b0a0 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java
@@ -74,11 +74,11 @@ class FilterEstimationTest {
         FilterEstimation filterEstimation = new FilterEstimation();
         Statistics expected = filterEstimation.estimate(or, stat);
         Assertions.assertTrue(
-                Precision.equals(expected.getRowCount(), 550,
+                Precision.equals(expected.getRowCount(), 750,
                          0.01));
     }
 
-    // a > 500 or b < 100
+    // a > 500 and b < 100
     // b isNaN
     @Test
     public void testAndNaN() {
@@ -103,7 +103,7 @@ class FilterEstimationTest {
         FilterEstimation filterEstimation = new FilterEstimation();
         Statistics expected = filterEstimation.estimate(and, stat);
         Assertions.assertTrue(
-                Precision.equals(expected.getRowCount(), 50,
+                Precision.equals(expected.getRowCount(), 250,
                         0.01));
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/CacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/CacheTest.java
index 5a92bbdfbe..991167e973 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/statistics/CacheTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/CacheTest.java
@@ -62,10 +62,10 @@ public class CacheTest extends TestWithFeService {
         };
         StatisticsCache statisticsCache = new StatisticsCache();
         ColumnStatistic c = statisticsCache.getColumnStatistics(1, "col");
-        Assertions.assertEquals(c, ColumnStatistic.UNKNOWN);
+        Assertions.assertTrue(c.isUnKnown);
         Thread.sleep(100);
         c = statisticsCache.getColumnStatistics(1, "col");
-        Assertions.assertEquals(c, ColumnStatistic.UNKNOWN);
+        Assertions.assertTrue(c.isUnKnown);
     }
 
     @Test
@@ -126,7 +126,7 @@ public class CacheTest extends TestWithFeService {
         };
         StatisticsCache statisticsCache = new StatisticsCache();
         ColumnStatistic columnStatistic = statisticsCache.getColumnStatistics(0, "col");
-        Assertions.assertEquals(ColumnStatistic.UNKNOWN, columnStatistic);
+        Assertions.assertTrue(columnStatistic.isUnKnown);
         Thread.sleep(1000);
         columnStatistic = statisticsCache.getColumnStatistics(0, "col");
         Assertions.assertEquals(1, columnStatistic.count);


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 32/36: [enhancement](profile) add build get child next time (#20460)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 43f6dbab248fd2f4f87aa4eebc665e9573bcb482
Author: yiguolei <67...@qq.com>
AuthorDate: Tue Jun 6 08:55:19 2023 +0800

    [enhancement](profile) add build get child next time (#20460)
    
    Currently, build time not include child(1)->get next time, it is very confusing during shared hash table scenario. So that I add a profile.
    
    ---------
    
    Co-authored-by: yiguolei <yi...@gmail.com>
---
 be/src/vec/exec/join/vhash_join_node.cpp | 24 ++++++++++++++----------
 be/src/vec/exec/join/vjoin_node_base.h   |  1 +
 2 files changed, 15 insertions(+), 10 deletions(-)

diff --git a/be/src/vec/exec/join/vhash_join_node.cpp b/be/src/vec/exec/join/vhash_join_node.cpp
index cd6bea45e9..670953252d 100644
--- a/be/src/vec/exec/join/vhash_join_node.cpp
+++ b/be/src/vec/exec/join/vhash_join_node.cpp
@@ -434,6 +434,7 @@ Status HashJoinNode::prepare(RuntimeState* state) {
     // Build phase
     _build_phase_profile = runtime_profile()->create_child("BuildPhase", true, true);
     runtime_profile()->add_child(_build_phase_profile, false, nullptr);
+    _build_get_next_timer = ADD_TIMER(_build_phase_profile, "BuildGetNextTime");
     _build_timer = ADD_TIMER(_build_phase_profile, "BuildTime");
     _build_table_timer = ADD_TIMER(_build_phase_profile, "BuildTableTime");
     _build_side_merge_block_timer = ADD_TIMER(_build_phase_profile, "BuildSideMergeBlockTime");
@@ -780,8 +781,10 @@ void HashJoinNode::release_resource(RuntimeState* state) {
 }
 
 Status HashJoinNode::_materialize_build_side(RuntimeState* state) {
-    RETURN_IF_ERROR(child(1)->open(state));
-
+    {
+        SCOPED_TIMER(_build_get_next_timer);
+        RETURN_IF_ERROR(child(1)->open(state));
+    }
     if (_should_build_hash_table) {
         bool eos = false;
         Block block;
@@ -790,14 +793,15 @@ Status HashJoinNode::_materialize_build_side(RuntimeState* state) {
         while (!eos && !_short_circuit_for_null_in_probe_side) {
             block.clear_column_data();
             RETURN_IF_CANCELLED(state);
-
-            RETURN_IF_ERROR(child(1)->get_next_after_projects(
-                    state, &block, &eos,
-                    std::bind((Status(ExecNode::*)(RuntimeState*, vectorized::Block*, bool*)) &
-                                      ExecNode::get_next,
-                              _children[1], std::placeholders::_1, std::placeholders::_2,
-                              std::placeholders::_3)));
-
+            {
+                SCOPED_TIMER(_build_get_next_timer);
+                RETURN_IF_ERROR(child(1)->get_next_after_projects(
+                        state, &block, &eos,
+                        std::bind((Status(ExecNode::*)(RuntimeState*, vectorized::Block*, bool*)) &
+                                          ExecNode::get_next,
+                                  _children[1], std::placeholders::_1, std::placeholders::_2,
+                                  std::placeholders::_3)));
+            }
             RETURN_IF_ERROR(sink(state, &block, eos));
         }
         RETURN_IF_ERROR(child(1)->close(state));
diff --git a/be/src/vec/exec/join/vjoin_node_base.h b/be/src/vec/exec/join/vjoin_node_base.h
index 6972357083..757670b31e 100644
--- a/be/src/vec/exec/join/vjoin_node_base.h
+++ b/be/src/vec/exec/join/vjoin_node_base.h
@@ -124,6 +124,7 @@ protected:
     MutableColumnPtr _tuple_is_null_right_flag_column;
 
     RuntimeProfile::Counter* _build_timer;
+    RuntimeProfile::Counter* _build_get_next_timer;
     RuntimeProfile::Counter* _probe_timer;
     RuntimeProfile::Counter* _build_rows_counter;
     RuntimeProfile::Counter* _probe_rows_counter;


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 14/36: [fix](nereids) avg size of column stats always be 0 (#20341)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 2d762f6da967a6f9a68bc4a5bf9df0361945e5f8
Author: minghong <en...@gmail.com>
AuthorDate: Mon Jun 5 13:01:58 2023 +0800

    [fix](nereids) avg size of column stats always be 0 (#20341)
    
    it takes lot of effort to compute the avgSizeByte for col stats.
    we use schema information to avoid compute actual average size
---
 .../doris/nereids/stats/StatsCalculator.java       |    5 +
 regression-test/data/cte_reuse/q11.out             |   65 -
 regression-test/data/cte_reuse/q14.out             |  161 -
 regression-test/data/cte_reuse/q23.out             |   98 -
 regression-test/data/cte_reuse/q24.out             |   54 -
 regression-test/data/cte_reuse/q31.out             |   74 -
 regression-test/data/cte_reuse/q4.out              |   97 -
 regression-test/data/cte_reuse/q47.out             |   49 -
 regression-test/data/cte_reuse/q57.out             |   49 -
 regression-test/data/cte_reuse/q59.out             |   45 -
 regression-test/data/cte_reuse/q64.out             |  115 -
 regression-test/data/cte_reuse/q74.out             |   64 -
 .../suites/cte_reuse/ddl/call_center.sql           |   38 -
 .../suites/cte_reuse/ddl/catalog_page.sql          |   17 -
 .../suites/cte_reuse/ddl/catalog_returns.sql       |   34 -
 .../suites/cte_reuse/ddl/catalog_sales.sql         |   42 -
 regression-test/suites/cte_reuse/ddl/customer.sql  |   26 -
 .../suites/cte_reuse/ddl/customer_address.sql      |   21 -
 .../suites/cte_reuse/ddl/customer_demographics.sql |   16 -
 regression-test/suites/cte_reuse/ddl/date_dim.sql  |   35 -
 .../cte_reuse/ddl/household_demographics.sql       |   13 -
 .../suites/cte_reuse/ddl/income_band.sql           |   11 -
 regression-test/suites/cte_reuse/ddl/inventory.sql |   12 -
 regression-test/suites/cte_reuse/ddl/item.sql      |   29 -
 regression-test/suites/cte_reuse/ddl/promotion.sql |   27 -
 regression-test/suites/cte_reuse/ddl/reason.sql    |   11 -
 regression-test/suites/cte_reuse/ddl/ship_mode.sql |   14 -
 regression-test/suites/cte_reuse/ddl/store.sql     |   36 -
 .../suites/cte_reuse/ddl/store_returns.sql         |   28 -
 .../suites/cte_reuse/ddl/store_sales.sql           |   32 -
 regression-test/suites/cte_reuse/ddl/time_dim.sql  |   17 -
 regression-test/suites/cte_reuse/ddl/warehouse.sql |   22 -
 regression-test/suites/cte_reuse/ddl/web_page.sql  |   21 -
 .../suites/cte_reuse/ddl/web_returns.sql           |   31 -
 regression-test/suites/cte_reuse/ddl/web_sales.sql |   42 -
 regression-test/suites/cte_reuse/ddl/web_site.sql  |   36 -
 regression-test/suites/cte_reuse/load.groovy       | 4228 --------------------
 regression-test/suites/cte_reuse/q11.groovy        |  108 -
 regression-test/suites/cte_reuse/q14.groovy        |  207 -
 regression-test/suites/cte_reuse/q23.groovy        |  129 -
 regression-test/suites/cte_reuse/q24.groovy        |   88 -
 regression-test/suites/cte_reuse/q31.groovy        |  104 -
 regression-test/suites/cte_reuse/q4.groovy         |  134 -
 regression-test/suites/cte_reuse/q47.groovy        |  103 -
 regression-test/suites/cte_reuse/q57.groovy        |   99 -
 regression-test/suites/cte_reuse/q59.groovy        |  115 -
 regression-test/suites/cte_reuse/q64.groovy        |  149 -
 regression-test/suites/cte_reuse/q74.groovy        |  100 -
 48 files changed, 5 insertions(+), 7046 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
index 9341371646..c166bd8d0b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
@@ -553,6 +553,11 @@ public class StatsCalculator extends DefaultPlanVisitor<Statistics, Void> {
                 throw new RuntimeException(String.format("Invalid slot: %s", slotReference.getExprId()));
             }
             ColumnStatistic cache = Config.enable_stats ? getColumnStatistic(table, colName) : ColumnStatistic.UNKNOWN;
+            if (cache.avgSizeByte <= 0) {
+                cache = new ColumnStatisticBuilder(cache)
+                        .setAvgSizeByte(slotReference.getColumn().get().getType().getSlotSize())
+                        .build();
+            }
             if (cache == ColumnStatistic.UNKNOWN && !colName.equals("__DORIS_DELETE_SIGN__")) {
                 if (forbidUnknownColStats) {
                     if (StatisticsUtil.statsTblAvailable()) {
diff --git a/regression-test/data/cte_reuse/q11.out b/regression-test/data/cte_reuse/q11.out
deleted file mode 100644
index 6965148ee4..0000000000
--- a/regression-test/data/cte_reuse/q11.out
+++ /dev/null
@@ -1,65 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !sql --
-CteAnchor[cteId= ( CTEId#4=] )
---CteProducer[cteId= ( CTEId#4=] )
-----PhysicalProject
-------PhysicalUnion
---------PhysicalProject
-----------hashAgg[GLOBAL]
-------------PhysicalDistribute
---------------hashAgg[LOCAL]
-----------------PhysicalProject
-------------------hashJoin[INNER_JOIN](customer.c_customer_sk = store_sales.ss_customer_sk)
---------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk)
-----------------------PhysicalProject
-------------------------filter((('s' = 'w') OR ('s' = 's')))
---------------------------PhysicalOlapScan[store_sales]
-----------------------PhysicalDistribute
-------------------------PhysicalProject
---------------------------filter(((date_dim.d_year = 2001) OR (date_dim.d_year = 2002))(('s' = 'w') OR ('s' = 's')))
-----------------------------PhysicalOlapScan[date_dim]
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------filter((('s' = 'w') OR ('s' = 's')))
---------------------------PhysicalOlapScan[customer]
---------PhysicalProject
-----------hashAgg[GLOBAL]
-------------PhysicalDistribute
---------------hashAgg[LOCAL]
-----------------PhysicalProject
-------------------hashJoin[INNER_JOIN](customer.c_customer_sk = web_sales.ws_bill_customer_sk)
---------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk)
-----------------------PhysicalProject
-------------------------filter((('w' = 'w') OR ('w' = 's')))
---------------------------PhysicalOlapScan[web_sales]
-----------------------PhysicalDistribute
-------------------------PhysicalProject
---------------------------filter(((date_dim.d_year = 2001) OR (date_dim.d_year = 2002))(('w' = 'w') OR ('w' = 's')))
-----------------------------PhysicalOlapScan[date_dim]
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------filter((('w' = 'w') OR ('w' = 's')))
---------------------------PhysicalOlapScan[customer]
---PhysicalTopN
-----PhysicalDistribute
-------PhysicalTopN
---------PhysicalProject
-----------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END > CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END)
-------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)
---------------PhysicalProject
-----------------filter((t_w_secyear.dyear = 2002)(t_w_secyear.sale_type = 'w'))
-------------------CteConsumer[cteId= ( CTEId#4=] )
---------------PhysicalDistribute
-----------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id)
-------------------PhysicalProject
---------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2002))
-----------------------CteConsumer[cteId= ( CTEId#4=] )
-------------------PhysicalDistribute
---------------------PhysicalProject
-----------------------filter((t_s_firstyear.dyear = 2001)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.00))
-------------------------CteConsumer[cteId= ( CTEId#4=] )
-------------PhysicalDistribute
---------------PhysicalProject
-----------------filter((t_w_firstyear.year_total > 0.00)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.dyear = 2001))
-------------------CteConsumer[cteId= ( CTEId#4=] )
-
diff --git a/regression-test/data/cte_reuse/q14.out b/regression-test/data/cte_reuse/q14.out
deleted file mode 100644
index 36d63de9e5..0000000000
--- a/regression-test/data/cte_reuse/q14.out
+++ /dev/null
@@ -1,161 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !sql --
-CteAnchor[cteId= ( CTEId#8=] )
---CteProducer[cteId= ( CTEId#8=] )
-----PhysicalProject
-------hashJoin[INNER_JOIN](item.i_brand_id = y.brand_id)(item.i_class_id = y.class_id)(item.i_category_id = y.category_id)
---------PhysicalIntersect
-----------PhysicalProject
-------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = iss.i_item_sk)
---------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = d1.d_date_sk)
-----------------PhysicalProject
-------------------PhysicalOlapScan[store_sales]
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------filter((d1.d_year <= 2001)(d1.d_year >= 1999))
-----------------------PhysicalOlapScan[date_dim]
---------------PhysicalDistribute
-----------------PhysicalProject
-------------------PhysicalOlapScan[item]
-----------PhysicalProject
-------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = ics.i_item_sk)
---------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d2.d_date_sk)
-----------------PhysicalProject
-------------------PhysicalOlapScan[catalog_sales]
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------filter((d2.d_year >= 1999)(d2.d_year <= 2001))
-----------------------PhysicalOlapScan[date_dim]
---------------PhysicalDistribute
-----------------PhysicalProject
-------------------PhysicalOlapScan[item]
-----------PhysicalProject
-------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = iws.i_item_sk)
---------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = d3.d_date_sk)
-----------------PhysicalProject
-------------------PhysicalOlapScan[web_sales]
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------filter((d3.d_year <= 2001)(d3.d_year >= 1999))
-----------------------PhysicalOlapScan[date_dim]
---------------PhysicalDistribute
-----------------PhysicalProject
-------------------PhysicalOlapScan[item]
---------PhysicalDistribute
-----------PhysicalProject
-------------PhysicalOlapScan[item]
---CteAnchor[cteId= ( CTEId#10=] )
-----CteProducer[cteId= ( CTEId#10=] )
-------hashAgg[GLOBAL]
---------PhysicalDistribute
-----------hashAgg[LOCAL]
-------------PhysicalUnion
---------------PhysicalProject
-----------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk)
-------------------PhysicalProject
---------------------PhysicalOlapScan[store_sales]
-------------------PhysicalDistribute
---------------------PhysicalProject
-----------------------filter((date_dim.d_year >= 1999)(date_dim.d_year <= 2001))
-------------------------PhysicalOlapScan[date_dim]
---------------PhysicalProject
-----------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)
-------------------PhysicalProject
---------------------PhysicalOlapScan[catalog_sales]
-------------------PhysicalDistribute
---------------------PhysicalProject
-----------------------filter((date_dim.d_year >= 1999)(date_dim.d_year <= 2001))
-------------------------PhysicalOlapScan[date_dim]
---------------PhysicalProject
-----------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk)
-------------------PhysicalProject
---------------------PhysicalOlapScan[web_sales]
-------------------PhysicalDistribute
---------------------PhysicalProject
-----------------------filter((date_dim.d_year >= 1999)(date_dim.d_year <= 2001))
-------------------------PhysicalOlapScan[date_dim]
-----PhysicalTopN
-------PhysicalDistribute
---------PhysicalTopN
-----------PhysicalProject
-------------hashAgg[GLOBAL]
---------------PhysicalDistribute
-----------------hashAgg[LOCAL]
-------------------PhysicalRepeat
---------------------PhysicalUnion
-----------------------PhysicalProject
-------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE))
---------------------------PhysicalProject
-----------------------------hashAgg[GLOBAL]
-------------------------------PhysicalDistribute
---------------------------------hashAgg[LOCAL]
-----------------------------------PhysicalProject
-------------------------------------hashJoin[RIGHT_SEMI_JOIN](store_sales.ss_item_sk = cross_items.ss_item_sk)
---------------------------------------PhysicalDistribute
-----------------------------------------CteConsumer[cteId= ( CTEId#8=] )
---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk)
-----------------------------------------PhysicalDistribute
-------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk)
---------------------------------------------PhysicalProject
-----------------------------------------------PhysicalOlapScan[store_sales]
---------------------------------------------PhysicalDistribute
-----------------------------------------------PhysicalProject
-------------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 2001))
---------------------------------------------------PhysicalOlapScan[date_dim]
-----------------------------------------PhysicalProject
-------------------------------------------PhysicalOlapScan[item]
---------------------------PhysicalDistribute
-----------------------------PhysicalAssertNumRows
-------------------------------PhysicalDistribute
---------------------------------CteConsumer[cteId= ( CTEId#10=] )
-----------------------PhysicalProject
-------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE))
---------------------------PhysicalProject
-----------------------------hashAgg[GLOBAL]
-------------------------------PhysicalDistribute
---------------------------------hashAgg[LOCAL]
-----------------------------------PhysicalProject
-------------------------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = cross_items.ss_item_sk)
---------------------------------------PhysicalDistribute
-----------------------------------------CteConsumer[cteId= ( CTEId#8=] )
---------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk)
-----------------------------------------PhysicalDistribute
-------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)
---------------------------------------------PhysicalProject
-----------------------------------------------PhysicalOlapScan[catalog_sales]
---------------------------------------------PhysicalDistribute
-----------------------------------------------PhysicalProject
-------------------------------------------------filter((date_dim.d_year = 2001)(date_dim.d_moy = 11))
---------------------------------------------------PhysicalOlapScan[date_dim]
-----------------------------------------PhysicalProject
-------------------------------------------PhysicalOlapScan[item]
---------------------------PhysicalDistribute
-----------------------------PhysicalAssertNumRows
-------------------------------PhysicalDistribute
---------------------------------CteConsumer[cteId= ( CTEId#10=] )
-----------------------PhysicalProject
-------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE))
---------------------------PhysicalProject
-----------------------------hashAgg[GLOBAL]
-------------------------------PhysicalDistribute
---------------------------------hashAgg[LOCAL]
-----------------------------------PhysicalProject
-------------------------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = cross_items.ss_item_sk)
---------------------------------------PhysicalDistribute
-----------------------------------------CteConsumer[cteId= ( CTEId#8=] )
---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk)
-----------------------------------------PhysicalDistribute
-------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk)
---------------------------------------------PhysicalProject
-----------------------------------------------PhysicalOlapScan[web_sales]
---------------------------------------------PhysicalDistribute
-----------------------------------------------PhysicalProject
-------------------------------------------------filter((date_dim.d_year = 2001)(date_dim.d_moy = 11))
---------------------------------------------------PhysicalOlapScan[date_dim]
-----------------------------------------PhysicalProject
-------------------------------------------PhysicalOlapScan[item]
---------------------------PhysicalDistribute
-----------------------------PhysicalAssertNumRows
-------------------------------PhysicalDistribute
---------------------------------CteConsumer[cteId= ( CTEId#10=] )
-
diff --git a/regression-test/data/cte_reuse/q23.out b/regression-test/data/cte_reuse/q23.out
deleted file mode 100644
index 63f621b681..0000000000
--- a/regression-test/data/cte_reuse/q23.out
+++ /dev/null
@@ -1,98 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !sql --
-CteAnchor[cteId= ( CTEId#1=] )
---CteProducer[cteId= ( CTEId#1=] )
-----PhysicalProject
-------filter((cnt > 4))
---------hashAgg[GLOBAL]
-----------PhysicalDistribute
-------------hashAgg[LOCAL]
---------------PhysicalProject
-----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk)
-------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk)
---------------------PhysicalProject
-----------------------PhysicalOlapScan[store_sales]
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------filter(d_year IN (2000, 2001, 2002, 2003))
---------------------------PhysicalOlapScan[date_dim]
-------------------PhysicalDistribute
---------------------PhysicalProject
-----------------------PhysicalOlapScan[item]
---CteAnchor[cteId= ( CTEId#4=] )
-----CteProducer[cteId= ( CTEId#4=] )
-------PhysicalProject
---------NestedLoopJoin[INNER_JOIN](cast(ssales as DOUBLE) > cast(((cast(50 as DECIMALV3(9, 6)) / cast('100.0' as DECIMALV3(5, 2))) * tpcds_cmax) as DOUBLE))
-----------hashAgg[GLOBAL]
-------------PhysicalDistribute
---------------hashAgg[LOCAL]
-----------------PhysicalProject
-------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk)
---------------------PhysicalProject
-----------------------PhysicalOlapScan[store_sales]
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------PhysicalOlapScan[customer]
-----------PhysicalDistribute
-------------PhysicalAssertNumRows
---------------PhysicalProject
-----------------hashAgg[GLOBAL]
-------------------PhysicalDistribute
---------------------hashAgg[LOCAL]
-----------------------PhysicalProject
-------------------------hashAgg[GLOBAL]
---------------------------PhysicalDistribute
-----------------------------hashAgg[LOCAL]
-------------------------------PhysicalProject
---------------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk)
-----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk)
-------------------------------------PhysicalProject
---------------------------------------PhysicalOlapScan[store_sales]
-------------------------------------PhysicalDistribute
---------------------------------------PhysicalProject
-----------------------------------------filter(d_year IN (2000, 2001, 2002, 2003))
-------------------------------------------PhysicalOlapScan[date_dim]
-----------------------------------PhysicalDistribute
-------------------------------------PhysicalProject
---------------------------------------PhysicalOlapScan[customer]
-----PhysicalLimit
-------PhysicalLimit
---------hashAgg[GLOBAL]
-----------PhysicalDistribute
-------------hashAgg[LOCAL]
---------------PhysicalUnion
-----------------PhysicalProject
-------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------CteConsumer[cteId= ( CTEId#4=] )
---------------------PhysicalDistribute
-----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)
-------------------------hashJoin[LEFT_SEMI_JOIN](catalog_sales.cs_item_sk = frequent_ss_items.item_sk)
---------------------------PhysicalProject
-----------------------------PhysicalOlapScan[catalog_sales]
---------------------------PhysicalDistribute
-----------------------------PhysicalProject
-------------------------------CteConsumer[cteId= ( CTEId#1=] )
-------------------------PhysicalDistribute
---------------------------PhysicalProject
-----------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2))
-------------------------------PhysicalOlapScan[date_dim]
-----------------PhysicalProject
-------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------CteConsumer[cteId= ( CTEId#4=] )
---------------------PhysicalDistribute
-----------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk)
-------------------------hashJoin[LEFT_SEMI_JOIN](web_sales.ws_item_sk = frequent_ss_items.item_sk)
---------------------------PhysicalProject
-----------------------------PhysicalOlapScan[web_sales]
---------------------------PhysicalDistribute
-----------------------------PhysicalProject
-------------------------------CteConsumer[cteId= ( CTEId#1=] )
-------------------------PhysicalDistribute
---------------------------PhysicalProject
-----------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2))
-------------------------------PhysicalOlapScan[date_dim]
-
diff --git a/regression-test/data/cte_reuse/q24.out b/regression-test/data/cte_reuse/q24.out
deleted file mode 100644
index fa0d4e45a6..0000000000
--- a/regression-test/data/cte_reuse/q24.out
+++ /dev/null
@@ -1,54 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !sql --
-CteAnchor[cteId= ( CTEId#0=] )
---CteProducer[cteId= ( CTEId#0=] )
-----PhysicalProject
-------hashAgg[GLOBAL]
---------PhysicalDistribute
-----------hashAgg[LOCAL]
-------------PhysicalProject
---------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)
-----------------PhysicalProject
-------------------PhysicalOlapScan[store_returns]
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk)
-----------------------PhysicalProject
-------------------------PhysicalOlapScan[item]
-----------------------PhysicalDistribute
-------------------------hashJoin[INNER_JOIN](store.s_zip = customer_address.ca_zip)(customer.c_birth_country = expr_upper(ca_country))
---------------------------PhysicalProject
-----------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk)
-------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk)
---------------------------------PhysicalProject
-----------------------------------PhysicalOlapScan[store_sales]
---------------------------------PhysicalDistribute
-----------------------------------PhysicalProject
-------------------------------------filter((store.s_market_id = 8))
---------------------------------------PhysicalOlapScan[store]
-------------------------------PhysicalDistribute
---------------------------------PhysicalProject
-----------------------------------PhysicalOlapScan[customer]
---------------------------PhysicalDistribute
-----------------------------PhysicalProject
-------------------------------PhysicalOlapScan[customer_address]
---PhysicalQuickSort
-----PhysicalDistribute
-------PhysicalQuickSort
---------PhysicalProject
-----------NestedLoopJoin[INNER_JOIN](cast(paid as DOUBLE) > cast((cast('0.05' as DECIMALV3(5, 2)) * avg(netpaid)) as DOUBLE))
-------------hashAgg[GLOBAL]
---------------PhysicalDistribute
-----------------hashAgg[LOCAL]
-------------------PhysicalProject
---------------------filter((cast(i_color as VARCHAR(*)) = 'pale'))
-----------------------CteConsumer[cteId= ( CTEId#0=] )
-------------PhysicalDistribute
---------------PhysicalAssertNumRows
-----------------PhysicalProject
-------------------hashAgg[GLOBAL]
---------------------PhysicalDistribute
-----------------------hashAgg[LOCAL]
-------------------------PhysicalProject
---------------------------CteConsumer[cteId= ( CTEId#0=] )
-
diff --git a/regression-test/data/cte_reuse/q31.out b/regression-test/data/cte_reuse/q31.out
deleted file mode 100644
index b767d12c63..0000000000
--- a/regression-test/data/cte_reuse/q31.out
+++ /dev/null
@@ -1,74 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !sql --
-CteAnchor[cteId= ( CTEId#6=] )
---CteProducer[cteId= ( CTEId#6=] )
-----PhysicalProject
-------hashAgg[GLOBAL]
---------PhysicalDistribute
-----------hashAgg[LOCAL]
-------------PhysicalProject
---------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk)
-----------------PhysicalProject
-------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk)
---------------------PhysicalProject
-----------------------PhysicalOlapScan[store_sales]
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------filter(d_qoy IN (1, 2, 3)(ss.d_year = 2000))
---------------------------PhysicalOlapScan[date_dim]
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------PhysicalOlapScan[customer_address]
---CteAnchor[cteId= ( CTEId#7=] )
-----CteProducer[cteId= ( CTEId#7=] )
-------PhysicalProject
---------hashAgg[GLOBAL]
-----------PhysicalDistribute
-------------hashAgg[LOCAL]
---------------PhysicalProject
-----------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk)
-------------------PhysicalProject
---------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk)
-----------------------PhysicalProject
-------------------------PhysicalOlapScan[web_sales]
-----------------------PhysicalDistribute
-------------------------PhysicalProject
---------------------------filter((ws.d_year = 2000)d_qoy IN (1, 2, 3))
-----------------------------PhysicalOlapScan[date_dim]
-------------------PhysicalDistribute
---------------------PhysicalProject
-----------------------PhysicalOlapScan[customer_address]
-----PhysicalQuickSort
-------PhysicalDistribute
---------PhysicalQuickSort
-----------PhysicalProject
-------------hashJoin[INNER_JOIN](ws1.ca_county = ws3.ca_county)(CASE WHEN (web_sales > 0.00) THEN (cast(cast(web_sales as DECIMALV3(21, 3)) as DECIMALV3(27, 9)) / web_sales) ELSE NULL END > CASE WHEN (store_sales > 0.00) THEN (cast(cast(store_sales as DECIMALV3(21, 3)) as DECIMALV3(27, 9)) / store_sales) ELSE NULL END)
---------------PhysicalProject
-----------------filter((ws3.d_year = 2000)(ws3.d_qoy = 3))
-------------------CteConsumer[cteId= ( CTEId#7=] )
---------------PhysicalDistribute
-----------------PhysicalProject
-------------------hashJoin[INNER_JOIN](ss2.ca_county = ss3.ca_county)
---------------------PhysicalProject
-----------------------filter((ss3.d_year = 2000)(ss3.d_qoy = 3))
-------------------------CteConsumer[cteId= ( CTEId#6=] )
---------------------PhysicalDistribute
-----------------------hashJoin[INNER_JOIN](ss1.ca_county = ss2.ca_county)(CASE WHEN (web_sales > 0.00) THEN (cast(cast(web_sales as DECIMALV3(21, 3)) as DECIMALV3(27, 9)) / web_sales) ELSE NULL END > CASE WHEN (store_sales > 0.00) THEN (cast(cast(store_sales as DECIMALV3(21, 3)) as DECIMALV3(27, 9)) / store_sales) ELSE NULL END)
-------------------------PhysicalProject
---------------------------filter((ss2.d_year = 2000)(ss2.d_qoy = 2))
-----------------------------CteConsumer[cteId= ( CTEId#6=] )
-------------------------PhysicalDistribute
---------------------------hashJoin[INNER_JOIN](ss1.ca_county = ws1.ca_county)
-----------------------------PhysicalProject
-------------------------------filter((ss1.d_year = 2000)(ss1.d_qoy = 1))
---------------------------------CteConsumer[cteId= ( CTEId#6=] )
-----------------------------PhysicalDistribute
-------------------------------hashJoin[INNER_JOIN](ws1.ca_county = ws2.ca_county)
---------------------------------PhysicalProject
-----------------------------------filter((ws1.d_year = 2000)(ws1.d_qoy = 1))
-------------------------------------CteConsumer[cteId= ( CTEId#7=] )
---------------------------------PhysicalDistribute
-----------------------------------PhysicalProject
-------------------------------------filter((ws2.d_qoy = 2)(ws2.d_year = 2000))
---------------------------------------CteConsumer[cteId= ( CTEId#7=] )
-
diff --git a/regression-test/data/cte_reuse/q4.out b/regression-test/data/cte_reuse/q4.out
deleted file mode 100644
index 6f72ca89f4..0000000000
--- a/regression-test/data/cte_reuse/q4.out
+++ /dev/null
@@ -1,97 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !sql --
-CteAnchor[cteId= ( CTEId#6=] )
---CteProducer[cteId= ( CTEId#6=] )
-----PhysicalProject
-------PhysicalUnion
---------PhysicalProject
-----------hashAgg[GLOBAL]
-------------PhysicalDistribute
---------------hashAgg[LOCAL]
-----------------PhysicalProject
-------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk)
---------------------PhysicalProject
-----------------------hashJoin[INNER_JOIN](customer.c_customer_sk = store_sales.ss_customer_sk)
-------------------------PhysicalProject
---------------------------filter(((('s' = 's') OR ('s' = 'c')) OR ('s' = 'w')))
-----------------------------PhysicalOlapScan[store_sales]
-------------------------PhysicalDistribute
---------------------------PhysicalProject
-----------------------------filter(((('s' = 's') OR ('s' = 'c')) OR ('s' = 'w')))
-------------------------------PhysicalOlapScan[customer]
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------filter(((date_dim.d_year = 2001) OR (date_dim.d_year = 2002))((('s' = 's') OR ('s' = 'c')) OR ('s' = 'w')))
---------------------------PhysicalOlapScan[date_dim]
---------PhysicalProject
-----------hashAgg[GLOBAL]
-------------PhysicalDistribute
---------------hashAgg[LOCAL]
-----------------PhysicalProject
-------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)
---------------------PhysicalProject
-----------------------hashJoin[INNER_JOIN](customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)
-------------------------PhysicalProject
---------------------------filter(((('c' = 's') OR ('c' = 'c')) OR ('c' = 'w')))
-----------------------------PhysicalOlapScan[catalog_sales]
-------------------------PhysicalDistribute
---------------------------PhysicalProject
-----------------------------filter(((('c' = 's') OR ('c' = 'c')) OR ('c' = 'w')))
-------------------------------PhysicalOlapScan[customer]
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------filter(((date_dim.d_year = 2001) OR (date_dim.d_year = 2002))((('c' = 's') OR ('c' = 'c')) OR ('c' = 'w')))
---------------------------PhysicalOlapScan[date_dim]
---------PhysicalProject
-----------hashAgg[GLOBAL]
-------------PhysicalDistribute
---------------hashAgg[LOCAL]
-----------------PhysicalProject
-------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk)
---------------------PhysicalProject
-----------------------hashJoin[INNER_JOIN](customer.c_customer_sk = web_sales.ws_bill_customer_sk)
-------------------------PhysicalProject
---------------------------filter(((('w' = 's') OR ('w' = 'c')) OR ('w' = 'w')))
-----------------------------PhysicalOlapScan[web_sales]
-------------------------PhysicalDistribute
---------------------------PhysicalProject
-----------------------------filter(((('w' = 's') OR ('w' = 'c')) OR ('w' = 'w')))
-------------------------------PhysicalOlapScan[customer]
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------filter(((date_dim.d_year = 2001) OR (date_dim.d_year = 2002))((('w' = 's') OR ('w' = 'c')) OR ('w' = 'w')))
---------------------------PhysicalOlapScan[date_dim]
---PhysicalTopN
-----PhysicalDistribute
-------PhysicalTopN
---------PhysicalProject
-----------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END)
-------------PhysicalProject
---------------filter((t_w_firstyear.dyear = 2001)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.year_total > 0.000000))
-----------------CteConsumer[cteId= ( CTEId#6=] )
-------------PhysicalDistribute
---------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)
-----------------PhysicalProject
-------------------filter((t_w_secyear.sale_type = 'w')(t_w_secyear.dyear = 2002))
---------------------CteConsumer[cteId= ( CTEId#6=] )
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_firstyear.customer_id)(CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END)
-----------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id)
-------------------------PhysicalProject
---------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2002))
-----------------------------CteConsumer[cteId= ( CTEId#6=] )
-------------------------PhysicalDistribute
---------------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_secyear.customer_id)
-----------------------------PhysicalProject
-------------------------------filter((t_c_secyear.sale_type = 'c')(t_c_secyear.dyear = 2002))
---------------------------------CteConsumer[cteId= ( CTEId#6=] )
-----------------------------PhysicalDistribute
-------------------------------PhysicalProject
---------------------------------filter((t_s_firstyear.year_total > 0.000000)(t_s_firstyear.dyear = 2001)(t_s_firstyear.sale_type = 's'))
-----------------------------------CteConsumer[cteId= ( CTEId#6=] )
-----------------------PhysicalDistribute
-------------------------PhysicalProject
---------------------------filter((t_c_firstyear.year_total > 0.000000)(t_c_firstyear.sale_type = 'c')(t_c_firstyear.dyear = 2001))
-----------------------------CteConsumer[cteId= ( CTEId#6=] )
-
diff --git a/regression-test/data/cte_reuse/q47.out b/regression-test/data/cte_reuse/q47.out
deleted file mode 100644
index 820340a660..0000000000
--- a/regression-test/data/cte_reuse/q47.out
+++ /dev/null
@@ -1,49 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !sql --
-CteAnchor[cteId= ( CTEId#0=] )
---CteProducer[cteId= ( CTEId#0=] )
-----PhysicalProject
-------PhysicalWindow
---------PhysicalQuickSort
-----------PhysicalDistribute
-------------PhysicalWindow
---------------PhysicalQuickSort
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------hashAgg[GLOBAL]
-----------------------PhysicalDistribute
-------------------------hashAgg[LOCAL]
---------------------------PhysicalProject
-----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk)
-------------------------------PhysicalProject
---------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk)
-----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk)
-------------------------------------PhysicalProject
---------------------------------------PhysicalOlapScan[store_sales]
-------------------------------------PhysicalDistribute
---------------------------------------PhysicalProject
-----------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))))
-------------------------------------------PhysicalOlapScan[date_dim]
-----------------------------------PhysicalDistribute
-------------------------------------PhysicalProject
---------------------------------------PhysicalOlapScan[item]
-------------------------------PhysicalDistribute
---------------------------------PhysicalProject
-----------------------------------PhysicalOlapScan[store]
---PhysicalProject
-----PhysicalTopN
-------PhysicalDistribute
---------PhysicalTopN
-----------PhysicalProject
-------------hashJoin[INNER_JOIN](i_category = v1_lag.i_category)(i_brand = v1_lag.i_brand)(s_store_name = v1_lag.s_store_name)(s_company_name = v1_lag.s_company_name)(v1.rn = expr_(rn + 1))
---------------PhysicalProject
-----------------CteConsumer[cteId= ( CTEId#0=] )
---------------PhysicalDistribute
-----------------hashJoin[INNER_JOIN](i_category = v1_lead.i_category)(i_brand = v1_lead.i_brand)(s_store_name = v1_lead.s_store_name)(s_company_name = v1_lead.s_company_name)(v1.rn = expr_(rn - 1))
-------------------PhysicalProject
---------------------CteConsumer[cteId= ( CTEId#0=] )
-------------------PhysicalDistribute
---------------------PhysicalProject
-----------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.d_year = 1999)(v2.avg_monthly_sales > 0.0000))
-------------------------CteConsumer[cteId= ( CTEId#0=] )
-
diff --git a/regression-test/data/cte_reuse/q57.out b/regression-test/data/cte_reuse/q57.out
deleted file mode 100644
index 2eecc377cc..0000000000
--- a/regression-test/data/cte_reuse/q57.out
+++ /dev/null
@@ -1,49 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !sql --
-CteAnchor[cteId= ( CTEId#0=] )
---CteProducer[cteId= ( CTEId#0=] )
-----PhysicalProject
-------PhysicalWindow
---------PhysicalQuickSort
-----------PhysicalDistribute
-------------PhysicalWindow
---------------PhysicalQuickSort
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------hashAgg[GLOBAL]
-----------------------PhysicalDistribute
-------------------------hashAgg[LOCAL]
---------------------------PhysicalProject
-----------------------------hashJoin[INNER_JOIN](call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)
-------------------------------PhysicalProject
---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk)
-----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)
-------------------------------------PhysicalProject
---------------------------------------PhysicalOlapScan[catalog_sales]
-------------------------------------PhysicalDistribute
---------------------------------------PhysicalProject
-----------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1))))
-------------------------------------------PhysicalOlapScan[date_dim]
-----------------------------------PhysicalDistribute
-------------------------------------PhysicalProject
---------------------------------------PhysicalOlapScan[item]
-------------------------------PhysicalDistribute
---------------------------------PhysicalProject
-----------------------------------PhysicalOlapScan[call_center]
---PhysicalProject
-----PhysicalTopN
-------PhysicalDistribute
---------PhysicalTopN
-----------PhysicalProject
-------------hashJoin[INNER_JOIN](i_category = v1_lag.i_category)(i_brand = v1_lag.i_brand)(cc_name = v1_lag.cc_name)(v1.rn = expr_(rn + 1))
---------------PhysicalProject
-----------------CteConsumer[cteId= ( CTEId#0=] )
---------------PhysicalDistribute
-----------------hashJoin[INNER_JOIN](i_category = v1_lead.i_category)(i_brand = v1_lead.i_brand)(cc_name = v1_lead.cc_name)(v1.rn = expr_(rn - 1))
-------------------PhysicalProject
---------------------CteConsumer[cteId= ( CTEId#0=] )
-------------------PhysicalDistribute
---------------------PhysicalProject
-----------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.avg_monthly_sales > 0.0000)(v2.d_year = 1999))
-------------------------CteConsumer[cteId= ( CTEId#0=] )
-
diff --git a/regression-test/data/cte_reuse/q59.out b/regression-test/data/cte_reuse/q59.out
deleted file mode 100644
index 29b5e53ef1..0000000000
--- a/regression-test/data/cte_reuse/q59.out
+++ /dev/null
@@ -1,45 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !sql --
-CteAnchor[cteId= ( CTEId#4=] )
---CteProducer[cteId= ( CTEId#4=] )
-----PhysicalProject
-------hashAgg[GLOBAL]
---------PhysicalDistribute
-----------hashAgg[LOCAL]
-------------PhysicalProject
---------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk)
-----------------PhysicalProject
-------------------PhysicalOlapScan[store_sales]
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------PhysicalOlapScan[date_dim]
---PhysicalTopN
-----PhysicalDistribute
-------PhysicalTopN
---------PhysicalProject
-----------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq2)
-------------PhysicalProject
---------------filter((d.d_month_seq <= 1235)(d.d_month_seq >= 1224))
-----------------PhysicalOlapScan[date_dim]
-------------PhysicalDistribute
---------------hashJoin[INNER_JOIN](y.s_store_id1 = x.s_store_id2)(wss.ss_store_sk = store.s_store_sk)
-----------------PhysicalProject
-------------------PhysicalOlapScan[store]
-----------------PhysicalDistribute
-------------------hashJoin[INNER_JOIN](expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 52))
---------------------PhysicalProject
-----------------------CteConsumer[cteId= ( CTEId#4=] )
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk)
---------------------------PhysicalProject
-----------------------------PhysicalOlapScan[store]
---------------------------PhysicalDistribute
-----------------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq1)
-------------------------------PhysicalProject
---------------------------------CteConsumer[cteId= ( CTEId#4=] )
-------------------------------PhysicalDistribute
---------------------------------PhysicalProject
-----------------------------------filter((d.d_month_seq <= 1223)(d.d_month_seq >= 1212))
-------------------------------------PhysicalOlapScan[date_dim]
-
diff --git a/regression-test/data/cte_reuse/q64.out b/regression-test/data/cte_reuse/q64.out
deleted file mode 100644
index 9346d33b3e..0000000000
--- a/regression-test/data/cte_reuse/q64.out
+++ /dev/null
@@ -1,115 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !sql --
-CteAnchor[cteId= ( CTEId#14=] )
---CteProducer[cteId= ( CTEId#14=] )
-----PhysicalProject
-------hashAgg[LOCAL]
---------PhysicalProject
-----------hashJoin[INNER_JOIN](hd2.hd_income_band_sk = ib2.ib_income_band_sk)
-------------PhysicalProject
---------------PhysicalOlapScan[income_band]
-------------PhysicalDistribute
---------------PhysicalProject
-----------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = ad2.ca_address_sk)
-------------------PhysicalProject
---------------------PhysicalOlapScan[customer_address]
-------------------PhysicalDistribute
---------------------PhysicalProject
-----------------------hashJoin[INNER_JOIN](customer.c_first_shipto_date_sk = d3.d_date_sk)
-------------------------PhysicalProject
---------------------------PhysicalOlapScan[date_dim]
-------------------------PhysicalDistribute
---------------------------PhysicalProject
-----------------------------hashJoin[INNER_JOIN](customer.c_first_sales_date_sk = d2.d_date_sk)
-------------------------------PhysicalProject
---------------------------------PhysicalOlapScan[date_dim]
-------------------------------PhysicalDistribute
---------------------------------PhysicalProject
-----------------------------------hashJoin[INNER_JOIN](customer.c_current_hdemo_sk = hd2.hd_demo_sk)
-------------------------------------PhysicalProject
---------------------------------------PhysicalOlapScan[household_demographics]
-------------------------------------PhysicalDistribute
---------------------------------------PhysicalProject
-----------------------------------------hashJoin[INNER_JOIN](customer.c_current_cdemo_sk = cd2.cd_demo_sk)( not (cd_marital_status = cd_marital_status))
-------------------------------------------PhysicalProject
---------------------------------------------PhysicalOlapScan[customer_demographics]
-------------------------------------------PhysicalDistribute
---------------------------------------------PhysicalProject
-----------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk)
-------------------------------------------------PhysicalProject
---------------------------------------------------PhysicalOlapScan[customer]
-------------------------------------------------PhysicalDistribute
---------------------------------------------------PhysicalProject
-----------------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = cs_ui.cs_item_sk)
-------------------------------------------------------PhysicalProject
---------------------------------------------------------filter((sale > (2 * refund)))
-----------------------------------------------------------hashAgg[GLOBAL]
-------------------------------------------------------------PhysicalDistribute
---------------------------------------------------------------hashAgg[LOCAL]
-----------------------------------------------------------------PhysicalProject
-------------------------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number)
---------------------------------------------------------------------PhysicalProject
-----------------------------------------------------------------------PhysicalOlapScan[catalog_sales]
---------------------------------------------------------------------PhysicalProject
-----------------------------------------------------------------------PhysicalOlapScan[catalog_returns]
-------------------------------------------------------PhysicalDistribute
---------------------------------------------------------PhysicalProject
-----------------------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = cd1.cd_demo_sk)
-------------------------------------------------------------PhysicalProject
---------------------------------------------------------------PhysicalOlapScan[customer_demographics]
-------------------------------------------------------------PhysicalDistribute
---------------------------------------------------------------PhysicalProject
-----------------------------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = d1.d_date_sk)
-------------------------------------------------------------------PhysicalProject
---------------------------------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = hd1.hd_demo_sk)
-----------------------------------------------------------------------PhysicalProject
-------------------------------------------------------------------------hashJoin[INNER_JOIN](hd1.hd_income_band_sk = ib1.ib_income_band_sk)
---------------------------------------------------------------------------PhysicalProject
-----------------------------------------------------------------------------PhysicalOlapScan[household_demographics]
---------------------------------------------------------------------------PhysicalDistribute
-----------------------------------------------------------------------------PhysicalProject
-------------------------------------------------------------------------------PhysicalOlapScan[income_band]
-----------------------------------------------------------------------PhysicalDistribute
-------------------------------------------------------------------------PhysicalProject
---------------------------------------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk)
-----------------------------------------------------------------------------PhysicalProject
-------------------------------------------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk)
---------------------------------------------------------------------------------PhysicalProject
-----------------------------------------------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = ad1.ca_address_sk)
-------------------------------------------------------------------------------------PhysicalProject
---------------------------------------------------------------------------------------PhysicalOlapScan[customer_address]
-------------------------------------------------------------------------------------PhysicalDistribute
---------------------------------------------------------------------------------------PhysicalProject
-----------------------------------------------------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)
-------------------------------------------------------------------------------------------PhysicalProject
---------------------------------------------------------------------------------------------PhysicalOlapScan[store_returns]
-------------------------------------------------------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item_sk)
---------------------------------------------------------------------------------------------PhysicalProject
-----------------------------------------------------------------------------------------------PhysicalOlapScan[store_sales]
---------------------------------------------------------------------------------------------PhysicalDistribute
-----------------------------------------------------------------------------------------------PhysicalProject
-------------------------------------------------------------------------------------------------filter((item.i_current_price <= 74.00)(item.i_current_price >= 65.00)i_color IN ('purple', 'burlywood', 'indian', 'spring', 'floral', 'medium'))
---------------------------------------------------------------------------------------------------PhysicalOlapScan[item]
---------------------------------------------------------------------------------PhysicalDistribute
-----------------------------------------------------------------------------------PhysicalProject
-------------------------------------------------------------------------------------PhysicalOlapScan[store]
-----------------------------------------------------------------------------PhysicalDistribute
-------------------------------------------------------------------------------PhysicalProject
---------------------------------------------------------------------------------PhysicalOlapScan[promotion]
-------------------------------------------------------------------PhysicalDistribute
---------------------------------------------------------------------PhysicalProject
-----------------------------------------------------------------------filter(((d1.d_year = 1999) OR (d1.d_year = 2000)))
-------------------------------------------------------------------------PhysicalOlapScan[date_dim]
---PhysicalQuickSort
-----PhysicalDistribute
-------PhysicalQuickSort
---------PhysicalProject
-----------hashJoin[INNER_JOIN](cs1.item_sk = cs2.item_sk)(cs1.store_name = cs2.store_name)(cs1.store_zip = cs2.store_zip)(cs2.cnt <= cs1.cnt)
-------------PhysicalProject
---------------filter((cs1.syear = 1999))
-----------------CteConsumer[cteId= ( CTEId#14=] )
-------------PhysicalDistribute
---------------PhysicalProject
-----------------filter((cs2.syear = 2000))
-------------------CteConsumer[cteId= ( CTEId#14=] )
-
diff --git a/regression-test/data/cte_reuse/q74.out b/regression-test/data/cte_reuse/q74.out
deleted file mode 100644
index ed9dcb3648..0000000000
--- a/regression-test/data/cte_reuse/q74.out
+++ /dev/null
@@ -1,64 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !sql --
-CteAnchor[cteId= ( CTEId#4=] )
---CteProducer[cteId= ( CTEId#4=] )
-----PhysicalUnion
-------PhysicalProject
---------hashAgg[GLOBAL]
-----------PhysicalDistribute
-------------hashAgg[LOCAL]
---------------PhysicalProject
-----------------hashJoin[INNER_JOIN](customer.c_customer_sk = store_sales.ss_customer_sk)
-------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk)
---------------------PhysicalProject
-----------------------filter((('s' = 's') OR ('s' = 'w')))
-------------------------PhysicalOlapScan[store_sales]
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------filter(((date_dim.d_year = 2001) OR (date_dim.d_year = 2002))(('s' = 's') OR ('s' = 'w')))
---------------------------PhysicalOlapScan[date_dim]
-------------------PhysicalDistribute
---------------------PhysicalProject
-----------------------filter((('s' = 's') OR ('s' = 'w')))
-------------------------PhysicalOlapScan[customer]
-------PhysicalProject
---------hashAgg[GLOBAL]
-----------PhysicalDistribute
-------------hashAgg[LOCAL]
---------------PhysicalProject
-----------------hashJoin[INNER_JOIN](customer.c_customer_sk = web_sales.ws_bill_customer_sk)
-------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk)
---------------------PhysicalProject
-----------------------filter((('w' = 's') OR ('w' = 'w')))
-------------------------PhysicalOlapScan[web_sales]
---------------------PhysicalDistribute
-----------------------PhysicalProject
-------------------------filter(((date_dim.d_year = 2001) OR (date_dim.d_year = 2002))(('w' = 's') OR ('w' = 'w')))
---------------------------PhysicalOlapScan[date_dim]
-------------------PhysicalDistribute
---------------------PhysicalProject
-----------------------filter((('w' = 's') OR ('w' = 'w')))
-------------------------PhysicalOlapScan[customer]
---PhysicalTopN
-----PhysicalDistribute
-------PhysicalTopN
---------PhysicalProject
-----------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE NULL END)
-------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)
---------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id)
-----------------PhysicalProject
-------------------filter((t_s_firstyear.YEAR = 2001)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.00))
---------------------CteConsumer[cteId= ( CTEId#4=] )
-----------------PhysicalDistribute
-------------------PhysicalProject
---------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.YEAR = 2002))
-----------------------CteConsumer[cteId= ( CTEId#4=] )
---------------PhysicalDistribute
-----------------PhysicalProject
-------------------filter((t_w_secyear.YEAR = 2002)(t_w_secyear.sale_type = 'w'))
---------------------CteConsumer[cteId= ( CTEId#4=] )
-------------PhysicalDistribute
---------------PhysicalProject
-----------------filter((t_w_firstyear.YEAR = 2001)(t_w_firstyear.year_total > 0.00)(t_w_firstyear.sale_type = 'w'))
-------------------CteConsumer[cteId= ( CTEId#4=] )
-
diff --git a/regression-test/suites/cte_reuse/ddl/call_center.sql b/regression-test/suites/cte_reuse/ddl/call_center.sql
deleted file mode 100644
index dd92911177..0000000000
--- a/regression-test/suites/cte_reuse/ddl/call_center.sql
+++ /dev/null
@@ -1,38 +0,0 @@
-CREATE TABLE IF NOT EXISTS call_center (
-    cc_call_center_sk bigint,
-    cc_call_center_id char(16),
-    cc_rec_start_date date,
-    cc_rec_end_date date,
-    cc_closed_date_sk integer,
-    cc_open_date_sk integer,
-    cc_name varchar(50),
-    cc_class varchar(50),
-    cc_employees integer,
-    cc_sq_ft integer,
-    cc_hours char(20),
-    cc_manager varchar(40),
-    cc_mkt_id integer,
-    cc_mkt_class char(50),
-    cc_mkt_desc varchar(100),
-    cc_market_manager varchar(40),
-    cc_division integer,
-    cc_division_name varchar(50),
-    cc_company integer,
-    cc_company_name char(50),
-    cc_street_number char(10),
-    cc_street_name varchar(60),
-    cc_street_type char(15),
-    cc_suite_number char(10),
-    cc_city varchar(60),
-    cc_county varchar(30),
-    cc_state char(2),
-    cc_zip char(10),
-    cc_country varchar(20),
-    cc_gmt_offset decimal(5,2),
-    cc_tax_percentage decimal(5,2)
- )
-DUPLICATE KEY(cc_call_center_sk, cc_call_center_id)
-DISTRIBUTED BY HASH(cc_call_center_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
diff --git a/regression-test/suites/cte_reuse/ddl/catalog_page.sql b/regression-test/suites/cte_reuse/ddl/catalog_page.sql
deleted file mode 100644
index 7b06b67fa5..0000000000
--- a/regression-test/suites/cte_reuse/ddl/catalog_page.sql
+++ /dev/null
@@ -1,17 +0,0 @@
-CREATE TABLE IF NOT EXISTS catalog_page (
-    cp_catalog_page_sk bigint,
-    cp_catalog_page_id char(16),
-    cp_start_date_sk integer,
-    cp_end_date_sk integer,
-    cp_department varchar(50),
-    cp_catalog_number integer,
-    cp_catalog_page_number integer,
-    cp_description varchar(100),
-    cp_type varchar(100)
-)
-DUPLICATE KEY(cp_catalog_page_sk, cp_catalog_page_id)
-DISTRIBUTED BY HASH(cp_catalog_page_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/catalog_returns.sql b/regression-test/suites/cte_reuse/ddl/catalog_returns.sql
deleted file mode 100644
index 56a335624f..0000000000
--- a/regression-test/suites/cte_reuse/ddl/catalog_returns.sql
+++ /dev/null
@@ -1,34 +0,0 @@
-CREATE TABLE IF NOT EXISTS catalog_returns (
-    cr_returned_date_sk bigint,
-    cr_returned_time_sk bigint,
-    cr_item_sk bigint,
-    cr_refunded_customer_sk bigint,
-    cr_refunded_cdemo_sk bigint,
-    cr_refunded_hdemo_sk bigint,
-    cr_refunded_addr_sk bigint,
-    cr_returning_customer_sk bigint,
-    cr_returning_cdemo_sk bigint,
-    cr_returning_hdemo_sk bigint,
-    cr_returning_addr_sk bigint,
-    cr_call_center_sk bigint,
-    cr_catalog_page_sk bigint,
-    cr_ship_mode_sk bigint,
-    cr_warehouse_sk bigint,
-    cr_reason_sk bigint,
-    cr_order_number bigint,
-    cr_return_quantity integer,
-    cr_return_amount decimal(7,2),
-    cr_return_tax decimal(7,2),
-    cr_return_amt_inc_tax decimal(7,2),
-    cr_fee decimal(7,2),
-    cr_return_ship_cost decimal(7,2),
-    cr_refunded_cash decimal(7,2),
-    cr_reversed_charge decimal(7,2),
-    cr_store_credit decimal(7,2),
-    cr_net_loss decimal(7,2)
-)
-DUPLICATE KEY(cr_returned_date_sk, cr_returned_time_sk, cr_item_sk, cr_refunded_customer_sk)
-DISTRIBUTED BY HASH(cr_refunded_customer_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
diff --git a/regression-test/suites/cte_reuse/ddl/catalog_sales.sql b/regression-test/suites/cte_reuse/ddl/catalog_sales.sql
deleted file mode 100644
index f47e23fd20..0000000000
--- a/regression-test/suites/cte_reuse/ddl/catalog_sales.sql
+++ /dev/null
@@ -1,42 +0,0 @@
-CREATE TABLE IF NOT EXISTS catalog_sales (
-    cs_sold_date_sk bigint,
-    cs_sold_time_sk bigint,
-    cs_ship_date_sk bigint,
-    cs_bill_customer_sk bigint,
-    cs_bill_cdemo_sk bigint,
-    cs_bill_hdemo_sk bigint,
-    cs_bill_addr_sk bigint,
-    cs_ship_customer_sk bigint,
-    cs_ship_cdemo_sk bigint,
-    cs_ship_hdemo_sk bigint,
-    cs_ship_addr_sk bigint,
-    cs_call_center_sk bigint,
-    cs_catalog_page_sk bigint,
-    cs_ship_mode_sk bigint,
-    cs_warehouse_sk bigint,
-    cs_item_sk bigint,
-    cs_promo_sk bigint,
-    cs_order_number bigint,
-    cs_quantity integer,
-    cs_wholesale_cost decimal(7,2),
-    cs_list_price decimal(7,2),
-    cs_sales_price decimal(7,2),
-    cs_ext_discount_amt decimal(7,2),
-    cs_ext_sales_price decimal(7,2),
-    cs_ext_wholesale_cost decimal(7,2),
-    cs_ext_list_price decimal(7,2),
-    cs_ext_tax decimal(7,2),
-    cs_coupon_amt decimal(7,2),
-    cs_ext_ship_cost decimal(7,2),
-    cs_net_paid decimal(7,2),
-    cs_net_paid_inc_tax decimal(7,2),
-    cs_net_paid_inc_ship decimal(7,2),
-    cs_net_paid_inc_ship_tax decimal(7,2),
-    cs_net_profit decimal(7,2)
-)
-DUPLICATE KEY(cs_sold_date_sk, cs_sold_time_sk, cs_ship_date_sk, cs_bill_customer_sk)
-DISTRIBUTED BY HASH(cs_bill_customer_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/customer.sql b/regression-test/suites/cte_reuse/ddl/customer.sql
deleted file mode 100644
index debf5526da..0000000000
--- a/regression-test/suites/cte_reuse/ddl/customer.sql
+++ /dev/null
@@ -1,26 +0,0 @@
-CREATE TABLE IF NOT EXISTS customer (
-    c_customer_sk bigint,
-    c_customer_id char(16),
-    c_current_cdemo_sk bigint,
-    c_current_hdemo_sk bigint,
-    c_current_addr_sk bigint,
-    c_first_shipto_date_sk bigint,
-    c_first_sales_date_sk bigint,
-    c_salutation char(10),
-    c_first_name char(20),
-    c_last_name char(30),
-    c_preferred_cust_flag char(1),
-    c_birth_day integer,
-    c_birth_month integer,
-    c_birth_year integer,
-    c_birth_country varchar(20),
-    c_login char(13),
-    c_email_address char(50),
-    c_last_review_date_sk bigint
-)
-DUPLICATE KEY(c_customer_sk, c_customer_id)
-DISTRIBUTED BY HASH(c_customer_id) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/customer_address.sql b/regression-test/suites/cte_reuse/ddl/customer_address.sql
deleted file mode 100644
index 4c9f849b22..0000000000
--- a/regression-test/suites/cte_reuse/ddl/customer_address.sql
+++ /dev/null
@@ -1,21 +0,0 @@
-CREATE TABLE IF NOT EXISTS customer_address (
-    ca_address_sk bigint,
-    ca_address_id char(16),
-    ca_street_number char(10),
-    ca_street_name varchar(60),
-    ca_street_type char(15),
-    ca_suite_number char(10),
-    ca_city varchar(60),
-    ca_county varchar(30),
-    ca_state char(2),
-    ca_zip char(10),
-    ca_country varchar(20),
-    ca_gmt_offset decimal(5,2),
-    ca_location_type char(20)
-)
-DUPLICATE KEY(ca_address_sk, ca_address_id)
-DISTRIBUTED BY HASH(ca_address_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/customer_demographics.sql b/regression-test/suites/cte_reuse/ddl/customer_demographics.sql
deleted file mode 100644
index cc5f03f8db..0000000000
--- a/regression-test/suites/cte_reuse/ddl/customer_demographics.sql
+++ /dev/null
@@ -1,16 +0,0 @@
-CREATE TABLE IF NOT EXISTS customer_demographics (
-    cd_demo_sk bigint,
-    cd_gender char(1),
-    cd_marital_status char(1),
-    cd_education_status char(20),
-    cd_purchase_estimate integer,
-    cd_credit_rating char(10),
-    cd_dep_count integer,
-    cd_dep_employed_count integer,
-    cd_dep_college_count integer
-)
-DUPLICATE KEY(cd_demo_sk, cd_gender)
-DISTRIBUTED BY HASH(cd_gender) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
diff --git a/regression-test/suites/cte_reuse/ddl/date_dim.sql b/regression-test/suites/cte_reuse/ddl/date_dim.sql
deleted file mode 100644
index ff77c7e434..0000000000
--- a/regression-test/suites/cte_reuse/ddl/date_dim.sql
+++ /dev/null
@@ -1,35 +0,0 @@
-CREATE TABLE IF NOT EXISTS date_dim (
-    d_date_sk bigint,
-    d_date_id char(16),
-    d_date date,
-    d_month_seq integer,
-    d_week_seq integer,
-    d_quarter_seq integer,
-    d_year integer,
-    d_dow integer,
-    d_moy integer,
-    d_dom integer,
-    d_qoy integer,
-    d_fy_year integer,
-    d_fy_quarter_seq integer,
-    d_fy_week_seq integer,
-    d_day_name char(9),
-    d_quarter_name char(6),
-    d_holiday char(1),
-    d_weekend char(1),
-    d_following_holiday char(1),
-    d_first_dom integer,
-    d_last_dom integer,
-    d_same_day_ly integer,
-    d_same_day_lq integer,
-    d_current_day char(1),
-    d_current_week char(1),
-    d_current_month char(1),
-    d_current_quarter char(1),
-    d_current_year char(1)
-)
-DUPLICATE KEY(d_date_sk, d_date_id)
-DISTRIBUTED BY HASH(d_date_id) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
diff --git a/regression-test/suites/cte_reuse/ddl/household_demographics.sql b/regression-test/suites/cte_reuse/ddl/household_demographics.sql
deleted file mode 100644
index f050e2a128..0000000000
--- a/regression-test/suites/cte_reuse/ddl/household_demographics.sql
+++ /dev/null
@@ -1,13 +0,0 @@
-CREATE TABLE IF NOT EXISTS household_demographics (
-    hd_demo_sk bigint,
-    hd_income_band_sk bigint,
-    hd_buy_potential char(15),
-    hd_dep_count integer,
-    hd_vehicle_count integer
-)
-DUPLICATE KEY(hd_demo_sk, hd_income_band_sk)
-DISTRIBUTED BY HASH(hd_demo_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/income_band.sql b/regression-test/suites/cte_reuse/ddl/income_band.sql
deleted file mode 100644
index 98f0d8cf64..0000000000
--- a/regression-test/suites/cte_reuse/ddl/income_band.sql
+++ /dev/null
@@ -1,11 +0,0 @@
-CREATE TABLE IF NOT EXISTS income_band (
-    ib_income_band_sk bigint,
-    ib_lower_bound integer,
-    ib_upper_bound integer
-)
-DUPLICATE KEY(ib_income_band_sk)
-DISTRIBUTED BY HASH(ib_income_band_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/inventory.sql b/regression-test/suites/cte_reuse/ddl/inventory.sql
deleted file mode 100644
index b4bcb1e931..0000000000
--- a/regression-test/suites/cte_reuse/ddl/inventory.sql
+++ /dev/null
@@ -1,12 +0,0 @@
-CREATE TABLE IF NOT EXISTS inventory (
-    inv_date_sk bigint,
-    inv_item_sk bigint,
-    inv_warehouse_sk bigint,
-    inv_quantity_on_hand integer
-)
-DUPLICATE KEY(inv_date_sk, inv_item_sk)
-DISTRIBUTED BY HASH(inv_warehouse_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/item.sql b/regression-test/suites/cte_reuse/ddl/item.sql
deleted file mode 100644
index 4b9ac8f53c..0000000000
--- a/regression-test/suites/cte_reuse/ddl/item.sql
+++ /dev/null
@@ -1,29 +0,0 @@
-CREATE TABLE IF NOT EXISTS item (
-    i_item_sk bigint,
-    i_item_id char(16),
-    i_rec_start_date date,
-    i_rec_end_date date,
-    i_item_desc varchar(200),
-    i_current_price decimal(7,2),
-    i_wholesale_cost decimal(7,2),
-    i_brand_id integer,
-    i_brand char(50),
-    i_class_id integer,
-    i_class char(50),
-    i_category_id integer,
-    i_category char(50),
-    i_manufact_id integer,
-    i_manufact char(50),
-    i_size char(20),
-    i_formulation char(20),
-    i_color char(20),
-    i_units char(10),
-    i_container char(10),
-    i_manager_id integer,
-    i_product_name char(50)
-)
-DUPLICATE KEY(i_item_sk, i_item_id)
-DISTRIBUTED BY HASH(i_item_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
diff --git a/regression-test/suites/cte_reuse/ddl/promotion.sql b/regression-test/suites/cte_reuse/ddl/promotion.sql
deleted file mode 100644
index 4a8d2ac5c3..0000000000
--- a/regression-test/suites/cte_reuse/ddl/promotion.sql
+++ /dev/null
@@ -1,27 +0,0 @@
-CREATE TABLE IF NOT EXISTS promotion (
-    p_promo_sk bigint,
-    p_promo_id char(16),
-    p_start_date_sk bigint,
-    p_end_date_sk bigint,
-    p_item_sk bigint,
-    p_cost decimal(15,2),
-    p_response_targe integer,
-    p_promo_name char(50),
-    p_channel_dmail char(1),
-    p_channel_email char(1),
-    p_channel_catalog char(1),
-    p_channel_tv char(1),
-    p_channel_radio char(1),
-    p_channel_press char(1),
-    p_channel_event char(1),
-    p_channel_demo char(1),
-    p_channel_details varchar(100),
-    p_purpose char(15),
-    p_discount_active char(1)
-)
-DUPLICATE KEY(p_promo_sk, p_promo_id)
-DISTRIBUTED BY HASH(p_promo_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/reason.sql b/regression-test/suites/cte_reuse/ddl/reason.sql
deleted file mode 100644
index 8fb573615e..0000000000
--- a/regression-test/suites/cte_reuse/ddl/reason.sql
+++ /dev/null
@@ -1,11 +0,0 @@
-CREATE TABLE IF NOT EXISTS reason (
-    r_reason_sk bigint,
-    r_reason_id char(16),
-    r_reason_desc char(100)
- )
-DUPLICATE KEY(r_reason_sk, r_reason_id)
-DISTRIBUTED BY HASH(r_reason_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/ship_mode.sql b/regression-test/suites/cte_reuse/ddl/ship_mode.sql
deleted file mode 100644
index 6ccaf51b57..0000000000
--- a/regression-test/suites/cte_reuse/ddl/ship_mode.sql
+++ /dev/null
@@ -1,14 +0,0 @@
-CREATE TABLE IF NOT EXISTS ship_mode (
-    sm_ship_mode_sk bigint,
-    sm_ship_mode_id char(16),
-    sm_type char(30),
-    sm_code char(10),
-    sm_carrier char(20),
-    sm_contract char(20)
-)
-DUPLICATE KEY(sm_ship_mode_sk, sm_ship_mode_id)
-DISTRIBUTED BY HASH(sm_ship_mode_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/store.sql b/regression-test/suites/cte_reuse/ddl/store.sql
deleted file mode 100644
index 57cac7ad52..0000000000
--- a/regression-test/suites/cte_reuse/ddl/store.sql
+++ /dev/null
@@ -1,36 +0,0 @@
-CREATE TABLE IF NOT EXISTS store (
-    s_store_sk bigint,
-    s_store_id char(16),
-    s_rec_start_date date,
-    s_rec_end_date date,
-    s_closed_date_sk bigint,
-    s_store_name varchar(50),
-    s_number_employees integer,
-    s_floor_space integer,
-    s_hours char(20),
-    s_manager varchar(40),
-    s_market_id integer,
-    s_geography_class varchar(100),
-    s_market_desc varchar(100),
-    s_market_manager varchar(40),
-    s_division_id integer,
-    s_division_name varchar(50),
-    s_company_id integer,
-    s_company_name varchar(50),
-    s_street_number varchar(10),
-    s_street_name varchar(60),
-    s_street_type char(15),
-    s_suite_number char(10),
-    s_city varchar(60),
-    s_county varchar(30),
-    s_state char(2),
-    s_zip char(10),
-    s_country varchar(20),
-    s_gmt_offset decimal(5,2),
-    s_tax_precentage decimal(5,2)
-)
-DUPLICATE KEY(s_store_sk, s_store_id)
-DISTRIBUTED BY HASH(s_store_id) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
diff --git a/regression-test/suites/cte_reuse/ddl/store_returns.sql b/regression-test/suites/cte_reuse/ddl/store_returns.sql
deleted file mode 100644
index 8c994f4f06..0000000000
--- a/regression-test/suites/cte_reuse/ddl/store_returns.sql
+++ /dev/null
@@ -1,28 +0,0 @@
-CREATE TABLE IF NOT EXISTS store_returns (
-    sr_returned_date_sk bigint,
-    sr_return_time_sk bigint,
-    sr_item_sk bigint,
-    sr_customer_sk bigint,
-    sr_cdemo_sk bigint,
-    sr_hdemo_sk bigint,
-    sr_addr_sk bigint,
-    sr_store_sk bigint,
-    sr_reason_sk bigint,
-    sr_ticket_number bigint,
-    sr_return_quantity integer,
-    sr_return_amt decimal(7,2),
-    sr_return_tax decimal(7,2),
-    sr_return_amt_inc_tax decimal(7,2),
-    sr_fee decimal(7,2),
-    sr_return_ship_cost decimal(7,2),
-    sr_refunded_cash decimal(7,2),
-    sr_reversed_charge decimal(7,2),
-    sr_store_credit decimal(7,2),
-    sr_net_loss decimal(7,2)
-)
-DUPLICATE KEY(sr_returned_date_sk, sr_return_time_sk, sr_item_sk)
-DISTRIBUTED BY HASH(sr_return_time_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/store_sales.sql b/regression-test/suites/cte_reuse/ddl/store_sales.sql
deleted file mode 100644
index 81003befb9..0000000000
--- a/regression-test/suites/cte_reuse/ddl/store_sales.sql
+++ /dev/null
@@ -1,32 +0,0 @@
-CREATE TABLE IF NOT EXISTS store_sales (
-    ss_sold_date_sk bigint,
-    ss_sold_time_sk bigint,
-    ss_item_sk bigint,
-    ss_customer_sk bigint,
-    ss_cdemo_sk bigint,
-    ss_hdemo_sk bigint,
-    ss_addr_sk bigint,
-    ss_store_sk bigint,
-    ss_promo_sk bigint,
-    ss_ticket_number bigint,
-    ss_quantity integer,
-    ss_wholesale_cost decimal(7,2),
-    ss_list_price decimal(7,2),
-    ss_sales_price decimal(7,2),
-    ss_ext_discount_amt decimal(7,2),
-    ss_ext_sales_price decimal(7,2),
-    ss_ext_wholesale_cost decimal(7,2),
-    ss_ext_list_price decimal(7,2),
-    ss_ext_tax decimal(7,2),
-    ss_coupon_amt decimal(7,2),
-    ss_net_paid decimal(7,2),
-    ss_net_paid_inc_tax decimal(7,2),
-    ss_net_profit decimal(7,2)
-)
-DUPLICATE KEY(ss_sold_date_sk, ss_sold_time_sk, ss_item_sk, ss_customer_sk)
-DISTRIBUTED BY HASH(ss_customer_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
-
diff --git a/regression-test/suites/cte_reuse/ddl/time_dim.sql b/regression-test/suites/cte_reuse/ddl/time_dim.sql
deleted file mode 100644
index c302f0d5c3..0000000000
--- a/regression-test/suites/cte_reuse/ddl/time_dim.sql
+++ /dev/null
@@ -1,17 +0,0 @@
-CREATE TABLE IF NOT EXISTS time_dim (
-    t_time_sk bigint,
-    t_time_id char(16),
-    t_time integer,
-    t_hour integer,
-    t_minute integer,
-    t_second integer,
-    t_am_pm char(2),
-    t_shift char(20),
-    t_sub_shift char(20),
-    t_meal_time char(20)
-)
-DUPLICATE KEY(t_time_sk, t_time_id)
-DISTRIBUTED BY HASH(t_time_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
diff --git a/regression-test/suites/cte_reuse/ddl/warehouse.sql b/regression-test/suites/cte_reuse/ddl/warehouse.sql
deleted file mode 100644
index 9e88325b8c..0000000000
--- a/regression-test/suites/cte_reuse/ddl/warehouse.sql
+++ /dev/null
@@ -1,22 +0,0 @@
-CREATE TABLE IF NOT EXISTS warehouse (
-    w_warehouse_sk bigint,
-    w_warehouse_id char(16),
-    w_warehouse_name varchar(20),
-    w_warehouse_sq_ft integer,
-    w_street_number char(10),
-    w_street_name varchar(60),
-    w_street_type char(15),
-    w_suite_number char(10),
-    w_city varchar(60),
-    w_county varchar(30),
-    w_state char(2),
-    w_zip char(10),
-    w_country varchar(20),
-    w_gmt_offset decimal(5,2)
-)
-DUPLICATE KEY(w_warehouse_sk, w_warehouse_id)
-DISTRIBUTED BY HASH(w_warehouse_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/web_page.sql b/regression-test/suites/cte_reuse/ddl/web_page.sql
deleted file mode 100644
index ce59ec8319..0000000000
--- a/regression-test/suites/cte_reuse/ddl/web_page.sql
+++ /dev/null
@@ -1,21 +0,0 @@
-CREATE TABLE IF NOT EXISTS web_page (
-        wp_web_page_sk bigint,
-        wp_web_page_id char(16),
-        wp_rec_start_date date,
-        wp_rec_end_date date,
-        wp_creation_date_sk bigint,
-        wp_access_date_sk bigint,
-        wp_autogen_flag char(1),
-        wp_customer_sk bigint,
-        wp_url varchar(100),
-        wp_type char(50),
-        wp_char_count integer,
-        wp_link_count integer,
-        wp_image_count integer,
-        wp_max_ad_count integer
-)
-DUPLICATE KEY(wp_web_page_sk, wp_web_page_id)
-DISTRIBUTED BY HASH(wp_web_page_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
diff --git a/regression-test/suites/cte_reuse/ddl/web_returns.sql b/regression-test/suites/cte_reuse/ddl/web_returns.sql
deleted file mode 100644
index 8552e2634a..0000000000
--- a/regression-test/suites/cte_reuse/ddl/web_returns.sql
+++ /dev/null
@@ -1,31 +0,0 @@
-CREATE TABLE IF NOT EXISTS web_returns (
-    wr_returned_date_sk bigint,
-    wr_returned_time_sk bigint,
-    wr_item_sk bigint,
-    wr_refunded_customer_sk bigint,
-    wr_refunded_cdemo_sk bigint,
-    wr_refunded_hdemo_sk bigint,
-    wr_refunded_addr_sk bigint,
-    wr_returning_customer_sk bigint,
-    wr_returning_cdemo_sk bigint,
-    wr_returning_hdemo_sk bigint,
-    wr_returning_addr_sk bigint,
-    wr_web_page_sk bigint,
-    wr_reason_sk bigint,
-    wr_order_number bigint,
-    wr_return_quantity integer,
-    wr_return_amt decimal(7,2),
-    wr_return_tax decimal(7,2),
-    wr_return_amt_inc_tax decimal(7,2),
-    wr_fee decimal(7,2),
-    wr_return_ship_cost decimal(7,2),
-    wr_refunded_cash decimal(7,2),
-    wr_reversed_charge decimal(7,2),
-    wr_account_credit decimal(7,2),
-    wr_net_loss decimal(7,2)
-)
-DUPLICATE KEY(wr_returned_date_sk, wr_returned_time_sk, wr_item_sk, wr_refunded_customer_sk)
-DISTRIBUTED BY HASH(wr_refunded_customer_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
diff --git a/regression-test/suites/cte_reuse/ddl/web_sales.sql b/regression-test/suites/cte_reuse/ddl/web_sales.sql
deleted file mode 100644
index 807a116896..0000000000
--- a/regression-test/suites/cte_reuse/ddl/web_sales.sql
+++ /dev/null
@@ -1,42 +0,0 @@
-CREATE TABLE IF NOT EXISTS web_sales (
-    ws_sold_date_sk bigint,
-    ws_sold_time_sk bigint,
-    ws_ship_date_sk bigint,
-    ws_item_sk bigint,
-    ws_bill_customer_sk bigint,
-    ws_bill_cdemo_sk bigint,
-    ws_bill_hdemo_sk bigint,
-    ws_bill_addr_sk bigint,
-    ws_ship_customer_sk bigint,
-    ws_ship_cdemo_sk bigint,
-    ws_ship_hdemo_sk bigint,
-    ws_ship_addr_sk bigint,
-    ws_web_page_sk bigint,
-    ws_web_site_sk bigint,
-    ws_ship_mode_sk bigint,
-    ws_warehouse_sk bigint,
-    ws_promo_sk bigint,
-    ws_order_number bigint,
-    ws_quantity integer,
-    ws_wholesale_cost decimal(7,2),
-    ws_list_price decimal(7,2),
-    ws_sales_price decimal(7,2),
-    ws_ext_discount_amt decimal(7,2),
-    ws_ext_sales_price decimal(7,2),
-    ws_ext_wholesale_cost decimal(7,2),
-    ws_ext_list_price decimal(7,2),
-    ws_ext_tax decimal(7,2),
-    ws_coupon_amt decimal(7,2),
-    ws_ext_ship_cost decimal(7,2),
-    ws_net_paid decimal(7,2),
-    ws_net_paid_inc_tax decimal(7,2),
-    ws_net_paid_inc_ship decimal(7,2),
-    ws_net_paid_inc_ship_tax decimal(7,2),
-    ws_net_profit decimal(7,2)
-)
-DUPLICATE KEY(ws_sold_date_sk, ws_sold_time_sk, ws_ship_date_sk, ws_item_sk)
-DISTRIBUTED BY HASH(ws_item_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
diff --git a/regression-test/suites/cte_reuse/ddl/web_site.sql b/regression-test/suites/cte_reuse/ddl/web_site.sql
deleted file mode 100644
index 316aaffb21..0000000000
--- a/regression-test/suites/cte_reuse/ddl/web_site.sql
+++ /dev/null
@@ -1,36 +0,0 @@
-CREATE TABLE IF NOT EXISTS web_site (
-    web_site_sk bigint,
-    web_site_id char(16),
-    web_rec_start_date date,
-    web_rec_end_date date,
-    web_name varchar(50),
-    web_open_date_sk bigint,
-    web_close_date_sk bigint,
-    web_class varchar(50),
-    web_manager varchar(40),
-    web_mkt_id integer,
-    web_mkt_class varchar(50),
-    web_mkt_desc varchar(100),
-    web_market_manager varchar(40),
-    web_company_id integer,
-    web_company_name char(50),
-    web_street_number char(10),
-    web_street_name varchar(60),
-    web_street_type char(15),
-    web_suite_number char(10),
-    web_city varchar(60),
-    web_county varchar(30),
-    web_state char(2),
-    web_zip char(10),
-    web_country varchar(20),
-    web_gmt_offset decimal(5,2),
-    web_tax_percentage decimal(5,2)
-)
-DUPLICATE KEY(web_site_sk, web_site_id)
-DISTRIBUTED BY HASH(web_site_sk) BUCKETS 3
-PROPERTIES (
-  "replication_num" = "1"
-)
-
-
-
diff --git a/regression-test/suites/cte_reuse/load.groovy b/regression-test/suites/cte_reuse/load.groovy
deleted file mode 100644
index 14aa1c19ef..0000000000
--- a/regression-test/suites/cte_reuse/load.groovy
+++ /dev/null
@@ -1,4228 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-suite("load") {
-    String database = context.config.getDbNameByFile(context.file)
-    sql "drop database if exists ${database}"
-    sql "create database ${database}"
-    sql "use ${database}"
-
-    sql '''
-    drop table if exists customer_demographics
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS customer_demographics (
-        cd_demo_sk bigint not null,
-        cd_gender char(1),
-        cd_marital_status char(1),
-        cd_education_status char(20),
-        cd_purchase_estimate integer,
-        cd_credit_rating char(10),
-        cd_dep_count integer,
-        cd_dep_employed_count integer,
-        cd_dep_college_count integer
-    )
-    DUPLICATE KEY(cd_demo_sk)
-    DISTRIBUTED BY HASH(cd_gender) BUCKETS 12
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists reason
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS reason (
-        r_reason_sk bigint not null,
-        r_reason_id char(16) not null,
-        r_reason_desc char(100)
-    )
-    DUPLICATE KEY(r_reason_sk)
-    DISTRIBUTED BY HASH(r_reason_sk) BUCKETS 1
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists date_dim
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS date_dim (
-        d_date_sk bigint not null,
-        d_date_id char(16) not null,
-        d_date datev2,
-        d_month_seq integer,
-        d_week_seq integer,
-        d_quarter_seq integer,
-        d_year integer,
-        d_dow integer,
-        d_moy integer,
-        d_dom integer,
-        d_qoy integer,
-        d_fy_year integer,
-        d_fy_quarter_seq integer,
-        d_fy_week_seq integer,
-        d_day_name char(9),
-        d_quarter_name char(6),
-        d_holiday char(1),
-        d_weekend char(1),
-        d_following_holiday char(1),
-        d_first_dom integer,
-        d_last_dom integer,
-        d_same_day_ly integer,
-        d_same_day_lq integer,
-        d_current_day char(1),
-        d_current_week char(1),
-        d_current_month char(1),
-        d_current_quarter char(1),
-        d_current_year char(1)
-    )
-    DUPLICATE KEY(d_date_sk)
-    DISTRIBUTED BY HASH(d_date_sk) BUCKETS 12
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists warehouse
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS warehouse (
-        w_warehouse_sk bigint not null,
-        w_warehouse_id char(16) not null,
-        w_warehouse_name varchar(20),
-        w_warehouse_sq_ft integer,
-        w_street_number char(10),
-        w_street_name varchar(60),
-        w_street_type char(15),
-        w_suite_number char(10),
-        w_city varchar(60),
-        w_county varchar(30),
-        w_state char(2),
-        w_zip char(10),
-        w_country varchar(20),
-        w_gmt_offset decimalv3(5,2)
-    )
-    DUPLICATE KEY(w_warehouse_sk)
-    DISTRIBUTED BY HASH(w_warehouse_sk) BUCKETS 1
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists catalog_sales
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS catalog_sales (
-        cs_sold_date_sk bigint,
-        cs_item_sk bigint not null,
-        cs_order_number bigint not null,
-        cs_sold_time_sk bigint,
-        cs_ship_date_sk bigint,
-        cs_bill_customer_sk bigint,
-        cs_bill_cdemo_sk bigint,
-        cs_bill_hdemo_sk bigint,
-        cs_bill_addr_sk bigint,
-        cs_ship_customer_sk bigint,
-        cs_ship_cdemo_sk bigint,
-        cs_ship_hdemo_sk bigint,
-        cs_ship_addr_sk bigint,
-        cs_call_center_sk bigint,
-        cs_catalog_page_sk bigint,
-        cs_ship_mode_sk bigint,
-        cs_warehouse_sk bigint,
-        cs_promo_sk bigint,
-        cs_quantity integer,
-        cs_wholesale_cost decimalv3(7,2),
-        cs_list_price decimalv3(7,2),
-        cs_sales_price decimalv3(7,2),
-        cs_ext_discount_amt decimalv3(7,2),
-        cs_ext_sales_price decimalv3(7,2),
-        cs_ext_wholesale_cost decimalv3(7,2),
-        cs_ext_list_price decimalv3(7,2),
-        cs_ext_tax decimalv3(7,2),
-        cs_coupon_amt decimalv3(7,2),
-        cs_ext_ship_cost decimalv3(7,2),
-        cs_net_paid decimalv3(7,2),
-        cs_net_paid_inc_tax decimalv3(7,2),
-        cs_net_paid_inc_ship decimalv3(7,2),
-        cs_net_paid_inc_ship_tax decimalv3(7,2),
-        cs_net_profit decimalv3(7,2)
-    )
-    DUPLICATE KEY(cs_sold_date_sk, cs_item_sk)
-    DISTRIBUTED BY HASH(cs_item_sk, cs_order_number) BUCKETS 32
-    PROPERTIES (
-    "replication_num" = "1",
-    "colocate_with" = "catalog"
-    )
-    '''
-
-    sql '''
-    drop table if exists call_center
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS call_center (
-    cc_call_center_sk bigint not null,
-    cc_call_center_id char(16) not null,
-    cc_rec_start_date datev2,
-    cc_rec_end_date datev2,
-    cc_closed_date_sk integer,
-    cc_open_date_sk integer,
-    cc_name varchar(50),
-    cc_class varchar(50),
-    cc_employees integer,
-    cc_sq_ft integer,
-    cc_hours char(20),
-    cc_manager varchar(40),
-    cc_mkt_id integer,
-    cc_mkt_class char(50),
-    cc_mkt_desc varchar(100),
-    cc_market_manager varchar(40),
-    cc_division integer,
-    cc_division_name varchar(50),
-    cc_company integer,
-    cc_company_name char(50),
-    cc_street_number char(10),
-    cc_street_name varchar(60),
-    cc_street_type char(15),
-    cc_suite_number char(10),
-    cc_city varchar(60),
-    cc_county varchar(30),
-    cc_state char(2),
-    cc_zip char(10),
-    cc_country varchar(20),
-    cc_gmt_offset decimalv3(5,2),
-    cc_tax_percentage decimalv3(5,2)
-    )
-    DUPLICATE KEY(cc_call_center_sk)
-    DISTRIBUTED BY HASH(cc_call_center_sk) BUCKETS 1
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists inventory
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS inventory (
-        inv_date_sk bigint not null,
-        inv_item_sk bigint not null,
-        inv_warehouse_sk bigint,
-        inv_quantity_on_hand integer
-    )
-    DUPLICATE KEY(inv_date_sk, inv_item_sk, inv_warehouse_sk)
-    DISTRIBUTED BY HASH(inv_date_sk, inv_item_sk, inv_warehouse_sk) BUCKETS 32
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists catalog_returns
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS catalog_returns (
-    cr_item_sk bigint not null,
-    cr_order_number bigint not null,
-    cr_returned_date_sk bigint,
-    cr_returned_time_sk bigint,
-    cr_refunded_customer_sk bigint,
-    cr_refunded_cdemo_sk bigint,
-    cr_refunded_hdemo_sk bigint,
-    cr_refunded_addr_sk bigint,
-    cr_returning_customer_sk bigint,
-    cr_returning_cdemo_sk bigint,
-    cr_returning_hdemo_sk bigint,
-    cr_returning_addr_sk bigint,
-    cr_call_center_sk bigint,
-    cr_catalog_page_sk bigint,
-    cr_ship_mode_sk bigint,
-    cr_warehouse_sk bigint,
-    cr_reason_sk bigint,
-    cr_return_quantity integer,
-    cr_return_amount decimalv3(7,2),
-    cr_return_tax decimalv3(7,2),
-    cr_return_amt_inc_tax decimalv3(7,2),
-    cr_fee decimalv3(7,2),
-    cr_return_ship_cost decimalv3(7,2),
-    cr_refunded_cash decimalv3(7,2),
-    cr_reversed_charge decimalv3(7,2),
-    cr_store_credit decimalv3(7,2),
-    cr_net_loss decimalv3(7,2)
-    )
-    DUPLICATE KEY(cr_item_sk, cr_order_number)
-    DISTRIBUTED BY HASH(cr_item_sk, cr_order_number) BUCKETS 32
-    PROPERTIES (
-    "replication_num" = "1",
-    "colocate_with" = "catalog"
-    )
-    '''
-
-    sql '''
-    drop table if exists household_demographics
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS household_demographics (
-        hd_demo_sk bigint not null,
-        hd_income_band_sk bigint,
-        hd_buy_potential char(15),
-        hd_dep_count integer,
-        hd_vehicle_count integer
-    )
-    DUPLICATE KEY(hd_demo_sk)
-    DISTRIBUTED BY HASH(hd_demo_sk) BUCKETS 3
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists customer_address
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS customer_address (
-        ca_address_sk bigint not null,
-        ca_address_id char(16) not null,
-        ca_street_number char(10),
-        ca_street_name varchar(60),
-        ca_street_type char(15),
-        ca_suite_number char(10),
-        ca_city varchar(60),
-        ca_county varchar(30),
-        ca_state char(2),
-        ca_zip char(10),
-        ca_country varchar(20),
-        ca_gmt_offset decimalv3(5,2),
-        ca_location_type char(20)
-    )
-    DUPLICATE KEY(ca_address_sk)
-    DISTRIBUTED BY HASH(ca_address_sk) BUCKETS 12
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists income_band
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS income_band (
-        ib_income_band_sk bigint not null,
-        ib_lower_bound integer,
-        ib_upper_bound integer
-    )
-    DUPLICATE KEY(ib_income_band_sk)
-    DISTRIBUTED BY HASH(ib_income_band_sk) BUCKETS 1
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists catalog_page
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS catalog_page (
-    cp_catalog_page_sk bigint not null,
-    cp_catalog_page_id char(16) not null,
-    cp_start_date_sk integer,
-    cp_end_date_sk integer,
-    cp_department varchar(50),
-    cp_catalog_number integer,
-    cp_catalog_page_number integer,
-    cp_description varchar(100),
-    cp_type varchar(100)
-    )
-    DUPLICATE KEY(cp_catalog_page_sk)
-    DISTRIBUTED BY HASH(cp_catalog_page_sk) BUCKETS 3
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists item
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS item (
-        i_item_sk bigint not null,
-        i_item_id char(16) not null,
-        i_rec_start_date datev2,
-        i_rec_end_date datev2,
-        i_item_desc varchar(200),
-        i_current_price decimalv3(7,2),
-        i_wholesale_cost decimalv3(7,2),
-        i_brand_id integer,
-        i_brand char(50),
-        i_class_id integer,
-        i_class char(50),
-        i_category_id integer,
-        i_category char(50),
-        i_manufact_id integer,
-        i_manufact char(50),
-        i_size char(20),
-        i_formulation char(20),
-        i_color char(20),
-        i_units char(10),
-        i_container char(10),
-        i_manager_id integer,
-        i_product_name char(50)
-    )
-    DUPLICATE KEY(i_item_sk)
-    DISTRIBUTED BY HASH(i_item_sk) BUCKETS 12
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists web_returns
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS web_returns (
-        wr_item_sk bigint not null,
-        wr_order_number bigint not null,
-        wr_returned_date_sk bigint,
-        wr_returned_time_sk bigint,
-        wr_refunded_customer_sk bigint,
-        wr_refunded_cdemo_sk bigint,
-        wr_refunded_hdemo_sk bigint,
-        wr_refunded_addr_sk bigint,
-        wr_returning_customer_sk bigint,
-        wr_returning_cdemo_sk bigint,
-        wr_returning_hdemo_sk bigint,
-        wr_returning_addr_sk bigint,
-        wr_web_page_sk bigint,
-        wr_reason_sk bigint,
-        wr_return_quantity integer,
-        wr_return_amt decimalv3(7,2),
-        wr_return_tax decimalv3(7,2),
-        wr_return_amt_inc_tax decimalv3(7,2),
-        wr_fee decimalv3(7,2),
-        wr_return_ship_cost decimalv3(7,2),
-        wr_refunded_cash decimalv3(7,2),
-        wr_reversed_charge decimalv3(7,2),
-        wr_account_credit decimalv3(7,2),
-        wr_net_loss decimalv3(7,2)
-    )
-    DUPLICATE KEY(wr_item_sk, wr_order_number)
-    DISTRIBUTED BY HASH(wr_item_sk, wr_order_number) BUCKETS 32
-    PROPERTIES (
-    "replication_num" = "1",
-    "colocate_with" = "web"
-    )
-    '''
-
-    sql '''
-    drop table if exists web_site
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS web_site (
-        web_site_sk bigint not null,
-        web_site_id char(16) not null,
-        web_rec_start_date datev2,
-        web_rec_end_date datev2,
-        web_name varchar(50),
-        web_open_date_sk bigint,
-        web_close_date_sk bigint,
-        web_class varchar(50),
-        web_manager varchar(40),
-        web_mkt_id integer,
-        web_mkt_class varchar(50),
-        web_mkt_desc varchar(100),
-        web_market_manager varchar(40),
-        web_company_id integer,
-        web_company_name char(50),
-        web_street_number char(10),
-        web_street_name varchar(60),
-        web_street_type char(15),
-        web_suite_number char(10),
-        web_city varchar(60),
-        web_county varchar(30),
-        web_state char(2),
-        web_zip char(10),
-        web_country varchar(20),
-        web_gmt_offset decimalv3(5,2),
-        web_tax_percentage decimalv3(5,2)
-    )
-    DUPLICATE KEY(web_site_sk)
-    DISTRIBUTED BY HASH(web_site_sk) BUCKETS 1
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists promotion
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS promotion (
-        p_promo_sk bigint not null,
-        p_promo_id char(16) not null,
-        p_start_date_sk bigint,
-        p_end_date_sk bigint,
-        p_item_sk bigint,
-        p_cost decimalv3(15,2),
-        p_response_targe integer,
-        p_promo_name char(50),
-        p_channel_dmail char(1),
-        p_channel_email char(1),
-        p_channel_catalog char(1),
-        p_channel_tv char(1),
-        p_channel_radio char(1),
-        p_channel_press char(1),
-        p_channel_event char(1),
-        p_channel_demo char(1),
-        p_channel_details varchar(100),
-        p_purpose char(15),
-        p_discount_active char(1)
-    )
-    DUPLICATE KEY(p_promo_sk)
-    DISTRIBUTED BY HASH(p_promo_sk) BUCKETS 1
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists web_sales
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS web_sales (
-        ws_sold_date_sk bigint,
-        ws_item_sk bigint not null,
-        ws_order_number bigint not null,
-        ws_sold_time_sk bigint,
-        ws_ship_date_sk bigint,
-        ws_bill_customer_sk bigint,
-        ws_bill_cdemo_sk bigint,
-        ws_bill_hdemo_sk bigint,
-        ws_bill_addr_sk bigint,
-        ws_ship_customer_sk bigint,
-        ws_ship_cdemo_sk bigint,
-        ws_ship_hdemo_sk bigint,
-        ws_ship_addr_sk bigint,
-        ws_web_page_sk bigint,
-        ws_web_site_sk bigint,
-        ws_ship_mode_sk bigint,
-        ws_warehouse_sk bigint,
-        ws_promo_sk bigint,
-        ws_quantity integer,
-        ws_wholesale_cost decimalv3(7,2),
-        ws_list_price decimalv3(7,2),
-        ws_sales_price decimalv3(7,2),
-        ws_ext_discount_amt decimalv3(7,2),
-        ws_ext_sales_price decimalv3(7,2),
-        ws_ext_wholesale_cost decimalv3(7,2),
-        ws_ext_list_price decimalv3(7,2),
-        ws_ext_tax decimalv3(7,2),
-        ws_coupon_amt decimalv3(7,2),
-        ws_ext_ship_cost decimalv3(7,2),
-        ws_net_paid decimalv3(7,2),
-        ws_net_paid_inc_tax decimalv3(7,2),
-        ws_net_paid_inc_ship decimalv3(7,2),
-        ws_net_paid_inc_ship_tax decimalv3(7,2),
-        ws_net_profit decimalv3(7,2)
-    )
-    DUPLICATE KEY(ws_sold_date_sk, ws_item_sk)
-    DISTRIBUTED BY HASH(ws_item_sk, ws_order_number) BUCKETS 32
-    PROPERTIES (
-    "replication_num" = "1",
-    "colocate_with" = "web"
-    )
-    '''
-
-    sql '''
-    drop table if exists store
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS store (
-        s_store_sk bigint not null,
-        s_store_id char(16) not null,
-        s_rec_start_date datev2,
-        s_rec_end_date datev2,
-        s_closed_date_sk bigint,
-        s_store_name varchar(50),
-        s_number_employees integer,
-        s_floor_space integer,
-        s_hours char(20),
-        s_manager varchar(40),
-        s_market_id integer,
-        s_geography_class varchar(100),
-        s_market_desc varchar(100),
-        s_market_manager varchar(40),
-        s_division_id integer,
-        s_division_name varchar(50),
-        s_company_id integer,
-        s_company_name varchar(50),
-        s_street_number varchar(10),
-        s_street_name varchar(60),
-        s_street_type char(15),
-        s_suite_number char(10),
-        s_city varchar(60),
-        s_county varchar(30),
-        s_state char(2),
-        s_zip char(10),
-        s_country varchar(20),
-        s_gmt_offset decimalv3(5,2),
-        s_tax_precentage decimalv3(5,2)
-    )
-    DUPLICATE KEY(s_store_sk)
-    DISTRIBUTED BY HASH(s_store_sk) BUCKETS 1
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists time_dim
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS time_dim (
-        t_time_sk bigint not null,
-        t_time_id char(16) not null,
-        t_time integer,
-        t_hour integer,
-        t_minute integer,
-        t_second integer,
-        t_am_pm char(2),
-        t_shift char(20),
-        t_sub_shift char(20),
-        t_meal_time char(20)
-    )
-    DUPLICATE KEY(t_time_sk)
-    DISTRIBUTED BY HASH(t_time_sk) BUCKETS 12
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists web_page
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS web_page (
-            wp_web_page_sk bigint not null,
-            wp_web_page_id char(16) not null,
-            wp_rec_start_date datev2,
-            wp_rec_end_date datev2,
-            wp_creation_date_sk bigint,
-            wp_access_date_sk bigint,
-            wp_autogen_flag char(1),
-            wp_customer_sk bigint,
-            wp_url varchar(100),
-            wp_type char(50),
-            wp_char_count integer,
-            wp_link_count integer,
-            wp_image_count integer,
-            wp_max_ad_count integer
-    )
-    DUPLICATE KEY(wp_web_page_sk)
-    DISTRIBUTED BY HASH(wp_web_page_sk) BUCKETS 1
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists store_returns
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS store_returns (
-        sr_item_sk bigint not null,
-        sr_ticket_number bigint not null,
-        sr_returned_date_sk bigint,
-        sr_return_time_sk bigint,
-        sr_customer_sk bigint,
-        sr_cdemo_sk bigint,
-        sr_hdemo_sk bigint,
-        sr_addr_sk bigint,
-        sr_store_sk bigint,
-        sr_reason_sk bigint,
-        sr_return_quantity integer,
-        sr_return_amt decimalv3(7,2),
-        sr_return_tax decimalv3(7,2),
-        sr_return_amt_inc_tax decimalv3(7,2),
-        sr_fee decimalv3(7,2),
-        sr_return_ship_cost decimalv3(7,2),
-        sr_refunded_cash decimalv3(7,2),
-        sr_reversed_charge decimalv3(7,2),
-        sr_store_credit decimalv3(7,2),
-        sr_net_loss decimalv3(7,2)
-    )
-    duplicate key(sr_item_sk, sr_ticket_number)
-    distributed by hash (sr_item_sk, sr_ticket_number) buckets 32
-    properties (
-    "replication_num" = "1",
-    "colocate_with" = "store"
-    )
-    '''
-
-    sql '''
-    drop table if exists store_sales
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS store_sales (
-        ss_sold_date_sk bigint,
-        ss_item_sk bigint not null,
-        ss_ticket_number bigint not null,
-        ss_sold_time_sk bigint,
-        ss_customer_sk bigint,
-        ss_cdemo_sk bigint,
-        ss_hdemo_sk bigint,
-        ss_addr_sk bigint,
-        ss_store_sk bigint,
-        ss_promo_sk bigint,
-        ss_quantity integer,
-        ss_wholesale_cost decimalv3(7,2),
-        ss_list_price decimalv3(7,2),
-        ss_sales_price decimalv3(7,2),
-        ss_ext_discount_amt decimalv3(7,2),
-        ss_ext_sales_price decimalv3(7,2),
-        ss_ext_wholesale_cost decimalv3(7,2),
-        ss_ext_list_price decimalv3(7,2),
-        ss_ext_tax decimalv3(7,2),
-        ss_coupon_amt decimalv3(7,2),
-        ss_net_paid decimalv3(7,2),
-        ss_net_paid_inc_tax decimalv3(7,2),
-        ss_net_profit decimalv3(7,2)
-    )
-    DUPLICATE KEY(ss_sold_date_sk, ss_item_sk)
-    DISTRIBUTED BY HASH(ss_item_sk, ss_ticket_number) BUCKETS 32
-    PROPERTIES (
-    "replication_num" = "1",
-    "colocate_with" = "store"
-    )
-    '''
-
-    sql '''
-    drop table if exists ship_mode
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS ship_mode (
-        sm_ship_mode_sk bigint not null,
-        sm_ship_mode_id char(16) not null,
-        sm_type char(30),
-        sm_code char(10),
-        sm_carrier char(20),
-        sm_contract char(20)
-    )
-    DUPLICATE KEY(sm_ship_mode_sk)
-    DISTRIBUTED BY HASH(sm_ship_mode_sk) BUCKETS 1
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists customer
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS customer (
-        c_customer_sk bigint not null,
-        c_customer_id char(16) not null,
-        c_current_cdemo_sk bigint,
-        c_current_hdemo_sk bigint,
-        c_current_addr_sk bigint,
-        c_first_shipto_date_sk bigint,
-        c_first_sales_date_sk bigint,
-        c_salutation char(10),
-        c_first_name char(20),
-        c_last_name char(30),
-        c_preferred_cust_flag char(1),
-        c_birth_day integer,
-        c_birth_month integer,
-        c_birth_year integer,
-        c_birth_country varchar(20),
-        c_login char(13),
-        c_email_address char(50),
-        c_last_review_date_sk bigint
-    )
-    DUPLICATE KEY(c_customer_sk)
-    DISTRIBUTED BY HASH(c_customer_id) BUCKETS 12
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-    sql '''
-    drop table if exists dbgen_version
-    '''
-
-    sql '''
-    CREATE TABLE IF NOT EXISTS dbgen_version
-    (
-        dv_version                varchar(16)                   ,
-        dv_create_date            datev2                        ,
-        dv_create_time            datetime                      ,
-        dv_cmdline_args           varchar(200)                  
-    )
-    DUPLICATE KEY(dv_version)
-    DISTRIBUTED BY HASH(dv_version) BUCKETS 1
-    PROPERTIES (
-    "replication_num" = "1"
-    )
-    '''
-
-
-    sql '''
-    alter table customer_demographics modify column cd_dep_college_count set stats ('row_count'='1920800', 'ndv'='7', 'min_value'='0', 'max_value'='6', 'avg_size'='7683200', 'max_size'='7683200' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_education_status set stats ('row_count'='1920800', 'ndv'='7', 'min_value'='2 yr Degree', 'max_value'='Unknown', 'avg_size'='18384800', 'max_size'='18384800' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_gender set stats ('row_count'='1920800', 'ndv'='2', 'min_value'='F', 'max_value'='M', 'avg_size'='1920800', 'max_size'='1920800' )
-    '''
-
-    sql '''
-    alter table reason modify column r_reason_id set stats ('row_count'='55', 'ndv'='55', 'min_value'='AAAAAAAAABAAAAAA', 'max_value'='AAAAAAAAPCAAAAAA', 'avg_size'='880', 'max_size'='880' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_current_quarter set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_current_week set stats ('row_count'='73049', 'ndv'='1', 'min_value'='N', 'max_value'='N', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_current_year set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_last_dom set stats ('row_count'='73049', 'ndv'='2419', 'min_value'='2415020', 'max_value'='2488372', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_month_seq set stats ('row_count'='73049', 'ndv'='2398', 'min_value'='0', 'max_value'='2400', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_quarter_name set stats ('row_count'='73049', 'ndv'='799', 'min_value'='1900Q1', 'max_value'='2100Q1', 'avg_size'='438294', 'max_size'='438294' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_county set stats ('row_count'='15', 'ndv'='8', 'min_value'='Barrow County', 'max_value'='Ziebach County', 'avg_size'='207', 'max_size'='207' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_warehouse_name set stats ('row_count'='15', 'ndv'='15', 'min_value'='', 'max_value'='Rooms cook ', 'avg_size'='230', 'max_size'='230' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_warehouse_sk set stats ('row_count'='15', 'ndv'='15', 'min_value'='1', 'max_value'='15', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_zip set stats ('row_count'='15', 'ndv'='15', 'min_value'='28721', 'max_value'='78721', 'avg_size'='75', 'max_size'='75' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_bill_addr_sk set stats ('row_count'='143997065', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_bill_hdemo_sk set stats ('row_count'='143997065', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_call_center_sk set stats ('row_count'='143997065', 'ndv'='30', 'min_value'='1', 'max_value'='30', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_catalog_page_sk set stats ('row_count'='143997065', 'ndv'='11515', 'min_value'='1', 'max_value'='17108', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_ship_cost set stats ('row_count'='143997065', 'ndv'='14266', 'min_value'='0.00', 'max_value'='14896.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_list_price set stats ('row_count'='143997065', 'ndv'='301', 'min_value'='1.00', 'max_value'='300.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_net_paid set stats ('row_count'='143997065', 'ndv'='27448', 'min_value'='0.00', 'max_value'='29760.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_promo_sk set stats ('row_count'='143997065', 'ndv'='986', 'min_value'='1', 'max_value'='1000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_sold_time_sk set stats ('row_count'='143997065', 'ndv'='87677', 'min_value'='0', 'max_value'='86399', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_sold_time_sk set stats ('row_count'='143997065', 'ndv'='87677', 'min_value'='0', 'max_value'='86399', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_warehouse_sk set stats ('row_count'='143997065', 'ndv'='15', 'min_value'='1', 'max_value'='15', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_call_center_id set stats ('row_count'='30', 'ndv'='15', 'min_value'='AAAAAAAAABAAAAAA', 'max_value'='AAAAAAAAOAAAAAAA', 'avg_size'='480', 'max_size'='480' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_company set stats ('row_count'='30', 'ndv'='6', 'min_value'='1', 'max_value'='6', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_country set stats ('row_count'='30', 'ndv'='1', 'min_value'='United States', 'max_value'='United States', 'avg_size'='390', 'max_size'='390' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_county set stats ('row_count'='30', 'ndv'='8', 'min_value'='Barrow County', 'max_value'='Ziebach County', 'avg_size'='423', 'max_size'='423' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_division_name set stats ('row_count'='30', 'ndv'='6', 'min_value'='able', 'max_value'='pri', 'avg_size'='123', 'max_size'='123' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_manager set stats ('row_count'='30', 'ndv'='22', 'min_value'='Alden Snyder', 'max_value'='Wayne Ray', 'avg_size'='368', 'max_size'='368' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_mkt_class set stats ('row_count'='30', 'ndv'='25', 'min_value'='A bit narrow forms matter animals. Consist', 'max_value'='Yesterday new men can make moreov', 'avg_size'='1033', 'max_size'='1033' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_mkt_id set stats ('row_count'='30', 'ndv'='6', 'min_value'='1', 'max_value'='6', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_state set stats ('row_count'='30', 'ndv'='8', 'min_value'='AL', 'max_value'='TN', 'avg_size'='60', 'max_size'='60' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_street_number set stats ('row_count'='30', 'ndv'='15', 'min_value'='406', 'max_value'='984', 'avg_size'='88', 'max_size'='88' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_catalog_page_sk set stats ('row_count'='14404374', 'ndv'='11515', 'min_value'='1', 'max_value'='17108', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_fee set stats ('row_count'='14404374', 'ndv'='101', 'min_value'='0.50', 'max_value'='100.00', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_item_sk set stats ('row_count'='14404374', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_refunded_cdemo_sk set stats ('row_count'='14404374', 'ndv'='1900770', 'min_value'='1', 'max_value'='1920800', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_return_amt_inc_tax set stats ('row_count'='14404374', 'ndv'='21566', 'min_value'='0.00', 'max_value'='29353.87', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returned_date_sk set stats ('row_count'='14404374', 'ndv'='2105', 'min_value'='2450821', 'max_value'='2452921', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returned_date_sk set stats ('row_count'='14404374', 'ndv'='2105', 'min_value'='2450821', 'max_value'='2452921', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returned_time_sk set stats ('row_count'='14404374', 'ndv'='87677', 'min_value'='0', 'max_value'='86399', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returning_customer_sk set stats ('row_count'='14404374', 'ndv'='1991754', 'min_value'='1', 'max_value'='2000000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returning_customer_sk set stats ('row_count'='14404374', 'ndv'='1991754', 'min_value'='1', 'max_value'='2000000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returning_hdemo_sk set stats ('row_count'='14404374', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_store_credit set stats ('row_count'='14404374', 'ndv'='12156', 'min_value'='0.00', 'max_value'='22167.49', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table household_demographics modify column hd_income_band_sk set stats ('row_count'='7200', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='57600', 'max_size'='57600' )
-    '''
-
-    sql '''
-    alter table household_demographics modify column hd_income_band_sk set stats ('row_count'='7200', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='57600', 'max_size'='57600' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_gmt_offset set stats ('row_count'='1000000', 'ndv'='6', 'min_value'='-10.00', 'max_value'='-5.00', 'avg_size'='4000000', 'max_size'='4000000' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_zip set stats ('row_count'='1000000', 'ndv'='7733', 'min_value'='', 'max_value'='99981', 'avg_size'='4848150', 'max_size'='4848150' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_catalog_page_number set stats ('row_count'='20400', 'ndv'='189', 'min_value'='1', 'max_value'='188', 'avg_size'='81600', 'max_size'='81600' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_catalog_page_sk set stats ('row_count'='20400', 'ndv'='20554', 'min_value'='1', 'max_value'='20400', 'avg_size'='163200', 'max_size'='163200' )
-    '''
-
-    sql '''
-    alter table item modify column i_brand set stats ('row_count'='204000', 'ndv'='714', 'min_value'='', 'max_value'='univunivamalg #9', 'avg_size'='3287671', 'max_size'='3287671' )
-    '''
-
-    sql '''
-    alter table item modify column i_category set stats ('row_count'='204000', 'ndv'='11', 'min_value'='', 'max_value'='Women', 'avg_size'='1201703', 'max_size'='1201703' )
-    '''
-
-    sql '''
-    alter table item modify column i_color set stats ('row_count'='204000', 'ndv'='93', 'min_value'='', 'max_value'='yellow', 'avg_size'='1094247', 'max_size'='1094247' )
-    '''
-
-    sql '''
-    alter table item modify column i_formulation set stats ('row_count'='204000', 'ndv'='152702', 'min_value'='', 'max_value'='yellow98911509228741', 'avg_size'='4069400', 'max_size'='4069400' )
-    '''
-
-    sql '''
-    alter table item modify column i_item_id set stats ('row_count'='204000', 'ndv'='103230', 'min_value'='AAAAAAAAAAAABAAA', 'max_value'='AAAAAAAAPPPPBAAA', 'avg_size'='3264000', 'max_size'='3264000' )
-    '''
-
-    sql '''
-    alter table item modify column i_rec_start_date set stats ('row_count'='204000', 'ndv'='4', 'min_value'='1997-10-27', 'max_value'='2001-10-27', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_size set stats ('row_count'='204000', 'ndv'='8', 'min_value'='', 'max_value'='small', 'avg_size'='880961', 'max_size'='880961' )
-    '''
-
-    sql '''
-    alter table item modify column i_wholesale_cost set stats ('row_count'='204000', 'ndv'='89', 'min_value'='0.02', 'max_value'='88.91', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_item_sk set stats ('row_count'='7197670', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_return_amt_inc_tax set stats ('row_count'='7197670', 'ndv'='19975', 'min_value'='0.00', 'max_value'='29493.38', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returned_date_sk set stats ('row_count'='7197670', 'ndv'='2185', 'min_value'='2450820', 'max_value'='2453002', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returning_addr_sk set stats ('row_count'='7197670', 'ndv'='999584', 'min_value'='1', 'max_value'='1000000', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returning_customer_sk set stats ('row_count'='7197670', 'ndv'='1926139', 'min_value'='1', 'max_value'='2000000', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_reversed_charge set stats ('row_count'='7197670', 'ndv'='10979', 'min_value'='0.00', 'max_value'='22972.36', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_city set stats ('row_count'='24', 'ndv'='11', 'min_value'='Centerville', 'max_value'='Salem', 'avg_size'='232', 'max_size'='232' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_city set stats ('row_count'='24', 'ndv'='11', 'min_value'='Centerville', 'max_value'='Salem', 'avg_size'='232', 'max_size'='232' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_country set stats ('row_count'='24', 'ndv'='1', 'min_value'='United States', 'max_value'='United States', 'avg_size'='312', 'max_size'='312' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_country set stats ('row_count'='24', 'ndv'='1', 'min_value'='United States', 'max_value'='United States', 'avg_size'='312', 'max_size'='312' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_county set stats ('row_count'='24', 'ndv'='9', 'min_value'='Barrow County', 'max_value'='Ziebach County', 'avg_size'='331', 'max_size'='331' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_manager set stats ('row_count'='24', 'ndv'='19', 'min_value'='Adam Stonge', 'max_value'='Tommy Jones', 'avg_size'='297', 'max_size'='297' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_mkt_class set stats ('row_count'='24', 'ndv'='18', 'min_value'='About rural reasons shall no', 'max_value'='Wide, final representat', 'avg_size'='758', 'max_size'='758' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_mkt_class set stats ('row_count'='24', 'ndv'='18', 'min_value'='About rural reasons shall no', 'max_value'='Wide, final representat', 'avg_size'='758', 'max_size'='758' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_mkt_desc set stats ('row_count'='24', 'ndv'='15', 'min_value'='Acres see else children. Mutual too', 'max_value'='Well similar decisions used to keep hardly democratic, personal priorities.', 'avg_size'='1561', 'max_size'='1561' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_mkt_id set stats ('row_count'='24', 'ndv'='6', 'min_value'='1', 'max_value'='6', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_mkt_id set stats ('row_count'='24', 'ndv'='6', 'min_value'='1', 'max_value'='6', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_rec_end_date set stats ('row_count'='24', 'ndv'='3', 'min_value'='1999-08-16', 'max_value'='2001-08-15', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_rec_start_date set stats ('row_count'='24', 'ndv'='4', 'min_value'='1997-08-16', 'max_value'='2001-08-16', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_street_type set stats ('row_count'='24', 'ndv'='15', 'min_value'='Avenue', 'max_value'='Wy', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_tax_percentage set stats ('row_count'='24', 'ndv'='1', 'min_value'='0.00', 'max_value'='0.12', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_dmail set stats ('row_count'='1000', 'ndv'='3', 'min_value'='', 'max_value'='Y', 'avg_size'='987', 'max_size'='987' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_dmail set stats ('row_count'='1000', 'ndv'='3', 'min_value'='', 'max_value'='Y', 'avg_size'='987', 'max_size'='987' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_radio set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='987', 'max_size'='987' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_item_sk set stats ('row_count'='1000', 'ndv'='970', 'min_value'='280', 'max_value'='203966', 'avg_size'='8000', 'max_size'='8000' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_bill_addr_sk set stats ('row_count'='72001237', 'ndv'='998891', 'min_value'='1', 'max_value'='1000000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_discount_amt set stats ('row_count'='72001237', 'ndv'='27052', 'min_value'='0.00', 'max_value'='29982.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_ship_cost set stats ('row_count'='72001237', 'ndv'='13977', 'min_value'='0.00', 'max_value'='14927.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_tax set stats ('row_count'='72001237', 'ndv'='2466', 'min_value'='0.00', 'max_value'='2682.90', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_item_sk set stats ('row_count'='72001237', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_net_paid_inc_ship_tax set stats ('row_count'='72001237', 'ndv'='37541', 'min_value'='0.00', 'max_value'='44479.52', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_net_profit set stats ('row_count'='72001237', 'ndv'='27958', 'min_value'='-9997.00', 'max_value'='19840.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_sold_date_sk set stats ('row_count'='72001237', 'ndv'='1820', 'min_value'='2450816', 'max_value'='2452642', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_warehouse_sk set stats ('row_count'='72001237', 'ndv'='15', 'min_value'='1', 'max_value'='15', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_web_page_sk set stats ('row_count'='72001237', 'ndv'='2032', 'min_value'='1', 'max_value'='2040', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table store modify column s_city set stats ('row_count'='402', 'ndv'='19', 'min_value'='', 'max_value'='Union', 'avg_size'='3669', 'max_size'='3669' )
-    '''
-
-    sql '''
-    alter table store modify column s_company_name set stats ('row_count'='402', 'ndv'='2', 'min_value'='', 'max_value'='Unknown', 'avg_size'='2793', 'max_size'='2793' )
-    '''
-
-    sql '''
-    alter table store modify column s_county set stats ('row_count'='402', 'ndv'='10', 'min_value'='', 'max_value'='Ziebach County', 'avg_size'='5693', 'max_size'='5693' )
-    '''
-
-    sql '''
-    alter table store modify column s_division_name set stats ('row_count'='402', 'ndv'='2', 'min_value'='', 'max_value'='Unknown', 'avg_size'='2779', 'max_size'='2779' )
-    '''
-
-    sql '''
-    alter table store modify column s_floor_space set stats ('row_count'='402', 'ndv'='300', 'min_value'='5004767', 'max_value'='9997773', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_market_desc set stats ('row_count'='402', 'ndv'='311', 'min_value'='', 'max_value'='Years get acute years. Right likely players mus', 'avg_size'='23261', 'max_size'='23261' )
-    '''
-
-    sql '''
-    alter table store modify column s_store_id set stats ('row_count'='402', 'ndv'='201', 'min_value'='AAAAAAAAAABAAAAA', 'max_value'='AAAAAAAAPNAAAAAA', 'avg_size'='6432', 'max_size'='6432' )
-    '''
-
-    sql '''
-    alter table store modify column s_store_name set stats ('row_count'='402', 'ndv'='11', 'min_value'='', 'max_value'='pri', 'avg_size'='1575', 'max_size'='1575' )
-    '''
-
-    sql '''
-    alter table store modify column s_street_type set stats ('row_count'='402', 'ndv'='21', 'min_value'='', 'max_value'='Wy', 'avg_size'='1657', 'max_size'='1657' )
-    '''
-
-    sql '''
-    alter table store modify column s_zip set stats ('row_count'='402', 'ndv'='102', 'min_value'='', 'max_value'='79431', 'avg_size'='1980', 'max_size'='1980' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_am_pm set stats ('row_count'='86400', 'ndv'='2', 'min_value'='AM', 'max_value'='PM', 'avg_size'='172800', 'max_size'='172800' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_hour set stats ('row_count'='86400', 'ndv'='24', 'min_value'='0', 'max_value'='23', 'avg_size'='345600', 'max_size'='345600' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_time set stats ('row_count'='86400', 'ndv'='86684', 'min_value'='0', 'max_value'='86399', 'avg_size'='345600', 'max_size'='345600' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_customer_sk set stats ('row_count'='2040', 'ndv'='475', 'min_value'='711', 'max_value'='1996257', 'avg_size'='16320', 'max_size'='16320' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_image_count set stats ('row_count'='2040', 'ndv'='7', 'min_value'='1', 'max_value'='7', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_link_count set stats ('row_count'='2040', 'ndv'='24', 'min_value'='2', 'max_value'='25', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_rec_end_date set stats ('row_count'='2040', 'ndv'='3', 'min_value'='1999-09-03', 'max_value'='2001-09-02', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_web_page_sk set stats ('row_count'='2040', 'ndv'='2032', 'min_value'='1', 'max_value'='2040', 'avg_size'='16320', 'max_size'='16320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_addr_sk set stats ('row_count'='28795080', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_addr_sk set stats ('row_count'='28795080', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_customer_sk set stats ('row_count'='28795080', 'ndv'='1994323', 'min_value'='1', 'max_value'='2000000', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_customer_sk set stats ('row_count'='28795080', 'ndv'='1994323', 'min_value'='1', 'max_value'='2000000', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_hdemo_sk set stats ('row_count'='28795080', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_amt set stats ('row_count'='28795080', 'ndv'='15493', 'min_value'='0.00', 'max_value'='18973.20', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_amt_inc_tax set stats ('row_count'='28795080', 'ndv'='16190', 'min_value'='0.00', 'max_value'='20002.89', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_tax set stats ('row_count'='28795080', 'ndv'='1427', 'min_value'='0.00', 'max_value'='1611.71', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_store_sk set stats ('row_count'='28795080', 'ndv'='200', 'min_value'='1', 'max_value'='400', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_ticket_number set stats ('row_count'='28795080', 'ndv'='16790866', 'min_value'='1', 'max_value'='23999996', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_customer_sk set stats ('row_count'='287997024', 'ndv'='1994393', 'min_value'='1', 'max_value'='2000000', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ext_discount_amt set stats ('row_count'='287997024', 'ndv'='16198', 'min_value'='0.00', 'max_value'='19225.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_hdemo_sk set stats ('row_count'='287997024', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_list_price set stats ('row_count'='287997024', 'ndv'='201', 'min_value'='1.00', 'max_value'='200.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_promo_sk set stats ('row_count'='287997024', 'ndv'='986', 'min_value'='1', 'max_value'='1000', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_quantity set stats ('row_count'='287997024', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_ship_mode_id set stats ('row_count'='20', 'ndv'='20', 'min_value'='AAAAAAAAABAAAAAA', 'max_value'='AAAAAAAAPAAAAAAA', 'avg_size'='320', 'max_size'='320' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_ship_mode_sk set stats ('row_count'='20', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='160', 'max_size'='160' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_ship_mode_sk set stats ('row_count'='20', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='160', 'max_size'='160' )
-    '''
-
-    sql '''
-    alter table customer modify column c_birth_month set stats ('row_count'='2000000', 'ndv'='12', 'min_value'='1', 'max_value'='12', 'avg_size'='8000000', 'max_size'='8000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_birth_year set stats ('row_count'='2000000', 'ndv'='69', 'min_value'='1924', 'max_value'='1992', 'avg_size'='8000000', 'max_size'='8000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_email_address set stats ('row_count'='2000000', 'ndv'='1936613', 'min_value'='', 'max_value'='Zulma.Wright@AqokXsju9f2yj.org', 'avg_size'='53014147', 'max_size'='53014147' )
-    '''
-
-    sql '''
-    alter table customer modify column c_email_address set stats ('row_count'='2000000', 'ndv'='1936613', 'min_value'='', 'max_value'='Zulma.Wright@AqokXsju9f2yj.org', 'avg_size'='53014147', 'max_size'='53014147' )
-    '''
-
-    sql '''
-    alter table customer modify column c_last_review_date_sk set stats ('row_count'='2000000', 'ndv'='366', 'min_value'='2452283', 'max_value'='2452648', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_login set stats ('row_count'='2000000', 'ndv'='1', 'min_value'='', 'max_value'='', 'avg_size'='0', 'max_size'='0' )
-    '''
-
-    sql '''
-    alter table customer modify column c_preferred_cust_flag set stats ('row_count'='2000000', 'ndv'='3', 'min_value'='', 'max_value'='Y', 'avg_size'='1930222', 'max_size'='1930222' )
-    '''
-
-    sql '''
-    alter table customer modify column c_preferred_cust_flag set stats ('row_count'='2000000', 'ndv'='3', 'min_value'='', 'max_value'='Y', 'avg_size'='1930222', 'max_size'='1930222' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_dep_count set stats ('row_count'='1920800', 'ndv'='7', 'min_value'='0', 'max_value'='6', 'avg_size'='7683200', 'max_size'='7683200' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_dep_count set stats ('row_count'='1920800', 'ndv'='7', 'min_value'='0', 'max_value'='6', 'avg_size'='7683200', 'max_size'='7683200' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_purchase_estimate set stats ('row_count'='1920800', 'ndv'='20', 'min_value'='500', 'max_value'='10000', 'avg_size'='7683200', 'max_size'='7683200' )
-    '''
-
-    sql '''
-    alter table reason modify column r_reason_sk set stats ('row_count'='55', 'ndv'='55', 'min_value'='1', 'max_value'='55', 'avg_size'='440', 'max_size'='440' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_date_id set stats ('row_count'='73049', 'ndv'='72907', 'min_value'='AAAAAAAAAAAAFCAA', 'max_value'='AAAAAAAAPPPPECAA', 'avg_size'='1168784', 'max_size'='1168784' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_date_sk set stats ('row_count'='73049', 'ndv'='73042', 'min_value'='2415022', 'max_value'='2488070', 'avg_size'='584392', 'max_size'='584392' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_date_sk set stats ('row_count'='73049', 'ndv'='73042', 'min_value'='2415022', 'max_value'='2488070', 'avg_size'='584392', 'max_size'='584392' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_first_dom set stats ('row_count'='73049', 'ndv'='2410', 'min_value'='2415021', 'max_value'='2488070', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_fy_quarter_seq set stats ('row_count'='73049', 'ndv'='801', 'min_value'='1', 'max_value'='801', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_fy_week_seq set stats ('row_count'='73049', 'ndv'='10448', 'min_value'='1', 'max_value'='10436', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_week_seq set stats ('row_count'='73049', 'ndv'='10448', 'min_value'='1', 'max_value'='10436', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_city set stats ('row_count'='15', 'ndv'='11', 'min_value'='Bethel', 'max_value'='Union', 'avg_size'='111', 'max_size'='111' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_county set stats ('row_count'='15', 'ndv'='8', 'min_value'='Barrow County', 'max_value'='Ziebach County', 'avg_size'='207', 'max_size'='207' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_gmt_offset set stats ('row_count'='15', 'ndv'='2', 'min_value'='-6.00', 'max_value'='-5.00', 'avg_size'='60', 'max_size'='60' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_street_name set stats ('row_count'='15', 'ndv'='15', 'min_value'='', 'max_value'='Wilson Elm', 'avg_size'='128', 'max_size'='128' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_warehouse_id set stats ('row_count'='15', 'ndv'='15', 'min_value'='AAAAAAAABAAAAAAA', 'max_value'='AAAAAAAAPAAAAAAA', 'avg_size'='240', 'max_size'='240' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_bill_addr_sk set stats ('row_count'='143997065', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_bill_customer_sk set stats ('row_count'='143997065', 'ndv'='1993691', 'min_value'='1', 'max_value'='2000000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_coupon_amt set stats ('row_count'='143997065', 'ndv'='22020', 'min_value'='0.00', 'max_value'='28422.94', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_coupon_amt set stats ('row_count'='143997065', 'ndv'='22020', 'min_value'='0.00', 'max_value'='28422.94', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_discount_amt set stats ('row_count'='143997065', 'ndv'='27722', 'min_value'='0.00', 'max_value'='29765.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_sales_price set stats ('row_count'='143997065', 'ndv'='27598', 'min_value'='0.00', 'max_value'='29808.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_ship_cost set stats ('row_count'='143997065', 'ndv'='14266', 'min_value'='0.00', 'max_value'='14896.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_tax set stats ('row_count'='143997065', 'ndv'='2488', 'min_value'='0.00', 'max_value'='2619.36', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_item_sk set stats ('row_count'='143997065', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_quantity set stats ('row_count'='143997065', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_quantity set stats ('row_count'='143997065', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_call_center_sk set stats ('row_count'='30', 'ndv'='30', 'min_value'='1', 'max_value'='30', 'avg_size'='240', 'max_size'='240' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_closed_date_sk set stats ('row_count'='30', 'ndv'='0', 'min_value'='2415022', 'max_value'='2488070', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_company set stats ('row_count'='30', 'ndv'='6', 'min_value'='1', 'max_value'='6', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_country set stats ('row_count'='30', 'ndv'='1', 'min_value'='United States', 'max_value'='United States', 'avg_size'='390', 'max_size'='390' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_division set stats ('row_count'='30', 'ndv'='6', 'min_value'='1', 'max_value'='6', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_rec_end_date set stats ('row_count'='30', 'ndv'='3', 'min_value'='2000-01-01', 'max_value'='2001-12-31', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_street_type set stats ('row_count'='30', 'ndv'='9', 'min_value'='Avenue', 'max_value'='Way', 'avg_size'='140', 'max_size'='140' )
-    '''
-
-    sql '''
-    alter table inventory modify column inv_quantity_on_hand set stats ('row_count'='399330000', 'ndv'='1006', 'min_value'='0', 'max_value'='1000', 'avg_size'='1597320000', 'max_size'='1597320000' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_net_loss set stats ('row_count'='14404374', 'ndv'='11753', 'min_value'='0.50', 'max_value'='15781.83', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_refunded_cash set stats ('row_count'='14404374', 'ndv'='16271', 'min_value'='0.00', 'max_value'='24544.84', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returning_addr_sk set stats ('row_count'='14404374', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_reversed_charge set stats ('row_count'='14404374', 'ndv'='12359', 'min_value'='0.00', 'max_value'='23801.24', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table household_demographics modify column hd_buy_potential set stats ('row_count'='7200', 'ndv'='6', 'min_value'='0-500', 'max_value'='Unknown', 'avg_size'='54000', 'max_size'='54000' )
-    '''
-
-    sql '''
-    alter table household_demographics modify column hd_demo_sk set stats ('row_count'='7200', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='57600', 'max_size'='57600' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_address_sk set stats ('row_count'='1000000', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='8000000', 'max_size'='8000000' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_city set stats ('row_count'='1000000', 'ndv'='977', 'min_value'='', 'max_value'='Zion', 'avg_size'='8681993', 'max_size'='8681993' )
-    '''
-
-    sql '''
-    alter table income_band modify column ib_income_band_sk set stats ('row_count'='20', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='160', 'max_size'='160' )
-    '''
-
-    sql '''
-    alter table income_band modify column ib_lower_bound set stats ('row_count'='20', 'ndv'='20', 'min_value'='0', 'max_value'='190001', 'avg_size'='80', 'max_size'='80' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_type set stats ('row_count'='20400', 'ndv'='4', 'min_value'='', 'max_value'='quarterly', 'avg_size'='155039', 'max_size'='155039' )
-    '''
-
-    sql '''
-    alter table item modify column i_brand_id set stats ('row_count'='204000', 'ndv'='951', 'min_value'='1001001', 'max_value'='10016017', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_category_id set stats ('row_count'='204000', 'ndv'='10', 'min_value'='1', 'max_value'='10', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_current_price set stats ('row_count'='204000', 'ndv'='100', 'min_value'='0.09', 'max_value'='99.99', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_current_price set stats ('row_count'='204000', 'ndv'='100', 'min_value'='0.09', 'max_value'='99.99', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_item_desc set stats ('row_count'='204000', 'ndv'='148398', 'min_value'='', 'max_value'='Youngsters used to save quite colour', 'avg_size'='20471814', 'max_size'='20471814' )
-    '''
-
-    sql '''
-    alter table item modify column i_manufact set stats ('row_count'='204000', 'ndv'='1004', 'min_value'='', 'max_value'='pripripri', 'avg_size'='2298787', 'max_size'='2298787' )
-    '''
-
-    sql '''
-    alter table item modify column i_manufact_id set stats ('row_count'='204000', 'ndv'='1005', 'min_value'='1', 'max_value'='1000', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_product_name set stats ('row_count'='204000', 'ndv'='200390', 'min_value'='', 'max_value'='pripripripripriought', 'avg_size'='4546148', 'max_size'='4546148' )
-    '''
-
-    sql '''
-    alter table item modify column i_rec_start_date set stats ('row_count'='204000', 'ndv'='4', 'min_value'='1997-10-27', 'max_value'='2001-10-27', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_refunded_cash set stats ('row_count'='7197670', 'ndv'='14621', 'min_value'='0.00', 'max_value'='26466.56', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_refunded_hdemo_sk set stats ('row_count'='7197670', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_return_quantity set stats ('row_count'='7197670', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_return_ship_cost set stats ('row_count'='7197670', 'ndv'='10429', 'min_value'='0.00', 'max_value'='13602.60', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returning_hdemo_sk set stats ('row_count'='7197670', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_web_page_sk set stats ('row_count'='7197670', 'ndv'='2032', 'min_value'='1', 'max_value'='2040', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_class set stats ('row_count'='24', 'ndv'='1', 'min_value'='Unknown', 'max_value'='Unknown', 'avg_size'='168', 'max_size'='168' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_company_id set stats ('row_count'='24', 'ndv'='6', 'min_value'='1', 'max_value'='6', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_market_manager set stats ('row_count'='24', 'ndv'='21', 'min_value'='Albert Leung', 'max_value'='Zachery Oneil', 'avg_size'='294', 'max_size'='294' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_street_name set stats ('row_count'='24', 'ndv'='24', 'min_value'='11th ', 'max_value'='Wilson Ridge', 'avg_size'='219', 'max_size'='219' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_suite_number set stats ('row_count'='24', 'ndv'='20', 'min_value'='Suite 130', 'max_value'='Suite U', 'avg_size'='196', 'max_size'='196' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_suite_number set stats ('row_count'='24', 'ndv'='20', 'min_value'='Suite 130', 'max_value'='Suite U', 'avg_size'='196', 'max_size'='196' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_tax_percentage set stats ('row_count'='24', 'ndv'='1', 'min_value'='0.00', 'max_value'='0.12', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_zip set stats ('row_count'='24', 'ndv'='14', 'min_value'='28828', 'max_value'='78828', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_demo set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='984', 'max_size'='984' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_email set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='987', 'max_size'='987' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_end_date_sk set stats ('row_count'='1000', 'ndv'='571', 'min_value'='2450116', 'max_value'='2450967', 'avg_size'='8000', 'max_size'='8000' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_bill_customer_sk set stats ('row_count'='72001237', 'ndv'='1899439', 'min_value'='1', 'max_value'='2000000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_coupon_amt set stats ('row_count'='72001237', 'ndv'='20659', 'min_value'='0.00', 'max_value'='27591.16', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_quantity set stats ('row_count'='72001237', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_sales_price set stats ('row_count'='72001237', 'ndv'='302', 'min_value'='0.00', 'max_value'='300.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_sales_price set stats ('row_count'='72001237', 'ndv'='302', 'min_value'='0.00', 'max_value'='300.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_sold_time_sk set stats ('row_count'='72001237', 'ndv'='87677', 'min_value'='0', 'max_value'='86399', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table store modify column s_city set stats ('row_count'='402', 'ndv'='19', 'min_value'='', 'max_value'='Union', 'avg_size'='3669', 'max_size'='3669' )
-    '''
-
-    sql '''
-    alter table store modify column s_closed_date_sk set stats ('row_count'='402', 'ndv'='69', 'min_value'='2450823', 'max_value'='2451313', 'avg_size'='3216', 'max_size'='3216' )
-    '''
-
-    sql '''
-    alter table store modify column s_gmt_offset set stats ('row_count'='402', 'ndv'='2', 'min_value'='-6.00', 'max_value'='-5.00', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_gmt_offset set stats ('row_count'='402', 'ndv'='2', 'min_value'='-6.00', 'max_value'='-5.00', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_market_desc set stats ('row_count'='402', 'ndv'='311', 'min_value'='', 'max_value'='Years get acute years. Right likely players mus', 'avg_size'='23261', 'max_size'='23261' )
-    '''
-
-    sql '''
-    alter table store modify column s_market_manager set stats ('row_count'='402', 'ndv'='286', 'min_value'='', 'max_value'='Zane Perez', 'avg_size'='5129', 'max_size'='5129' )
-    '''
-
-    sql '''
-    alter table store modify column s_rec_end_date set stats ('row_count'='402', 'ndv'='3', 'min_value'='1999-03-13', 'max_value'='2001-03-12', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_rec_end_date set stats ('row_count'='402', 'ndv'='3', 'min_value'='1999-03-13', 'max_value'='2001-03-12', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_state set stats ('row_count'='402', 'ndv'='10', 'min_value'='', 'max_value'='TN', 'avg_size'='800', 'max_size'='800' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_minute set stats ('row_count'='86400', 'ndv'='60', 'min_value'='0', 'max_value'='59', 'avg_size'='345600', 'max_size'='345600' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_shift set stats ('row_count'='86400', 'ndv'='3', 'min_value'='first', 'max_value'='third', 'avg_size'='460800', 'max_size'='460800' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_customer_sk set stats ('row_count'='2040', 'ndv'='475', 'min_value'='711', 'max_value'='1996257', 'avg_size'='16320', 'max_size'='16320' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_type set stats ('row_count'='2040', 'ndv'='8', 'min_value'='', 'max_value'='welcome', 'avg_size'='12856', 'max_size'='12856' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_url set stats ('row_count'='2040', 'ndv'='2', 'min_value'='', 'max_value'='http://www.foo.com', 'avg_size'='36270', 'max_size'='36270' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_amt_inc_tax set stats ('row_count'='28795080', 'ndv'='16190', 'min_value'='0.00', 'max_value'='20002.89', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_quantity set stats ('row_count'='28795080', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_tax set stats ('row_count'='28795080', 'ndv'='1427', 'min_value'='0.00', 'max_value'='1611.71', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_time_sk set stats ('row_count'='28795080', 'ndv'='32660', 'min_value'='28799', 'max_value'='61199', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_returned_date_sk set stats ('row_count'='28795080', 'ndv'='2010', 'min_value'='2450820', 'max_value'='2452822', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_reversed_charge set stats ('row_count'='28795080', 'ndv'='9872', 'min_value'='0.00', 'max_value'='16099.52', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_addr_sk set stats ('row_count'='287997024', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ext_list_price set stats ('row_count'='287997024', 'ndv'='19770', 'min_value'='1.00', 'max_value'='20000.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ext_sales_price set stats ('row_count'='287997024', 'ndv'='19105', 'min_value'='0.00', 'max_value'='19878.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ext_wholesale_cost set stats ('row_count'='287997024', 'ndv'='10009', 'min_value'='1.00', 'max_value'='10000.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_item_sk set stats ('row_count'='287997024', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_net_paid set stats ('row_count'='287997024', 'ndv'='19028', 'min_value'='0.00', 'max_value'='19878.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_net_profit set stats ('row_count'='287997024', 'ndv'='19581', 'min_value'='-10000.00', 'max_value'='9889.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_store_sk set stats ('row_count'='287997024', 'ndv'='200', 'min_value'='1', 'max_value'='400', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_ship_mode_id set stats ('row_count'='20', 'ndv'='20', 'min_value'='AAAAAAAAABAAAAAA', 'max_value'='AAAAAAAAPAAAAAAA', 'avg_size'='320', 'max_size'='320' )
-    '''
-
-    sql '''
-    alter table customer modify column c_current_hdemo_sk set stats ('row_count'='2000000', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_first_name set stats ('row_count'='2000000', 'ndv'='5140', 'min_value'='', 'max_value'='Zulma', 'avg_size'='11267996', 'max_size'='11267996' )
-    '''
-
-    sql '''
-    alter table dbgen_version modify column dv_create_date set stats ('row_count'='1', 'ndv'='1', 'min_value'='2023-03-16', 'max_value'='2023-03-16', 'avg_size'='4', 'max_size'='4' )
-    '''
-
-    sql '''
-    alter table dbgen_version modify column dv_version set stats ('row_count'='1', 'ndv'='1', 'min_value'='3.2.0', 'max_value'='3.2.0', 'avg_size'='5', 'max_size'='5' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_credit_rating set stats ('row_count'='1920800', 'ndv'='4', 'min_value'='Good', 'max_value'='Unknown', 'avg_size'='13445600', 'max_size'='13445600' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_demo_sk set stats ('row_count'='1920800', 'ndv'='1916366', 'min_value'='1', 'max_value'='1920800', 'avg_size'='15366400', 'max_size'='15366400' )
-    '''
-
-    sql '''
-    alter table reason modify column r_reason_sk set stats ('row_count'='55', 'ndv'='55', 'min_value'='1', 'max_value'='55', 'avg_size'='440', 'max_size'='440' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_current_day set stats ('row_count'='73049', 'ndv'='1', 'min_value'='N', 'max_value'='N', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_day_name set stats ('row_count'='73049', 'ndv'='7', 'min_value'='Friday', 'max_value'='Wednesday', 'avg_size'='521779', 'max_size'='521779' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_fy_quarter_seq set stats ('row_count'='73049', 'ndv'='801', 'min_value'='1', 'max_value'='801', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_fy_year set stats ('row_count'='73049', 'ndv'='202', 'min_value'='1900', 'max_value'='2100', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_holiday set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_holiday set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_quarter_seq set stats ('row_count'='73049', 'ndv'='801', 'min_value'='1', 'max_value'='801', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_same_day_lq set stats ('row_count'='73049', 'ndv'='72231', 'min_value'='2414930', 'max_value'='2487978', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_week_seq set stats ('row_count'='73049', 'ndv'='10448', 'min_value'='1', 'max_value'='10436', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_state set stats ('row_count'='15', 'ndv'='8', 'min_value'='AL', 'max_value'='SD', 'avg_size'='30', 'max_size'='30' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_street_number set stats ('row_count'='15', 'ndv'='15', 'min_value'='', 'max_value'='957', 'avg_size'='40', 'max_size'='40' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_suite_number set stats ('row_count'='15', 'ndv'='14', 'min_value'='', 'max_value'='Suite X', 'avg_size'='111', 'max_size'='111' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_suite_number set stats ('row_count'='15', 'ndv'='14', 'min_value'='', 'max_value'='Suite X', 'avg_size'='111', 'max_size'='111' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_warehouse_id set stats ('row_count'='15', 'ndv'='15', 'min_value'='AAAAAAAABAAAAAAA', 'max_value'='AAAAAAAAPAAAAAAA', 'avg_size'='240', 'max_size'='240' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_warehouse_sq_ft set stats ('row_count'='15', 'ndv'='14', 'min_value'='73065', 'max_value'='977787', 'avg_size'='60', 'max_size'='60' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_bill_customer_sk set stats ('row_count'='143997065', 'ndv'='1993691', 'min_value'='1', 'max_value'='2000000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_call_center_sk set stats ('row_count'='143997065', 'ndv'='30', 'min_value'='1', 'max_value'='30', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_list_price set stats ('row_count'='143997065', 'ndv'='29336', 'min_value'='1.00', 'max_value'='29997.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_wholesale_cost set stats ('row_count'='143997065', 'ndv'='10009', 'min_value'='1.00', 'max_value'='10000.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_item_sk set stats ('row_count'='143997065', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_net_paid_inc_ship set stats ('row_count'='143997065', 'ndv'='37890', 'min_value'='0.00', 'max_value'='43725.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_net_paid_inc_tax set stats ('row_count'='143997065', 'ndv'='28777', 'min_value'='0.00', 'max_value'='31745.52', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_net_profit set stats ('row_count'='143997065', 'ndv'='28450', 'min_value'='-10000.00', 'max_value'='19840.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_order_number set stats ('row_count'='143997065', 'ndv'='16050730', 'min_value'='1', 'max_value'='16000000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_promo_sk set stats ('row_count'='143997065', 'ndv'='986', 'min_value'='1', 'max_value'='1000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_addr_sk set stats ('row_count'='143997065', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_cdemo_sk set stats ('row_count'='143997065', 'ndv'='1916125', 'min_value'='1', 'max_value'='1920800', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_sold_date_sk set stats ('row_count'='143997065', 'ndv'='1835', 'min_value'='2450815', 'max_value'='2452654', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_call_center_id set stats ('row_count'='30', 'ndv'='15', 'min_value'='AAAAAAAAABAAAAAA', 'max_value'='AAAAAAAAOAAAAAAA', 'avg_size'='480', 'max_size'='480' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_city set stats ('row_count'='30', 'ndv'='12', 'min_value'='Bethel', 'max_value'='Shady Grove', 'avg_size'='282', 'max_size'='282' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_closed_date_sk set stats ('row_count'='30', 'ndv'='0', 'min_value'='0', 'max_value'='0', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_company_name set stats ('row_count'='30', 'ndv'='6', 'min_value'='able', 'max_value'='pri', 'avg_size'='110', 'max_size'='110' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_county set stats ('row_count'='30', 'ndv'='8', 'min_value'='Barrow County', 'max_value'='Ziebach County', 'avg_size'='423', 'max_size'='423' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_division_name set stats ('row_count'='30', 'ndv'='6', 'min_value'='able', 'max_value'='pri', 'avg_size'='123', 'max_size'='123' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_market_manager set stats ('row_count'='30', 'ndv'='24', 'min_value'='Charles Corbett', 'max_value'='Tom Root', 'avg_size'='373', 'max_size'='373' )
-    '''
-
-
-    sql '''
-    alter table call_center modify column cc_open_date_sk set stats ('row_count'='30', 'ndv'='15', 'min_value'='2450794', 'max_value'='2451146', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_sq_ft set stats ('row_count'='30', 'ndv'='22', 'min_value'='1670015', 'max_value'='31896816', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_state set stats ('row_count'='30', 'ndv'='8', 'min_value'='AL', 'max_value'='TN', 'avg_size'='60', 'max_size'='60' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_street_number set stats ('row_count'='30', 'ndv'='15', 'min_value'='406', 'max_value'='984', 'avg_size'='88', 'max_size'='88' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_street_type set stats ('row_count'='30', 'ndv'='9', 'min_value'='Avenue', 'max_value'='Way', 'avg_size'='140', 'max_size'='140' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_suite_number set stats ('row_count'='30', 'ndv'='14', 'min_value'='Suite 0', 'max_value'='Suite W', 'avg_size'='234', 'max_size'='234' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_zip set stats ('row_count'='30', 'ndv'='14', 'min_value'='20059', 'max_value'='75281', 'avg_size'='150', 'max_size'='150' )
-    '''
-
-    sql '''
-    alter table inventory modify column inv_item_sk set stats ('row_count'='399330000', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='3194640000', 'max_size'='3194640000' )
-    '''
-
-    sql '''
-    alter table inventory modify column inv_warehouse_sk set stats ('row_count'='399330000', 'ndv'='15', 'min_value'='1', 'max_value'='15', 'avg_size'='3194640000', 'max_size'='3194640000' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_item_sk set stats ('row_count'='14404374', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_order_number set stats ('row_count'='14404374', 'ndv'='9425725', 'min_value'='2', 'max_value'='16000000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_reason_sk set stats ('row_count'='14404374', 'ndv'='55', 'min_value'='1', 'max_value'='55', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_reason_sk set stats ('row_count'='14404374', 'ndv'='55', 'min_value'='1', 'max_value'='55', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_refunded_addr_sk set stats ('row_count'='14404374', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_refunded_hdemo_sk set stats ('row_count'='14404374', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_return_tax set stats ('row_count'='14404374', 'ndv'='1926', 'min_value'='0.00', 'max_value'='2390.75', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returning_cdemo_sk set stats ('row_count'='14404374', 'ndv'='1913762', 'min_value'='1', 'max_value'='1920800', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_ship_mode_sk set stats ('row_count'='14404374', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_gmt_offset set stats ('row_count'='1000000', 'ndv'='6', 'min_value'='-10.00', 'max_value'='-5.00', 'avg_size'='4000000', 'max_size'='4000000' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_location_type set stats ('row_count'='1000000', 'ndv'='4', 'min_value'='', 'max_value'='single family', 'avg_size'='8728128', 'max_size'='8728128' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_state set stats ('row_count'='1000000', 'ndv'='52', 'min_value'='', 'max_value'='WY', 'avg_size'='1939752', 'max_size'='1939752' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_street_number set stats ('row_count'='1000000', 'ndv'='1002', 'min_value'='', 'max_value'='999', 'avg_size'='2805540', 'max_size'='2805540' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_suite_number set stats ('row_count'='1000000', 'ndv'='76', 'min_value'='', 'max_value'='Suite Y', 'avg_size'='7652799', 'max_size'='7652799' )
-    '''
-
-    sql '''
-    alter table income_band modify column ib_upper_bound set stats ('row_count'='20', 'ndv'='20', 'min_value'='10000', 'max_value'='200000', 'avg_size'='80', 'max_size'='80' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_catalog_number set stats ('row_count'='20400', 'ndv'='109', 'min_value'='1', 'max_value'='109', 'avg_size'='81600', 'max_size'='81600' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_catalog_page_id set stats ('row_count'='20400', 'ndv'='20341', 'min_value'='AAAAAAAAAAABAAAA', 'max_value'='AAAAAAAAPPPDAAAA', 'avg_size'='326400', 'max_size'='326400' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_end_date_sk set stats ('row_count'='20400', 'ndv'='97', 'min_value'='2450844', 'max_value'='2453186', 'avg_size'='81600', 'max_size'='81600' )
-    '''
-
-    sql '''
-    alter table item modify column i_brand set stats ('row_count'='204000', 'ndv'='714', 'min_value'='', 'max_value'='univunivamalg #9', 'avg_size'='3287671', 'max_size'='3287671' )
-    '''
-
-    sql '''
-    alter table item modify column i_category set stats ('row_count'='204000', 'ndv'='11', 'min_value'='', 'max_value'='Women', 'avg_size'='1201703', 'max_size'='1201703' )
-    '''
-
-    sql '''
-    alter table item modify column i_class_id set stats ('row_count'='204000', 'ndv'='16', 'min_value'='1', 'max_value'='16', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_class_id set stats ('row_count'='204000', 'ndv'='16', 'min_value'='1', 'max_value'='16', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_container set stats ('row_count'='204000', 'ndv'='2', 'min_value'='', 'max_value'='Unknown', 'avg_size'='1424430', 'max_size'='1424430' )
-    '''
-
-    sql '''
-    alter table item modify column i_manager_id set stats ('row_count'='204000', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_refunded_customer_sk set stats ('row_count'='7197670', 'ndv'='1923644', 'min_value'='1', 'max_value'='2000000', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_return_amt_inc_tax set stats ('row_count'='7197670', 'ndv'='19975', 'min_value'='0.00', 'max_value'='29493.38', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_return_quantity set stats ('row_count'='7197670', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returned_time_sk set stats ('row_count'='7197670', 'ndv'='87677', 'min_value'='0', 'max_value'='86399', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returning_hdemo_sk set stats ('row_count'='7197670', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_reversed_charge set stats ('row_count'='7197670', 'ndv'='10979', 'min_value'='0.00', 'max_value'='22972.36', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_class set stats ('row_count'='24', 'ndv'='1', 'min_value'='Unknown', 'max_value'='Unknown', 'avg_size'='168', 'max_size'='168' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_close_date_sk set stats ('row_count'='24', 'ndv'='8', 'min_value'='2443328', 'max_value'='2447131', 'avg_size'='192', 'max_size'='192' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_close_date_sk set stats ('row_count'='24', 'ndv'='8', 'min_value'='2443328', 'max_value'='2447131', 'avg_size'='192', 'max_size'='192' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_state set stats ('row_count'='24', 'ndv'='9', 'min_value'='AL', 'max_value'='TN', 'avg_size'='48', 'max_size'='48' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_street_number set stats ('row_count'='24', 'ndv'='14', 'min_value'='184', 'max_value'='973', 'avg_size'='70', 'max_size'='70' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_catalog set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='986', 'max_size'='986' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_details set stats ('row_count'='1000', 'ndv'='992', 'min_value'='', 'max_value'='Young, valuable companies watch walls. Payments can flour', 'avg_size'='39304', 'max_size'='39304' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_event set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='986', 'max_size'='986' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_event set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='986', 'max_size'='986' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_radio set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='987', 'max_size'='987' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_response_targe set stats ('row_count'='1000', 'ndv'='1', 'min_value'='1', 'max_value'='1', 'avg_size'='4000', 'max_size'='4000' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_bill_cdemo_sk set stats ('row_count'='72001237', 'ndv'='1835731', 'min_value'='1', 'max_value'='1920800', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_list_price set stats ('row_count'='72001237', 'ndv'='301', 'min_value'='1.00', 'max_value'='300.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_net_paid_inc_ship set stats ('row_count'='72001237', 'ndv'='36553', 'min_value'='0.00', 'max_value'='43468.92', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_net_paid_inc_ship_tax set stats ('row_count'='72001237', 'ndv'='37541', 'min_value'='0.00', 'max_value'='44479.52', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_order_number set stats ('row_count'='72001237', 'ndv'='6015811', 'min_value'='1', 'max_value'='6000000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table store modify column s_company_id set stats ('row_count'='402', 'ndv'='1', 'min_value'='1', 'max_value'='1', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_country set stats ('row_count'='402', 'ndv'='2', 'min_value'='', 'max_value'='United States', 'avg_size'='5174', 'max_size'='5174' )
-    '''
-
-    sql '''
-    alter table store modify column s_county set stats ('row_count'='402', 'ndv'='10', 'min_value'='', 'max_value'='Ziebach County', 'avg_size'='5693', 'max_size'='5693' )
-    '''
-
-    sql '''
-    alter table store modify column s_division_id set stats ('row_count'='402', 'ndv'='1', 'min_value'='1', 'max_value'='1', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_manager set stats ('row_count'='402', 'ndv'='301', 'min_value'='', 'max_value'='Zachary Price', 'avg_size'='5075', 'max_size'='5075' )
-    '''
-
-    sql '''
-    alter table store modify column s_manager set stats ('row_count'='402', 'ndv'='301', 'min_value'='', 'max_value'='Zachary Price', 'avg_size'='5075', 'max_size'='5075' )
-    '''
-
-    sql '''
-    alter table store modify column s_market_id set stats ('row_count'='402', 'ndv'='10', 'min_value'='1', 'max_value'='10', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_store_sk set stats ('row_count'='402', 'ndv'='398', 'min_value'='1', 'max_value'='402', 'avg_size'='3216', 'max_size'='3216' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_am_pm set stats ('row_count'='86400', 'ndv'='2', 'min_value'='AM', 'max_value'='PM', 'avg_size'='172800', 'max_size'='172800' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_minute set stats ('row_count'='86400', 'ndv'='60', 'min_value'='0', 'max_value'='59', 'avg_size'='345600', 'max_size'='345600' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_time_id set stats ('row_count'='86400', 'ndv'='85663', 'min_value'='AAAAAAAAAAAABAAA', 'max_value'='AAAAAAAAPPPPAAAA', 'avg_size'='1382400', 'max_size'='1382400' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_image_count set stats ('row_count'='2040', 'ndv'='7', 'min_value'='1', 'max_value'='7', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_max_ad_count set stats ('row_count'='2040', 'ndv'='5', 'min_value'='0', 'max_value'='4', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_type set stats ('row_count'='2040', 'ndv'='8', 'min_value'='', 'max_value'='welcome', 'avg_size'='12856', 'max_size'='12856' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_url set stats ('row_count'='2040', 'ndv'='2', 'min_value'='', 'max_value'='http://www.foo.com', 'avg_size'='36270', 'max_size'='36270' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_web_page_id set stats ('row_count'='2040', 'ndv'='1019', 'min_value'='AAAAAAAAAABAAAAA', 'max_value'='AAAAAAAAPPEAAAAA', 'avg_size'='32640', 'max_size'='32640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_reason_sk set stats ('row_count'='28795080', 'ndv'='55', 'min_value'='1', 'max_value'='55', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_ship_cost set stats ('row_count'='28795080', 'ndv'='8186', 'min_value'='0.00', 'max_value'='9578.25', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_store_sk set stats ('row_count'='28795080', 'ndv'='200', 'min_value'='1', 'max_value'='400', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_addr_sk set stats ('row_count'='287997024', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_coupon_amt set stats ('row_count'='287997024', 'ndv'='16198', 'min_value'='0.00', 'max_value'='19225.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ext_discount_amt set stats ('row_count'='287997024', 'ndv'='16198', 'min_value'='0.00', 'max_value'='19225.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ext_tax set stats ('row_count'='287997024', 'ndv'='1722', 'min_value'='0.00', 'max_value'='1762.38', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_hdemo_sk set stats ('row_count'='287997024', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_list_price set stats ('row_count'='287997024', 'ndv'='201', 'min_value'='1.00', 'max_value'='200.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_net_paid_inc_tax set stats ('row_count'='287997024', 'ndv'='20203', 'min_value'='0.00', 'max_value'='21344.38', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_sold_time_sk set stats ('row_count'='287997024', 'ndv'='47252', 'min_value'='28800', 'max_value'='75599', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_code set stats ('row_count'='20', 'ndv'='4', 'min_value'='AIR', 'max_value'='SURFACE', 'avg_size'='87', 'max_size'='87' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_contract set stats ('row_count'='20', 'ndv'='20', 'min_value'='2mM8l', 'max_value'='yVfotg7Tio3MVhBg6Bkn', 'avg_size'='252', 'max_size'='252' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_type set stats ('row_count'='20', 'ndv'='6', 'min_value'='EXPRESS', 'max_value'='TWO DAY', 'avg_size'='150', 'max_size'='150' )
-    '''
-
-    sql '''
-    alter table customer modify column c_birth_day set stats ('row_count'='2000000', 'ndv'='31', 'min_value'='1', 'max_value'='31', 'avg_size'='8000000', 'max_size'='8000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_first_sales_date_sk set stats ('row_count'='2000000', 'ndv'='3644', 'min_value'='2448998', 'max_value'='2452648', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_last_name set stats ('row_count'='2000000', 'ndv'='4990', 'min_value'='', 'max_value'='Zuniga', 'avg_size'='11833714', 'max_size'='11833714' )
-    '''
-
-    sql '''
-    alter table dbgen_version modify column dv_version set stats ('row_count'='1', 'ndv'='1', 'min_value'='3.2.0', 'max_value'='3.2.0', 'avg_size'='5', 'max_size'='5' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_dep_employed_count set stats ('row_count'='1920800', 'ndv'='7', 'min_value'='0', 'max_value'='6', 'avg_size'='7683200', 'max_size'='7683200' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_education_status set stats ('row_count'='1920800', 'ndv'='7', 'min_value'='2 yr Degree', 'max_value'='Unknown', 'avg_size'='18384800', 'max_size'='18384800' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_purchase_estimate set stats ('row_count'='1920800', 'ndv'='20', 'min_value'='500', 'max_value'='10000', 'avg_size'='7683200', 'max_size'='7683200' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_dow set stats ('row_count'='73049', 'ndv'='7', 'min_value'='0', 'max_value'='6', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_following_holiday set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_moy set stats ('row_count'='73049', 'ndv'='12', 'min_value'='1', 'max_value'='12', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_quarter_seq set stats ('row_count'='73049', 'ndv'='801', 'min_value'='1', 'max_value'='801', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_street_number set stats ('row_count'='15', 'ndv'='15', 'min_value'='', 'max_value'='957', 'avg_size'='40', 'max_size'='40' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_street_type set stats ('row_count'='15', 'ndv'='11', 'min_value'='', 'max_value'='Wy', 'avg_size'='58', 'max_size'='58' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_warehouse_sk set stats ('row_count'='15', 'ndv'='15', 'min_value'='1', 'max_value'='15', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_catalog_page_sk set stats ('row_count'='143997065', 'ndv'='11515', 'min_value'='1', 'max_value'='17108', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_net_profit set stats ('row_count'='143997065', 'ndv'='28450', 'min_value'='-10000.00', 'max_value'='19840.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_cdemo_sk set stats ('row_count'='143997065', 'ndv'='1916125', 'min_value'='1', 'max_value'='1920800', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_customer_sk set stats ('row_count'='143997065', 'ndv'='1993190', 'min_value'='1', 'max_value'='2000000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_mode_sk set stats ('row_count'='143997065', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_mode_sk set stats ('row_count'='143997065', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_employees set stats ('row_count'='30', 'ndv'='22', 'min_value'='2935', 'max_value'='69020', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_gmt_offset set stats ('row_count'='30', 'ndv'='2', 'min_value'='-6.00', 'max_value'='-5.00', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_manager set stats ('row_count'='30', 'ndv'='22', 'min_value'='Alden Snyder', 'max_value'='Wayne Ray', 'avg_size'='368', 'max_size'='368' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_market_manager set stats ('row_count'='30', 'ndv'='24', 'min_value'='Charles Corbett', 'max_value'='Tom Root', 'avg_size'='373', 'max_size'='373' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_mkt_id set stats ('row_count'='30', 'ndv'='6', 'min_value'='1', 'max_value'='6', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_rec_end_date set stats ('row_count'='30', 'ndv'='3', 'min_value'='2000-01-01', 'max_value'='2001-12-31', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_rec_start_date set stats ('row_count'='30', 'ndv'='4', 'min_value'='1998-01-01', 'max_value'='2002-01-01', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table inventory modify column inv_date_sk set stats ('row_count'='399330000', 'ndv'='261', 'min_value'='2450815', 'max_value'='2452635', 'avg_size'='3194640000', 'max_size'='3194640000' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_call_center_sk set stats ('row_count'='14404374', 'ndv'='30', 'min_value'='1', 'max_value'='30', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_fee set stats ('row_count'='14404374', 'ndv'='101', 'min_value'='0.50', 'max_value'='100.00', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_refunded_addr_sk set stats ('row_count'='14404374', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_refunded_customer_sk set stats ('row_count'='14404374', 'ndv'='1977657', 'min_value'='1', 'max_value'='2000000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_return_amt_inc_tax set stats ('row_count'='14404374', 'ndv'='21566', 'min_value'='0.00', 'max_value'='29353.87', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_return_quantity set stats ('row_count'='14404374', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_return_ship_cost set stats ('row_count'='14404374', 'ndv'='11144', 'min_value'='0.00', 'max_value'='14130.96', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returning_hdemo_sk set stats ('row_count'='14404374', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_reversed_charge set stats ('row_count'='14404374', 'ndv'='12359', 'min_value'='0.00', 'max_value'='23801.24', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_ship_mode_sk set stats ('row_count'='14404374', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_warehouse_sk set stats ('row_count'='14404374', 'ndv'='15', 'min_value'='1', 'max_value'='15', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table household_demographics modify column hd_buy_potential set stats ('row_count'='7200', 'ndv'='6', 'min_value'='0-500', 'max_value'='Unknown', 'avg_size'='54000', 'max_size'='54000' )
-    '''
-
-    sql '''
-    alter table household_demographics modify column hd_demo_sk set stats ('row_count'='7200', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='57600', 'max_size'='57600' )
-    '''
-
-    sql '''
-    alter table household_demographics modify column hd_dep_count set stats ('row_count'='7200', 'ndv'='10', 'min_value'='0', 'max_value'='9', 'avg_size'='28800', 'max_size'='28800' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_address_id set stats ('row_count'='1000000', 'ndv'='999950', 'min_value'='AAAAAAAAAAAABAAA', 'max_value'='AAAAAAAAPPPPOAAA', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_address_sk set stats ('row_count'='1000000', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='8000000', 'max_size'='8000000' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_country set stats ('row_count'='1000000', 'ndv'='2', 'min_value'='', 'max_value'='United States', 'avg_size'='12608739', 'max_size'='12608739' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_country set stats ('row_count'='1000000', 'ndv'='2', 'min_value'='', 'max_value'='United States', 'avg_size'='12608739', 'max_size'='12608739' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_county set stats ('row_count'='1000000', 'ndv'='1825', 'min_value'='', 'max_value'='Ziebach County', 'avg_size'='13540273', 'max_size'='13540273' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_zip set stats ('row_count'='1000000', 'ndv'='7733', 'min_value'='', 'max_value'='99981', 'avg_size'='4848150', 'max_size'='4848150' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_catalog_page_number set stats ('row_count'='20400', 'ndv'='189', 'min_value'='1', 'max_value'='188', 'avg_size'='81600', 'max_size'='81600' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_description set stats ('row_count'='20400', 'ndv'='20501', 'min_value'='', 'max_value'='Youngsters should get very. Bad, necessary years must pick telecommunications. Co', 'avg_size'='1507423', 'max_size'='1507423' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_start_date_sk set stats ('row_count'='20400', 'ndv'='91', 'min_value'='2450815', 'max_value'='2453005', 'avg_size'='81600', 'max_size'='81600' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_type set stats ('row_count'='20400', 'ndv'='4', 'min_value'='', 'max_value'='quarterly', 'avg_size'='155039', 'max_size'='155039' )
-    '''
-
-    sql '''
-    alter table item modify column i_category_id set stats ('row_count'='204000', 'ndv'='10', 'min_value'='1', 'max_value'='10', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_color set stats ('row_count'='204000', 'ndv'='93', 'min_value'='', 'max_value'='yellow', 'avg_size'='1094247', 'max_size'='1094247' )
-    '''
-
-    sql '''
-    alter table item modify column i_item_id set stats ('row_count'='204000', 'ndv'='103230', 'min_value'='AAAAAAAAAAAABAAA', 'max_value'='AAAAAAAAPPPPBAAA', 'avg_size'='3264000', 'max_size'='3264000' )
-    '''
-
-    sql '''
-    alter table item modify column i_product_name set stats ('row_count'='204000', 'ndv'='200390', 'min_value'='', 'max_value'='pripripripripriought', 'avg_size'='4546148', 'max_size'='4546148' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_fee set stats ('row_count'='7197670', 'ndv'='101', 'min_value'='0.50', 'max_value'='100.00', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_fee set stats ('row_count'='7197670', 'ndv'='101', 'min_value'='0.50', 'max_value'='100.00', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_order_number set stats ('row_count'='7197670', 'ndv'='4249346', 'min_value'='1', 'max_value'='5999999', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_refunded_addr_sk set stats ('row_count'='7197670', 'ndv'='999503', 'min_value'='1', 'max_value'='1000000', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_refunded_cdemo_sk set stats ('row_count'='7197670', 'ndv'='1868495', 'min_value'='1', 'max_value'='1920800', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_refunded_customer_sk set stats ('row_count'='7197670', 'ndv'='1923644', 'min_value'='1', 'max_value'='2000000', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_return_amt set stats ('row_count'='7197670', 'ndv'='19263', 'min_value'='0.00', 'max_value'='28346.31', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returning_addr_sk set stats ('row_count'='7197670', 'ndv'='999584', 'min_value'='1', 'max_value'='1000000', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returning_cdemo_sk set stats ('row_count'='7197670', 'ndv'='1865149', 'min_value'='1', 'max_value'='1920800', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_web_page_sk set stats ('row_count'='7197670', 'ndv'='2032', 'min_value'='1', 'max_value'='2040', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_company_id set stats ('row_count'='24', 'ndv'='6', 'min_value'='1', 'max_value'='6', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_manager set stats ('row_count'='24', 'ndv'='19', 'min_value'='Adam Stonge', 'max_value'='Tommy Jones', 'avg_size'='297', 'max_size'='297' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_site_id set stats ('row_count'='24', 'ndv'='12', 'min_value'='AAAAAAAAABAAAAAA', 'max_value'='AAAAAAAAOAAAAAAA', 'avg_size'='384', 'max_size'='384' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_site_sk set stats ('row_count'='24', 'ndv'='24', 'min_value'='1', 'max_value'='24', 'avg_size'='192', 'max_size'='192' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_street_number set stats ('row_count'='24', 'ndv'='14', 'min_value'='184', 'max_value'='973', 'avg_size'='70', 'max_size'='70' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_catalog set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='986', 'max_size'='986' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_press set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='985', 'max_size'='985' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_discount_active set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='981', 'max_size'='981' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_item_sk set stats ('row_count'='1000', 'ndv'='970', 'min_value'='280', 'max_value'='203966', 'avg_size'='8000', 'max_size'='8000' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_promo_sk set stats ('row_count'='1000', 'ndv'='986', 'min_value'='1', 'max_value'='1000', 'avg_size'='8000', 'max_size'='8000' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_start_date_sk set stats ('row_count'='1000', 'ndv'='574', 'min_value'='2450100', 'max_value'='2450915', 'avg_size'='8000', 'max_size'='8000' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_start_date_sk set stats ('row_count'='1000', 'ndv'='574', 'min_value'='2450100', 'max_value'='2450915', 'avg_size'='8000', 'max_size'='8000' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_bill_hdemo_sk set stats ('row_count'='72001237', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_net_paid_inc_ship set stats ('row_count'='72001237', 'ndv'='36553', 'min_value'='0.00', 'max_value'='43468.92', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_addr_sk set stats ('row_count'='72001237', 'ndv'='997336', 'min_value'='1', 'max_value'='1000000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_mode_sk set stats ('row_count'='72001237', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_sold_time_sk set stats ('row_count'='72001237', 'ndv'='87677', 'min_value'='0', 'max_value'='86399', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table store modify column s_number_employees set stats ('row_count'='402', 'ndv'='97', 'min_value'='200', 'max_value'='300', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_number_employees set stats ('row_count'='402', 'ndv'='97', 'min_value'='200', 'max_value'='300', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_state set stats ('row_count'='402', 'ndv'='10', 'min_value'='', 'max_value'='TN', 'avg_size'='800', 'max_size'='800' )
-    '''
-
-    sql '''
-    alter table store modify column s_suite_number set stats ('row_count'='402', 'ndv'='75', 'min_value'='', 'max_value'='Suite Y', 'avg_size'='3140', 'max_size'='3140' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_meal_time set stats ('row_count'='86400', 'ndv'='4', 'min_value'='', 'max_value'='lunch', 'avg_size'='248400', 'max_size'='248400' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_second set stats ('row_count'='86400', 'ndv'='60', 'min_value'='0', 'max_value'='59', 'avg_size'='345600', 'max_size'='345600' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_second set stats ('row_count'='86400', 'ndv'='60', 'min_value'='0', 'max_value'='59', 'avg_size'='345600', 'max_size'='345600' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_autogen_flag set stats ('row_count'='2040', 'ndv'='3', 'min_value'='', 'max_value'='Y', 'avg_size'='2015', 'max_size'='2015' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_creation_date_sk set stats ('row_count'='2040', 'ndv'='134', 'min_value'='2450672', 'max_value'='2450815', 'avg_size'='16320', 'max_size'='16320' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_link_count set stats ('row_count'='2040', 'ndv'='24', 'min_value'='2', 'max_value'='25', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_item_sk set stats ('row_count'='28795080', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_net_loss set stats ('row_count'='28795080', 'ndv'='8663', 'min_value'='0.50', 'max_value'='10447.72', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_net_loss set stats ('row_count'='28795080', 'ndv'='8663', 'min_value'='0.50', 'max_value'='10447.72', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_refunded_cash set stats ('row_count'='28795080', 'ndv'='12626', 'min_value'='0.00', 'max_value'='17556.95', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_time_sk set stats ('row_count'='28795080', 'ndv'='32660', 'min_value'='28799', 'max_value'='61199', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_item_sk set stats ('row_count'='287997024', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_sold_date_sk set stats ('row_count'='287997024', 'ndv'='1820', 'min_value'='2450816', 'max_value'='2452642', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_sold_time_sk set stats ('row_count'='287997024', 'ndv'='47252', 'min_value'='28800', 'max_value'='75599', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ticket_number set stats ('row_count'='287997024', 'ndv'='23905324', 'min_value'='1', 'max_value'='24000000', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_wholesale_cost set stats ('row_count'='287997024', 'ndv'='100', 'min_value'='1.00', 'max_value'='100.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_wholesale_cost set stats ('row_count'='287997024', 'ndv'='100', 'min_value'='1.00', 'max_value'='100.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_carrier set stats ('row_count'='20', 'ndv'='20', 'min_value'='AIRBORNE', 'max_value'='ZOUROS', 'avg_size'='133', 'max_size'='133' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_code set stats ('row_count'='20', 'ndv'='4', 'min_value'='AIR', 'max_value'='SURFACE', 'avg_size'='87', 'max_size'='87' )
-    '''
-
-    sql '''
-    alter table customer modify column c_birth_country set stats ('row_count'='2000000', 'ndv'='211', 'min_value'='', 'max_value'='ZIMBABWE', 'avg_size'='16787900', 'max_size'='16787900' )
-    '''
-
-    sql '''
-    alter table customer modify column c_birth_month set stats ('row_count'='2000000', 'ndv'='12', 'min_value'='1', 'max_value'='12', 'avg_size'='8000000', 'max_size'='8000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_current_cdemo_sk set stats ('row_count'='2000000', 'ndv'='1221921', 'min_value'='1', 'max_value'='1920798', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_customer_sk set stats ('row_count'='2000000', 'ndv'='1994393', 'min_value'='1', 'max_value'='2000000', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_customer_sk set stats ('row_count'='2000000', 'ndv'='1994393', 'min_value'='1', 'max_value'='2000000', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_first_name set stats ('row_count'='2000000', 'ndv'='5140', 'min_value'='', 'max_value'='Zulma', 'avg_size'='11267996', 'max_size'='11267996' )
-    '''
-
-    sql '''
-    alter table customer modify column c_salutation set stats ('row_count'='2000000', 'ndv'='7', 'min_value'='', 'max_value'='Sir', 'avg_size'='6257882', 'max_size'='6257882' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_gender set stats ('row_count'='1920800', 'ndv'='2', 'min_value'='F', 'max_value'='M', 'avg_size'='1920800', 'max_size'='1920800' )
-    '''
-
-    sql '''
-    alter table reason modify column r_reason_id set stats ('row_count'='55', 'ndv'='55', 'min_value'='AAAAAAAAABAAAAAA', 'max_value'='AAAAAAAAPCAAAAAA', 'avg_size'='880', 'max_size'='880' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_current_day set stats ('row_count'='73049', 'ndv'='1', 'min_value'='N', 'max_value'='N', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_current_month set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_current_quarter set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_date_id set stats ('row_count'='73049', 'ndv'='72907', 'min_value'='AAAAAAAAAAAAFCAA', 'max_value'='AAAAAAAAPPPPECAA', 'avg_size'='1168784', 'max_size'='1168784' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_dow set stats ('row_count'='73049', 'ndv'='7', 'min_value'='0', 'max_value'='6', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_following_holiday set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_fy_year set stats ('row_count'='73049', 'ndv'='202', 'min_value'='1900', 'max_value'='2100', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_last_dom set stats ('row_count'='73049', 'ndv'='2419', 'min_value'='2415020', 'max_value'='2488372', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_month_seq set stats ('row_count'='73049', 'ndv'='2398', 'min_value'='0', 'max_value'='2400', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_quarter_name set stats ('row_count'='73049', 'ndv'='799', 'min_value'='1900Q1', 'max_value'='2100Q1', 'avg_size'='438294', 'max_size'='438294' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_same_day_lq set stats ('row_count'='73049', 'ndv'='72231', 'min_value'='2414930', 'max_value'='2487978', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_same_day_ly set stats ('row_count'='73049', 'ndv'='72450', 'min_value'='2414657', 'max_value'='2487705', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_weekend set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_year set stats ('row_count'='73049', 'ndv'='202', 'min_value'='1900', 'max_value'='2100', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_city set stats ('row_count'='15', 'ndv'='11', 'min_value'='Bethel', 'max_value'='Union', 'avg_size'='111', 'max_size'='111' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_country set stats ('row_count'='15', 'ndv'='1', 'min_value'='United States', 'max_value'='United States', 'avg_size'='195', 'max_size'='195' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_gmt_offset set stats ('row_count'='15', 'ndv'='2', 'min_value'='-6.00', 'max_value'='-5.00', 'avg_size'='60', 'max_size'='60' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_street_name set stats ('row_count'='15', 'ndv'='15', 'min_value'='', 'max_value'='Wilson Elm', 'avg_size'='128', 'max_size'='128' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_bill_cdemo_sk set stats ('row_count'='143997065', 'ndv'='1915709', 'min_value'='1', 'max_value'='1920800', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_sales_price set stats ('row_count'='143997065', 'ndv'='27598', 'min_value'='0.00', 'max_value'='29808.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_net_paid_inc_ship_tax set stats ('row_count'='143997065', 'ndv'='38890', 'min_value'='0.00', 'max_value'='45460.80', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_order_number set stats ('row_count'='143997065', 'ndv'='16050730', 'min_value'='1', 'max_value'='16000000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_customer_sk set stats ('row_count'='143997065', 'ndv'='1993190', 'min_value'='1', 'max_value'='2000000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_hdemo_sk set stats ('row_count'='143997065', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_sold_date_sk set stats ('row_count'='143997065', 'ndv'='1835', 'min_value'='2450815', 'max_value'='2452654', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_city set stats ('row_count'='30', 'ndv'='12', 'min_value'='Bethel', 'max_value'='Shady Grove', 'avg_size'='282', 'max_size'='282' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_employees set stats ('row_count'='30', 'ndv'='22', 'min_value'='2935', 'max_value'='69020', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_gmt_offset set stats ('row_count'='30', 'ndv'='2', 'min_value'='-6.00', 'max_value'='-5.00', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_hours set stats ('row_count'='30', 'ndv'='3', 'min_value'='8AM-12AM', 'max_value'='8AM-8AM', 'avg_size'='214', 'max_size'='214' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_hours set stats ('row_count'='30', 'ndv'='3', 'min_value'='8AM-12AM', 'max_value'='8AM-8AM', 'avg_size'='214', 'max_size'='214' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_mkt_desc set stats ('row_count'='30', 'ndv'='22', 'min_value'='As existing eyebrows miss as the matters. Realistic stories may not face almost by a ', 'max_value'='Young tests could buy comfortable, local users; o', 'avg_size'='1766', 'max_size'='1766' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_name set stats ('row_count'='30', 'ndv'='15', 'min_value'='California', 'max_value'='Pacific Northwest_1', 'avg_size'='401', 'max_size'='401' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_name set stats ('row_count'='30', 'ndv'='15', 'min_value'='California', 'max_value'='Pacific Northwest_1', 'avg_size'='401', 'max_size'='401' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_sq_ft set stats ('row_count'='30', 'ndv'='22', 'min_value'='1670015', 'max_value'='31896816', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table inventory modify column inv_item_sk set stats ('row_count'='399330000', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='3194640000', 'max_size'='3194640000' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_call_center_sk set stats ('row_count'='14404374', 'ndv'='30', 'min_value'='1', 'max_value'='30', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_refunded_customer_sk set stats ('row_count'='14404374', 'ndv'='1977657', 'min_value'='1', 'max_value'='2000000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_return_ship_cost set stats ('row_count'='14404374', 'ndv'='11144', 'min_value'='0.00', 'max_value'='14130.96', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returned_time_sk set stats ('row_count'='14404374', 'ndv'='87677', 'min_value'='0', 'max_value'='86399', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returning_cdemo_sk set stats ('row_count'='14404374', 'ndv'='1913762', 'min_value'='1', 'max_value'='1920800', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table household_demographics modify column hd_vehicle_count set stats ('row_count'='7200', 'ndv'='6', 'min_value'='-1', 'max_value'='4', 'avg_size'='28800', 'max_size'='28800' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_location_type set stats ('row_count'='1000000', 'ndv'='4', 'min_value'='', 'max_value'='single family', 'avg_size'='8728128', 'max_size'='8728128' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_state set stats ('row_count'='1000000', 'ndv'='52', 'min_value'='', 'max_value'='WY', 'avg_size'='1939752', 'max_size'='1939752' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_street_number set stats ('row_count'='1000000', 'ndv'='1002', 'min_value'='', 'max_value'='999', 'avg_size'='2805540', 'max_size'='2805540' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_department set stats ('row_count'='20400', 'ndv'='2', 'min_value'='', 'max_value'='DEPARTMENT', 'avg_size'='201950', 'max_size'='201950' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_description set stats ('row_count'='20400', 'ndv'='20501', 'min_value'='', 'max_value'='Youngsters should get very. Bad, necessary years must pick telecommunications. Co', 'avg_size'='1507423', 'max_size'='1507423' )
-    '''
-
-    sql '''
-    alter table item modify column i_brand_id set stats ('row_count'='204000', 'ndv'='951', 'min_value'='1001001', 'max_value'='10016017', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_class set stats ('row_count'='204000', 'ndv'='100', 'min_value'='', 'max_value'='womens watch', 'avg_size'='1585937', 'max_size'='1585937' )
-    '''
-
-    sql '''
-    alter table item modify column i_units set stats ('row_count'='204000', 'ndv'='22', 'min_value'='', 'max_value'='Unknown', 'avg_size'='852562', 'max_size'='852562' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_account_credit set stats ('row_count'='7197670', 'ndv'='10868', 'min_value'='0.00', 'max_value'='23028.27', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_reason_sk set stats ('row_count'='7197670', 'ndv'='55', 'min_value'='1', 'max_value'='55', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_refunded_hdemo_sk set stats ('row_count'='7197670', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returned_date_sk set stats ('row_count'='7197670', 'ndv'='2185', 'min_value'='2450820', 'max_value'='2453002', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returned_time_sk set stats ('row_count'='7197670', 'ndv'='87677', 'min_value'='0', 'max_value'='86399', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_company_name set stats ('row_count'='24', 'ndv'='6', 'min_value'='able', 'max_value'='pri', 'avg_size'='97', 'max_size'='97' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_gmt_offset set stats ('row_count'='24', 'ndv'='2', 'min_value'='-6.00', 'max_value'='-5.00', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_market_manager set stats ('row_count'='24', 'ndv'='21', 'min_value'='Albert Leung', 'max_value'='Zachery Oneil', 'avg_size'='294', 'max_size'='294' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_mkt_desc set stats ('row_count'='24', 'ndv'='15', 'min_value'='Acres see else children. Mutual too', 'max_value'='Well similar decisions used to keep hardly democratic, personal priorities.', 'avg_size'='1561', 'max_size'='1561' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_rec_end_date set stats ('row_count'='24', 'ndv'='3', 'min_value'='1999-08-16', 'max_value'='2001-08-15', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_site_id set stats ('row_count'='24', 'ndv'='12', 'min_value'='AAAAAAAAABAAAAAA', 'max_value'='AAAAAAAAOAAAAAAA', 'avg_size'='384', 'max_size'='384' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_site_sk set stats ('row_count'='24', 'ndv'='24', 'min_value'='1', 'max_value'='24', 'avg_size'='192', 'max_size'='192' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_state set stats ('row_count'='24', 'ndv'='9', 'min_value'='AL', 'max_value'='TN', 'avg_size'='48', 'max_size'='48' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_tv set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='986', 'max_size'='986' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_promo_name set stats ('row_count'='1000', 'ndv'='11', 'min_value'='', 'max_value'='pri', 'avg_size'='3924', 'max_size'='3924' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_promo_sk set stats ('row_count'='1000', 'ndv'='986', 'min_value'='1', 'max_value'='1000', 'avg_size'='8000', 'max_size'='8000' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_list_price set stats ('row_count'='72001237', 'ndv'='29104', 'min_value'='1.02', 'max_value'='29997.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_order_number set stats ('row_count'='72001237', 'ndv'='6015811', 'min_value'='1', 'max_value'='6000000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_addr_sk set stats ('row_count'='72001237', 'ndv'='997336', 'min_value'='1', 'max_value'='1000000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_cdemo_sk set stats ('row_count'='72001237', 'ndv'='1822804', 'min_value'='1', 'max_value'='1920800', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_customer_sk set stats ('row_count'='72001237', 'ndv'='1898561', 'min_value'='1', 'max_value'='2000000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_date_sk set stats ('row_count'='72001237', 'ndv'='1952', 'min_value'='2450817', 'max_value'='2452762', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_hdemo_sk set stats ('row_count'='72001237', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_warehouse_sk set stats ('row_count'='72001237', 'ndv'='15', 'min_value'='1', 'max_value'='15', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table store modify column s_company_id set stats ('row_count'='402', 'ndv'='1', 'min_value'='1', 'max_value'='1', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_country set stats ('row_count'='402', 'ndv'='2', 'min_value'='', 'max_value'='United States', 'avg_size'='5174', 'max_size'='5174' )
-    '''
-
-    sql '''
-    alter table store modify column s_division_id set stats ('row_count'='402', 'ndv'='1', 'min_value'='1', 'max_value'='1', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_division_name set stats ('row_count'='402', 'ndv'='2', 'min_value'='', 'max_value'='Unknown', 'avg_size'='2779', 'max_size'='2779' )
-    '''
-
-    sql '''
-    alter table store modify column s_floor_space set stats ('row_count'='402', 'ndv'='300', 'min_value'='5004767', 'max_value'='9997773', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_geography_class set stats ('row_count'='402', 'ndv'='2', 'min_value'='', 'max_value'='Unknown', 'avg_size'='2793', 'max_size'='2793' )
-    '''
-
-    sql '''
-    alter table store modify column s_geography_class set stats ('row_count'='402', 'ndv'='2', 'min_value'='', 'max_value'='Unknown', 'avg_size'='2793', 'max_size'='2793' )
-    '''
-
-    sql '''
-    alter table store modify column s_hours set stats ('row_count'='402', 'ndv'='4', 'min_value'='', 'max_value'='8AM-8AM', 'avg_size'='2848', 'max_size'='2848' )
-    '''
-
-    sql '''
-    alter table store modify column s_hours set stats ('row_count'='402', 'ndv'='4', 'min_value'='', 'max_value'='8AM-8AM', 'avg_size'='2848', 'max_size'='2848' )
-    '''
-
-    sql '''
-    alter table store modify column s_street_name set stats ('row_count'='402', 'ndv'='256', 'min_value'='', 'max_value'='Woodland ', 'avg_size'='3384', 'max_size'='3384' )
-    '''
-
-    sql '''
-    alter table store modify column s_street_number set stats ('row_count'='402', 'ndv'='267', 'min_value'='', 'max_value'='986', 'avg_size'='1150', 'max_size'='1150' )
-    '''
-
-    sql '''
-    alter table store modify column s_street_type set stats ('row_count'='402', 'ndv'='21', 'min_value'='', 'max_value'='Wy', 'avg_size'='1657', 'max_size'='1657' )
-    '''
-
-    sql '''
-    alter table store modify column s_tax_precentage set stats ('row_count'='402', 'ndv'='1', 'min_value'='0.00', 'max_value'='0.11', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_meal_time set stats ('row_count'='86400', 'ndv'='4', 'min_value'='', 'max_value'='lunch', 'avg_size'='248400', 'max_size'='248400' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_sub_shift set stats ('row_count'='86400', 'ndv'='4', 'min_value'='afternoon', 'max_value'='night', 'avg_size'='597600', 'max_size'='597600' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_time_id set stats ('row_count'='86400', 'ndv'='85663', 'min_value'='AAAAAAAAAAAABAAA', 'max_value'='AAAAAAAAPPPPAAAA', 'avg_size'='1382400', 'max_size'='1382400' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_char_count set stats ('row_count'='2040', 'ndv'='1363', 'min_value'='303', 'max_value'='8523', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_max_ad_count set stats ('row_count'='2040', 'ndv'='5', 'min_value'='0', 'max_value'='4', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_rec_start_date set stats ('row_count'='2040', 'ndv'='4', 'min_value'='1997-09-03', 'max_value'='2001-09-03', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_web_page_sk set stats ('row_count'='2040', 'ndv'='2032', 'min_value'='1', 'max_value'='2040', 'avg_size'='16320', 'max_size'='16320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_cdemo_sk set stats ('row_count'='28795080', 'ndv'='1916366', 'min_value'='1', 'max_value'='1920800', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_hdemo_sk set stats ('row_count'='28795080', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_item_sk set stats ('row_count'='28795080', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_reason_sk set stats ('row_count'='28795080', 'ndv'='55', 'min_value'='1', 'max_value'='55', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_cdemo_sk set stats ('row_count'='287997024', 'ndv'='1916366', 'min_value'='1', 'max_value'='1920800', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ext_wholesale_cost set stats ('row_count'='287997024', 'ndv'='10009', 'min_value'='1.00', 'max_value'='10000.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_net_paid_inc_tax set stats ('row_count'='287997024', 'ndv'='20203', 'min_value'='0.00', 'max_value'='21344.38', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_promo_sk set stats ('row_count'='287997024', 'ndv'='986', 'min_value'='1', 'max_value'='1000', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_sold_date_sk set stats ('row_count'='287997024', 'ndv'='1820', 'min_value'='2450816', 'max_value'='2452642', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ticket_number set stats ('row_count'='287997024', 'ndv'='23905324', 'min_value'='1', 'max_value'='24000000', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_contract set stats ('row_count'='20', 'ndv'='20', 'min_value'='2mM8l', 'max_value'='yVfotg7Tio3MVhBg6Bkn', 'avg_size'='252', 'max_size'='252' )
-    '''
-
-    sql '''
-    alter table customer modify column c_birth_country set stats ('row_count'='2000000', 'ndv'='211', 'min_value'='', 'max_value'='ZIMBABWE', 'avg_size'='16787900', 'max_size'='16787900' )
-    '''
-
-    sql '''
-    alter table customer modify column c_birth_year set stats ('row_count'='2000000', 'ndv'='69', 'min_value'='1924', 'max_value'='1992', 'avg_size'='8000000', 'max_size'='8000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_current_addr_sk set stats ('row_count'='2000000', 'ndv'='866672', 'min_value'='1', 'max_value'='1000000', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_current_addr_sk set stats ('row_count'='2000000', 'ndv'='866672', 'min_value'='1', 'max_value'='1000000', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_first_sales_date_sk set stats ('row_count'='2000000', 'ndv'='3644', 'min_value'='2448998', 'max_value'='2452648', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_first_shipto_date_sk set stats ('row_count'='2000000', 'ndv'='3644', 'min_value'='2449028', 'max_value'='2452678', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_last_review_date_sk set stats ('row_count'='2000000', 'ndv'='366', 'min_value'='2452283', 'max_value'='2452648', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_salutation set stats ('row_count'='2000000', 'ndv'='7', 'min_value'='', 'max_value'='Sir', 'avg_size'='6257882', 'max_size'='6257882' )
-    '''
-
-    sql '''
-    alter table dbgen_version modify column dv_cmdline_args set stats ('row_count'='1', 'ndv'='1', 'min_value'='-SCALE 100 -PARALLEL 10 -CHILD 1 -TERMINATE N -DIR /mnt/datadisk0/doris/tools/tpcds-tools/bin/tpcds-data ', 'max_value'='-SCALE 100 -PARALLEL 10 -CHILD 1 -TERMINATE N -DIR /mnt/datadisk0/doris/tools/tpcds-tools/bin/tpcds-data ', 'avg_size'='105', 'max_size'='105' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_marital_status set stats ('row_count'='1920800', 'ndv'='5', 'min_value'='D', 'max_value'='W', 'avg_size'='1920800', 'max_size'='1920800' )
-    '''
-
-    sql '''
-    alter table reason modify column r_reason_desc set stats ('row_count'='55', 'ndv'='54', 'min_value'='Did not fit', 'max_value'='unauthoized purchase', 'avg_size'='758', 'max_size'='758' )
-    '''
-
-    sql '''
-    alter table reason modify column r_reason_desc set stats ('row_count'='55', 'ndv'='54', 'min_value'='Did not fit', 'max_value'='unauthoized purchase', 'avg_size'='758', 'max_size'='758' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_current_week set stats ('row_count'='73049', 'ndv'='1', 'min_value'='N', 'max_value'='N', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_date set stats ('row_count'='73049', 'ndv'='73250', 'min_value'='1900-01-02', 'max_value'='2100-01-01', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_day_name set stats ('row_count'='73049', 'ndv'='7', 'min_value'='Friday', 'max_value'='Wednesday', 'avg_size'='521779', 'max_size'='521779' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_dom set stats ('row_count'='73049', 'ndv'='31', 'min_value'='1', 'max_value'='31', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_dom set stats ('row_count'='73049', 'ndv'='31', 'min_value'='1', 'max_value'='31', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_fy_week_seq set stats ('row_count'='73049', 'ndv'='10448', 'min_value'='1', 'max_value'='10436', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_moy set stats ('row_count'='73049', 'ndv'='12', 'min_value'='1', 'max_value'='12', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_qoy set stats ('row_count'='73049', 'ndv'='4', 'min_value'='1', 'max_value'='4', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_qoy set stats ('row_count'='73049', 'ndv'='4', 'min_value'='1', 'max_value'='4', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_same_day_ly set stats ('row_count'='73049', 'ndv'='72450', 'min_value'='2414657', 'max_value'='2487705', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_weekend set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_year set stats ('row_count'='73049', 'ndv'='202', 'min_value'='1900', 'max_value'='2100', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_country set stats ('row_count'='15', 'ndv'='1', 'min_value'='United States', 'max_value'='United States', 'avg_size'='195', 'max_size'='195' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_street_type set stats ('row_count'='15', 'ndv'='11', 'min_value'='', 'max_value'='Wy', 'avg_size'='58', 'max_size'='58' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_warehouse_name set stats ('row_count'='15', 'ndv'='15', 'min_value'='', 'max_value'='Rooms cook ', 'avg_size'='230', 'max_size'='230' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_warehouse_sq_ft set stats ('row_count'='15', 'ndv'='14', 'min_value'='73065', 'max_value'='977787', 'avg_size'='60', 'max_size'='60' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_tax set stats ('row_count'='143997065', 'ndv'='2488', 'min_value'='0.00', 'max_value'='2619.36', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_wholesale_cost set stats ('row_count'='143997065', 'ndv'='10009', 'min_value'='1.00', 'max_value'='10000.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_list_price set stats ('row_count'='143997065', 'ndv'='301', 'min_value'='1.00', 'max_value'='300.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_net_paid set stats ('row_count'='143997065', 'ndv'='27448', 'min_value'='0.00', 'max_value'='29760.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_sales_price set stats ('row_count'='143997065', 'ndv'='302', 'min_value'='0.00', 'max_value'='300.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_addr_sk set stats ('row_count'='143997065', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_warehouse_sk set stats ('row_count'='143997065', 'ndv'='15', 'min_value'='1', 'max_value'='15', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_wholesale_cost set stats ('row_count'='143997065', 'ndv'='100', 'min_value'='1.00', 'max_value'='100.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_call_center_sk set stats ('row_count'='30', 'ndv'='30', 'min_value'='1', 'max_value'='30', 'avg_size'='240', 'max_size'='240' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_class set stats ('row_count'='30', 'ndv'='3', 'min_value'='large', 'max_value'='small', 'avg_size'='166', 'max_size'='166' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_open_date_sk set stats ('row_count'='30', 'ndv'='15', 'min_value'='2450794', 'max_value'='2451146', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_rec_start_date set stats ('row_count'='30', 'ndv'='4', 'min_value'='1998-01-01', 'max_value'='2002-01-01', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_street_name set stats ('row_count'='30', 'ndv'='15', 'min_value'='1st ', 'max_value'='View ', 'avg_size'='240', 'max_size'='240' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_street_name set stats ('row_count'='30', 'ndv'='15', 'min_value'='1st ', 'max_value'='View ', 'avg_size'='240', 'max_size'='240' )
-    '''
-
-    sql '''
-    alter table inventory modify column inv_date_sk set stats ('row_count'='399330000', 'ndv'='261', 'min_value'='2450815', 'max_value'='2452635', 'avg_size'='3194640000', 'max_size'='3194640000' )
-    '''
-
-    sql '''
-    alter table inventory modify column inv_quantity_on_hand set stats ('row_count'='399330000', 'ndv'='1006', 'min_value'='0', 'max_value'='1000', 'avg_size'='1597320000', 'max_size'='1597320000' )
-    '''
-
-    sql '''
-    alter table inventory modify column inv_warehouse_sk set stats ('row_count'='399330000', 'ndv'='15', 'min_value'='1', 'max_value'='15', 'avg_size'='3194640000', 'max_size'='3194640000' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_refunded_hdemo_sk set stats ('row_count'='14404374', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_return_amount set stats ('row_count'='14404374', 'ndv'='20656', 'min_value'='0.00', 'max_value'='28778.31', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_return_amount set stats ('row_count'='14404374', 'ndv'='20656', 'min_value'='0.00', 'max_value'='28778.31', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_return_tax set stats ('row_count'='14404374', 'ndv'='1926', 'min_value'='0.00', 'max_value'='2390.75', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_store_credit set stats ('row_count'='14404374', 'ndv'='12156', 'min_value'='0.00', 'max_value'='22167.49', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_warehouse_sk set stats ('row_count'='14404374', 'ndv'='15', 'min_value'='1', 'max_value'='15', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table household_demographics modify column hd_dep_count set stats ('row_count'='7200', 'ndv'='10', 'min_value'='0', 'max_value'='9', 'avg_size'='28800', 'max_size'='28800' )
-    '''
-
-    sql '''
-    alter table household_demographics modify column hd_vehicle_count set stats ('row_count'='7200', 'ndv'='6', 'min_value'='-1', 'max_value'='4', 'avg_size'='28800', 'max_size'='28800' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_address_id set stats ('row_count'='1000000', 'ndv'='999950', 'min_value'='AAAAAAAAAAAABAAA', 'max_value'='AAAAAAAAPPPPOAAA', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_county set stats ('row_count'='1000000', 'ndv'='1825', 'min_value'='', 'max_value'='Ziebach County', 'avg_size'='13540273', 'max_size'='13540273' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_street_name set stats ('row_count'='1000000', 'ndv'='8155', 'min_value'='', 'max_value'='Woodland Woodland', 'avg_size'='8445649', 'max_size'='8445649' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_street_name set stats ('row_count'='1000000', 'ndv'='8155', 'min_value'='', 'max_value'='Woodland Woodland', 'avg_size'='8445649', 'max_size'='8445649' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_street_type set stats ('row_count'='1000000', 'ndv'='21', 'min_value'='', 'max_value'='Wy', 'avg_size'='4073296', 'max_size'='4073296' )
-    '''
-
-    sql '''
-    alter table income_band modify column ib_income_band_sk set stats ('row_count'='20', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='160', 'max_size'='160' )
-    '''
-
-    sql '''
-    alter table income_band modify column ib_upper_bound set stats ('row_count'='20', 'ndv'='20', 'min_value'='10000', 'max_value'='200000', 'avg_size'='80', 'max_size'='80' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_start_date_sk set stats ('row_count'='20400', 'ndv'='91', 'min_value'='2450815', 'max_value'='2453005', 'avg_size'='81600', 'max_size'='81600' )
-    '''
-
-    sql '''
-    alter table item modify column i_class set stats ('row_count'='204000', 'ndv'='100', 'min_value'='', 'max_value'='womens watch', 'avg_size'='1585937', 'max_size'='1585937' )
-    '''
-
-    sql '''
-    alter table item modify column i_manufact set stats ('row_count'='204000', 'ndv'='1004', 'min_value'='', 'max_value'='pripripri', 'avg_size'='2298787', 'max_size'='2298787' )
-    '''
-
-    sql '''
-    alter table item modify column i_manufact_id set stats ('row_count'='204000', 'ndv'='1005', 'min_value'='1', 'max_value'='1000', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_rec_end_date set stats ('row_count'='204000', 'ndv'='3', 'min_value'='1999-10-27', 'max_value'='2001-10-26', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_units set stats ('row_count'='204000', 'ndv'='22', 'min_value'='', 'max_value'='Unknown', 'avg_size'='852562', 'max_size'='852562' )
-    '''
-
-    sql '''
-    alter table item modify column i_wholesale_cost set stats ('row_count'='204000', 'ndv'='89', 'min_value'='0.02', 'max_value'='88.91', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_account_credit set stats ('row_count'='7197670', 'ndv'='10868', 'min_value'='0.00', 'max_value'='23028.27', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_item_sk set stats ('row_count'='7197670', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_net_loss set stats ('row_count'='7197670', 'ndv'='11012', 'min_value'='0.50', 'max_value'='15068.96', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_net_loss set stats ('row_count'='7197670', 'ndv'='11012', 'min_value'='0.50', 'max_value'='15068.96', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_order_number set stats ('row_count'='7197670', 'ndv'='4249346', 'min_value'='1', 'max_value'='5999999', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_refunded_addr_sk set stats ('row_count'='7197670', 'ndv'='999503', 'min_value'='1', 'max_value'='1000000', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_refunded_cash set stats ('row_count'='7197670', 'ndv'='14621', 'min_value'='0.00', 'max_value'='26466.56', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_refunded_cdemo_sk set stats ('row_count'='7197670', 'ndv'='1868495', 'min_value'='1', 'max_value'='1920800', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_return_tax set stats ('row_count'='7197670', 'ndv'='1820', 'min_value'='0.00', 'max_value'='2551.16', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returning_customer_sk set stats ('row_count'='7197670', 'ndv'='1926139', 'min_value'='1', 'max_value'='2000000', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_county set stats ('row_count'='24', 'ndv'='9', 'min_value'='Barrow County', 'max_value'='Ziebach County', 'avg_size'='331', 'max_size'='331' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_name set stats ('row_count'='24', 'ndv'='4', 'min_value'='site_0', 'max_value'='site_3', 'avg_size'='144', 'max_size'='144' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_street_name set stats ('row_count'='24', 'ndv'='24', 'min_value'='11th ', 'max_value'='Wilson Ridge', 'avg_size'='219', 'max_size'='219' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_street_type set stats ('row_count'='24', 'ndv'='15', 'min_value'='Avenue', 'max_value'='Wy', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_demo set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='984', 'max_size'='984' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_email set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='987', 'max_size'='987' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_tv set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='986', 'max_size'='986' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_cost set stats ('row_count'='1000', 'ndv'='1', 'min_value'='1000.00', 'max_value'='1000.00', 'avg_size'='8000', 'max_size'='8000' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_cost set stats ('row_count'='1000', 'ndv'='1', 'min_value'='1000.00', 'max_value'='1000.00', 'avg_size'='8000', 'max_size'='8000' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_discount_active set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='981', 'max_size'='981' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_promo_id set stats ('row_count'='1000', 'ndv'='1004', 'min_value'='AAAAAAAAAABAAAAA', 'max_value'='AAAAAAAAPPCAAAAA', 'avg_size'='16000', 'max_size'='16000' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_response_targe set stats ('row_count'='1000', 'ndv'='1', 'min_value'='1', 'max_value'='1', 'avg_size'='4000', 'max_size'='4000' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_bill_addr_sk set stats ('row_count'='72001237', 'ndv'='998891', 'min_value'='1', 'max_value'='1000000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_bill_cdemo_sk set stats ('row_count'='72001237', 'ndv'='1835731', 'min_value'='1', 'max_value'='1920800', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_bill_customer_sk set stats ('row_count'='72001237', 'ndv'='1899439', 'min_value'='1', 'max_value'='2000000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_discount_amt set stats ('row_count'='72001237', 'ndv'='27052', 'min_value'='0.00', 'max_value'='29982.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_list_price set stats ('row_count'='72001237', 'ndv'='29104', 'min_value'='1.02', 'max_value'='29997.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_sales_price set stats ('row_count'='72001237', 'ndv'='27115', 'min_value'='0.00', 'max_value'='29810.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_list_price set stats ('row_count'='72001237', 'ndv'='301', 'min_value'='1.00', 'max_value'='300.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_net_paid set stats ('row_count'='72001237', 'ndv'='26912', 'min_value'='0.00', 'max_value'='29810.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_promo_sk set stats ('row_count'='72001237', 'ndv'='986', 'min_value'='1', 'max_value'='1000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_promo_sk set stats ('row_count'='72001237', 'ndv'='986', 'min_value'='1', 'max_value'='1000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_cdemo_sk set stats ('row_count'='72001237', 'ndv'='1822804', 'min_value'='1', 'max_value'='1920800', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_customer_sk set stats ('row_count'='72001237', 'ndv'='1898561', 'min_value'='1', 'max_value'='2000000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_date_sk set stats ('row_count'='72001237', 'ndv'='1952', 'min_value'='2450817', 'max_value'='2452762', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_hdemo_sk set stats ('row_count'='72001237', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_web_site_sk set stats ('row_count'='72001237', 'ndv'='24', 'min_value'='1', 'max_value'='24', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table store modify column s_closed_date_sk set stats ('row_count'='402', 'ndv'='69', 'min_value'='2450823', 'max_value'='2451313', 'avg_size'='3216', 'max_size'='3216' )
-    '''
-
-    sql '''
-    alter table store modify column s_company_name set stats ('row_count'='402', 'ndv'='2', 'min_value'='', 'max_value'='Unknown', 'avg_size'='2793', 'max_size'='2793' )
-    '''
-
-    sql '''
-    alter table store modify column s_market_manager set stats ('row_count'='402', 'ndv'='286', 'min_value'='', 'max_value'='Zane Perez', 'avg_size'='5129', 'max_size'='5129' )
-    '''
-
-    sql '''
-    alter table store modify column s_store_id set stats ('row_count'='402', 'ndv'='201', 'min_value'='AAAAAAAAAABAAAAA', 'max_value'='AAAAAAAAPNAAAAAA', 'avg_size'='6432', 'max_size'='6432' )
-    '''
-
-    sql '''
-    alter table store modify column s_store_sk set stats ('row_count'='402', 'ndv'='398', 'min_value'='1', 'max_value'='402', 'avg_size'='3216', 'max_size'='3216' )
-    '''
-
-    sql '''
-    alter table store modify column s_street_number set stats ('row_count'='402', 'ndv'='267', 'min_value'='', 'max_value'='986', 'avg_size'='1150', 'max_size'='1150' )
-    '''
-
-    sql '''
-    alter table store modify column s_suite_number set stats ('row_count'='402', 'ndv'='75', 'min_value'='', 'max_value'='Suite Y', 'avg_size'='3140', 'max_size'='3140' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_hour set stats ('row_count'='86400', 'ndv'='24', 'min_value'='0', 'max_value'='23', 'avg_size'='345600', 'max_size'='345600' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_shift set stats ('row_count'='86400', 'ndv'='3', 'min_value'='first', 'max_value'='third', 'avg_size'='460800', 'max_size'='460800' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_time set stats ('row_count'='86400', 'ndv'='86684', 'min_value'='0', 'max_value'='86399', 'avg_size'='345600', 'max_size'='345600' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_creation_date_sk set stats ('row_count'='2040', 'ndv'='134', 'min_value'='2450672', 'max_value'='2450815', 'avg_size'='16320', 'max_size'='16320' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_rec_end_date set stats ('row_count'='2040', 'ndv'='3', 'min_value'='1999-09-03', 'max_value'='2001-09-02', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_rec_start_date set stats ('row_count'='2040', 'ndv'='4', 'min_value'='1997-09-03', 'max_value'='2001-09-03', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_web_page_id set stats ('row_count'='2040', 'ndv'='1019', 'min_value'='AAAAAAAAAABAAAAA', 'max_value'='AAAAAAAAPPEAAAAA', 'avg_size'='32640', 'max_size'='32640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_fee set stats ('row_count'='28795080', 'ndv'='101', 'min_value'='0.50', 'max_value'='100.00', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_amt set stats ('row_count'='28795080', 'ndv'='15493', 'min_value'='0.00', 'max_value'='18973.20', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_returned_date_sk set stats ('row_count'='28795080', 'ndv'='2010', 'min_value'='2450820', 'max_value'='2452822', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_store_credit set stats ('row_count'='28795080', 'ndv'='9907', 'min_value'='0.00', 'max_value'='15642.11', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_store_credit set stats ('row_count'='28795080', 'ndv'='9907', 'min_value'='0.00', 'max_value'='15642.11', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_ticket_number set stats ('row_count'='28795080', 'ndv'='16790866', 'min_value'='1', 'max_value'='23999996', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_customer_sk set stats ('row_count'='287997024', 'ndv'='1994393', 'min_value'='1', 'max_value'='2000000', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ext_list_price set stats ('row_count'='287997024', 'ndv'='19770', 'min_value'='1.00', 'max_value'='20000.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ext_sales_price set stats ('row_count'='287997024', 'ndv'='19105', 'min_value'='0.00', 'max_value'='19878.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_ext_tax set stats ('row_count'='287997024', 'ndv'='1722', 'min_value'='0.00', 'max_value'='1762.38', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_quantity set stats ('row_count'='287997024', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_sales_price set stats ('row_count'='287997024', 'ndv'='202', 'min_value'='0.00', 'max_value'='200.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_store_sk set stats ('row_count'='287997024', 'ndv'='200', 'min_value'='1', 'max_value'='400', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_carrier set stats ('row_count'='20', 'ndv'='20', 'min_value'='AIRBORNE', 'max_value'='ZOUROS', 'avg_size'='133', 'max_size'='133' )
-    '''
-
-    sql '''
-    alter table customer modify column c_current_cdemo_sk set stats ('row_count'='2000000', 'ndv'='1221921', 'min_value'='1', 'max_value'='1920798', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_current_hdemo_sk set stats ('row_count'='2000000', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_customer_id set stats ('row_count'='2000000', 'ndv'='1994557', 'min_value'='AAAAAAAAAAAAABAA', 'max_value'='AAAAAAAAPPPPPAAA', 'avg_size'='32000000', 'max_size'='32000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_login set stats ('row_count'='2000000', 'ndv'='1', 'min_value'='', 'max_value'='', 'avg_size'='0', 'max_size'='0' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_credit_rating set stats ('row_count'='1920800', 'ndv'='4', 'min_value'='Good', 'max_value'='Unknown', 'avg_size'='13445600', 'max_size'='13445600' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_demo_sk set stats ('row_count'='1920800', 'ndv'='1916366', 'min_value'='1', 'max_value'='1920800', 'avg_size'='15366400', 'max_size'='15366400' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_dep_college_count set stats ('row_count'='1920800', 'ndv'='7', 'min_value'='0', 'max_value'='6', 'avg_size'='7683200', 'max_size'='7683200' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_dep_employed_count set stats ('row_count'='1920800', 'ndv'='7', 'min_value'='0', 'max_value'='6', 'avg_size'='7683200', 'max_size'='7683200' )
-    '''
-
-    sql '''
-    alter table customer_demographics modify column cd_marital_status set stats ('row_count'='1920800', 'ndv'='5', 'min_value'='D', 'max_value'='W', 'avg_size'='1920800', 'max_size'='1920800' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_current_month set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_current_year set stats ('row_count'='73049', 'ndv'='2', 'min_value'='N', 'max_value'='Y', 'avg_size'='73049', 'max_size'='73049' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_date set stats ('row_count'='73049', 'ndv'='73250', 'min_value'='1900-01-02', 'max_value'='2100-01-01', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table date_dim modify column d_first_dom set stats ('row_count'='73049', 'ndv'='2410', 'min_value'='2415021', 'max_value'='2488070', 'avg_size'='292196', 'max_size'='292196' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_state set stats ('row_count'='15', 'ndv'='8', 'min_value'='AL', 'max_value'='SD', 'avg_size'='30', 'max_size'='30' )
-    '''
-
-    sql '''
-    alter table warehouse modify column w_zip set stats ('row_count'='15', 'ndv'='15', 'min_value'='28721', 'max_value'='78721', 'avg_size'='75', 'max_size'='75' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_bill_cdemo_sk set stats ('row_count'='143997065', 'ndv'='1915709', 'min_value'='1', 'max_value'='1920800', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_bill_hdemo_sk set stats ('row_count'='143997065', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_discount_amt set stats ('row_count'='143997065', 'ndv'='27722', 'min_value'='0.00', 'max_value'='29765.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ext_list_price set stats ('row_count'='143997065', 'ndv'='29336', 'min_value'='1.00', 'max_value'='29997.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_net_paid_inc_ship set stats ('row_count'='143997065', 'ndv'='37890', 'min_value'='0.00', 'max_value'='43725.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_net_paid_inc_ship_tax set stats ('row_count'='143997065', 'ndv'='38890', 'min_value'='0.00', 'max_value'='45460.80', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_net_paid_inc_tax set stats ('row_count'='143997065', 'ndv'='28777', 'min_value'='0.00', 'max_value'='31745.52', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_sales_price set stats ('row_count'='143997065', 'ndv'='302', 'min_value'='0.00', 'max_value'='300.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_date_sk set stats ('row_count'='143997065', 'ndv'='1933', 'min_value'='2450817', 'max_value'='2452744', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_date_sk set stats ('row_count'='143997065', 'ndv'='1933', 'min_value'='2450817', 'max_value'='2452744', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_ship_hdemo_sk set stats ('row_count'='143997065', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='1151976520', 'max_size'='1151976520' )
-    '''
-
-    sql '''
-    alter table catalog_sales modify column cs_wholesale_cost set stats ('row_count'='143997065', 'ndv'='100', 'min_value'='1.00', 'max_value'='100.00', 'avg_size'='575988260', 'max_size'='575988260' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_class set stats ('row_count'='30', 'ndv'='3', 'min_value'='large', 'max_value'='small', 'avg_size'='166', 'max_size'='166' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_company_name set stats ('row_count'='30', 'ndv'='6', 'min_value'='able', 'max_value'='pri', 'avg_size'='110', 'max_size'='110' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_division set stats ('row_count'='30', 'ndv'='6', 'min_value'='1', 'max_value'='6', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_mkt_class set stats ('row_count'='30', 'ndv'='25', 'min_value'='A bit narrow forms matter animals. Consist', 'max_value'='Yesterday new men can make moreov', 'avg_size'='1033', 'max_size'='1033' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_suite_number set stats ('row_count'='30', 'ndv'='14', 'min_value'='Suite 0', 'max_value'='Suite W', 'avg_size'='234', 'max_size'='234' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_tax_percentage set stats ('row_count'='30', 'ndv'='1', 'min_value'='0.00', 'max_value'='0.12', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_tax_percentage set stats ('row_count'='30', 'ndv'='1', 'min_value'='0.00', 'max_value'='0.12', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_zip set stats ('row_count'='30', 'ndv'='14', 'min_value'='20059', 'max_value'='75281', 'avg_size'='150', 'max_size'='150' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_catalog_page_sk set stats ('row_count'='14404374', 'ndv'='11515', 'min_value'='1', 'max_value'='17108', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_net_loss set stats ('row_count'='14404374', 'ndv'='11753', 'min_value'='0.50', 'max_value'='15781.83', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_order_number set stats ('row_count'='14404374', 'ndv'='9425725', 'min_value'='2', 'max_value'='16000000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_refunded_cash set stats ('row_count'='14404374', 'ndv'='16271', 'min_value'='0.00', 'max_value'='24544.84', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_refunded_cdemo_sk set stats ('row_count'='14404374', 'ndv'='1900770', 'min_value'='1', 'max_value'='1920800', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_return_quantity set stats ('row_count'='14404374', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='57617496', 'max_size'='57617496' )
-    '''
-
-    sql '''
-    alter table catalog_returns modify column cr_returning_addr_sk set stats ('row_count'='14404374', 'ndv'='1000237', 'min_value'='1', 'max_value'='1000000', 'avg_size'='115234992', 'max_size'='115234992' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_city set stats ('row_count'='1000000', 'ndv'='977', 'min_value'='', 'max_value'='Zion', 'avg_size'='8681993', 'max_size'='8681993' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_street_type set stats ('row_count'='1000000', 'ndv'='21', 'min_value'='', 'max_value'='Wy', 'avg_size'='4073296', 'max_size'='4073296' )
-    '''
-
-    sql '''
-    alter table customer_address modify column ca_suite_number set stats ('row_count'='1000000', 'ndv'='76', 'min_value'='', 'max_value'='Suite Y', 'avg_size'='7652799', 'max_size'='7652799' )
-    '''
-
-    sql '''
-    alter table income_band modify column ib_lower_bound set stats ('row_count'='20', 'ndv'='20', 'min_value'='0', 'max_value'='190001', 'avg_size'='80', 'max_size'='80' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_catalog_number set stats ('row_count'='20400', 'ndv'='109', 'min_value'='1', 'max_value'='109', 'avg_size'='81600', 'max_size'='81600' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_catalog_page_id set stats ('row_count'='20400', 'ndv'='20341', 'min_value'='AAAAAAAAAAABAAAA', 'max_value'='AAAAAAAAPPPDAAAA', 'avg_size'='326400', 'max_size'='326400' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_catalog_page_sk set stats ('row_count'='20400', 'ndv'='20554', 'min_value'='1', 'max_value'='20400', 'avg_size'='163200', 'max_size'='163200' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_department set stats ('row_count'='20400', 'ndv'='2', 'min_value'='', 'max_value'='DEPARTMENT', 'avg_size'='201950', 'max_size'='201950' )
-    '''
-
-    sql '''
-    alter table catalog_page modify column cp_end_date_sk set stats ('row_count'='20400', 'ndv'='97', 'min_value'='2450844', 'max_value'='2453186', 'avg_size'='81600', 'max_size'='81600' )
-    '''
-
-    sql '''
-    alter table item modify column i_container set stats ('row_count'='204000', 'ndv'='2', 'min_value'='', 'max_value'='Unknown', 'avg_size'='1424430', 'max_size'='1424430' )
-    '''
-
-    sql '''
-    alter table item modify column i_formulation set stats ('row_count'='204000', 'ndv'='152702', 'min_value'='', 'max_value'='yellow98911509228741', 'avg_size'='4069400', 'max_size'='4069400' )
-    '''
-
-    sql '''
-    alter table item modify column i_item_desc set stats ('row_count'='204000', 'ndv'='148398', 'min_value'='', 'max_value'='Youngsters used to save quite colour', 'avg_size'='20471814', 'max_size'='20471814' )
-    '''
-
-    sql '''
-    alter table item modify column i_item_sk set stats ('row_count'='204000', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='1632000', 'max_size'='1632000' )
-    '''
-
-    sql '''
-    alter table item modify column i_item_sk set stats ('row_count'='204000', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='1632000', 'max_size'='1632000' )
-    '''
-
-    sql '''
-    alter table item modify column i_manager_id set stats ('row_count'='204000', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_rec_end_date set stats ('row_count'='204000', 'ndv'='3', 'min_value'='1999-10-27', 'max_value'='2001-10-26', 'avg_size'='816000', 'max_size'='816000' )
-    '''
-
-    sql '''
-    alter table item modify column i_size set stats ('row_count'='204000', 'ndv'='8', 'min_value'='', 'max_value'='small', 'avg_size'='880961', 'max_size'='880961' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_reason_sk set stats ('row_count'='7197670', 'ndv'='55', 'min_value'='1', 'max_value'='55', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_return_amt set stats ('row_count'='7197670', 'ndv'='19263', 'min_value'='0.00', 'max_value'='28346.31', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_return_ship_cost set stats ('row_count'='7197670', 'ndv'='10429', 'min_value'='0.00', 'max_value'='13602.60', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_return_tax set stats ('row_count'='7197670', 'ndv'='1820', 'min_value'='0.00', 'max_value'='2551.16', 'avg_size'='28790680', 'max_size'='28790680' )
-    '''
-
-    sql '''
-    alter table web_returns modify column wr_returning_cdemo_sk set stats ('row_count'='7197670', 'ndv'='1865149', 'min_value'='1', 'max_value'='1920800', 'avg_size'='57581360', 'max_size'='57581360' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_company_name set stats ('row_count'='24', 'ndv'='6', 'min_value'='able', 'max_value'='pri', 'avg_size'='97', 'max_size'='97' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_gmt_offset set stats ('row_count'='24', 'ndv'='2', 'min_value'='-6.00', 'max_value'='-5.00', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_name set stats ('row_count'='24', 'ndv'='4', 'min_value'='site_0', 'max_value'='site_3', 'avg_size'='144', 'max_size'='144' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_open_date_sk set stats ('row_count'='24', 'ndv'='12', 'min_value'='2450628', 'max_value'='2450807', 'avg_size'='192', 'max_size'='192' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_open_date_sk set stats ('row_count'='24', 'ndv'='12', 'min_value'='2450628', 'max_value'='2450807', 'avg_size'='192', 'max_size'='192' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_rec_start_date set stats ('row_count'='24', 'ndv'='4', 'min_value'='1997-08-16', 'max_value'='2001-08-16', 'avg_size'='96', 'max_size'='96' )
-    '''
-
-    sql '''
-    alter table web_site modify column web_zip set stats ('row_count'='24', 'ndv'='14', 'min_value'='28828', 'max_value'='78828', 'avg_size'='120', 'max_size'='120' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_details set stats ('row_count'='1000', 'ndv'='992', 'min_value'='', 'max_value'='Young, valuable companies watch walls. Payments can flour', 'avg_size'='39304', 'max_size'='39304' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_channel_press set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='N', 'avg_size'='985', 'max_size'='985' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_end_date_sk set stats ('row_count'='1000', 'ndv'='571', 'min_value'='2450116', 'max_value'='2450967', 'avg_size'='8000', 'max_size'='8000' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_promo_id set stats ('row_count'='1000', 'ndv'='1004', 'min_value'='AAAAAAAAAABAAAAA', 'max_value'='AAAAAAAAPPCAAAAA', 'avg_size'='16000', 'max_size'='16000' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_promo_name set stats ('row_count'='1000', 'ndv'='11', 'min_value'='', 'max_value'='pri', 'avg_size'='3924', 'max_size'='3924' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_purpose set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='Unknown', 'avg_size'='6909', 'max_size'='6909' )
-    '''
-
-    sql '''
-    alter table promotion modify column p_purpose set stats ('row_count'='1000', 'ndv'='2', 'min_value'='', 'max_value'='Unknown', 'avg_size'='6909', 'max_size'='6909' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_bill_hdemo_sk set stats ('row_count'='72001237', 'ndv'='7251', 'min_value'='1', 'max_value'='7200', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_coupon_amt set stats ('row_count'='72001237', 'ndv'='20659', 'min_value'='0.00', 'max_value'='27591.16', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_sales_price set stats ('row_count'='72001237', 'ndv'='27115', 'min_value'='0.00', 'max_value'='29810.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_ship_cost set stats ('row_count'='72001237', 'ndv'='13977', 'min_value'='0.00', 'max_value'='14927.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_tax set stats ('row_count'='72001237', 'ndv'='2466', 'min_value'='0.00', 'max_value'='2682.90', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_wholesale_cost set stats ('row_count'='72001237', 'ndv'='10009', 'min_value'='1.00', 'max_value'='10000.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ext_wholesale_cost set stats ('row_count'='72001237', 'ndv'='10009', 'min_value'='1.00', 'max_value'='10000.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_item_sk set stats ('row_count'='72001237', 'ndv'='205012', 'min_value'='1', 'max_value'='204000', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_net_paid set stats ('row_count'='72001237', 'ndv'='26912', 'min_value'='0.00', 'max_value'='29810.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_net_paid_inc_tax set stats ('row_count'='72001237', 'ndv'='28263', 'min_value'='0.00', 'max_value'='32492.90', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_net_paid_inc_tax set stats ('row_count'='72001237', 'ndv'='28263', 'min_value'='0.00', 'max_value'='32492.90', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_net_profit set stats ('row_count'='72001237', 'ndv'='27958', 'min_value'='-9997.00', 'max_value'='19840.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_quantity set stats ('row_count'='72001237', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_ship_mode_sk set stats ('row_count'='72001237', 'ndv'='20', 'min_value'='1', 'max_value'='20', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_sold_date_sk set stats ('row_count'='72001237', 'ndv'='1820', 'min_value'='2450816', 'max_value'='2452642', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_web_page_sk set stats ('row_count'='72001237', 'ndv'='2032', 'min_value'='1', 'max_value'='2040', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_web_site_sk set stats ('row_count'='72001237', 'ndv'='24', 'min_value'='1', 'max_value'='24', 'avg_size'='576009896', 'max_size'='576009896' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_wholesale_cost set stats ('row_count'='72001237', 'ndv'='100', 'min_value'='1.00', 'max_value'='100.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table web_sales modify column ws_wholesale_cost set stats ('row_count'='72001237', 'ndv'='100', 'min_value'='1.00', 'max_value'='100.00', 'avg_size'='288004948', 'max_size'='288004948' )
-    '''
-
-    sql '''
-    alter table store modify column s_market_id set stats ('row_count'='402', 'ndv'='10', 'min_value'='1', 'max_value'='10', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_rec_start_date set stats ('row_count'='402', 'ndv'='4', 'min_value'='1997-03-13', 'max_value'='2001-03-13', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_rec_start_date set stats ('row_count'='402', 'ndv'='4', 'min_value'='1997-03-13', 'max_value'='2001-03-13', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_store_name set stats ('row_count'='402', 'ndv'='11', 'min_value'='', 'max_value'='pri', 'avg_size'='1575', 'max_size'='1575' )
-    '''
-
-    sql '''
-    alter table store modify column s_street_name set stats ('row_count'='402', 'ndv'='256', 'min_value'='', 'max_value'='Woodland ', 'avg_size'='3384', 'max_size'='3384' )
-    '''
-
-    sql '''
-    alter table store modify column s_tax_precentage set stats ('row_count'='402', 'ndv'='1', 'min_value'='0.00', 'max_value'='0.11', 'avg_size'='1608', 'max_size'='1608' )
-    '''
-
-    sql '''
-    alter table store modify column s_zip set stats ('row_count'='402', 'ndv'='102', 'min_value'='', 'max_value'='79431', 'avg_size'='1980', 'max_size'='1980' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_sub_shift set stats ('row_count'='86400', 'ndv'='4', 'min_value'='afternoon', 'max_value'='night', 'avg_size'='597600', 'max_size'='597600' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_time_sk set stats ('row_count'='86400', 'ndv'='87677', 'min_value'='0', 'max_value'='86399', 'avg_size'='691200', 'max_size'='691200' )
-    '''
-
-    sql '''
-    alter table time_dim modify column t_time_sk set stats ('row_count'='86400', 'ndv'='87677', 'min_value'='0', 'max_value'='86399', 'avg_size'='691200', 'max_size'='691200' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_access_date_sk set stats ('row_count'='2040', 'ndv'='101', 'min_value'='2452548', 'max_value'='2452648', 'avg_size'='16320', 'max_size'='16320' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_access_date_sk set stats ('row_count'='2040', 'ndv'='101', 'min_value'='2452548', 'max_value'='2452648', 'avg_size'='16320', 'max_size'='16320' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_autogen_flag set stats ('row_count'='2040', 'ndv'='3', 'min_value'='', 'max_value'='Y', 'avg_size'='2015', 'max_size'='2015' )
-    '''
-
-    sql '''
-    alter table web_page modify column wp_char_count set stats ('row_count'='2040', 'ndv'='1363', 'min_value'='303', 'max_value'='8523', 'avg_size'='8160', 'max_size'='8160' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_cdemo_sk set stats ('row_count'='28795080', 'ndv'='1916366', 'min_value'='1', 'max_value'='1920800', 'avg_size'='230360640', 'max_size'='230360640' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_fee set stats ('row_count'='28795080', 'ndv'='101', 'min_value'='0.50', 'max_value'='100.00', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_refunded_cash set stats ('row_count'='28795080', 'ndv'='12626', 'min_value'='0.00', 'max_value'='17556.95', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_quantity set stats ('row_count'='28795080', 'ndv'='100', 'min_value'='1', 'max_value'='100', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_return_ship_cost set stats ('row_count'='28795080', 'ndv'='8186', 'min_value'='0.00', 'max_value'='9578.25', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_returns modify column sr_reversed_charge set stats ('row_count'='28795080', 'ndv'='9872', 'min_value'='0.00', 'max_value'='16099.52', 'avg_size'='115180320', 'max_size'='115180320' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_cdemo_sk set stats ('row_count'='287997024', 'ndv'='1916366', 'min_value'='1', 'max_value'='1920800', 'avg_size'='2303976192', 'max_size'='2303976192' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_coupon_amt set stats ('row_count'='287997024', 'ndv'='16198', 'min_value'='0.00', 'max_value'='19225.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_net_paid set stats ('row_count'='287997024', 'ndv'='19028', 'min_value'='0.00', 'max_value'='19878.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_net_profit set stats ('row_count'='287997024', 'ndv'='19581', 'min_value'='-10000.00', 'max_value'='9889.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table store_sales modify column ss_sales_price set stats ('row_count'='287997024', 'ndv'='202', 'min_value'='0.00', 'max_value'='200.00', 'avg_size'='1151988096', 'max_size'='1151988096' )
-    '''
-
-    sql '''
-    alter table ship_mode modify column sm_type set stats ('row_count'='20', 'ndv'='6', 'min_value'='EXPRESS', 'max_value'='TWO DAY', 'avg_size'='150', 'max_size'='150' )
-    '''
-
-    sql '''
-    alter table customer modify column c_birth_day set stats ('row_count'='2000000', 'ndv'='31', 'min_value'='1', 'max_value'='31', 'avg_size'='8000000', 'max_size'='8000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_customer_id set stats ('row_count'='2000000', 'ndv'='1994557', 'min_value'='AAAAAAAAAAAAABAA', 'max_value'='AAAAAAAAPPPPPAAA', 'avg_size'='32000000', 'max_size'='32000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_first_shipto_date_sk set stats ('row_count'='2000000', 'ndv'='3644', 'min_value'='2449028', 'max_value'='2452678', 'avg_size'='16000000', 'max_size'='16000000' )
-    '''
-
-    sql '''
-    alter table customer modify column c_last_name set stats ('row_count'='2000000', 'ndv'='4990', 'min_value'='', 'max_value'='Zuniga', 'avg_size'='11833714', 'max_size'='11833714' )
-    '''
-
-    sql '''
-    alter table call_center modify column cc_mkt_desc set stats ('row_count'='30', 'ndv'='22', 'min_value'='As existing eyebrows miss as the matters. Realistic stories may not face almost by a ', 'max_value'='Young tests could buy comfortable, local users', 'avg_size'='1766', 'max_size'='1766' )
-    '''
-
-}
\ No newline at end of file
diff --git a/regression-test/suites/cte_reuse/q11.groovy b/regression-test/suites/cte_reuse/q11.groovy
deleted file mode 100644
index 00ce7e71b1..0000000000
--- a/regression-test/suites/cte_reuse/q11.groovy
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-suite("cte_reuse_q11") {
-    
-    sql "use regression_test_cte_reuse"
-    sql 'set enable_nereids_planner=true'
-    sql 'set enable_fallback_to_original_planner=false'
-    sql """
-        SET enable_pipeline_engine = true
-    """
-    sql "set forbid_unknown_col_stats=true"
-
-
-    def result = sql "show backends;"
-    if (result.size() != 1) {
-        print("backends num: ${result.size()}");
-        return;
-    }
-
-    qt_sql """
-        EXPLAIN SHAPE PLAN
-WITH
-  year_total AS (
-   SELECT
-     c_customer_id customer_id
-   , c_first_name customer_first_name
-   , c_last_name customer_last_name
-   , c_preferred_cust_flag customer_preferred_cust_flag
-   , c_birth_country customer_birth_country
-   , c_login customer_login
-   , c_email_address customer_email_address
-   , d_year dyear
-   , sum((ss_ext_list_price - ss_ext_discount_amt)) year_total
-   , 's' sale_type
-   FROM
-     customer
-   , store_sales
-   , date_dim
-   WHERE (c_customer_sk = ss_customer_sk)
-      AND (ss_sold_date_sk = d_date_sk)
-   GROUP BY c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-UNION ALL    SELECT
-     c_customer_id customer_id
-   , c_first_name customer_first_name
-   , c_last_name customer_last_name
-   , c_preferred_cust_flag customer_preferred_cust_flag
-   , c_birth_country customer_birth_country
-   , c_login customer_login
-   , c_email_address customer_email_address
-   , d_year dyear
-   , sum((ws_ext_list_price - ws_ext_discount_amt)) year_total
-   , 'w' sale_type
-   FROM
-     customer
-   , web_sales
-   , date_dim
-   WHERE (c_customer_sk = ws_bill_customer_sk)
-      AND (ws_sold_date_sk = d_date_sk)
-   GROUP BY c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-)
-SELECT
-  t_s_secyear.customer_id
-, t_s_secyear.customer_first_name
-, t_s_secyear.customer_last_name
-, t_s_secyear.customer_preferred_cust_flag
-, t_s_secyear.customer_birth_country
-, t_s_secyear.customer_login
-FROM
-  year_total t_s_firstyear
-, year_total t_s_secyear
-, year_total t_w_firstyear
-, year_total t_w_secyear
-WHERE (t_s_secyear.customer_id = t_s_firstyear.customer_id)
-   AND (t_s_firstyear.customer_id = t_w_secyear.customer_id)
-   AND (t_s_firstyear.customer_id = t_w_firstyear.customer_id)
-   AND (t_s_firstyear.sale_type = 's')
-   AND (t_w_firstyear.sale_type = 'w')
-   AND (t_s_secyear.sale_type = 's')
-   AND (t_w_secyear.sale_type = 'w')
-   AND (t_s_firstyear.dyear = 2001)
-   AND (t_s_secyear.dyear = (2001 + 1))
-   AND (t_w_firstyear.dyear = 2001)
-   AND (t_w_secyear.dyear = (2001 + 1))
-   AND (t_s_firstyear.year_total > 0)
-   AND (t_w_firstyear.year_total > 0)
-   AND ((CASE WHEN (t_w_firstyear.year_total > 0) THEN (t_w_secyear.year_total / t_w_firstyear.year_total) ELSE CAST('0.0' AS DECIMAL(2,1)) END) > (CASE WHEN (t_s_firstyear.year_total > 0) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE CAST('0.0' AS DECIMAL(2,1)) END))
-ORDER BY t_s_secyear.customer_id ASC, t_s_secyear.customer_first_name ASC, t_s_secyear.customer_last_name ASC, t_s_secyear.customer_preferred_cust_flag ASC
-LIMIT 100
-
-    """
-}
diff --git a/regression-test/suites/cte_reuse/q14.groovy b/regression-test/suites/cte_reuse/q14.groovy
deleted file mode 100644
index 4b37ecc002..0000000000
--- a/regression-test/suites/cte_reuse/q14.groovy
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-suite("cte_reuse_q14") {
-    
-    sql "use regression_test_cte_reuse"
-    sql 'set enable_nereids_planner=true'
-    sql 'set enable_fallback_to_original_planner=false'
-    sql """
-        SET enable_pipeline_engine = true
-    """
-    sql "set forbid_unknown_col_stats=true"
-
-
-    def result = sql "show backends;"
-    if (result.size() != 1) {
-        print("backends num: ${result.size()}");
-        return;
-    }
-
-    qt_sql """
-        EXPLAIN SHAPE PLAN
-WITH
-  cross_items AS (
-   SELECT i_item_sk ss_item_sk
-   FROM
-     item
-   , (
-      SELECT
-        iss.i_brand_id brand_id
-      , iss.i_class_id class_id
-      , iss.i_category_id category_id
-      FROM
-        store_sales
-      , item iss
-      , date_dim d1
-      WHERE (ss_item_sk = iss.i_item_sk)
-         AND (ss_sold_date_sk = d1.d_date_sk)
-         AND (d1.d_year BETWEEN 1999 AND (1999 + 2))
-INTERSECT       SELECT
-        ics.i_brand_id
-      , ics.i_class_id
-      , ics.i_category_id
-      FROM
-        catalog_sales
-      , item ics
-      , date_dim d2
-      WHERE (cs_item_sk = ics.i_item_sk)
-         AND (cs_sold_date_sk = d2.d_date_sk)
-         AND (d2.d_year BETWEEN 1999 AND (1999 + 2))
-INTERSECT       SELECT
-        iws.i_brand_id
-      , iws.i_class_id
-      , iws.i_category_id
-      FROM
-        web_sales
-      , item iws
-      , date_dim d3
-      WHERE (ws_item_sk = iws.i_item_sk)
-         AND (ws_sold_date_sk = d3.d_date_sk)
-         AND (d3.d_year BETWEEN 1999 AND (1999 + 2))
-   ) y
-   WHERE (i_brand_id = brand_id)
-      AND (i_class_id = class_id)
-      AND (i_category_id = category_id)
-)
-, avg_sales AS (
-   SELECT avg((quantity * list_price)) average_sales
-   FROM
-     (
-      SELECT
-        ss_quantity quantity
-      , ss_list_price list_price
-      FROM
-        store_sales
-      , date_dim
-      WHERE (ss_sold_date_sk = d_date_sk)
-         AND (d_year BETWEEN 1999 AND (1999 + 2))
-UNION ALL       SELECT
-        cs_quantity quantity
-      , cs_list_price list_price
-      FROM
-        catalog_sales
-      , date_dim
-      WHERE (cs_sold_date_sk = d_date_sk)
-         AND (d_year BETWEEN 1999 AND (1999 + 2))
-UNION ALL       SELECT
-        ws_quantity quantity
-      , ws_list_price list_price
-      FROM
-        web_sales
-      , date_dim
-      WHERE (ws_sold_date_sk = d_date_sk)
-         AND (d_year BETWEEN 1999 AND (1999 + 2))
-   )  x
-)
-SELECT
-  channel
-, i_brand_id
-, i_class_id
-, i_category_id
-, sum(sales)
-, sum(number_sales)
-FROM
-  (
-   SELECT
-     'store' channel
-   , i_brand_id
-   , i_class_id
-   , i_category_id
-   , sum((ss_quantity * ss_list_price)) sales
-   , count(*) number_sales
-   FROM
-     store_sales
-   , item
-   , date_dim
-   WHERE (ss_item_sk IN (
-      SELECT ss_item_sk
-      FROM
-        cross_items
-   ))
-      AND (ss_item_sk = i_item_sk)
-      AND (ss_sold_date_sk = d_date_sk)
-      AND (d_year = (1999 + 2))
-      AND (d_moy = 11)
-   GROUP BY i_brand_id, i_class_id, i_category_id
-   HAVING (sum((ss_quantity * ss_list_price)) > (
-         SELECT average_sales
-         FROM
-           avg_sales
-      ))
-UNION ALL    SELECT
-     'catalog' channel
-   , i_brand_id
-   , i_class_id
-   , i_category_id
-   , sum((cs_quantity * cs_list_price)) sales
-   , count(*) number_sales
-   FROM
-     catalog_sales
-   , item
-   , date_dim
-   WHERE (cs_item_sk IN (
-      SELECT ss_item_sk
-      FROM
-        cross_items
-   ))
-      AND (cs_item_sk = i_item_sk)
-      AND (cs_sold_date_sk = d_date_sk)
-      AND (d_year = (1999 + 2))
-      AND (d_moy = 11)
-   GROUP BY i_brand_id, i_class_id, i_category_id
-   HAVING (sum((cs_quantity * cs_list_price)) > (
-         SELECT average_sales
-         FROM
-           avg_sales
-      ))
-UNION ALL    SELECT
-     'web' channel
-   , i_brand_id
-   , i_class_id
-   , i_category_id
-   , sum((ws_quantity * ws_list_price)) sales
-   , count(*) number_sales
-   FROM
-     web_sales
-   , item
-   , date_dim
-   WHERE (ws_item_sk IN (
-      SELECT ss_item_sk
-      FROM
-        cross_items
-   ))
-      AND (ws_item_sk = i_item_sk)
-      AND (ws_sold_date_sk = d_date_sk)
-      AND (d_year = (1999 + 2))
-      AND (d_moy = 11)
-   GROUP BY i_brand_id, i_class_id, i_category_id
-   HAVING (sum((ws_quantity * ws_list_price)) > (
-         SELECT average_sales
-         FROM
-           avg_sales
-      ))
-)  y
-GROUP BY ROLLUP (channel, i_brand_id, i_class_id, i_category_id)
-ORDER BY channel ASC, i_brand_id ASC, i_class_id ASC, i_category_id ASC
-LIMIT 100
-
-
-    """
-}
diff --git a/regression-test/suites/cte_reuse/q23.groovy b/regression-test/suites/cte_reuse/q23.groovy
deleted file mode 100644
index 0a40405190..0000000000
--- a/regression-test/suites/cte_reuse/q23.groovy
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-suite("cte_reuse_q23") {
-    
-    sql "use regression_test_cte_reuse"
-    sql 'set enable_nereids_planner=true'
-    sql 'set enable_fallback_to_original_planner=false'
-    sql """
-        SET enable_pipeline_engine = true
-    """
-    sql "set forbid_unknown_col_stats=true"
-
-    def result = sql "show backends;"
-    if (result.size() != 1) {
-        print("backends num: ${result.size()}");
-        return;
-    }
-
-    qt_sql """
-        EXPLAIN SHAPE PLAN
-
-WITH
-  frequent_ss_items AS (
-   SELECT
-     substr(i_item_desc, 1, 30) itemdesc
-   , i_item_sk item_sk
-   , d_date solddate
-   , count(*) cnt
-   FROM
-     store_sales
-   , date_dim
-   , item
-   WHERE (ss_sold_date_sk = d_date_sk)
-      AND (ss_item_sk = i_item_sk)
-      AND (d_year IN (2000   , (2000 + 1)   , (2000 + 2)   , (2000 + 3)))
-   GROUP BY substr(i_item_desc, 1, 30), i_item_sk, d_date
-   HAVING (count(*) > 4)
-)
-, max_store_sales AS (
-   SELECT max(csales) tpcds_cmax
-   FROM
-     (
-      SELECT
-        c_customer_sk
-      , sum((ss_quantity * ss_sales_price)) csales
-      FROM
-        store_sales
-      , customer
-      , date_dim
-      WHERE (ss_customer_sk = c_customer_sk)
-         AND (ss_sold_date_sk = d_date_sk)
-         AND (d_year IN (2000      , (2000 + 1)      , (2000 + 2)      , (2000 + 3)))
-      GROUP BY c_customer_sk
-   ) x
-)
-, best_ss_customer AS (
-   SELECT
-     c_customer_sk
-   , sum((ss_quantity * ss_sales_price)) ssales
-   FROM
-     store_sales
-   , customer
-   WHERE (ss_customer_sk = c_customer_sk)
-   GROUP BY c_customer_sk
-   HAVING (sum((ss_quantity * ss_sales_price)) > ((50 / CAST('100.0' AS DECIMAL(5,2))) * (
-            SELECT *
-            FROM
-              max_store_sales
-         )))
-)
-SELECT sum(sales)
-FROM
-  (
-   SELECT (cs_quantity * cs_list_price) sales
-   FROM
-     catalog_sales
-   , date_dim
-   WHERE (d_year = 2000)
-      AND (d_moy = 2)
-      AND (cs_sold_date_sk = d_date_sk)
-      AND (cs_item_sk IN (
-      SELECT item_sk
-      FROM
-        frequent_ss_items
-   ))
-      AND (cs_bill_customer_sk IN (
-      SELECT c_customer_sk
-      FROM
-        best_ss_customer
-   ))
-UNION ALL    SELECT (ws_quantity * ws_list_price) sales
-   FROM
-     web_sales
-   , date_dim
-   WHERE (d_year = 2000)
-      AND (d_moy = 2)
-      AND (ws_sold_date_sk = d_date_sk)
-      AND (ws_item_sk IN (
-      SELECT item_sk
-      FROM
-        frequent_ss_items
-   ))
-      AND (ws_bill_customer_sk IN (
-      SELECT c_customer_sk
-      FROM
-        best_ss_customer
-   ))
-) y
-LIMIT 100
-
-    """
-}
diff --git a/regression-test/suites/cte_reuse/q24.groovy b/regression-test/suites/cte_reuse/q24.groovy
deleted file mode 100644
index d22868de16..0000000000
--- a/regression-test/suites/cte_reuse/q24.groovy
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-suite("cte_reuse_q24") {
-    
-    sql "use regression_test_cte_reuse"
-    sql 'set enable_nereids_planner=true'
-    sql 'set enable_fallback_to_original_planner=false'
-    sql """
-        SET enable_pipeline_engine = true
-    """
-    sql "set forbid_unknown_col_stats=true"
-
-    def result = sql "show backends;"
-    if (result.size() != 1) {
-        print("backends num: ${result.size()}");
-        return;
-    }
-    
-
-    qt_sql """
-        EXPLAIN SHAPE PLAN
-WITH
-  ssales AS (
-   SELECT
-     c_last_name
-   , c_first_name
-   , s_store_name
-   , ca_state
-   , s_state
-   , i_color
-   , i_current_price
-   , i_manager_id
-   , i_units
-   , i_size
-   , sum(ss_net_paid) netpaid
-   FROM
-     store_sales
-   , store_returns
-   , store
-   , item
-   , customer
-   , customer_address
-   WHERE (ss_ticket_number = sr_ticket_number)
-      AND (ss_item_sk = sr_item_sk)
-      AND (ss_customer_sk = c_customer_sk)
-      AND (ss_item_sk = i_item_sk)
-      AND (ss_store_sk = s_store_sk)
-      AND (c_birth_country = upper(ca_country))
-      AND (s_zip = ca_zip)
-      AND (s_market_id = 8)
-   GROUP BY c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size
-)
-SELECT
-  c_last_name
-, c_first_name
-, s_store_name
-, sum(netpaid) paid
-FROM
-  ssales
-WHERE (i_color = 'pale')
-GROUP BY c_last_name, c_first_name, s_store_name
-HAVING (sum(netpaid) > (
-      SELECT (CAST('0.05' AS DECIMAL(5,2)) * avg(netpaid))
-      FROM
-        ssales
-   ))
-ORDER BY c_last_name, c_first_name, s_store_name
-
-
-    """
-}
diff --git a/regression-test/suites/cte_reuse/q31.groovy b/regression-test/suites/cte_reuse/q31.groovy
deleted file mode 100644
index 61d53e83d1..0000000000
--- a/regression-test/suites/cte_reuse/q31.groovy
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-suite("cte_reuse_q31") {
-    
-    sql "use regression_test_cte_reuse"
-    sql 'set enable_nereids_planner=true'
-    sql 'set enable_fallback_to_original_planner=false'
-    sql """
-        SET enable_pipeline_engine = true
-    """
-    sql "set forbid_unknown_col_stats=true"
-
-    def result = sql "show backends;"
-    if (result.size() != 1) {
-        print("backends num: ${result.size()}");
-        return;
-    }
-
-    qt_sql """
-        EXPLAIN SHAPE PLAN
-WITH
-  ss AS (
-   SELECT
-     ca_county
-   , d_qoy
-   , d_year
-   , sum(ss_ext_sales_price) store_sales
-   FROM
-     store_sales
-   , date_dim
-   , customer_address
-   WHERE (ss_sold_date_sk = d_date_sk)
-      AND (ss_addr_sk = ca_address_sk)
-   GROUP BY ca_county, d_qoy, d_year
-)
-, ws AS (
-   SELECT
-     ca_county
-   , d_qoy
-   , d_year
-   , sum(ws_ext_sales_price) web_sales
-   FROM
-     web_sales
-   , date_dim
-   , customer_address
-   WHERE (ws_sold_date_sk = d_date_sk)
-      AND (ws_bill_addr_sk = ca_address_sk)
-   GROUP BY ca_county, d_qoy, d_year
-)
-SELECT
-  ss1.ca_county
-, ss1.d_year
-, (ws2.web_sales / ws1.web_sales) web_q1_q2_increase
-, (ss2.store_sales / ss1.store_sales) store_q1_q2_increase
-, (ws3.web_sales / ws2.web_sales) web_q2_q3_increase
-, (ss3.store_sales / ss2.store_sales) store_q2_q3_increase
-FROM
-  ss ss1
-, ss ss2
-, ss ss3
-, ws ws1
-, ws ws2
-, ws ws3
-WHERE (ss1.d_qoy = 1)
-   AND (ss1.d_year = 2000)
-   AND (ss1.ca_county = ss2.ca_county)
-   AND (ss2.d_qoy = 2)
-   AND (ss2.d_year = 2000)
-   AND (ss2.ca_county = ss3.ca_county)
-   AND (ss3.d_qoy = 3)
-   AND (ss3.d_year = 2000)
-   AND (ss1.ca_county = ws1.ca_county)
-   AND (ws1.d_qoy = 1)
-   AND (ws1.d_year = 2000)
-   AND (ws1.ca_county = ws2.ca_county)
-   AND (ws2.d_qoy = 2)
-   AND (ws2.d_year = 2000)
-   AND (ws1.ca_county = ws3.ca_county)
-   AND (ws3.d_qoy = 3)
-   AND (ws3.d_year = 2000)
-   AND ((CASE WHEN (ws1.web_sales > 0) THEN (CAST(ws2.web_sales AS DECIMAL(21,3)) / ws1.web_sales) ELSE null END) > (CASE WHEN (ss1.store_sales > 0) THEN (CAST(ss2.store_sales AS DECIMAL(21,3)) / ss1.store_sales) ELSE null END))
-   AND ((CASE WHEN (ws2.web_sales > 0) THEN (CAST(ws3.web_sales AS DECIMAL(21,3)) / ws2.web_sales) ELSE null END) > (CASE WHEN (ss2.store_sales > 0) THEN (CAST(ss3.store_sales AS DECIMAL(21,3)) / ss2.store_sales) ELSE null END))
-ORDER BY ss1.ca_county ASC
-
-
-    """
-}
diff --git a/regression-test/suites/cte_reuse/q4.groovy b/regression-test/suites/cte_reuse/q4.groovy
deleted file mode 100644
index 9cbff3344d..0000000000
--- a/regression-test/suites/cte_reuse/q4.groovy
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-suite("cte_reuse_q4") {
-    
-    sql "use regression_test_cte_reuse"
-    sql 'set enable_nereids_planner=true'
-    sql 'set enable_fallback_to_original_planner=false'
-    sql """
-        SET enable_pipeline_engine = true
-    """
-    sql "set forbid_unknown_col_stats=true"
-    
-
-    def result = sql "show backends;"
-    if (result.size() != 1) {
-        print("backends num: ${result.size()}");
-        return;
-    }
-
-    qt_sql """
-        EXPLAIN SHAPE PLAN
-        WITH
-  year_total AS (
-   SELECT
-     c_customer_id customer_id
-   , c_first_name customer_first_name
-   , c_last_name customer_last_name
-   , c_preferred_cust_flag customer_preferred_cust_flag
-   , c_birth_country customer_birth_country
-   , c_login customer_login
-   , c_email_address customer_email_address
-   , d_year dyear
-   , sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)) year_total
-   , 's' sale_type
-   FROM
-     customer
-   , store_sales
-   , date_dim
-   WHERE (c_customer_sk = ss_customer_sk)
-      AND (ss_sold_date_sk = d_date_sk)
-   GROUP BY c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-UNION ALL    SELECT
-     c_customer_id customer_id
-   , c_first_name customer_first_name
-   , c_last_name customer_last_name
-   , c_preferred_cust_flag customer_preferred_cust_flag
-   , c_birth_country customer_birth_country
-   , c_login customer_login
-   , c_email_address customer_email_address
-   , d_year dyear
-   , sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)) year_total
-   , 'c' sale_type
-   FROM
-     customer
-   , catalog_sales
-   , date_dim
-   WHERE (c_customer_sk = cs_bill_customer_sk)
-      AND (cs_sold_date_sk = d_date_sk)
-   GROUP BY c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-UNION ALL    SELECT
-     c_customer_id customer_id
-   , c_first_name customer_first_name
-   , c_last_name customer_last_name
-   , c_preferred_cust_flag customer_preferred_cust_flag
-   , c_birth_country customer_birth_country
-   , c_login customer_login
-   , c_email_address customer_email_address
-   , d_year dyear
-   , sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)) year_total
-   , 'w' sale_type
-   FROM
-     customer
-   , web_sales
-   , date_dim
-   WHERE (c_customer_sk = ws_bill_customer_sk)
-      AND (ws_sold_date_sk = d_date_sk)
-   GROUP BY c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-)
-SELECT
-  t_s_secyear.customer_id
-, t_s_secyear.customer_first_name
-, t_s_secyear.customer_last_name
-, t_s_secyear.customer_preferred_cust_flag
-FROM
-  year_total t_s_firstyear
-, year_total t_s_secyear
-, year_total t_c_firstyear
-, year_total t_c_secyear
-, year_total t_w_firstyear
-, year_total t_w_secyear
-WHERE (t_s_secyear.customer_id = t_s_firstyear.customer_id)
-   AND (t_s_firstyear.customer_id = t_c_secyear.customer_id)
-   AND (t_s_firstyear.customer_id = t_c_firstyear.customer_id)
-   AND (t_s_firstyear.customer_id = t_w_firstyear.customer_id)
-   AND (t_s_firstyear.customer_id = t_w_secyear.customer_id)
-   AND (t_s_firstyear.sale_type = 's')
-   AND (t_c_firstyear.sale_type = 'c')
-   AND (t_w_firstyear.sale_type = 'w')
-   AND (t_s_secyear.sale_type = 's')
-   AND (t_c_secyear.sale_type = 'c')
-   AND (t_w_secyear.sale_type = 'w')
-   AND (t_s_firstyear.dyear = 2001)
-   AND (t_s_secyear.dyear = (2001 + 1))
-   AND (t_c_firstyear.dyear = 2001)
-   AND (t_c_secyear.dyear = (2001 + 1))
-   AND (t_w_firstyear.dyear = 2001)
-   AND (t_w_secyear.dyear = (2001 + 1))
-   AND (t_s_firstyear.year_total > 0)
-   AND (t_c_firstyear.year_total > 0)
-   AND (t_w_firstyear.year_total > 0)
-   AND ((CASE WHEN (t_c_firstyear.year_total > 0) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE null END) > (CASE WHEN (t_s_firstyear.year_total > 0) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE null END))
-   AND ((CASE WHEN (t_c_firstyear.year_total > 0) THEN (t_c_secyear.year_total / t_c_firstyear.year_total) ELSE null END) > (CASE WHEN (t_w_firstyear.year_total > 0) THEN (t_w_secyear.year_total / t_w_firstyear.year_total) ELSE null END))
-ORDER BY t_s_secyear.customer_id ASC, t_s_secyear.customer_first_name ASC, t_s_secyear.customer_last_name ASC, t_s_secyear.customer_preferred_cust_flag ASC
-LIMIT 100
-
-    """
-}
diff --git a/regression-test/suites/cte_reuse/q47.groovy b/regression-test/suites/cte_reuse/q47.groovy
deleted file mode 100644
index 42e563275a..0000000000
--- a/regression-test/suites/cte_reuse/q47.groovy
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-suite("cte_reuse_q47") {
-    
-    sql "use regression_test_cte_reuse"
-    sql 'set enable_nereids_planner=true'
-    sql 'set enable_fallback_to_original_planner=false'
-    sql """
-        SET enable_pipeline_engine = true
-    """
-    sql "set forbid_unknown_col_stats=true"
-
-    def result = sql "show backends;"
-    if (result.size() != 1) {
-        print("backends num: ${result.size()}");
-        return;
-    }
-
-    qt_sql """
-        EXPLAIN SHAPE PLAN
-WITH
-  v1 AS (
-   SELECT
-     i_category
-   , i_brand
-   , s_store_name
-   , s_company_name
-   , d_year
-   , d_moy
-   , sum(ss_sales_price) sum_sales
-   , avg(sum(ss_sales_price)) OVER (PARTITION BY i_category, i_brand, s_store_name, s_company_name, d_year) avg_monthly_sales
-   , rank() OVER (PARTITION BY i_category, i_brand, s_store_name, s_company_name ORDER BY d_year ASC, d_moy ASC) rn
-   FROM
-     item
-   , store_sales
-   , date_dim
-   , store
-   WHERE (ss_item_sk = i_item_sk)
-      AND (ss_sold_date_sk = d_date_sk)
-      AND (ss_store_sk = s_store_sk)
-      AND ((d_year = 1999)
-         OR ((d_year = (1999 - 1))
-            AND (d_moy = 12))
-         OR ((d_year = (1999 + 1))
-            AND (d_moy = 1)))
-   GROUP BY i_category, i_brand, s_store_name, s_company_name, d_year, d_moy
-)
-, v2 AS (
-   SELECT
-     v1.i_category
-   , v1.i_brand
-   , v1.s_store_name
-   , v1.s_company_name
-   , v1.d_year
-   , v1.d_moy
-   , v1.avg_monthly_sales
-   , v1.sum_sales
-   , v1_lag.sum_sales psum
-   , v1_lead.sum_sales nsum
-   FROM
-     v1
-   , v1 v1_lag
-   , v1 v1_lead
-   WHERE (v1.i_category = v1_lag.i_category)
-      AND (v1.i_category = v1_lead.i_category)
-      AND (v1.i_brand = v1_lag.i_brand)
-      AND (v1.i_brand = v1_lead.i_brand)
-      AND (v1.s_store_name = v1_lag.s_store_name)
-      AND (v1.s_store_name = v1_lead.s_store_name)
-      AND (v1.s_company_name = v1_lag.s_company_name)
-      AND (v1.s_company_name = v1_lead.s_company_name)
-      AND (v1.rn = (v1_lag.rn + 1))
-      AND (v1.rn = (v1_lead.rn - 1))
-)
-SELECT *
-FROM
-  v2
-WHERE (d_year = 1999)
-   AND (avg_monthly_sales > 0)
-   AND ((CASE WHEN (avg_monthly_sales > 0) THEN (abs((sum_sales - avg_monthly_sales)) / avg_monthly_sales) ELSE null END) > CAST('0.1' AS DECIMAL(2,1)))
-ORDER BY (sum_sales - avg_monthly_sales) ASC, 3 ASC
-LIMIT 100
-
-
-    """
-}
diff --git a/regression-test/suites/cte_reuse/q57.groovy b/regression-test/suites/cte_reuse/q57.groovy
deleted file mode 100644
index 8332a69178..0000000000
--- a/regression-test/suites/cte_reuse/q57.groovy
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-suite("cte_reuse_q57") {
-    
-    sql "use regression_test_cte_reuse"
-    sql 'set enable_nereids_planner=true'
-    sql 'set enable_fallback_to_original_planner=false'
-    sql """
-        SET enable_pipeline_engine = true
-    """
-    sql "set forbid_unknown_col_stats=true"
-
-    def result = sql "show backends;"
-    if (result.size() != 1) {
-        print("backends num: ${result.size()}");
-        return;
-    }
-
-    qt_sql """
-        EXPLAIN SHAPE PLAN
-WITH
-  v1 AS (
-   SELECT
-     i_category
-   , i_brand
-   , cc_name
-   , d_year
-   , d_moy
-   , sum(cs_sales_price) sum_sales
-   , avg(sum(cs_sales_price)) OVER (PARTITION BY i_category, i_brand, cc_name, d_year) avg_monthly_sales
-   , rank() OVER (PARTITION BY i_category, i_brand, cc_name ORDER BY d_year ASC, d_moy ASC) rn
-   FROM
-     item
-   , catalog_sales
-   , date_dim
-   , call_center
-   WHERE (cs_item_sk = i_item_sk)
-      AND (cs_sold_date_sk = d_date_sk)
-      AND (cc_call_center_sk = cs_call_center_sk)
-      AND ((d_year = 1999)
-         OR ((d_year = (1999 - 1))
-            AND (d_moy = 12))
-         OR ((d_year = (1999 + 1))
-            AND (d_moy = 1)))
-   GROUP BY i_category, i_brand, cc_name, d_year, d_moy
-)
-, v2 AS (
-   SELECT
-     v1.i_category
-   , v1.i_brand
-   , v1.cc_name
-   , v1.d_year
-   , v1.d_moy
-   , v1.avg_monthly_sales
-   , v1.sum_sales
-   , v1_lag.sum_sales psum
-   , v1_lead.sum_sales nsum
-   FROM
-     v1
-   , v1 v1_lag
-   , v1 v1_lead
-   WHERE (v1.i_category = v1_lag.i_category)
-      AND (v1.i_category = v1_lead.i_category)
-      AND (v1.i_brand = v1_lag.i_brand)
-      AND (v1.i_brand = v1_lead.i_brand)
-      AND (v1.cc_name = v1_lag.cc_name)
-      AND (v1.cc_name = v1_lead.cc_name)
-      AND (v1.rn = (v1_lag.rn + 1))
-      AND (v1.rn = (v1_lead.rn - 1))
-)
-SELECT *
-FROM
-  v2
-WHERE (d_year = 1999)
-   AND (avg_monthly_sales > 0)
-   AND ((CASE WHEN (avg_monthly_sales > 0) THEN (abs((sum_sales - avg_monthly_sales)) / avg_monthly_sales) ELSE null END) > CAST('0.1' AS DECIMAL(2,1)))
-ORDER BY (sum_sales - avg_monthly_sales) ASC, 3 ASC
-LIMIT 100
-
-
-    """
-}
diff --git a/regression-test/suites/cte_reuse/q59.groovy b/regression-test/suites/cte_reuse/q59.groovy
deleted file mode 100644
index 2a0aeb2799..0000000000
--- a/regression-test/suites/cte_reuse/q59.groovy
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-suite("cte_reuse_q59") {
-    
-    sql "use regression_test_cte_reuse"
-    sql 'set enable_nereids_planner=true'
-    sql 'set enable_fallback_to_original_planner=false'
-    sql """
-        SET enable_pipeline_engine = true
-    """
-    sql "set forbid_unknown_col_stats=true"
-
-    def result = sql "show backends;"
-    if (result.size() != 1) {
-        print("backends num: ${result.size()}");
-        return;
-    }
-
-    qt_sql """
-        EXPLAIN SHAPE PLAN
-
-        WITH
-  wss AS (
-   SELECT
-     d_week_seq
-   , ss_store_sk
-   , sum((CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE null END)) sun_sales
-   , sum((CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE null END)) mon_sales
-   , sum((CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE null END)) tue_sales
-   , sum((CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE null END)) wed_sales
-   , sum((CASE WHEN (d_day_name = 'Thursday ') THEN ss_sales_price ELSE null END)) thu_sales
-   , sum((CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE null END)) fri_sales
-   , sum((CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE null END)) sat_sales
-   FROM
-     store_sales
-   , date_dim
-   WHERE (d_date_sk = ss_sold_date_sk)
-   GROUP BY d_week_seq, ss_store_sk
-)
-SELECT
-  s_store_name1
-, s_store_id1
-, d_week_seq1
-, (sun_sales1 / sun_sales2)
-, (mon_sales1 / mon_sales2)
-, (tue_sales1 / tue_sales2)
-, (wed_sales1 / wed_sales2)
-, (thu_sales1 / thu_sales2)
-, (fri_sales1 / fri_sales2)
-, (sat_sales1 / sat_sales2)
-FROM
-  (
-   SELECT
-     s_store_name s_store_name1
-   , wss.d_week_seq d_week_seq1
-   , s_store_id s_store_id1
-   , sun_sales sun_sales1
-   , mon_sales mon_sales1
-   , tue_sales tue_sales1
-   , wed_sales wed_sales1
-   , thu_sales thu_sales1
-   , fri_sales fri_sales1
-   , sat_sales sat_sales1
-   FROM
-     wss
-   , store
-   , date_dim d
-   WHERE (d.d_week_seq = wss.d_week_seq)
-      AND (ss_store_sk = s_store_sk)
-      AND (d_month_seq BETWEEN 1212 AND (1212 + 11))
-)  y
-, (
-   SELECT
-     s_store_name s_store_name2
-   , wss.d_week_seq d_week_seq2
-   , s_store_id s_store_id2
-   , sun_sales sun_sales2
-   , mon_sales mon_sales2
-   , tue_sales tue_sales2
-   , wed_sales wed_sales2
-   , thu_sales thu_sales2
-   , fri_sales fri_sales2
-   , sat_sales sat_sales2
-   FROM
-     wss
-   , store
-   , date_dim d
-   WHERE (d.d_week_seq = wss.d_week_seq)
-      AND (ss_store_sk = s_store_sk)
-      AND (d_month_seq BETWEEN (1212 + 12) AND (1212 + 23))
-)  x
-WHERE (s_store_id1 = s_store_id2)
-   AND (d_week_seq1 = (d_week_seq2 - 52))
-ORDER BY s_store_name1 ASC, s_store_id1 ASC, d_week_seq1 ASC
-LIMIT 100
-
-    """
-}
diff --git a/regression-test/suites/cte_reuse/q64.groovy b/regression-test/suites/cte_reuse/q64.groovy
deleted file mode 100644
index 1e7bc485f6..0000000000
--- a/regression-test/suites/cte_reuse/q64.groovy
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-suite("cte_reuse_q64") {
-    
-    //    sql "use regression_test_cte_reuse"
-    //    sql 'set enable_nereids_planner=true'
-    //    sql 'set enable_fallback_to_original_planner=false'
-    //    sql """
-    //        SET enable_pipeline_engine = true
-    //    """
-    //    sql "set forbid_unknown_col_stats=true"
-    //
-    //    def result = sql "show backends;"
-    //    if (result.size() != 1) {
-    //        print("backends num: ${result.size()}");
-    //        return;
-    //    }
-
-    //    qt_sql """
-    //        EXPLAIN SHAPE PLAN
-    //WITH
-    //  cs_ui AS (
-    //   SELECT
-    //     cs_item_sk
-    //   , sum(cs_ext_list_price) sale
-    //   , sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)) refund
-    //   FROM
-    //     catalog_sales
-    //   , catalog_returns
-    //   WHERE (cs_item_sk = cr_item_sk)
-    //      AND (cs_order_number = cr_order_number)
-    //   GROUP BY cs_item_sk
-    //   HAVING (sum(cs_ext_list_price) > (2 * sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))))
-    //)
-    //, cross_sales AS (
-    //   SELECT
-    //     i_product_name product_name
-    //   , i_item_sk item_sk
-    //   , s_store_name store_name
-    //   , s_zip store_zip
-    //   , ad1.ca_street_number b_street_number
-    //   , ad1.ca_street_name b_street_name
-    //   , ad1.ca_city b_city
-    //   , ad1.ca_zip b_zip
-    //   , ad2.ca_street_number c_street_number
-    //   , ad2.ca_street_name c_street_name
-    //   , ad2.ca_city c_city
-    //   , ad2.ca_zip c_zip
-    //   , d1.d_year syear
-    //   , d2.d_year fsyear
-    //   , d3.d_year s2year
-    //   , count(*) cnt
-    //   , sum(ss_wholesale_cost) s1
-    //   , sum(ss_list_price) s2
-    //   , sum(ss_coupon_amt) s3
-    //   FROM
-    //     store_sales
-    //   , store_returns
-    //   , cs_ui
-    //   , date_dim d1
-    //   , date_dim d2
-    //   , date_dim d3
-    //   , store
-    //   , customer
-    //   , customer_demographics cd1
-    //   , customer_demographics cd2
-    //   , promotion
-    //   , household_demographics hd1
-    //   , household_demographics hd2
-    //   , customer_address ad1
-    //   , customer_address ad2
-    //   , income_band ib1
-    //   , income_band ib2
-    //   , item
-    //   WHERE (ss_store_sk = s_store_sk)
-    //      AND (ss_sold_date_sk = d1.d_date_sk)
-    //      AND (ss_customer_sk = c_customer_sk)
-    //      AND (ss_cdemo_sk = cd1.cd_demo_sk)
-    //      AND (ss_hdemo_sk = hd1.hd_demo_sk)
-    //      AND (ss_addr_sk = ad1.ca_address_sk)
-    //      AND (ss_item_sk = i_item_sk)
-    //      AND (ss_item_sk = sr_item_sk)
-    //      AND (ss_ticket_number = sr_ticket_number)
-    //      AND (ss_item_sk = cs_ui.cs_item_sk)
-    //      AND (c_current_cdemo_sk = cd2.cd_demo_sk)
-    //      AND (c_current_hdemo_sk = hd2.hd_demo_sk)
-    //      AND (c_current_addr_sk = ad2.ca_address_sk)
-    //      AND (c_first_sales_date_sk = d2.d_date_sk)
-    //      AND (c_first_shipto_date_sk = d3.d_date_sk)
-    //      AND (ss_promo_sk = p_promo_sk)
-    //      AND (hd1.hd_income_band_sk = ib1.ib_income_band_sk)
-    //      AND (hd2.hd_income_band_sk = ib2.ib_income_band_sk)
-    //      AND (cd1.cd_marital_status <> cd2.cd_marital_status)
-    //      AND (i_color IN ('purple'   , 'burlywood'   , 'indian'   , 'spring'   , 'floral'   , 'medium'))
-    //      AND (i_current_price BETWEEN 64 AND (64 + 10))
-    //      AND (i_current_price BETWEEN (64 + 1) AND (64 + 15))
-    //   GROUP BY i_product_name, i_item_sk, s_store_name, s_zip, ad1.ca_street_number, ad1.ca_street_name, ad1.ca_city, ad1.ca_zip, ad2.ca_street_number, ad2.ca_street_name, ad2.ca_city, ad2.ca_zip, d1.d_year, d2.d_year, d3.d_year
-    //)
-    //SELECT
-    //  cs1.product_name
-    //, cs1.store_name
-    //, cs1.store_zip
-    //, cs1.b_street_number
-    //, cs1.b_street_name
-    //, cs1.b_city
-    //, cs1.b_zip
-    //, cs1.c_street_number
-    //, cs1.c_street_name
-    //, cs1.c_city
-    //, cs1.c_zip
-    //, cs1.syear
-    //, cs1.cnt
-    //, cs1.s1 s11
-    //, cs1.s2 s21
-    //, cs1.s3 s31
-    //, cs2.s1 s12
-    //, cs2.s2 s22
-    //, cs2.s3 s32
-    //, cs2.syear
-    //, cs2.cnt
-    //FROM
-    //  cross_sales cs1
-    //, cross_sales cs2
-    //WHERE (cs1.item_sk = cs2.item_sk)
-    //   AND (cs1.syear = 1999)
-    //   AND (cs2.syear = (1999 + 1))
-    //   AND (cs2.cnt <= cs1.cnt)
-    //   AND (cs1.store_name = cs2.store_name)
-    //   AND (cs1.store_zip = cs2.store_zip)
-    //ORDER BY cs1.product_name ASC, cs1.store_name ASC, cs2.cnt ASC, 14, 15, 16, 17, 18
-    //    """
-}
diff --git a/regression-test/suites/cte_reuse/q74.groovy b/regression-test/suites/cte_reuse/q74.groovy
deleted file mode 100644
index 0341d5190f..0000000000
--- a/regression-test/suites/cte_reuse/q74.groovy
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-suite("cte_reuse_q74") {
-    
-    sql "use regression_test_cte_reuse"
-    sql 'set enable_nereids_planner=true'
-    sql 'set enable_fallback_to_original_planner=false'
-    sql """
-        SET enable_pipeline_engine = true
-    """
-    sql "set forbid_unknown_col_stats=true"
-
-    def result = sql "show backends;"
-    if (result.size() != 1) {
-        print("backends num: ${result.size()}");
-        return;
-    }
-
-    qt_sql """
-EXPLAIN SHAPE PLAN
-       WITH
-  year_total AS (
-   SELECT
-     c_customer_id customer_id
-   , c_first_name customer_first_name
-   , c_last_name customer_last_name
-   , d_year YEAR
-   , sum(ss_net_paid) year_total
-   , 's' sale_type
-   FROM
-     customer
-   , store_sales
-   , date_dim
-   WHERE (c_customer_sk = ss_customer_sk)
-      AND (ss_sold_date_sk = d_date_sk)
-      AND (d_year IN (2001   , (2001 + 1)))
-   GROUP BY c_customer_id, c_first_name, c_last_name, d_year
-UNION ALL    SELECT
-     c_customer_id customer_id
-   , c_first_name customer_first_name
-   , c_last_name customer_last_name
-   , d_year YEAR
-   , sum(ws_net_paid) year_total
-   , 'w' sale_type
-   FROM
-     customer
-   , web_sales
-   , date_dim
-   WHERE (c_customer_sk = ws_bill_customer_sk)
-      AND (ws_sold_date_sk = d_date_sk)
-      AND (d_year IN (2001   , (2001 + 1)))
-   GROUP BY c_customer_id, c_first_name, c_last_name, d_year
-)
-SELECT
-  t_s_secyear.customer_id
-, t_s_secyear.customer_first_name
-, t_s_secyear.customer_last_name
-FROM
-  year_total t_s_firstyear
-, year_total t_s_secyear
-, year_total t_w_firstyear
-, year_total t_w_secyear
-WHERE (t_s_secyear.customer_id = t_s_firstyear.customer_id)
-   AND (t_s_firstyear.customer_id = t_w_secyear.customer_id)
-   AND (t_s_firstyear.customer_id = t_w_firstyear.customer_id)
-   AND (t_s_firstyear.sale_type = 's')
-   AND (t_w_firstyear.sale_type = 'w')
-   AND (t_s_secyear.sale_type = 's')
-   AND (t_w_secyear.sale_type = 'w')
-   AND (t_s_firstyear.year = 2001)
-   AND (t_s_secyear.year = (2001 + 1))
-   AND (t_w_firstyear.year = 2001)
-   AND (t_w_secyear.year = (2001 + 1))
-   AND (t_s_firstyear.year_total > 0)
-   AND (t_w_firstyear.year_total > 0)
-   AND ((CASE WHEN (t_w_firstyear.year_total > 0) THEN (t_w_secyear.year_total / t_w_firstyear.year_total) ELSE null END) > (CASE WHEN (t_s_firstyear.year_total > 0) THEN (t_s_secyear.year_total / t_s_firstyear.year_total) ELSE null END))
-ORDER BY 1 ASC, 1 ASC, 1 ASC
-LIMIT 100
-
-
-
-    """
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 21/36: [fix](sequence) value predicates shouldn't be push down when has sequence column (#20408)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 0b16cde515389ef20f3cc3e5e406337e7dfe53a6
Author: Xin Liao <li...@126.com>
AuthorDate: Mon Jun 5 19:18:34 2023 +0800

    [fix](sequence) value predicates shouldn't be push down when has sequence column (#20408)
    
    * (fix)[sequence] value predicates shouldn't be push down when has sequence column
    
    * add case
---
 be/src/olap/rowset/beta_rowset_reader.cpp          |   6 +-
 .../unique/test_unique_value_predicate.out         |  45 +++++++++
 .../unique/test_unique_value_predicate.groovy      | 111 +++++++++++++++++++++
 3 files changed, 160 insertions(+), 2 deletions(-)

diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp b/be/src/olap/rowset/beta_rowset_reader.cpp
index ab69aea140..2df50d1177 100644
--- a/be/src/olap/rowset/beta_rowset_reader.cpp
+++ b/be/src/olap/rowset/beta_rowset_reader.cpp
@@ -304,10 +304,12 @@ Status BetaRowsetReader::next_block_view(vectorized::BlockView* block_view) {
 
 bool BetaRowsetReader::_should_push_down_value_predicates() const {
     // if unique table with rowset [0-x] or [0-1] [2-y] [...],
-    // value column predicates can be pushdown on rowset [0-x] or [2-y], [2-y] must be compaction and not overlapping
+    // value column predicates can be pushdown on rowset [0-x] or [2-y], [2-y]
+    // must be compaction, not overlapping and don't have sequence column
     return _rowset->keys_type() == UNIQUE_KEYS &&
            (((_rowset->start_version() == 0 || _rowset->start_version() == 2) &&
-             !_rowset->_rowset_meta->is_segments_overlapping()) ||
+             !_rowset->_rowset_meta->is_segments_overlapping() &&
+             _context->sequence_id_idx == -1) ||
             _context->enable_unique_key_merge_on_write);
 }
 
diff --git a/regression-test/data/data_model_p0/unique/test_unique_value_predicate.out b/regression-test/data/data_model_p0/unique/test_unique_value_predicate.out
new file mode 100644
index 0000000000..0fe421460f
--- /dev/null
+++ b/regression-test/data/data_model_p0/unique/test_unique_value_predicate.out
@@ -0,0 +1,45 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select --
+0	1	test char	2000-01-01
+
+-- !select --
+0	1	test char	2000-01-01
+
+-- !select --
+
+-- !select --
+0	1	test char	2000-01-01
+
+-- !select --
+0	2	test int	2000-02-02
+
+-- !select --
+0	2	test int	2000-02-02
+
+-- !select --
+0	2	test int	2000-02-02
+
+-- !select --
+
+-- !select --
+0	1	test char	2000-01-01
+
+-- !select --
+0	1	test char	2000-01-01
+
+-- !select --
+
+-- !select --
+0	1	test char	2000-01-01
+
+-- !select --
+0	2	test int	2000-02-02
+
+-- !select --
+0	2	test int	2000-02-02
+
+-- !select --
+0	2	test int	2000-02-02
+
+-- !select --
+
diff --git a/regression-test/suites/data_model_p0/unique/test_unique_value_predicate.groovy b/regression-test/suites/data_model_p0/unique/test_unique_value_predicate.groovy
new file mode 100644
index 0000000000..c5a1e5b4ca
--- /dev/null
+++ b/regression-test/suites/data_model_p0/unique/test_unique_value_predicate.groovy
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_unique_value_predicate") {
+    // test uniq table
+    def tbName = "test_uniq_value_predicate"
+    // mor without seq
+    sql "DROP TABLE IF EXISTS ${tbName}"
+    sql """
+            CREATE TABLE IF NOT EXISTS ${tbName} (
+                k int,
+                int_value int,
+                char_value char(10),
+                date_value date
+            )
+            UNIQUE KEY(k)
+            DISTRIBUTED BY HASH(k) BUCKETS 1
+            properties("replication_num" = "1",
+                       "disable_auto_compaction" = "true",
+                       "enable_unique_key_merge_on_write" = "false");
+        """
+    sql "insert into ${tbName} values(0, 2, 'test int', '2000-02-02')"
+    sql "insert into ${tbName} values(0, 1, 'test char', '2000-01-01')"
+    qt_select "select * from ${tbName}"
+    qt_select "select * from ${tbName} where k = 0"
+    qt_select "select * from ${tbName} where k = 0 && int_value = 2"
+    qt_select "select * from ${tbName} where k = 0 && int_value = 1"
+
+    // mor with seq
+    sql "DROP TABLE IF EXISTS ${tbName}"
+    sql """
+            CREATE TABLE IF NOT EXISTS ${tbName} (
+                k int,
+                int_value int,
+                char_value char(10),
+                date_value date
+            )
+            UNIQUE KEY(k)
+            DISTRIBUTED BY HASH(k) BUCKETS 1
+            properties("replication_num" = "1",
+                       "function_column.sequence_col" = "int_value",
+                       "disable_auto_compaction" = "true",
+                       "enable_unique_key_merge_on_write" = "false");
+        """
+    sql "insert into ${tbName} values(0, 2, 'test int', '2000-02-02')"
+    sql "insert into ${tbName} values(0, 1, 'test char', '2000-01-01')"
+    qt_select "select * from ${tbName}"
+    qt_select "select * from ${tbName} where k = 0"
+    qt_select "select * from ${tbName} where k = 0 && int_value = 2"
+    qt_select "select * from ${tbName} where k = 0 && int_value = 1"
+
+    // mow without seq
+    sql "DROP TABLE IF EXISTS ${tbName}"
+    sql """
+            CREATE TABLE IF NOT EXISTS ${tbName} (
+                k int,
+                int_value int,
+                char_value char(10),
+                date_value date
+            )
+            UNIQUE KEY(k)
+            DISTRIBUTED BY HASH(k) BUCKETS 1
+            properties("replication_num" = "1",
+                       "disable_auto_compaction" = "true",
+                       "enable_unique_key_merge_on_write" = "true");
+        """
+    sql "insert into ${tbName} values(0, 2, 'test int', '2000-02-02')"
+    sql "insert into ${tbName} values(0, 1, 'test char', '2000-01-01')"
+    qt_select "select * from ${tbName}"
+    qt_select "select * from ${tbName} where k = 0"
+    qt_select "select * from ${tbName} where k = 0 && int_value = 2"
+    qt_select "select * from ${tbName} where k = 0 && int_value = 1"
+
+    // mow with seq
+    sql "DROP TABLE IF EXISTS ${tbName}"
+    sql """
+            CREATE TABLE IF NOT EXISTS ${tbName} (
+                k int,
+                int_value int,
+                char_value char(10),
+                date_value date
+            )
+            UNIQUE KEY(k)
+            DISTRIBUTED BY HASH(k) BUCKETS 1
+            properties("replication_num" = "1",
+                       "function_column.sequence_col" = "int_value",
+                       "disable_auto_compaction" = "true",
+                       "enable_unique_key_merge_on_write" = "true");
+        """
+    sql "insert into ${tbName} values(0, 2, 'test int', '2000-02-02')"
+    sql "insert into ${tbName} values(0, 1, 'test char', '2000-01-01')"
+    qt_select "select * from ${tbName}"
+    qt_select "select * from ${tbName} where k = 0"
+    qt_select "select * from ${tbName} where k = 0 && int_value = 2"
+    qt_select "select * from ${tbName} where k = 0 && int_value = 1"
+    sql "DROP TABLE ${tbName}"
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 12/36: [feature](planner)(nereids) support user defined variable (#20334)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 1cc61fbf1d96c22eb034bcabe7c5d21997753261
Author: Chengpeng Yan <41...@users.noreply.github.com>
AuthorDate: Tue Jun 6 14:35:16 2023 +0800

    [feature](planner)(nereids) support user defined variable (#20334)
    
    Support user-defined variables.
    After this PR, we can use `set @a = xx` to define a user variable and use it in the query like `select @a`.
    
    the changes of this PR:
    1. Support the grammar for `set user variable` in the parser.
    2. Add the `userVars` in `VariableMgr` to store the user-defined variables.
    3. For the `set @a = xx`, we will store the variable name and its value in the `userVars` in `VariableMgr`.
    4. For the `select @a`, we will get the value for the variable name in `userVars`.
---
 .../antlr4/org/apache/doris/nereids/DorisParser.g4 |  2 +-
 fe/fe-core/src/main/cup/sql_parser.cup             | 12 ++--
 .../main/java/org/apache/doris/analysis/Expr.java  |  3 +-
 .../apache/doris/analysis/ExpressionFunctions.java |  6 +-
 .../java/org/apache/doris/analysis/SetType.java    |  3 +-
 .../{SetType.java => SetUserDefinedVar.java}       | 49 ++++---------
 .../java/org/apache/doris/analysis/SetVar.java     | 37 ++++++++--
 .../{SysVariableDesc.java => VariableExpr.java}    | 66 ++++++++++++------
 .../doris/nereids/parser/LogicalPlanBuilder.java   |  3 +-
 .../nereids/trees/expressions/literal/Literal.java |  7 ++
 .../main/java/org/apache/doris/qe/SetExecutor.java |  3 +
 .../main/java/org/apache/doris/qe/VariableMgr.java | 80 ++++++++++++++++++++--
 .../apache/doris/rewrite/FoldConstantsRule.java    | 14 ++--
 .../java/org/apache/doris/analysis/SetVarTest.java |  2 +-
 ...VariableDescTest.java => VariableExprTest.java} |  6 +-
 .../java/org/apache/doris/qe/VariableMgrTest.java  |  4 +-
 regression-test/data/nereids_p0/test_user_var.out  | 16 +++++
 .../data/query_p0/set/test_user_var.out            | 16 +++++
 .../suites/nereids_p0/test_user_var.groovy         | 32 +++++++++
 .../suites/query_p0/set/test_user_var.groovy       | 30 ++++++++
 20 files changed, 299 insertions(+), 92 deletions(-)

diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
index 642b65a182..29faec5a32 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
@@ -355,7 +355,7 @@ primaryExpression
       (COMMA arguments+=expression)* (ORDER BY sortItem (COMMA sortItem)*)?)? RIGHT_PAREN
       (OVER windowSpec)?                                                                        #functionCall
     | LEFT_PAREN query RIGHT_PAREN                                                             #subqueryExpression
-    | ATSIGN identifier                                                                        #userVariable
+    | ATSIGN identifierOrText                                                                        #userVariable
     | DOUBLEATSIGN (kind=(GLOBAL | SESSION) DOT)? identifier                                     #systemVariable
     | identifier                                                                               #columnReference
     | base=primaryExpression DOT fieldName=identifier                                          #dereference
diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index 3a85b99dd4..4f2014b001 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -4914,9 +4914,9 @@ option_value_no_option_type ::=
     {:
         RESULT = new SetVar(variable, expr);
     :}
-    | AT ident_or_text:var equal literal:expr
+    | AT ident_or_text:var equal expr:expr
     {:
-        RESULT = new SetVar(var, expr);
+        RESULT = new SetUserDefinedVar(var, expr);
     :}
     /* Ident */
     | AT AT variable_name:variable equal set_expr_or_default:expr
@@ -6398,13 +6398,17 @@ exists_predicate ::=
 non_pred_expr ::=
   sign_chain_expr:e
   {: RESULT = e; :}
+  | AT ident:l
+  {:
+    RESULT = new VariableExpr(l, SetType.USER);
+  :}
   | AT AT ident:l
   {:
-    RESULT = new SysVariableDesc(l);
+    RESULT = new VariableExpr(l);
   :}
   | AT AT var_ident_type:type ident:l
   {:
-    RESULT = new SysVariableDesc(l, type);
+    RESULT = new VariableExpr(l, type);
   :}
   | literal:l
   {: RESULT = l; :}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
index a6cc659794..8633b868eb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
@@ -990,7 +990,8 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
             // Hack to ensure BE never sees TYPE_NULL. If an expr makes it this far without
             // being cast to a non-NULL type, the type doesn't matter and we can cast it
             // arbitrarily.
-            Preconditions.checkState(this instanceof NullLiteral || this instanceof SlotRef);
+            Preconditions.checkState(this instanceof NullLiteral || this instanceof SlotRef
+                    || this instanceof VariableExpr);
             return NullLiteral.create(ScalarType.BOOLEAN).treeToThrift();
         }
         TExpr result = new TExpr();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExpressionFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExpressionFunctions.java
index e7dae9617b..e39748b570 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExpressionFunctions.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExpressionFunctions.java
@@ -58,7 +58,7 @@ public enum ExpressionFunctions {
     public Expr evalExpr(Expr constExpr) {
         // Function's arg are all LiteralExpr.
         for (Expr child : constExpr.getChildren()) {
-            if (!(child instanceof LiteralExpr) && !(child instanceof SysVariableDesc)) {
+            if (!(child instanceof LiteralExpr) && !(child instanceof VariableExpr)) {
                 return constExpr;
             }
         }
@@ -115,8 +115,8 @@ public enum ExpressionFunctions {
                     return constExpr;
                 }
             }
-        } else if (constExpr instanceof SysVariableDesc) {
-            return ((SysVariableDesc) constExpr).getLiteralExpr();
+        } else if (constExpr instanceof VariableExpr) {
+            return ((VariableExpr) constExpr).getLiteralExpr();
         }
         return constExpr;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetType.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetType.java
index 924c1f38a9..ae0fbd27bc 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetType.java
@@ -23,7 +23,8 @@ import org.apache.doris.thrift.TVarType;
 public enum SetType {
     DEFAULT("DEFAULT"),
     GLOBAL("GLOBAL"),
-    SESSION("SESSION");
+    SESSION("SESSION"),
+    USER("USER");
 
     private String desc;
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetType.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetUserDefinedVar.java
similarity index 53%
copy from fe/fe-core/src/main/java/org/apache/doris/analysis/SetType.java
copy to fe/fe-core/src/main/java/org/apache/doris/analysis/SetUserDefinedVar.java
index 924c1f38a9..7b0c5d98aa 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetUserDefinedVar.java
@@ -17,44 +17,23 @@
 
 package org.apache.doris.analysis;
 
-import org.apache.doris.thrift.TVarType;
+import org.apache.doris.catalog.ScalarType;
+import org.apache.doris.common.AnalysisException;
 
-// Set statement type.
-public enum SetType {
-    DEFAULT("DEFAULT"),
-    GLOBAL("GLOBAL"),
-    SESSION("SESSION");
-
-    private String desc;
-
-    SetType(String desc) {
-        this.desc = desc;
-    }
-
-    public TVarType toThrift() {
-        switch (this) {
-            case GLOBAL:
-                return TVarType.GLOBAL;
-            default:
-                return TVarType.SESSION;
-        }
-    }
-
-    public static SetType fromThrift(TVarType tType) {
-        switch (tType) {
-            case GLOBAL:
-                return SetType.GLOBAL;
-            default:
-                return SetType.SESSION;
-        }
-    }
-
-    public String toSql() {
-        return desc;
+public class SetUserDefinedVar extends SetVar {
+    public SetUserDefinedVar(String variable, Expr value) {
+        super(SetType.USER, variable, value, SetVarType.SET_USER_DEFINED_VAR);
     }
 
     @Override
-    public String toString() {
-        return toSql();
+    public void analyze(Analyzer analyzer) throws AnalysisException  {
+        Expr expression = getValue();
+        if (expression instanceof NullLiteral) {
+            setResult(NullLiteral.create(ScalarType.NULL));
+        } else if (expression instanceof LiteralExpr) {
+            setResult((LiteralExpr) expression);
+        } else {
+            throw new AnalysisException("Unsupported to set the non-literal for user defined variables.");
+        }
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetVar.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetVar.java
index ccb505bbe4..8afa942dba 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetVar.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetVar.java
@@ -43,7 +43,8 @@ public class SetVar {
         SET_LDAP_PASS_VAR,
         SET_NAMES_VAR,
         SET_TRANSACTION,
-        SET_USER_PROPERTY_VAR
+        SET_USER_PROPERTY_VAR,
+        SET_USER_DEFINED_VAR,
     }
 
     private String variable;
@@ -75,14 +76,36 @@ public class SetVar {
         }
     }
 
+    public SetVar(SetType setType, String variable, Expr value, SetVarType varType) {
+        this.type = setType;
+        this.varType = varType;
+        this.variable = variable;
+        this.value = value;
+        if (value instanceof LiteralExpr) {
+            this.result = (LiteralExpr) value;
+        }
+    }
+
     public String getVariable() {
         return variable;
     }
 
-    public LiteralExpr getValue() {
+    public Expr getValue() {
+        return value;
+    }
+
+    public void setValue(Expr value) {
+        this.value = value;
+    }
+
+    public LiteralExpr getResult() {
         return result;
     }
 
+    public void setResult(LiteralExpr result) {
+        this.result = result;
+    }
+
     public SetType getType() {
         return type;
     }
@@ -144,7 +167,7 @@ public class SetVar {
         }
 
         if (getVariable().equalsIgnoreCase(SessionVariable.PREFER_JOIN_METHOD)) {
-            String value = getValue().getStringValue();
+            String value = getResult().getStringValue();
             if (!value.equalsIgnoreCase("broadcast") && !value.equalsIgnoreCase("shuffle")) {
                 ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_VALUE_FOR_VAR,
                         SessionVariable.PREFER_JOIN_METHOD, value);
@@ -153,11 +176,11 @@ public class SetVar {
 
         // Check variable time_zone value is valid
         if (getVariable().equalsIgnoreCase(SessionVariable.TIME_ZONE)) {
-            this.value = new StringLiteral(TimeUtils.checkTimeZoneValidAndStandardize(getValue().getStringValue()));
+            this.value = new StringLiteral(TimeUtils.checkTimeZoneValidAndStandardize(getResult().getStringValue()));
             this.result = (LiteralExpr) this.value;
         }
         if (getVariable().equalsIgnoreCase(SessionVariable.PARALLEL_FRAGMENT_EXEC_INSTANCE_NUM)) {
-            int instanceNum = Integer.parseInt(getValue().getStringValue());
+            int instanceNum = Integer.parseInt(getResult().getStringValue());
             if (instanceNum > Config.max_instance_num) {
                 ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_VALUE_FOR_VAR,
                         SessionVariable.PARALLEL_FRAGMENT_EXEC_INSTANCE_NUM,
@@ -165,11 +188,11 @@ public class SetVar {
             }
         }
         if (getVariable().equalsIgnoreCase(SessionVariable.EXEC_MEM_LIMIT)) {
-            this.value = new StringLiteral(Long.toString(ParseUtil.analyzeDataVolumn(getValue().getStringValue())));
+            this.value = new StringLiteral(Long.toString(ParseUtil.analyzeDataVolumn(getResult().getStringValue())));
             this.result = (LiteralExpr) this.value;
         }
         if (getVariable().equalsIgnoreCase(SessionVariable.SCAN_QUEUE_MEM_LIMIT)) {
-            this.value = new StringLiteral(Long.toString(ParseUtil.analyzeDataVolumn(getValue().getStringValue())));
+            this.value = new StringLiteral(Long.toString(ParseUtil.analyzeDataVolumn(getResult().getStringValue())));
             this.result = (LiteralExpr) this.value;
         }
         if (getVariable().equalsIgnoreCase("is_report_success")) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SysVariableDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/VariableExpr.java
similarity index 74%
rename from fe/fe-core/src/main/java/org/apache/doris/analysis/SysVariableDesc.java
rename to fe/fe-core/src/main/java/org/apache/doris/analysis/VariableExpr.java
index 023be528e6..649bb4017f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SysVariableDesc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/VariableExpr.java
@@ -32,33 +32,37 @@ import org.apache.doris.thrift.TStringLiteral;
 
 import com.google.common.base.Strings;
 
+import java.math.BigDecimal;
 import java.util.Objects;
 
-// System variable
+// Variable expr: including the system variable and user define variable.
 // Converted to StringLiteral in analyze, if this variable is not exist, throw AnalysisException.
-public class SysVariableDesc extends Expr {
+public class VariableExpr extends Expr {
     private String name;
     private SetType setType;
+    private boolean isNull;
     private boolean boolValue;
     private long intValue;
     private double floatValue;
+    private BigDecimal decimalValue;
     private String strValue;
 
     private LiteralExpr literalExpr;
 
-    public SysVariableDesc(String name) {
+    public VariableExpr(String name) {
         this(name, SetType.SESSION);
     }
 
-    public SysVariableDesc(String name, SetType setType) {
+    public VariableExpr(String name, SetType setType) {
         this.name = name;
         this.setType = setType;
     }
 
-    protected SysVariableDesc(SysVariableDesc other) {
+    protected VariableExpr(VariableExpr other) {
         super(other);
         name = other.name;
         setType = other.setType;
+        isNull = other.isNull;
         boolValue = other.boolValue;
         intValue = other.intValue;
         floatValue = other.floatValue;
@@ -67,18 +71,22 @@ public class SysVariableDesc extends Expr {
 
     @Override
     public Expr clone() {
-        return new SysVariableDesc(this);
+        return new VariableExpr(this);
     }
 
     @Override
     public void analyzeImpl(Analyzer analyzer) throws AnalysisException {
-        VariableMgr.fillValue(analyzer.getContext().getSessionVariable(), this);
-        if (!Strings.isNullOrEmpty(name) && VariableVarConverters.hasConverter(name)) {
-            setType(Type.VARCHAR);
-            try {
-                setStringValue(VariableVarConverters.decode(name, intValue));
-            } catch (DdlException e) {
-                ErrorReport.reportAnalysisException(e.getMessage());
+        if (setType == SetType.USER) {
+            VariableMgr.fillValueForUserDefinedVar(this);
+        } else {
+            VariableMgr.fillValue(analyzer.getContext().getSessionVariable(), this);
+            if (!Strings.isNullOrEmpty(name) && VariableVarConverters.hasConverter(name)) {
+                setType(Type.VARCHAR);
+                try {
+                    setStringValue(VariableVarConverters.decode(name, intValue));
+                } catch (DdlException e) {
+                    ErrorReport.reportAnalysisException(e.getMessage());
+                }
             }
         }
     }
@@ -91,6 +99,14 @@ public class SysVariableDesc extends Expr {
         return setType;
     }
 
+    public void setIsNull() {
+        isNull = true;
+    }
+
+    public boolean isNull() {
+        return isNull;
+    }
+
     public void setBoolValue(boolean value) {
         this.boolValue = value;
         this.literalExpr = new BoolLiteral(value);
@@ -106,6 +122,11 @@ public class SysVariableDesc extends Expr {
         this.literalExpr = new FloatLiteral(value);
     }
 
+    public void setDecimalValue(BigDecimal value) {
+        this.decimalValue = value;
+        this.literalExpr = new DecimalLiteral(value);
+    }
+
     public void setStringValue(String value) {
         this.strValue = value;
         this.literalExpr = new StringLiteral(value);
@@ -164,9 +185,14 @@ public class SysVariableDesc extends Expr {
 
     @Override
     public String toSqlImpl() {
-        StringBuilder sb = new StringBuilder("@@");
-        if (setType == SetType.GLOBAL) {
-            sb.append("GLOBAL.");
+        StringBuilder sb = new StringBuilder();
+        if (setType == SetType.USER) {
+            sb.append("@");
+        } else {
+            sb.append("@@");
+            if (setType == SetType.GLOBAL) {
+                sb.append("GLOBAL.");
+            }
         }
         sb.append(name);
         return sb.toString();
@@ -182,16 +208,16 @@ public class SysVariableDesc extends Expr {
         if (this == obj) {
             return true;
         }
-        if (!(obj instanceof SysVariableDesc)) {
+        if (!(obj instanceof VariableExpr)) {
             return false;
         }
-        if (!name.equals(((SysVariableDesc) obj).getName())) {
+        if (!name.equals(((VariableExpr) obj).getName())) {
             return false;
         }
-        if (!setType.equals(((SysVariableDesc) obj).getSetType())) {
+        if (!setType.equals(((VariableExpr) obj).getSetType())) {
             return false;
         }
 
-        return Objects.equals(literalExpr, ((SysVariableDesc) obj).getLiteralExpr());
+        return Objects.equals(literalExpr, ((VariableExpr) obj).getLiteralExpr());
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
index f0cc946500..4cb7423c12 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
@@ -654,7 +654,8 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
 
     @Override
     public Expression visitUserVariable(UserVariableContext ctx) {
-        throw new ParseException("Unsupported user variable :" + ctx.getText(), ctx);
+        String name = ctx.identifierOrText().getText();
+        return VariableMgr.getLiteralForUserVar(name);
     }
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Literal.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Literal.java
index 109357499f..123aa618f4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Literal.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Literal.java
@@ -19,6 +19,7 @@ package org.apache.doris.nereids.trees.expressions.literal;
 
 import org.apache.doris.analysis.LiteralExpr;
 import org.apache.doris.catalog.Type;
+import org.apache.doris.common.Config;
 import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.exceptions.UnboundException;
 import org.apache.doris.nereids.trees.expressions.Expression;
@@ -75,6 +76,12 @@ public abstract class Literal extends Expression implements LeafExpression, Comp
             return new FloatLiteral((Float) value);
         } else if (value instanceof Double) {
             return new DoubleLiteral((Double) value);
+        } else if (value instanceof BigDecimal) {
+            if (Config.enable_decimal_conversion) {
+                return new DecimalV3Literal((BigDecimal) value);
+            } else {
+                return new DecimalLiteral((BigDecimal) value);
+            }
         } else if (value instanceof Boolean) {
             return BooleanLiteral.of((Boolean) value);
         } else if (value instanceof String) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SetExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SetExecutor.java
index c88e7e3336..9dbe9ca8a9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/SetExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SetExecutor.java
@@ -22,6 +22,7 @@ import org.apache.doris.analysis.SetNamesVar;
 import org.apache.doris.analysis.SetPassVar;
 import org.apache.doris.analysis.SetStmt;
 import org.apache.doris.analysis.SetTransaction;
+import org.apache.doris.analysis.SetUserDefinedVar;
 import org.apache.doris.analysis.SetVar;
 import org.apache.doris.common.DdlException;
 
@@ -54,6 +55,8 @@ public class SetExecutor {
         } else if (var instanceof SetTransaction) {
             // do nothing
             return;
+        } else if (var instanceof SetUserDefinedVar) {
+            VariableMgr.setUserVar(var);
         } else {
             VariableMgr.setVar(ctx.getSessionVariable(), var);
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java b/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java
index 5408e74071..24b7468e77 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java
@@ -17,9 +17,16 @@
 
 package org.apache.doris.qe;
 
+import org.apache.doris.analysis.BoolLiteral;
+import org.apache.doris.analysis.DecimalLiteral;
+import org.apache.doris.analysis.FloatLiteral;
+import org.apache.doris.analysis.IntLiteral;
+import org.apache.doris.analysis.LiteralExpr;
+import org.apache.doris.analysis.NullLiteral;
 import org.apache.doris.analysis.SetType;
 import org.apache.doris.analysis.SetVar;
-import org.apache.doris.analysis.SysVariableDesc;
+import org.apache.doris.analysis.StringLiteral;
+import org.apache.doris.analysis.VariableExpr;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
@@ -52,6 +59,7 @@ import java.lang.annotation.RetentionPolicy;
 import java.lang.reflect.Field;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.locks.Lock;
@@ -116,6 +124,8 @@ public class VariableMgr {
     // its display name is "experimental_foo"
     private static ImmutableMap<String, VarContext> ctxByDisplayVarName;
 
+    private static Map<String, LiteralExpr> userVars = new HashMap<String, LiteralExpr>();
+
     // This variable is equivalent to the default value of session variables.
     // Whenever a new session is established, the value in this object is copied to the session-level variable.
     private static SessionVariable defaultSessionVariable;
@@ -258,6 +268,10 @@ public class VariableMgr {
         }
     }
 
+    public static void setUserVar(SetVar setVar) {
+        userVars.put(setVar.getVariable(), setVar.getResult());
+    }
+
     // Entry of handling SetVarStmt
     // Input:
     //      sessionVariable: the variable of current session
@@ -314,8 +328,8 @@ public class VariableMgr {
         VarAttr attr = ctx.getField().getAnnotation(VarAttr.class);
         String value;
         // If value is null, this is `set variable = DEFAULT`
-        if (setVar.getValue() != null) {
-            value = setVar.getValue().getStringValue();
+        if (setVar.getResult() != null) {
+            value = setVar.getResult().getStringValue();
         } else {
             value = ctx.getDefaultValue();
             if (value == null) {
@@ -416,7 +430,7 @@ public class VariableMgr {
     }
 
     // Get variable value through variable name, used to satisfy statement like `SELECT @@comment_version`
-    public static void fillValue(SessionVariable var, SysVariableDesc desc) throws AnalysisException {
+    public static void fillValue(SessionVariable var, VariableExpr desc) throws AnalysisException {
         VarContext ctx = ctxByVarName.get(desc.getName());
         if (ctx == null) {
             ErrorReport.reportAnalysisException(ErrorCode.ERR_UNKNOWN_SYSTEM_VARIABLE, desc.getName());
@@ -434,7 +448,7 @@ public class VariableMgr {
         }
     }
 
-    private static void fillValue(Object obj, Field field, SysVariableDesc desc) {
+    private static void fillValue(Object obj, Field field, VariableExpr desc) {
         try {
             switch (field.getType().getSimpleName()) {
                 case "boolean":
@@ -479,6 +493,36 @@ public class VariableMgr {
         }
     }
 
+    // Get variable value through variable name, used to satisfy statement like `SELECT @@comment_version`
+    public static void fillValueForUserDefinedVar(VariableExpr desc) {
+        String varName = desc.getName();
+        if (userVars.containsKey(varName)) {
+            LiteralExpr literalExpr = userVars.get(varName);
+            desc.setType(literalExpr.getType());
+            if (literalExpr instanceof BoolLiteral) {
+                desc.setBoolValue(((BoolLiteral) literalExpr).getValue());
+            } else if (literalExpr instanceof IntLiteral) {
+                desc.setIntValue(((IntLiteral) literalExpr).getValue());
+            } else if (literalExpr instanceof FloatLiteral) {
+                desc.setFloatValue(((FloatLiteral) literalExpr).getValue());
+            } else if (literalExpr instanceof DecimalLiteral) {
+                desc.setDecimalValue(((DecimalLiteral) literalExpr).getValue());
+            } else if (literalExpr instanceof StringLiteral) {
+                desc.setStringValue(((StringLiteral) literalExpr).getValue());
+            } else if (literalExpr instanceof NullLiteral) {
+                desc.setType(Type.NULL);
+                desc.setIsNull();
+            } else {
+                desc.setType(Type.VARCHAR);
+                desc.setStringValue("");
+            }
+        } else {
+            // If there are no such user defined var, just fill the NULL value.
+            desc.setType(Type.NULL);
+            desc.setIsNull();
+        }
+    }
+
     private static String getValue(SessionVariable var, String name, SetType setType) throws AnalysisException {
         VarContext ctx = ctxByVarName.get(name);
         if (ctx == null) {
@@ -499,7 +543,7 @@ public class VariableMgr {
 
     // Get variable value through variable name, used to satisfy statement like `SELECT @@comment_version`
     // For test only
-    public static String getValue(SessionVariable var, SysVariableDesc desc) throws AnalysisException {
+    public static String getValue(SessionVariable var, VariableExpr desc) throws AnalysisException {
         return getValue(var, desc.getName(), desc.getSetType());
     }
 
@@ -550,6 +594,30 @@ public class VariableMgr {
         return Literal.of("");
     }
 
+    public static @Nullable Literal getLiteralForUserVar(String varName) {
+        if (userVars.containsKey(varName)) {
+            LiteralExpr literalExpr = userVars.get(varName);
+            if (literalExpr instanceof BoolLiteral) {
+                return Literal.of(((BoolLiteral) literalExpr).getValue());
+            } else if (literalExpr instanceof IntLiteral) {
+                return Literal.of(((IntLiteral) literalExpr).getValue());
+            } else if (literalExpr instanceof FloatLiteral) {
+                return Literal.of(((FloatLiteral) literalExpr).getValue());
+            } else if (literalExpr instanceof DecimalLiteral) {
+                return Literal.of(((DecimalLiteral) literalExpr).getValue());
+            } else if (literalExpr instanceof StringLiteral) {
+                return Literal.of(((StringLiteral) literalExpr).getValue());
+            } else if (literalExpr instanceof NullLiteral) {
+                return Literal.of(null);
+            } else {
+                return Literal.of("");
+            }
+        } else {
+            // If there are no such user defined var, just return the NULL value.
+            return Literal.of(null);
+        }
+    }
+
     private static String getValue(Object obj, Field field) {
         try {
             switch (field.getType().getSimpleName()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/rewrite/FoldConstantsRule.java b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FoldConstantsRule.java
index a4baf0ca3e..dcc0bede70 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/rewrite/FoldConstantsRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FoldConstantsRule.java
@@ -29,7 +29,7 @@ import org.apache.doris.analysis.Expr;
 import org.apache.doris.analysis.InformationFunction;
 import org.apache.doris.analysis.LiteralExpr;
 import org.apache.doris.analysis.NullLiteral;
-import org.apache.doris.analysis.SysVariableDesc;
+import org.apache.doris.analysis.VariableExpr;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.ScalarType;
@@ -216,7 +216,7 @@ public class FoldConstantsRule implements ExprRewriteRule {
                 return;
             }
             // collect sysVariableDesc expr
-            if (expr.contains(Predicates.instanceOf(SysVariableDesc.class))) {
+            if (expr.contains(Predicates.instanceOf(VariableExpr.class))) {
                 getSysVarDescExpr(expr, sysVarMap);
                 return;
             }
@@ -244,14 +244,14 @@ public class FoldConstantsRule implements ExprRewriteRule {
     }
 
     private void getSysVarDescExpr(Expr expr, Map<String, Expr> sysVarMap) {
-        if (expr instanceof SysVariableDesc) {
-            Expr literalExpr = ((SysVariableDesc) expr).getLiteralExpr();
+        if (expr instanceof VariableExpr) {
+            Expr literalExpr = ((VariableExpr) expr).getLiteralExpr();
             if (literalExpr == null) {
                 try {
-                    VariableMgr.fillValue(ConnectContext.get().getSessionVariable(), (SysVariableDesc) expr);
-                    literalExpr = ((SysVariableDesc) expr).getLiteralExpr();
+                    VariableMgr.fillValue(ConnectContext.get().getSessionVariable(), (VariableExpr) expr);
+                    literalExpr = ((VariableExpr) expr).getLiteralExpr();
                 } catch (AnalysisException e) {
-                    LOG.warn("failed to get session variable value: " + ((SysVariableDesc) expr).getName());
+                    LOG.warn("failed to get session variable value: " + ((VariableExpr) expr).getName());
                 }
             }
             sysVarMap.put(expr.getId().toString(), literalExpr);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/SetVarTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/SetVarTest.java
index 7b5360c87c..d88538a9a5 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/SetVarTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/SetVarTest.java
@@ -52,7 +52,7 @@ public class SetVarTest {
         var.setType(SetType.GLOBAL);
         Assert.assertEquals(SetType.GLOBAL, var.getType());
         Assert.assertEquals("names", var.getVariable());
-        Assert.assertEquals("utf-8", var.getValue().getStringValue());
+        Assert.assertEquals("utf-8", var.getResult().getStringValue());
 
         Assert.assertEquals("GLOBAL names = 'utf-8'", var.toString());
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/SysVariableDescTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/VariableExprTest.java
similarity index 92%
rename from fe/fe-core/src/test/java/org/apache/doris/analysis/SysVariableDescTest.java
rename to fe/fe-core/src/test/java/org/apache/doris/analysis/VariableExprTest.java
index fcc8b8ddf2..28334f4102 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/SysVariableDescTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/VariableExprTest.java
@@ -26,11 +26,11 @@ import org.apache.doris.thrift.TExprNodeType;
 import org.junit.Assert;
 import org.junit.Test;
 
-public class SysVariableDescTest {
+public class VariableExprTest {
 
     @Test
     public void testNormal() throws AnalysisException {
-        SysVariableDesc desc = new SysVariableDesc("version_comment");
+        VariableExpr desc = new VariableExpr("version_comment");
         desc.analyze(AccessTestUtil.fetchAdminAnalyzer(false));
         Assert.assertEquals("@@version_comment", desc.toSql());
         Assert.assertEquals("version_comment", desc.getName());
@@ -45,7 +45,7 @@ public class SysVariableDescTest {
 
     @Test(expected = AnalysisException.class)
     public void testNoVar() throws AnalysisException {
-        SysVariableDesc desc = new SysVariableDesc("zcPrivate");
+        VariableExpr desc = new VariableExpr("zcPrivate");
         desc.analyze(AccessTestUtil.fetchAdminAnalyzer(false));
         Assert.fail("No exception throws.");
     }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/VariableMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/VariableMgrTest.java
index b8abd92d32..f1b8e7b5a7 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/qe/VariableMgrTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/qe/VariableMgrTest.java
@@ -22,7 +22,7 @@ import org.apache.doris.analysis.SetStmt;
 import org.apache.doris.analysis.SetType;
 import org.apache.doris.analysis.SetVar;
 import org.apache.doris.analysis.StringLiteral;
-import org.apache.doris.analysis.SysVariableDesc;
+import org.apache.doris.analysis.VariableExpr;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
@@ -154,7 +154,7 @@ public class VariableMgrTest {
         Assert.assertEquals(8L, ctx.getSessionVariable().getRuntimeFilterType());
 
         // Get from name
-        SysVariableDesc desc = new SysVariableDesc("exec_mem_limit");
+        VariableExpr desc = new VariableExpr("exec_mem_limit");
         Assert.assertEquals(var.getMaxExecMemByte() + "", VariableMgr.getValue(var, desc));
     }
 
diff --git a/regression-test/data/nereids_p0/test_user_var.out b/regression-test/data/nereids_p0/test_user_var.out
new file mode 100644
index 0000000000..41aebfe022
--- /dev/null
+++ b/regression-test/data/nereids_p0/test_user_var.out
@@ -0,0 +1,16 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select1 --
+1	0	-1
+
+-- !select2 --
+1.1	0.0	-1.1
+
+-- !select3 --
+H	
+
+-- !select4 --
+true	false
+
+-- !select5 --
+\N	\N
+
diff --git a/regression-test/data/query_p0/set/test_user_var.out b/regression-test/data/query_p0/set/test_user_var.out
new file mode 100644
index 0000000000..41aebfe022
--- /dev/null
+++ b/regression-test/data/query_p0/set/test_user_var.out
@@ -0,0 +1,16 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select1 --
+1	0	-1
+
+-- !select2 --
+1.1	0.0	-1.1
+
+-- !select3 --
+H	
+
+-- !select4 --
+true	false
+
+-- !select5 --
+\N	\N
+
diff --git a/regression-test/suites/nereids_p0/test_user_var.groovy b/regression-test/suites/nereids_p0/test_user_var.groovy
new file mode 100644
index 0000000000..f54e139632
--- /dev/null
+++ b/regression-test/suites/nereids_p0/test_user_var.groovy
@@ -0,0 +1,32 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_user_var") {
+    sql "SET enable_nereids_planner=true"
+    sql "SET enable_fallback_to_original_planner=false"
+    sql "SET @a1=1, @a2=0, @a3=-1"
+    sql "SET @b1=1.1, @b2=0.0, @b3=-1.1"
+    sql "SET @c1='H', @c2=''"
+    sql "SET @d1=true, @d2=false"
+    sql "SET @f1=null"
+
+    qt_select1 'select @a1, @a2, @a3;'
+    qt_select2 'select @b1, @b2, @b3;'
+    qt_select3 'select @c1, @c2;'
+    qt_select4 'select @d1, @d2;'
+    qt_select5 'select @f1, @f2;'
+}
\ No newline at end of file
diff --git a/regression-test/suites/query_p0/set/test_user_var.groovy b/regression-test/suites/query_p0/set/test_user_var.groovy
new file mode 100644
index 0000000000..af2d2caadc
--- /dev/null
+++ b/regression-test/suites/query_p0/set/test_user_var.groovy
@@ -0,0 +1,30 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_user_var") {
+    sql "SET @a1=1, @a2=0, @a3=-1"
+    sql "SET @b1=1.1, @b2=0.0, @b3=-1.1"
+    sql "SET @c1='H', @c2=''"
+    sql "SET @d1=true, @d2=false"
+    sql "SET @f1=null"
+
+    qt_select1 'select @a1, @a2, @a3;'
+    qt_select2 'select @b1, @b2, @b3;'
+    qt_select3 'select @c1, @c2;'
+    qt_select4 'select @d1, @d2;'
+    qt_select5 'select @f1, @f2;'
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 10/36: [Feature](Nereids) support update unique table statement (#20313)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit d8c6dfbf09faf350b6f2139688c55fad3f6ff04c
Author: mch_ucchi <41...@users.noreply.github.com>
AuthorDate: Tue Jun 6 20:32:43 2023 +0800

    [Feature](Nereids) support update unique table statement (#20313)
---
 .../antlr4/org/apache/doris/nereids/DorisLexer.g4  |   1 +
 .../antlr4/org/apache/doris/nereids/DorisParser.g4 |  13 ++
 .../doris/nereids/parser/LogicalPlanBuilder.java   |  74 +++++++---
 .../doris/nereids/trees/plans/Explainable.java     |  27 ++++
 .../trees/plans/commands/ExplainCommand.java       |   9 +-
 .../plans/commands/InsertIntoTableCommand.java     |  19 ++-
 .../trees/plans/commands/UpdateCommand.java        | 156 +++++++++++++++++++++
 .../trees/plans/logical/AbstractLogicalPlan.java   |   9 +-
 .../trees/plans/physical/AbstractPhysicalPlan.java |   9 +-
 .../trees/plans/visitor/CommandVisitor.java        |  46 ++++++
 .../nereids/trees/plans/visitor/PlanVisitor.java   |  18 +--
 .../nereids/trees/plans/UpdateCommandTest.java     | 126 +++++++++++++++++
 .../data/nereids_p0/update/update_unique_table.out |   6 +-
 .../suites/nereids_p0/update/load.groovy           |  81 +++++++++++
 .../nereids_p0/update/update_unique_table.groovy   |  39 ++++++
 15 files changed, 586 insertions(+), 47 deletions(-)

diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
index 11a1c88618..3c54093cdd 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
@@ -159,6 +159,7 @@ DATE_ADD: 'DATE_ADD';
 DATE_SUB: 'DATE_SUB';
 DATE_DIFF: 'DATE_DIFF';
 DBPROPERTIES: 'DBPROPERTIES';
+DEFAULT: 'DEFAULT';
 DEFINED: 'DEFINED';
 DELETE: 'DELETE';
 DELIMITED: 'DELIMITED';
diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
index 14ad9c35c4..642b65a182 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
@@ -45,6 +45,10 @@ statement
         (WITH LABEL labelName=identifier)? cols=identifierList?  // label and columns define
         (LEFT_BRACKET hints=identifierSeq RIGHT_BRACKET)?  // hint define
         query                                                          #insertIntoQuery
+    | explain? UPDATE tableName=multipartIdentifier tableAlias
+        SET updateAssignmentSeq
+        fromClause?
+        whereClause                                                    #update
     ;
 
 // -----------------Command accessories-----------------
@@ -184,11 +188,20 @@ hintStatement
 hintAssignment
     : key=identifierOrText (EQ (constantValue=constant | identifierValue=identifier))?
     ;
+    
+updateAssignment
+    : col=multipartIdentifier EQ (expression | DEFAULT)
+    ;
+    
+updateAssignmentSeq
+    : assignments+=updateAssignment (COMMA assignments+=updateAssignment)*
+    ;
 
 lateralView
     : LATERAL VIEW functionName=identifier LEFT_PAREN (expression (COMMA expression)*)? RIGHT_PAREN
       tableName=identifier AS columnName=identifier
     ;
+
 queryOrganization
     : sortClause? limitClause?
     ;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
index 491b1e80f6..f0cc946500 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
@@ -102,6 +102,9 @@ import org.apache.doris.nereids.DorisParser.TvfPropertyContext;
 import org.apache.doris.nereids.DorisParser.TvfPropertyItemContext;
 import org.apache.doris.nereids.DorisParser.TypeConstructorContext;
 import org.apache.doris.nereids.DorisParser.UnitIdentifierContext;
+import org.apache.doris.nereids.DorisParser.UpdateAssignmentContext;
+import org.apache.doris.nereids.DorisParser.UpdateAssignmentSeqContext;
+import org.apache.doris.nereids.DorisParser.UpdateContext;
 import org.apache.doris.nereids.DorisParser.UserIdentifyContext;
 import org.apache.doris.nereids.DorisParser.UserVariableContext;
 import org.apache.doris.nereids.DorisParser.WhereClauseContext;
@@ -210,6 +213,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand;
 import org.apache.doris.nereids.trees.plans.commands.ExplainCommand;
 import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel;
 import org.apache.doris.nereids.trees.plans.commands.InsertIntoTableCommand;
+import org.apache.doris.nereids.trees.plans.commands.UpdateCommand;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
 import org.apache.doris.nereids.trees.plans.logical.LogicalCTE;
 import org.apache.doris.nereids.trees.plans.logical.LogicalCheckPolicy;
@@ -315,6 +319,23 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
         return new InsertIntoTableCommand(sink, labelName);
     }
 
+    @Override
+    public LogicalPlan visitUpdate(UpdateContext ctx) {
+        LogicalPlan query = withCheckPolicy(new UnboundRelation(
+                RelationUtil.newRelationId(), visitMultipartIdentifier(ctx.tableName)));
+        query = withTableAlias(query, ctx.tableAlias());
+        if (ctx.fromClause() != null) {
+            query = withRelations(query, ctx.fromClause());
+        }
+        query = withFilter(query, Optional.of(ctx.whereClause()));
+        String tableAlias = null;
+        if (ctx.tableAlias().strictIdentifier() != null) {
+            tableAlias = ctx.tableAlias().getText();
+        }
+        return withExplain(new UpdateCommand(visitMultipartIdentifier(ctx.tableName), tableAlias,
+                visitUpdateAssignmentSeq(ctx.updateAssignmentSeq()), query), ctx.explain());
+    }
+
     /**
      * Visit multi-statements.
      */
@@ -1268,25 +1289,7 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
 
     @Override
     public LogicalPlan visitFromClause(FromClauseContext ctx) {
-        return ParserUtils.withOrigin(ctx, () -> {
-            LogicalPlan left = null;
-            for (RelationContext relation : ctx.relation()) {
-                // build left deep join tree
-                LogicalPlan right = visitRelation(relation);
-                left = (left == null) ? right :
-                        new LogicalJoin<>(
-                                JoinType.CROSS_JOIN,
-                                ExpressionUtils.EMPTY_CONDITION,
-                                ExpressionUtils.EMPTY_CONDITION,
-                                JoinHint.NONE,
-                                Optional.empty(),
-                                left,
-                                right);
-                left = withJoinRelations(left, relation);
-                // TODO: pivot and lateral view
-            }
-            return left;
-        });
+        return ParserUtils.withOrigin(ctx, () -> withRelations(null, ctx));
     }
 
     /* ********************************************************************************************
@@ -1318,6 +1321,19 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
             .collect(ImmutableList.toImmutableList());
     }
 
+    @Override
+    public EqualTo visitUpdateAssignment(UpdateAssignmentContext ctx) {
+        return new EqualTo(new UnboundSlot(visitMultipartIdentifier(ctx.multipartIdentifier())),
+                getExpression(ctx.expression()));
+    }
+
+    @Override
+    public List<EqualTo> visitUpdateAssignmentSeq(UpdateAssignmentSeqContext ctx) {
+        return ctx.assignments.stream()
+                .map(this::visitUpdateAssignment)
+                .collect(Collectors.toList());
+    }
+
     /**
      * get OrderKey.
      *
@@ -1626,6 +1642,26 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
         });
     }
 
+    private LogicalPlan withRelations(LogicalPlan inputPlan, FromClauseContext ctx) {
+        LogicalPlan left = inputPlan;
+        for (RelationContext relation : ctx.relation()) {
+            // build left deep join tree
+            LogicalPlan right = visitRelation(relation);
+            left = (left == null) ? right :
+                    new LogicalJoin<>(
+                            JoinType.CROSS_JOIN,
+                            ExpressionUtils.EMPTY_CONDITION,
+                            ExpressionUtils.EMPTY_CONDITION,
+                            JoinHint.NONE,
+                            Optional.empty(),
+                            left,
+                            right);
+            left = withJoinRelations(left, relation);
+            // TODO: pivot and lateral view
+        }
+        return left;
+    }
+
     private LogicalPlan withFilter(LogicalPlan input, Optional<WhereClauseContext> whereCtx) {
         return input.optionalMap(whereCtx, () ->
             new LogicalFilter<>(ExpressionUtils.extractConjunctionToSet(
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Explainable.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Explainable.java
new file mode 100644
index 0000000000..46771392e5
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Explainable.java
@@ -0,0 +1,27 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.trees.plans;
+
+import org.apache.doris.qe.ConnectContext;
+
+/**
+ * plan can be explained.
+ */
+public interface Explainable {
+    Plan getExplainPlan(ConnectContext ctx) throws Exception;
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java
index a312c1738c..820e3861d8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java
@@ -18,8 +18,10 @@
 package org.apache.doris.nereids.trees.plans.commands;
 
 import org.apache.doris.analysis.ExplainOptions;
+import org.apache.doris.common.AnalysisException;
 import org.apache.doris.nereids.NereidsPlanner;
 import org.apache.doris.nereids.glue.LogicalPlanAdapter;
+import org.apache.doris.nereids.trees.plans.Explainable;
 import org.apache.doris.nereids.trees.plans.PlanType;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
@@ -65,7 +67,12 @@ public class ExplainCommand extends Command implements NoForward {
 
     @Override
     public void run(ConnectContext ctx, StmtExecutor executor) throws Exception {
-        LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(logicalPlan, ctx.getStatementContext());
+        LogicalPlan explainPlan = null;
+        if (!(logicalPlan instanceof Explainable)) {
+            throw new AnalysisException("explain a plan cannot be explained");
+        }
+        explainPlan = ((LogicalPlan) ((Explainable) logicalPlan).getExplainPlan(ctx));
+        LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(explainPlan, ctx.getStatementContext());
         logicalPlanAdapter.setIsExplain(new ExplainOptions(level));
         executor.setParsedStmt(logicalPlanAdapter);
         NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java
index 291cf1687c..8e7c688597 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java
@@ -24,6 +24,8 @@ import org.apache.doris.nereids.NereidsPlanner;
 import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.glue.LogicalPlanAdapter;
 import org.apache.doris.nereids.trees.TreeNode;
+import org.apache.doris.nereids.trees.plans.Explainable;
+import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.PlanType;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink;
@@ -40,8 +42,10 @@ import com.google.common.base.Strings;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import java.util.Objects;
 import java.util.Optional;
 import java.util.Set;
+import javax.annotation.Nullable;
 
 /**
  * insert into select command implementation
@@ -51,22 +55,22 @@ import java.util.Set;
  *  InsertIntoTableCommand(Query())
  *  ExplainCommand(Query())
  */
-public class InsertIntoTableCommand extends Command implements ForwardWithSync {
+public class InsertIntoTableCommand extends Command implements ForwardWithSync, Explainable {
 
     public static final Logger LOG = LogManager.getLogger(InsertIntoTableCommand.class);
 
     private final LogicalPlan logicalQuery;
-    private final String labelName;
+    private final @Nullable String labelName;
     private NereidsPlanner planner;
     private boolean isTxnBegin = false;
 
     /**
      * constructor
      */
-    public InsertIntoTableCommand(LogicalPlan logicalQuery, String labelName) {
+    public InsertIntoTableCommand(LogicalPlan logicalQuery, @Nullable String labelName) {
         super(PlanType.INSERT_INTO_TABLE_COMMAND);
-        Preconditions.checkNotNull(logicalQuery, "logicalQuery cannot be null in InsertIntoTableCommand");
-        this.logicalQuery = logicalQuery;
+        this.logicalQuery = Objects.requireNonNull(logicalQuery,
+                "logicalQuery cannot be null in InsertIntoTableCommand");
         this.labelName = labelName;
     }
 
@@ -150,6 +154,11 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync {
         }
     }
 
+    @Override
+    public Plan getExplainPlan(ConnectContext ctx) {
+        return this.logicalQuery;
+    }
+
     @Override
     public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
         return visitor.visitInsertIntoCommand(this, context);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java
new file mode 100644
index 0000000000..a582c22394
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java
@@ -0,0 +1,156 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.trees.plans.commands;
+
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.catalog.TableIf;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.nereids.analyzer.UnboundOlapTableSink;
+import org.apache.doris.nereids.analyzer.UnboundSlot;
+import org.apache.doris.nereids.trees.expressions.Alias;
+import org.apache.doris.nereids.trees.expressions.EqualTo;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Explainable;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.nereids.util.RelationUtil;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.SessionVariable;
+import org.apache.doris.qe.StmtExecutor;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.annotation.Nullable;
+
+/**
+ * update command
+ * the two case will be handled as:
+ * case 1:
+ *  update table t1 set v1 = v1 + 1 where k1 = 1 and k2 = 2;
+ * =>
+ *  insert into table (v1) select v1 + 1 from table t1 where k1 = 1 and k2 = 2
+ * case 2:
+ *  update t1 set t1.c1 = t2.c1, t1.c3 = t2.c3 * 100
+ *  from t2 inner join t3 on t2.id = t3.id
+ *  where t1.id = t2.id;
+ * =>
+ *  insert into t1 (c1, c3) select t2.c1, t2.c3 * 100 from t1 join t2 inner join t3 on t2.id = t3.id where t1.id = t2.id
+ */
+public class UpdateCommand extends Command implements ForwardWithSync, Explainable {
+    private final List<EqualTo> assignments;
+    private final List<String> nameParts;
+    private final @Nullable String tableAlias;
+    private final LogicalPlan logicalQuery;
+    private OlapTable targetTable;
+
+    /**
+     * constructor
+     */
+    public UpdateCommand(List<String> nameParts, @Nullable String tableAlias, List<EqualTo> assignments,
+            LogicalPlan logicalQuery) {
+        super(PlanType.UPDATE_COMMAND);
+        this.nameParts = ImmutableList.copyOf(Objects.requireNonNull(nameParts,
+                "tableName is required in update command"));
+        this.assignments = ImmutableList.copyOf(Objects.requireNonNull(assignments,
+                "assignment is required in update command"));
+        this.tableAlias = tableAlias;
+        this.logicalQuery = Objects.requireNonNull(logicalQuery, "logicalQuery is required in update command");
+    }
+
+    @Override
+    public void run(ConnectContext ctx, StmtExecutor executor) throws Exception {
+        new InsertIntoTableCommand(completeQueryPlan(ctx, logicalQuery), null).run(ctx, executor);
+    }
+
+    /**
+     * add LogicalOlapTableSink node, public for test.
+     */
+    public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuery) throws AnalysisException {
+        checkTable(ctx);
+
+        Map<String, Expression> colNameToExpression = Maps.newHashMap();
+        for (EqualTo equalTo : assignments) {
+            List<String> nameParts = ((UnboundSlot) equalTo.left()).getNameParts();
+            colNameToExpression.put(nameParts.get(nameParts.size() - 1), equalTo.right());
+        }
+        List<NamedExpression> selectItems = Lists.newArrayList();
+        String tableName = tableAlias != null ? tableAlias : targetTable.getName();
+        for (Column column : targetTable.getFullSchema()) {
+            if (!column.isVisible()) {
+                continue;
+            }
+            if (colNameToExpression.containsKey(column.getName())) {
+                Expression expr = colNameToExpression.get(column.getName());
+                selectItems.add(expr instanceof UnboundSlot
+                        ? ((NamedExpression) expr)
+                        : new Alias(expr, expr.toSql()));
+            } else {
+                selectItems.add(new UnboundSlot(tableName, column.getName()));
+            }
+        }
+
+        logicalQuery = new LogicalProject<>(selectItems, logicalQuery);
+
+        // make UnboundTableSink
+        return new UnboundOlapTableSink<>(nameParts, null, null, null, logicalQuery);
+    }
+
+    private void checkTable(ConnectContext ctx) throws AnalysisException {
+        if (ctx.getSessionVariable().isInDebugMode()) {
+            throw new AnalysisException("Update is forbidden since current session is in debug mode."
+                    + " Please check the following session variables: "
+                    + String.join(", ", SessionVariable.DEBUG_VARIABLES));
+        }
+        List<String> tableQualifier = RelationUtil.getQualifierName(ctx, nameParts);
+        TableIf table = RelationUtil.getTable(tableQualifier, ctx.getEnv());
+        if (!(table instanceof OlapTable)) {
+            throw new AnalysisException("target table in update command should be an olapTable");
+        }
+        targetTable = ((OlapTable) table);
+        if (targetTable.getType() != Table.TableType.OLAP
+                || targetTable.getKeysType() != KeysType.UNIQUE_KEYS) {
+            throw new AnalysisException("Only unique table could be updated.");
+        }
+    }
+
+    @Override
+    public Plan getExplainPlan(ConnectContext ctx) throws AnalysisException {
+        return completeQueryPlan(ctx, logicalQuery);
+    }
+
+    public LogicalPlan getLogicalQuery() {
+        return logicalQuery;
+    }
+
+    @Override
+    public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
+        return visitor.visitUpdateCommand(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java
index 15ea90e909..d8fb322073 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java
@@ -21,8 +21,10 @@ import org.apache.doris.nereids.memo.GroupExpression;
 import org.apache.doris.nereids.properties.LogicalProperties;
 import org.apache.doris.nereids.properties.UnboundLogicalProperties;
 import org.apache.doris.nereids.trees.plans.AbstractPlan;
+import org.apache.doris.nereids.trees.plans.Explainable;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.qe.ConnectContext;
 
 import java.util.Optional;
 import java.util.function.Supplier;
@@ -30,7 +32,7 @@ import java.util.function.Supplier;
 /**
  * Abstract class for all concrete logical plan.
  */
-public abstract class AbstractLogicalPlan extends AbstractPlan implements LogicalPlan {
+public abstract class AbstractLogicalPlan extends AbstractPlan implements LogicalPlan, Explainable {
 
     private Supplier<Boolean> hasUnboundExpressions = () -> super.hasUnboundExpression();
 
@@ -63,4 +65,9 @@ public abstract class AbstractLogicalPlan extends AbstractPlan implements Logica
             return new LogicalProperties(this::computeOutput);
         }
     }
+
+    @Override
+    public Plan getExplainPlan(ConnectContext ctx) {
+        return this;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java
index 2cdab8c32b..14b989f414 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java
@@ -21,8 +21,10 @@ import org.apache.doris.nereids.memo.GroupExpression;
 import org.apache.doris.nereids.properties.LogicalProperties;
 import org.apache.doris.nereids.properties.PhysicalProperties;
 import org.apache.doris.nereids.trees.plans.AbstractPlan;
+import org.apache.doris.nereids.trees.plans.Explainable;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.statistics.Statistics;
 
 import java.util.Optional;
@@ -31,7 +33,7 @@ import javax.annotation.Nullable;
 /**
  * Abstract class for all concrete physical plan.
  */
-public abstract class AbstractPhysicalPlan extends AbstractPlan implements PhysicalPlan {
+public abstract class AbstractPhysicalPlan extends AbstractPlan implements PhysicalPlan, Explainable {
 
     protected final PhysicalProperties physicalProperties;
 
@@ -54,4 +56,9 @@ public abstract class AbstractPhysicalPlan extends AbstractPlan implements Physi
     public PhysicalProperties getPhysicalProperties() {
         return physicalProperties;
     }
+
+    @Override
+    public Plan getExplainPlan(ConnectContext ctx) {
+        return this;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java
new file mode 100644
index 0000000000..5adab2cc96
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java
@@ -0,0 +1,46 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.trees.plans.visitor;
+
+import org.apache.doris.nereids.trees.plans.commands.Command;
+import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand;
+import org.apache.doris.nereids.trees.plans.commands.ExplainCommand;
+import org.apache.doris.nereids.trees.plans.commands.InsertIntoTableCommand;
+import org.apache.doris.nereids.trees.plans.commands.UpdateCommand;
+
+/** CommandVisitor. */
+public interface CommandVisitor<R, C> {
+    R visitCommand(Command command, C context);
+
+    default R visitExplainCommand(ExplainCommand explain, C context) {
+        return visitCommand(explain, context);
+    }
+
+    default R visitCreatePolicyCommand(CreatePolicyCommand createPolicy, C context) {
+        return visitCommand(createPolicy, context);
+    }
+
+    default R visitInsertIntoCommand(InsertIntoTableCommand insertIntoSelectCommand,
+            C context) {
+        return visitCommand(insertIntoSelectCommand, context);
+    }
+
+    default R visitUpdateCommand(UpdateCommand updateCommand, C context) {
+        return visitCommand(updateCommand, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java
index 4b9e480453..799988c449 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java
@@ -24,9 +24,6 @@ import org.apache.doris.nereids.analyzer.UnboundTVFRelation;
 import org.apache.doris.nereids.trees.plans.GroupPlan;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.commands.Command;
-import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand;
-import org.apache.doris.nereids.trees.plans.commands.ExplainCommand;
-import org.apache.doris.nereids.trees.plans.commands.InsertIntoTableCommand;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
 import org.apache.doris.nereids.trees.plans.logical.LogicalApply;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAssertNumRows;
@@ -103,7 +100,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalWindow;
  * @param <R> Return type of each visit method.
  * @param <C> Context type.
  */
-public abstract class PlanVisitor<R, C> {
+public abstract class PlanVisitor<R, C> implements CommandVisitor<R, C> {
 
     public abstract R visit(Plan plan, C context);
 
@@ -115,19 +112,6 @@ public abstract class PlanVisitor<R, C> {
         return visit(command, context);
     }
 
-    public R visitExplainCommand(ExplainCommand explain, C context) {
-        return visitCommand(explain, context);
-    }
-
-    public R visitCreatePolicyCommand(CreatePolicyCommand createPolicy, C context) {
-        return visitCommand(createPolicy, context);
-    }
-
-    public R visitInsertIntoCommand(InsertIntoTableCommand insertIntoSelectCommand,
-            C context) {
-        return visit(insertIntoSelectCommand, context);
-    }
-
     // *******************************
     // Logical plans
     // *******************************
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/UpdateCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/UpdateCommandTest.java
new file mode 100644
index 0000000000..440d5345f6
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/UpdateCommandTest.java
@@ -0,0 +1,126 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.trees.plans;
+
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.nereids.parser.NereidsParser;
+import org.apache.doris.nereids.trees.plans.commands.UpdateCommand;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.util.PlanChecker;
+import org.apache.doris.nereids.util.PlanPatternMatchSupported;
+import org.apache.doris.utframe.TestWithFeService;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+public class UpdateCommandTest extends TestWithFeService implements PlanPatternMatchSupported {
+    @Override
+    public void runBeforeAll() throws Exception {
+        createDatabase("test");
+        connectContext.setDatabase("default_cluster:test");
+        createTable("create table t1 (\n"
+                + "    k1 int,\n"
+                + "    k2 int,\n"
+                + "    v1 int,\n"
+                + "    v2 int\n"
+                + ")\n"
+                + "unique key(k1, k2)\n"
+                + "distributed by hash(k1) buckets 4\n"
+                + "properties(\n"
+                + "    \"replication_num\"=\"1\"\n"
+                + ")");
+        createTable("create table t2 (\n"
+                + "    k1 int,\n"
+                + "    k2 int,\n"
+                + "    v1 int,\n"
+                + "    v2 int\n"
+                + ")\n"
+                + "unique key(k1, k2)\n"
+                + "distributed by hash(k1) buckets 4\n"
+                + "properties(\n"
+                + "    \"replication_num\"=\"1\"\n"
+                + ")");
+        createTable("create table src (\n"
+                + "    k1 int,\n"
+                + "    k2 int,\n"
+                + "    v1 int,\n"
+                + "    v2 int\n"
+                + ")\n"
+                + "duplicate key(k1, k2)\n"
+                + "distributed by hash(k1) buckets 4\n"
+                + "properties(\n"
+                + "    \"replication_num\"=\"1\"\n"
+                + ")");
+    }
+
+    @Test
+    public void testSimpleUpdate() throws AnalysisException {
+        String sql = "update t1 set v1 = v1 + 2, v2 = v1 * 2 where k1 = 3";
+        LogicalPlan parsed = new NereidsParser().parseSingle(sql);
+        Assertions.assertTrue(parsed instanceof UpdateCommand);
+        UpdateCommand command = ((UpdateCommand) parsed);
+        LogicalPlan plan = command.completeQueryPlan(connectContext, command.getLogicalQuery());
+        PlanChecker.from(connectContext, plan)
+                .analyze(plan)
+                .rewrite()
+                .matches(
+                        logicalOlapTableSink(
+                                logicalProject(
+                                        logicalFilter(
+                                                logicalOlapScan()
+                                        )
+                                )
+                        )
+                );
+    }
+
+    @Test
+    public void testFromClauseUpdate() throws AnalysisException {
+        String sql = "update t1 a set v1 = t2.v1 + 2, v2 = a.v1 * 2 "
+                + "from src join t2 on src.k1 = t2.k1 where t2.k1 = a.k1";
+        LogicalPlan parsed = new NereidsParser().parseSingle(sql);
+        Assertions.assertTrue(parsed instanceof UpdateCommand);
+        UpdateCommand command = ((UpdateCommand) parsed);
+        LogicalPlan plan = command.completeQueryPlan(connectContext, command.getLogicalQuery());
+        PlanChecker.from(connectContext, plan)
+                .analyze(plan)
+                .rewrite()
+                .matches(
+                        logicalOlapTableSink(
+                                logicalProject(
+                                        logicalJoin(
+                                                logicalJoin(
+                                                        logicalProject(
+                                                                logicalFilter(
+                                                                        logicalOlapScan()
+                                                                )
+                                                        ),
+                                                        logicalProject(
+                                                                logicalOlapScan())
+                                                ),
+                                                logicalProject(
+                                                        logicalFilter(
+                                                                logicalOlapScan()
+                                                        )
+                                                )
+                                        )
+                                )
+                        )
+                );
+    }
+}
diff --git a/regression-test/data/nereids_p0/update/update_unique_table.out b/regression-test/data/nereids_p0/update/update_unique_table.out
index 9c04fa150f..24a7d0462f 100644
--- a/regression-test/data/nereids_p0/update/update_unique_table.out
+++ b/regression-test/data/nereids_p0/update/update_unique_table.out
@@ -2,15 +2,15 @@
 -- !sql --
 1	10	1	1	1.0	2000-01-01
 2	20	2	2	2.0	2000-01-02
-3	30	3	3	3.0	2000-01-03
+3	30	5	3	3.0	2000-01-03
 
 -- !sql --
 1	10	2	1	2.0	2000-01-01
 2	20	2	2	2.0	2000-01-02
-3	30	3	3	3.0	2000-01-03
+3	30	5	3	3.0	2000-01-03
 
 -- !sql --
 1	10	10	1	1000.0	2000-01-01
 2	20	2	2	2.0	2000-01-02
-3	30	3	3	3.0	2000-01-03
+3	30	5	3	3.0	2000-01-03
 
diff --git a/regression-test/suites/nereids_p0/update/load.groovy b/regression-test/suites/nereids_p0/update/load.groovy
new file mode 100644
index 0000000000..16f7ba5a07
--- /dev/null
+++ b/regression-test/suites/nereids_p0/update/load.groovy
@@ -0,0 +1,81 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("load") {
+    sql '''
+        create table t1 (
+            id int,
+            id1 int,
+            c1 bigint,
+            c2 string,
+            c3 double,
+            c4 date
+        ) unique key (id, id1)
+        distributed by hash(id, id1)
+        properties(
+            'replication_num'='1',
+            "function_column.sequence_col" = "c4"
+        );
+    '''
+
+    sql '''
+        create table t2 (
+            id int,
+            c1 bigint,
+            c2 string,
+            c3 double,
+            c4 date
+        ) unique key (id)
+        distributed by hash(id)
+        properties(
+            'replication_num'='1'
+        );
+    '''
+
+    sql '''
+        create table t3 (
+            id int
+        ) distributed by hash(id)
+        properties(
+            'replication_num'='1'
+        );
+    '''
+
+    sql '''
+        INSERT INTO t1 VALUES
+            (1, 10, 1, '1', 1.0, '2000-01-01'),
+            (2, 20, 2, '2', 2.0, '2000-01-02'),
+            (3, 30, 3, '3', 3.0, '2000-01-03');
+    '''
+
+    sql '''
+
+        INSERT INTO t2 VALUES
+            (1, 10, '10', 10.0, '2000-01-10'),
+            (2, 20, '20', 20.0, '2000-01-20'),
+            (3, 30, '30', 30.0, '2000-01-30'),
+            (4, 4, '4', 4.0, '2000-01-04'),
+            (5, 5, '5', 5.0, '2000-01-05');
+    '''
+
+    sql '''
+        INSERT INTO t3 VALUES
+            (1),
+            (4),
+            (5);
+    '''
+}
diff --git a/regression-test/suites/nereids_p0/update/update_unique_table.groovy b/regression-test/suites/nereids_p0/update/update_unique_table.groovy
new file mode 100644
index 0000000000..358a3bb9cf
--- /dev/null
+++ b/regression-test/suites/nereids_p0/update/update_unique_table.groovy
@@ -0,0 +1,39 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite('update_unique_table') {
+    sql 'set enable_nereids_planner=true'
+    sql 'set enable_fallback_to_original_planner=false'
+    sql 'set enable_nereids_dml=true'
+
+    sql 'update t1 set c1 = 5 where id = 3'
+    
+    qt_sql 'select * from t1 order by id'
+
+    sql 'update t1 set c1 = c1 + 1, c3 = c2 * 2 where id = 1'
+
+    qt_sql 'select * from t1 order by id'
+
+    sql '''
+        update t1
+        set t1.c1 = t2.c1, t1.c3 = t2.c3 * 100
+        from t2 inner join t3 on t2.id = t3.id
+        where t1.id = t2.id;
+    '''
+
+    qt_sql 'select * from t1 order by id'
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 06/36: [fix](nereids) select with specified partition name is not work as expected (#20269)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 5c4c653aac2e0abb8427266ab11010a883a31755
Author: AKIRA <33...@users.noreply.github.com>
AuthorDate: Mon Jun 5 13:48:54 2023 +0900

    [fix](nereids) select with specified partition name is not work as expected (#20269)
    
    This PR is to fix the select specific partition issue, certain codes related to this feature were accidentally deleted.
---
 .../rewrite/logical/PruneOlapScanPartition.java    | 15 ++++++++----
 .../data/nereids_syntax_p0/select_partition.out    |  3 +++
 .../nereids_syntax_p0/select_partition.groovy      | 28 ++++++++++++++++++++++
 3 files changed, 42 insertions(+), 4 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PruneOlapScanPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PruneOlapScanPartition.java
index 96c7dc94c2..a148273991 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PruneOlapScanPartition.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PruneOlapScanPartition.java
@@ -28,6 +28,10 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
 import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
 import org.apache.doris.nereids.util.Utils;
 
+import com.google.common.collect.ImmutableList;
+import org.apache.commons.collections.CollectionUtils;
+
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -61,10 +65,13 @@ public class PruneOlapScanPartition extends OneRewriteRuleFactory {
                     .map(column -> scanOutput.get(column.getName().toLowerCase()))
                     .collect(Collectors.toList());
 
-            List<Long> prunedPartitions = PartitionPruner.prune(
-                    partitionSlots, filter.getPredicate(), partitionInfo, ctx.cascadesContext);
-
-            LogicalOlapScan rewrittenScan = scan.withSelectedPartitionIds(prunedPartitions);
+            List<Long> prunedPartitions = new ArrayList<>(PartitionPruner.prune(
+                    partitionSlots, filter.getPredicate(), partitionInfo, ctx.cascadesContext));
+            List<Long> manuallySpecifiedPartitions = scan.getManuallySpecifiedPartitions();
+            if (!CollectionUtils.isEmpty(manuallySpecifiedPartitions)) {
+                prunedPartitions.retainAll(manuallySpecifiedPartitions);
+            }
+            LogicalOlapScan rewrittenScan = scan.withSelectedPartitionIds(ImmutableList.copyOf(prunedPartitions));
             return new LogicalFilter<>(filter.getConjuncts(), rewrittenScan);
         }).toRule(RuleType.OLAP_SCAN_PARTITION_PRUNE);
     }
diff --git a/regression-test/data/nereids_syntax_p0/select_partition.out b/regression-test/data/nereids_syntax_p0/select_partition.out
index 5383ef8afb..db982e24c0 100644
--- a/regression-test/data/nereids_syntax_p0/select_partition.out
+++ b/regression-test/data/nereids_syntax_p0/select_partition.out
@@ -21,3 +21,6 @@
 -- !sql --
 16	1234	t
 
+-- !sql --
+1	aaa	aaa
+
diff --git a/regression-test/suites/nereids_syntax_p0/select_partition.groovy b/regression-test/suites/nereids_syntax_p0/select_partition.groovy
index 4a61471d2b..de90c7d5c2 100644
--- a/regression-test/suites/nereids_syntax_p0/select_partition.groovy
+++ b/regression-test/suites/nereids_syntax_p0/select_partition.groovy
@@ -60,4 +60,32 @@ suite("query_on_specific_partition") {
     qt_sql """select * from t_p temporary partitions(tp1);"""
 
     qt_sql """select * from t_p temporary partition tp1;"""
+
+    sql """
+        CREATE TABLE IF NOT EXISTS test_iot (
+                `test_int` int NOT NULL,
+                `test_varchar` varchar(150) NULL,
+        `test_text` text NULL
+        ) ENGINE=OLAP
+        UNIQUE KEY(`test_int`)
+        PARTITION BY LIST (`test_int`)
+        (
+                PARTITION p1 VALUES IN ("1","2","3"),
+                        PARTITION p2 VALUES IN ("4","5","6")
+        )
+        DISTRIBUTED BY HASH(`test_int`) BUCKETS 3
+        PROPERTIES (
+                "replication_allocation" = "tag.location.default: 1",
+                "in_memory" = "false",
+                "storage_format" = "V2"
+        )
+    """
+
+    sql """
+        INSERT INTO test_iot VALUES(1,'aaa','aaa'),(4,'ccc','ccc');
+    """
+
+    qt_sql """
+        SELECT * FROM test_iot PARTITION p1;
+    """
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 17/36: [fix](load) in strict mode, return error for insert if datatype convert fails (#20378)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit b31747a37dc53a4157067a1361a5a727db095d52
Author: TengJianPing <18...@users.noreply.github.com>
AuthorDate: Tue Jun 6 12:04:03 2023 +0800

    [fix](load) in strict mode, return error for insert if datatype convert fails (#20378)
    
    * [fix](load) in strict mode, return error for load and insert if datatype convert fails
    
    Revert "[fix](MySQL) the way Doris handles boolean type is consistent with MySQL (#19416)"
    
    This reverts commit 68eb420cabe5b26b09d6d4a2724ae12699bdee87.
    
    Since it changed other behaviours, e.g. in strict mode insert into t_int values ("a"),
    it will result 0 is inserted into table, but it should return error instead.
    
    * fix be ut
    
    * fix regression tests
---
 be/src/pipeline/pipeline_task.cpp                  |   4 +-
 be/src/runtime/runtime_state.h                     |   4 +
 be/src/vec/functions/function_cast.h               | 101 ++++++----
 .../org/apache/doris/analysis/BinaryPredicate.java |   4 -
 .../org/apache/doris/analysis/StringLiteral.java   |   9 +-
 .../java/org/apache/doris/qe/SessionVariable.java  |   2 +
 gensrc/thrift/PaloInternalService.thrift           |   2 +
 regression-test/data/insert_p0/insert_invalid.out  |  11 ++
 .../datetime_functions/test_date_function.out      |   6 +-
 .../datetime_functions/test_date_function.out      |   6 +-
 .../storage/test_dup_tab_date_nullable.groovy      |   2 +-
 .../suites/insert_p0/insert_invalid.groovy         | 215 +++++++++++++++++++++
 .../datetime_functions/test_date_function.groovy   |   6 +-
 .../datetime_functions/test_date_function.groovy   |   6 +-
 14 files changed, 314 insertions(+), 64 deletions(-)

diff --git a/be/src/pipeline/pipeline_task.cpp b/be/src/pipeline/pipeline_task.cpp
index 853cd8ec0d..7c2379796a 100644
--- a/be/src/pipeline/pipeline_task.cpp
+++ b/be/src/pipeline/pipeline_task.cpp
@@ -219,11 +219,13 @@ Status PipelineTask::execute(bool* eos) {
         if (_block->rows() != 0 || *eos) {
             SCOPED_TIMER(_sink_timer);
             auto status = _sink->sink(_state, block, _data_state);
+            if (!status.is<ErrorCode::END_OF_FILE>()) {
+                RETURN_IF_ERROR(status);
+            }
             *eos = status.is<ErrorCode::END_OF_FILE>() ? true : *eos;
             if (*eos) { // just return, the scheduler will do finish work
                 break;
             }
-            RETURN_IF_ERROR(status);
         }
     }
 
diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h
index 6394a78a82..53382c925a 100644
--- a/be/src/runtime/runtime_state.h
+++ b/be/src/runtime/runtime_state.h
@@ -409,6 +409,10 @@ public:
                        : 0;
     }
 
+    bool enable_insert_strict() const {
+        return _query_options.__isset.enable_insert_strict && _query_options.enable_insert_strict;
+    }
+
 private:
     Status create_error_log_file();
 
diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h
index 90bafadf96..a716a01a75 100644
--- a/be/src/vec/functions/function_cast.h
+++ b/be/src/vec/functions/function_cast.h
@@ -41,6 +41,7 @@
 // IWYU pragma: no_include <opentelemetry/common/threadlocal.h>
 #include "common/compiler_util.h" // IWYU pragma: keep
 #include "common/status.h"
+#include "runtime/runtime_state.h"
 #include "udf/udf.h"
 #include "util/jsonb_document.h"
 #include "util/jsonb_stream.h"
@@ -82,6 +83,7 @@
 #include "vec/io/io_helper.h"
 #include "vec/io/reader_buffer.h"
 #include "vec/runtime/vdatetime_value.h"
+#include "vec/utils/template_helpers.hpp"
 
 class DateLUTImpl;
 
@@ -220,8 +222,9 @@ struct ConvertImpl {
     using ToFieldType = typename ToDataType::FieldType;
 
     template <typename Additions = void*>
-    static Status execute(Block& block, const ColumnNumbers& arguments, size_t result,
-                          size_t /*input_rows_count*/, bool check_overflow [[maybe_unused]] = false,
+    static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                          size_t result, size_t /*input_rows_count*/,
+                          bool check_overflow [[maybe_unused]] = false,
                           Additions additions [[maybe_unused]] = Additions()) {
         const ColumnWithTypeAndName& named_from = block.get_by_position(arguments[0]);
 
@@ -414,8 +417,8 @@ struct ConvertImpl {
   */
 template <typename T, typename Name>
 struct ConvertImpl<std::enable_if_t<!T::is_parametric, T>, T, Name> {
-    static Status execute(Block& block, const ColumnNumbers& arguments, size_t result,
-                          size_t /*input_rows_count*/) {
+    static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                          size_t result, size_t /*input_rows_count*/) {
         block.get_by_position(result).column = block.get_by_position(arguments[0]).column;
         return Status::OK();
     }
@@ -751,8 +754,9 @@ template <typename ToDataType, typename Name>
 struct ConvertImpl<DataTypeString, ToDataType, Name> {
     template <typename Additions = void*>
 
-    static Status execute(Block& block, const ColumnNumbers& arguments, size_t result,
-                          size_t /*input_rows_count*/, bool check_overflow [[maybe_unused]] = false,
+    static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                          size_t result, size_t /*input_rows_count*/,
+                          bool check_overflow [[maybe_unused]] = false,
                           Additions additions [[maybe_unused]] = Additions()) {
         return Status::RuntimeError("not support convert from string");
     }
@@ -1073,18 +1077,18 @@ public:
                     UInt32 scale = extract_to_decimal_scale(scale_column);
 
                     ret_status = ConvertImpl<LeftDataType, RightDataType, Name>::execute(
-                            block, arguments, result, input_rows_count,
+                            context, block, arguments, result, input_rows_count,
                             context->check_overflow_for_decimal(), scale);
                 } else if constexpr (IsDataTypeDateTimeV2<RightDataType>) {
                     const ColumnWithTypeAndName& scale_column = block.get_by_position(result);
                     auto type =
                             check_and_get_data_type<DataTypeDateTimeV2>(scale_column.type.get());
                     ret_status = ConvertImpl<LeftDataType, RightDataType, Name>::execute(
-                            block, arguments, result, input_rows_count,
+                            context, block, arguments, result, input_rows_count,
                             context->check_overflow_for_decimal(), type->get_scale());
                 } else {
                     ret_status = ConvertImpl<LeftDataType, RightDataType, Name>::execute(
-                            block, arguments, result, input_rows_count);
+                            context, block, arguments, result, input_rows_count);
                 }
                 return true;
             };
@@ -1268,8 +1272,9 @@ struct ConvertThroughParsing {
     static bool is_all_read(ReadBuffer& in) { return in.eof(); }
 
     template <typename Additions = void*>
-    static Status execute(Block& block, const ColumnNumbers& arguments, size_t result,
-                          size_t input_rows_count, bool check_overflow [[maybe_unused]] = false,
+    static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                          size_t result, size_t input_rows_count,
+                          bool check_overflow [[maybe_unused]] = false,
                           Additions additions [[maybe_unused]] = Additions()) {
         using ColVecTo = std::conditional_t<IsDecimalNumber<ToFieldType>,
                                             ColumnDecimal<ToFieldType>, ColumnVector<ToFieldType>>;
@@ -1311,34 +1316,54 @@ struct ConvertThroughParsing {
             offsets = &col_from_string->get_offsets();
         }
 
+        bool is_load = (context && context->state()->query_type() == TQueryType::type::LOAD);
+        bool is_strict_insert = (context && context->state()->enable_insert_strict());
         size_t current_offset = 0;
-        for (size_t i = 0; i < size; ++i) {
-            size_t next_offset = std::is_same_v<FromDataType, DataTypeString>
-                                         ? (*offsets)[i]
-                                         : (current_offset + fixed_string_size);
-            size_t string_size = std::is_same_v<FromDataType, DataTypeString>
-                                         ? next_offset - current_offset
-                                         : fixed_string_size;
-
-            ReadBuffer read_buffer(&(*chars)[current_offset], string_size);
+        auto status = std::visit(
+                [&](auto is_load_, auto is_strict_insert_) {
+                    for (size_t i = 0; i < size; ++i) {
+                        size_t next_offset = std::is_same_v<FromDataType, DataTypeString>
+                                                     ? (*offsets)[i]
+                                                     : (current_offset + fixed_string_size);
+                        size_t string_size = std::is_same_v<FromDataType, DataTypeString>
+                                                     ? next_offset - current_offset
+                                                     : fixed_string_size;
+
+                        ReadBuffer read_buffer(&(*chars)[current_offset], string_size);
+
+                        bool parsed;
+                        if constexpr (IsDataTypeDecimal<ToDataType>) {
+                            parsed = try_parse_impl<ToDataType>(
+                                    vec_to[i], read_buffer, local_time_zone, vec_to.get_scale());
+                        } else if constexpr (IsDataTypeDateTimeV2<ToDataType>) {
+                            auto type = check_and_get_data_type<DataTypeDateTimeV2>(
+                                    block.get_by_position(result).type.get());
+                            parsed = try_parse_impl<ToDataType>(vec_to[i], read_buffer,
+                                                                local_time_zone, type->get_scale());
+                        } else {
+                            parsed = try_parse_impl<ToDataType, void*, FromDataType>(
+                                    vec_to[i], read_buffer, local_time_zone);
+                        }
+                        (*vec_null_map_to)[i] = !parsed || !is_all_read(read_buffer);
+                        if constexpr (is_load_ && is_strict_insert_) {
+                            if (string_size != 0 && (*vec_null_map_to)[i]) {
+                                return Status::InternalError(
+                                        "Invalid value in strict mode for function {}, source "
+                                        "column {}, from "
+                                        "type "
+                                        "{} to type {}",
+                                        Name::name, col_from->get_name(), FromDataType().get_name(),
+                                        ToDataType().get_name());
+                            }
+                        }
 
-            bool parsed;
-            if constexpr (IsDataTypeDecimal<ToDataType>) {
-                parsed = try_parse_impl<ToDataType>(vec_to[i], read_buffer, local_time_zone,
-                                                    vec_to.get_scale());
-            } else if constexpr (IsDataTypeDateTimeV2<ToDataType>) {
-                auto type = check_and_get_data_type<DataTypeDateTimeV2>(
-                        block.get_by_position(result).type.get());
-                parsed = try_parse_impl<ToDataType>(vec_to[i], read_buffer, local_time_zone,
-                                                    type->get_scale());
-            } else {
-                parsed = try_parse_impl<ToDataType, void*, FromDataType>(vec_to[i], read_buffer,
-                                                                         local_time_zone);
-            }
-            (*vec_null_map_to)[i] = !parsed || !is_all_read(read_buffer);
+                        current_offset = next_offset;
+                    }
+                    return Status::OK();
+                },
+                make_bool_variant(is_load), make_bool_variant(is_strict_insert));
 
-            current_offset = next_offset;
-        }
+        RETURN_IF_ERROR(status);
 
         block.get_by_position(result).column =
                 ColumnNullable::create(std::move(col_to), std::move(col_null_map_to));
@@ -1390,7 +1415,7 @@ public:
 
         if (check_and_get_data_type<DataTypeString>(from_type)) {
             return ConvertThroughParsing<DataTypeString, ToDataType, Name>::execute(
-                    block, arguments, result, input_rows_count);
+                    context, block, arguments, result, input_rows_count);
         }
 
         return Status::RuntimeError(
@@ -1563,7 +1588,7 @@ private:
                         using RightDataType = typename Types::RightType;
 
                         ConvertImpl<LeftDataType, RightDataType, NameCast>::execute(
-                                block, arguments, result, input_rows_count,
+                                context, block, arguments, result, input_rows_count,
                                 context->check_overflow_for_decimal(), scale);
                         return true;
                     });
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java
index 64f802efaf..8587d801e8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java
@@ -426,10 +426,6 @@ public class BinaryPredicate extends Predicate implements Writable {
                 && (t2 == PrimitiveType.BIGINT || t2 == PrimitiveType.LARGEINT)) {
             return Type.LARGEINT;
         }
-        // MySQL will try to parse string as bigint, if failed, will take string as 0.
-        if (t1 == PrimitiveType.BIGINT && t2.isCharFamily()) {
-            return Type.BIGINT;
-        }
 
         // Implicit conversion affects query performance.
         // For a common example datekey='20200825' which datekey is int type.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java
index 3ce734d8c2..116155e0ca 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java
@@ -227,14 +227,7 @@ public class StringLiteral extends LiteralExpr {
                             throw new AnalysisException(e.getMessage());
                         }
                     }
-                    // MySQL will try to parse string as bigint, if failed, will cast string as 0.
-                    long longValue;
-                    try {
-                        longValue = Long.parseLong(value);
-                    } catch (NumberFormatException e) {
-                        longValue = 0L;
-                    }
-                    return new IntLiteral(longValue, targetType);
+                    return new IntLiteral(value, targetType);
                 case LARGEINT:
                     if (VariableVarConverters.hasConverter(beConverted)) {
                         try {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
index fe76449a1e..e1fc2b973f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
@@ -1954,6 +1954,8 @@ public class SessionVariable implements Serializable, Writable {
         tResult.setEnableParquetLazyMat(enableParquetLazyMat);
         tResult.setEnableOrcLazyMat(enableOrcLazyMat);
 
+        tResult.setEnableInsertStrict(enableInsertStrict);
+
         return tResult;
     }
 
diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift
index 6a8ae55fd4..ca14a548b0 100644
--- a/gensrc/thrift/PaloInternalService.thrift
+++ b/gensrc/thrift/PaloInternalService.thrift
@@ -223,6 +223,8 @@ struct TQueryOptions {
   73: optional i64 scan_queue_mem_limit
 
   74: optional bool enable_scan_node_run_serial = false; 
+
+  75: optional bool enable_insert_strict = false;
 }
     
 
diff --git a/regression-test/data/insert_p0/insert_invalid.out b/regression-test/data/insert_p0/insert_invalid.out
new file mode 100644
index 0000000000..7e268e6c7e
--- /dev/null
+++ b/regression-test/data/insert_p0/insert_invalid.out
@@ -0,0 +1,11 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select_inserted0 --
+12345678908876643	a
+1234567890887664643	b
+
+-- !select_inserted1 --
+12345678908876643	a
+1234567890887664643	b
+
+-- !select_inserted2 --
+
diff --git a/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out b/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out
index b2ed2aae64..d74dd598a1 100644
--- a/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out
+++ b/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out
@@ -585,21 +585,21 @@ true
 2019-08-01T13:21:02.111111
 
 -- !sql --
-\N	\N	\N	\N
 2000-02-29	2000-02-29	2000-02-29	2000-02-29
 2022-01-31	2022-01-31	2022-01-31	2022-01-31
 2022-02-28	2022-02-28	2022-02-28	2022-02-28
+2022-02-28	2022-02-28	2022-02-28	2022-02-28
 
 -- !sql --
-\N	\N
 2000-02-29	2000-02-29
 2022-01-31	2022-01-31
 2022-02-28	2022-02-28
+2022-02-28	2022-02-28
 
 -- !sql --
-\N	\N	\N	\N
 1970-01-01	1970-01-01	1970-01-01	1970-01-01
 2000-01-31	2000-01-31	2000-01-31	2000-01-31
 2021-12-27	2021-12-27	2021-12-27	2021-12-27
+2022-02-21	2022-02-21	2022-02-21	2022-02-21
 2022-02-28	2022-02-28	2022-02-28	2022-02-28
 
diff --git a/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out b/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out
index 3e532edaec..d179d5360f 100644
--- a/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out
+++ b/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out
@@ -617,22 +617,22 @@ true
 -1096
 
 -- !sql --
-\N	\N	\N	\N
 2000-02-29	2000-02-29	2000-02-29	2000-02-29
 2022-01-31	2022-01-31	2022-01-31	2022-01-31
 2022-02-28	2022-02-28	2022-02-28	2022-02-28
+2022-02-28	2022-02-28	2022-02-28	2022-02-28
 
 -- !sql --
-\N	\N
 2000-02-29	2000-02-29
 2022-01-31	2022-01-31
 2022-02-28	2022-02-28
+2022-02-28	2022-02-28
 
 -- !sql --
-\N	\N	\N	\N
 1970-01-01	1970-01-01	1970-01-01	1970-01-01
 2000-01-31	2000-01-31	2000-01-31	2000-01-31
 2021-12-27	2021-12-27	2021-12-27	2021-12-27
+2022-02-21	2022-02-21	2022-02-21	2022-02-21
 2022-02-28	2022-02-28	2022-02-28	2022-02-28
 
 -- !sql --
diff --git a/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_date_nullable.groovy b/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_date_nullable.groovy
index a4e43d4ce0..0f9942231f 100644
--- a/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_date_nullable.groovy
+++ b/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_date_nullable.groovy
@@ -47,7 +47,7 @@ PROPERTIES (
         (1, '2021-03-01', '2021-03-02', '2021-03-03', '2021-03-01', '2021-03-02', '2021-03-03'),
         (1, '2021-02-01', '2021-02-02', '2021-02-03', '2021-02-01', '2021-02-02', '2021-02-03'),
         (1, '2021-01-01', '2021-01-02', '2021-01-03', '2021-01-01', '2021-01-02', '2021-01-03'),
-        (null, '2021-05-01', 'null', '2021-04-03', '2021-05-01', 'null', '2021-04-03')
+        (null, '2021-05-01', null, '2021-04-03', '2021-05-01', null, '2021-04-03')
     """
 
     qt_sql1 "select date1 from ${table1} order by date1"
diff --git a/regression-test/suites/insert_p0/insert_invalid.groovy b/regression-test/suites/insert_p0/insert_invalid.groovy
new file mode 100644
index 0000000000..45703d4a50
--- /dev/null
+++ b/regression-test/suites/insert_p0/insert_invalid.groovy
@@ -0,0 +1,215 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// The cases is copied from https://github.com/trinodb/trino/tree/master
+// /testing/trino-product-tests/src/main/resources/sql-tests/testcases
+// and modified by Doris.
+suite("insert_invalid") {
+    sql """ DROP TABLE IF EXISTS datatype_invalid; """
+    sql """
+    CREATE TABLE `datatype_invalid` (`timea` bigint NOT NULL, `creatr` varchar(30) NULL)
+        UNIQUE KEY(`timea`)
+        DISTRIBUTED BY HASH(`timea`) BUCKETS 1
+        PROPERTIES ("replication_num" = "1");
+    """
+    
+    // strict insert
+    sql """ set enable_insert_strict=true; """
+
+    // test insert select: out of range value
+    sql """ DROP TABLE IF EXISTS datatype_invalid_base; """
+    sql """ 
+    CREATE TABLE `datatype_invalid_base` (
+        `timea` varchar(30) NULL,
+        `creatr` varchar(30) NULL
+    ) UNIQUE KEY(`timea`)
+      DISTRIBUTED BY HASH(`timea`) BUCKETS 1
+      PROPERTIES ("replication_num" = "1");
+    """
+
+    sql """
+    insert into
+        datatype_invalid_base
+    values
+        ("12345678908876643", "a"),
+        ("1234567890887664643", "b"),
+        ("123456789088766445456", "c");
+    """
+
+    test {
+        sql """ insert into datatype_invalid select * from datatype_invalid_base;"""
+        exception "Invalid value in strict mode"
+    }
+
+    // test insert select: invalid value
+    sql """ DROP TABLE IF EXISTS datatype_invalid_base; """
+    sql """ 
+    CREATE TABLE `datatype_invalid_base` (
+        `timea` varchar(30) NULL,
+        `creatr` varchar(30) NULL
+    ) UNIQUE KEY(`timea`)
+      DISTRIBUTED BY HASH(`timea`) BUCKETS 1
+      PROPERTIES ("replication_num" = "1");
+    """
+
+    sql """
+    insert into
+        datatype_invalid_base
+    values
+        ("a", "a");
+    """
+    test {
+        sql """ insert into datatype_invalid select * from datatype_invalid_base;"""
+        exception "Invalid value in strict mode"
+    }
+
+    // test insert select: invalid value
+    sql """ DROP TABLE IF EXISTS datatype_invalid_base; """
+    sql """ 
+    CREATE TABLE `datatype_invalid_base` (
+        `timea` varchar(30) NULL,
+        `creatr` varchar(30) NULL
+    ) UNIQUE KEY(`timea`)
+      DISTRIBUTED BY HASH(`timea`) BUCKETS 1
+      PROPERTIES ("replication_num" = "1");
+    """
+
+    sql """
+    insert into
+        datatype_invalid_base
+    values
+        (" ", "a");
+    """
+    test {
+        sql """ insert into datatype_invalid select * from datatype_invalid_base;"""
+        exception "Invalid value in strict mode"
+    }
+
+    // test insert select: null into not nullable
+    sql """ DROP TABLE IF EXISTS datatype_invalid_base; """
+    sql """ 
+    CREATE TABLE `datatype_invalid_base` (
+        `timea` varchar(30) NULL,
+        `creatr` varchar(30) NULL
+    ) UNIQUE KEY(`timea`)
+      DISTRIBUTED BY HASH(`timea`) BUCKETS 1
+      PROPERTIES ("replication_num" = "1");
+    """
+
+    sql """
+    insert into
+        datatype_invalid_base
+    values
+        (null, "a");
+    """
+    test {
+        sql """ insert into datatype_invalid select * from datatype_invalid_base;"""
+        exception "Insert has filtered data in strict mode"
+    }
+
+    // test insert
+    test {
+        sql """ insert into datatype_invalid values("a", "a");"""
+        exception "Invalid number format"
+    }
+    test {
+        sql """ insert into datatype_invalid values(" ", "a");"""
+        exception "Invalid number format"
+    }
+    test {
+        sql """ insert into datatype_invalid values(123456789088766445456, "a");"""
+        exception "Number out of range"
+    }
+    test {
+        sql """ insert into datatype_invalid values(null, "a");"""
+        exception "Insert has filtered data in strict mode"
+    }
+
+    sql """ DROP TABLE IF EXISTS datatype_invalid; """
+    sql """
+    CREATE TABLE `datatype_invalid` (`timea` datetime NOT NULL, `creatr` varchar(30) NULL)
+        UNIQUE KEY(`timea`)
+        DISTRIBUTED BY HASH(`timea`) BUCKETS 1
+        PROPERTIES ("replication_num" = "1");
+    """
+    test {
+        sql """ insert into datatype_invalid values ('2022-02-29', 'a'); """
+        exception "Invalid value in strict mode"
+    }
+
+    sql """ set enable_insert_strict=false; """
+    sql """ DROP TABLE IF EXISTS datatype_invalid; """
+    sql """
+    CREATE TABLE `datatype_invalid` (`timea` bigint NOT NULL, `creatr` varchar(30) NULL)
+        UNIQUE KEY(`timea`)
+        DISTRIBUTED BY HASH(`timea`) BUCKETS 1
+        PROPERTIES ("replication_num" = "1");
+    """
+
+    // non strict insert into select
+    sql """ DROP TABLE IF EXISTS datatype_invalid_base; """
+    sql """
+    CREATE TABLE `datatype_invalid_base` (
+        `timea` varchar(30) NULL,
+        `creatr` varchar(30) NULL
+    ) UNIQUE KEY(`timea`)
+      DISTRIBUTED BY HASH(`timea`) BUCKETS 1
+      PROPERTIES ("replication_num" = "1");
+    """
+    sql """
+    insert into
+        datatype_invalid_base
+    values
+        ("a", "a"),
+        (" ", "a"),
+        ("12345678908876643", "a"),
+        ("1234567890887664643", "b"),
+        ("123456789088766445456", "c");
+    """
+    sql """ insert into datatype_invalid select * from datatype_invalid_base;"""
+
+    qt_select_inserted0 """ select * from datatype_invalid order by timea """
+
+    sql """ DROP TABLE IF EXISTS datatype_invalid_base; """
+    sql """
+    CREATE TABLE `datatype_invalid_base` (
+        `timea` varchar(30) NULL,
+        `creatr` varchar(30) NULL
+    ) UNIQUE KEY(`timea`)
+      DISTRIBUTED BY HASH(`timea`) BUCKETS 1
+      PROPERTIES ("replication_num" = "1");
+    """
+    sql """
+    insert into
+        datatype_invalid_base
+    values
+        (null, "a");
+    """
+    sql """ insert into datatype_invalid select * from datatype_invalid_base;"""
+
+    qt_select_inserted1 """ select * from datatype_invalid order by timea """
+
+    sql """ DROP TABLE IF EXISTS datatype_invalid; """
+    sql """
+    CREATE TABLE `datatype_invalid` (`timea` datetime NOT NULL, `creatr` varchar(30) NULL)
+        UNIQUE KEY(`timea`)
+        DISTRIBUTED BY HASH(`timea`) BUCKETS 1
+        PROPERTIES ("replication_num" = "1");
+    """
+    sql """ insert into datatype_invalid values ('2022-02-29', 'a'); """
+    qt_select_inserted2 """ select * from datatype_invalid order by timea """
+}
diff --git a/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy b/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy
index 787ead79ba..64c0fe2476 100644
--- a/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy
+++ b/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy
@@ -566,7 +566,7 @@ suite("test_date_function") {
         insert into ${tableName} values 
         ('2022-01-01', '2022-01-01', '2022-01-01 00:00:00', '2022-01-01 00:00:00'), 
         ('2000-02-01', '2000-02-01', '2000-02-01 00:00:00', '2000-02-01 00:00:00.123'), 
-        ('2022-02-29', '2022-02-29', '2022-02-29 00:00:00', '2022-02-29 00:00:00'),
+        ('2022-02-27', '2022-02-27', '2022-02-27 00:00:00', '2022-02-27 00:00:00'),
         ('2022-02-28', '2022-02-28', '2022-02-28T23:59:59', '2022-02-28T23:59:59');"""
     qt_sql """
         select last_day(birth), last_day(birth1), 
@@ -588,7 +588,7 @@ suite("test_date_function") {
         insert into ${tableName} values 
         ('2022-01-01', '2022-01-01 00:00:00'), 
         ('2000-02-01', '2000-02-01 00:00:00'), 
-        ('2022-02-29', '2022-02-29 00:00:00'),
+        ('2022-02-27', '2022-02-27 00:00:00'),
         ('2022-02-28', '2022-02-28 23:59:59');"""
     qt_sql """
         select last_day(birth), last_day(birth1) from ${tableName};
@@ -611,7 +611,7 @@ suite("test_date_function") {
         insert into ${tableName} values 
         ('2022-01-01', '2022-01-01', '2022-01-01 00:00:00', '2022-01-01 00:00:00'), 
         ('2000-02-01', '2000-02-01', '2000-02-01 00:00:00', '2000-02-01 00:00:00.123'), 
-        ('2022-02-29', '2022-02-29', '2022-02-29 00:00:00', '2022-02-29 00:00:00'),
+        ('2022-02-27', '2022-02-27', '2022-02-27 00:00:00', '2022-02-27 00:00:00'),
         ('2022-02-28', '2022-02-28', '2022-02-28 23:59:59', '2022-02-28 23:59:59'),
         ('1970-01-02', '1970-01-02', '1970-01-02 01:02:03', '1970-01-02 02:03:04');"""
     qt_sql """
diff --git a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy
index ae8ff4861c..86641d242b 100644
--- a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy
+++ b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy
@@ -590,7 +590,7 @@ suite("test_date_function") {
         insert into ${tableName} values 
         ('2022-01-01', '2022-01-01', '2022-01-01 00:00:00', '2022-01-01 00:00:00'), 
         ('2000-02-01', '2000-02-01', '2000-02-01 00:00:00', '2000-02-01 00:00:00.123'), 
-        ('2022-02-29', '2022-02-29', '2022-02-29 00:00:00', '2022-02-29 00:00:00'),
+        ('2022-02-27', '2022-02-27', '2022-02-27 00:00:00', '2022-02-27 00:00:00'),
         ('2022-02-28', '2022-02-28', '2022-02-28T23:59:59', '2022-02-28T23:59:59');"""
     qt_sql """
         select last_day(birth), last_day(birth1), 
@@ -612,7 +612,7 @@ suite("test_date_function") {
         insert into ${tableName} values 
         ('2022-01-01', '2022-01-01 00:00:00'), 
         ('2000-02-01', '2000-02-01 00:00:00'), 
-        ('2022-02-29', '2022-02-29 00:00:00'),
+        ('2022-02-27', '2022-02-27 00:00:00'),
         ('2022-02-28', '2022-02-28 23:59:59');"""
     qt_sql """
         select last_day(birth), last_day(birth1) from ${tableName};
@@ -646,7 +646,7 @@ suite("test_date_function") {
         insert into ${tableName} values 
         ('2022-01-01', '2022-01-01', '2022-01-01 00:00:00', '2022-01-01 00:00:00'), 
         ('2000-02-01', '2000-02-01', '2000-02-01 00:00:00', '2000-02-01 00:00:00.123'), 
-        ('2022-02-29', '2022-02-29', '2022-02-29 00:00:00', '2022-02-29 00:00:00'),
+        ('2022-02-27', '2022-02-27', '2022-02-27 00:00:00', '2022-02-27 00:00:00'),
         ('2022-02-28', '2022-02-28', '2022-02-28 23:59:59', '2022-02-28 23:59:59'),
         ('1970-01-02', '1970-01-02', '1970-01-02 01:02:03', '1970-01-02 02:03:04');"""
     qt_sql """


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 16/36: [Fix](Nereids) Fix duplicated name in view does not throw exception (#20374)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit e64ab13b45b102b87f7e4fb68b792f181ef6e4ab
Author: LiBinfeng <46...@users.noreply.github.com>
AuthorDate: Mon Jun 5 16:10:54 2023 +0800

    [Fix](Nereids) Fix duplicated name in view does not throw exception (#20374)
    
    When using nereids, if we have duplicated name in output of view, we need to throw an exception. A check rule was added in bindExpression rule set
---
 .../nereids/rules/analysis/BindExpression.java     | 20 +++++++
 .../trees/plans/ExplainInsertCommandTest.java      |  6 --
 .../subquery/test_duplicate_name_in_view.groovy    | 69 ++++++++++++++++++++++
 3 files changed, 89 insertions(+), 6 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java
index f1bd3802c7..286dbebfc1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java
@@ -67,6 +67,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
 import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat;
 import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation;
 import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
+import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias;
 import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation;
 import org.apache.doris.nereids.trees.plans.logical.UsingJoin;
 import org.apache.doris.nereids.util.TypeCoercionUtils;
@@ -538,6 +539,13 @@ public class BindExpression implements AnalysisRuleFactory {
                     UnboundTVFRelation relation = ctx.root;
                     return bindTableValuedFunction(relation, ctx.statementContext);
                 })
+            ),
+            RuleType.BINDING_SUBQUERY_ALIAS_SLOT.build(
+                logicalSubQueryAlias().thenApply(ctx -> {
+                    LogicalSubQueryAlias<Plan> subQueryAlias = ctx.root;
+                    checkSameNameSlot(subQueryAlias.child(0).getOutput(), subQueryAlias.getAlias());
+                    return subQueryAlias;
+                })
             )
         ).stream().map(ruleCondition).collect(ImmutableList.toImmutableList());
     }
@@ -677,6 +685,18 @@ public class BindExpression implements AnalysisRuleFactory {
         return new LogicalTVFRelation(unboundTVFRelation.getId(), (TableValuedFunction) function);
     }
 
+    private void checkSameNameSlot(List<Slot> childOutputs, String subQueryAlias) {
+        Set<String> nameSlots = new HashSet<>();
+        for (Slot s : childOutputs) {
+            if (nameSlots.contains(s.getName())) {
+                throw new AnalysisException("Duplicated inline view column alias: '" + s.getName()
+                        + "'" + " in inline view: '" + subQueryAlias + "'");
+            } else {
+                nameSlots.add(s.getName());
+            }
+        }
+    }
+
     private BoundFunction bindTableGeneratingFunction(UnboundFunction unboundFunction,
             CascadesContext cascadesContext) {
         List<Expression> boundArguments = unboundFunction.getArguments().stream()
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/ExplainInsertCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/ExplainInsertCommandTest.java
index 7eda6fae58..d82dd209b4 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/ExplainInsertCommandTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/ExplainInsertCommandTest.java
@@ -104,12 +104,6 @@ public class ExplainInsertCommandTest extends TestWithFeService {
 
     }
 
-    @Test
-    public void testInsertIntoDuplicateKeyTableWithCast() throws Exception {
-        String sql = "explain insert into t1 select * from (select cast(k1 as varchar), 1, 1, 1 from src) t";
-        Assertions.assertEquals(4, getOutputFragment(sql).getOutputExprs().size());
-    }
-
     @Test
     public void testInsertIntoSomeColumns() throws Exception {
         String sql = "explain insert into t1 (v1, v2) select v1 + 1, v2 + 4 from src";
diff --git a/regression-test/suites/nereids_p0/subquery/test_duplicate_name_in_view.groovy b/regression-test/suites/nereids_p0/subquery/test_duplicate_name_in_view.groovy
new file mode 100644
index 0000000000..fd5f093a3e
--- /dev/null
+++ b/regression-test/suites/nereids_p0/subquery/test_duplicate_name_in_view.groovy
@@ -0,0 +1,69 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("inlineview_with_project") {
+    sql "SET enable_nereids_planner=true"
+    sql "SET enable_fallback_to_original_planner=false"
+    sql """
+        drop table if exists issue_19611_t0;
+    """
+
+    sql """
+        drop table if exists issue_19611_t1;
+    """
+
+    sql """
+        create table issue_19611_t0 (c0 int)
+        ENGINE=OLAP
+        DISTRIBUTED BY HASH(c0) BUCKETS 5
+        PROPERTIES (
+        "replication_allocation" = "tag.location.default: 1",
+        "in_memory" = "false",
+        "storage_format" = "V2"
+        );
+    """
+
+    sql """
+        create table issue_19611_t1 (c0 int)
+        ENGINE=OLAP
+        DISTRIBUTED BY HASH(c0) BUCKETS 5
+        PROPERTIES (
+        "replication_allocation" = "tag.location.default: 1",
+        "in_memory" = "false",
+        "storage_format" = "V2"
+        );
+    """
+
+    test {
+        sql """
+             select * from (
+                select * from issue_19611_t0, issue_19611_t1 where issue_19611_t1.c0 != 0 
+                    union select * from issue_19611_t0, issue_19611_t1 where issue_19611_t1.c0 = 0) tmp;
+        """
+        exception "errCode = 2, detailMessage = Unexpected exception: Duplicated inline view column alias: 'c0' in inline view: 'tmp'"
+
+    }
+
+
+    sql """
+        drop table if exists issue_19611_t0;
+    """
+
+    sql """
+        drop table if exists issue_19611_t1;
+    """
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 02/36: [fix](Nereids) give clean error message when there are subquery in the on clause (#20211)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 8edb297534d84259ee6ab297f359e4cbbcf4c4c9
Author: Chengpeng Yan <41...@users.noreply.github.com>
AuthorDate: Tue Jun 6 16:50:20 2023 +0800

    [fix](Nereids) give clean error message when there are subquery in the on clause (#20211)
    
    Add the rule for checking the join node in `analysis/CheckAnalysis.java` file. When we check the join node, we should check its' on clause. If there are some subquery expression, we should throw exception.
    
    Before this PR
    ```
    mysql> select a.k1 from baseall a join test b on b.k2 in (select 49);
    ERROR 1105 (HY000): errCode = 2, detailMessage = Unexpected exception: nul
    ```
    
    After this PR
    ```
    mysql> select a.k1 from baseall a join test b on b.k2 in (select 49);
    ERROR 1105 (HY000): errCode = 2, detailMessage = Unexpected exception: Not support OnClause contain Subquery, expr:k2 IN (INSUBQUERY) (LogicalOneRowRelation ( projects=[49 AS `49`#28], buildUnionNode=true ))
    ```
---
 .../doris/nereids/rules/analysis/CheckAnalysis.java      | 16 ++++++++++------
 1 file changed, 10 insertions(+), 6 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java
index 80ce746776..c37c05084f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java
@@ -21,6 +21,7 @@ import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.rules.Rule;
 import org.apache.doris.nereids.rules.RuleType;
 import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.SubqueryExpr;
 import org.apache.doris.nereids.trees.expressions.WindowExpression;
 import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.GroupingScalarFunction;
@@ -28,6 +29,7 @@ import org.apache.doris.nereids.trees.expressions.typecoercion.TypeCheckResult;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
 import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 
 import com.google.common.collect.ImmutableList;
@@ -46,12 +48,14 @@ import java.util.Set;
 public class CheckAnalysis implements AnalysisRuleFactory {
 
     private static final Map<Class<? extends LogicalPlan>, Set<Class<? extends Expression>>>
-            UNEXPECTED_EXPRESSION_TYPE_MAP = ImmutableMap.of(
-                    LogicalFilter.class, ImmutableSet.of(
-                            AggregateFunction.class,
-                            GroupingScalarFunction.class,
-                            WindowExpression.class)
-    );
+            UNEXPECTED_EXPRESSION_TYPE_MAP = ImmutableMap.<Class<? extends LogicalPlan>,
+                Set<Class<? extends Expression>>>builder()
+            .put(LogicalFilter.class, ImmutableSet.of(
+                AggregateFunction.class,
+                GroupingScalarFunction.class,
+                WindowExpression.class))
+            .put(LogicalJoin.class, ImmutableSet.of(SubqueryExpr.class))
+            .build();
 
     @Override
     public List<Rule> buildRules() {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 23/36: [fix](conf) fix fe host in doris-cluster.conf #20422

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 8c79108d8e9972a9c18d0fb5f7a0ee0f78b551c3
Author: ElvinWei <zh...@outlook.com>
AuthorDate: Tue Jun 6 09:15:36 2023 +0800

    [fix](conf) fix fe host in doris-cluster.conf #20422
---
 tools/tpch-tools/conf/doris-cluster.conf | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/tools/tpch-tools/conf/doris-cluster.conf b/tools/tpch-tools/conf/doris-cluster.conf
index 21fe5c09ff..b2f6d41125 100644
--- a/tools/tpch-tools/conf/doris-cluster.conf
+++ b/tools/tpch-tools/conf/doris-cluster.conf
@@ -16,7 +16,7 @@
 # under the License.
 
 # Any of FE host
-export FE_HOST='172.16.1.0'
+export FE_HOST='127.0.0.1'
 # http_port in fe.conf
 export FE_HTTP_PORT=8030
 # query_port in fe.conf


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 08/36: [Enchancement](Agg State) storage function name and result is nullable in agg state type (#20298)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit a01fb1197256724e03934d3f230d333d0afff0f6
Author: Pxl <px...@qq.com>
AuthorDate: Sun Jun 4 22:44:48 2023 +0800

    [Enchancement](Agg State)  storage function name and result is nullable in agg state type  (#20298)
    
    storage function name and result is nullable in agg state type
---
 be/CMakeLists.txt                                  |   2 +-
 be/src/olap/tablet_meta.cpp                        |   5 +
 be/src/olap/tablet_schema.cpp                      |  19 +--
 be/src/olap/tablet_schema.h                        |   4 +
 be/src/runtime/types.h                             |   8 ++
 be/src/vec/data_types/data_type_agg_state.h        |  42 ++++++-
 be/src/vec/data_types/data_type_factory.cpp        |  28 +++--
 be/src/vec/exec/vaggregation_node.cpp              |  13 +-
 be/src/vec/exprs/vectorized_agg_fn.cpp             |  27 +++--
 be/src/vec/exprs/vectorized_fn_call.cpp            |   4 +-
 be/src/vec/functions/function_agg_state.h          |  23 +++-
 .../org/apache/doris/catalog/AggStateType.java     | 134 +++++++++++++++++++++
 .../java/org/apache/doris/catalog/ScalarType.java  |  62 +---------
 .../main/java/org/apache/doris/catalog/Type.java   |   2 +-
 .../java/org/apache/doris/analysis/ColumnDef.java  |  13 +-
 .../main/java/org/apache/doris/analysis/Expr.java  |  76 ++++++++++--
 .../main/java/org/apache/doris/catalog/Column.java |   6 +-
 .../java/org/apache/doris/catalog/Function.java    |  41 +++++--
 .../apache/doris/nereids/types/AggStateType.java   |  14 ++-
 .../org/apache/doris/nereids/types/DataType.java   |   5 +-
 .../org/apache/doris/persist/gson/GsonUtils.java   |   4 +-
 gensrc/proto/data.proto                            |   2 +
 gensrc/proto/olap_file.proto                       |   1 +
 gensrc/thrift/Descriptors.thrift                   |   1 +
 gensrc/thrift/Types.thrift                         |   4 +-
 25 files changed, 386 insertions(+), 154 deletions(-)

diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index 5f5058e483..712f11f672 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -454,7 +454,7 @@ endif()
 
 add_subdirectory(${SRC_DIR}/clucene EXCLUDE_FROM_ALL)
 
-set(clucene_options -w -Wall -Wno-non-virtual-dtor)
+set(clucene_options -w -Wall)
 if (COMPILER_CLANG)
     set(clucene_options ${clucene_options} -Wno-c++11-narrowing)
 else ()
diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp
index 432fd34ff3..3ee508afb9 100644
--- a/be/src/olap/tablet_meta.cpp
+++ b/be/src/olap/tablet_meta.cpp
@@ -315,6 +315,11 @@ void TabletMeta::init_column_from_tcolumn(uint32_t unique_id, const TColumn& tco
     column->set_index_length(length);
     column->set_precision(tcolumn.column_type.precision);
     column->set_frac(tcolumn.column_type.scale);
+
+    if (tcolumn.__isset.result_is_nullable) {
+        column->set_result_is_nullable(tcolumn.result_is_nullable);
+    }
+
     if (tcolumn.column_type.type == TPrimitiveType::VARCHAR ||
         tcolumn.column_type.type == TPrimitiveType::STRING) {
         if (!tcolumn.column_type.__isset.index_len) {
diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp
index 20dd40aed1..f5f62e6857 100644
--- a/be/src/olap/tablet_schema.cpp
+++ b/be/src/olap/tablet_schema.cpp
@@ -427,6 +427,9 @@ void TabletColumn::init_from_pb(const ColumnPB& column) {
         _aggregation = get_aggregation_type_by_string(column.aggregation());
         _aggregation_name = column.aggregation();
     }
+    if (column.has_result_is_nullable()) {
+        _result_is_nullable = column.result_is_nullable();
+    }
     if (column.has_visible()) {
         _visible = column.visible();
     }
@@ -464,6 +467,7 @@ void TabletColumn::to_schema_pb(ColumnPB* column) const {
     if (!_aggregation_name.empty()) {
         column->set_aggregation(_aggregation_name);
     }
+    column->set_result_is_nullable(_result_is_nullable);
     if (_has_bitmap_index) {
         column->set_has_bitmap_index(_has_bitmap_index);
     }
@@ -494,19 +498,8 @@ bool TabletColumn::is_row_store_column() const {
 
 vectorized::AggregateFunctionPtr TabletColumn::get_aggregate_function_union(
         vectorized::DataTypePtr type) const {
-    auto state_type = dynamic_cast<const vectorized::DataTypeAggState*>(type.get());
-    if (!state_type) {
-        return nullptr;
-    }
-    vectorized::DataTypes argument_types;
-    for (auto col : _sub_columns) {
-        auto sub_type = vectorized::DataTypeFactory::instance().create_data_type(col);
-        state_type->add_sub_type(sub_type);
-    }
-    auto agg_function = vectorized::AggregateFunctionSimpleFactory::instance().get(
-            _aggregation_name, state_type->get_sub_types(), false);
-
-    return vectorized::AggregateStateUnion::create(agg_function, {type}, type);
+    auto state_type = assert_cast<const vectorized::DataTypeAggState*>(type.get());
+    return vectorized::AggregateStateUnion::create(state_type->get_nested_function(), {type}, type);
 }
 
 vectorized::AggregateFunctionPtr TabletColumn::get_aggregate_function(std::string suffix) const {
diff --git a/be/src/olap/tablet_schema.h b/be/src/olap/tablet_schema.h
index 00ec5d6aa9..c82e1e8f2f 100644
--- a/be/src/olap/tablet_schema.h
+++ b/be/src/olap/tablet_schema.h
@@ -115,6 +115,8 @@ public:
     static FieldAggregationMethod get_aggregation_type_by_string(const std::string& str);
     static uint32_t get_field_length_by_type(TPrimitiveType::type type, uint32_t string_length);
     bool is_row_store_column() const;
+    std::string get_aggregation_name() const { return _aggregation_name; }
+    bool get_result_is_nullable() const { return _result_is_nullable; }
 
 private:
     int32_t _unique_id;
@@ -143,6 +145,8 @@ private:
     TabletColumn* _parent = nullptr;
     std::vector<TabletColumn> _sub_columns;
     uint32_t _sub_column_count = 0;
+
+    bool _result_is_nullable = false;
 };
 
 bool operator==(const TabletColumn& a, const TabletColumn& b);
diff --git a/be/src/runtime/types.h b/be/src/runtime/types.h
index 1d261caeb2..e4ae6f9461 100644
--- a/be/src/runtime/types.h
+++ b/be/src/runtime/types.h
@@ -61,6 +61,10 @@ struct TypeDescriptor {
 
     std::vector<TypeDescriptor> children;
 
+    bool result_is_nullable = false;
+
+    std::string function_name;
+
     // Only set if type == TYPE_STRUCT. The field name of each child.
     std::vector<std::string> field_names;
 
@@ -153,6 +157,10 @@ struct TypeDescriptor {
                 result.children.push_back(from_thrift(sub));
                 result.contains_nulls.push_back(sub.is_nullable);
             }
+            DCHECK(t.__isset.result_is_nullable);
+            result.result_is_nullable = t.result_is_nullable;
+            DCHECK(t.__isset.function_name);
+            result.function_name = t.function_name;
         }
         return result;
     }
diff --git a/be/src/vec/data_types/data_type_agg_state.h b/be/src/vec/data_types/data_type_agg_state.h
index 1bb5dc4e00..eaf2c122d2 100644
--- a/be/src/vec/data_types/data_type_agg_state.h
+++ b/be/src/vec/data_types/data_type_agg_state.h
@@ -28,6 +28,8 @@
 #include <memory>
 #include <string>
 
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/aggregate_functions/aggregate_function_simple_factory.h"
 #include "vec/data_types/data_type.h"
 #include "vec/data_types/data_type_string.h"
 
@@ -43,8 +45,24 @@ namespace doris::vectorized {
 
 class DataTypeAggState : public DataTypeString {
 public:
+    DataTypeAggState(DataTypes sub_types, bool result_is_nullable, std::string function_name)
+            : _sub_types(sub_types),
+              _result_is_nullable(result_is_nullable),
+              _function_name(function_name) {}
+
     const char* get_family_name() const override { return "AggState"; }
 
+    std::string do_get_name() const override {
+        std::string types;
+        for (auto type : _sub_types) {
+            if (!types.empty()) {
+                types += ", ";
+            }
+            types += type->get_name();
+        }
+        return "AggState(" + types + ")";
+    }
+
     TypeIndex get_type_id() const override { return TypeIndex::AggState; }
 
     PrimitiveType get_type_as_primitive_type() const override { return TYPE_AGG_STATE; }
@@ -52,19 +70,35 @@ public:
         return TPrimitiveType::AGG_STATE;
     }
 
-    const DataTypes& get_sub_types() const { return sub_types; }
+    std::string to_string(const IColumn& column, size_t row_num) const override {
+        std::string res = "binary(";
+        StringRef str = column.get_data_at(row_num);
+        for (auto c : str.to_string()) {
+            res += std::to_string(int(c));
+            res += ' ';
+        }
+        res += ")";
+        return res;
+    }
 
-    void add_sub_type(DataTypePtr type) const { sub_types.push_back(type); }
+    const DataTypes& get_sub_types() const { return _sub_types; }
 
     void to_pb_column_meta(PColumnMeta* col_meta) const override {
         IDataType::to_pb_column_meta(col_meta);
-        for (auto type : sub_types) {
+        for (auto type : _sub_types) {
             type->to_pb_column_meta(col_meta->add_children());
         }
     }
 
+    AggregateFunctionPtr get_nested_function() const {
+        return AggregateFunctionSimpleFactory::instance().get(_function_name, _sub_types,
+                                                              _result_is_nullable);
+    }
+
 private:
-    mutable DataTypes sub_types;
+    DataTypes _sub_types;
+    bool _result_is_nullable;
+    std::string _function_name;
 };
 
 } // namespace doris::vectorized
diff --git a/be/src/vec/data_types/data_type_factory.cpp b/be/src/vec/data_types/data_type_factory.cpp
index 438e266658..3a2bf13ea7 100644
--- a/be/src/vec/data_types/data_type_factory.cpp
+++ b/be/src/vec/data_types/data_type_factory.cpp
@@ -40,6 +40,7 @@
 #include "runtime/define_primitive_type.h"
 #include "vec/common/uint128.h"
 #include "vec/core/types.h"
+#include "vec/data_types/data_type.h"
 #include "vec/data_types/data_type_agg_state.h"
 #include "vec/data_types/data_type_array.h"
 #include "vec/data_types/data_type_bitmap.h"
@@ -115,6 +116,13 @@ DataTypePtr DataTypeFactory::create_data_type(const TabletColumn& col_desc, bool
             names.push_back(col_desc.get_sub_column(i).name());
         }
         nested = std::make_shared<DataTypeStruct>(dataTypes, names);
+    } else if (col_desc.type() == FieldType::OLAP_FIELD_TYPE_AGG_STATE) {
+        DataTypes dataTypes;
+        for (size_t i = 0; i < col_desc.get_subtype_count(); i++) {
+            dataTypes.push_back(create_data_type(col_desc.get_sub_column(i)));
+        }
+        nested = std::make_shared<vectorized::DataTypeAggState>(
+                dataTypes, col_desc.get_result_is_nullable(), col_desc.get_aggregation_name());
     } else {
         nested =
                 _create_primitive_data_type(col_desc.type(), col_desc.precision(), col_desc.frac());
@@ -128,6 +136,7 @@ DataTypePtr DataTypeFactory::create_data_type(const TabletColumn& col_desc, bool
 
 DataTypePtr DataTypeFactory::create_data_type(const TypeDescriptor& col_desc, bool is_nullable) {
     DataTypePtr nested = nullptr;
+    DataTypes subTypes;
     switch (col_desc.type) {
     case TYPE_BOOLEAN:
         nested = std::make_shared<vectorized::DataTypeUInt8>();
@@ -177,12 +186,11 @@ DataTypePtr DataTypeFactory::create_data_type(const TypeDescriptor& col_desc, bo
         nested = std::make_shared<vectorized::DataTypeString>();
         break;
     case TYPE_AGG_STATE:
-        nested = std::make_shared<vectorized::DataTypeAggState>();
         for (size_t i = 0; i < col_desc.children.size(); i++) {
-            ((DataTypeAggState*)nested.get())
-                    ->add_sub_type(
-                            create_data_type(col_desc.children[i], col_desc.contains_nulls[i]));
+            subTypes.push_back(create_data_type(col_desc.children[i], col_desc.contains_nulls[i]));
         }
+        nested = std::make_shared<vectorized::DataTypeAggState>(
+                subTypes, col_desc.result_is_nullable, col_desc.function_name);
         break;
     case TYPE_JSONB:
         nested = std::make_shared<vectorized::DataTypeJsonb>();
@@ -336,9 +344,6 @@ DataTypePtr DataTypeFactory::create_data_type(const TypeIndex& type_index, bool
     case TypeIndex::TimeV2:
         nested = std::make_shared<vectorized::DataTypeTime>();
         break;
-    case TypeIndex::AggState:
-        nested = std::make_shared<vectorized::DataTypeAggState>();
-        break;
     default:
         DCHECK(false) << "invalid typeindex:" << getTypeName(type_index);
         break;
@@ -395,9 +400,6 @@ DataTypePtr DataTypeFactory::_create_primitive_data_type(const FieldType& type,
     case FieldType::OLAP_FIELD_TYPE_STRING:
         result = std::make_shared<vectorized::DataTypeString>();
         break;
-    case FieldType::OLAP_FIELD_TYPE_AGG_STATE:
-        result = std::make_shared<vectorized::DataTypeAggState>();
-        break;
     case FieldType::OLAP_FIELD_TYPE_JSONB:
         result = std::make_shared<vectorized::DataTypeJsonb>();
         break;
@@ -545,10 +547,12 @@ DataTypePtr DataTypeFactory::create_data_type(const PColumnMeta& pcolumn) {
         break;
     }
     case PGenericType::AGG_STATE: {
-        nested = std::make_shared<DataTypeAggState>();
+        DataTypes sub_types;
         for (auto child : pcolumn.children()) {
-            ((DataTypeAggState*)nested.get())->add_sub_type(create_data_type(child));
+            sub_types.push_back(create_data_type(child));
         }
+        nested = std::make_shared<DataTypeAggState>(sub_types, pcolumn.result_is_nullable(),
+                                                    pcolumn.function_name());
         break;
     }
     default: {
diff --git a/be/src/vec/exec/vaggregation_node.cpp b/be/src/vec/exec/vaggregation_node.cpp
index f4bde7569a..bb2ce8377e 100644
--- a/be/src/vec/exec/vaggregation_node.cpp
+++ b/be/src/vec/exec/vaggregation_node.cpp
@@ -668,13 +668,12 @@ Status AggregationNode::_get_without_key_result(RuntimeState* state, Block* bloc
         const auto column_type = block_schema[i].type;
         if (!column_type->equals(*data_types[i])) {
             if (!is_array(remove_nullable(column_type))) {
-                DCHECK(column_type->is_nullable());
-                DCHECK(!data_types[i]->is_nullable())
-                        << " column type: " << column_type->get_name()
-                        << ", data type: " << data_types[i]->get_name();
-                DCHECK(remove_nullable(column_type)->equals(*data_types[i]))
-                        << " column type: " << remove_nullable(column_type)->get_name()
-                        << ", data type: " << data_types[i]->get_name();
+                if (!column_type->is_nullable() || data_types[i]->is_nullable() ||
+                    !remove_nullable(column_type)->equals(*data_types[i])) {
+                    return Status::InternalError(
+                            "column_type not match data_types, column_type={}, data_types={}",
+                            column_type->get_name(), data_types[i]->get_name());
+                }
             }
 
             ColumnPtr ptr = std::move(columns[i]);
diff --git a/be/src/vec/exprs/vectorized_agg_fn.cpp b/be/src/vec/exprs/vectorized_agg_fn.cpp
index 513eb8fe98..cb2f4cadab 100644
--- a/be/src/vec/exprs/vectorized_agg_fn.cpp
+++ b/be/src/vec/exprs/vectorized_agg_fn.cpp
@@ -56,13 +56,10 @@ class IColumn;
 namespace doris::vectorized {
 
 template <class FunctionType>
-AggregateFunctionPtr get_agg_state_function(const std::string& name,
-                                            const DataTypes& argument_types,
+AggregateFunctionPtr get_agg_state_function(const DataTypes& argument_types,
                                             DataTypePtr return_type) {
     return FunctionType::create(
-            AggregateFunctionSimpleFactory::instance().get(
-                    name, ((DataTypeAggState*)argument_types[0].get())->get_sub_types(),
-                    return_type->is_nullable()),
+            assert_cast<const DataTypeAggState*>(argument_types[0].get())->get_nested_function(),
             argument_types, return_type);
 }
 
@@ -159,14 +156,22 @@ Status AggFnEvaluator::prepare(RuntimeState* state, const RowDescriptor& desc,
     } else if (_fn.binary_type == TFunctionBinaryType::RPC) {
         _function = AggregateRpcUdaf::create(_fn, argument_types, _data_type);
     } else if (_fn.binary_type == TFunctionBinaryType::AGG_STATE) {
+        if (argument_types.size() != 1) {
+            return Status::InternalError("Agg state Function must input 1 argument but get {}",
+                                         argument_types.size());
+        }
+        if (argument_types[0]->is_nullable()) {
+            return Status::InternalError("Agg state function input type must be not nullable");
+        }
+        if (argument_types[0]->get_type_as_primitive_type() != PrimitiveType::TYPE_AGG_STATE) {
+            return Status::InternalError(
+                    "Agg state function input type must be agg_state but get {}",
+                    argument_types[0]->get_family_name());
+        }
         if (match_suffix(_fn.name.function_name, AGG_UNION_SUFFIX)) {
-            _function = get_agg_state_function<AggregateStateUnion>(
-                    remove_suffix(_fn.name.function_name, AGG_UNION_SUFFIX), argument_types,
-                    _data_type);
+            _function = get_agg_state_function<AggregateStateUnion>(argument_types, _data_type);
         } else if (match_suffix(_fn.name.function_name, AGG_MERGE_SUFFIX)) {
-            _function = get_agg_state_function<AggregateStateMerge>(
-                    remove_suffix(_fn.name.function_name, AGG_MERGE_SUFFIX), argument_types,
-                    _data_type);
+            _function = get_agg_state_function<AggregateStateMerge>(argument_types, _data_type);
         } else {
             return Status::InternalError(
                     "Aggregate Function {} is not endwith '_merge' or '_union'", _fn.signature);
diff --git a/be/src/vec/exprs/vectorized_fn_call.cpp b/be/src/vec/exprs/vectorized_fn_call.cpp
index 53cd19ed0d..582acb9da9 100644
--- a/be/src/vec/exprs/vectorized_fn_call.cpp
+++ b/be/src/vec/exprs/vectorized_fn_call.cpp
@@ -96,9 +96,7 @@ Status VectorizedFnCall::prepare(RuntimeState* state, const RowDescriptor& desc,
             }
             _function = FunctionAggState::create(
                     argument_types, _data_type,
-                    AggregateFunctionSimpleFactory::instance().get(
-                            remove_suffix(_fn.name.function_name, AGG_STATE_SUFFIX), argument_types,
-                            _data_type->is_nullable()));
+                    assert_cast<const DataTypeAggState*>(_data_type.get())->get_nested_function());
         } else {
             return Status::InternalError("Function {} is not endwith '_state'", _fn.signature);
         }
diff --git a/be/src/vec/functions/function_agg_state.h b/be/src/vec/functions/function_agg_state.h
index e17fc06c60..820eebaff9 100644
--- a/be/src/vec/functions/function_agg_state.h
+++ b/be/src/vec/functions/function_agg_state.h
@@ -21,10 +21,13 @@
 
 #include "common/status.h"
 #include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/columns/column.h"
+#include "vec/columns/column_nullable.h"
 #include "vec/common/arena.h"
 #include "vec/core/block.h"
 #include "vec/core/types.h"
 #include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_agg_state.h"
 #include "vec/functions/function.h"
 
 namespace doris::vectorized {
@@ -62,9 +65,23 @@ public:
                         size_t result, size_t input_rows_count) override {
         auto col = _return_type->create_column();
         std::vector<const IColumn*> agg_columns;
-
-        for (size_t index : arguments) {
-            agg_columns.push_back(block.get_by_position(index).column);
+        std::vector<ColumnPtr> save_columns;
+
+        for (size_t i = 0; i < arguments.size(); i++) {
+            DataTypePtr signature =
+                    assert_cast<const DataTypeAggState*>(_return_type.get())->get_sub_types()[i];
+            ColumnPtr column = block.get_by_position(arguments[i]).column;
+
+            if (!signature->is_nullable() && column->is_nullable()) {
+                return Status::InternalError(
+                        "State function meet input nullable column, but signature is not nullable");
+            }
+            if (!column->is_nullable() && signature->is_nullable()) {
+                column = make_nullable(column);
+                save_columns.push_back(column);
+            }
+
+            agg_columns.push_back(column);
         }
 
         VectorBufferWriter writter(assert_cast<ColumnString&>(*col));
diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/AggStateType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/AggStateType.java
new file mode 100644
index 0000000000..47477a08f8
--- /dev/null
+++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/AggStateType.java
@@ -0,0 +1,134 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.catalog;
+
+import org.apache.doris.thrift.TTypeDesc;
+import org.apache.doris.thrift.TTypeNode;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.annotations.SerializedName;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class AggStateType extends ScalarType {
+
+    @SerializedName(value = "subTypes")
+    private List<Type> subTypes;
+
+    @SerializedName(value = "subTypeNullables")
+    private List<Boolean> subTypeNullables;
+
+    @SerializedName(value = "resultIsNullable")
+    private Boolean resultIsNullable;
+
+    @SerializedName(value = "functionName")
+    private String functionName;
+
+    public AggStateType(String functionName, Boolean resultIsNullable, List<Type> subTypes,
+            List<Boolean> subTypeNullables) {
+        super(PrimitiveType.AGG_STATE);
+        Preconditions.checkState((subTypes == null) == (subTypeNullables == null));
+        if (subTypes != null && subTypeNullables != null) {
+            Preconditions.checkState(subTypes.size() == subTypeNullables.size(),
+                    "AggStateType' subTypes.size()!=subTypeNullables.size()");
+        }
+        this.functionName = functionName;
+        this.subTypes = subTypes;
+        this.subTypeNullables = subTypeNullables;
+        this.resultIsNullable = resultIsNullable;
+    }
+
+    public List<Type> getSubTypes() {
+        return subTypes;
+    }
+
+    public List<Boolean> getSubTypeNullables() {
+        return subTypeNullables;
+    }
+
+    public String getFunctionName() {
+        return functionName;
+    }
+
+    public boolean getResultIsNullable() {
+        return resultIsNullable;
+    }
+
+    @Override
+    public String toSql(int depth) {
+        StringBuilder stringBuilder = new StringBuilder();
+        stringBuilder.append("agg_state(");
+        for (int i = 0; i < subTypes.size(); i++) {
+            if (i > 0) {
+                stringBuilder.append(", ");
+            }
+            stringBuilder.append(subTypes.get(i).toSql());
+            if (subTypeNullables.get(i)) {
+                stringBuilder.append(" NULL");
+            }
+        }
+        stringBuilder.append(")");
+        return stringBuilder.toString();
+    }
+
+    @Override
+    public void toThrift(TTypeDesc container) {
+        super.toThrift(container);
+        if (subTypes != null) {
+            List<TTypeDesc> types = new ArrayList<TTypeDesc>();
+            for (int i = 0; i < subTypes.size(); i++) {
+                TTypeDesc desc = new TTypeDesc();
+                desc.setTypes(new ArrayList<TTypeNode>());
+                subTypes.get(i).toThrift(desc);
+                desc.setIsNullable(subTypeNullables.get(i));
+                types.add(desc);
+            }
+            container.setSubTypes(types);
+        }
+        container.setResultIsNullable(resultIsNullable);
+        container.setFunctionName(functionName);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof AggStateType)) {
+            return false;
+        }
+        AggStateType other = (AggStateType) o;
+        if ((subTypes == null) != (other.getSubTypes() == null)) {
+            return false;
+        }
+        if (subTypes == null) {
+            return true;
+        }
+        int subTypeNumber = subTypeNullables.size();
+        if (subTypeNumber != other.subTypeNullables.size()) {
+            return false;
+        }
+        for (int i = 0; i < subTypeNumber; i++) {
+            if (!subTypeNullables.get(i).equals(other.subTypeNullables.get(i))) {
+                return false;
+            }
+            if (!subTypes.get(i).equals(other.subTypes.get(i))) {
+                return false;
+            }
+        }
+        return true;
+    }
+}
diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java
index 5e4a54f948..9493b6d6ae 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java
@@ -31,8 +31,6 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Objects;
 
 /**
@@ -125,30 +123,10 @@ public class ScalarType extends Type {
     @SerializedName(value = "lenStr")
     private String lenStr;
 
-    @SerializedName(value = "subTypes")
-    private List<Type> subTypes;
-
-    @SerializedName(value = "subTypeNullables")
-    private List<Boolean> subTypeNullables;
-
-    public List<Type> getSubTypes() {
-        return subTypes;
-    }
-
-    public List<Boolean> getSubTypeNullables() {
-        return subTypeNullables;
-    }
-
     public ScalarType(PrimitiveType type) {
         this.type = type;
     }
 
-    public ScalarType(List<Type> subTypes, List<Boolean> subTypeNullables) {
-        this.type = PrimitiveType.AGG_STATE;
-        this.subTypes = subTypes;
-        this.subTypeNullables = subTypeNullables;
-    }
-
     public static ScalarType createType(PrimitiveType type, int len, int precision, int scale) {
         switch (type) {
             case CHAR:
@@ -667,17 +645,7 @@ public class ScalarType extends Type {
                 stringBuilder.append("json");
                 break;
             case AGG_STATE:
-                stringBuilder.append("agg_state(");
-                for (int i = 0; i < subTypes.size(); i++) {
-                    if (i > 0) {
-                        stringBuilder.append(", ");
-                    }
-                    stringBuilder.append(subTypes.get(i).toSql());
-                    if (subTypeNullables.get(i)) {
-                        stringBuilder.append(" NULL");
-                    }
-                }
-                stringBuilder.append(")");
+                stringBuilder.append("agg_state(unknown)");
                 break;
             default:
                 stringBuilder.append("unknown type: " + type.toString());
@@ -723,18 +691,6 @@ public class ScalarType extends Type {
                 break;
         }
         node.setScalarType(scalarType);
-
-        if (subTypes != null) {
-            List<TTypeDesc> types = new ArrayList<TTypeDesc>();
-            for (int i = 0; i < subTypes.size(); i++) {
-                TTypeDesc desc = new TTypeDesc();
-                desc.setTypes(new ArrayList<TTypeNode>());
-                subTypes.get(i).toThrift(desc);
-                desc.setIsNullable(subTypeNullables.get(i));
-                types.add(desc);
-            }
-            container.setSubTypes(types);
-        }
     }
 
     public int decimalPrecision() {
@@ -908,22 +864,6 @@ public class ScalarType extends Type {
             return false;
         }
         ScalarType other = (ScalarType) o;
-        if (this.isAggStateType() && other.isAggStateType()) {
-            int subTypeNumber = subTypeNullables.size();
-            if (subTypeNumber != other.subTypeNullables.size()) {
-                return false;
-            }
-            for (int i = 0; i < subTypeNumber; i++) {
-                if (!subTypeNullables.get(i).equals(other.subTypeNullables.get(i))) {
-                    return false;
-                }
-                if (!subTypes.get(i).equals(other.subTypes.get(i))) {
-                    return false;
-                }
-            }
-            return true;
-        }
-
         if ((this.isDatetimeV2() && other.isDatetimeV2()) || (this.isTimeV2() && other.isTimeV2())) {
             return this.decimalScale() == other.decimalScale();
         }
diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java
index c801921dc6..22d0bcd65c 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java
@@ -102,7 +102,7 @@ public abstract class Type {
     public static final ScalarType CHAR = ScalarType.createCharType(-1);
     public static final ScalarType BITMAP = new ScalarType(PrimitiveType.BITMAP);
     public static final ScalarType QUANTILE_STATE = new ScalarType(PrimitiveType.QUANTILE_STATE);
-    public static final ScalarType AGG_STATE = new ScalarType(PrimitiveType.AGG_STATE);
+    public static final AggStateType AGG_STATE = new AggStateType(null, null, null, null);
     public static final ScalarType LAMBDA_FUNCTION = new ScalarType(PrimitiveType.LAMBDA_FUNCTION);
     // Only used for alias function, to represent any type in function args
     public static final ScalarType ALL = new ScalarType(PrimitiveType.ALL);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java
index cbe308d1cd..7cd5ca9aa3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java
@@ -497,19 +497,18 @@ public class ColumnDef {
     public Column toColumn() {
         List<Type> typeList = null;
         List<Boolean> nullableList = null;
+
+        Type type = typeDef.getType();
         if (genericAggregationArguments != null) {
             typeList = genericAggregationArguments.stream().map(TypeDef::getType).collect(Collectors.toList());
             nullableList = genericAggregationArguments.stream().map(TypeDef::getNullable).collect(Collectors.toList());
-        }
 
-        Type type = typeDef.getType();
-        if (type.isAggStateType()) {
-            type = new ScalarType(typeList, nullableList);
+            type = Expr.createAggStateType(genericAggregationName, typeList, nullableList);
         }
 
-        return new Column(name, type, isKey, aggregateType, isAllowNull, defaultValue.value, comment,
-                visible, defaultValue.defaultValueExprDef, Column.COLUMN_UNIQUE_ID_INIT_VALUE,
-                defaultValue.getValue(), genericAggregationName, typeList, nullableList);
+        return new Column(name, type, isKey, aggregateType, isAllowNull, defaultValue.value, comment, visible,
+                defaultValue.defaultValueExprDef, Column.COLUMN_UNIQUE_ID_INIT_VALUE, defaultValue.getValue(),
+                genericAggregationName, typeList, nullableList);
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
index 93d5437564..a6cc659794 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
@@ -21,6 +21,7 @@
 package org.apache.doris.analysis;
 
 import org.apache.doris.analysis.ArithmeticExpr.Operator;
+import org.apache.doris.catalog.AggStateType;
 import org.apache.doris.catalog.AggregateFunction;
 import org.apache.doris.catalog.ArrayType;
 import org.apache.doris.catalog.Env;
@@ -444,8 +445,8 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
             setSelectivity();
         }
         analysisDone();
-        if (type.isAggStateType() && !(this instanceof SlotRef) && ((ScalarType) type).getSubTypes() == null) {
-            type = new ScalarType(Arrays.asList(collectChildReturnTypes()),
+        if (type.isAggStateType() && !(this instanceof SlotRef) && ((AggStateType) type).getSubTypes() == null) {
+            type = createAggStateType(((AggStateType) type), Arrays.asList(collectChildReturnTypes()),
                     Arrays.asList(collectChildReturnNullables()));
         }
     }
@@ -1004,8 +1005,8 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
     // Append a flattened version of this expr, including all children, to 'container'.
     protected void treeToThriftHelper(TExpr container) {
         TExprNode msg = new TExprNode();
-        if (type.isAggStateType() && ((ScalarType) type).getSubTypes() == null) {
-            type = new ScalarType(Arrays.asList(collectChildReturnTypes()),
+        if (type.isAggStateType() && ((AggStateType) type).getSubTypes() == null) {
+            type = createAggStateType(((AggStateType) type), Arrays.asList(collectChildReturnTypes()),
                     Arrays.asList(collectChildReturnNullables()));
         }
         msg.type = type.toThrift();
@@ -1482,7 +1483,7 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
         }
 
         if (this.type.isAggStateType()) {
-            List<Type> subTypes = ((ScalarType) targetType).getSubTypes();
+            List<Type> subTypes = ((AggStateType) targetType).getSubTypes();
 
             if (this instanceof FunctionCallExpr) {
                 if (subTypes.size() != getChildren().size()) {
@@ -1493,7 +1494,7 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
                 }
                 type = targetType;
             } else {
-                List<Type> selfSubTypes = ((ScalarType) type).getSubTypes();
+                List<Type> selfSubTypes = ((AggStateType) type).getSubTypes();
                 if (subTypes.size() != selfSubTypes.size()) {
                     throw new AnalysisException("AggState's subTypes size did not match");
                 }
@@ -1879,7 +1880,7 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
                 if (argList.size() != 1 || !argList.get(0).isAggStateType()) {
                     throw new AnalysisException("merge/union function must input one agg_state");
                 }
-                ScalarType aggState = (ScalarType) argList.get(0);
+                AggStateType aggState = (AggStateType) argList.get(0);
                 if (aggState.getSubTypes() == null) {
                     throw new AnalysisException("agg_state's subTypes is null");
                 }
@@ -1895,9 +1896,10 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
 
             if (isState) {
                 f = new ScalarFunction(new FunctionName(name + AGG_STATE_SUFFIX), Arrays.asList(f.getArgs()),
-                        Type.AGG_STATE, f.hasVarArgs(), f.isUserVisible());
+                        Expr.createAggStateType(name, null, null), f.hasVarArgs(), f.isUserVisible());
                 f.setNullableMode(NullableMode.ALWAYS_NOT_NULLABLE);
             } else {
+                Function original = f;
                 f = ((AggregateFunction) f).clone();
                 f.setArgs(argList);
                 if (isUnion) {
@@ -1907,6 +1909,8 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
                 }
                 if (isMerge) {
                     f.setName(new FunctionName(name + AGG_MERGE_SUFFIX));
+                    f.setNullableMode(NullableMode.CUSTOM);
+                    f.setNestedFunction(original);
                 }
             }
             f.setBinaryType(TFunctionBinaryType.AGG_STATE);
@@ -2221,6 +2225,10 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
     }
 
     protected boolean hasNullableChild() {
+        return hasNullableChild(children);
+    }
+
+    protected static boolean hasNullableChild(List<Expr> children) {
         for (Expr expr : children) {
             if (expr.isNullable()) {
                 return true;
@@ -2244,24 +2252,34 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
      * overwrite this method to plan correct
      */
     public boolean isNullable() {
+        return isNullable(fn, children);
+    }
+
+    public static boolean isNullable(Function fn, List<Expr> children) {
         if (fn == null) {
             return true;
         }
         switch (fn.getNullableMode()) {
             case DEPEND_ON_ARGUMENT:
-                return hasNullableChild();
+                return hasNullableChild(children);
             case ALWAYS_NOT_NULLABLE:
                 return false;
             case CUSTOM:
-                return customNullableAlgorithm();
+                return customNullableAlgorithm(fn, children);
             case ALWAYS_NULLABLE:
             default:
                 return true;
         }
     }
 
-    private boolean customNullableAlgorithm() {
+    private static boolean customNullableAlgorithm(Function fn, List<Expr> children) {
         Preconditions.checkState(fn.getNullableMode() == Function.NullableMode.CUSTOM);
+
+        if (fn.functionName().endsWith(AGG_MERGE_SUFFIX)) {
+            AggStateType type = (AggStateType) fn.getArgs()[0];
+            return isNullable(fn.getNestedFunction(), getMockedExprs(type));
+        }
+
         if (fn.functionName().equalsIgnoreCase("if")) {
             Preconditions.checkState(children.size() == 3);
             for (int i = 1; i < children.size(); i++) {
@@ -2295,7 +2313,7 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
                     && ConnectContext.get().getSessionVariable().checkOverflowForDecimal()) {
                 return true;
             } else {
-                return hasNullableChild();
+                return hasNullableChild(children);
             }
         }
         if ((fn.functionName().equalsIgnoreCase(Operator.ADD.getName())
@@ -2305,7 +2323,7 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
                     && ConnectContext.get().getSessionVariable().checkOverflowForDecimal()) {
                 return true;
             } else {
-                return hasNullableChild();
+                return hasNullableChild(children);
             }
         }
         if (fn.functionName().equalsIgnoreCase("group_concat")) {
@@ -2323,6 +2341,38 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
         return true;
     }
 
+    public static Boolean getResultIsNullable(String name, List<Type> typeList, List<Boolean> nullableList) {
+        if (name == null || typeList == null || nullableList == null) {
+            return false;
+        }
+        FunctionName fnName = new FunctionName(name);
+        Function searchDesc = new Function(fnName, typeList, Type.INVALID, false, true);
+        List<Expr> mockedExprs = getMockedExprs(typeList, nullableList);
+        Function f = Env.getCurrentEnv().getFunction(searchDesc, Function.CompareMode.IS_IDENTICAL);
+        return isNullable(f, mockedExprs);
+    }
+
+    public static AggStateType createAggStateType(String name, List<Type> typeList, List<Boolean> nullableList) {
+        return new AggStateType(name, Expr.getResultIsNullable(name, typeList, nullableList), typeList, nullableList);
+    }
+
+    public static AggStateType createAggStateType(AggStateType type, List<Type> typeList, List<Boolean> nullableList) {
+        return new AggStateType(type.getFunctionName(),
+                Expr.getResultIsNullable(type.getFunctionName(), typeList, nullableList), typeList, nullableList);
+    }
+
+    public static List<Expr> getMockedExprs(List<Type> typeList, List<Boolean> nullableList) {
+        List<Expr> mockedExprs = Lists.newArrayList();
+        for (int i = 0; i < typeList.size(); i++) {
+            mockedExprs.add(new SlotRef(typeList.get(i), nullableList.get(i)));
+        }
+        return mockedExprs;
+    }
+
+    public static List<Expr> getMockedExprs(AggStateType type) {
+        return getMockedExprs(type.getSubTypes(), type.getSubTypeNullables());
+    }
+
     public void materializeSrcExpr() {
         if (this instanceof SlotRef) {
             SlotRef thisRef = (SlotRef) this;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
index f9722b47c0..936db90a35 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
@@ -496,9 +496,7 @@ public class Column implements Writable, GsonPostProcessable {
         if (null != this.aggregationType) {
             tColumn.setAggregationType(this.aggregationType.toThrift());
         }
-        if (genericAggregationName != null) {
-            tColumn.setAggregation(genericAggregationName);
-        }
+
         tColumn.setIsKey(this.isKey);
         tColumn.setIsAllowNull(this.isAllowNull);
         // keep compatibility
@@ -509,6 +507,8 @@ public class Column implements Writable, GsonPostProcessable {
         tColumn.setColUniqueId(uniqueId);
 
         if (type.isAggStateType()) {
+            tColumn.setAggregation(genericAggregationName);
+            tColumn.setResultIsNullable(((AggStateType) type).getResultIsNullable());
             for (Column column : children) {
                 tColumn.addToChildrenColumn(column.toThrift());
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java
index 22b36f604c..b59f974eda 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java
@@ -123,6 +123,8 @@ public class Function implements Writable {
     private URI location;
     private TFunctionBinaryType binaryType;
 
+    private Function nestedFunction = null;
+
     protected NullableMode nullableMode = NullableMode.DEPEND_ON_ARGUMENT;
 
     protected boolean vectorized = true;
@@ -192,6 +194,14 @@ public class Function implements Writable {
         this.checksum = other.checksum;
     }
 
+    public void setNestedFunction(Function nestedFunction) {
+        this.nestedFunction = nestedFunction;
+    }
+
+    public Function getNestedFunction() {
+        return nestedFunction;
+    }
+
     public Function clone() {
         return new Function(this);
     }
@@ -532,7 +542,8 @@ public class Function implements Writable {
         }
 
         if (realReturnType.isAggStateType()) {
-            realReturnType = new ScalarType(Arrays.asList(realArgTypes), Arrays.asList(realArgTypeNullables));
+            realReturnType = Expr.createAggStateType(((AggStateType) realReturnType), Arrays.asList(realArgTypes),
+                    Arrays.asList(realArgTypeNullables));
         }
 
         // For types with different precisions and scales, return type only indicates a
@@ -835,14 +846,30 @@ public class Function implements Writable {
     public static FunctionCallExpr convertToStateCombinator(FunctionCallExpr fnCall) {
         Function aggFunction = fnCall.getFn();
         List<Type> arguments = Arrays.asList(aggFunction.getArgs());
-        ScalarFunction fn = new org.apache.doris.catalog.ScalarFunction(
-                new FunctionName(aggFunction.getFunctionName().getFunction() + Expr.AGG_STATE_SUFFIX),
-                arguments,
-                new ScalarType(arguments, fnCall.getChildren().stream().map(expr -> {
-                    return expr.isNullable();
-                }).collect(Collectors.toList())), aggFunction.hasVarArgs(), aggFunction.isUserVisible());
+        ScalarFunction fn = new ScalarFunction(
+                new FunctionName(aggFunction.getFunctionName().getFunction() + Expr.AGG_STATE_SUFFIX), arguments,
+                Expr.createAggStateType(aggFunction.getFunctionName().getFunction(),
+                        fnCall.getChildren().stream().map(expr -> {
+                            return expr.getType();
+                        }).collect(Collectors.toList()), fnCall.getChildren().stream().map(expr -> {
+                            return expr.isNullable();
+                        }).collect(Collectors.toList())),
+                aggFunction.hasVarArgs(), aggFunction.isUserVisible());
         fn.setNullableMode(NullableMode.ALWAYS_NOT_NULLABLE);
         fn.setBinaryType(TFunctionBinaryType.AGG_STATE);
         return new FunctionCallExpr(fn, fnCall.getParams());
     }
+
+    public static FunctionCallExpr convertToMergeCombinator(FunctionCallExpr fnCall) {
+        Function aggFunction = fnCall.getFn();
+        aggFunction.setName(new FunctionName(aggFunction.getFunctionName().getFunction() + Expr.AGG_MERGE_SUFFIX));
+        aggFunction.setArgs(Arrays.asList(Expr.createAggStateType(aggFunction.getFunctionName().getFunction(),
+                fnCall.getChildren().stream().map(expr -> {
+                    return expr.getType();
+                }).collect(Collectors.toList()), fnCall.getChildren().stream().map(expr -> {
+                    return expr.isNullable();
+                }).collect(Collectors.toList()))));
+        aggFunction.setBinaryType(TFunctionBinaryType.AGG_STATE);
+        return fnCall;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java
index 84d0d7ce06..ac245c3066 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java
@@ -17,7 +17,7 @@
 
 package org.apache.doris.nereids.types;
 
-import org.apache.doris.catalog.ScalarType;
+import org.apache.doris.analysis.Expr;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.nereids.types.coercion.AbstractDataType;
 import org.apache.doris.nereids.types.coercion.PrimitiveType;
@@ -30,22 +30,28 @@ import java.util.stream.Collectors;
  */
 public class AggStateType extends PrimitiveType {
 
-    public static final AggStateType SYSTEM_DEFAULT = new AggStateType(null, null);
+    public static final AggStateType SYSTEM_DEFAULT = new AggStateType(null, null, null);
 
     public static final int WIDTH = 16;
 
     private final List<DataType> subTypes;
     private final List<Boolean> subTypeNullables;
+    private final String functionName;
 
-    public AggStateType(List<DataType> subTypes, List<Boolean> subTypeNullables) {
+    public AggStateType(String functionName, List<DataType> subTypes, List<Boolean> subTypeNullables) {
         this.subTypes = subTypes;
         this.subTypeNullables = subTypeNullables;
+        this.functionName = functionName;
+    }
+
+    public List<DataType> getSubTypes() {
+        return subTypes;
     }
 
     @Override
     public Type toCatalogDataType() {
         List<Type> types = subTypes.stream().map(t -> t.toCatalogDataType()).collect(Collectors.toList());
-        return new ScalarType(types, subTypeNullables);
+        return Expr.createAggStateType(functionName, types, subTypeNullables);
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java
index 17cb0ed9a5..0941a4a1e9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java
@@ -325,9 +325,10 @@ public abstract class DataType implements AbstractDataType {
             org.apache.doris.catalog.ArrayType arrayType = (org.apache.doris.catalog.ArrayType) type;
             return ArrayType.of(fromCatalogType(arrayType.getItemType()), arrayType.getContainsNull());
         } else if (type.isAggStateType()) {
-            List<DataType> types = ((ScalarType) type).getSubTypes().stream().map(t -> fromCatalogType(t))
+            org.apache.doris.catalog.AggStateType catalogType = ((org.apache.doris.catalog.AggStateType) type);
+            List<DataType> types = catalogType.getSubTypes().stream().map(t -> fromCatalogType(t))
                     .collect(Collectors.toList());
-            return new AggStateType(types, ((ScalarType) type).getSubTypeNullables());
+            return new AggStateType(catalogType.getFunctionName(), types, catalogType.getSubTypeNullables());
         }
         throw new AnalysisException("Nereids do not support type: " + type);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java
index 12623cfd31..bf7c0a5484 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java
@@ -20,6 +20,7 @@ package org.apache.doris.persist.gson;
 import org.apache.doris.alter.AlterJobV2;
 import org.apache.doris.alter.RollupJobV2;
 import org.apache.doris.alter.SchemaChangeJobV2;
+import org.apache.doris.catalog.AggStateType;
 import org.apache.doris.catalog.ArrayType;
 import org.apache.doris.catalog.DatabaseIf;
 import org.apache.doris.catalog.DistributionInfo;
@@ -140,7 +141,8 @@ public class GsonUtils {
             .registerSubtype(ScalarType.class, ScalarType.class.getSimpleName())
             .registerSubtype(ArrayType.class, ArrayType.class.getSimpleName())
             .registerSubtype(MapType.class, MapType.class.getSimpleName())
-            .registerSubtype(StructType.class, StructType.class.getSimpleName());
+            .registerSubtype(StructType.class, StructType.class.getSimpleName())
+            .registerSubtype(AggStateType.class, AggStateType.class.getSimpleName());
 
     // runtime adapter for class "DistributionInfo"
     private static RuntimeTypeAdapterFactory<DistributionInfo> distributionInfoTypeAdapterFactory
diff --git a/gensrc/proto/data.proto b/gensrc/proto/data.proto
index d57ed2e24f..e9ced52391 100644
--- a/gensrc/proto/data.proto
+++ b/gensrc/proto/data.proto
@@ -58,6 +58,8 @@ message PColumnMeta {
     optional bool is_nullable = 3 [default = false];
     optional Decimal decimal_param = 4;
     repeated PColumnMeta children = 5;
+    optional bool result_is_nullable = 6;
+    optional string function_name = 7;
 }
 
 message PBlock {
diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto
index 2d6a331c37..9bf2abe5c8 100644
--- a/gensrc/proto/olap_file.proto
+++ b/gensrc/proto/olap_file.proto
@@ -189,6 +189,7 @@ message ColumnPB {
     optional bool visible = 16 [default=true];
     repeated ColumnPB children_columns = 17;
     repeated string children_column_names = 18;
+    optional bool result_is_nullable = 19;
 }
 
 enum IndexType {
diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift
index 284c728b8d..631132f934 100644
--- a/gensrc/thrift/Descriptors.thrift
+++ b/gensrc/thrift/Descriptors.thrift
@@ -38,6 +38,7 @@ struct TColumn {
     14: optional i32 gram_size
     15: optional i32 gram_bf_size
     16: optional string aggregation
+    17: optional bool result_is_nullable
 }
 
 struct TSlotDescriptor {
diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift
index b0b5da4550..ee62f59aa5 100644
--- a/gensrc/thrift/Types.thrift
+++ b/gensrc/thrift/Types.thrift
@@ -161,7 +161,9 @@ struct TTypeDesc {
     1: list<TTypeNode> types
     2: optional bool is_nullable
     3: optional i64  byte_size
-    4: optional list<TTypeDesc> sub_types;
+    4: optional list<TTypeDesc> sub_types
+    5: optional bool result_is_nullable
+    6: optional string function_name
 }
 
 enum TAggregationType {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 19/36: [performance](load) improve memtable sort performance (#20392)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 2e0ff4f788aee30aa8f0242915c66725ec79e10c
Author: Kaijie Chen <ck...@apache.org>
AuthorDate: Sun Jun 4 20:33:15 2023 +0800

    [performance](load) improve memtable sort performance (#20392)
---
 be/src/olap/memtable.cpp            | 48 +++++++++++++++++----
 be/src/olap/memtable.h              | 63 +++++++++++++++++++++++++++-
 be/src/vec/core/block.h             |  8 ++++
 be/test/CMakeLists.txt              |  1 +
 be/test/olap/memtable_sort_test.cpp | 83 +++++++++++++++++++++++++++++++++++++
 thirdparty/build-thirdparty.sh      | 10 ++---
 thirdparty/vars.sh                  |  8 ++--
 7 files changed, 203 insertions(+), 18 deletions(-)

diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp
index cd4db46d20..d61a25c8be 100644
--- a/be/src/olap/memtable.cpp
+++ b/be/src/olap/memtable.cpp
@@ -19,6 +19,7 @@
 
 #include <fmt/format.h>
 #include <gen_cpp/olap_file.pb.h>
+#include <pdqsort.h>
 
 #include <algorithm>
 #include <cstddef>
@@ -249,13 +250,32 @@ void MemTable::_put_into_output(vectorized::Block& in_block) {
     _output_mutable_block.add_rows(&in_block, row_pos_vec.data(),
                                    row_pos_vec.data() + in_block.rows());
 }
-int MemTable::_sort() {
+
+size_t MemTable::_sort() {
     SCOPED_RAW_TIMER(&_stat.sort_ns);
     _stat.sort_times++;
-    _vec_row_comparator->set_block(&_input_mutable_block);
-    auto new_row_it = std::next(_row_in_blocks.begin(), _last_sorted_pos);
     size_t same_keys_num = 0;
+    // sort new rows
+    Tie tie = Tie(_last_sorted_pos, _row_in_blocks.size());
+    for (size_t i = 0; i < _schema->num_key_columns(); i++) {
+        auto cmp = [&](const RowInBlock* lhs, const RowInBlock* rhs) -> int {
+            return _input_mutable_block.compare_one_column(lhs->_row_pos, rhs->_row_pos, i, -1);
+        };
+        _sort_one_column(_row_in_blocks, tie, cmp);
+    }
     bool is_dup = (_keys_type == KeysType::DUP_KEYS);
+    // sort extra round by _row_pos to make the sort stable
+    auto iter = tie.iter();
+    while (iter.next()) {
+        pdqsort(std::next(_row_in_blocks.begin(), iter.left()),
+                std::next(_row_in_blocks.begin(), iter.right()),
+                [&is_dup](const RowInBlock* lhs, const RowInBlock* rhs) -> bool {
+                    return is_dup ? lhs->_row_pos > rhs->_row_pos : lhs->_row_pos < rhs->_row_pos;
+                });
+        same_keys_num += iter.right() - iter.left();
+    }
+    // merge new rows and old rows
+    _vec_row_comparator->set_block(&_input_mutable_block);
     auto cmp_func = [this, is_dup, &same_keys_num](const RowInBlock* l,
                                                    const RowInBlock* r) -> bool {
         auto value = (*(this->_vec_row_comparator))(l, r);
@@ -266,14 +286,26 @@ int MemTable::_sort() {
             return value < 0;
         }
     };
-    // sort new rows
-    std::sort(new_row_it, _row_in_blocks.end(), cmp_func);
-    // merge new rows and old rows
+    auto new_row_it = std::next(_row_in_blocks.begin(), _last_sorted_pos);
     std::inplace_merge(_row_in_blocks.begin(), new_row_it, _row_in_blocks.end(), cmp_func);
     _last_sorted_pos = _row_in_blocks.size();
     return same_keys_num;
 }
 
+void MemTable::_sort_one_column(std::vector<RowInBlock*>& row_in_blocks, Tie& tie,
+                                std::function<int(const RowInBlock*, const RowInBlock*)> cmp) {
+    auto iter = tie.iter();
+    while (iter.next()) {
+        pdqsort(std::next(row_in_blocks.begin(), iter.left()),
+                std::next(row_in_blocks.begin(), iter.right()),
+                [&cmp](auto lhs, auto rhs) -> bool { return cmp(lhs, rhs) < 0; });
+        tie[iter.left()] = 0;
+        for (int i = iter.left() + 1; i < iter.right(); i++) {
+            tie[i] = (cmp(row_in_blocks[i - 1], row_in_blocks[i]) == 0);
+        }
+    }
+}
+
 template <bool is_final>
 void MemTable::_finalize_one_row(RowInBlock* row,
                                  const vectorized::ColumnsWithTypeAndName& block_data,
@@ -379,7 +411,7 @@ void MemTable::shrink_memtable_by_agg() {
     if (_keys_type == KeysType::DUP_KEYS) {
         return;
     }
-    int same_keys_num = _sort();
+    size_t same_keys_num = _sort();
     if (same_keys_num == 0) {
         vectorized::Block in_block = _input_mutable_block.to_block();
         _put_into_output(in_block);
@@ -465,7 +497,7 @@ Status MemTable::flush() {
 
 Status MemTable::_do_flush() {
     SCOPED_CONSUME_MEM_TRACKER(_flush_mem_tracker);
-    int same_keys_num = _sort();
+    size_t same_keys_num = _sort();
     if (_keys_type == KeysType::DUP_KEYS || same_keys_num == 0) {
         if (_keys_type == KeysType::DUP_KEYS && _schema->num_key_columns() == 0) {
             _output_mutable_block.swap(_input_mutable_block);
diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h
index d6acc5211c..e7844f5d93 100644
--- a/be/src/olap/memtable.h
+++ b/be/src/olap/memtable.h
@@ -20,6 +20,7 @@
 #include <stddef.h>
 #include <stdint.h>
 
+#include <cstring>
 #include <functional>
 #include <memory>
 #include <optional>
@@ -69,6 +70,64 @@ struct RowInBlock {
     inline void remove_init_agg() { _has_init_agg = false; }
 };
 
+class Tie {
+public:
+    class Iter {
+    public:
+        Iter(Tie& tie) : _tie(tie), _next(tie._begin + 1) {}
+        size_t left() { return _left; }
+        size_t right() { return _right; }
+
+        // return false means no more ranges
+        bool next() {
+            if (_next >= _tie._end) {
+                return false;
+            }
+            _next = _find(1, _next);
+            if (_next >= _tie._end) {
+                return false;
+            }
+            _left = _next - 1;
+            _next = _find(0, _next);
+            _right = _next;
+            return true;
+        }
+
+    private:
+        size_t _find(uint8_t value, size_t start) {
+            if (start >= _tie._end) {
+                return start;
+            }
+            size_t offset = start - _tie._begin;
+            size_t size = _tie._end - start;
+            void* p = std::memchr(_tie._bits.data() + offset, value, size);
+            if (p == nullptr) {
+                return _tie._end;
+            }
+            return static_cast<uint8_t*>(p) - _tie._bits.data() + _tie._begin;
+        }
+
+    private:
+        Tie& _tie;
+        size_t _left;
+        size_t _right;
+        size_t _next;
+    };
+
+public:
+    Tie(size_t begin, size_t end) : _begin(begin), _end(end) {
+        _bits = std::vector<uint8_t>(_end - _begin, 1);
+    }
+    uint8_t operator[](int i) const { return _bits[i - _begin]; }
+    uint8_t& operator[](int i) { return _bits[i - _begin]; }
+    Iter iter() { return Iter(*this); }
+
+private:
+    const size_t _begin;
+    const size_t _end;
+    std::vector<uint8_t> _bits;
+};
+
 class RowInBlockComparator {
 public:
     RowInBlockComparator(const Schema* schema) : _schema(schema) {}
@@ -220,7 +279,9 @@ private:
     size_t _last_sorted_pos = 0;
 
     //return number of same keys
-    int _sort();
+    size_t _sort();
+    void _sort_one_column(std::vector<RowInBlock*>& row_in_blocks, Tie& tie,
+                          std::function<int(const RowInBlock*, const RowInBlock*)> cmp);
     template <bool is_final>
     void _finalize_one_row(RowInBlock* row, const vectorized::ColumnsWithTypeAndName& block_data,
                            int row_pos);
diff --git a/be/src/vec/core/block.h b/be/src/vec/core/block.h
index 381323c049..9823d0f984 100644
--- a/be/src/vec/core/block.h
+++ b/be/src/vec/core/block.h
@@ -465,6 +465,14 @@ public:
         return _data_types[position];
     }
 
+    int compare_one_column(size_t n, size_t m, size_t column_id, int nan_direction_hint) const {
+        DCHECK_LE(column_id, columns());
+        DCHECK_LE(n, rows());
+        DCHECK_LE(m, rows());
+        auto& column = get_column_by_position(column_id);
+        return column->compare_at(n, m, *column, nan_direction_hint);
+    }
+
     int compare_at(size_t n, size_t m, size_t num_columns, const MutableBlock& rhs,
                    int nan_direction_hint) const {
         DCHECK_GE(columns(), num_columns);
diff --git a/be/test/CMakeLists.txt b/be/test/CMakeLists.txt
index d035c8a103..055480f814 100644
--- a/be/test/CMakeLists.txt
+++ b/be/test/CMakeLists.txt
@@ -87,6 +87,7 @@ set(OLAP_TEST_FILES
     olap/cumulative_compaction_policy_test.cpp
     #olap/row_cursor_test.cpp
     olap/skiplist_test.cpp
+    olap/memtable_sort_test.cpp
     olap/olap_meta_test.cpp
     olap/decimal12_test.cpp
     olap/storage_types_test.cpp
diff --git a/be/test/olap/memtable_sort_test.cpp b/be/test/olap/memtable_sort_test.cpp
new file mode 100644
index 0000000000..b9aa0b6652
--- /dev/null
+++ b/be/test/olap/memtable_sort_test.cpp
@@ -0,0 +1,83 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+
+#include "olap/memtable.h"
+
+namespace doris {
+
+class MemTableSortTest : public ::testing::Test {};
+
+TEST_F(MemTableSortTest, Tie) {
+    auto t0 = Tie {0, 0};
+    EXPECT_FALSE(t0.iter().next());
+
+    auto tie = Tie {0, 1};
+    EXPECT_FALSE(tie.iter().next());
+
+    auto t = Tie {10, 30};
+    for (int i = 10; i < 30; i++) {
+        EXPECT_EQ(t[i], 1);
+    }
+
+    auto it1 = t.iter();
+    EXPECT_TRUE(it1.next());
+    EXPECT_EQ(it1.left(), 10);
+    EXPECT_EQ(it1.right(), 30);
+
+    EXPECT_FALSE(it1.next());
+
+    t[13] = t[14] = t[22] = t[29] = 0;
+    auto it2 = t.iter();
+
+    EXPECT_TRUE(it2.next());
+    EXPECT_EQ(it2.left(), 10);
+    EXPECT_EQ(it2.right(), 13);
+
+    EXPECT_TRUE(it2.next());
+    EXPECT_EQ(it2.left(), 14);
+    EXPECT_EQ(it2.right(), 22);
+
+    EXPECT_TRUE(it2.next());
+    EXPECT_EQ(it2.left(), 22);
+    EXPECT_EQ(it2.right(), 29);
+
+    EXPECT_FALSE(it2.next());
+    EXPECT_FALSE(it2.next());
+
+    // 100000000...
+    for (int i = 11; i < 30; i++) {
+        t[i] = 0;
+    }
+    EXPECT_FALSE(t.iter().next());
+
+    // 000000000...
+    t[10] = 0;
+    EXPECT_FALSE(t.iter().next());
+
+    // 000000000...001
+    t[29] = 1;
+    auto it3 = t.iter();
+    EXPECT_TRUE(it3.next());
+    EXPECT_EQ(it3.left(), 28);
+    EXPECT_EQ(it3.right(), 30);
+
+    EXPECT_FALSE(it3.next());
+}
+
+} // namespace doris
diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh
index f311080709..ad348b72c9 100755
--- a/thirdparty/build-thirdparty.sh
+++ b/thirdparty/build-thirdparty.sh
@@ -266,9 +266,9 @@ check_if_source_exist() {
     echo "===== begin build $1"
 }
 
-check_if_archieve_exist() {
+check_if_archive_exist() {
     if [[ -z $1 ]]; then
-        echo "archieve should specified to check if exist."
+        echo "archive should specified to check if exist."
         exit 1
     fi
 
@@ -1167,9 +1167,9 @@ build_parallel_hashmap() {
 
 # pdqsort
 build_pdqsort() {
-    check_if_source_exist "${PDQSORT_SOURCE}"
-    cd "${TP_SOURCE_DIR}/${PDQSORT_SOURCE}"
-    cp -r pdqsort.h "${TP_INSTALL_DIR}/include/"
+    check_if_archive_exist "${PDQSORT_FILE}"
+    cd "${TP_SOURCE_DIR}"
+    cp "${PDQSORT_FILE}" "${TP_INSTALL_DIR}/include/"
 }
 
 # libdivide
diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh
index bc5cc885a3..144a61e10d 100644
--- a/thirdparty/vars.sh
+++ b/thirdparty/vars.sh
@@ -380,10 +380,10 @@ LIBDIVIDE_SOURCE="libdivide-5.0"
 LIBDIVIDE_MD5SUM="7fd16b0bb4ab6812b2e2fdc7bfb81641"
 
 #pdqsort
-PDQSORT_DOWNLOAD="http://ftp.cise.ufl.edu/ubuntu/pool/universe/p/pdqsort/pdqsort_0.0.0+git20180419.orig.tar.gz"
-PDQSORT_NAME="pdqsort.tar.gz"
-PDQSORT_SOURCE="pdqsort-0.0.0+git20180419"
-PDQSORT_MD5SUM="39261c3e7b40aa7505662fac29f22d20"
+PDQSORT_DOWNLOAD="https://raw.githubusercontent.com/orlp/pdqsort/b1ef26a55cdb60d236a5cb199c4234c704f46726/pdqsort.h"
+PDQSORT_NAME="pdqsort.h"
+PDQSORT_FILE="pdqsort.h"
+PDQSORT_MD5SUM="af28f79d5d7d7a5486f54d9f1244c2b5"
 
 # benchmark
 BENCHMARK_DOWNLOAD="https://github.com/google/benchmark/archive/v1.5.6.tar.gz"


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 30/36: [fix](executor)Fix duplicate timer and add open timer #20448

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit c3ef8e0a294f800cbd0614f5f570d381b2876371
Author: wangbo <wa...@apache.org>
AuthorDate: Tue Jun 6 08:55:52 2023 +0800

    [fix](executor)Fix duplicate timer and add open timer #20448
    
    1 Currently, Node's total timer couter has timed twice(in Open and alloc_resource), this may cause timer in profile is not correct.
    2 Add more timer to find more code which may cost much time.
---
 be/src/vec/exec/join/vhash_join_node.cpp | 5 ++++-
 be/src/vec/exec/join/vhash_join_node.h   | 3 +++
 be/src/vec/exec/scan/vscan_node.cpp      | 6 +++++-
 be/src/vec/exec/scan/vscan_node.h        | 2 ++
 4 files changed, 14 insertions(+), 2 deletions(-)

diff --git a/be/src/vec/exec/join/vhash_join_node.cpp b/be/src/vec/exec/join/vhash_join_node.cpp
index 2e91f199c2..cd6bea45e9 100644
--- a/be/src/vec/exec/join/vhash_join_node.cpp
+++ b/be/src/vec/exec/join/vhash_join_node.cpp
@@ -457,6 +457,8 @@ Status HashJoinNode::prepare(RuntimeState* state) {
     _probe_side_output_timer = ADD_TIMER(probe_phase_profile, "ProbeWhenProbeSideOutputTime");
 
     _join_filter_timer = ADD_TIMER(runtime_profile(), "JoinFilterTimer");
+    _open_timer = ADD_TIMER(runtime_profile(), "OpenTime");
+    _allocate_resource_timer = ADD_TIMER(runtime_profile(), "AllocateResourceTime");
 
     _push_down_timer = ADD_TIMER(runtime_profile(), "PublishRuntimeFilterTime");
     _push_compute_timer = ADD_TIMER(runtime_profile(), "PushDownComputeTime");
@@ -744,14 +746,15 @@ void HashJoinNode::_prepare_probe_block() {
 
 Status HashJoinNode::open(RuntimeState* state) {
     SCOPED_TIMER(_runtime_profile->total_time_counter());
+    SCOPED_TIMER(_open_timer);
     RETURN_IF_ERROR(VJoinNodeBase::open(state));
     RETURN_IF_CANCELLED(state);
     return Status::OK();
 }
 
 Status HashJoinNode::alloc_resource(doris::RuntimeState* state) {
+    SCOPED_TIMER(_allocate_resource_timer);
     RETURN_IF_ERROR(VJoinNodeBase::alloc_resource(state));
-    SCOPED_TIMER(_runtime_profile->total_time_counter());
     for (size_t i = 0; i < _runtime_filter_descs.size(); i++) {
         if (auto bf = _runtime_filters[i]->get_bloomfilter()) {
             RETURN_IF_ERROR(bf->init_with_fixed_length());
diff --git a/be/src/vec/exec/join/vhash_join_node.h b/be/src/vec/exec/join/vhash_join_node.h
index 33ae53f923..408dacce02 100644
--- a/be/src/vec/exec/join/vhash_join_node.h
+++ b/be/src/vec/exec/join/vhash_join_node.h
@@ -290,6 +290,9 @@ private:
     RuntimeProfile::Counter* _build_side_merge_block_timer;
     RuntimeProfile::Counter* _build_runtime_filter_timer;
 
+    RuntimeProfile::Counter* _open_timer;
+    RuntimeProfile::Counter* _allocate_resource_timer;
+
     RuntimeProfile::Counter* _build_blocks_memory_usage;
     RuntimeProfile::Counter* _hash_table_memory_usage;
     RuntimeProfile::HighWaterMarkCounter* _build_arena_memory_usage;
diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp
index cf5516471c..692b65376d 100644
--- a/be/src/vec/exec/scan/vscan_node.cpp
+++ b/be/src/vec/exec/scan/vscan_node.cpp
@@ -150,22 +150,26 @@ Status VScanNode::prepare(RuntimeState* state) {
     // could add here, not in the _init_profile() function
     _get_next_timer = ADD_TIMER(_runtime_profile, "GetNextTime");
     _acquire_runtime_filter_timer = ADD_TIMER(_runtime_profile, "AcuireRuntimeFilterTime");
+
+    _open_timer = ADD_TIMER(_runtime_profile, "OpenTime");
+    _alloc_resource_timer = ADD_TIMER(_runtime_profile, "AllocateResourceTime");
     return Status::OK();
 }
 
 Status VScanNode::open(RuntimeState* state) {
     SCOPED_TIMER(_runtime_profile->total_time_counter());
+    SCOPED_TIMER(_open_timer);
     RETURN_IF_CANCELLED(state);
     return ExecNode::open(state);
 }
 
 Status VScanNode::alloc_resource(RuntimeState* state) {
+    SCOPED_TIMER(_alloc_resource_timer);
     if (_opened) {
         return Status::OK();
     }
     _input_tuple_desc = state->desc_tbl().get_tuple_descriptor(_input_tuple_id);
     _output_tuple_desc = state->desc_tbl().get_tuple_descriptor(_output_tuple_id);
-    SCOPED_TIMER(_runtime_profile->total_time_counter());
     RETURN_IF_ERROR(ExecNode::alloc_resource(state));
     RETURN_IF_ERROR(_acquire_runtime_filter());
     RETURN_IF_ERROR(_process_conjuncts());
diff --git a/be/src/vec/exec/scan/vscan_node.h b/be/src/vec/exec/scan/vscan_node.h
index c447505a61..69692b032d 100644
--- a/be/src/vec/exec/scan/vscan_node.h
+++ b/be/src/vec/exec/scan/vscan_node.h
@@ -343,6 +343,8 @@ protected:
     RuntimeProfile::Counter* _num_scanners;
 
     RuntimeProfile::Counter* _get_next_timer = nullptr;
+    RuntimeProfile::Counter* _open_timer = nullptr;
+    RuntimeProfile::Counter* _alloc_resource_timer = nullptr;
     RuntimeProfile::Counter* _acquire_runtime_filter_timer = nullptr;
     // time of get block from scanner
     RuntimeProfile::Counter* _scan_timer = nullptr;


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 20/36: [fix][refactor](backend-policy)(compute) refactor the hierarchy of external scan node and fix compute node bug #20402

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit b6a1092d87ef317c295a8c5488db5cf1ad7d25b8
Author: Mingyu Chen <mo...@163.com>
AuthorDate: Tue Jun 6 10:35:30 2023 +0800

    [fix][refactor](backend-policy)(compute) refactor the hierarchy of external scan node and fix compute node bug #20402
    
    There should be 2 kinds of ScanNode:
    
    OlapScanNode
    ExternalScanNode
    The Backends used for ExternalScanNode should be controlled by FederationBackendPolicy.
    But currently, only FileScanNode is controlled by FederationBackendPolicy, other scan node such as MysqlScanNode,
    JdbcScanNode will use Mix Backend even if we enable and prefer to use Compute Backend.
    
    In this PR, I modified the hierarchy of ExternalScanNode, the new hierarchy is:
    
    ScanNode
        OlapScanNode
        SchemaScanNode
        ExternalScanNode
            MetadataScanNode
            DataGenScanNode
            EsScanNode
            OdbcScanNode
            MysqlScanNode
            JdbcScanNode
            FileScanNode
                FileLoadScanNode
                FileQueryScanNode
                    MaxComputeScanNode
                    IcebergScanNode
                    TVFScanNode
                    HiveScanNode
                        HudiScanNode
    And previously, the BackendPolicy is the member of FileScanNode, now I moved it to the ExternalScanNode.
    So that all subtype ExternalScanNode can use BackendPolicy to choose Compute Backend to execute the query.
    
    All all ExternalScanNode should implement the abstract method createScanRangeLocations().
    
    For scan node like jdbc scan node/mysql scan node, the scan range locations will be selected randomly from
    compute node(if preferred).
    
    And for compute node selection. If all scan nodes are external scan nodes, and prefer_compute_node_for_external_table
    is set to true, the BE for this query will only select compute nodes.
---
 .../planner/BackendPartitionedSchemaScanNode.java  | 18 +++-----
 .../org/apache/doris/planner/DataGenScanNode.java  | 37 +++++----------
 .../java/org/apache/doris/planner/EsScanNode.java  | 48 ++++++--------------
 .../org/apache/doris/planner/FileLoadScanNode.java | 29 ++++++++++--
 .../org/apache/doris/planner/JdbcScanNode.java     | 49 ++++++++++++--------
 .../org/apache/doris/planner/MysqlScanNode.java    | 27 ++++-------
 .../org/apache/doris/planner/OdbcScanNode.java     | 32 ++++---------
 .../org/apache/doris/planner/OlapScanNode.java     | 52 ++++++++--------------
 .../java/org/apache/doris/planner/ScanNode.java    | 25 ++++++++++-
 .../org/apache/doris/planner/SchemaScanNode.java   | 15 ++++---
 .../doris/planner/TestExternalTableScanNode.java   | 31 +++++--------
 .../doris/planner/external/ExternalScanNode.java   | 30 +++++++++++--
 .../planner/external/FederationBackendPolicy.java  |  4 ++
 .../doris/planner/external/FileQueryScanNode.java  | 18 ++++----
 .../doris/planner/external/FileScanNode.java       | 12 +----
 .../doris/planner/external/MetadataScanNode.java   | 32 ++++++++-----
 .../apache/doris/planner/external/TVFScanNode.java |  9 ----
 .../doris/planner/external/hudi/HudiScanNode.java  |  2 +-
 .../planner/external/iceberg/IcebergScanNode.java  | 20 ++++-----
 .../main/java/org/apache/doris/qe/Coordinator.java | 48 +++++++++++---------
 .../java/org/apache/doris/qe/CoordinatorTest.java  |  8 ++--
 regression-test/data/es_p0/test_es_query.out       | 46 +++++++++----------
 .../data/es_p0/test_es_query_no_http_url.out       | 10 ++---
 regression-test/suites/es_p0/test_es_query.groovy  |  4 --
 .../suites/es_p0/test_es_query_nereids.groovy      |  5 ---
 .../suites/es_p0/test_es_query_no_http_url.groovy  |  4 --
 .../hive/test_external_yandex_nereids.groovy       |  1 +
 27 files changed, 295 insertions(+), 321 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java
index 8889373af3..6b2e278a01 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java
@@ -60,7 +60,6 @@ public class BackendPartitionedSchemaScanNode extends SchemaScanNode {
         return false;
     }
 
-    private List<TScanRangeLocations> shardScanRanges;
     // backendPartitionInfo is set in generatePartitionInfo().
     // `backendPartitionInfo` is `List Partition` of Backend_ID, one PartitionItem only have one partitionKey
     // for example: if the alive be are: 10001, 10002, 10003, `backendPartitionInfo` like
@@ -86,24 +85,20 @@ public class BackendPartitionedSchemaScanNode extends SchemaScanNode {
     @Override
     public void finalize(Analyzer analyzer) throws UserException {
         super.finalize(analyzer);
-        shardScanRanges = getScanRangeLocations();
+        createScanRangeLocations();
     }
 
     @Override
     public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
-        return shardScanRanges;
+        return scanRangeLocations;
     }
 
     @Override
-    public int getNumInstances() {
-        return shardScanRanges.size();
-    }
-
-    private List<TScanRangeLocations> getScanRangeLocations() throws AnalysisException {
-        List<TScanRangeLocations> result = new ArrayList<>();
+    protected void createScanRangeLocations() throws UserException {
+        scanRangeLocations = new ArrayList<>();
         for (Long partitionID : selectedPartitionIds) {
             Long backendId = partitionIDToBackendID.get(partitionID);
-            Backend be  = Env.getCurrentSystemInfo().getIdToBackend().get(backendId);
+            Backend be = Env.getCurrentSystemInfo().getIdToBackend().get(backendId);
             if (!be.isAlive()) {
                 throw new AnalysisException("backend " + be.getId() + " is not alive.");
             }
@@ -113,9 +108,8 @@ public class BackendPartitionedSchemaScanNode extends SchemaScanNode {
             location.setServer(new TNetworkAddress(be.getHost(), be.getBePort()));
             locations.addToLocations(location);
             locations.setScanRange(new TScanRange());
-            result.add(locations);
+            scanRangeLocations.add(locations);
         }
-        return result;
     }
 
     private void computePartitionInfo() throws AnalysisException {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/DataGenScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/DataGenScanNode.java
index da4dfa1906..46af5ec1ae 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/DataGenScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/DataGenScanNode.java
@@ -19,9 +19,9 @@ package org.apache.doris.planner;
 
 import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.TupleDescriptor;
-import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.NereidsException;
 import org.apache.doris.common.UserException;
+import org.apache.doris.planner.external.ExternalScanNode;
 import org.apache.doris.statistics.StatisticalType;
 import org.apache.doris.tablefunction.DataGenTableValuedFunction;
 import org.apache.doris.tablefunction.TableValuedFunctionTask;
@@ -41,32 +41,25 @@ import java.util.List;
 /**
  * This scan node is used for data source generated from memory.
  */
-public class DataGenScanNode extends ScanNode {
+public class DataGenScanNode extends ExternalScanNode {
     private static final Logger LOG = LogManager.getLogger(DataGenScanNode.class.getName());
 
-    private List<TScanRangeLocations> shardScanRanges;
     private DataGenTableValuedFunction tvf;
     private boolean isFinalized = false;
 
     public DataGenScanNode(PlanNodeId id, TupleDescriptor desc, DataGenTableValuedFunction tvf) {
-        super(id, desc, "DataGenScanNode", StatisticalType.TABLE_VALUED_FUNCTION_NODE);
+        super(id, desc, "DataGenScanNode", StatisticalType.TABLE_VALUED_FUNCTION_NODE, false);
         this.tvf = tvf;
     }
 
     @Override
     public void init(Analyzer analyzer) throws UserException {
         super.init(analyzer);
-        computeStats(analyzer);
-    }
-
-    @Override
-    public int getNumInstances() {
-        return shardScanRanges.size();
     }
 
     @Override
     public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
-        return shardScanRanges;
+        return scanRangeLocations;
     }
 
     @Override
@@ -74,12 +67,7 @@ public class DataGenScanNode extends ScanNode {
         if (isFinalized) {
             return;
         }
-        try {
-            shardScanRanges = getShardLocations();
-        } catch (AnalysisException e) {
-            throw new UserException(e.getMessage());
-        }
-
+        createScanRangeLocations();
         isFinalized = true;
     }
 
@@ -92,8 +80,9 @@ public class DataGenScanNode extends ScanNode {
         msg.data_gen_scan_node = dataGenScanNode;
     }
 
-    private List<TScanRangeLocations> getShardLocations() throws AnalysisException {
-        List<TScanRangeLocations> result = Lists.newArrayList();
+    @Override
+    protected void createScanRangeLocations() throws UserException {
+        scanRangeLocations = Lists.newArrayList();
         for (TableValuedFunctionTask task : tvf.getTasks()) {
             TScanRangeLocations locations = new TScanRangeLocations();
             TScanRangeLocation location = new TScanRangeLocation();
@@ -101,19 +90,15 @@ public class DataGenScanNode extends ScanNode {
             location.setServer(new TNetworkAddress(task.getBackend().getHost(), task.getBackend().getBePort()));
             locations.addToLocations(location);
             locations.setScanRange(task.getExecParams());
-            result.add(locations);
+            scanRangeLocations.add(locations);
         }
-        return result;
     }
 
     @Override
     public void finalizeForNereids() {
-        if (shardScanRanges != null) {
-            return;
-        }
         try {
-            shardScanRanges = getShardLocations();
-        } catch (AnalysisException e) {
+            createScanRangeLocations();
+        } catch (UserException e) {
             throw new NereidsException("Can not compute shard locations for DataGenScanNode: " + e.getMessage(), e);
         }
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java
index e2ddb34509..1e80ce17e8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java
@@ -37,6 +37,7 @@ import org.apache.doris.external.elasticsearch.QueryBuilders;
 import org.apache.doris.external.elasticsearch.QueryBuilders.BoolQueryBuilder;
 import org.apache.doris.external.elasticsearch.QueryBuilders.BuilderOptions;
 import org.apache.doris.external.elasticsearch.QueryBuilders.QueryBuilder;
+import org.apache.doris.planner.external.ExternalScanNode;
 import org.apache.doris.planner.external.FederationBackendPolicy;
 import org.apache.doris.statistics.StatisticalType;
 import org.apache.doris.statistics.query.StatsDelta;
@@ -53,7 +54,6 @@ import org.apache.doris.thrift.TScanRangeLocations;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.collect.Multimap;
 import lombok.SneakyThrows;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -63,20 +63,16 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.Set;
 
 /**
  * ScanNode for Elasticsearch.
  **/
-public class EsScanNode extends ScanNode {
+public class EsScanNode extends ExternalScanNode {
 
     private static final Logger LOG = LogManager.getLogger(EsScanNode.class);
 
-    private final Random random = new Random(System.currentTimeMillis());
-    private Multimap<String, Backend> backendMap;
     private EsTablePartitions esTablePartitions;
-    private List<TScanRangeLocations> shardScanRanges = Lists.newArrayList();
     private EsTable table;
     private QueryBuilder queryBuilder;
     private boolean isFinalized = false;
@@ -89,7 +85,7 @@ public class EsScanNode extends ScanNode {
      * For multicatalog es.
      **/
     public EsScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, boolean esExternalTable) {
-        super(id, desc, planNodeName, StatisticalType.ES_SCAN_NODE);
+        super(id, desc, planNodeName, StatisticalType.ES_SCAN_NODE, false);
         if (esExternalTable) {
             EsExternalTable externalTable = (EsExternalTable) (desc.getTable());
             table = externalTable.getEsTable();
@@ -102,54 +98,36 @@ public class EsScanNode extends ScanNode {
     @Override
     public void init(Analyzer analyzer) throws UserException {
         super.init(analyzer);
-        computeColumnFilter();
-        computeStats(analyzer);
         buildQuery();
     }
 
+    @Override
     public void init() throws UserException {
-        computeColumnFilter();
+        super.init();
         buildQuery();
     }
 
     @Override
-    public int getNumInstances() {
-        return shardScanRanges.size();
+    public void finalize(Analyzer analyzer) throws UserException {
+        doFinalize();
     }
 
     @Override
-    public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
-        return shardScanRanges;
+    public void finalizeForNereids() throws UserException {
+        doFinalize();
     }
 
-    @Override
-    public void finalize(Analyzer analyzer) throws UserException {
+    private void doFinalize() throws UserException {
         if (isFinalized) {
             return;
         }
-
-        try {
-            shardScanRanges = getShardLocations();
-        } catch (AnalysisException e) {
-            throw new UserException(e.getMessage());
-        }
-
+        createScanRangeLocations();
         isFinalized = true;
     }
 
     @Override
-    public void finalizeForNereids() throws UserException {
-        if (isFinalized) {
-            return;
-        }
-
-        try {
-            shardScanRanges = getShardLocations();
-        } catch (AnalysisException e) {
-            throw new UserException(e.getMessage());
-        }
-
-        isFinalized = true;
+    protected void createScanRangeLocations() throws UserException {
+        scanRangeLocations = getShardLocations();
     }
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/FileLoadScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/FileLoadScanNode.java
index ef581280c0..c20a916bf9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/FileLoadScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/FileLoadScanNode.java
@@ -40,11 +40,13 @@ import org.apache.doris.catalog.TableIf;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.UserException;
 import org.apache.doris.load.BrokerFileGroup;
+import org.apache.doris.planner.external.FederationBackendPolicy;
 import org.apache.doris.planner.external.FileGroupInfo;
 import org.apache.doris.planner.external.FileScanNode;
 import org.apache.doris.planner.external.LoadScanProvider;
 import org.apache.doris.rewrite.ExprRewriter;
 import org.apache.doris.statistics.StatisticalType;
+import org.apache.doris.system.BeSelectionPolicy;
 import org.apache.doris.thrift.TBrokerFileStatus;
 import org.apache.doris.thrift.TFileScanRangeParams;
 import org.apache.doris.thrift.TFileType;
@@ -120,8 +122,6 @@ public class FileLoadScanNode extends FileScanNode {
         for (FileGroupInfo fileGroupInfo : fileGroupInfos) {
             this.scanProviders.add(new LoadScanProvider(fileGroupInfo, desc));
         }
-        backendPolicy.init();
-        numNodes = backendPolicy.numBackends();
         initParamCreateContexts(analyzer);
     }
 
@@ -193,18 +193,39 @@ public class FileLoadScanNode extends FileScanNode {
     public void finalize(Analyzer analyzer) throws UserException {
         Preconditions.checkState(contexts.size() == scanProviders.size(),
                 contexts.size() + " vs. " + scanProviders.size());
+        // ATTN: for load scan node, do not use backend policy in ExternalScanNode.
+        // Because backend policy in ExternalScanNode may only contain compute backend.
+        // But for load job, we should select backends from all backends, both compute and mix.
+        BeSelectionPolicy policy = new BeSelectionPolicy.Builder()
+                .needQueryAvailable()
+                .needLoadAvailable()
+                .build();
+        FederationBackendPolicy localBackendPolicy = new FederationBackendPolicy();
+        localBackendPolicy.init(policy);
         for (int i = 0; i < contexts.size(); ++i) {
             FileLoadScanNode.ParamCreateContext context = contexts.get(i);
             LoadScanProvider scanProvider = scanProviders.get(i);
             finalizeParamsForLoad(context, analyzer);
-            createScanRangeLocations(context, scanProvider);
+            createScanRangeLocations(context, scanProvider, localBackendPolicy);
             this.inputSplitsNum += scanProvider.getInputSplitNum();
             this.totalFileSize += scanProvider.getInputFileSize();
         }
     }
 
+    // TODO: This api is for load job only. Will remove it later.
+    private void createScanRangeLocations(FileLoadScanNode.ParamCreateContext context,
+            LoadScanProvider scanProvider, FederationBackendPolicy backendPolicy)
+            throws UserException {
+        scanProvider.createScanRangeLocations(context, backendPolicy, scanRangeLocations);
+    }
+
+    @Override
+    protected void createScanRangeLocations() throws UserException {
+        // do nothing, we have already created scan range locations in finalize
+    }
+
     protected void finalizeParamsForLoad(ParamCreateContext context,
-                                         Analyzer analyzer) throws UserException {
+            Analyzer analyzer) throws UserException {
         Map<String, SlotDescriptor> slotDescByName = context.srcSlotDescByName;
         Map<String, Expr> exprMap = context.exprMap;
         TupleDescriptor srcTupleDesc = context.srcTupleDescriptor;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/JdbcScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/JdbcScanNode.java
index c839d5ea95..34bc96255a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/JdbcScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/JdbcScanNode.java
@@ -22,6 +22,7 @@ import org.apache.doris.analysis.Expr;
 import org.apache.doris.analysis.ExprSubstitutionMap;
 import org.apache.doris.analysis.FunctionCallExpr;
 import org.apache.doris.analysis.SlotDescriptor;
+import org.apache.doris.analysis.SlotId;
 import org.apache.doris.analysis.SlotRef;
 import org.apache.doris.analysis.TupleDescriptor;
 import org.apache.doris.catalog.Column;
@@ -31,6 +32,8 @@ import org.apache.doris.catalog.OdbcTable;
 import org.apache.doris.catalog.external.JdbcExternalTable;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.UserException;
+import org.apache.doris.nereids.glue.translator.PlanTranslatorContext;
+import org.apache.doris.planner.external.ExternalScanNode;
 import org.apache.doris.statistics.StatisticalType;
 import org.apache.doris.statistics.StatsRecursiveDerive;
 import org.apache.doris.statistics.query.StatsDelta;
@@ -39,7 +42,6 @@ import org.apache.doris.thrift.TJdbcScanNode;
 import org.apache.doris.thrift.TOdbcTableType;
 import org.apache.doris.thrift.TPlanNode;
 import org.apache.doris.thrift.TPlanNodeType;
-import org.apache.doris.thrift.TScanRangeLocations;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.MoreObjects;
@@ -49,8 +51,9 @@ import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
-public class JdbcScanNode extends ScanNode {
+public class JdbcScanNode extends ExternalScanNode {
     private static final Logger LOG = LogManager.getLogger(JdbcScanNode.class);
 
     private final List<String> columns = new ArrayList<String>();
@@ -62,7 +65,7 @@ public class JdbcScanNode extends ScanNode {
     private JdbcTable tbl;
 
     public JdbcScanNode(PlanNodeId id, TupleDescriptor desc, boolean isJdbcExternalTable) {
-        super(id, desc, "JdbcScanNode", StatisticalType.JDBC_SCAN_NODE);
+        super(id, desc, "JdbcScanNode", StatisticalType.JDBC_SCAN_NODE, false);
         if (isJdbcExternalTable) {
             JdbcExternalTable jdbcExternalTable = (JdbcExternalTable) (desc.getTable());
             tbl = jdbcExternalTable.getJdbcTable();
@@ -76,10 +79,20 @@ public class JdbcScanNode extends ScanNode {
     @Override
     public void init(Analyzer analyzer) throws UserException {
         super.init(analyzer);
-        computeStats(analyzer);
         getGraphQueryString();
     }
 
+    /**
+     * Used for Nereids. Should NOT use this function in anywhere else.
+     */
+    @Override
+    public void init() throws UserException {
+        super.init();
+        numNodes = numNodes <= 0 ? 1 : numNodes;
+        StatsRecursiveDerive.getStatsRecursiveDerive().statsRecursiveDerive(this);
+        cardinality = (long) statsDeriveResult.getRowCount();
+    }
+
     private boolean isNebula() {
         return jdbcType == TOdbcTableType.NEBULA;
     }
@@ -99,20 +112,6 @@ public class JdbcScanNode extends ScanNode {
         conjuncts = Lists.newArrayList();
     }
 
-    /**
-     * Used for Nereids. Should NOT use this function in anywhere else.
-     */
-    public void init() throws UserException {
-        numNodes = numNodes <= 0 ? 1 : numNodes;
-        StatsRecursiveDerive.getStatsRecursiveDerive().statsRecursiveDerive(this);
-        cardinality = (long) statsDeriveResult.getRowCount();
-    }
-
-    @Override
-    public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
-        return null;
-    }
-
     private void createJdbcFilters() {
         if (conjuncts.isEmpty()) {
             return;
@@ -139,6 +138,7 @@ public class JdbcScanNode extends ScanNode {
     }
 
     private void createJdbcColumns() {
+        columns.clear();
         for (SlotDescriptor slot : desc.getSlots()) {
             if (!slot.isMaterialized()) {
                 continue;
@@ -212,12 +212,25 @@ public class JdbcScanNode extends ScanNode {
         // Convert predicates to Jdbc columns and filters.
         createJdbcColumns();
         createJdbcFilters();
+        createScanRangeLocations();
     }
 
     @Override
     public void finalizeForNereids() throws UserException {
         createJdbcColumns();
         createJdbcFilters();
+        createScanRangeLocations();
+    }
+
+    @Override
+    public void updateRequiredSlots(PlanTranslatorContext context, Set<SlotId> requiredByProjectSlotIdSet)
+            throws UserException {
+        createJdbcColumns();
+    }
+
+    @Override
+    protected void createScanRangeLocations() throws UserException {
+        scanRangeLocations = Lists.newArrayList(createSingleScanRangeLocations(backendPolicy));
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/MysqlScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/MysqlScanNode.java
index a04b531159..506e95c4b8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/MysqlScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/MysqlScanNode.java
@@ -26,13 +26,13 @@ import org.apache.doris.analysis.TupleDescriptor;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.MysqlTable;
 import org.apache.doris.common.UserException;
+import org.apache.doris.planner.external.ExternalScanNode;
 import org.apache.doris.statistics.StatisticalType;
 import org.apache.doris.statistics.StatsRecursiveDerive;
 import org.apache.doris.thrift.TExplainLevel;
 import org.apache.doris.thrift.TMySQLScanNode;
 import org.apache.doris.thrift.TPlanNode;
 import org.apache.doris.thrift.TPlanNodeType;
-import org.apache.doris.thrift.TScanRangeLocations;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.MoreObjects;
@@ -46,7 +46,7 @@ import java.util.List;
 /**
  * Full scan of an MySQL table.
  */
-public class MysqlScanNode extends ScanNode {
+public class MysqlScanNode extends ExternalScanNode {
     private static final Logger LOG = LogManager.getLogger(MysqlScanNode.class);
 
     private final List<String> columns = new ArrayList<String>();
@@ -57,16 +57,10 @@ public class MysqlScanNode extends ScanNode {
      * Constructs node to scan given data files of table 'tbl'.
      */
     public MysqlScanNode(PlanNodeId id, TupleDescriptor desc, MysqlTable tbl) {
-        super(id, desc, "SCAN MYSQL", StatisticalType.MYSQL_SCAN_NODE);
+        super(id, desc, "SCAN MYSQL", StatisticalType.MYSQL_SCAN_NODE, false);
         tblName = "`" + tbl.getMysqlTableName() + "`";
     }
 
-    @Override
-    public void init(Analyzer analyzer) throws UserException {
-        super.init(analyzer);
-        computeStats(analyzer);
-    }
-
     @Override
     protected String debugString() {
         MoreObjects.ToStringHelper helper = MoreObjects.toStringHelper(this);
@@ -78,6 +72,12 @@ public class MysqlScanNode extends ScanNode {
         // Convert predicates to MySQL columns and filters.
         createMySQLColumns(analyzer);
         createMySQLFilters(analyzer);
+        createScanRangeLocations();
+    }
+
+    @Override
+    protected void createScanRangeLocations() throws UserException {
+        scanRangeLocations = Lists.newArrayList(createSingleScanRangeLocations(backendPolicy));
     }
 
     @Override
@@ -150,15 +150,6 @@ public class MysqlScanNode extends ScanNode {
         msg.mysql_scan_node = new TMySQLScanNode(desc.getId().asInt(), tblName, columns, filters);
     }
 
-    /**
-     * We query MySQL Meta to get request's data location
-     * extra result info will pass to backend ScanNode
-     */
-    @Override
-    public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
-        return null;
-    }
-
     @Override
     public int getNumInstances() {
         return 1;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OdbcScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OdbcScanNode.java
index 32f8fe2f65..e21af36af1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/OdbcScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OdbcScanNode.java
@@ -33,6 +33,7 @@ import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.UserException;
+import org.apache.doris.planner.external.ExternalScanNode;
 import org.apache.doris.statistics.StatisticalType;
 import org.apache.doris.statistics.StatsRecursiveDerive;
 import org.apache.doris.statistics.query.StatsDelta;
@@ -41,7 +42,6 @@ import org.apache.doris.thrift.TOdbcScanNode;
 import org.apache.doris.thrift.TOdbcTableType;
 import org.apache.doris.thrift.TPlanNode;
 import org.apache.doris.thrift.TPlanNodeType;
-import org.apache.doris.thrift.TScanRangeLocations;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.MoreObjects;
@@ -55,7 +55,7 @@ import java.util.List;
 /**
  * Full scan of an ODBC table.
  */
-public class OdbcScanNode extends ScanNode {
+public class OdbcScanNode extends ExternalScanNode {
     private static final Logger LOG = LogManager.getLogger(OdbcScanNode.class);
 
     // Now some database have different function call like doris, now doris do not
@@ -100,19 +100,13 @@ public class OdbcScanNode extends ScanNode {
      * Constructs node to scan given data files of table 'tbl'.
      */
     public OdbcScanNode(PlanNodeId id, TupleDescriptor desc, OdbcTable tbl) {
-        super(id, desc, "SCAN ODBC", StatisticalType.ODBC_SCAN_NODE);
+        super(id, desc, "SCAN ODBC", StatisticalType.ODBC_SCAN_NODE, false);
         connectString = tbl.getConnectString();
         odbcType = tbl.getOdbcTableType();
         tblName = OdbcTable.databaseProperName(odbcType, tbl.getOdbcTableName());
         this.tbl = tbl;
     }
 
-    @Override
-    public void init(Analyzer analyzer) throws UserException {
-        super.init(analyzer);
-        computeStats(analyzer);
-    }
-
     @Override
     protected String debugString() {
         MoreObjects.ToStringHelper helper = MoreObjects.toStringHelper(this);
@@ -124,6 +118,12 @@ public class OdbcScanNode extends ScanNode {
         // Convert predicates to Odbc columns and filters.
         createOdbcColumns(analyzer);
         createOdbcFilters(analyzer);
+        createScanRangeLocations();
+    }
+
+    @Override
+    protected void createScanRangeLocations() throws UserException {
+        scanRangeLocations = Lists.newArrayList(createSingleScanRangeLocations(backendPolicy));
     }
 
     @Override
@@ -229,20 +229,6 @@ public class OdbcScanNode extends ScanNode {
         msg.odbc_scan_node = odbcScanNode;
     }
 
-    /**
-     * We query Odbc Meta to get request's data location
-     * extra result info will pass to backend ScanNode
-     */
-    @Override
-    public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
-        return null;
-    }
-
-    @Override
-    public int getNumInstances() {
-        return 1;
-    }
-
     @Override
     public void computeStats(Analyzer analyzer) throws UserException {
         super.computeStats(analyzer);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java
index 908b428aa5..80bac9fb55 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java
@@ -111,7 +111,6 @@ public class OlapScanNode extends ScanNode {
     // average compression ratio in doris storage engine
     private static final int COMPRESSION_RATIO = 5;
 
-    private List<TScanRangeLocations> result = new ArrayList<>();
     /*
      * When the field value is ON, the storage engine can return the data directly
      * without pre-aggregation.
@@ -569,7 +568,7 @@ public class OlapScanNode extends ScanNode {
     public void init() throws UserException {
         selectedPartitionNum = selectedPartitionIds.size();
         try {
-            getScanRangeLocations();
+            createScanRangeLocations();
         } catch (AnalysisException e) {
             throw new UserException(e.getMessage());
         }
@@ -603,7 +602,7 @@ public class OlapScanNode extends ScanNode {
         // prepare stmt evaluate lazily in Coordinator execute
         if (!isFromPrepareStmt) {
             try {
-                getScanRangeLocations();
+                createScanRangeLocations();
             } catch (AnalysisException e) {
                 throw new UserException(e.getMessage());
             }
@@ -730,7 +729,7 @@ public class OlapScanNode extends ScanNode {
                     visibleVersionStr = String.valueOf(visibleVersion);
                 }
             }
-            TScanRangeLocations scanRangeLocations = new TScanRangeLocations();
+            TScanRangeLocations locations = new TScanRangeLocations();
             TPaloScanRange paloRange = new TPaloScanRange();
             paloRange.setDbName("");
             paloRange.setSchemaHash("0");
@@ -796,7 +795,7 @@ public class OlapScanNode extends ScanNode {
                 int port = backend.getBePort();
                 TScanRangeLocation scanRangeLocation = new TScanRangeLocation(new TNetworkAddress(ip, port));
                 scanRangeLocation.setBackendId(replica.getBackendId());
-                scanRangeLocations.addToLocations(scanRangeLocation);
+                locations.addToLocations(scanRangeLocation);
                 paloRange.addToHosts(new TNetworkAddress(ip, port));
                 tabletIsNull = false;
 
@@ -817,11 +816,11 @@ public class OlapScanNode extends ScanNode {
             }
             TScanRange scanRange = new TScanRange();
             scanRange.setPaloScanRange(paloRange);
-            scanRangeLocations.setScanRange(scanRange);
+            locations.setScanRange(scanRange);
 
-            bucketSeq2locations.put(tabletId2BucketSeq.get(tabletId), scanRangeLocations);
+            bucketSeq2locations.put(tabletId2BucketSeq.get(tabletId), locations);
 
-            result.add(scanRangeLocations);
+            scanRangeLocations.add(locations);
         }
 
         if (tablets.size() == 0) {
@@ -884,11 +883,13 @@ public class OlapScanNode extends ScanNode {
         final RollupSelector rollupSelector = new RollupSelector(analyzer, desc, olapTable);
         selectedIndexId = rollupSelector.selectBestRollup(selectedPartitionIds, conjuncts, isPreAggregation);
         updateSlotUniqueId();
-        LOG.debug("select best roll up cost: {} ms, best index id: {}",
-                (System.currentTimeMillis() - start), selectedIndexId);
+        LOG.debug("select best roll up cost: {} ms, best index id: {}", (System.currentTimeMillis() - start),
+                selectedIndexId);
     }
 
-    private void getScanRangeLocations() throws UserException {
+    @Override
+    protected void createScanRangeLocations() throws UserException {
+        scanRangeLocations = Lists.newArrayList();
         if (selectedPartitionIds.size() == 0) {
             desc.setCardinality(0);
             return;
@@ -1088,7 +1089,7 @@ public class OlapScanNode extends ScanNode {
      */
     @Override
     public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
-        return result;
+        return scanRangeLocations;
     }
 
     // Only called when Coordinator exec in high performance point query
@@ -1100,14 +1101,13 @@ public class OlapScanNode extends ScanNode {
         computePartitionInfo();
         scanBackendIds.clear();
         scanTabletIds.clear();
-        result.clear();
         bucketSeq2locations.clear();
         try {
-            getScanRangeLocations();
+            createScanRangeLocations();
         } catch (AnalysisException e) {
             throw new UserException(e.getMessage());
         }
-        return result;
+        return scanRangeLocations;
     }
 
     public void setDescTable(DescriptorTable descTable) {
@@ -1207,11 +1207,11 @@ public class OlapScanNode extends ScanNode {
         // so here we need count distinct be_num to do the work. make sure get right instance
         if (ConnectContext.get().getSessionVariable().enablePipelineEngine()) {
             int parallelInstance = ConnectContext.get().getSessionVariable().getParallelExecInstanceNum();
-            long numBackend = result.stream().flatMap(rangeLoc -> rangeLoc.getLocations().stream())
-                        .map(loc -> loc.backend_id).distinct().count();
+            long numBackend = scanRangeLocations.stream().flatMap(rangeLoc -> rangeLoc.getLocations().stream())
+                    .map(loc -> loc.backend_id).distinct().count();
             return (int) (parallelInstance * numBackend);
         }
-        return result.size();
+        return scanRangeLocations.size();
     }
 
     @Override
@@ -1353,22 +1353,6 @@ public class OlapScanNode extends ScanNode {
         }
     }
 
-    // export some tablets
-    public static OlapScanNode createOlapScanNodeByLocation(
-            PlanNodeId id, TupleDescriptor desc, String planNodeName, List<TScanRangeLocations> locationsList) {
-        OlapScanNode olapScanNode = new OlapScanNode(id, desc, planNodeName);
-        olapScanNode.numInstances = 1;
-
-        olapScanNode.selectedIndexId = olapScanNode.olapTable.getBaseIndexId();
-        olapScanNode.selectedPartitionNum = 1;
-        olapScanNode.selectedTabletsNum = 1;
-        olapScanNode.totalTabletsNum = 1;
-        olapScanNode.setIsPreAggregation(false, "Export job");
-        olapScanNode.result.addAll(locationsList);
-
-        return olapScanNode;
-    }
-
     public void collectColumns(Analyzer analyzer, Set<String> equivalenceColumns, Set<String> unequivalenceColumns) {
         // 1. Get columns which has predicate on it.
         for (Expr expr : conjuncts) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java
index e63662b053..7dc0a7b3de 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java
@@ -41,10 +41,14 @@ import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.NotImplementedException;
 import org.apache.doris.common.UserException;
 import org.apache.doris.nereids.glue.translator.PlanTranslatorContext;
+import org.apache.doris.planner.external.FederationBackendPolicy;
 import org.apache.doris.spi.Split;
 import org.apache.doris.statistics.StatisticalType;
 import org.apache.doris.statistics.query.StatsDelta;
+import org.apache.doris.system.Backend;
 import org.apache.doris.thrift.TNetworkAddress;
+import org.apache.doris.thrift.TScanRange;
+import org.apache.doris.thrift.TScanRangeLocation;
 import org.apache.doris.thrift.TScanRangeLocations;
 
 import com.google.common.base.MoreObjects;
@@ -74,6 +78,7 @@ public abstract class ScanNode extends PlanNode {
     protected Map<String, ColumnRange> columnNameToRange = Maps.newHashMap();
     protected String sortColumn = null;
     protected Analyzer analyzer;
+    protected List<TScanRangeLocations> scanRangeLocations = Lists.newArrayList();
 
     public ScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, StatisticalType statisticalType) {
         super(id, desc.getId().asList(), planNodeName, statisticalType);
@@ -128,6 +133,8 @@ public abstract class ScanNode extends PlanNode {
         }
     }
 
+    protected abstract void createScanRangeLocations() throws UserException;
+
     /**
      * Returns all scan ranges plus their locations. Needs to be preceded by a call to
      * finalize().
@@ -170,7 +177,10 @@ public abstract class ScanNode extends PlanNode {
     }
 
     public void computeColumnFilter() {
-        computeColumnFilter(desc.getTable().getBaseSchema());
+        // for load scan node, table is null
+        if (desc.getTable() != null) {
+            computeColumnFilter(desc.getTable().getBaseSchema());
+        }
     }
 
     public static ColumnRange createColumnRange(SlotDescriptor desc,
@@ -593,4 +603,17 @@ public abstract class ScanNode extends PlanNode {
         }
         return delta;
     }
+
+    // Create a single scan range locations for the given backend policy.
+    // Used for those scan nodes which do not require data location.
+    public static TScanRangeLocations createSingleScanRangeLocations(FederationBackendPolicy backendPolicy) {
+        TScanRangeLocations scanRangeLocation = new TScanRangeLocations();
+        scanRangeLocation.setScanRange(new TScanRange());
+        TScanRangeLocation location = new TScanRangeLocation();
+        Backend be = backendPolicy.getNextBe();
+        location.setServer(new TNetworkAddress(be.getHost(), be.getBePort()));
+        location.setBackendId(be.getId());
+        scanRangeLocation.addToLocations(location);
+        return scanRangeLocation;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/SchemaScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/SchemaScanNode.java
index 91867f84c6..595d09792f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/SchemaScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/SchemaScanNode.java
@@ -24,6 +24,7 @@ import org.apache.doris.common.Config;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.Util;
 import org.apache.doris.datasource.InternalCatalog;
+import org.apache.doris.planner.external.FederationBackendPolicy;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.service.FrontendOptions;
 import org.apache.doris.statistics.StatisticalType;
@@ -34,6 +35,7 @@ import org.apache.doris.thrift.TSchemaScanNode;
 import org.apache.doris.thrift.TUserIdentity;
 
 import com.google.common.base.MoreObjects;
+import com.google.common.collect.Lists;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -127,13 +129,16 @@ public class SchemaScanNode extends ScanNode {
         msg.schema_scan_node.setCurrentUserIdent(tCurrentUser);
     }
 
-    /**
-     * We query MySQL Meta to get request's data location
-     * extra result info will pass to backend ScanNode
-     */
     @Override
     public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
-        return null;
+        return scanRangeLocations;
+    }
+
+    @Override
+    protected void createScanRangeLocations() throws UserException {
+        FederationBackendPolicy backendPolicy = new FederationBackendPolicy();
+        backendPolicy.init();
+        scanRangeLocations = Lists.newArrayList(createSingleScanRangeLocations(backendPolicy));
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/TestExternalTableScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/TestExternalTableScanNode.java
index f537d091be..2ce78391e7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/TestExternalTableScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/TestExternalTableScanNode.java
@@ -20,40 +20,28 @@ package org.apache.doris.planner;
 import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.TupleDescriptor;
 import org.apache.doris.common.UserException;
+import org.apache.doris.planner.external.ExternalScanNode;
 import org.apache.doris.statistics.StatisticalType;
 import org.apache.doris.statistics.StatsRecursiveDerive;
 import org.apache.doris.thrift.TExplainLevel;
 import org.apache.doris.thrift.TPlanNode;
 import org.apache.doris.thrift.TPlanNodeType;
-import org.apache.doris.thrift.TScanRangeLocations;
 import org.apache.doris.thrift.TTestExternalScanNode;
 
 import com.google.common.base.MoreObjects;
+import com.google.common.collect.Lists;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import java.util.List;
-
-public class TestExternalTableScanNode extends ScanNode {
+public class TestExternalTableScanNode extends ExternalScanNode {
     private static final Logger LOG = LogManager.getLogger(TestExternalTableScanNode.class);
     private String tableName;
 
     public TestExternalTableScanNode(PlanNodeId id, TupleDescriptor desc) {
-        super(id, desc, "TestExternalTableScanNode", StatisticalType.TEST_EXTERNAL_TABLE);
+        super(id, desc, "TestExternalTableScanNode", StatisticalType.TEST_EXTERNAL_TABLE, true);
         tableName = desc.getTable().getName();
     }
 
-    @Override
-    public void init(Analyzer analyzer) throws UserException {
-        super.init(analyzer);
-        computeStats(analyzer);
-    }
-
-    @Override
-    public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
-        return null;
-    }
-
     @Override
     public String getNodeExplainString(String prefix, TExplainLevel detailLevel) {
         StringBuilder output = new StringBuilder();
@@ -63,6 +51,12 @@ public class TestExternalTableScanNode extends ScanNode {
 
     @Override
     public void finalize(Analyzer analyzer) throws UserException {
+        createScanRangeLocations();
+    }
+
+    @Override
+    protected void createScanRangeLocations() throws UserException {
+        scanRangeLocations = Lists.newArrayList(createSingleScanRangeLocations(backendPolicy));
     }
 
     @Override
@@ -87,9 +81,4 @@ public class TestExternalTableScanNode extends ScanNode {
         MoreObjects.ToStringHelper helper = MoreObjects.toStringHelper(this);
         return helper.addValue(super.debugString()).toString();
     }
-
-    @Override
-    public int getNumInstances() {
-        return 1;
-    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/ExternalScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/ExternalScanNode.java
index 24137e1351..2fda6fbd28 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/ExternalScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/ExternalScanNode.java
@@ -17,13 +17,14 @@
 
 package org.apache.doris.planner.external;
 
+import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.TupleDescriptor;
+import org.apache.doris.common.UserException;
 import org.apache.doris.planner.PlanNodeId;
 import org.apache.doris.planner.ScanNode;
 import org.apache.doris.statistics.StatisticalType;
 import org.apache.doris.thrift.TScanRangeLocations;
 
-import com.google.common.collect.Lists;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -42,8 +43,7 @@ public abstract class ExternalScanNode extends ScanNode {
     // set to false means this scan node does not need to check column priv.
     protected boolean needCheckColumnPriv;
 
-    // Final output of this file scan node
-    protected List<TScanRangeLocations> scanRangeLocations = Lists.newArrayList();
+    protected final FederationBackendPolicy backendPolicy = new FederationBackendPolicy();
 
     public ExternalScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, StatisticalType statisticalType,
             boolean needCheckColumnPriv) {
@@ -51,6 +51,30 @@ public abstract class ExternalScanNode extends ScanNode {
         this.needCheckColumnPriv = needCheckColumnPriv;
     }
 
+    @Override
+    public void init(Analyzer analyzer) throws UserException {
+        super.init(analyzer);
+        computeStats(analyzer);
+        computeColumnFilter();
+        initBackendPolicy();
+    }
+
+    // For Nereids
+    @Override
+    public void init() throws UserException {
+        computeColumnFilter();
+        initBackendPolicy();
+    }
+
+    protected void initBackendPolicy() throws UserException {
+        backendPolicy.init();
+        numNodes = backendPolicy.numBackends();
+    }
+
+    public FederationBackendPolicy getBackendPolicy() {
+        return backendPolicy;
+    }
+
     @Override
     public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
         LOG.debug("There is {} scanRangeLocations for execution.", scanRangeLocations.size());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FederationBackendPolicy.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FederationBackendPolicy.java
index aee9252e2e..91300be1ad 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FederationBackendPolicy.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FederationBackendPolicy.java
@@ -66,6 +66,10 @@ public class FederationBackendPolicy {
                 .assignExpectBeNum(Config.min_backend_num_for_external_table)
                 .addPreLocations(preLocations)
                 .build();
+        init(policy);
+    }
+
+    public void init(BeSelectionPolicy policy) throws UserException {
         backends.addAll(policy.getCandidateBackends(Env.getCurrentSystemInfo().getIdToBackend().values()));
         if (backends.isEmpty()) {
             throw new UserException("No available backends");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java
index f83c2529c2..b07a48da2b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java
@@ -101,6 +101,7 @@ public abstract class FileQueryScanNode extends FileScanNode {
     /**
      * Init ExternalFileScanNode, ONLY used for Nereids. Should NOT use this function in anywhere else.
      */
+    @Override
     public void init() throws UserException {
         doInitialize();
     }
@@ -108,10 +109,13 @@ public abstract class FileQueryScanNode extends FileScanNode {
     // Init scan provider and schema related params.
     protected void doInitialize() throws UserException {
         Preconditions.checkNotNull(desc);
-        ExternalTable table = (ExternalTable) desc.getTable();
-        if (table.isView()) {
-            throw new AnalysisException(
-                String.format("Querying external view '%s.%s' is not supported", table.getDbName(), table.getName()));
+        if (desc.getTable() instanceof ExternalTable) {
+            ExternalTable table = (ExternalTable) desc.getTable();
+            if (table.isView()) {
+                throw new AnalysisException(
+                        String.format("Querying external view '%s.%s' is not supported", table.getDbName(),
+                                table.getName()));
+            }
         }
         computeColumnFilter();
         initBackendPolicy();
@@ -144,11 +148,6 @@ public abstract class FileQueryScanNode extends FileScanNode {
         params.setSrcTupleId(-1);
     }
 
-    protected void initBackendPolicy() throws UserException {
-        backendPolicy.init();
-        numNodes = backendPolicy.numBackends();
-    }
-
     /**
      * Reset required_slots in contexts. This is called after Nereids planner do the projection.
      * In the projection process, some slots may be removed. So call this to update the slots info.
@@ -211,6 +210,7 @@ public abstract class FileQueryScanNode extends FileScanNode {
         params.setColumnIdxs(columnIdxs);
     }
 
+    @Override
     public void createScanRangeLocations() throws UserException {
         long start = System.currentTimeMillis();
         List<Split> inputSplits = getSplits();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileScanNode.java
index aa0e923203..694e97a7d9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileScanNode.java
@@ -25,7 +25,6 @@ import org.apache.doris.analysis.TupleDescriptor;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.TableIf;
 import org.apache.doris.common.UserException;
-import org.apache.doris.planner.FileLoadScanNode;
 import org.apache.doris.planner.PlanNodeId;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.spi.Split;
@@ -57,7 +56,7 @@ import java.util.Map;
 /**
  * Base class for External File Scan, including external query and load.
  */
-public class FileScanNode extends ExternalScanNode {
+public abstract class FileScanNode extends ExternalScanNode {
     private static final Logger LOG = LogManager.getLogger(FileScanNode.class);
 
     public static final long DEFAULT_SPLIT_SIZE = 128 * 1024 * 1024; // 128MB
@@ -68,8 +67,6 @@ public class FileScanNode extends ExternalScanNode {
     protected long totalPartitionNum = 0;
     protected long readPartitionNum = 0;
 
-    protected final FederationBackendPolicy backendPolicy = new FederationBackendPolicy();
-
     public FileScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, StatisticalType statisticalType,
                             boolean needCheckColumnPriv) {
         super(id, desc, planNodeName, statisticalType, needCheckColumnPriv);
@@ -159,13 +156,6 @@ public class FileScanNode extends ExternalScanNode {
         return output.toString();
     }
 
-    // TODO: This api is for load job only. Will remove it later.
-    protected void createScanRangeLocations(FileLoadScanNode.ParamCreateContext context,
-                                            LoadScanProvider scanProvider)
-            throws UserException {
-        scanProvider.createScanRangeLocations(context, backendPolicy, scanRangeLocations);
-    }
-
     protected void setDefaultValueExprs(TableIf tbl,
                                         Map<String, SlotDescriptor> slotDescByName,
                                         TFileScanRangeParams params,
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/MetadataScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/MetadataScanNode.java
index ba48ab5ed2..d16126fd8f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/MetadataScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/MetadataScanNode.java
@@ -21,7 +21,6 @@ import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.TupleDescriptor;
 import org.apache.doris.common.UserException;
 import org.apache.doris.planner.PlanNodeId;
-import org.apache.doris.planner.ScanNode;
 import org.apache.doris.statistics.StatisticalType;
 import org.apache.doris.system.Backend;
 import org.apache.doris.tablefunction.MetadataTableValuedFunction;
@@ -37,25 +36,17 @@ import com.google.common.collect.Lists;
 
 import java.util.List;
 
-public class MetadataScanNode extends ScanNode {
+public class MetadataScanNode extends ExternalScanNode {
 
     private MetadataTableValuedFunction tvf;
 
     private List<TScanRangeLocations> scanRangeLocations = Lists.newArrayList();
 
-    private final FederationBackendPolicy backendPolicy = new FederationBackendPolicy();
-
     public MetadataScanNode(PlanNodeId id, TupleDescriptor desc, MetadataTableValuedFunction tvf) {
-        super(id, desc, "METADATA_SCAN_NODE", StatisticalType.METADATA_SCAN_NODE);
+        super(id, desc, "METADATA_SCAN_NODE", StatisticalType.METADATA_SCAN_NODE, false);
         this.tvf = tvf;
     }
 
-    @Override
-    public void init(Analyzer analyzer) throws UserException {
-        super.init(analyzer);
-        backendPolicy.init();
-    }
-
     @Override
     protected void toThrift(TPlanNode planNode) {
         planNode.setNodeType(TPlanNodeType.META_SCAN_NODE);
@@ -65,6 +56,23 @@ public class MetadataScanNode extends ScanNode {
         planNode.setMetaScanNode(metaScanNode);
     }
 
+    @Override
+    protected void createScanRangeLocations() throws UserException {
+        TScanRange scanRange = new TScanRange();
+        scanRange.setMetaScanRange(tvf.getMetaScanRange());
+        // set location
+        TScanRangeLocation location = new TScanRangeLocation();
+        Backend backend = backendPolicy.getNextBe();
+        location.setBackendId(backend.getId());
+        location.setServer(new TNetworkAddress(backend.getHost(), backend.getBePort()));
+
+        TScanRangeLocations locations = new TScanRangeLocations();
+        locations.addToLocations(location);
+        locations.setScanRange(scanRange);
+
+        scanRangeLocations.add(locations);
+    }
+
     @Override
     public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
         return scanRangeLocations;
@@ -72,7 +80,7 @@ public class MetadataScanNode extends ScanNode {
 
     @Override
     public void finalize(Analyzer analyzer) throws UserException {
-        buildScanRanges();
+        createScanRangeLocations();
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/TVFScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/TVFScanNode.java
index 9ec6c6c618..0dfb78abed 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/TVFScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/TVFScanNode.java
@@ -34,7 +34,6 @@ import org.apache.doris.thrift.TFileCompressType;
 import org.apache.doris.thrift.TFileFormatType;
 import org.apache.doris.thrift.TFileType;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import org.apache.hadoop.fs.Path;
 import org.apache.logging.log4j.LogManager;
@@ -62,14 +61,6 @@ public class TVFScanNode extends FileQueryScanNode {
         tableValuedFunction = (ExternalFileTableValuedFunction) table.getTvf();
     }
 
-    @Override
-    protected void doInitialize() throws UserException {
-        Preconditions.checkNotNull(desc);
-        computeColumnFilter();
-        initBackendPolicy();
-        initSchemaParams();
-    }
-
     @Override
     protected String getFsName(FileSplit split) {
         return tableValuedFunction.getFsName();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/hudi/HudiScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/hudi/HudiScanNode.java
index aa7a9c50cc..a0277b0377 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/hudi/HudiScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/hudi/HudiScanNode.java
@@ -102,11 +102,11 @@ public class HudiScanNode extends HiveScanNode {
         }
         computeColumnFilter();
         initBackendPolicy();
+        initSchemaParams();
         if (table instanceof HMSExternalTable) {
             source = new HudiHMSSource((HMSExternalTable) table, desc, columnNameToRange);
         }
         Preconditions.checkNotNull(source);
-        initSchemaParams();
     }
 
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanNode.java
index 5038ecb476..09320e6f59 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanNode.java
@@ -24,7 +24,6 @@ import org.apache.doris.catalog.TableIf;
 import org.apache.doris.catalog.external.ExternalTable;
 import org.apache.doris.catalog.external.HMSExternalTable;
 import org.apache.doris.catalog.external.IcebergExternalTable;
-import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.S3Util;
@@ -83,17 +82,8 @@ public class IcebergScanNode extends FileQueryScanNode {
      */
     public IcebergScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckColumnPriv) {
         super(id, desc, "ICEBERG_SCAN_NODE", StatisticalType.ICEBERG_SCAN_NODE, needCheckColumnPriv);
-    }
 
-    @Override
-    protected void doInitialize() throws UserException {
         ExternalTable table = (ExternalTable) desc.getTable();
-        if (table.isView()) {
-            throw new AnalysisException(
-                String.format("Querying external view '%s.%s' is not supported", table.getDbName(), table.getName()));
-        }
-        computeColumnFilter();
-        initBackendPolicy();
         if (table instanceof HMSExternalTable) {
             source = new IcebergHMSSource((HMSExternalTable) table, desc, columnNameToRange);
         } else if (table instanceof IcebergExternalTable) {
@@ -106,11 +96,17 @@ public class IcebergScanNode extends FileQueryScanNode {
                     source = new IcebergApiSource((IcebergExternalTable) table, desc, columnNameToRange);
                     break;
                 default:
-                    throw new UserException("Unknown iceberg catalog type: " + catalogType);
+                    Preconditions.checkState(false, "Unknown iceberg catalog type: " + catalogType);
+                    break;
             }
         }
         Preconditions.checkNotNull(source);
-        initSchemaParams();
+    }
+
+    @Override
+    protected void doInitialize() throws UserException {
+        super.doInitialize();
+
     }
 
     public static void setIcebergParams(TFileRangeDesc rangeDesc, IcebergSplit icebergSplit) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
index 1ec5fe623b..a042c0a8ad 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
@@ -63,7 +63,7 @@ import org.apache.doris.planner.ScanNode;
 import org.apache.doris.planner.SetOperationNode;
 import org.apache.doris.planner.UnionNode;
 import org.apache.doris.planner.external.ExternalScanNode;
-import org.apache.doris.planner.external.FederationBackendPolicy;
+import org.apache.doris.planner.external.FileQueryScanNode;
 import org.apache.doris.proto.InternalService;
 import org.apache.doris.proto.InternalService.PExecPlanFragmentResult;
 import org.apache.doris.proto.InternalService.PExecPlanFragmentStartRequest;
@@ -268,6 +268,9 @@ public class Coordinator {
         return executionProfile;
     }
 
+    // True if all scan node are ExternalScanNode.
+    private boolean isAllExternalScan = true;
+
     private static class BackendHash implements Funnel<Backend> {
         @Override
         public void funnel(Backend backend, PrimitiveSink primitiveSink) {
@@ -1548,10 +1551,14 @@ public class Coordinator {
             if (fragment.getDataPartition() == DataPartition.UNPARTITIONED) {
                 Reference<Long> backendIdRef = new Reference<Long>();
                 TNetworkAddress execHostport;
-                if (ConnectContext.get() != null && ConnectContext.get().isResourceTagsSet()
-                        && !addressToBackendID.isEmpty()) {
-                    // In this case, we only use the BE where the replica selected by the tag is located to execute
-                    // this query. Otherwise, except for the scan node, the rest of the execution nodes of the query
+                if (((ConnectContext.get() != null && ConnectContext.get().isResourceTagsSet()) || (isAllExternalScan
+                        && Config.prefer_compute_node_for_external_table)) && !addressToBackendID.isEmpty()) {
+                    // 2 cases:
+                    // case 1: user set resource tag, we need to use the BE with the specified resource tags.
+                    // case 2: All scan nodes are external scan node,
+                    //         and prefer_compute_node_for_external_table is true, we should only select BE which scan
+                    //         nodes are used.
+                    // Otherwise, except for the scan node, the rest of the execution nodes of the query
                     // can be executed on any BE. addressToBackendID can be empty when this is a constant
                     // select stmt like:
                     //      SELECT  @@session.auto_increment_increment AS auto_increment_increment;
@@ -1731,8 +1738,7 @@ public class Coordinator {
                     // backendIdRef can be null is we call getHostByCurrentBackend() before
                     this.addressToBackendID.put(execHostport, backendIdRef.getRef());
                 }
-                FInstanceExecParam instanceParam = new FInstanceExecParam(null, execHostport,
-                        0, params);
+                FInstanceExecParam instanceParam = new FInstanceExecParam(null, execHostport, 0, params);
                 params.instanceExecParams.add(instanceParam);
             }
         }
@@ -1745,8 +1751,8 @@ public class Coordinator {
             FragmentExecParams params = fragmentExecParamsMap.get(fragment.getFragmentId());
             // Transform <fragment, runtimeFilterId> to <runtimeFilterId, fragment>
             for (RuntimeFilterId rid : fragment.getTargetRuntimeFilterIds()) {
-                List<FRuntimeFilterTargetParam> targetFragments =
-                        ridToTargetParam.computeIfAbsent(rid, k -> new ArrayList<>());
+                List<FRuntimeFilterTargetParam> targetFragments = ridToTargetParam.computeIfAbsent(rid,
+                        k -> new ArrayList<>());
                 for (final FInstanceExecParam instance : params.instanceExecParams) {
                     targetFragments.add(new FRuntimeFilterTargetParam(instance.instanceId, toBrpcHost(instance.host)));
                 }
@@ -1891,14 +1897,10 @@ public class Coordinator {
         }
     }
 
-    private Map<TNetworkAddress, Long> getReplicaNumPerHost() {
+    private Map<TNetworkAddress, Long> getReplicaNumPerHostForOlapTable() {
         Map<TNetworkAddress, Long> replicaNumPerHost = Maps.newHashMap();
         for (ScanNode scanNode : scanNodes) {
             List<TScanRangeLocations> locationsList = scanNode.getScanRangeLocations(0);
-            if (locationsList == null) {
-                // only analysis olap scan node
-                continue;
-            }
             for (TScanRangeLocations locations : locationsList) {
                 for (TScanRangeLocation location : locations.locations) {
                     if (replicaNumPerHost.containsKey(location.server)) {
@@ -1922,11 +1924,15 @@ public class Coordinator {
             Preconditions.checkNotNull(locations);
             return;
         }
+
         Map<TNetworkAddress, Long> assignedBytesPerHost = Maps.newHashMap();
-        Map<TNetworkAddress, Long> replicaNumPerHost = getReplicaNumPerHost();
+        Map<TNetworkAddress, Long> replicaNumPerHost = getReplicaNumPerHostForOlapTable();
         Collections.shuffle(scanNodes);
         // set scan ranges/locations for scan nodes
         for (ScanNode scanNode : scanNodes) {
+            if (!(scanNode instanceof ExternalScanNode)) {
+                isAllExternalScan = false;
+            }
             List<TScanRangeLocations> locations;
             // the parameters of getScanRangeLocations may ignore, It doesn't take effect
             locations = scanNode.getScanRangeLocations(0);
@@ -2066,17 +2072,17 @@ public class Coordinator {
     }
 
     private void computeScanRangeAssignmentByConsistentHash(
-            ScanNode scanNode,
+            FileQueryScanNode scanNode,
             final List<TScanRangeLocations> locations,
             FragmentScanRangeAssignment assignment,
             Map<TNetworkAddress, Long> assignedBytesPerHost,
             Map<TNetworkAddress, Long> replicaNumPerHost) throws Exception {
-        FederationBackendPolicy federationBackendPolicy = new FederationBackendPolicy();
-        federationBackendPolicy.init();
-        Collection<Backend> aliveBEs = federationBackendPolicy.getBackends();
+
+        Collection<Backend> aliveBEs = scanNode.getBackendPolicy().getBackends();
         if (aliveBEs.isEmpty()) {
             throw new UserException("No available backends");
         }
+
         int virtualNumber = Math.max(Math.min(512 / aliveBEs.size(), 32), 2);
         ConsistentHash<TScanRangeLocations, Backend> consistentHash = new ConsistentHash<>(
                 Hashing.murmur3_128(), new ScanRangeHash(), new BackendHash(), aliveBEs, virtualNumber);
@@ -2111,10 +2117,10 @@ public class Coordinator {
             FragmentScanRangeAssignment assignment,
             Map<TNetworkAddress, Long> assignedBytesPerHost,
             Map<TNetworkAddress, Long> replicaNumPerHost) throws Exception {
-        if (scanNode instanceof ExternalScanNode) {
+        if (scanNode instanceof FileQueryScanNode) {
             // Use consistent hash to assign the same scan range into the same backend among different queries
             computeScanRangeAssignmentByConsistentHash(
-                    scanNode, locations, assignment, assignedBytesPerHost, replicaNumPerHost);
+                    (FileQueryScanNode) scanNode, locations, assignment, assignedBytesPerHost, replicaNumPerHost);
             return;
         }
         for (TScanRangeLocations scanRangeLocations : locations) {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/CoordinatorTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/CoordinatorTest.java
index 941a587a6d..ca8109e40e 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/qe/CoordinatorTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/qe/CoordinatorTest.java
@@ -64,8 +64,6 @@ import java.util.Map;
 import java.util.Set;
 
 public class CoordinatorTest extends Coordinator {
-
-
     @Mocked
     static Env env;
     @Mocked
@@ -768,7 +766,7 @@ public class CoordinatorTest extends Coordinator {
         olapScanNode.setFragment(fragment);
         List<TScanRangeLocations> locations = new ArrayList<>();
         locations.add(tScanRangeLocations);
-        Deencapsulation.setField(olapScanNode, "result", locations);
+        Deencapsulation.setField(olapScanNode, "scanRangeLocations", locations);
 
         //scanNode2
         PlanFragmentId planFragmentId2 = new PlanFragmentId(2);
@@ -783,7 +781,7 @@ public class CoordinatorTest extends Coordinator {
         olapScanNode2.setFragment(fragment2);
         List<TScanRangeLocations> locations2 = new ArrayList<>();
         locations2.add(tScanRangeLocations);
-        Deencapsulation.setField(olapScanNode2, "result", locations2);
+        Deencapsulation.setField(olapScanNode2, "scanRangeLocations", locations2);
 
         //scanNode3
         PlanFragmentId planFragmentId3 = new PlanFragmentId(3);
@@ -798,7 +796,7 @@ public class CoordinatorTest extends Coordinator {
         olapScanNode3.setFragment(fragment3);
         List<TScanRangeLocations> locations3 = new ArrayList<>();
         locations3.add(tScanRangeLocations);
-        Deencapsulation.setField(olapScanNode3, "result", locations3);
+        Deencapsulation.setField(olapScanNode3, "scanRangeLocations", locations3);
 
         //scan nodes
         List<ScanNode> scanNodes = new ArrayList<>();
diff --git a/regression-test/data/es_p0/test_es_query.out b/regression-test/data/es_p0/test_es_query.out
index 57ddfafd02..4da0342d86 100644
--- a/regression-test/data/es_p0/test_es_query.out
+++ b/regression-test/data/es_p0/test_es_query.out
@@ -1,9 +1,9 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
 -- !sql51 --
-[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	['d', 'e', 'f']	[128, 129, -129, -130]	['192.168.0.1', '127.0.0.1']	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	['a', 'b', 'c']	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
+[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	["d", "e", "f"]	[128, 129, -129, -130]	["192.168.0.1", "127.0.0.1"]	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	["a", "b", "c"]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
 
 -- !sql52 --
-[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	['d', 'e', 'f']	[128, 129, -129, -130]	['192.168.0.1', '127.0.0.1']	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	['a', 'b', 'c']	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
+[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	["d", "e", "f"]	[128, 129, -129, -130]	["192.168.0.1", "127.0.0.1"]	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	["a", "b", "c"]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
 
 -- !sql53 --
 2022-08-08	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T04:10:10	2022-08-08T20:10:10
@@ -12,10 +12,10 @@
 2022-08-08	2022-08-11T12:10:10	2022-08-11T12:10:10	2022-08-11T12:10:10	2022-08-11T11:10:10
 
 -- !sql53 --
-[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	['d', 'e', 'f']	[128, 129, -129, -130]	['192.168.0.1', '127.0.0.1']	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	['a', 'b', 'c']	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
+[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	["d", "e", "f"]	[128, 129, -129, -130]	["192.168.0.1", "127.0.0.1"]	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	["a", "b", "c"]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
 
 -- !sql54 --
-[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	['d', 'e', 'f']	[128, 129, -129, -130]	['192.168.0.1', '127.0.0.1']	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	['a', 'b', 'c']	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
+[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	["d", "e", "f"]	[128, 129, -129, -130]	["192.168.0.1", "127.0.0.1"]	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	["a", "b", "c"]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
 
 -- !sql55 --
 2022-08-08	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T04:10:10	2022-08-08T20:10:10
@@ -24,15 +24,15 @@
 2022-08-08	2022-08-11T12:10:10	2022-08-11T12:10:10	2022-08-11T12:10:10	2022-08-11T11:10:10
 
 -- !sql62 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	['2020-01-01 12:00:00', '2020-01-02 13:01:01']	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	["2020-01-01 12:00:00", "2020-01-02 13:01:01"]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00
 
 -- !sql63 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	['2020-01-01 12:00:00', '2020-01-02 13:01:01']	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	['2020-01-01 12:00:00', '2020-01-02 13:01:01']	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string2	text2	4.0	2022-08-08T00:00
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	['2020-01-01 12:00:00', '2020-01-02 13:01:01']	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string3	text3_4*5	5.0	2022-08-08T00:00
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	["2020-01-01 12:00:00", "2020-01-02 13:01:01"]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	["2020-01-01 12:00:00", "2020-01-02 13:01:01"]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string2	text2	4.0	2022-08-08T00:00
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	["2020-01-01 12:00:00", "2020-01-02 13:01:01"]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string3	text3_4*5	5.0	2022-08-08T00:00
 
 -- !sql64 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	['2020-01-01 12:00:00', '2020-01-02 13:01:01']	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string2	text2	4.0	2022-08-08T00:00
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	["2020-01-01 12:00:00", "2020-01-02 13:01:01"]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string2	text2	4.0	2022-08-08T00:00
 
 -- !sql65 --
 true	1	128	32768	-1	0	1.0	1	1	1	2020-01-01T00:00	2020-01-01 12:00:00	a	d	192.168.0.1	{"name":"Andy","age":18}
@@ -45,19 +45,19 @@ true	1	128	32768	-1	0	1.0	1	1	1	2020-01-01T00:00	2020-01-01 12:00:00	a	d	192.168
 true	1	128	32768	-1	0	1.0	1	1	1	2020-01-01T00:00	2020-01-01 12:00:00	a	d	192.168.0.1	{"name":"Andy","age":18}
 
 -- !sql67 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	['2020-01-01 12:00:00', '2020-01-02 13:01:01']	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	["2020-01-01 12:00:00", "2020-01-02 13:01:01"]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00
 
 -- !sql72 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
 
 -- !sql73 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string2	text2	4.0	2022-08-08T00:00	2022-08-09T12:10:10	1660018210000	2022-08-09T12:10:10	2022-08-09T12:10:10
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string3	text3_4*5	5.0	2022-08-08T00:00	2022-08-10T12:10:10	1660104610000	2022-08-10T12:10:10	2022-08-10T20:10:10
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string4	text3_4*5	6.0	2022-08-08T00:00	2022-08-11T12:10:10	1660191010000	2022-08-11T12:10:10	2022-08-11T11:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string2	text2	4.0	2022-08-08T00:00	2022-08-09T12:10:10	1660018210000	2022-08-09T12:10:10	2022-08-09T12:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string3	text3_4*5	5.0	2022-08-08T00:00	2022-08-10T12:10:10	1660104610000	2022-08-10T12:10:10	2022-08-10T20:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string4	text3_4*5	6.0	2022-08-08T00:00	2022-08-11T12:10:10	1660191010000	2022-08-11T12:10:10	2022-08-11T11:10:10
 
 -- !sql74 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string2	text2	4.0	2022-08-08T00:00	2022-08-09T12:10:10	1660018210000	2022-08-09T12:10:10	2022-08-09T12:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string2	text2	4.0	2022-08-08T00:00	2022-08-09T12:10:10	1660018210000	2022-08-09T12:10:10	2022-08-09T12:10:10
 
 -- !sql75 --
 true	1	128	32768	-1	0	1.0	1.0	1.0	1.0	2020-01-01	2020-01-01T12:00	a	d	192.168.0.1	{"name":"Andy","age":18}
@@ -72,16 +72,16 @@ true	1	128	32768	-1	0	1.0	1.0	1.0	1.0	2020-01-01	2020-01-01T12:00	a	d	192.168.0.
 true	1	128	32768	-1	0	1.0	1.0	1.0	1.0	2020-01-01	2020-01-01T12:00	a	d	192.168.0.1	{"name":"Andy","age":18}
 
 -- !sql77 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
 
 -- !sql81 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
 
 -- !sql82 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string2	text2	4.0	2022-08-08T00:00	2022-08-09T12:10:10	1660018210000	2022-08-09T12:10:10	2022-08-09T12:10:10
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string3	text3_4*5	5.0	2022-08-08T00:00	2022-08-10T12:10:10	1660104610000	2022-08-10T12:10:10	2022-08-10T20:10:10
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string4	text3_4*5	6.0	2022-08-08T00:00	2022-08-11T12:10:10	1660191010000	2022-08-11T12:10:10	2022-08-11T11:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string2	text2	4.0	2022-08-08T00:00	2022-08-09T12:10:10	1660018210000	2022-08-09T12:10:10	2022-08-09T12:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string3	text3_4*5	5.0	2022-08-08T00:00	2022-08-10T12:10:10	1660104610000	2022-08-10T12:10:10	2022-08-10T20:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string4	text3_4*5	6.0	2022-08-08T00:00	2022-08-11T12:10:10	1660191010000	2022-08-11T12:10:10	2022-08-11T11:10:10
 
 -- !sql83 --
 true	1	128	32768	-1	0	1.0	1.0	1.0	1.0	2020-01-01	2020-01-01T12:00	a	d	192.168.0.1	{"name":"Andy","age":18}
@@ -96,5 +96,5 @@ true	1	128	32768	-1	0	1.0	1.0	1.0	1.0	2020-01-01	2020-01-01T12:00	a	d	192.168.0.
 true	1	128	32768	-1	0	1.0	1.0	1.0	1.0	2020-01-01	2020-01-01T12:00	a	d	192.168.0.1	{"name":"Andy","age":18}
 
 -- !sql85 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
 
diff --git a/regression-test/data/es_p0/test_es_query_no_http_url.out b/regression-test/data/es_p0/test_es_query_no_http_url.out
index b195d6d187..e1c30290dc 100644
--- a/regression-test/data/es_p0/test_es_query_no_http_url.out
+++ b/regression-test/data/es_p0/test_es_query_no_http_url.out
@@ -1,16 +1,16 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
 -- !sql51 --
-[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	['d', 'e', 'f']	[128, 129, -129, -130]	['192.168.0.1', '127.0.0.1']	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	['a', 'b', 'c']	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
+[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	["d", "e", "f"]	[128, 129, -129, -130]	["192.168.0.1", "127.0.0.1"]	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	["a", "b", "c"]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
 
 -- !sql52 --
-[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	['d', 'e', 'f']	[128, 129, -129, -130]	['192.168.0.1', '127.0.0.1']	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	['a', 'b', 'c']	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
+[2020-01-01, 2020-01-02]	[-1, 0, 1, 2]	[0, 1, 2, 3]	["d", "e", "f"]	[128, 129, -129, -130]	["192.168.0.1", "127.0.0.1"]	string1	[1, 2, 3, 4]	2022-08-08	2022-08-08T12:10:10	text#1	[2020-01-01, 2020-01-02]	3.14	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	["a", "b", "c"]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	2022-08-08T12:10:10	2022-08-08T12:10:10	2022-08-08T20:10:10	[1, -2, -3, 4]	[1, 0, 1, 1]	[32768, 32769, -32769, -32770]
 
 -- !sql61 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	['2020-01-01 12:00:00', '2020-01-02 13:01:01']	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01 00:00:00, 2020-01-02 00:00:00]	["2020-01-01 12:00:00", "2020-01-02 13:01:01"]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00
 
 -- !sql71 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
 
 -- !sql81 --
-[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	['192.168.0.1', '127.0.0.1']	['a', 'b', 'c']	[-1, 0, 1, 2]	['{"name":"Andy","age":18}', '{"name":"Tim","age":28}']	[1, 2, 3, 4]	[128, 129, -129, -130]	['d', 'e', 'f']	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
+[1, 0, 1, 1]	[1, -2, -3, 4]	[2020-01-01, 2020-01-02]	[2020-01-01 12:00:00, 2020-01-02 13:01:01]	[1, 2, 3, 4]	[1, 1.1, 1.2, 1.3]	[1, 2, 3, 4]	[32768, 32769, -32769, -32770]	["192.168.0.1", "127.0.0.1"]	["a", "b", "c"]	[-1, 0, 1, 2]	["{"name":"Andy","age":18}", "{"name":"Tim","age":28}"]	[1, 2, 3, 4]	[128, 129, -129, -130]	["d", "e", "f"]	[0, 1, 2, 3]	string1	text#1	3.14	2022-08-08T00:00	2022-08-08T12:10:10	1659931810000	2022-08-08T12:10:10	2022-08-08T20:10:10
 
diff --git a/regression-test/suites/es_p0/test_es_query.groovy b/regression-test/suites/es_p0/test_es_query.groovy
index 2b0251c1e1..24b1817732 100644
--- a/regression-test/suites/es_p0/test_es_query.groovy
+++ b/regression-test/suites/es_p0/test_es_query.groovy
@@ -161,9 +161,5 @@ suite("test_es_query", "p0") {
         order_qt_sql84 """select c_bool[1], c_byte[1], c_short[1], c_integer[1], c_long[1], c_unsigned_long[1], c_float[1], c_half_float[1], c_double[1], c_scaled_float[1], c_date[1], c_datetime[1], c_keyword[1], c_text[1], c_ip[1], c_person[1] from test2"""
         order_qt_sql85 """select * from test1 where esquery(test2, '{"match":{"test2":"text#1"}}')"""
 
-
-        sql """drop catalog if exists es6;"""
-        sql """drop catalog if exists es7;"""
-        sql """drop catalog if exists es8;"""
     }
 }
diff --git a/regression-test/suites/es_p0/test_es_query_nereids.groovy b/regression-test/suites/es_p0/test_es_query_nereids.groovy
index 407d6c2d17..4921ced70f 100644
--- a/regression-test/suites/es_p0/test_es_query_nereids.groovy
+++ b/regression-test/suites/es_p0/test_es_query_nereids.groovy
@@ -154,10 +154,5 @@ suite("test_es_query_nereids", "p0") {
         order_qt_sql82 """select * from test2_20220808 where test4='2022-08-08'"""
         order_qt_sql83 """select c_bool[1], c_byte[1], c_short[1], c_integer[1], c_long[1], c_unsigned_long[1], c_float[1], c_half_float[1], c_double[1], c_scaled_float[1], c_date[1], c_datetime[1], c_keyword[1], c_text[1], c_ip[1], c_person[1] from test1"""
         order_qt_sql84 """select c_bool[1], c_byte[1], c_short[1], c_integer[1], c_long[1], c_unsigned_long[1], c_float[1], c_half_float[1], c_double[1], c_scaled_float[1], c_date[1], c_datetime[1], c_keyword[1], c_text[1], c_ip[1], c_person[1] from test2"""
-
-
-        sql """drop catalog if exists es6_nereids;"""
-        sql """drop catalog if exists es7_nereids;"""
-        sql """drop catalog if exists es8_nereids;"""
     }
 }
diff --git a/regression-test/suites/es_p0/test_es_query_no_http_url.groovy b/regression-test/suites/es_p0/test_es_query_no_http_url.groovy
index e7e6af5b95..3f712fe778 100644
--- a/regression-test/suites/es_p0/test_es_query_no_http_url.groovy
+++ b/regression-test/suites/es_p0/test_es_query_no_http_url.groovy
@@ -143,9 +143,5 @@ suite("test_es_query_no_http_url", "p0") {
         // es8
         sql """switch es8"""
         order_qt_sql81 """select * from test1 where test2='text#1'"""
-
-        sql """drop catalog if exists es6;"""
-        sql """drop catalog if exists es7;"""
-        sql """drop catalog if exists es8;"""
     }
 }
diff --git a/regression-test/suites/external_table_emr_p2/hive/test_external_yandex_nereids.groovy b/regression-test/suites/external_table_emr_p2/hive/test_external_yandex_nereids.groovy
index 9f06df3100..af1a0e5d15 100644
--- a/regression-test/suites/external_table_emr_p2/hive/test_external_yandex_nereids.groovy
+++ b/regression-test/suites/external_table_emr_p2/hive/test_external_yandex_nereids.groovy
@@ -62,6 +62,7 @@ suite("test_external_yandex_nereids", "p2") {
         sql """use multi_catalog;"""
         logger.info("use multi_catalog")
         sql """set enable_nereids_planner=true"""
+        sql """set enable_fallback_to_original_planner=false"""
 
         for (String format in formats) {
             logger.info("Process format " + format)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 05/36: [fix](memory) Fix query memory tracking #20253

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 4ebf48b437b3a20255ad570c29471b661c26048e
Author: Xinyi Zou <zo...@gmail.com>
AuthorDate: Mon Jun 5 08:33:38 2023 +0800

    [fix](memory) Fix query memory tracking #20253
    
    The memory released by the query end is recorded in the query mem tracker, main memory in _runtime_state.
    fix page no cache memory tracking
    Now the main reason for the inaccurate query memory tracking is that the virtual memory used by the query is sometimes much larger than the actual memory. And the mem hook counts virtual memory.
---
 be/src/olap/page_cache.h                      |  2 --
 be/src/olap/rowset/segment_v2/page_handle.h   |  5 ++++-
 be/src/pipeline/pipeline_fragment_context.cpp |  9 ++++++++-
 be/src/runtime/plan_fragment_executor.cpp     | 10 ++++++++--
 4 files changed, 20 insertions(+), 6 deletions(-)

diff --git a/be/src/olap/page_cache.h b/be/src/olap/page_cache.h
index cc52f4a252..7e5ca4de6e 100644
--- a/be/src/olap/page_cache.h
+++ b/be/src/olap/page_cache.h
@@ -42,7 +42,6 @@ public:
 
     PageBase(size_t b) : _size(b), _capacity(b) {
         _data = reinterpret_cast<char*>(TAllocator::alloc(_capacity, ALLOCATOR_ALIGNMENT_16));
-        ExecEnv::GetInstance()->page_no_cache_mem_tracker()->consume(_capacity);
     }
 
     PageBase(const PageBase&) = delete;
@@ -52,7 +51,6 @@ public:
         if (_data != nullptr) {
             DCHECK(_capacity != 0 && _size != 0);
             TAllocator::free(_data, _capacity);
-            ExecEnv::GetInstance()->page_no_cache_mem_tracker()->release(_capacity);
         }
     }
 
diff --git a/be/src/olap/rowset/segment_v2/page_handle.h b/be/src/olap/rowset/segment_v2/page_handle.h
index 7e4f766524..6494048774 100644
--- a/be/src/olap/rowset/segment_v2/page_handle.h
+++ b/be/src/olap/rowset/segment_v2/page_handle.h
@@ -36,7 +36,9 @@ public:
 
     // This class will take the ownership of input data's memory. It will
     // free it when deconstructs.
-    PageHandle(DataPage* data) : _is_data_owner(true), _data(data) {}
+    PageHandle(DataPage* data) : _is_data_owner(true), _data(data) {
+        ExecEnv::GetInstance()->page_no_cache_mem_tracker()->consume(_data->capacity());
+    }
 
     // This class will take the content of cache data, and will make input
     // cache_data to a invalid cache handle.
@@ -59,6 +61,7 @@ public:
 
     ~PageHandle() {
         if (_is_data_owner) {
+            ExecEnv::GetInstance()->page_no_cache_mem_tracker()->release(_data->capacity());
             delete _data;
         } else {
             DCHECK(_data == nullptr);
diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp
index eea8d34b08..32207ad216 100644
--- a/be/src/pipeline/pipeline_fragment_context.cpp
+++ b/be/src/pipeline/pipeline_fragment_context.cpp
@@ -143,7 +143,14 @@ PipelineFragmentContext::PipelineFragmentContext(
 }
 
 PipelineFragmentContext::~PipelineFragmentContext() {
-    _call_back(_runtime_state.get(), &_exec_status);
+    if (_runtime_state != nullptr) {
+        // The memory released by the query end is recorded in the query mem tracker, main memory in _runtime_state.
+        SCOPED_ATTACH_TASK(_runtime_state.get());
+        _call_back(_runtime_state.get(), &_exec_status);
+        _runtime_state.reset();
+    } else {
+        _call_back(_runtime_state.get(), &_exec_status);
+    }
     DCHECK(!_report_thread_active);
 }
 
diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp
index bd99a9a2e3..079719927b 100644
--- a/be/src/runtime/plan_fragment_executor.cpp
+++ b/be/src/runtime/plan_fragment_executor.cpp
@@ -88,7 +88,14 @@ PlanFragmentExecutor::PlanFragmentExecutor(ExecEnv* exec_env,
 }
 
 PlanFragmentExecutor::~PlanFragmentExecutor() {
-    close();
+    if (_runtime_state != nullptr) {
+        // The memory released by the query end is recorded in the query mem tracker, main memory in _runtime_state.
+        SCOPED_ATTACH_TASK(_runtime_state.get());
+        close();
+        _runtime_state.reset();
+    } else {
+        close();
+    }
     // at this point, the report thread should have been stopped
     DCHECK(!_report_thread_active);
 }
@@ -274,7 +281,6 @@ Status PlanFragmentExecutor::open() {
         if (_cancel_reason == PPlanFragmentCancelReason::CALL_RPC_ERROR) {
             status = Status::RuntimeError(_cancel_msg);
         } else if (_cancel_reason == PPlanFragmentCancelReason::MEMORY_LIMIT_EXCEED) {
-            // status = Status::MemoryAllocFailed(_cancel_msg);
             status = Status::MemoryLimitExceeded(_cancel_msg);
         }
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 25/36: Fix query hang when using queue (#20434)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 1f273fd8560dbfb2ab9b58a8cf967955d309359b
Author: wangbo <wa...@apache.org>
AuthorDate: Mon Jun 5 18:12:26 2023 +0800

    Fix query hang when using queue (#20434)
---
 .../java/org/apache/doris/qe/StmtExecutor.java     | 63 +++++++++++-----------
 1 file changed, 33 insertions(+), 30 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
index c5c23055ff..a06f6ee855 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
@@ -577,34 +577,40 @@ public class StmtExecutor {
         }
 
         int retryTime = Config.max_query_retry_time;
-        for (int i = 0; i < retryTime; i++) {
-            try {
-                //reset query id for each retry
-                if (i > 0) {
-                    UUID uuid = UUID.randomUUID();
-                    TUniqueId newQueryId = new TUniqueId(uuid.getMostSignificantBits(),
-                            uuid.getLeastSignificantBits());
-                    AuditLog.getQueryAudit().log("Query {} {} times with new query id: {}",
-                            DebugUtil.printId(queryId), i, DebugUtil.printId(newQueryId));
-                    context.setQueryId(newQueryId);
-                }
-                handleQueryStmt();
-                break;
-            } catch (RpcException e) {
-                if (i == retryTime - 1) {
-                    throw e;
-                }
-                if (!context.getMysqlChannel().isSend()) {
-                    LOG.warn("retry {} times. stmt: {}", (i + 1), parsedStmt.getOrigStmt().originStmt);
-                } else {
-                    throw e;
+        try {
+            for (int i = 0; i < retryTime; i++) {
+                try {
+                    //reset query id for each retry
+                    if (i > 0) {
+                        UUID uuid = UUID.randomUUID();
+                        TUniqueId newQueryId = new TUniqueId(uuid.getMostSignificantBits(),
+                                uuid.getLeastSignificantBits());
+                        AuditLog.getQueryAudit().log("Query {} {} times with new query id: {}",
+                                DebugUtil.printId(queryId), i, DebugUtil.printId(newQueryId));
+                        context.setQueryId(newQueryId);
+                    }
+                    handleQueryStmt();
+                    break;
+                } catch (RpcException e) {
+                    if (i == retryTime - 1) {
+                        throw e;
+                    }
+                    if (!context.getMysqlChannel().isSend()) {
+                        LOG.warn("retry {} times. stmt: {}", (i + 1), parsedStmt.getOrigStmt().originStmt);
+                    } else {
+                        throw e;
+                    }
+                } finally {
+                    // The final profile report occurs after be returns the query data, and the profile cannot be
+                    // received after unregisterQuery(), causing the instance profile to be lost, so we should wait
+                    // for the profile before unregisterQuery().
+                    updateProfile(true);
+                    QeProcessorImpl.INSTANCE.unregisterQuery(context.queryId());
                 }
-            } finally {
-                // The final profile report occurs after be returns the query data, and the profile cannot be
-                // received after unregisterQuery(), causing the instance profile to be lost, so we should wait
-                // for the profile before unregisterQuery().
-                updateProfile(true);
-                QeProcessorImpl.INSTANCE.unregisterQuery(context.queryId());
+            }
+        } finally {
+            if (offerRet.isOfferSuccess()) {
+                queryQueue.poll();
             }
         }
     }
@@ -645,9 +651,6 @@ public class StmtExecutor {
                     throw e;
                 } finally {
                     queryAnalysisSpan.end();
-                    if (offerRet.isOfferSuccess()) {
-                        queryQueue.poll();
-                    }
                 }
                 if (isForwardToMaster()) {
                     if (isProxy) {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 35/36: [fix](Nereids) join condition not extract as conjunctions (#20498)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 193cf917d0813ec6eb9084e75b6475f6cc01ad34
Author: morrySnow <10...@users.noreply.github.com>
AuthorDate: Tue Jun 6 20:34:19 2023 +0800

    [fix](Nereids) join condition not extract as conjunctions (#20498)
---
 .../doris/nereids/rules/expression/ExpressionRewrite.java  | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java
index 3dc61bcbc7..e42f82b13f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java
@@ -42,7 +42,6 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
@@ -189,7 +188,7 @@ public class ExpressionRewrite implements RewriteRuleFactory {
                 for (Expression expr : hashJoinConjuncts) {
                     Expression newExpr = rewriter.rewrite(expr, context);
                     hashJoinConjunctsChanged = hashJoinConjunctsChanged || !newExpr.equals(expr);
-                    rewriteHashJoinConjuncts.add(newExpr);
+                    rewriteHashJoinConjuncts.addAll(ExpressionUtils.extractConjunction(newExpr));
                 }
 
                 List<Expression> rewriteOtherJoinConjuncts = Lists.newArrayList();
@@ -197,7 +196,7 @@ public class ExpressionRewrite implements RewriteRuleFactory {
                 for (Expression expr : otherJoinConjuncts) {
                     Expression newExpr = rewriter.rewrite(expr, context);
                     otherJoinConjunctsChanged = otherJoinConjunctsChanged || !newExpr.equals(expr);
-                    rewriteOtherJoinConjuncts.add(newExpr);
+                    rewriteOtherJoinConjuncts.addAll(ExpressionUtils.extractConjunction(newExpr));
                 }
 
                 if (!hashJoinConjunctsChanged && !otherJoinConjunctsChanged) {
@@ -233,12 +232,13 @@ public class ExpressionRewrite implements RewriteRuleFactory {
         public Rule build() {
             return logicalHaving().thenApply(ctx -> {
                 LogicalHaving<Plan> having = ctx.root;
-                Set<Expression> rewrittenExpr = new HashSet<>();
                 ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext);
-                for (Expression e : having.getExpressions()) {
-                    rewrittenExpr.add(rewriter.rewrite(e, context));
+                Set<Expression> newConjuncts = ImmutableSet.copyOf(ExpressionUtils.extractConjunction(
+                        rewriter.rewrite(having.getPredicate(), context)));
+                if (newConjuncts.equals(having.getConjuncts())) {
+                    return having;
                 }
-                return having.withExpressions(rewrittenExpr);
+                return having.withExpressions(newConjuncts);
             }).toRule(RuleType.REWRITE_HAVING_EXPRESSION);
         }
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 27/36: [fix](Nereids) throw NPE when sql cannot be parsed by all planner (#20440)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit e30e986aa5c14d2753af23612bc629e5ad451af4
Author: morrySnow <10...@users.noreply.github.com>
AuthorDate: Mon Jun 5 17:49:08 2023 +0800

    [fix](Nereids) throw NPE when sql cannot be parsed by all planner (#20440)
---
 fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
index 1c47f2f651..779c8ab772 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
@@ -652,7 +652,7 @@ public class ConnectProcessor {
         // note(wb) we should write profile after return result to mysql client
         // because write profile maybe take too much time
         // explain query stmt do not have profile
-        if (executor != null && !executor.getParsedStmt().isExplain()
+        if (executor != null && executor.getParsedStmt() != null && !executor.getParsedStmt().isExplain()
                 && (executor.getParsedStmt() instanceof QueryStmt // currently only QueryStmt and insert need profile
                 || executor.getParsedStmt() instanceof LogicalPlanAdapter
                 || executor.getParsedStmt() instanceof InsertStmt)) {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 11/36: [feature](profile)Add the filtering info of the in filter in profile #20321

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit a368255baa1133047c46d64c48a635abfd8fb5fe
Author: Mryange <59...@users.noreply.github.com>
AuthorDate: Tue Jun 6 10:24:59 2023 +0800

    [feature](profile)Add the filtering info of the in filter in profile #20321
    
    image Currently, it is difficult to obtain the id of in filters,so, the some in filters's id is -1.
---
 be/src/exprs/hybrid_set.h                          |  3 +++
 be/src/exprs/runtime_filter.cpp                    |  3 +++
 be/src/olap/in_list_predicate.h                    | 15 ++++++++++-----
 be/src/olap/rowset/segment_v2/segment_iterator.cpp |  6 +-----
 be/src/vec/exec/scan/new_olap_scan_node.cpp        |  5 +----
 5 files changed, 18 insertions(+), 14 deletions(-)

diff --git a/be/src/exprs/hybrid_set.h b/be/src/exprs/hybrid_set.h
index 73508d56d2..fa7bd97176 100644
--- a/be/src/exprs/hybrid_set.h
+++ b/be/src/exprs/hybrid_set.h
@@ -222,6 +222,9 @@ public:
         LOG(FATAL) << "HybridSetBase not support find_batch_nullable_negative";
     }
 
+    void set_filter_id(int filter_id) { _filter_id = filter_id; }
+    int get_filter_id() const { return _filter_id; }
+    int _filter_id = -1;
     class IteratorBase {
     public:
         IteratorBase() = default;
diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp
index 499c291ae5..ef8543e136 100644
--- a/be/src/exprs/runtime_filter.cpp
+++ b/be/src/exprs/runtime_filter.cpp
@@ -1069,6 +1069,9 @@ public:
         if (_context.bitmap_filter_func) {
             _context.bitmap_filter_func->set_filter_id(id);
         }
+        if (_context.hybrid_set) {
+            _context.hybrid_set->set_filter_id(id);
+        }
     }
 
 private:
diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h
index 7751e02eb6..8af69efc66 100644
--- a/be/src/olap/in_list_predicate.h
+++ b/be/src/olap/in_list_predicate.h
@@ -260,6 +260,8 @@ public:
 
     uint16_t evaluate(const vectorized::IColumn& column, uint16_t* sel,
                       uint16_t size) const override {
+        int64_t new_size = 0;
+
         if (column.is_nullable()) {
             auto* nullable_col =
                     vectorized::check_and_get_column<vectorized::ColumnNullable>(column);
@@ -269,19 +271,22 @@ public:
             auto& nested_col = nullable_col->get_nested_column();
 
             if (_opposite) {
-                return _base_evaluate<true, true>(&nested_col, &null_map, sel, size);
+                new_size = _base_evaluate<true, true>(&nested_col, &null_map, sel, size);
             } else {
-                return _base_evaluate<true, false>(&nested_col, &null_map, sel, size);
+                new_size = _base_evaluate<true, false>(&nested_col, &null_map, sel, size);
             }
         } else {
             if (_opposite) {
-                return _base_evaluate<false, true>(&column, nullptr, sel, size);
+                new_size = _base_evaluate<false, true>(&column, nullptr, sel, size);
             } else {
-                return _base_evaluate<false, false>(&column, nullptr, sel, size);
+                new_size = _base_evaluate<false, false>(&column, nullptr, sel, size);
             }
         }
+        _evaluated_rows += size;
+        _passed_rows += new_size;
+        return new_size;
     }
-
+    int get_filter_id() const override { return _values->get_filter_id(); }
     template <bool is_and>
     void _evaluate_bit(const vectorized::IColumn& column, const uint16_t* sel, uint16_t size,
                        bool* flags) const {
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index 8eed9410a9..284ae71f1a 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -1262,9 +1262,7 @@ Status SegmentIterator::_vec_init_lazy_materialization() {
             } else {
                 short_cir_pred_col_id_set.insert(cid);
                 _short_cir_eval_predicate.push_back(predicate);
-                if (predicate->get_filter_id() != -1) {
-                    _filter_info_id.push_back(predicate);
-                }
+                _filter_info_id.push_back(predicate);
             }
         }
 
@@ -1644,8 +1642,6 @@ uint16_t SegmentIterator::_evaluate_short_circuit_predicate(uint16_t* vec_sel_ro
     }
     _opts.stats->short_circuit_cond_input_rows += original_size;
     _opts.stats->rows_short_circuit_cond_filtered += original_size - selected_size;
-    _opts.stats->short_circuit_cond_input_rows += original_size;
-    _opts.stats->rows_short_circuit_cond_filtered += original_size - selected_size;
 
     // evaluate delete condition
     original_size = selected_size;
diff --git a/be/src/vec/exec/scan/new_olap_scan_node.cpp b/be/src/vec/exec/scan/new_olap_scan_node.cpp
index 3c3c272ff7..a5a18a97dd 100644
--- a/be/src/vec/exec/scan/new_olap_scan_node.cpp
+++ b/be/src/vec/exec/scan/new_olap_scan_node.cpp
@@ -645,9 +645,6 @@ bool NewOlapScanNode::_is_key_column(const std::string& key_name) {
 }
 
 void NewOlapScanNode::add_filter_info(int id, const PredicateFilterInfo& update_info) {
-    static std::vector<std::string> PredicateTypeName(20, "Unknow");
-    PredicateTypeName[static_cast<int>(PredicateType::BF)] = "BloomFilter";
-    PredicateTypeName[static_cast<int>(PredicateType::BITMAP_FILTER)] = "BitmapFilter";
     // update
     _filter_info[id].filtered_row += update_info.filtered_row;
     _filter_info[id].input_row += update_info.input_row;
@@ -657,7 +654,7 @@ void NewOlapScanNode::add_filter_info(int id, const PredicateFilterInfo& update_
     std::string filter_name = "RuntimeFilterInfo id ";
     filter_name += std::to_string(id);
     std::string info_str;
-    info_str += "type = " + PredicateTypeName[info.type] + ", ";
+    info_str += "type = " + type_to_string(static_cast<PredicateType>(info.type)) + ", ";
     info_str += "input = " + std::to_string(info.input_row) + ", ";
     info_str += "filtered = " + std::to_string(info.filtered_row);
     info_str = "[" + info_str + "]";


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 09/36: [fix](Nereids): fix filter can't be pushdown unionAll (#20310)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 31bd9ba03cc26818bd2d440702815c2b7956e8e4
Author: jakevin <ja...@gmail.com>
AuthorDate: Mon Jun 5 16:56:25 2023 +0800

    [fix](Nereids): fix filter can't be pushdown unionAll (#20310)
---
 .../logical/PushdownFilterThroughSetOperation.java |  21 ++---
 .../nereids/trees/plans/logical/LogicalExcept.java |   5 -
 .../trees/plans/logical/LogicalIntersect.java      |   5 -
 .../trees/plans/logical/LogicalSetOperation.java   |   3 -
 .../nereids/trees/plans/logical/LogicalUnion.java  |   5 -
 .../nereids_tpcds_shape_sf100_p0/shape/query75.out | 104 ++++++++++-----------
 6 files changed, 58 insertions(+), 85 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushdownFilterThroughSetOperation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushdownFilterThroughSetOperation.java
index 218bf9e2ee..f678951d78 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushdownFilterThroughSetOperation.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/PushdownFilterThroughSetOperation.java
@@ -45,11 +45,11 @@ public class PushdownFilterThroughSetOperation extends OneRewriteRuleFactory {
 
     @Override
     public Rule build() {
-        return logicalFilter(logicalSetOperation()).then(filter -> {
-            LogicalSetOperation setOperation = filter.child();
+        return logicalFilter(logicalSetOperation()).when(f -> f.child().getQualifier() == Qualifier.ALL).then(f -> {
+            LogicalSetOperation setOperation = f.child();
 
             if (setOperation instanceof LogicalUnion && ((LogicalUnion) setOperation).hasPushedFilter()) {
-                return filter;
+                return f;
             }
 
             List<Plan> newChildren = new ArrayList<>();
@@ -70,26 +70,19 @@ public class PushdownFilterThroughSetOperation extends OneRewriteRuleFactory {
                     replaceMap.put(output, child.getOutput().get(i));
                 }
 
-                Set<Expression> newFilterPredicates = filter.getConjuncts().stream().map(conjunct ->
+                Set<Expression> newFilterPredicates = f.getConjuncts().stream().map(conjunct ->
                         ExpressionUtils.replace(conjunct, replaceMap)).collect(ImmutableSet.toImmutableSet());
                 newChildren.add(new LogicalFilter<>(newFilterPredicates, child));
             }
-
             if (allOneRowRelation) {
-                return filter;
-            }
-
-            if (setOperation instanceof LogicalUnion && setOperation.getQualifier() == Qualifier.DISTINCT) {
-                return new LogicalFilter<>(filter.getConjuncts(),
-                        ((LogicalUnion) setOperation).withHasPushedFilter().withChildren(newChildren));
+                return f;
             }
 
             if (hasOneRowRelation) {
                 // If there are some `OneRowRelation` exists, we need to keep the `filter`.
-                return filter.withChildren(
-                        ((LogicalUnion) setOperation).withHasPushedFilter().withNewChildren(newChildren));
+                return f.withChildren(((LogicalUnion) setOperation).withHasPushedFilter().withChildren(newChildren));
             }
-            return setOperation.withNewChildren(newChildren);
+            return setOperation.withChildren(newChildren);
         }).toRule(RuleType.PUSHDOWN_FILTER_THROUGH_SET_OPERATION);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalExcept.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalExcept.java
index cd55848405..f11e9782a2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalExcept.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalExcept.java
@@ -80,9 +80,4 @@ public class LogicalExcept extends LogicalSetOperation {
     public LogicalExcept withNewOutputs(List<NamedExpression> newOutputs) {
         return new LogicalExcept(qualifier, newOutputs, Optional.empty(), Optional.empty(), children);
     }
-
-    @Override
-    public LogicalExcept withNewChildren(List<Plan> children) {
-        return withChildren(children);
-    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalIntersect.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalIntersect.java
index 2f3b37453e..6c831da1d0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalIntersect.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalIntersect.java
@@ -82,9 +82,4 @@ public class LogicalIntersect extends LogicalSetOperation {
         return new LogicalIntersect(qualifier, newOutputs,
                 Optional.empty(), Optional.empty(), children);
     }
-
-    @Override
-    public LogicalIntersect withNewChildren(List<Plan> children) {
-        return withChildren(children);
-    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java
index 31d8ab6dcf..308cd2a9e9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java
@@ -224,7 +224,4 @@ public abstract class LogicalSetOperation extends AbstractLogicalPlan implements
     public int getArity() {
         return children.size();
     }
-
-    public abstract LogicalSetOperation withNewChildren(List<Plan> children);
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnion.java
index 1f53f89d6d..934df533f7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnion.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnion.java
@@ -116,11 +116,6 @@ public class LogicalUnion extends LogicalSetOperation implements OutputPrunable
         return new LogicalUnion(qualifier, outputs, true, Optional.empty(), Optional.empty(), children);
     }
 
-    @Override
-    public LogicalUnion withNewChildren(List<Plan> children) {
-        return withChildren(children);
-    }
-
     @Override
     public LogicalUnion pruneOutputs(List<NamedExpression> prunedOutputs) {
         return withNewOutputs(prunedOutputs);
diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out
index 319e8a5922..4997e1c25c 100644
--- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out
+++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out
@@ -8,64 +8,62 @@ CteAnchor[cteId= ( CTEId#3=] )
 ----------hashAgg[GLOBAL]
 ------------PhysicalDistribute
 --------------hashAgg[LOCAL]
-----------------filter(((all_sales.d_year = 1998) OR (all_sales.d_year = 1999)))
-------------------PhysicalUnion
---------------------hashAgg[GLOBAL]
-----------------------PhysicalDistribute
-------------------------hashAgg[LOCAL]
---------------------------filter(((d_year = 1998) OR (d_year = 1999)))
-----------------------------PhysicalUnion
+----------------PhysicalUnion
+------------------hashAgg[GLOBAL]
+--------------------PhysicalDistribute
+----------------------hashAgg[LOCAL]
+------------------------PhysicalUnion
+--------------------------PhysicalProject
+----------------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number)
 ------------------------------PhysicalProject
---------------------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number)
-----------------------------------PhysicalProject
-------------------------------------PhysicalOlapScan[catalog_returns]
-----------------------------------PhysicalProject
-------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)
---------------------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk)
-----------------------------------------PhysicalProject
-------------------------------------------PhysicalOlapScan[catalog_sales]
-----------------------------------------PhysicalDistribute
-------------------------------------------PhysicalProject
---------------------------------------------filter((cast(i_category as VARCHAR(*)) = 'Home'))
-----------------------------------------------PhysicalOlapScan[item]
---------------------------------------PhysicalDistribute
-----------------------------------------PhysicalProject
-------------------------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999)))
---------------------------------------------PhysicalOlapScan[date_dim]
+--------------------------------PhysicalOlapScan[catalog_returns]
 ------------------------------PhysicalProject
---------------------------------hashJoin[RIGHT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)
-----------------------------------PhysicalProject
-------------------------------------PhysicalOlapScan[store_returns]
-----------------------------------PhysicalProject
-------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk)
---------------------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk)
-----------------------------------------PhysicalProject
-------------------------------------------PhysicalOlapScan[store_sales]
-----------------------------------------PhysicalDistribute
-------------------------------------------PhysicalProject
---------------------------------------------filter((cast(i_category as VARCHAR(*)) = 'Home'))
-----------------------------------------------PhysicalOlapScan[item]
---------------------------------------PhysicalDistribute
-----------------------------------------PhysicalProject
-------------------------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999)))
---------------------------------------------PhysicalOlapScan[date_dim]
---------------------PhysicalProject
-----------------------hashJoin[RIGHT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number)
-------------------------PhysicalProject
---------------------------PhysicalOlapScan[web_returns]
-------------------------PhysicalProject
---------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk)
-----------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk)
+--------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)
+----------------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk)
+------------------------------------PhysicalProject
+--------------------------------------PhysicalOlapScan[catalog_sales]
+------------------------------------PhysicalDistribute
+--------------------------------------PhysicalProject
+----------------------------------------filter((cast(i_category as VARCHAR(*)) = 'Home'))
+------------------------------------------PhysicalOlapScan[item]
+----------------------------------PhysicalDistribute
+------------------------------------PhysicalProject
+--------------------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999)))
+----------------------------------------PhysicalOlapScan[date_dim]
+--------------------------PhysicalProject
+----------------------------hashJoin[RIGHT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)
 ------------------------------PhysicalProject
---------------------------------PhysicalOlapScan[web_sales]
-------------------------------PhysicalDistribute
---------------------------------PhysicalProject
-----------------------------------filter((cast(i_category as VARCHAR(*)) = 'Home'))
-------------------------------------PhysicalOlapScan[item]
+--------------------------------PhysicalOlapScan[store_returns]
+------------------------------PhysicalProject
+--------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk)
+----------------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk)
+------------------------------------PhysicalProject
+--------------------------------------PhysicalOlapScan[store_sales]
+------------------------------------PhysicalDistribute
+--------------------------------------PhysicalProject
+----------------------------------------filter((cast(i_category as VARCHAR(*)) = 'Home'))
+------------------------------------------PhysicalOlapScan[item]
+----------------------------------PhysicalDistribute
+------------------------------------PhysicalProject
+--------------------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999)))
+----------------------------------------PhysicalOlapScan[date_dim]
+------------------PhysicalProject
+--------------------hashJoin[RIGHT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number)
+----------------------PhysicalProject
+------------------------PhysicalOlapScan[web_returns]
+----------------------PhysicalProject
+------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk)
+--------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk)
+----------------------------PhysicalProject
+------------------------------PhysicalOlapScan[web_sales]
 ----------------------------PhysicalDistribute
 ------------------------------PhysicalProject
---------------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999)))
-----------------------------------PhysicalOlapScan[date_dim]
+--------------------------------filter((cast(i_category as VARCHAR(*)) = 'Home'))
+----------------------------------PhysicalOlapScan[item]
+--------------------------PhysicalDistribute
+----------------------------PhysicalProject
+------------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999)))
+--------------------------------PhysicalOlapScan[date_dim]
 --PhysicalTopN
 ----PhysicalDistribute
 ------PhysicalTopN


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 18/36: [fix](nereids) change defaultConcreteType function's return value for decimal (#20380)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit b5027ef1bab2e86d3a0e1d2cd20ecd82235eef78
Author: starocean999 <40...@users.noreply.github.com>
AuthorDate: Mon Jun 5 10:50:07 2023 +0800

    [fix](nereids) change defaultConcreteType function's return value for decimal (#20380)
    
    1. add default decimalv2 and decimalv3 for NullType
    2. change defaultConcreteType of decimalv3 to this
---
 .../trees/expressions/functions/scalar/Nvl.java    |  2 +-
 .../apache/doris/nereids/types/DecimalV2Type.java  |  3 ++-
 .../apache/doris/nereids/types/DecimalV3Type.java  |  3 ++-
 .../conditional_functions/test_coalesce.out        | 24 +++++++++++-----------
 .../conditional_functions/test_nullif.out          |  8 ++++----
 5 files changed, 21 insertions(+), 19 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Nvl.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Nvl.java
index cf553199fe..5e5e8736b6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Nvl.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Nvl.java
@@ -58,8 +58,8 @@ public class Nvl extends ScalarFunction
             FunctionSignature.ret(IntegerType.INSTANCE).args(IntegerType.INSTANCE, IntegerType.INSTANCE),
             FunctionSignature.ret(BigIntType.INSTANCE).args(BigIntType.INSTANCE, BigIntType.INSTANCE),
             FunctionSignature.ret(LargeIntType.INSTANCE).args(LargeIntType.INSTANCE, LargeIntType.INSTANCE),
-            FunctionSignature.ret(FloatType.INSTANCE).args(FloatType.INSTANCE, FloatType.INSTANCE),
             FunctionSignature.ret(DoubleType.INSTANCE).args(DoubleType.INSTANCE, DoubleType.INSTANCE),
+            FunctionSignature.ret(FloatType.INSTANCE).args(FloatType.INSTANCE, FloatType.INSTANCE),
             FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, DateType.INSTANCE),
             FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, DateTimeType.INSTANCE),
             FunctionSignature.ret(DateTimeType.INSTANCE).args(DateType.INSTANCE, DateTimeType.INSTANCE),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV2Type.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV2Type.java
index bbee1c1ecc..d599fbe0c3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV2Type.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV2Type.java
@@ -62,6 +62,7 @@ public class DecimalV2Type extends FractionalType {
             .put(DoubleType.INSTANCE, DOUBLE_DECIMAL)
             .put(TimeType.INSTANCE, DOUBLE_DECIMAL)
             .put(TimeV2Type.INSTANCE, DOUBLE_DECIMAL)
+            .put(NullType.INSTANCE, BOOLEAN_DECIMAL)
             .build();
 
     private final int precision;
@@ -137,7 +138,7 @@ public class DecimalV2Type extends FractionalType {
 
     @Override
     public DataType defaultConcreteType() {
-        return SYSTEM_DEFAULT;
+        return this;
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV3Type.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV3Type.java
index 259da3ea90..4c78513b50 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV3Type.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV3Type.java
@@ -60,6 +60,7 @@ public class DecimalV3Type extends FractionalType {
             .put(LargeIntType.INSTANCE, LARGEINT_DECIMAL)
             .put(FloatType.INSTANCE, FLOAT_DECIMAL)
             .put(DoubleType.INSTANCE, DOUBLE_DECIMAL)
+            .put(NullType.INSTANCE, BOOLEAN_DECIMAL)
             .build();
 
     protected final int precision;
@@ -146,7 +147,7 @@ public class DecimalV3Type extends FractionalType {
 
     @Override
     public DataType defaultConcreteType() {
-        return SYSTEM_DEFAULT;
+        return this;
     }
 
     @Override
diff --git a/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_coalesce.out b/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_coalesce.out
index 82dbea9a25..f25ac04c28 100644
--- a/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_coalesce.out
+++ b/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_coalesce.out
@@ -326,11 +326,11 @@
 -- !coalesce3 --
 \N	\N
 1	123.123
-2	1243.5
-3	24453.324
-4	243243.33
+2	1243.500
+3	24453.325
+4	243243.325
 5	243.325
-6	604587.0
+6	604587.000
 7	3.141
 8	-0.123
 9	-654.654
@@ -338,17 +338,17 @@
 11	0.666
 12	243.325
 13	100.001
-14	0.0
-15	0.0
+14	0.000
+15	0.000
 
 -- !coalesce4 --
 \N	\N
 1	123.123
-2	1243.5
-3	24453.324
-4	243243.33
+2	1243.500
+3	24453.325
+4	243243.325
 5	243.325
-6	604587.0
+6	604587.000
 7	3.141
 8	-0.123
 9	-654.654
@@ -356,8 +356,8 @@
 11	0.666
 12	243.325
 13	100.001
-14	0.0
-15	0.0
+14	0.000
+15	0.000
 
 -- !coalesce1 --
 \N	\N
diff --git a/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_nullif.out b/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_nullif.out
index dfda4676d3..f0746996f0 100644
--- a/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_nullif.out
+++ b/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_nullif.out
@@ -490,13 +490,13 @@ null	NULL	null	NULLL
 
 -- !if_nullif23 --
 123.123
-1243.5
-24453.324
+1243.500
+24453.325
 
 -- !if_nullif24 --
 123.123
-1243.5
-24453.324
+1243.500
+24453.325
 
 -- !if_nullif23 --
 false


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 22/36: [Fix] (tablet) fix tablet queryable set (#20413) (#20414)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 4889512689af123ed5b998312e38f502ec7aa1ea
Author: yujun <yu...@gmail.com>
AuthorDate: Tue Jun 6 15:38:01 2023 +0800

    [Fix] (tablet) fix tablet queryable set (#20413) (#20414)
---
 fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java    | 9 +++++++++
 .../src/main/java/org/apache/doris/clone/TabletScheduler.java    | 4 ++--
 2 files changed, 11 insertions(+), 2 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java
index 755406cf30..d9a4ee46c6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java
@@ -243,6 +243,7 @@ public class Tablet extends MetaObject implements Writable {
     // for query
     public List<Replica> getQueryableReplicas(long visibleVersion) {
         List<Replica> allQueryableReplica = Lists.newArrayListWithCapacity(replicas.size());
+        List<Replica> auxiliaryReplica = Lists.newArrayListWithCapacity(replicas.size());
         for (Replica replica : replicas) {
             if (replica.isBad()) {
                 continue;
@@ -258,9 +259,17 @@ public class Tablet extends MetaObject implements Writable {
                 if (replica.checkVersionCatchUp(visibleVersion, false)) {
                     allQueryableReplica.add(replica);
                 }
+            } else if (state == ReplicaState.DECOMMISSION) {
+                if (replica.checkVersionCatchUp(visibleVersion, false)) {
+                    auxiliaryReplica.add(replica);
+                }
             }
         }
 
+        if (allQueryableReplica.isEmpty()) {
+            allQueryableReplica = auxiliaryReplica;
+        }
+
         if (Config.skip_compaction_slower_replica && allQueryableReplica.size() > 1) {
             long minVersionCount = Long.MAX_VALUE;
             for (Replica replica : allQueryableReplica) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java
index 7578580563..c9cf4b1948 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java
@@ -1097,8 +1097,8 @@ public class TabletScheduler extends MasterDaemon {
             replica.setState(ReplicaState.DECOMMISSION);
             // set priority to normal because it may wait for a long time. Remain it as VERY_HIGH may block other task.
             tabletCtx.setOrigPriority(Priority.NORMAL);
-            LOG.debug("set replica {} on backend {} of tablet {} state to DECOMMISSION",
-                    replica.getId(), replica.getBackendId(), tabletCtx.getTabletId());
+            LOG.info("set replica {} on backend {} of tablet {} state to DECOMMISSION due to reason {}",
+                    replica.getId(), replica.getBackendId(), tabletCtx.getTabletId(), reason);
             throw new SchedException(Status.SCHEDULE_FAILED, "set watermark txn " + nextTxnId);
         } else if (replica.getState() == ReplicaState.DECOMMISSION && replica.getWatermarkTxnId() != -1) {
             long watermarkTxnId = replica.getWatermarkTxnId();


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 24/36: [fix](workload-group) fix workload group non-existence error (#20428)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 7b0de61f0f0cf495834c73e1eebd8d71b8905e4d
Author: luozenglin <lu...@baidu.com>
AuthorDate: Mon Jun 5 15:33:26 2023 +0800

    [fix](workload-group)  fix workload group non-existence error (#20428)
---
 .../main/java/org/apache/doris/qe/StmtExecutor.java   |  6 +++---
 .../resource/workloadgroup/WorkloadGroupMgr.java      | 19 ++++++++++++++-----
 2 files changed, 17 insertions(+), 8 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
index e3f76c649e..c5c23055ff 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
@@ -559,9 +559,9 @@ public class StmtExecutor {
 
     private void handleQueryWithRetry(TUniqueId queryId) throws Exception {
         // queue query here
-        if (!parsedStmt.isExplain() && Config.enable_workload_group && Config.enable_query_queue) {
-            this.queryQueue = analyzer.getEnv().getWorkloadGroupMgr()
-                    .getWorkloadGroupQueryQueue(context.sessionVariable.workloadGroup);
+        if (!parsedStmt.isExplain() && Config.enable_workload_group && Config.enable_query_queue
+                && context.getSessionVariable().enablePipelineEngine()) {
+            this.queryQueue = analyzer.getEnv().getWorkloadGroupMgr().getWorkloadGroupQueryQueue(context);
             try {
                 this.offerRet = queryQueue.offer();
             } catch (InterruptedException e) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java
index 1813271a4d..98a8d6891d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java
@@ -102,10 +102,7 @@ public class WorkloadGroupMgr implements Writable, GsonPostProcessable {
     }
 
     public List<TPipelineWorkloadGroup> getWorkloadGroup(ConnectContext context) throws UserException {
-        String groupName = context.getSessionVariable().getWorkloadGroup();
-        if (Strings.isNullOrEmpty(groupName)) {
-            groupName = Env.getCurrentEnv().getAuth().getWorkloadGroup(context.getQualifiedUser());
-        }
+        String groupName = getWorkloadGroupName(context);
         List<TPipelineWorkloadGroup> workloadGroups = Lists.newArrayList();
         readLock();
         try {
@@ -120,7 +117,8 @@ public class WorkloadGroupMgr implements Writable, GsonPostProcessable {
         return workloadGroups;
     }
 
-    public QueryQueue getWorkloadGroupQueryQueue(String groupName) throws UserException {
+    public QueryQueue getWorkloadGroupQueryQueue(ConnectContext context) throws UserException {
+        String groupName = getWorkloadGroupName(context);
         readLock();
         try {
             WorkloadGroup workloadGroup = nameToWorkloadGroup.get(groupName);
@@ -133,6 +131,17 @@ public class WorkloadGroupMgr implements Writable, GsonPostProcessable {
         }
     }
 
+    private String getWorkloadGroupName(ConnectContext context) {
+        String groupName = context.getSessionVariable().getWorkloadGroup();
+        if (Strings.isNullOrEmpty(groupName)) {
+            groupName = Env.getCurrentEnv().getAuth().getWorkloadGroup(context.getQualifiedUser());
+        }
+        if (Strings.isNullOrEmpty(groupName)) {
+            groupName = DEFAULT_GROUP_NAME;
+        }
+        return groupName;
+    }
+
     private void checkAndCreateDefaultGroup() {
         WorkloadGroup defaultWorkloadGroup = null;
         writeLock();


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 04/36: [fix](dynamic partition) partition create failed after alter distributed column (#20239)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit b79fd59c7dd888d344901d473c1077843d7677b9
Author: camby <zh...@baidu.com>
AuthorDate: Mon Jun 5 12:20:50 2023 +0800

    [fix](dynamic partition) partition create failed after alter distributed column (#20239)
    
    This pr fix following two problems:
    
    Problem1: Alter column comment make add dynamic partition failed inside issue #10811
    
    create table with dynamic partition policy;
    restart FE;
    alter distribution column comment;
    alter dynamic_partition.end to trigger add new partition by dynamic partition scheduler;
    Then we got the error log, and the new partition create failed.
    dynamic add partition failed: errCode = 2, detailMessage =      Cannot assign hash distribution with different distribution cols. default is: [id int(11) NULL COMMENT 'new_comment_of_id'], db: default_cluster:example_db, table: test_2
    Problem2: rename distributed column, make old partition insert failed. inside #20405
    
    The key point of the reproduce steps is restart FE.
    
    It seems all versions will be affected, include master and lts-1.1 and so on.
---
 .../doris/analysis/HashDistributionDesc.java       |  4 +-
 .../main/java/org/apache/doris/catalog/Column.java | 22 ++++++++
 .../main/java/org/apache/doris/catalog/Env.java    | 25 ++++++---
 .../apache/doris/catalog/HashDistributionInfo.java | 14 ++++-
 .../apache/doris/datasource/InternalCatalog.java   | 13 ++---
 regression-test/pipeline/p0/conf/fe.conf           |  2 +
 regression-test/pipeline/p1/conf/fe.conf           |  2 +
 .../test_dynamic_partition_with_alter.groovy       | 56 ++++++++++++++++++++
 .../test_dynamic_partition_with_rename.groovy      | 60 ++++++++++++++++++++++
 9 files changed, 181 insertions(+), 17 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/HashDistributionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/HashDistributionDesc.java
index d54049f793..374bf10bdb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/HashDistributionDesc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/HashDistributionDesc.java
@@ -131,7 +131,9 @@ public class HashDistributionDesc extends DistributionDesc {
                                 + column.getName() + "].");
                     }
 
-                    distributionColumns.add(column);
+                    // distribution info and base columns persist seperately inside OlapTable, so we need deep copy
+                    // to avoid modify table columns also modify columns inside distribution info.
+                    distributionColumns.add(new Column(column));
                     find = true;
                     break;
                 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
index 4e6a5a3edf..f9722b47c0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
@@ -784,6 +784,28 @@ public class Column implements Writable, GsonPostProcessable {
                 && Objects.equals(realDefaultValue, other.realDefaultValue);
     }
 
+    // distribution column compare only care about attrs which affect data,
+    // do not care about attrs, such as comment
+    public boolean equalsForDistribution(Column other) {
+        if (other == this) {
+            return true;
+        }
+
+        return name.equalsIgnoreCase(other.name)
+                && Objects.equals(getDefaultValue(), other.getDefaultValue())
+                && Objects.equals(aggregationType, other.aggregationType)
+                && isAggregationTypeImplicit == other.isAggregationTypeImplicit
+                && isKey == other.isKey
+                && isAllowNull == other.isAllowNull
+                && getDataType().equals(other.getDataType())
+                && getStrLen() == other.getStrLen()
+                && getPrecision() == other.getPrecision()
+                && getScale() == other.getScale()
+                && visible == other.visible
+                && Objects.equals(children, other.children)
+                && Objects.equals(realDefaultValue, other.realDefaultValue);
+    }
+
     @Override
     public void write(DataOutput out) throws IOException {
         String json = GsonUtils.GSON.toJson(this);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
index 4565a6bc48..011b0c3ebd 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
@@ -4279,12 +4279,26 @@ public class Env {
         // 4. modify distribution info
         DistributionInfo distributionInfo = table.getDefaultDistributionInfo();
         if (distributionInfo.getType() == DistributionInfoType.HASH) {
+            // modify default distribution info
             List<Column> distributionColumns = ((HashDistributionInfo) distributionInfo).getDistributionColumns();
             for (Column column : distributionColumns) {
                 if (column.getName().equalsIgnoreCase(colName)) {
                     column.setName(newColName);
                 }
             }
+            // modify distribution info inside partitions
+            for (Partition p : table.getPartitions()) {
+                DistributionInfo partDistInfo = p.getDistributionInfo();
+                if (partDistInfo.getType() != DistributionInfoType.HASH) {
+                    continue;
+                }
+                List<Column> partDistColumns = ((HashDistributionInfo) partDistInfo).getDistributionColumns();
+                for (Column column : partDistColumns) {
+                    if (column.getName().equalsIgnoreCase(colName)) {
+                        column.setName(newColName);
+                    }
+                }
+            }
         }
 
         // 5. modify sequence map col
@@ -4535,13 +4549,10 @@ public class Env {
                 }
                 if (distributionInfo.getType() == DistributionInfoType.HASH) {
                     HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo;
-                    List<Column> newDistriCols = hashDistributionInfo.getDistributionColumns();
-                    List<Column> defaultDistriCols
-                            = ((HashDistributionInfo) defaultDistributionInfo).getDistributionColumns();
-                    if (!newDistriCols.equals(defaultDistriCols)) {
-                        throw new DdlException(
-                                "Cannot assign hash distribution with different distribution cols. " + "default is: "
-                                        + defaultDistriCols);
+                    if (!hashDistributionInfo.sameDistributionColumns((HashDistributionInfo) defaultDistributionInfo)) {
+                        throw new DdlException("Cannot assign hash distribution with different distribution cols. "
+                                + "new is: " + hashDistributionInfo.getDistributionColumns() + " default is: "
+                                + ((HashDistributionInfo) distributionInfo).getDistributionColumns());
                     }
                 }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/HashDistributionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/HashDistributionInfo.java
index d746bd355f..5f30bc2098 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/HashDistributionInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/HashDistributionInfo.java
@@ -95,6 +95,18 @@ public class HashDistributionInfo extends DistributionInfo {
         return distributionInfo;
     }
 
+    public boolean sameDistributionColumns(HashDistributionInfo other) {
+        if (distributionColumns.size() != other.distributionColumns.size()) {
+            return false;
+        }
+        for (int i = 0; i < distributionColumns.size(); ++i) {
+            if (!distributionColumns.get(i).equalsForDistribution(other.distributionColumns.get(i))) {
+                return false;
+            }
+        }
+        return true;
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o) {
@@ -107,7 +119,7 @@ public class HashDistributionInfo extends DistributionInfo {
             return false;
         }
         HashDistributionInfo that = (HashDistributionInfo) o;
-        return bucketNum == that.bucketNum && Objects.equals(distributionColumns, that.distributionColumns);
+        return bucketNum == that.bucketNum && sameDistributionColumns(that);
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
index d1c0354337..1fc3b20bcc 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
@@ -1410,17 +1410,14 @@ public class InternalCatalog implements CatalogIf<Database> {
 
                 if (distributionInfo.getType() == DistributionInfoType.HASH) {
                     HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo;
-                    List<Column> newDistriCols = hashDistributionInfo.getDistributionColumns();
-                    List<Column> defaultDistriCols = ((HashDistributionInfo) defaultDistributionInfo)
-                            .getDistributionColumns();
-                    if (!newDistriCols.equals(defaultDistriCols)) {
-                        throw new DdlException(
-                                "Cannot assign hash distribution with different distribution cols. " + "default is: "
-                                        + defaultDistriCols);
-                    }
                     if (hashDistributionInfo.getBucketNum() <= 0) {
                         throw new DdlException("Cannot assign hash distribution buckets less than 1");
                     }
+                    if (!hashDistributionInfo.sameDistributionColumns((HashDistributionInfo) defaultDistributionInfo)) {
+                        throw new DdlException("Cannot assign hash distribution with different distribution cols. "
+                                + "new is: " + hashDistributionInfo.getDistributionColumns() + " default is: "
+                                + ((HashDistributionInfo) distributionInfo).getDistributionColumns());
+                    }
                 } else if (distributionInfo.getType() == DistributionInfoType.RANDOM) {
                     RandomDistributionInfo randomDistributionInfo = (RandomDistributionInfo) distributionInfo;
                     if (randomDistributionInfo.getBucketNum() <= 0) {
diff --git a/regression-test/pipeline/p0/conf/fe.conf b/regression-test/pipeline/p0/conf/fe.conf
index 5879c463ee..d2e8cbd282 100644
--- a/regression-test/pipeline/p0/conf/fe.conf
+++ b/regression-test/pipeline/p0/conf/fe.conf
@@ -80,3 +80,5 @@ enable_struct_type=true
 
 # enable mtmv
 enable_mtmv = true
+
+dynamic_partition_check_interval_seconds=5
diff --git a/regression-test/pipeline/p1/conf/fe.conf b/regression-test/pipeline/p1/conf/fe.conf
index b7fbf07bcb..ba7de606b5 100644
--- a/regression-test/pipeline/p1/conf/fe.conf
+++ b/regression-test/pipeline/p1/conf/fe.conf
@@ -82,3 +82,5 @@ enable_mtmv = true
 # enable auto collect statistics
 enable_auto_collect_statistics=true
 auto_check_statistics_in_sec=60
+
+dynamic_partition_check_interval_seconds=5
diff --git a/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_alter.groovy b/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_alter.groovy
new file mode 100644
index 0000000000..32cfc742a7
--- /dev/null
+++ b/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_alter.groovy
@@ -0,0 +1,56 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_dynamic_partition_with_alter") {
+    def tbl = "test_dynamic_partition_with_alter"
+    sql "drop table if exists ${tbl}"
+    sql """
+        CREATE TABLE IF NOT EXISTS ${tbl}
+        ( k1 date NOT NULL, k2 varchar(20) NOT NULL, k3 int sum NOT NULL )
+        AGGREGATE KEY(k1,k2)
+        PARTITION BY RANGE(k1) ( )
+        DISTRIBUTED BY HASH(k1) BUCKETS 1
+        PROPERTIES (
+            "dynamic_partition.enable"="true",
+            "dynamic_partition.end"="3",
+            "dynamic_partition.buckets"="1",
+            "dynamic_partition.start"="-3",
+            "dynamic_partition.prefix"="p",
+            "dynamic_partition.time_unit"="DAY",
+            "dynamic_partition.create_history_partition"="true",
+            "dynamic_partition.replication_allocation" = "tag.location.default: 1")
+        """
+    result = sql "show partitions from ${tbl}"
+    assertEquals(7, result.size())
+
+    // modify distributed column comment, then try to add too more dynamic partition
+    sql """ alter table ${tbl} modify column k1 comment 'new_comment_for_k1' """
+    sql """ ADMIN SET FRONTEND CONFIG ('dynamic_partition_check_interval_seconds' = '1') """
+    sql """ alter table ${tbl} set('dynamic_partition.end'='5') """
+    result = sql "show partitions from ${tbl}"
+    for (def retry = 0; retry < 15; retry++) {
+        if (result.size() == 9) {
+            break;
+        }
+        logger.info("wait dynamic partition scheduler, sleep 1s")
+        sleep(1000);
+        result = sql "show partitions from ${tbl}"
+    }
+    assertEquals(9, result.size())
+
+    sql "drop table ${tbl}"
+}
diff --git a/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_rename.groovy b/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_rename.groovy
new file mode 100644
index 0000000000..b07a2f1a63
--- /dev/null
+++ b/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_rename.groovy
@@ -0,0 +1,60 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_dynamic_partition_with_rename") {
+    def tbl = "test_dynamic_partition_with_rename"
+    sql "drop table if exists ${tbl}"
+    sql """
+        CREATE TABLE IF NOT EXISTS ${tbl}
+        ( k1 date NOT NULL, k2 varchar(20) NOT NULL, k3 int sum NOT NULL )
+        AGGREGATE KEY(k1,k2)
+        PARTITION BY RANGE(k1) ( )
+        DISTRIBUTED BY HASH(k1) BUCKETS 1
+        PROPERTIES (
+            "dynamic_partition.enable"="true",
+            "dynamic_partition.end"="3",
+            "dynamic_partition.buckets"="1",
+            "dynamic_partition.start"="-3",
+            "dynamic_partition.prefix"="p",
+            "dynamic_partition.time_unit"="DAY",
+            "dynamic_partition.create_history_partition"="true",
+            "dynamic_partition.replication_allocation" = "tag.location.default: 1")
+        """
+    result = sql "show partitions from ${tbl}"
+    assertEquals(7, result.size())
+
+    // rename distributed column, then try to add too more dynamic partition
+    sql "alter table ${tbl} rename column k1 renamed_k1"
+    sql """ ADMIN SET FRONTEND CONFIG ('dynamic_partition_check_interval_seconds' = '1') """
+    sql """ alter table ${tbl} set('dynamic_partition.end'='5') """
+    result = sql "show partitions from ${tbl}"
+    for (def retry = 0; retry < 15; retry++) {
+        if (result.size() == 9) {
+            break;
+        }
+        logger.info("wait dynamic partition scheduler, sleep 1s")
+        sleep(1000);
+        result = sql "show partitions from ${tbl}"
+    }
+    assertEquals(9, result.size())
+    for (def line = 0; line < result.size(); line++) {
+        // XXX: DistributionKey at pos(7), next maybe impl by sql meta
+        assertEquals("renamed_k1", result.get(line).get(7))
+    }
+
+    sql "drop table ${tbl}"
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 34/36: [Fix](WorkloadGroup)Fix query queue nereids bug #20484

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 8e2ae30b20551de2998a5e9eccd023b6c4339abf
Author: Lijia Liu <li...@yeah.net>
AuthorDate: Tue Jun 6 16:44:35 2023 +0800

    [Fix](WorkloadGroup)Fix query queue nereids bug #20484
---
 .../src/main/java/org/apache/doris/analysis/Analyzer.java     | 11 -----------
 fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java |  5 ++++-
 .../src/main/java/org/apache/doris/qe/StmtExecutor.java       | 10 ++++------
 3 files changed, 8 insertions(+), 18 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
index 117124f6ac..c76cb0fa0b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
@@ -69,7 +69,6 @@ import org.apache.doris.rewrite.mvrewrite.ExprToSlotRefRule;
 import org.apache.doris.rewrite.mvrewrite.HLLHashToSlotRefRule;
 import org.apache.doris.rewrite.mvrewrite.NDVToHll;
 import org.apache.doris.rewrite.mvrewrite.ToBitmapToSlotRefRule;
-import org.apache.doris.thrift.TPipelineWorkloadGroup;
 import org.apache.doris.thrift.TQueryGlobals;
 
 import com.google.common.base.Joiner;
@@ -408,8 +407,6 @@ public class Analyzer {
 
         private final Map<InlineViewRef, Set<Expr>> migrateFailedConjuncts = Maps.newHashMap();
 
-        public List<TPipelineWorkloadGroup> tWorkloadGroups;
-
         public GlobalState(Env env, ConnectContext context) {
             this.env = env;
             this.context = context;
@@ -597,14 +594,6 @@ public class Analyzer {
         return explicitViewAlias;
     }
 
-    public void setWorkloadGroups(List<TPipelineWorkloadGroup> tWorkloadGroups) {
-        globalState.tWorkloadGroups = tWorkloadGroups;
-    }
-
-    public List<TPipelineWorkloadGroup> getWorkloadGroups() {
-        return globalState.tWorkloadGroups;
-    }
-
     /**
      * Registers a local view definition with this analyzer. Throws an exception if a view
      * definition with the same alias has already been registered or if the number of
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
index a042c0a8ad..91e01e6399 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
@@ -260,6 +260,10 @@ public class Coordinator {
 
     private StatsErrorEstimator statsErrorEstimator;
 
+    public void setTWorkloadGroups(List<TPipelineWorkloadGroup> tWorkloadGroups) {
+        this.tWorkloadGroups = tWorkloadGroups;
+    }
+
     private List<TPipelineWorkloadGroup> tWorkloadGroups = Lists.newArrayList();
 
     private final ExecutionProfile executionProfile;
@@ -351,7 +355,6 @@ public class Coordinator {
         nextInstanceId.setHi(queryId.hi);
         nextInstanceId.setLo(queryId.lo + 1);
         this.assignedRuntimeFilters = planner.getRuntimeFilters();
-        this.tWorkloadGroups = analyzer == null ? null : analyzer.getWorkloadGroups();
         this.executionProfile = new ExecutionProfile(queryId, fragments.size());
 
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
index a06f6ee855..4799b21448 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
@@ -561,7 +561,7 @@ public class StmtExecutor {
         // queue query here
         if (!parsedStmt.isExplain() && Config.enable_workload_group && Config.enable_query_queue
                 && context.getSessionVariable().enablePipelineEngine()) {
-            this.queryQueue = analyzer.getEnv().getWorkloadGroupMgr().getWorkloadGroupQueryQueue(context);
+            this.queryQueue = context.getEnv().getWorkloadGroupMgr().getWorkloadGroupQueryQueue(context);
             try {
                 this.offerRet = queryQueue.offer();
             } catch (InterruptedException e) {
@@ -1098,11 +1098,6 @@ public class StmtExecutor {
                     parsedStmt.setIsExplain(explainOptions);
                 }
             }
-            if (parsedStmt instanceof QueryStmt && Config.enable_workload_group
-                    && context.sessionVariable.enablePipelineEngine()) {
-                analyzer.setWorkloadGroups(analyzer.getEnv().getWorkloadGroupMgr()
-                        .getWorkloadGroup(context));
-            }
         }
         profile.getSummaryProfile().setQueryAnalysisFinishTime();
         planner = new OriginalPlanner(analyzer);
@@ -1361,6 +1356,9 @@ public class StmtExecutor {
         // 2. If this is a query, send the result expr fields first, and send result data back to client.
         RowBatch batch;
         coord = new Coordinator(context, analyzer, planner, context.getStatsErrorEstimator());
+        if (Config.enable_workload_group && context.sessionVariable.enablePipelineEngine()) {
+            coord.setTWorkloadGroups(context.getEnv().getWorkloadGroupMgr().getWorkloadGroup(context));
+        }
         QeProcessorImpl.INSTANCE.registerQuery(context.queryId(),
                 new QeProcessorImpl.QueryInfo(context, originStmt.originStmt, coord));
         profile.addExecutionProfile(coord.getExecutionProfile());


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 28/36: [bug](jdbc) fix trino date/datetime filter (#20443)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 5146a71c4f975c48150557b0d68e4d581fc8b5c8
Author: zy-kkk <zh...@gmail.com>
AuthorDate: Tue Jun 6 11:20:42 2023 +0800

    [bug](jdbc) fix trino date/datetime filter (#20443)
    
    When querying Trino's JDBC catalog, if our WHERE filter condition is k1 >= '2022-01-01', this format is incorrect.
    In Trino, the correct format should be k1 >= date '2022-01-01' or k1 >= timestamp '2022-01-01 00:00:00'.
    Therefore, the date string in the WHERE condition needs to be converted to the date or timestamp format supported by Trino.
---
 .../java/org/apache/doris/planner/OdbcScanNode.java   | 19 +++++++++++++++++++
 1 file changed, 19 insertions(+)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OdbcScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OdbcScanNode.java
index e21af36af1..3439987f16 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/OdbcScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OdbcScanNode.java
@@ -85,6 +85,25 @@ public class OdbcScanNode extends ExternalScanNode {
                 return filter;
             }
         }
+        if (tableType.equals(TOdbcTableType.TRINO) && expr.contains(DateLiteral.class)
+                && (expr instanceof BinaryPredicate)) {
+            ArrayList<Expr> children = expr.getChildren();
+            if (children.get(1).isConstant() && (children.get(1).getType().isDate()) || children
+                    .get(1).getType().isDateV2()) {
+                String filter = children.get(0).toSql();
+                filter += ((BinaryPredicate) expr).getOp().toString();
+                filter += "date '" + children.get(1).getStringValue() + "'";
+                return filter;
+            }
+            if (children.get(1).isConstant() && (children.get(1).getType().isDatetime() || children
+                    .get(1).getType().isDatetimeV2())) {
+                String filter = children.get(0).toSql();
+                filter += ((BinaryPredicate) expr).getOp().toString();
+                filter += "timestamp '" + children.get(1).getStringValue() + "'";
+                return filter;
+            }
+        }
+
         return expr.toMySql();
     }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 26/36: [fix](execution) result_filter_data should be filled by 0 when can_filter_all is true (#20438)

Posted by kx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 34eca4d69251bade0835d8468f0cb24c2610b32d
Author: starocean999 <40...@users.noreply.github.com>
AuthorDate: Mon Jun 5 17:05:35 2023 +0800

    [fix](execution) result_filter_data should be filled by 0 when can_filter_all is true (#20438)
---
 be/src/vec/exprs/vexpr_context.cpp                 |  1 +
 .../data/correctness_p0/test_function_conjunct.out |  5 +++
 .../correctness_p0/test_function_conjunct.groovy   | 45 ++++++++++++++++++++++
 3 files changed, 51 insertions(+)

diff --git a/be/src/vec/exprs/vexpr_context.cpp b/be/src/vec/exprs/vexpr_context.cpp
index 98af614797..88796ecc24 100644
--- a/be/src/vec/exprs/vexpr_context.cpp
+++ b/be/src/vec/exprs/vexpr_context.cpp
@@ -195,6 +195,7 @@ Status VExprContext::execute_conjuncts(const VExprContextSPtrs& ctxs,
             // filter all
             if (!const_column->get_bool(0)) {
                 *can_filter_all = true;
+                memset(result_filter_data, 0, result_filter->size());
                 return Status::OK();
             }
         } else {
diff --git a/regression-test/data/correctness_p0/test_function_conjunct.out b/regression-test/data/correctness_p0/test_function_conjunct.out
new file mode 100644
index 0000000000..863b85b658
--- /dev/null
+++ b/regression-test/data/correctness_p0/test_function_conjunct.out
@@ -0,0 +1,5 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select_default --
+
+-- !select_default --
+
diff --git a/regression-test/suites/correctness_p0/test_function_conjunct.groovy b/regression-test/suites/correctness_p0/test_function_conjunct.groovy
new file mode 100644
index 0000000000..1d747bb973
--- /dev/null
+++ b/regression-test/suites/correctness_p0/test_function_conjunct.groovy
@@ -0,0 +1,45 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_function_conjunct") {
+    def tableName = "test_function_conjunct_table"
+    sql """ DROP TABLE IF EXISTS ${tableName} """
+    sql """
+            CREATE TABLE IF NOT EXISTS ${tableName} (
+                a varchar(10) not null, 
+                b int not null
+            ) ENGINE=OLAP
+            DUPLICATE KEY(a)
+            DISTRIBUTED BY HASH(a) BUCKETS 2
+            PROPERTIES (
+            "replication_num" = "1",
+            "in_memory" = "false",
+            "storage_format" = "V2"
+            );
+    """
+
+    sql """ INSERT INTO ${tableName} VALUES ("1","1"); """
+
+    sql "set enable_nereids_planner=true"
+    qt_select_default """select a, b from ${tableName} where coalesce(b, null) is NULL;"""
+
+    sql "set enable_nereids_planner=false"
+    qt_select_default """select a, b from ${tableName} where coalesce(b, null) is NULL;"""
+
+    sql """ DROP TABLE IF EXISTS ${tableName} """
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org