You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2023/04/19 13:34:26 UTC

[doris] branch branch-2.0-alpha updated (7dd96bc341 -> 701397fb50)

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

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


    from 7dd96bc341 [fix](olap) remove zorder support when create table (#18698)
     new d64e483163 [Bug](pipeline) regression heap use after free (#18701)
     new 18ccf8a66d [feature](multicatalog) enable doris hive/iceberg catalog to read data on tencent GooseFS (#18685)
     new 08f4cf03e8 [regression-test](prepared statement) Fix connection error when test framework used lower jdbc version (#18665)
     new e4d11ca3ad [fix](executor)Fix scanner's _max_thread_num may == 0 #18465
     new f10f9c640e [regression-test] add some suites (#18286)
     new 6976f3ae32 [feature](config) support "experimental" prefix for FE config (#18699)
     new 9013b3712c [Fix](catalog)Fix hudi-catalog get file split error (#18644) (#18673)
     new 982e437470 [chore](build) add apache-orc git submodule path (#18695)
     new b883b2019a [BugFix](functions) fix multi_search_all_positions #18682
     new dce24e6383 [chore](build) Use include-what-you-use to optimize includes (#18681)
     new 64bb7c278e [Bug](DECIMALV3) report failure if a decimal value is overflow (#18336)
     new b0014fb2a9 [fix](bitmap) fix bitmap_count errors to set nullable to non-nullable bitmap col (#18689)
     new 2c67b8de34 [fix](Nereids): fix sum func in eager agg (#18675)
     new bb4a28d806 [Bug](DECIMAL) Fix bug for arithmatic expr DECIMALV2 / DECIMALV3 (#18723)
     new 7e5790c995 [feature](segcompaction) enable segcompaction by default (#18722)
     new cd37185860 [fix](build) Fix missing header files (#18740)
     new 06b61711e9 [fix](Nereids): when GroupExpr already exists, we need to remove ParentExpression (#18749)
     new cf224e932d [Compile](BE) Fix compile failed with tcmalloc (#18748)
     new f28944d9fe [fix](merge-on-write) the mow keyset check is quite slow when loading large data set, remove it from release version (#18750)
     new 47b652b13d [fix](multi-catalog)fix old s3 properties check  (#18430)
     new 353f5fc246 [chore](hashtable) Use doris' Allocator to replace std::allocator in phmap (#18735)
     new 9d199e3868 [vectorzied](function) fix array_map function analyzed failed with order by clause (#18676)
     new 941f4655fa [fix](const column) fix coredump caused by const column for some functions (#18737)
     new 6dc7237346 [fix](olap) fix lost disable_auto_compaction info when fe restart (#18757)
     new 1f08b77200 [fix](stats) Stats still in cache after user dropped it (#18720)
     new 23a3229d9a [fix](merge-on-write) enable_unique_key_merge_on_write property should only be used for unique table (#18734)
     new fb1f54ddd5 [enhancement](exception) Column filter/replicate supports exception safety (#18503)
     new 93655e446d fix doc (#18772)
     new 588e3623f1 [Fix](vertical compaction) Preserve _segment_num_rows during final segment flush (#18779)
     new 4f1e136206 [typo](docs) fix insert load doc error (#18773)
     new 8a3fc223fe [fix](compile) Fix block.cpp compilation failure (#18797)
     new a0f1cbe023 [Doc] Fix error test example (#18764)
     new 2f6f3ff208 [minor](decimal) forbid to create table with decimal type exceeds 18 (#18763)
     new 8bc6413866 [Fix](planner)Fix TupleDescriptor include not materialized slot bug (#18783)
     new 563900e23d [regression-test](iceberg)add iceberg in regression case (#18792)
     new 701397fb50 [fix](Nereids): need update parentExpression after replace child. (#18771)

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:
 .gitignore                                         |    1 +
 be/src/agent/agent_server.cpp                      |   12 +
 be/src/agent/agent_server.h                        |   10 +-
 be/src/agent/heartbeat_server.cpp                  |   19 +-
 be/src/agent/heartbeat_server.h                    |    8 +-
 be/src/agent/task_worker_pool.cpp                  |   34 +-
 be/src/agent/task_worker_pool.h                    |   19 +-
 be/src/agent/topic_subscriber.cpp                  |    7 +
 be/src/agent/topic_subscriber.h                    |    5 +-
 be/src/agent/user_resource_listener.cpp            |   19 +-
 be/src/agent/user_resource_listener.h              |    7 +-
 be/src/agent/utils.cpp                             |   23 +-
 be/src/agent/utils.h                               |   10 +
 be/src/common/config.h                             |    4 +-
 be/src/common/configbase.cpp                       |   21 +-
 be/src/common/configbase.h                         |    1 +
 be/src/common/daemon.cpp                           |   26 +-
 be/src/common/exception.h                          |   11 +
 be/src/common/logconfig.cpp                        |    6 +-
 be/src/common/logging.h                            |    2 +-
 be/src/common/resource_tls.cpp                     |    4 +-
 be/src/common/status.cpp                           |    8 +-
 be/src/common/status.h                             |    7 +-
 be/src/exec/arrow/arrow_reader.cpp                 |   23 +-
 be/src/exec/arrow/arrow_reader.h                   |   28 +-
 be/src/exec/arrow/parquet_reader.cpp               |   23 +-
 be/src/exec/arrow/parquet_reader.h                 |   18 +-
 be/src/exec/base_scanner.cpp                       |   30 +-
 be/src/exec/base_scanner.h                         |   16 +
 be/src/exec/data_sink.cpp                          |   21 +-
 be/src/exec/data_sink.h                            |   14 +-
 be/src/exec/decompressor.cpp                       |    4 +
 be/src/exec/decompressor.h                         |    4 +
 be/src/exec/es/es_scan_reader.cpp                  |    4 +
 be/src/exec/es/es_scan_reader.h                    |    6 +
 be/src/exec/es/es_scroll_parser.cpp                |   24 +-
 be/src/exec/es/es_scroll_parser.h                  |    6 +
 be/src/exec/es/es_scroll_query.cpp                 |    5 +
 be/src/exec/exec_node.cpp                          |   21 +-
 be/src/exec/exec_node.h                            |   13 +-
 be/src/exec/odbc_connector.cpp                     |   16 +-
 be/src/exec/odbc_connector.h                       |   12 +
 be/src/exec/olap_common.cpp                        |    4 -
 be/src/exec/olap_common.h                          |   13 +
 be/src/exec/rowid_fetcher.cpp                      |   26 +-
 be/src/exec/rowid_fetcher.h                        |   14 +-
 be/src/exec/scan_node.cpp                          |    9 +
 be/src/exec/scan_node.h                            |   11 +-
 be/src/exec/schema_scanner.cpp                     |   21 +-
 be/src/exec/schema_scanner.h                       |   12 +-
 .../schema_scanner/schema_charsets_scanner.cpp     |    8 +
 .../exec/schema_scanner/schema_charsets_scanner.h  |    6 +
 .../schema_scanner/schema_collations_scanner.cpp   |    9 +-
 .../schema_scanner/schema_collations_scanner.h     |    6 +
 .../exec/schema_scanner/schema_columns_scanner.cpp |   16 +-
 .../exec/schema_scanner/schema_columns_scanner.h   |    9 +-
 .../exec/schema_scanner/schema_dummy_scanner.cpp   |    9 +
 be/src/exec/schema_scanner/schema_dummy_scanner.h  |    5 +
 .../exec/schema_scanner/schema_files_scanner.cpp   |   10 +-
 be/src/exec/schema_scanner/schema_files_scanner.h  |   10 +-
 be/src/exec/schema_scanner/schema_helper.cpp       |   17 +-
 be/src/exec/schema_scanner/schema_helper.h         |   18 +-
 .../schema_scanner/schema_partitions_scanner.cpp   |   12 +-
 .../schema_scanner/schema_partitions_scanner.h     |   10 +-
 .../exec/schema_scanner/schema_rowsets_scanner.cpp |   24 +-
 .../exec/schema_scanner/schema_rowsets_scanner.h   |    8 +
 .../schema_schema_privileges_scanner.cpp           |   12 +-
 .../schema_schema_privileges_scanner.h             |   10 +-
 .../schema_scanner/schema_schemata_scanner.cpp     |   12 +-
 .../exec/schema_scanner/schema_schemata_scanner.h  |   10 +-
 .../schema_scanner/schema_statistics_scanner.cpp   |    4 +-
 .../schema_scanner/schema_statistics_scanner.h     |    2 +
 .../schema_table_privileges_scanner.cpp            |   10 +-
 .../schema_table_privileges_scanner.h              |   10 +-
 .../exec/schema_scanner/schema_tables_scanner.cpp  |   18 +-
 be/src/exec/schema_scanner/schema_tables_scanner.h |   10 +-
 .../schema_user_privileges_scanner.cpp             |   12 +-
 .../schema_user_privileges_scanner.h               |   10 +-
 .../schema_scanner/schema_variables_scanner.cpp    |   16 +-
 .../exec/schema_scanner/schema_variables_scanner.h |   11 +-
 .../exec/schema_scanner/schema_views_scanner.cpp   |   12 +-
 be/src/exec/schema_scanner/schema_views_scanner.h  |   10 +-
 be/src/exec/table_connector.cpp                    |   21 +-
 be/src/exec/table_connector.h                      |   12 +
 be/src/exec/tablet_info.cpp                        |   18 +
 be/src/exec/tablet_info.h                          |   21 +-
 be/src/exec/text_converter.cpp                     |   14 +-
 be/src/exec/text_converter.h                       |    3 +
 be/src/exprs/block_bloom_filter_avx_impl.cc        |    6 +-
 be/src/exprs/block_bloom_filter_impl.cc            |    9 +-
 be/src/exprs/hybrid_set.h                          |    7 +-
 be/src/exprs/json_functions.cpp                    |   16 +-
 be/src/exprs/json_functions.h                      |   13 +
 be/src/exprs/math_functions.cpp                    |   11 +-
 be/src/exprs/math_functions.h                      |    1 +
 be/src/exprs/runtime_filter.cpp                    |   21 +-
 be/src/exprs/runtime_filter.h                      |   22 +
 be/src/exprs/runtime_filter_rpc.cpp                |   15 +-
 be/src/exprs/string_functions.cpp                  |    7 +-
 be/src/exprs/string_functions.h                    |    2 +
 be/src/geo/ByteOrderDataInStream.h                 |    2 +
 be/src/geo/ByteOrderValues.cpp                     |    3 +-
 be/src/geo/ByteOrderValues.h                       |    5 +
 be/src/geo/geo_tobinary.cpp                        |   10 +-
 be/src/geo/geo_tobinary.h                          |    8 +
 be/src/geo/geo_types.cpp                           |   14 +-
 be/src/geo/geo_types.h                             |    3 +-
 be/src/geo/machine.h                               |    2 +
 be/src/geo/wkb_parse.cpp                           |   12 +-
 be/src/geo/wkb_parse.h                             |   12 +
 be/src/geo/wkt_parse.cpp                           |    3 +-
 be/src/geo/wkt_parse.h                             |    4 +
 be/src/http/action/check_rpc_channel_action.cpp    |   14 +-
 be/src/http/action/check_rpc_channel_action.h      |    2 +
 be/src/http/action/check_tablet_segment_action.cpp |    7 +
 be/src/http/action/check_tablet_segment_action.h   |    1 +
 be/src/http/action/checksum_action.cpp             |    7 +-
 be/src/http/action/checksum_action.h               |    1 +
 be/src/http/action/compaction_action.cpp           |   13 +-
 be/src/http/action/compaction_action.h             |    5 +
 be/src/http/action/config_action.cpp               |    8 +-
 be/src/http/action/config_action.h                 |    1 +
 be/src/http/action/download_action.cpp             |   12 +-
 be/src/http/action/download_action.h               |    5 +
 be/src/http/action/health_action.cpp               |    1 -
 be/src/http/action/health_action.h                 |    1 +
 be/src/http/action/jeprofile_actions.cpp           |    9 +-
 be/src/http/action/meta_action.cpp                 |   13 +-
 be/src/http/action/meta_action.h                   |    3 +
 be/src/http/action/metrics_action.cpp              |    7 -
 be/src/http/action/monitor_action.cpp              |    5 +-
 be/src/http/action/pad_rowset_action.cpp           |   15 +-
 be/src/http/action/pad_rowset_action.h             |    2 +
 be/src/http/action/pprof_actions.cpp               |   17 +-
 be/src/http/action/reload_tablet_action.cpp        |    7 +-
 be/src/http/action/reload_tablet_action.h          |    7 +-
 be/src/http/action/reset_rpc_channel_action.cpp    |    8 +-
 be/src/http/action/reset_rpc_channel_action.h      |    2 +
 be/src/http/action/restore_tablet_action.cpp       |   19 +-
 be/src/http/action/restore_tablet_action.h         |    6 +-
 be/src/http/action/snapshot_action.cpp             |   10 +-
 be/src/http/action/snapshot_action.h               |    1 +
 be/src/http/action/stream_load.cpp                 |   32 +-
 be/src/http/action/stream_load.h                   |    7 +-
 be/src/http/action/stream_load_2pc.cpp             |   11 +-
 be/src/http/action/stream_load_2pc.h               |    1 +
 be/src/http/action/tablet_migration_action.cpp     |    7 +-
 be/src/http/action/tablet_migration_action.h       |   14 +-
 be/src/http/action/tablets_distribution_action.cpp |   10 +-
 be/src/http/action/tablets_distribution_action.h   |    3 +
 be/src/http/action/tablets_info_action.cpp         |    9 +
 be/src/http/action/tablets_info_action.h           |    1 +
 be/src/http/action/version_action.cpp              |    5 +-
 be/src/http/action/version_action.h                |    1 +
 be/src/http/default_path_handlers.cpp              |   18 +-
 be/src/http/ev_http_server.cpp                     |   19 +-
 be/src/http/ev_http_server.h                       |    3 +
 be/src/http/http_channel.cpp                       |    7 +-
 be/src/http/http_channel.h                         |    2 +
 be/src/http/http_client.cpp                        |    6 +
 be/src/http/http_client.h                          |    3 +
 be/src/http/http_method.cpp                        |    1 +
 be/src/http/http_request.cpp                       |    7 +-
 be/src/http/http_status.cpp                        |    1 +
 be/src/http/utils.cpp                              |   10 +-
 be/src/http/utils.h                                |    1 +
 be/src/http/web_page_handler.cpp                   |   10 +-
 be/src/http/web_page_handler.h                     |    3 +
 be/src/io/fs/file_reader_writer_fwd.h              |    3 +
 be/src/io/fs/hdfs.h                                |    4 +-
 be/src/io/fs/hdfs_file_reader.h                    |   12 +
 be/src/io/fs/hdfs_file_writer.h                    |    7 +
 be/src/olap/hll.cpp                                |    2 +-
 be/src/olap/hll.h                                  |    4 +-
 be/src/olap/in_list_predicate.h                    |    2 -
 be/src/olap/reader.cpp                             |    2 -
 be/src/olap/rowset/beta_rowset_writer.cpp          |    2 +
 be/src/olap/rowset/segment_v2/binary_dict_page.h   |    2 -
 be/src/olap/rowset/segment_v2/segment_iterator.cpp |   15 +-
 be/src/olap/rowset/segment_v2/segment_writer.cpp   |    4 +
 be/src/olap/rowset/vertical_beta_rowset_writer.cpp |    5 +-
 be/src/olap/tablet.cpp                             |    2 +-
 be/src/runtime/decimalv2_value.cpp                 |    2 +-
 be/src/runtime/thread_context.h                    |   48 +-
 be/src/service/backend_options.cpp                 |    1 +
 be/src/service/backend_options.h                   |    1 +
 be/src/service/backend_service.cpp                 |   46 +-
 be/src/service/backend_service.h                   |   23 +-
 be/src/service/brpc_service.cpp                    |    9 +-
 be/src/service/doris_main.cpp                      |   34 +-
 be/src/service/http_service.cpp                    |    5 +
 be/src/service/internal_service.cpp                |   62 +-
 be/src/service/internal_service.h                  |   15 +-
 be/src/service/point_query_executor.cpp            |   15 +-
 be/src/service/point_query_executor.h              |   33 +-
 .../single_replica_load_download_service.cpp       |    6 +
 be/src/udf/udf.cpp                                 |   11 +-
 be/src/udf/udf.h                                   |    2 +-
 be/src/util/sse_util.hpp                           |    6 +-
 be/src/util/string_parser.hpp                      |    8 +
 .../vec/aggregate_functions/aggregate_function.h   |    5 +-
 .../aggregate_functions/aggregate_function_topn.h  |    4 +-
 .../aggregate_functions/aggregate_function_uniq.h  |    5 +-
 be/src/vec/columns/column_array.cpp                |   24 +-
 be/src/vec/columns/column_complex.h                |   13 +-
 be/src/vec/columns/column_const.cpp                |   15 +-
 be/src/vec/columns/column_decimal.cpp              |   12 +-
 be/src/vec/columns/column_dummy.h                  |    4 +-
 be/src/vec/columns/column_nullable.cpp             |    6 +-
 be/src/vec/columns/column_nullable.h               |    3 +-
 be/src/vec/columns/column_string.cpp               |    4 +-
 be/src/vec/columns/column_vector.cpp               |   14 +-
 be/src/vec/columns/columns_common.cpp              |    8 +-
 be/src/vec/columns/columns_common.h                |   18 +
 be/src/vec/common/hash_table/ph_hash_map.h         |    3 +-
 be/src/vec/common/hash_table/phmap_fwd_decl.h      |   54 +
 be/src/vec/common/sort/heap_sorter.cpp             |    3 +-
 be/src/vec/common/sort/sorter.cpp                  |   10 +-
 be/src/vec/common/string_searcher.h                |   66 -
 be/src/vec/core/block.cpp                          |    5 +-
 be/src/vec/core/block.h                            |    3 +
 be/src/vec/core/block_spill_writer.cpp             |    6 +-
 be/src/vec/core/sort_cursor.h                      |    1 +
 be/src/vec/data_types/data_type_decimal.cpp        |   11 +-
 be/src/vec/data_types/data_type_decimal.h          |    2 +-
 .../exec/format/parquet/vparquet_group_reader.cpp  |   18 +-
 .../vec/exec/join/process_hash_table_probe_impl.h  |    4 +-
 be/src/vec/exec/join/vhash_join_node.cpp           |   10 +-
 be/src/vec/exec/join/vnested_loop_join_node.cpp    |    2 +
 be/src/vec/exec/join/vnested_loop_join_node.h      |   18 +-
 be/src/vec/exec/scan/scanner_context.cpp           |    3 +-
 be/src/vec/exec/vaggregation_node.cpp              |    2 +-
 be/src/vec/exec/varrow_scanner.cpp                 |    2 +-
 be/src/vec/exprs/vexpr.cpp                         |  166 +-
 be/src/vec/exprs/vexpr_context.cpp                 |    6 +-
 be/src/vec/exprs/vliteral.cpp                      |   59 +-
 .../vec/functions/array/function_array_apply.cpp   |    8 +-
 .../array/function_array_with_constant.cpp         |    3 +-
 be/src/vec/functions/function_bitmap.cpp           |    4 +-
 be/src/vec/functions/function_bitmap_min_or_max.h  |    2 +-
 be/src/vec/functions/function_conv.cpp             |    2 +-
 be/src/vec/functions/function_convert_tz.h         |    2 +-
 be/src/vec/functions/function_jsonb.cpp            |    2 +-
 be/src/vec/functions/function_regexp.cpp           |    2 +-
 be/src/vec/functions/function_string.h             |    2 +-
 be/src/vec/functions/function_timestamp.cpp        |    8 +-
 be/src/vec/functions/function_totype.h             |    4 +-
 .../functions/functions_multi_string_position.cpp  |  140 +-
 be/src/vec/io/io_helper.h                          |   12 +-
 be/src/vec/sink/vtablet_sink.cpp                   |   72 +-
 be/src/vec/sink/vtablet_sink.h                     |   10 +
 be/src/vec/utils/util.hpp                          |   13 +-
 build.sh                                           |    2 +-
 docs/en/docs/admin-manual/config/be-config.md      |   10 +-
 docs/en/docs/faq/data-faq.md                       |    4 +-
 .../en/docs/install/construct-docker/k8s-deploy.md |    4 +-
 .../search/multi_search_all_positions.md           |   16 +-
 docs/zh-CN/docs/admin-manual/config/be-config.md   |   10 +-
 .../data-operate/import/import-scenes/jdbc-load.md |    2 +-
 docs/zh-CN/docs/faq/data-faq.md                    |    6 +-
 .../docs/install/construct-docker/k8s-deploy.md    |    4 +-
 .../search/multi_search_all_positions.md           |   16 +-
 .../java/org/apache/doris/catalog/ScalarType.java  |    6 +-
 .../main/java/org/apache/doris/common/Config.java  |    8 +-
 .../java/org/apache/doris/common/ConfigBase.java   |   82 +-
 .../org/apache/doris/common/ExperimentalUtil.java} |   27 +-
 fe/fe-core/pom.xml                                 |    5 +-
 fe/fe-core/src/main/cup/sql_parser.cup             |   14 +-
 .../org/apache/doris/analysis/AnalyzeStmt.java     |   40 -
 .../org/apache/doris/analysis/ArithmeticExpr.java  |   11 +-
 .../org/apache/doris/analysis/DropStatsStmt.java   |  114 +-
 .../apache/doris/analysis/LambdaFunctionExpr.java  |    7 +-
 .../java/org/apache/doris/analysis/TypeDef.java    |    8 +-
 .../java/org/apache/doris/backup/Repository.java   |    3 +
 .../java/org/apache/doris/backup/S3Storage.java    |    7 -
 .../java/org/apache/doris/catalog/S3Resource.java  |    5 +-
 .../org/apache/doris/catalog/TableProperty.java    |    3 +-
 .../java/org/apache/doris/common/FeConstants.java  |    1 +
 .../apache/doris/datasource/InternalCatalog.java   |   12 +-
 .../datasource/property/PropertyConverter.java     |    8 +-
 .../property/constants/GlueProperties.java         |   10 +
 .../property/constants/S3Properties.java           |   15 +-
 .../apache/doris/nereids/memo/GroupExpression.java |    2 +-
 .../java/org/apache/doris/nereids/memo/Memo.java   |    6 +-
 .../doris/nereids/rules/exploration/CBOUtils.java  |   12 -
 .../nereids/rules/exploration/EagerCount.java      |   10 +-
 .../nereids/rules/exploration/EagerGroupBy.java    |    2 +-
 .../rules/exploration/EagerGroupByCount.java       |    8 +-
 .../nereids/rules/exploration/EagerSplit.java      |   18 +-
 .../exploration/join/InnerJoinLAsscomProject.java  |    2 +-
 .../join/InnerJoinLeftAssociateProject.java        |    2 +-
 .../join/InnerJoinRightAssociateProject.java       |    2 +-
 .../exploration/join/JoinExchangeBothProject.java  |    3 +-
 .../exploration/join/JoinExchangeLeftProject.java  |    2 +-
 .../exploration/join/JoinExchangeRightProject.java |    2 +-
 .../join/LogicalJoinSemiJoinTransposeProject.java  |    4 +-
 .../exploration/join/OuterJoinAssocProject.java    |    2 +-
 .../exploration/join/OuterJoinLAsscomProject.java  |    2 +-
 .../join/PushdownProjectThroughInnerJoin.java      |    5 +-
 .../join/PushdownProjectThroughSemiJoin.java       |    5 +-
 .../join/SemiJoinSemiJoinTransposeProject.java     |    2 +-
 .../trees/plans/logical/LogicalProject.java        |    4 +
 .../java/org/apache/doris/planner/ScanNode.java    |   12 +-
 .../doris/planner/external/HiveScanProvider.java   |    2 +
 .../external/iceberg/IcebergScanProvider.java      |    2 +
 .../java/org/apache/doris/qe/ConnectProcessor.java |    6 +
 .../java/org/apache/doris/qe/SessionVariable.java  |   30 +-
 .../main/java/org/apache/doris/qe/VariableMgr.java |   45 +-
 .../apache/doris/statistics/AnalysisManager.java   |   36 +-
 .../apache/doris/statistics/AnalysisTaskInfo.java  |    7 +-
 .../doris/statistics/AnalysisTaskInfoBuilder.java  |   10 +-
 .../org/apache/doris/statistics/HistogramTask.java |    2 +-
 .../apache/doris/statistics/MVAnalysisTask.java    |    2 +-
 .../apache/doris/statistics/OlapAnalysisTask.java  |    2 +-
 .../doris/statistics/StatisticConstants.java       |    1 +
 .../apache/doris/statistics/StatisticsCache.java   |    3 +-
 .../apache/doris/statistics/StatisticsCleaner.java |    8 +-
 .../doris/statistics/StatisticsRepository.java     |   63 +-
 .../ExternalFileTableValuedFunction.java           |    3 +
 .../doris/tablefunction/S3TableValuedFunction.java |   16 +-
 .../doris/analysis/AdminSetConfigStmtTest.java     |   39 +-
 .../datasource/property/PropertyConverterTest.java |  360 +
 .../nereids/rules/exploration/EagerCountTest.java  |    9 +-
 .../rules/exploration/EagerGroupByCountTest.java   |    8 +-
 .../nereids/rules/exploration/EagerSplitTest.java  |   12 +-
 .../join/InnerJoinLAsscomProjectTest.java          |   16 +-
 .../join/JoinExchangeBothProjectTest.java          |   24 +-
 .../join/JoinExchangeLeftProjectTest.java          |   16 +-
 .../join/JoinExchangeRightProjectTest.java         |   24 +-
 .../join/OuterJoinLAsscomProjectTest.java          |   38 +-
 .../org/apache/doris/planner/QueryPlanTest.java    |    4 +-
 .../org/apache/doris/qe/SessionVariablesTest.java  |   77 +
 .../apache/doris/statistics/AnalysisJobTest.java   |    2 -
 .../doris/statistics/AnalysisTaskExecutorTest.java |    2 -
 .../apache/doris/statistics/HistogramTaskTest.java |    3 -
 .../apache/doris/utframe/TestWithFeService.java    |    5 +
 fe/pom.xml                                         |    5 +
 .../doris/broker/hdfs/FileSystemManager.java       |   43 +-
 .../decimalv3/test_arithmetic_expressions.out      |    6 +
 .../data/datatype_p0/decimalv3/test_overflow.out   |   19 -
 .../iceberg/test_external_catalog_icebergv2.out    |    8 +
 regression-test/data/insert_p0/insert.out          |  182 +-
 regression-test/data/nereids_arith_p0/date.out     | 2968 ------
 regression-test/data/nereids_arith_p0/decimal.out  | 9995 ++------------------
 regression-test/data/nereids_arith_p0/integer.out  | 6096 +-----------
 regression-test/data/nereids_arith_p0/string.out   | 3047 +-----
 .../data/nereids_p0/aggregate/aggregate.out        |    8 +-
 .../operator/test_arithmetic_operators.out         |   10 -
 .../conditional_functions/test_nullif.out          |    2 +-
 .../test_arithmetic_operators.out                  |   18 -
 .../data/query_p0/join/test_bitmap_filter.out      |    9 +
 .../operator/test_arithmetic_operators.out         |  261 +-
 .../test_array_functions_by_literal.out            |    3 -
 .../array_functions/test_array_map_function.out    |    6 +
 .../bitmap_functions/test_bitmap_function.out      |   19 +
 .../cast_function/test_cast_function.out           |   16 +-
 .../conditional_functions/test_nullif.out          |    2 +-
 .../datetime_functions/test_date_function.out      |   20 +
 .../sql_functions/math_functions/test_conv.out     |   19 +
 .../test_multi_string_position.out                 |   43 +-
 .../string_functions/test_string_function.out      |   19 +
 .../test_string_function_regexp.out                |   19 +
 regression-test/data/statistics/analyze_test.out   |    3 +
 .../account_p0/test_information_schema.groovy      |    6 +-
 .../test_compaction_uniq_keys_row_store.groovy     |   25 +-
 .../test_outer_join_with_subquery.groovy           |   24 +-
 .../test_outer_join_with_inline_view.groovy        |    2 +-
 .../decimalv3/test_arithmetic_expressions.groovy   |   22 +
 .../decimalv3/test_data/test_overflow.csv          |    2 +
 .../datatype_p0/decimalv3/test_overflow.groovy     |   56 -
 .../datev2/tpch_sf0.1_p1/sql/pipeline_q11.sql      |    2 +-
 .../suites/ddl_p0/test_create_table.groovy         |    3 +-
 .../iceberg/test_external_catalog_icebergv2.groovy |    6 +
 .../stream_load/test_parquet_orc_case.groovy       |    4 +-
 .../suites/mysql_ssl_p0/test_ssl_wild.groovy       |   10 +-
 .../suites/nereids_arith_p0/load.groovy            |    4 +-
 .../nereids_function_p0/agg_function/agg.groovy    |    4 +-
 .../suites/nereids_p0/aggregate/aggregate.groovy   |    2 +-
 .../operator/test_arithmetic_operators.groovy      |   14 +-
 .../test_array_aggregation_functions.groovy        |    2 +-
 .../conditional_functions/test_nullif.groovy       |    4 +-
 .../test_arithmetic_operators.groovy               |   14 +-
 .../suites/point_query_p0/test_point_query.groovy  |   42 +-
 .../suites/query_p0/aggregate/aggregate.groovy     |    2 +-
 .../suites/query_p0/join/test_bitmap_filter.groovy |    2 +
 .../query_p0/join/test_join_result_count.groovy    |  198 +
 .../operator/test_arithmetic_operators.groovy      |   22 +-
 .../test_array_aggregation_functions.groovy        |    2 +-
 .../test_array_functions_by_literal.groovy         |    2 +-
 .../array_functions/test_array_map_function.groovy |    2 +
 .../bitmap_functions/test_bitmap_function.groovy   |   45 +
 .../cast_function/test_cast_function.groovy        |    6 +-
 .../conditional_functions/test_nullif.groovy       |    4 +-
 .../datetime_functions/test_date_function.groovy   |   22 +
 .../sql_functions/math_functions/test_conv.groovy  |   29 +
 .../test_multi_string_position.groovy              |   47 +-
 .../string_functions/test_string_function.groovy   |   29 +
 .../test_string_function_regexp.groovy             |   22 +-
 .../suites/statistics/analyze_test.groovy          |   43 +-
 .../unique_with_mow_p0/test_create_table.groovy    |   97 +
 .../unique_with_mow_p0/test_pk_uk_case.groovy      |    8 +-
 run-fe-ut.sh                                       |   16 +-
 402 files changed, 5085 insertions(+), 23025 deletions(-)
 create mode 100644 be/src/vec/common/hash_table/phmap_fwd_decl.h
 copy fe/{fe-core/src/main/java/org/apache/doris/common/ColumnAliasGenerator.java => fe-common/src/main/java/org/apache/doris/common/ExperimentalUtil.java} (56%)
 create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java
 delete mode 100644 regression-test/data/datatype_p0/decimalv3/test_overflow.out
 create mode 100644 regression-test/suites/datatype_p0/decimalv3/test_data/test_overflow.csv
 delete mode 100644 regression-test/suites/datatype_p0/decimalv3/test_overflow.groovy
 create mode 100644 regression-test/suites/query_p0/join/test_join_result_count.groovy
 create mode 100644 regression-test/suites/unique_with_mow_p0/test_create_table.groovy


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


[doris] 17/36: [fix](Nereids): when GroupExpr already exists, we need to remove ParentExpression (#18749)

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

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

commit 06b61711e92c9ca4ba68c91d21935996e848a21c
Author: jakevin <ja...@gmail.com>
AuthorDate: Mon Apr 17 23:12:26 2023 +0800

    [fix](Nereids): when GroupExpr already exists, we need to remove ParentExpression (#18749)
---
 .../src/main/java/org/apache/doris/nereids/memo/GroupExpression.java   | 2 +-
 fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java       | 3 +++
 2 files changed, 4 insertions(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java
index 12e9816f8e..afa6d51f9b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java
@@ -182,7 +182,7 @@ public class GroupExpression {
      */
     public boolean isUnused() {
         if (isUnused) {
-            Preconditions.checkState(children.isEmpty() || ownerGroup == null);
+            Preconditions.checkState(children.isEmpty() && ownerGroup == null);
             return true;
         }
         Preconditions.checkState(ownerGroup != null);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
index 7a8d62338d..4ad1b118dc 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
@@ -427,6 +427,9 @@ public class Memo {
             if (target != null && !target.getGroupId().equals(existedGroupExpression.getOwnerGroup().getGroupId())) {
                 mergeGroup(existedGroupExpression.getOwnerGroup(), target);
             }
+            // When we create a GroupExpression, we will add it into ParentExpression of childGroup.
+            // But if it already exists, we should remove it from ParentExpression of childGroup.
+            groupExpression.children().forEach(childGroup -> childGroup.removeParentExpression(groupExpression));
             return CopyInResult.of(false, existedGroupExpression);
         }
         if (target != null) {


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


[doris] 02/36: [feature](multicatalog) enable doris hive/iceberg catalog to read data on tencent GooseFS (#18685)

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

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

commit 18ccf8a66d172ada321b417f719cbad1b5e9fbab
Author: Yulei-Yang <yu...@gmail.com>
AuthorDate: Sun Apr 16 18:11:57 2023 +0800

    [feature](multicatalog) enable doris hive/iceberg catalog to read data on tencent GooseFS (#18685)
---
 .../java/org/apache/doris/common/FeConstants.java  |  1 +
 .../doris/planner/external/HiveScanProvider.java   |  2 +
 .../external/iceberg/IcebergScanProvider.java      |  2 +
 .../doris/broker/hdfs/FileSystemManager.java       | 43 +++++++++++++++++++++-
 4 files changed, 47 insertions(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java b/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java
index 19dff3e707..fc4d273178 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java
@@ -81,6 +81,7 @@ public class FeConstants {
     public static String FS_PREFIX_COS = "cos";
     public static String FS_PREFIX_OBS = "obs";
     public static String FS_PREFIX_OFS = "ofs";
+    public static String FS_PREFIX_GFS = "gfs";
     public static String FS_PREFIX_JFS = "jfs";
     public static String FS_PREFIX_HDFS = "hdfs";
     public static String FS_PREFIX_FILE = "file";
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanProvider.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanProvider.java
index 99f84a128f..b09b61fea5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanProvider.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanProvider.java
@@ -101,6 +101,8 @@ public class HiveScanProvider extends HMSTableScanProvider {
                 return TFileType.FILE_LOCAL;
             } else if (location.startsWith(FeConstants.FS_PREFIX_OFS)) {
                 return TFileType.FILE_BROKER;
+            } else if (location.startsWith(FeConstants.FS_PREFIX_GFS)) {
+                return TFileType.FILE_BROKER;
             } else if (location.startsWith(FeConstants.FS_PREFIX_JFS)) {
                 return TFileType.FILE_BROKER;
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanProvider.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanProvider.java
index c481362da7..c710afc3fa 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanProvider.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/iceberg/IcebergScanProvider.java
@@ -111,6 +111,8 @@ public class IcebergScanProvider extends QueryScanProvider {
                 return TFileType.FILE_LOCAL;
             } else if (location.startsWith(FeConstants.FS_PREFIX_OFS)) {
                 return TFileType.FILE_BROKER;
+            } else if (location.startsWith(FeConstants.FS_PREFIX_GFS)) {
+                return TFileType.FILE_BROKER;
             } else if (location.startsWith(FeConstants.FS_PREFIX_JFS)) {
                 return TFileType.FILE_BROKER;
             }
diff --git a/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java b/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java
index 21cbe48aca..17f4d13216 100644
--- a/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java
+++ b/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java
@@ -75,6 +75,7 @@ public class FileSystemManager {
     private static final String BOS_SCHEME = "bos";
     private static final String JFS_SCHEME = "jfs";
     private static final String AFS_SCHEME = "afs";
+    private static final String GFS_SCHEME = "gfs";
 
     private static final String USER_NAME_KEY = "username";
     private static final String PASSWORD_KEY = "password";
@@ -221,7 +222,9 @@ public class FileSystemManager {
             brokerFileSystem = getBOSFileSystem(path, properties);
         } else if (scheme.equals(JFS_SCHEME)) {
             brokerFileSystem = getJuiceFileSystem(path, properties);
-        }else {
+        } else if (scheme.equals(GFS_SCHEME)) {
+            brokerFileSystem = getGooseFSFileSystem(path, properties);
+        } else {
             throw new BrokerException(TBrokerOperationStatusCode.INVALID_INPUT_FILE_PATH,
                 "invalid path. scheme is not supported");
         }
@@ -973,6 +976,44 @@ public class FileSystemManager {
         }
     }
 
+    /**
+     * @param path
+     * @param properties
+     * @return
+     * @throws URISyntaxException
+     * @throws Exception
+     */
+    public BrokerFileSystem getGooseFSFileSystem(String path, Map<String, String> properties) {
+        WildcardURI pathUri = new WildcardURI(path);
+        // endpoint is the server host, pathUri.getUri().getHost() is the bucket
+        // we should use these two params as the host identity, because FileSystem will cache both.
+        String host = GFS_SCHEME + "://" + pathUri.getAuthority();
+
+        String username = properties.getOrDefault(USER_NAME_KEY, "");
+        String password = properties.getOrDefault(PASSWORD_KEY, "");
+        String gfsUgi = username + "," + password;
+        FileSystemIdentity fileSystemIdentity = new FileSystemIdentity(host, gfsUgi);
+        BrokerFileSystem brokerFileSystem = updateCachedFileSystem(fileSystemIdentity, properties);
+        brokerFileSystem.getLock().lock();
+        try {
+            if (brokerFileSystem.getDFSFileSystem() == null) {
+                logger.info("create goosefs client: " + path);
+                Configuration conf = new Configuration();
+                for (Map.Entry<String, String> propElement : properties.entrySet()) {
+                    conf.set(propElement.getKey(), propElement.getValue());
+                }
+                FileSystem fileSystem = FileSystem.get(pathUri.getUri(), conf);
+                brokerFileSystem.setFileSystem(fileSystem);
+            }
+            return brokerFileSystem;
+        } catch (Exception e) {
+            logger.error("errors while connect to " + path, e);
+            throw new BrokerException(TBrokerOperationStatusCode.NOT_AUTHORIZED, e);
+        } finally {
+            brokerFileSystem.getLock().unlock();
+        }
+    }
+
     public List<TBrokerFileStatus> listPath(String path, boolean fileNameOnly, Map<String, String> properties) {
         List<TBrokerFileStatus> resultFileStatus = null;
         WildcardURI pathUri = new WildcardURI(path);


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


[doris] 28/36: fix doc (#18772)

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

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

commit 93655e446d5c8a3cf7cb7ddbe430669fc5439532
Author: Liqf <10...@users.noreply.github.com>
AuthorDate: Tue Apr 18 19:54:28 2023 +0800

    fix doc (#18772)
---
 docs/en/docs/install/construct-docker/k8s-deploy.md    | 4 ++--
 docs/zh-CN/docs/install/construct-docker/k8s-deploy.md | 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/docs/en/docs/install/construct-docker/k8s-deploy.md b/docs/en/docs/install/construct-docker/k8s-deploy.md
index 29ac9f1dc5..259419265d 100644
--- a/docs/en/docs/install/construct-docker/k8s-deploy.md
+++ b/docs/en/docs/install/construct-docker/k8s-deploy.md
@@ -77,9 +77,9 @@ Connect to the FE using mysql-client and perform operations such as' show backen
   Users need to mount PVC on their own to persist metadata information, data information, or log information
 - How to safely shrink the BE node?
 
-  BE:User manual execution is required before current resizing[ALTER-SYSTEM-DECOMMISSION-BACKEND](../../docs/sql-manual/sql-reference/Cluster-Management-Statements/ALTER-SYSTEM-DECOMMISSION-BACKEND.md)
+  BE:User manual execution is required before current resizing[ALTER-SYSTEM-DECOMMISSION-BACKEND](../../docs/sql-manual/sql-reference/Cluster-Management-Statements/ALTER-SYSTEM-DECOMMISSION-BACKEND)
 
-  BE(The role type is Compute Node): Do not store data files and can directly shrink,[About Computing Nodes](../../docs/advanced/compute_node.md)
+  BE(The role type is Compute Node): Do not store data files and can directly shrink,[About Computing Nodes](../../docs/advanced/compute_node)
 - FE startup error "failed to init statefulSetName"
 
   doris_ The environment variables statefulSetName and serviceName for follower. yml must appear in pairs, such as CN configured_ SERVICE, CN must be configured_ STATEFULSET
diff --git a/docs/zh-CN/docs/install/construct-docker/k8s-deploy.md b/docs/zh-CN/docs/install/construct-docker/k8s-deploy.md
index f482784e40..0ed5a53030 100644
--- a/docs/zh-CN/docs/install/construct-docker/k8s-deploy.md
+++ b/docs/zh-CN/docs/install/construct-docker/k8s-deploy.md
@@ -77,9 +77,9 @@ under the License.
   用户需要自行挂载pvc,持久化元数据信息,数据信息或者日志信息等
 - 怎么安全缩容BE节点?
 
-  BE:当前缩容之前需要用户手动执行[ALTER-SYSTEM-DECOMMISSION-BACKEND](../../docs/sql-manual/sql-reference/Cluster-Management-Statements/ALTER-SYSTEM-DECOMMISSION-BACKEND.md)
+  BE:当前缩容之前需要用户手动执行[ALTER-SYSTEM-DECOMMISSION-BACKEND](../../docs/sql-manual/sql-reference/Cluster-Management-Statements/ALTER-SYSTEM-DECOMMISSION-BACKEND)
 
-  BE(角色类型为 Compute Node): 不存储数据文件,可以直接进行缩容,[关于计算节点](../../docs/advanced/compute_node.md)
+  BE(角色类型为 Compute Node): 不存储数据文件,可以直接进行缩容,[关于计算节点](../../docs/advanced/compute_node)
 - FE启动报错"failed to init statefulSetName"
 
   doris_follower.yml的环境变量 statefulSetName和serviceName必须成对出现,比如配置了CN_SERVICE,就必须配置CN_STATEFULSET


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


[doris] 10/36: [chore](build) Use include-what-you-use to optimize includes (#18681)

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

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

commit dce24e63830e21b62b56769cbeac817f6b8945af
Author: Adonis Ling <ad...@gmail.com>
AuthorDate: Mon Apr 17 11:44:58 2023 +0800

    [chore](build) Use include-what-you-use to optimize includes (#18681)
    
    Currently, there are some useless includes in the codebase. We can use a tool named include-what-you-use to optimize these includes. By using a strict include-what-you-use policy, we can get lots of benefits from it.
---
 be/src/agent/agent_server.cpp                      | 12 +++++
 be/src/agent/agent_server.h                        | 10 +++-
 be/src/agent/heartbeat_server.cpp                  | 19 +++++--
 be/src/agent/heartbeat_server.h                    |  8 ++-
 be/src/agent/task_worker_pool.cpp                  | 34 +++++++++---
 be/src/agent/task_worker_pool.h                    | 19 ++++++-
 be/src/agent/topic_subscriber.cpp                  |  7 +++
 be/src/agent/topic_subscriber.h                    |  5 +-
 be/src/agent/user_resource_listener.cpp            | 19 ++++---
 be/src/agent/user_resource_listener.h              |  7 ++-
 be/src/agent/utils.cpp                             | 23 +++++++-
 be/src/agent/utils.h                               | 10 ++++
 be/src/common/config.h                             |  2 +-
 be/src/common/configbase.cpp                       | 21 ++++++--
 be/src/common/configbase.h                         |  1 +
 be/src/common/daemon.cpp                           | 26 +++++++--
 be/src/common/exception.h                          | 11 ++++
 be/src/common/logconfig.cpp                        |  6 ++-
 be/src/common/logging.h                            |  2 +-
 be/src/common/resource_tls.cpp                     |  4 +-
 be/src/common/status.cpp                           |  8 ++-
 be/src/common/status.h                             |  5 +-
 be/src/exec/arrow/arrow_reader.cpp                 | 23 ++++----
 be/src/exec/arrow/arrow_reader.h                   | 28 ++++++++--
 be/src/exec/arrow/parquet_reader.cpp               | 23 +++++---
 be/src/exec/arrow/parquet_reader.h                 | 18 +++++--
 be/src/exec/base_scanner.cpp                       | 28 +++++++++-
 be/src/exec/base_scanner.h                         | 16 ++++++
 be/src/exec/data_sink.cpp                          | 12 +++--
 be/src/exec/data_sink.h                            | 14 ++++-
 be/src/exec/decompressor.cpp                       |  4 ++
 be/src/exec/decompressor.h                         |  4 ++
 be/src/exec/es/es_scan_reader.cpp                  |  4 ++
 be/src/exec/es/es_scan_reader.h                    |  6 +++
 be/src/exec/es/es_scroll_parser.cpp                | 24 +++++++--
 be/src/exec/es/es_scroll_parser.h                  |  6 +++
 be/src/exec/es/es_scroll_query.cpp                 |  5 ++
 be/src/exec/exec_node.cpp                          | 20 ++++++-
 be/src/exec/exec_node.h                            | 13 ++++-
 be/src/exec/odbc_connector.cpp                     | 16 ++++--
 be/src/exec/odbc_connector.h                       | 12 +++++
 be/src/exec/olap_common.cpp                        |  4 --
 be/src/exec/olap_common.h                          | 13 +++++
 be/src/exec/rowid_fetcher.cpp                      | 26 ++++++++-
 be/src/exec/rowid_fetcher.h                        | 14 ++++-
 be/src/exec/scan_node.cpp                          |  9 ++++
 be/src/exec/scan_node.h                            | 11 +++-
 be/src/exec/schema_scanner.cpp                     | 21 +++++++-
 be/src/exec/schema_scanner.h                       | 12 ++++-
 .../schema_scanner/schema_charsets_scanner.cpp     |  8 +++
 .../exec/schema_scanner/schema_charsets_scanner.h  |  6 +++
 .../schema_scanner/schema_collations_scanner.cpp   |  9 +++-
 .../schema_scanner/schema_collations_scanner.h     |  6 +++
 .../exec/schema_scanner/schema_columns_scanner.cpp | 16 ++++--
 .../exec/schema_scanner/schema_columns_scanner.h   |  9 +++-
 .../exec/schema_scanner/schema_dummy_scanner.cpp   |  9 ++++
 be/src/exec/schema_scanner/schema_dummy_scanner.h  |  5 ++
 .../exec/schema_scanner/schema_files_scanner.cpp   | 10 +++-
 be/src/exec/schema_scanner/schema_files_scanner.h  | 10 +++-
 be/src/exec/schema_scanner/schema_helper.cpp       | 17 ++++--
 be/src/exec/schema_scanner/schema_helper.h         | 18 ++++++-
 .../schema_scanner/schema_partitions_scanner.cpp   | 12 ++++-
 .../schema_scanner/schema_partitions_scanner.h     | 10 +++-
 .../exec/schema_scanner/schema_rowsets_scanner.cpp | 24 ++++++---
 .../exec/schema_scanner/schema_rowsets_scanner.h   |  8 +++
 .../schema_schema_privileges_scanner.cpp           | 12 ++++-
 .../schema_schema_privileges_scanner.h             | 10 +++-
 .../schema_scanner/schema_schemata_scanner.cpp     | 12 ++++-
 .../exec/schema_scanner/schema_schemata_scanner.h  | 10 +++-
 .../schema_scanner/schema_statistics_scanner.cpp   |  4 +-
 .../schema_scanner/schema_statistics_scanner.h     |  2 +
 .../schema_table_privileges_scanner.cpp            | 10 +++-
 .../schema_table_privileges_scanner.h              | 10 +++-
 .../exec/schema_scanner/schema_tables_scanner.cpp  | 18 ++++++-
 be/src/exec/schema_scanner/schema_tables_scanner.h | 10 +++-
 .../schema_user_privileges_scanner.cpp             | 12 ++++-
 .../schema_user_privileges_scanner.h               | 10 +++-
 .../schema_scanner/schema_variables_scanner.cpp    | 16 +++++-
 .../exec/schema_scanner/schema_variables_scanner.h | 11 +++-
 .../exec/schema_scanner/schema_views_scanner.cpp   | 12 ++++-
 be/src/exec/schema_scanner/schema_views_scanner.h  | 10 +++-
 be/src/exec/table_connector.cpp                    | 21 ++++++--
 be/src/exec/table_connector.h                      | 12 +++++
 be/src/exec/tablet_info.cpp                        | 18 +++++++
 be/src/exec/tablet_info.h                          | 21 +++++++-
 be/src/exec/text_converter.cpp                     | 14 ++++-
 be/src/exec/text_converter.h                       |  3 ++
 be/src/exprs/block_bloom_filter_avx_impl.cc        |  6 ++-
 be/src/exprs/block_bloom_filter_impl.cc            |  9 +++-
 be/src/exprs/json_functions.cpp                    | 16 +++---
 be/src/exprs/json_functions.h                      | 13 +++++
 be/src/exprs/math_functions.cpp                    | 11 ++--
 be/src/exprs/math_functions.h                      |  1 +
 be/src/exprs/runtime_filter.cpp                    | 21 +++++++-
 be/src/exprs/runtime_filter.h                      | 22 ++++++++
 be/src/exprs/runtime_filter_rpc.cpp                | 15 +++++-
 be/src/exprs/string_functions.cpp                  |  7 +--
 be/src/exprs/string_functions.h                    |  2 +
 be/src/geo/ByteOrderDataInStream.h                 |  2 +
 be/src/geo/ByteOrderValues.cpp                     |  3 +-
 be/src/geo/ByteOrderValues.h                       |  5 ++
 be/src/geo/geo_tobinary.cpp                        | 10 +++-
 be/src/geo/geo_tobinary.h                          |  8 +++
 be/src/geo/geo_types.cpp                           | 14 +++--
 be/src/geo/geo_types.h                             |  3 +-
 be/src/geo/machine.h                               |  2 +
 be/src/geo/wkb_parse.cpp                           | 12 +++--
 be/src/geo/wkb_parse.h                             | 12 +++++
 be/src/geo/wkt_parse.cpp                           |  3 +-
 be/src/geo/wkt_parse.h                             |  4 ++
 be/src/http/action/check_rpc_channel_action.cpp    | 14 +++--
 be/src/http/action/check_rpc_channel_action.h      |  2 +
 be/src/http/action/check_tablet_segment_action.cpp |  7 +++
 be/src/http/action/check_tablet_segment_action.h   |  1 +
 be/src/http/action/checksum_action.cpp             |  7 +--
 be/src/http/action/checksum_action.h               |  1 +
 be/src/http/action/compaction_action.cpp           | 13 ++++-
 be/src/http/action/compaction_action.h             |  5 ++
 be/src/http/action/config_action.cpp               |  8 ++-
 be/src/http/action/config_action.h                 |  1 +
 be/src/http/action/download_action.cpp             | 12 ++---
 be/src/http/action/download_action.h               |  5 ++
 be/src/http/action/health_action.cpp               |  1 -
 be/src/http/action/health_action.h                 |  1 +
 be/src/http/action/jeprofile_actions.cpp           |  9 ++--
 be/src/http/action/meta_action.cpp                 | 13 +++--
 be/src/http/action/meta_action.h                   |  3 ++
 be/src/http/action/metrics_action.cpp              |  7 ---
 be/src/http/action/monitor_action.cpp              |  5 +-
 be/src/http/action/pad_rowset_action.cpp           | 15 +++++-
 be/src/http/action/pad_rowset_action.h             |  2 +
 be/src/http/action/pprof_actions.cpp               | 14 ++---
 be/src/http/action/reload_tablet_action.cpp        |  7 +--
 be/src/http/action/reload_tablet_action.h          |  7 ++-
 be/src/http/action/reset_rpc_channel_action.cpp    |  8 ++-
 be/src/http/action/reset_rpc_channel_action.h      |  2 +
 be/src/http/action/restore_tablet_action.cpp       | 19 ++++---
 be/src/http/action/restore_tablet_action.h         |  6 ++-
 be/src/http/action/snapshot_action.cpp             | 10 ++--
 be/src/http/action/snapshot_action.h               |  1 +
 be/src/http/action/stream_load.cpp                 | 32 ++++++-----
 be/src/http/action/stream_load.h                   |  7 ++-
 be/src/http/action/stream_load_2pc.cpp             | 11 +++-
 be/src/http/action/stream_load_2pc.h               |  1 +
 be/src/http/action/tablet_migration_action.cpp     |  7 ++-
 be/src/http/action/tablet_migration_action.h       | 14 ++++-
 be/src/http/action/tablets_distribution_action.cpp | 10 +++-
 be/src/http/action/tablets_distribution_action.h   |  3 ++
 be/src/http/action/tablets_info_action.cpp         |  9 ++++
 be/src/http/action/tablets_info_action.h           |  1 +
 be/src/http/action/version_action.cpp              |  5 +-
 be/src/http/action/version_action.h                |  1 +
 be/src/http/default_path_handlers.cpp              | 18 +++++--
 be/src/http/ev_http_server.cpp                     | 19 ++++---
 be/src/http/ev_http_server.h                       |  3 ++
 be/src/http/http_channel.cpp                       |  7 ++-
 be/src/http/http_channel.h                         |  2 +
 be/src/http/http_client.cpp                        |  6 +++
 be/src/http/http_client.h                          |  3 ++
 be/src/http/http_method.cpp                        |  1 +
 be/src/http/http_request.cpp                       |  7 +--
 be/src/http/http_status.cpp                        |  1 +
 be/src/http/utils.cpp                              | 10 +++-
 be/src/http/utils.h                                |  1 +
 be/src/http/web_page_handler.cpp                   | 10 +++-
 be/src/http/web_page_handler.h                     |  3 ++
 be/src/io/fs/file_reader_writer_fwd.h              |  3 ++
 be/src/io/fs/hdfs.h                                |  4 +-
 be/src/io/fs/hdfs_file_reader.h                    | 12 +++++
 be/src/io/fs/hdfs_file_writer.h                    |  7 +++
 be/src/service/backend_options.cpp                 |  1 +
 be/src/service/backend_options.h                   |  1 +
 be/src/service/backend_service.cpp                 | 46 +++++++++-------
 be/src/service/backend_service.h                   | 23 ++++++--
 be/src/service/brpc_service.cpp                    |  9 +++-
 be/src/service/doris_main.cpp                      | 34 +++++++-----
 be/src/service/http_service.cpp                    |  5 ++
 be/src/service/internal_service.cpp                | 62 ++++++++++++++++++----
 be/src/service/internal_service.h                  | 15 +++++-
 be/src/service/point_query_executor.cpp            | 15 ++++--
 be/src/service/point_query_executor.h              | 33 +++++++++++-
 .../single_replica_load_download_service.cpp       |  6 +++
 be/src/udf/udf.cpp                                 | 11 +---
 be/src/udf/udf.h                                   |  2 +-
 be/src/util/sse_util.hpp                           |  6 +--
 185 files changed, 1619 insertions(+), 355 deletions(-)

diff --git a/be/src/agent/agent_server.cpp b/be/src/agent/agent_server.cpp
index 5fc46a912d..cb84e8b2e0 100644
--- a/be/src/agent/agent_server.cpp
+++ b/be/src/agent/agent_server.cpp
@@ -17,9 +17,14 @@
 
 #include "agent/agent_server.h"
 
+#include <gen_cpp/AgentService_types.h>
+#include <gen_cpp/HeartbeatService_types.h>
+#include <gen_cpp/Types_types.h>
+#include <stdint.h>
 #include <thrift/protocol/TDebugProtocol.h>
 
 #include <filesystem>
+#include <ostream>
 #include <string>
 
 #include "agent/task_worker_pool.h"
@@ -29,7 +34,14 @@
 #include "common/logging.h"
 #include "common/status.h"
 #include "gutil/strings/substitute.h"
+#include "olap/olap_define.h"
+#include "olap/options.h"
 #include "olap/snapshot_manager.h"
+#include "runtime/exec_env.h"
+
+namespace doris {
+class TopicListener;
+} // namespace doris
 
 using std::string;
 using std::vector;
diff --git a/be/src/agent/agent_server.h b/be/src/agent/agent_server.h
index c8136d00b8..11521eb09a 100644
--- a/be/src/agent/agent_server.h
+++ b/be/src/agent/agent_server.h
@@ -17,17 +17,25 @@
 
 #pragma once
 
+#include <butil/macros.h>
+#include <gen_cpp/AgentService_types.h>
+
 #include <memory>
 #include <string>
 #include <vector>
 
-#include "gen_cpp/AgentService_types.h"
 #include "runtime/exec_env.h"
 
 namespace doris {
 
 class TaskWorkerPool;
 class TopicSubscriber;
+class ExecEnv;
+class TAgentPublishRequest;
+class TAgentResult;
+class TAgentTaskRequest;
+class TMasterInfo;
+class TSnapshotRequest;
 
 // Each method corresponds to one RPC from FE Master, see BackendService.
 class AgentServer {
diff --git a/be/src/agent/heartbeat_server.cpp b/be/src/agent/heartbeat_server.cpp
index 705f14ab9a..41a61005ed 100644
--- a/be/src/agent/heartbeat_server.cpp
+++ b/be/src/agent/heartbeat_server.cpp
@@ -17,18 +17,31 @@
 
 #include "agent/heartbeat_server.h"
 
-#include <thrift/TProcessor.h>
+#include <gen_cpp/HeartbeatService.h>
+#include <gen_cpp/HeartbeatService_types.h>
+#include <gen_cpp/Types_types.h>
+#include <glog/logging.h>
 
+#include <memory>
+#include <ostream>
+#include <string>
+
+#include "common/config.h"
 #include "common/status.h"
-#include "gen_cpp/HeartbeatService.h"
-#include "gen_cpp/Status_types.h"
 #include "olap/storage_engine.h"
+#include "runtime/exec_env.h"
 #include "runtime/heartbeat_flags.h"
 #include "service/backend_options.h"
 #include "util/debug_util.h"
 #include "util/thrift_server.h"
 #include "util/time.h"
 
+namespace apache {
+namespace thrift {
+class TProcessor;
+} // namespace thrift
+} // namespace apache
+
 namespace doris {
 
 HeartbeatServer::HeartbeatServer(TMasterInfo* master_info)
diff --git a/be/src/agent/heartbeat_server.h b/be/src/agent/heartbeat_server.h
index 73bfbba7fa..65e228dd8c 100644
--- a/be/src/agent/heartbeat_server.h
+++ b/be/src/agent/heartbeat_server.h
@@ -17,14 +17,20 @@
 
 #pragma once
 
+#include <butil/macros.h>
+#include <gen_cpp/HeartbeatService.h>
+#include <stdint.h>
+
 #include <mutex>
 
 #include "common/status.h"
-#include "gen_cpp/HeartbeatService.h"
 #include "olap/olap_define.h"
 #include "runtime/exec_env.h"
 
 namespace doris {
+class ExecEnv;
+class THeartbeatResult;
+class TMasterInfo;
 
 const uint32_t HEARTBEAT_INTERVAL = 10;
 class StorageEngine;
diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp
index 9fe680ea96..ab7877fbf2 100644
--- a/be/src/agent/task_worker_pool.cpp
+++ b/be/src/agent/task_worker_pool.cpp
@@ -17,46 +17,66 @@
 
 #include "agent/task_worker_pool.h"
 
+#include <fmt/format.h>
 #include <gen_cpp/AgentService_types.h>
-#include <pthread.h>
-#include <sys/stat.h>
-
-#include <boost/lexical_cast.hpp>
-#include <chrono>
-#include <csignal>
+#include <gen_cpp/HeartbeatService_types.h>
+#include <gen_cpp/MasterService_types.h>
+#include <gen_cpp/Status_types.h>
+#include <gen_cpp/Types_types.h>
+#include <unistd.h>
+
+#include <algorithm>
+// IWYU pragma: no_include <bits/chrono.h>
+#include <chrono> // IWYU pragma: keep
 #include <ctime>
+#include <functional>
 #include <memory>
+#include <shared_mutex>
 #include <sstream>
 #include <string>
+#include <thread>
+#include <utility>
+#include <vector>
 
 #include "agent/utils.h"
+#include "common/config.h"
 #include "common/logging.h"
 #include "common/status.h"
-#include "gen_cpp/Types_types.h"
+#include "gutil/ref_counted.h"
+#include "gutil/stringprintf.h"
 #include "gutil/strings/substitute.h"
+#include "io/fs/file_system.h"
 #include "io/fs/local_file_system.h"
+#include "io/fs/path.h"
 #include "io/fs/s3_file_system.h"
 #include "olap/data_dir.h"
 #include "olap/olap_common.h"
+#include "olap/rowset/rowset_meta.h"
 #include "olap/snapshot_manager.h"
 #include "olap/storage_engine.h"
 #include "olap/storage_policy.h"
 #include "olap/tablet.h"
+#include "olap/tablet_manager.h"
+#include "olap/tablet_meta.h"
+#include "olap/tablet_schema.h"
 #include "olap/task/engine_alter_tablet_task.h"
 #include "olap/task/engine_batch_load_task.h"
 #include "olap/task/engine_checksum_task.h"
 #include "olap/task/engine_clone_task.h"
 #include "olap/task/engine_publish_version_task.h"
 #include "olap/task/engine_storage_migration_task.h"
+#include "olap/txn_manager.h"
 #include "olap/utils.h"
 #include "runtime/exec_env.h"
 #include "runtime/snapshot_loader.h"
 #include "service/backend_options.h"
 #include "util/doris_metrics.h"
 #include "util/random.h"
+#include "util/s3_util.h"
 #include "util/scoped_cleanup.h"
 #include "util/stopwatch.hpp"
 #include "util/threadpool.h"
+#include "util/time.h"
 #include "util/trace.h"
 
 namespace doris {
diff --git a/be/src/agent/task_worker_pool.h b/be/src/agent/task_worker_pool.h
index 3ea664014e..cd9b274d98 100644
--- a/be/src/agent/task_worker_pool.h
+++ b/be/src/agent/task_worker_pool.h
@@ -17,24 +17,39 @@
 
 #pragma once
 
+#include <butil/macros.h>
+#include <gen_cpp/AgentService_types.h>
+#include <gen_cpp/HeartbeatService_types.h>
+#include <gen_cpp/Types_types.h>
+#include <stdint.h>
+
 #include <atomic>
+#include <condition_variable>
 #include <deque>
+#include <map>
 #include <memory>
+#include <mutex>
+#include <set>
+#include <string>
 #include <utility>
 #include <vector>
 
 #include "common/status.h"
-#include "gen_cpp/AgentService_types.h"
-#include "gen_cpp/HeartbeatService_types.h"
 #include "olap/data_dir.h"
 #include "olap/tablet.h"
 #include "util/countdown_latch.h"
+#include "util/metrics.h"
 
 namespace doris {
 
 class ExecEnv;
 class ThreadPool;
 class AgentUtils;
+class DataDir;
+class TFinishTaskRequest;
+class TMasterInfo;
+class TReportRequest;
+class TTabletInfo;
 
 class TaskWorkerPool {
 public:
diff --git a/be/src/agent/topic_subscriber.cpp b/be/src/agent/topic_subscriber.cpp
index 88754efd2b..c5c4a324eb 100644
--- a/be/src/agent/topic_subscriber.cpp
+++ b/be/src/agent/topic_subscriber.cpp
@@ -17,6 +17,13 @@
 
 #include "agent/topic_subscriber.h"
 
+#include <gen_cpp/AgentService_types.h>
+
+#include <mutex>
+#include <utility>
+
+#include "agent/topic_listener.h"
+
 namespace doris {
 
 TopicSubscriber::TopicSubscriber() {}
diff --git a/be/src/agent/topic_subscriber.h b/be/src/agent/topic_subscriber.h
index 2d57f6927d..3151cbe54c 100644
--- a/be/src/agent/topic_subscriber.h
+++ b/be/src/agent/topic_subscriber.h
@@ -17,15 +17,18 @@
 
 #pragma once
 
+#include <gen_cpp/AgentService_types.h>
+
 #include <map>
 #include <mutex>
 #include <shared_mutex>
 #include <thread>
+#include <vector>
 
 #include "agent/topic_listener.h"
-#include "gen_cpp/AgentService_types.h"
 
 namespace doris {
+class TopicListener;
 
 class TopicSubscriber {
 public:
diff --git a/be/src/agent/user_resource_listener.cpp b/be/src/agent/user_resource_listener.cpp
index efef2c7f6b..07ae1ee6b0 100644
--- a/be/src/agent/user_resource_listener.cpp
+++ b/be/src/agent/user_resource_listener.cpp
@@ -17,17 +17,24 @@
 
 #include "agent/user_resource_listener.h"
 
-#include <thrift/TApplicationException.h>
+#include <gen_cpp/AgentService_types.h>
+#include <gen_cpp/FrontendService.h>
+#include <gen_cpp/HeartbeatService_types.h>
+#include <gen_cpp/MasterService_types.h>
+#include <gen_cpp/Types_types.h>
+#include <glog/logging.h>
 #include <thrift/Thrift.h>
-#include <thrift/protocol/TBinaryProtocol.h>
-#include <thrift/transport/TBufferTransports.h>
-#include <thrift/transport/TSocket.h>
+#include <thrift/transport/TTransportException.h>
 
 #include <future>
-#include <map>
+#include <ostream>
+#include <string>
+#include <vector>
 
-#include "gen_cpp/FrontendService.h"
+#include "common/config.h"
+#include "common/status.h"
 #include "runtime/client_cache.h"
+#include "runtime/exec_env.h"
 
 namespace doris {
 
diff --git a/be/src/agent/user_resource_listener.h b/be/src/agent/user_resource_listener.h
index 9bddb1856b..b26ae3f279 100644
--- a/be/src/agent/user_resource_listener.h
+++ b/be/src/agent/user_resource_listener.h
@@ -17,16 +17,19 @@
 
 #pragma once
 
+#include <gen_cpp/AgentService_types.h>
+#include <gen_cpp/HeartbeatService_types.h>
+#include <stdint.h>
+
 #include <string>
 
 #include "agent/topic_listener.h"
-#include "gen_cpp/AgentService_types.h"
-#include "gen_cpp/HeartbeatService_types.h"
 #include "runtime/exec_env.h"
 
 namespace doris {
 
 class ExecEnv;
+class TMasterInfo;
 
 class UserResourceListener : public TopicListener {
 public:
diff --git a/be/src/agent/utils.cpp b/be/src/agent/utils.cpp
index dfe4bc04c8..0e91eeaa49 100644
--- a/be/src/agent/utils.cpp
+++ b/be/src/agent/utils.cpp
@@ -17,18 +17,39 @@
 
 #include "agent/utils.h"
 
+#include <errno.h>
+#include <gen_cpp/FrontendService.h>
+#include <gen_cpp/HeartbeatService_types.h>
+#include <gen_cpp/Types_types.h>
+#include <glog/logging.h>
 #include <rapidjson/document.h>
+#include <rapidjson/encodings.h>
 #include <rapidjson/rapidjson.h>
 #include <rapidjson/stringbuffer.h>
 #include <rapidjson/writer.h>
+#include <stdint.h>
+#include <stdlib.h>
+#include <string.h>
+#include <thrift/transport/TTransportException.h>
 
 #include <cstdio>
+#include <exception>
 #include <fstream>
-#include <sstream>
+#include <memory>
+#include <utility>
 
+#include "common/config.h"
 #include "common/status.h"
 #include "runtime/client_cache.h"
 
+namespace doris {
+class TConfirmUnusedRemoteFilesRequest;
+class TConfirmUnusedRemoteFilesResult;
+class TFinishTaskRequest;
+class TMasterResult;
+class TReportRequest;
+} // namespace doris
+
 using std::map;
 using std::string;
 using std::stringstream;
diff --git a/be/src/agent/utils.h b/be/src/agent/utils.h
index cfce0c3413..8cd1895d6a 100644
--- a/be/src/agent/utils.h
+++ b/be/src/agent/utils.h
@@ -17,14 +17,24 @@
 
 #pragma once
 
+#include <butil/macros.h>
 #include <gen_cpp/FrontendService.h>
 #include <gen_cpp/HeartbeatService_types.h>
 #include <gen_cpp/MasterService_types.h>
 
+#include <map>
+#include <string>
+
 #include "common/status.h"
 #include "gutil/macros.h"
 
 namespace doris {
+class TConfirmUnusedRemoteFilesRequest;
+class TConfirmUnusedRemoteFilesResult;
+class TFinishTaskRequest;
+class TMasterInfo;
+class TMasterResult;
+class TReportRequest;
 
 class MasterServerClient {
 public:
diff --git a/be/src/common/config.h b/be/src/common/config.h
index 03dfb1cfc2..3f2034191c 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -17,7 +17,7 @@
 
 #pragma once
 
-#include "configbase.h"
+#include "configbase.h" // IWYU pragma: export
 
 namespace doris {
 namespace config {
diff --git a/be/src/common/configbase.cpp b/be/src/common/configbase.cpp
index 24e3a44e66..066aa0352d 100644
--- a/be/src/common/configbase.cpp
+++ b/be/src/common/configbase.cpp
@@ -15,20 +15,31 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <stdint.h>
+
 #include <algorithm>
+#include <cctype>
 #include <cerrno>
+#include <cstdlib>
 #include <cstring>
-#include <fstream>
+#include <fstream> // IWYU pragma: keep
+#include <functional>
 #include <iostream>
-#include <list>
 #include <map>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "common/logging.h"
 
 #define __IN_CONFIGBASE_CPP__
-#include "common/config.h"
+#include "common/config.h" // IWYU pragma: keep
 #undef __IN_CONFIGBASE_CPP__
 
 #include "common/status.h"
-#include "gutil/strings/substitute.h"
+#include "io/fs/file_reader_writer_fwd.h"
 #include "io/fs/file_writer.h"
 #include "io/fs/local_file_system.h"
 
@@ -390,7 +401,7 @@ Status persist_config(const std::string& field, const std::string& value) {
     // lock to make sure only one thread can modify the be_custom.conf
     std::lock_guard<std::mutex> l(custom_conf_lock);
 
-    static const string conffile = string(getenv("DORIS_HOME")) + "/conf/be_custom.conf";
+    static const std::string conffile = std::string(getenv("DORIS_HOME")) + "/conf/be_custom.conf";
 
     Properties tmp_props;
     if (!tmp_props.load(conffile.c_str(), false)) {
diff --git a/be/src/common/configbase.h b/be/src/common/configbase.h
index f3c79519ba..0bc11647a4 100644
--- a/be/src/common/configbase.h
+++ b/be/src/common/configbase.h
@@ -22,6 +22,7 @@
 #include <map>
 #include <mutex>
 #include <string>
+#include <utility>
 #include <vector>
 
 namespace doris {
diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp
index 9e75a5a784..4a64e67053 100644
--- a/be/src/common/daemon.cpp
+++ b/be/src/common/daemon.cpp
@@ -17,21 +17,35 @@
 
 #include "common/daemon.h"
 
+#include <bthread/errno.h>
 #include <gflags/gflags.h>
-#include <gperftools/malloc_extension.h>
+// IWYU pragma: no_include <bits/std_abs.h>
+#include <math.h>
 #include <signal.h>
+#include <stdint.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include <algorithm>
+// IWYU pragma: no_include <bits/chrono.h>
+#include <chrono> // IWYU pragma: keep
+#include <map>
+#include <memory>
+#include <ostream>
+#include <set>
+#include <string>
 
 #include "common/config.h"
 #include "common/logging.h"
-#include "exprs/math_functions.h"
-#include "exprs/string_functions.h"
+#include "common/status.h"
 #include "olap/options.h"
 #include "olap/storage_engine.h"
+#include "olap/tablet_manager.h"
 #include "runtime/block_spill_manager.h"
 #include "runtime/exec_env.h"
-#include "runtime/fragment_mgr.h"
 #include "runtime/load_channel_mgr.h"
-#include "runtime/memory/chunk_allocator.h"
+#include "runtime/memory/mem_tracker.h"
+#include "runtime/memory/mem_tracker_limiter.h"
 #include "runtime/user_function_cache.h"
 #include "service/backend_options.h"
 #include "util/cpu_info.h"
@@ -39,7 +53,9 @@
 #include "util/disk_info.h"
 #include "util/doris_metrics.h"
 #include "util/mem_info.h"
+#include "util/metrics.h"
 #include "util/network_util.h"
+#include "util/perf_counters.h"
 #include "util/system_metrics.h"
 #include "util/thrift_util.h"
 #include "util/time.h"
diff --git a/be/src/common/exception.h b/be/src/common/exception.h
index fff3215d31..3723db0891 100644
--- a/be/src/common/exception.h
+++ b/be/src/common/exception.h
@@ -17,6 +17,17 @@
 
 #pragma once
 
+#include <fmt/format.h>
+#include <gen_cpp/Status_types.h>
+#include <stdint.h>
+
+#include <exception>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <utility>
+
 #include "common/status.h"
 
 namespace doris {
diff --git a/be/src/common/logconfig.cpp b/be/src/common/logconfig.cpp
index 9e05e64a97..fc81103254 100644
--- a/be/src/common/logconfig.cpp
+++ b/be/src/common/logconfig.cpp
@@ -15,14 +15,16 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <glog/logging.h>
-#include <glog/vlog_is_on.h>
+#include <ctype.h>
+#include <stdint.h>
 
 #include <cerrno>
 #include <cstdlib>
 #include <cstring>
 #include <iostream>
 #include <mutex>
+#include <string>
+#include <vector>
 
 #include "common/config.h"
 #include "common/logging.h"
diff --git a/be/src/common/logging.h b/be/src/common/logging.h
index d8c55d5f6b..894db39cda 100644
--- a/be/src/common/logging.h
+++ b/be/src/common/logging.h
@@ -24,7 +24,7 @@
 #undef _XOPEN_SOURCE
 // This is including a glog internal file.  We want this to expose the
 // function to get the stack trace.
-#include <glog/logging.h>
+#include <glog/logging.h> // IWYU pragma: export
 #undef MutexLock
 
 // Define VLOG levels.  We want display per-row info less than per-file which
diff --git a/be/src/common/resource_tls.cpp b/be/src/common/resource_tls.cpp
index 5ac6c94d9d..9b5ddc6815 100644
--- a/be/src/common/resource_tls.cpp
+++ b/be/src/common/resource_tls.cpp
@@ -17,10 +17,12 @@
 
 #include "common/resource_tls.h"
 
+#include <gen_cpp/Types_types.h>
 #include <pthread.h>
 
+#include <ostream>
+
 #include "common/logging.h"
-#include "gen_cpp/Types_types.h"
 
 namespace doris {
 
diff --git a/be/src/common/status.cpp b/be/src/common/status.cpp
index 85f4d69938..d4403f1112 100644
--- a/be/src/common/status.cpp
+++ b/be/src/common/status.cpp
@@ -4,10 +4,16 @@
 
 #include "common/status.h"
 
+#include <gen_cpp/Status_types.h>
+#include <gen_cpp/types.pb.h> // for PStatus
+#include <rapidjson/encodings.h>
 #include <rapidjson/prettywriter.h>
 #include <rapidjson/stringbuffer.h>
 
-#include "gen_cpp/types.pb.h" // for PStatus
+#include <algorithm>
+#include <new>
+#include <vector>
+
 #include "service/backend_options.h"
 
 namespace doris {
diff --git a/be/src/common/status.h b/be/src/common/status.h
index 330f8114bf..ec8fbba4fe 100644
--- a/be/src/common/status.h
+++ b/be/src/common/status.h
@@ -5,14 +5,17 @@
 #pragma once
 
 #include <fmt/format.h>
+#include <gen_cpp/Status_types.h> // for TStatus
 #include <glog/logging.h>
+#include <stdint.h>
 
 #include <iostream>
+#include <memory>
 #include <string>
 #include <string_view>
+#include <utility>
 
 #include "common/compiler_util.h"
-#include "gen_cpp/Status_types.h" // for TStatus
 #ifdef ENABLE_STACKTRACE
 #include "util/stack_util.h"
 #endif
diff --git a/be/src/exec/arrow/arrow_reader.cpp b/be/src/exec/arrow/arrow_reader.cpp
index 2e34b343d2..6696d5489c 100644
--- a/be/src/exec/arrow/arrow_reader.cpp
+++ b/be/src/exec/arrow/arrow_reader.cpp
@@ -16,23 +16,24 @@
 // under the License.
 #include "exec/arrow/arrow_reader.h"
 
-#include <arrow/array.h>
-#include <arrow/status.h>
-#include <time.h>
+#include <arrow/buffer.h>
+#include <arrow/record_batch.h>
+#include <opentelemetry/common/threadlocal.h>
+
+#include <algorithm>
+// IWYU pragma: no_include <bits/chrono.h>
+#include <chrono> // IWYU pragma: keep
+#include <ostream>
+#include <utility>
 
 #include "common/logging.h"
-#include "gen_cpp/PaloBrokerService_types.h"
-#include "gen_cpp/TPaloBrokerService.h"
-#include "io/io_common.h"
-#include "olap/iterators.h"
-#include "runtime/broker_mgr.h"
-#include "runtime/client_cache.h"
+#include "io/fs/file_reader.h"
 #include "runtime/descriptors.h"
-#include "runtime/exec_env.h"
 #include "runtime/runtime_state.h"
+#include "util/slice.h"
 #include "util/string_util.h"
-#include "util/thrift_util.h"
 #include "vec/core/block.h"
+#include "vec/core/column_with_type_and_name.h"
 #include "vec/utils/arrow_column_to_doris_column.h"
 
 namespace doris {
diff --git a/be/src/exec/arrow/arrow_reader.h b/be/src/exec/arrow/arrow_reader.h
index fc068092e6..e685e7edaf 100644
--- a/be/src/exec/arrow/arrow_reader.h
+++ b/be/src/exec/arrow/arrow_reader.h
@@ -22,23 +22,40 @@
 #include <arrow/io/api.h>
 #include <arrow/io/file.h>
 #include <arrow/io/interfaces.h>
+#include <arrow/result.h>
+#include <gen_cpp/PaloBrokerService_types.h>
+#include <gen_cpp/PlanNodes_types.h>
+#include <gen_cpp/Types_types.h>
 #include <parquet/api/reader.h>
 #include <parquet/api/writer.h>
 #include <parquet/arrow/reader.h>
 #include <parquet/arrow/writer.h>
 #include <parquet/exception.h>
+#include <parquet/platform.h>
+#include <stddef.h>
 #include <stdint.h>
 
+#include <atomic>
+#include <condition_variable>
+#include <list>
 #include <map>
+#include <memory>
+#include <mutex>
 #include <string>
+#include <thread>
+#include <vector>
 
+#include "common/config.h"
 #include "common/status.h"
-#include "gen_cpp/PaloBrokerService_types.h"
-#include "gen_cpp/PlanNodes_types.h"
-#include "gen_cpp/Types_types.h"
 #include "io/fs/file_reader.h"
+#include "io/fs/file_reader_writer_fwd.h"
 #include "vec/exec/format/generic_reader.h"
 
+namespace arrow {
+class RecordBatch;
+class RecordBatchReader;
+} // namespace arrow
+
 namespace doris {
 
 class ExecEnv;
@@ -47,6 +64,11 @@ class TNetworkAddress;
 class RuntimeState;
 class SlotDescriptor;
 class FileReader;
+class TupleDescriptor;
+
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 struct Statistics {
     int32_t filtered_row_groups = 0;
diff --git a/be/src/exec/arrow/parquet_reader.cpp b/be/src/exec/arrow/parquet_reader.cpp
index 5423db509f..f0a1755324 100644
--- a/be/src/exec/arrow/parquet_reader.cpp
+++ b/be/src/exec/arrow/parquet_reader.cpp
@@ -16,23 +16,34 @@
 // under the License.
 #include "exec/arrow/parquet_reader.h"
 
-#include <arrow/array.h>
+#include <arrow/record_batch.h>
+#include <arrow/result.h>
 #include <arrow/status.h>
-#include <arrow/type_fwd.h>
-#include <time.h>
+#include <arrow/type.h>
+#include <opentelemetry/common/threadlocal.h>
+#include <parquet/exception.h>
+#include <parquet/file_reader.h>
+#include <parquet/metadata.h>
+#include <parquet/properties.h>
+#include <parquet/schema.h>
 
 #include <algorithm>
-#include <cinttypes>
+#include <atomic>
+// IWYU pragma: no_include <bits/chrono.h>
+#include <chrono> // IWYU pragma: keep
+#include <condition_variable>
+#include <list>
+#include <map>
 #include <mutex>
+#include <ostream>
 #include <thread>
 
 #include "common/logging.h"
 #include "common/status.h"
-#include "runtime/descriptors.h"
 #include "util/string_util.h"
-#include "vec/common/string_ref.h"
 
 namespace doris {
+class TupleDescriptor;
 
 // Broker
 ParquetReaderWrap::ParquetReaderWrap(RuntimeState* state,
diff --git a/be/src/exec/arrow/parquet_reader.h b/be/src/exec/arrow/parquet_reader.h
index 93dd64a081..4f57fdc7b9 100644
--- a/be/src/exec/arrow/parquet_reader.h
+++ b/be/src/exec/arrow/parquet_reader.h
@@ -23,6 +23,10 @@
 #include <arrow/io/file.h>
 #include <arrow/io/interfaces.h>
 #include <arrow/status.h>
+#include <arrow/type_fwd.h>
+#include <gen_cpp/PaloBrokerService_types.h>
+#include <gen_cpp/PlanNodes_types.h>
+#include <gen_cpp/Types_types.h>
 #include <parquet/api/reader.h>
 #include <parquet/api/writer.h>
 #include <parquet/arrow/reader.h>
@@ -34,16 +38,23 @@
 #include <condition_variable>
 #include <list>
 #include <map>
+#include <memory>
 #include <mutex>
 #include <string>
 #include <thread>
+#include <vector>
 
 #include "common/config.h"
 #include "common/status.h"
 #include "exec/arrow/arrow_reader.h"
-#include "gen_cpp/PaloBrokerService_types.h"
-#include "gen_cpp/PlanNodes_types.h"
-#include "gen_cpp/Types_types.h"
+#include "io/fs/file_reader_writer_fwd.h"
+
+namespace arrow {
+class RecordBatch;
+} // namespace arrow
+namespace parquet {
+class FileMetaData;
+} // namespace parquet
 
 namespace doris {
 
@@ -54,6 +65,7 @@ class RuntimeState;
 class SlotDescriptor;
 class FileReader;
 class RowGroupReader;
+class TupleDescriptor;
 
 // Reader of parquet file
 class ParquetReaderWrap final : public ArrowReaderWrap {
diff --git a/be/src/exec/base_scanner.cpp b/be/src/exec/base_scanner.cpp
index 84329c832d..043913a5f8 100644
--- a/be/src/exec/base_scanner.cpp
+++ b/be/src/exec/base_scanner.cpp
@@ -17,16 +17,40 @@
 
 #include "base_scanner.h"
 
+#include <assert.h>
 #include <fmt/format.h>
+#include <gen_cpp/Metrics_types.h>
+#include <gen_cpp/PlanNodes_types.h>
+#include <glog/logging.h>
+#include <opentelemetry/common/threadlocal.h>
+#include <parallel_hashmap/phmap.h>
+#include <stddef.h>
+
+#include <boost/iterator/iterator_facade.hpp>
+#include <iterator>
+#include <map>
+#include <string>
+#include <utility>
 
 #include "common/consts.h"
-#include "common/utils.h"
-#include "exec/exec_node.h"
+#include "gutil/casts.h"
+#include "runtime/define_primitive_type.h"
 #include "runtime/descriptors.h"
 #include "runtime/runtime_state.h"
+#include "runtime/types.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/columns/column_vector.h"
+#include "vec/columns/columns_number.h"
+#include "vec/common/string_ref.h"
+#include "vec/core/column_with_type_and_name.h"
+#include "vec/data_types/data_type.h"
 #include "vec/data_types/data_type_factory.hpp"
+#include "vec/data_types/data_type_number.h"
+#include "vec/exprs/vexpr_context.h"
 
 namespace doris {
+class TColumn;
+class TNetworkAddress;
 
 BaseScanner::BaseScanner(RuntimeState* state, RuntimeProfile* profile,
                          const TBrokerScanRangeParams& params,
diff --git a/be/src/exec/base_scanner.h b/be/src/exec/base_scanner.h
index 8fbcb8b01f..291f7d653d 100644
--- a/be/src/exec/base_scanner.h
+++ b/be/src/exec/base_scanner.h
@@ -17,9 +17,21 @@
 
 #pragma once
 
+#include <gen_cpp/Exprs_types.h>
+#include <stdint.h>
+
+#include <memory>
+#include <unordered_map>
+#include <vector>
+
+#include "common/global_types.h"
 #include "common/status.h"
+#include "runtime/descriptors.h"
 #include "util/runtime_profile.h"
+#include "util/slice.h"
+#include "vec/columns/column.h"
 #include "vec/common/schema_util.h"
+#include "vec/core/block.h"
 #include "vec/exprs/vexpr.h"
 #include "vec/exprs/vexpr_context.h"
 
@@ -28,10 +40,14 @@ namespace doris {
 class TupleDescriptor;
 class RowDescriptor;
 class RuntimeState;
+class TBrokerRangeDesc;
+class TBrokerScanRangeParams;
+class TNetworkAddress;
 
 namespace vectorized {
 class VExprContext;
 class IColumn;
+
 using MutableColumnPtr = IColumn::MutablePtr;
 } // namespace vectorized
 
diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp
index 3ce9d004e2..197a353124 100644
--- a/be/src/exec/data_sink.cpp
+++ b/be/src/exec/data_sink.cpp
@@ -20,22 +20,28 @@
 
 #include "exec/data_sink.h"
 
+#include <gen_cpp/DataSinks_types.h>
+#include <gen_cpp/PaloInternalService_types.h>
+#include <glog/logging.h>
+
 #include <map>
 #include <memory>
+#include <ostream>
 #include <string>
+#include <utility>
 
-#include "gen_cpp/PaloInternalService_types.h"
-#include "runtime/runtime_state.h"
+#include "common/config.h"
 #include "vec/sink/vdata_stream_sender.h"
 #include "vec/sink/vjdbc_table_sink.h"
 #include "vec/sink/vmemory_scratch_sink.h"
-#include "vec/sink/vmysql_table_sink.h"
 #include "vec/sink/vodbc_table_sink.h"
 #include "vec/sink/vresult_file_sink.h"
 #include "vec/sink/vresult_sink.h"
 #include "vec/sink/vtablet_sink.h"
 
 namespace doris {
+class DescriptorTbl;
+class TExpr;
 
 Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink,
                                   const std::vector<TExpr>& output_exprs,
diff --git a/be/src/exec/data_sink.h b/be/src/exec/data_sink.h
index 0e0b9e63fd..3ea5da0c3c 100644
--- a/be/src/exec/data_sink.h
+++ b/be/src/exec/data_sink.h
@@ -20,11 +20,16 @@
 
 #pragma once
 
+#include <gen_cpp/DataSinks_types.h>
+#include <gen_cpp/Exprs_types.h>
+#include <opentelemetry/trace/span.h>
+#include <stddef.h>
+// IWYU pragma: no_include <opentelemetry/nostd/shared_ptr.h>
+#include <memory>
+#include <string>
 #include <vector>
 
 #include "common/status.h"
-#include "gen_cpp/DataSinks_types.h"
-#include "gen_cpp/Exprs_types.h"
 #include "runtime/descriptors.h"
 #include "runtime/query_statistics.h"
 #include "util/runtime_profile.h"
@@ -36,6 +41,11 @@ class ObjectPool;
 class RuntimeState;
 class TPlanFragmentExecParams;
 class RowDescriptor;
+class DescriptorTbl;
+class QueryStatistics;
+class TDataSink;
+class TExpr;
+class TPipelineFragmentParams;
 
 namespace vectorized {
 class Block;
diff --git a/be/src/exec/decompressor.cpp b/be/src/exec/decompressor.cpp
index 18bf064fcb..af69a896a2 100644
--- a/be/src/exec/decompressor.cpp
+++ b/be/src/exec/decompressor.cpp
@@ -17,6 +17,10 @@
 
 #include "exec/decompressor.h"
 
+#include <strings.h>
+
+#include <ostream>
+
 #include "common/logging.h"
 
 namespace doris {
diff --git a/be/src/exec/decompressor.h b/be/src/exec/decompressor.h
index a9e6ef7b02..af37335f1f 100644
--- a/be/src/exec/decompressor.h
+++ b/be/src/exec/decompressor.h
@@ -19,8 +19,12 @@
 
 #include <bzlib.h>
 #include <lz4/lz4frame.h>
+#include <stddef.h>
+#include <stdint.h>
 #include <zlib.h>
 
+#include <string>
+
 #ifdef DORIS_WITH_LZO
 #include <lzo/lzo1x.h>
 #include <lzo/lzoconf.h>
diff --git a/be/src/exec/es/es_scan_reader.cpp b/be/src/exec/es/es_scan_reader.cpp
index 0002d8f2d6..10c36ab67f 100644
--- a/be/src/exec/es/es_scan_reader.cpp
+++ b/be/src/exec/es/es_scan_reader.cpp
@@ -17,6 +17,8 @@
 
 #include "exec/es/es_scan_reader.h"
 
+#include <stdlib.h>
+
 #include <map>
 #include <sstream>
 #include <string>
@@ -24,7 +26,9 @@
 #include "common/config.h"
 #include "common/logging.h"
 #include "common/status.h"
+#include "exec/es/es_scroll_parser.h"
 #include "exec/es/es_scroll_query.h"
+#include "http/http_method.h"
 
 namespace doris {
 
diff --git a/be/src/exec/es/es_scan_reader.h b/be/src/exec/es/es_scan_reader.h
index 79042838af..95b3e37980 100644
--- a/be/src/exec/es/es_scan_reader.h
+++ b/be/src/exec/es/es_scan_reader.h
@@ -17,11 +17,17 @@
 
 #pragma once
 
+#include <map>
+#include <memory>
 #include <string>
 
 #include "exec/es/es_scroll_parser.h"
 #include "http/http_client.h"
 
+namespace doris {
+class ScrollParser;
+} // namespace doris
+
 using std::string;
 
 namespace doris {
diff --git a/be/src/exec/es/es_scroll_parser.cpp b/be/src/exec/es/es_scroll_parser.cpp
index d0a57b341c..abe793984c 100644
--- a/be/src/exec/es/es_scroll_parser.cpp
+++ b/be/src/exec/es/es_scroll_parser.cpp
@@ -18,20 +18,34 @@
 #include "exec/es/es_scroll_parser.h"
 
 #include <cctz/time_zone.h>
+#include <glog/logging.h>
 #include <gutil/strings/substitute.h>
-
-#include <boost/algorithm/string.hpp>
+#include <rapidjson/allocators.h>
+#include <rapidjson/encodings.h>
+#include <stdint.h>
+#include <string.h>
+
+// IWYU pragma: no_include <bits/chrono.h>
+#include <chrono> // IWYU pragma: keep
+#include <cstdlib>
+#include <ostream>
 #include <string>
 
 #include "common/status.h"
+#include "gutil/integral_types.h"
 #include "rapidjson/document.h"
 #include "rapidjson/rapidjson.h"
 #include "rapidjson/stringbuffer.h"
 #include "rapidjson/writer.h"
-#include "runtime/memory/mem_tracker.h"
+#include "runtime/decimalv2_value.h"
+#include "runtime/define_primitive_type.h"
+#include "runtime/descriptors.h"
+#include "runtime/primitive_type.h"
+#include "runtime/types.h"
+#include "util/binary_cast.hpp"
 #include "util/string_parser.hpp"
-#include "vec/columns/column_array.h"
-#include "vec/common/string_ref.h"
+#include "vec/columns/column.h"
+#include "vec/columns/column_nullable.h"
 #include "vec/core/field.h"
 #include "vec/runtime/vdatetime_value.h"
 
diff --git a/be/src/exec/es/es_scroll_parser.h b/be/src/exec/es/es_scroll_parser.h
index 2a40bde974..d4edcb9ed2 100644
--- a/be/src/exec/es/es_scroll_parser.h
+++ b/be/src/exec/es/es_scroll_parser.h
@@ -17,15 +17,21 @@
 
 #pragma once
 
+#include <rapidjson/rapidjson.h>
+
+#include <map>
 #include <string>
+#include <vector>
 
 #include "rapidjson/document.h"
 #include "runtime/descriptors.h"
 #include "vec/core/block.h"
+#include "vec/data_types/data_type.h"
 
 namespace doris {
 
 class Status;
+class TupleDescriptor;
 
 class ScrollParser {
 public:
diff --git a/be/src/exec/es/es_scroll_query.cpp b/be/src/exec/es/es_scroll_query.cpp
index b68e8624c3..e8d214fd9b 100644
--- a/be/src/exec/es/es_scroll_query.cpp
+++ b/be/src/exec/es/es_scroll_query.cpp
@@ -17,6 +17,11 @@
 
 #include "exec/es/es_scroll_query.h"
 
+#include <glog/logging.h>
+#include <rapidjson/encodings.h>
+#include <rapidjson/rapidjson.h>
+#include <stdlib.h>
+
 #include <sstream>
 
 #include "exec/es/es_scan_reader.h"
diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp
index d0dde43c93..6c3a011dc5 100644
--- a/be/src/exec/exec_node.cpp
+++ b/be/src/exec/exec_node.cpp
@@ -20,19 +20,32 @@
 
 #include "exec/exec_node.h"
 
+#include <gen_cpp/Metrics_types.h>
+#include <gen_cpp/PlanNodes_types.h>
+#include <opentelemetry/common/threadlocal.h>
 #include <thrift/protocol/TDebugProtocol.h>
-#include <unistd.h>
 
+#include <map>
 #include <sstream>
+#include <typeinfo>
+#include <utility>
 
+#include "common/config.h"
+#include "common/logging.h"
 #include "common/object_pool.h"
 #include "common/status.h"
+#include "exec/scan_node.h"
 #include "runtime/descriptors.h"
 #include "runtime/memory/mem_tracker.h"
 #include "runtime/runtime_state.h"
 #include "util/debug_util.h"
 #include "util/runtime_profile.h"
+#include "util/uid_util.h"
+#include "vec/columns/column.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/common/pod_array_fwd.h"
 #include "vec/core/block.h"
+#include "vec/core/column_with_type_and_name.h"
 #include "vec/exec/join/vhash_join_node.h"
 #include "vec/exec/join/vnested_loop_join_node.h"
 #include "vec/exec/scan/new_es_scan_node.h"
@@ -41,13 +54,13 @@
 #include "vec/exec/scan/new_odbc_scan_node.h"
 #include "vec/exec/scan/new_olap_scan_node.h"
 #include "vec/exec/scan/vmeta_scan_node.h"
+#include "vec/exec/scan/vscan_node.h"
 #include "vec/exec/vaggregation_node.h"
 #include "vec/exec/vanalytic_eval_node.h"
 #include "vec/exec/vassert_num_rows_node.h"
 #include "vec/exec/vdata_gen_scan_node.h"
 #include "vec/exec/vempty_set_node.h"
 #include "vec/exec/vexchange_node.h"
-#include "vec/exec/vmysql_scan_node.h"
 #include "vec/exec/vrepeat_node.h"
 #include "vec/exec/vschema_scan_node.h"
 #include "vec/exec/vselect_node.h"
@@ -56,8 +69,11 @@
 #include "vec/exec/vtable_function_node.h"
 #include "vec/exec/vunion_node.h"
 #include "vec/exprs/vexpr.h"
+#include "vec/exprs/vexpr_context.h"
+#include "vec/utils/util.hpp"
 
 namespace doris {
+class QueryStatistics;
 
 const std::string ExecNode::ROW_THROUGHPUT_COUNTER = "RowsReturnedRate";
 
diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h
index d5a38890ab..d5b9688361 100644
--- a/be/src/exec/exec_node.h
+++ b/be/src/exec/exec_node.h
@@ -20,18 +20,27 @@
 
 #pragma once
 
+#include <gen_cpp/PlanNodes_types.h>
+#include <stddef.h>
+#include <stdint.h>
+
+#include <atomic>
+#include <functional>
+#include <memory>
 #include <mutex>
 #include <sstream>
+#include <string>
 #include <vector>
 
+#include "common/global_types.h"
 #include "common/status.h"
-#include "gen_cpp/PlanNodes_types.h"
 #include "runtime/descriptors.h"
 #include "runtime/query_statistics.h"
 #include "service/backend_options.h"
 #include "util/blocking_queue.hpp"
 #include "util/runtime_profile.h"
 #include "util/telemetry/telemetry.h"
+#include "vec/core/block.h"
 #include "vec/exprs/vexpr_context.h"
 
 namespace doris {
@@ -40,10 +49,12 @@ class Counters;
 class RuntimeState;
 class TPlan;
 class MemTracker;
+class QueryStatistics;
 
 namespace vectorized {
 class Block;
 class VExpr;
+class VExprContext;
 } // namespace vectorized
 
 namespace pipeline {
diff --git a/be/src/exec/odbc_connector.cpp b/be/src/exec/odbc_connector.cpp
index 255eb1715c..3b3d896ce6 100644
--- a/be/src/exec/odbc_connector.cpp
+++ b/be/src/exec/odbc_connector.cpp
@@ -17,12 +17,22 @@
 
 #include "exec/odbc_connector.h"
 
+#include <glog/logging.h>
+#include <sql.h>
 #include <sqlext.h>
+#include <wchar.h>
 
-#include <codecvt>
+#include <algorithm>
+#include <ostream>
 
-#include "runtime/primitive_type.h"
-#include "util/types.h"
+#include "runtime/define_primitive_type.h"
+#include "runtime/descriptors.h"
+#include "runtime/types.h"
+#include "util/runtime_profile.h"
+
+namespace doris {
+class RuntimeState;
+} // namespace doris
 
 #define ODBC_DISPOSE(h, ht, x, op)                                                        \
     {                                                                                     \
diff --git a/be/src/exec/odbc_connector.h b/be/src/exec/odbc_connector.h
index f7765a9fce..90ef031a10 100644
--- a/be/src/exec/odbc_connector.h
+++ b/be/src/exec/odbc_connector.h
@@ -16,12 +16,24 @@
 // under the License.
 
 #pragma once
+#include <fmt/format.h>
 #include <sqltypes.h>
+#include <stdint.h>
+#include <stdlib.h>
 
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "common/config.h"
 #include "common/status.h"
 #include "exec/table_connector.h"
 
 namespace doris {
+class RuntimeProfile;
+class RuntimeState;
+class TupleDescriptor;
+
 struct ODBCConnectorParam {
     std::string connect_string;
 
diff --git a/be/src/exec/olap_common.cpp b/be/src/exec/olap_common.cpp
index 7963139542..04ca8e5efc 100644
--- a/be/src/exec/olap_common.cpp
+++ b/be/src/exec/olap_common.cpp
@@ -17,10 +17,6 @@
 
 #include "exec/olap_common.h"
 
-#include <boost/lexical_cast.hpp>
-#include <set>
-#include <sstream>
-#include <string>
 #include <utility>
 #include <vector>
 
diff --git a/be/src/exec/olap_common.h b/be/src/exec/olap_common.h
index d0b6191927..e160d54fa3 100644
--- a/be/src/exec/olap_common.h
+++ b/be/src/exec/olap_common.h
@@ -17,17 +17,30 @@
 
 #pragma once
 
+#include <gen_cpp/PaloInternalService_types.h>
+#include <glog/logging.h>
+#include <stddef.h>
+
+#include <boost/container/detail/std_fwd.hpp>
 #include <boost/lexical_cast.hpp>
 #include <cstdint>
+#include <iterator>
 #include <map>
+#include <memory>
+#include <set>
 #include <sstream>
 #include <string>
 #include <type_traits>
+#include <utility>
 #include <variant>
+#include <vector>
 
+#include "common/status.h"
 #include "exec/olap_utils.h"
 #include "olap/olap_common.h"
 #include "olap/olap_tuple.h"
+#include "runtime/datetime_value.h"
+#include "runtime/define_primitive_type.h"
 #include "runtime/primitive_type.h"
 #include "runtime/type_limit.h"
 #include "vec/core/types.h"
diff --git a/be/src/exec/rowid_fetcher.cpp b/be/src/exec/rowid_fetcher.cpp
index a9a326c0e3..57bacb4fcb 100644
--- a/be/src/exec/rowid_fetcher.cpp
+++ b/be/src/exec/rowid_fetcher.cpp
@@ -17,14 +17,36 @@
 
 #include "exec/rowid_fetcher.h"
 
+#include <brpc/callback.h>
+#include <brpc/controller.h>
+#include <butil/endpoint.h>
+#include <fmt/format.h>
+#include <gen_cpp/internal_service.pb.h>
+#include <glog/logging.h>
+#include <stddef.h>
+#include <stdint.h>
+
+#include <algorithm>
+#include <ostream>
+#include <string>
+#include <unordered_map>
+#include <utility>
+
 #include "bthread/countdown_event.h"
+#include "common/config.h"
 #include "exec/tablet_info.h" // DorisNodesInfo
-#include "gen_cpp/Types_types.h"
-#include "gen_cpp/internal_service.pb.h"
+#include "olap/olap_common.h"
+#include "olap/utils.h"
+#include "runtime/descriptors.h"
 #include "runtime/exec_env.h"       // ExecEnv
 #include "runtime/runtime_state.h"  // RuntimeState
 #include "util/brpc_client_cache.h" // BrpcClientCache
 #include "util/defer_op.h"
+#include "vec/columns/column.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/columns/column_string.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/string_ref.h"
 #include "vec/core/block.h" // Block
 
 namespace doris {
diff --git a/be/src/exec/rowid_fetcher.h b/be/src/exec/rowid_fetcher.h
index 57101c5033..19dabdf901 100644
--- a/be/src/exec/rowid_fetcher.h
+++ b/be/src/exec/rowid_fetcher.h
@@ -17,13 +17,25 @@
 
 #pragma once
 
-#include "gen_cpp/internal_service.pb.h"
+#include <gen_cpp/internal_service.pb.h>
+
+#include <memory>
+#include <vector>
+
+#include "common/status.h"
 #include "vec/core/block.h"
+#include "vec/data_types/data_type.h"
 
 namespace doris {
 
 class DorisNodesInfo;
 class RuntimeState;
+class TupleDescriptor;
+
+namespace vectorized {
+class ColumnString;
+class MutableBlock;
+} // namespace vectorized
 
 // fetch rows by global rowid
 // tablet_id/rowset_name/segment_id/ordinal_id
diff --git a/be/src/exec/scan_node.cpp b/be/src/exec/scan_node.cpp
index 2b8bb6b5cf..281a55b6f3 100644
--- a/be/src/exec/scan_node.cpp
+++ b/be/src/exec/scan_node.cpp
@@ -20,9 +20,18 @@
 
 #include "exec/scan_node.h"
 
+#include <gen_cpp/Metrics_types.h>
+
+#include <memory>
+
+#include "vec/exprs/vexpr_context.h"
 #include "vec/utils/util.hpp"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class VExpr;
+} // namespace vectorized
 
 const std::string ScanNode::_s_bytes_read_counter = "BytesRead";
 const std::string ScanNode::_s_rows_read_counter = "RowsRead";
diff --git a/be/src/exec/scan_node.h b/be/src/exec/scan_node.h
index c83bf9884b..aed701dbe9 100644
--- a/be/src/exec/scan_node.h
+++ b/be/src/exec/scan_node.h
@@ -20,15 +20,24 @@
 
 #pragma once
 
+#include <gen_cpp/PaloInternalService_types.h>
+
+#include <functional>
 #include <string>
+#include <vector>
 
+#include "common/status.h"
 #include "exec/exec_node.h"
-#include "gen_cpp/PaloInternalService_types.h"
 #include "util/runtime_profile.h"
 
 namespace doris {
 
 class TScanRange;
+class DescriptorTbl;
+class ObjectPool;
+class RuntimeState;
+class TPlanNode;
+class TScanRangeParams;
 
 // Abstract base class of all scan nodes; introduces set_scan_range().
 //
diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp
index ebca8591a5..fd47976c89 100644
--- a/be/src/exec/schema_scanner.cpp
+++ b/be/src/exec/schema_scanner.cpp
@@ -17,7 +17,14 @@
 
 #include "exec/schema_scanner.h"
 
-#include <cstddef>
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/Types_types.h>
+#include <glog/logging.h>
+#include <string.h>
+
+#include <new>
+#include <ostream>
+#include <utility>
 
 #include "exec/schema_scanner/schema_charsets_scanner.h"
 #include "exec/schema_scanner/schema_collations_scanner.h"
@@ -33,13 +40,23 @@
 #include "exec/schema_scanner/schema_user_privileges_scanner.h"
 #include "exec/schema_scanner/schema_variables_scanner.h"
 #include "exec/schema_scanner/schema_views_scanner.h"
+#include "olap/hll.h"
 #include "runtime/define_primitive_type.h"
-#include "util/encryption_util.h"
+#include "util/types.h"
 #include "vec/columns/column.h"
+#include "vec/columns/column_complex.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/columns/column_string.h"
+#include "vec/columns/column_vector.h"
+#include "vec/columns/columns_number.h"
 #include "vec/common/string_ref.h"
 #include "vec/core/block.h"
+#include "vec/core/column_with_type_and_name.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type.h"
 
 namespace doris {
+class ObjectPool;
 
 DorisServer* SchemaScanner::_s_doris_server;
 
diff --git a/be/src/exec/schema_scanner.h b/be/src/exec/schema_scanner.h
index 3602defbd0..bc600bdf58 100644
--- a/be/src/exec/schema_scanner.h
+++ b/be/src/exec/schema_scanner.h
@@ -17,12 +17,18 @@
 
 #pragma once
 
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/Types_types.h>
+#include <stddef.h>
+#include <stdint.h>
+
+#include <memory>
 #include <string>
+#include <vector>
 
 #include "common/object_pool.h"
 #include "common/status.h"
-#include "gen_cpp/Descriptors_types.h"
-#include "gen_cpp/Types_types.h"
+#include "runtime/define_primitive_type.h"
 #include "util/runtime_profile.h"
 #include "vec/core/block.h"
 
@@ -31,6 +37,8 @@ namespace doris {
 // forehead declare class, because jni function init in DorisServer.
 class DorisServer;
 class RuntimeState;
+class ObjectPool;
+class TUserIdentity;
 
 namespace vectorized {
 class Block;
diff --git a/be/src/exec/schema_scanner/schema_charsets_scanner.cpp b/be/src/exec/schema_scanner/schema_charsets_scanner.cpp
index 824b5a76ea..4a801f50c1 100644
--- a/be/src/exec/schema_scanner/schema_charsets_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_charsets_scanner.cpp
@@ -17,10 +17,18 @@
 
 #include "exec/schema_scanner/schema_charsets_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+#include <string.h>
+
 #include "common/status.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaCharsetsScanner::_s_css_columns = {
         //   name,       type,          size
diff --git a/be/src/exec/schema_scanner/schema_charsets_scanner.h b/be/src/exec/schema_scanner/schema_charsets_scanner.h
index 37b2866bab..dfc83cc34b 100644
--- a/be/src/exec/schema_scanner/schema_charsets_scanner.h
+++ b/be/src/exec/schema_scanner/schema_charsets_scanner.h
@@ -19,9 +19,15 @@
 
 #include <stdint.h>
 
+#include <vector>
+
+#include "common/status.h"
 #include "exec/schema_scanner.h"
 
 namespace doris {
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaCharsetsScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_collations_scanner.cpp b/be/src/exec/schema_scanner/schema_collations_scanner.cpp
index 472ba92074..f63e7622b4 100644
--- a/be/src/exec/schema_scanner/schema_collations_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_collations_scanner.cpp
@@ -17,11 +17,18 @@
 
 #include "exec/schema_scanner/schema_collations_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+#include <string.h>
+
 #include "common/status.h"
-#include "runtime/primitive_type.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaCollationsScanner::_s_cols_columns = {
         //   name,       type,          size
diff --git a/be/src/exec/schema_scanner/schema_collations_scanner.h b/be/src/exec/schema_scanner/schema_collations_scanner.h
index d421cf3818..260ef0a170 100644
--- a/be/src/exec/schema_scanner/schema_collations_scanner.h
+++ b/be/src/exec/schema_scanner/schema_collations_scanner.h
@@ -19,9 +19,15 @@
 
 #include <stdint.h>
 
+#include <vector>
+
+#include "common/status.h"
 #include "exec/schema_scanner.h"
 
 namespace doris {
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaCollationsScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_columns_scanner.cpp b/be/src/exec/schema_scanner/schema_columns_scanner.cpp
index 963ed68ebd..e980cced64 100644
--- a/be/src/exec/schema_scanner/schema_columns_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_columns_scanner.cpp
@@ -17,16 +17,24 @@
 
 #include "exec/schema_scanner/schema_columns_scanner.h"
 
-#include <cstddef>
+#include <fmt/format.h>
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+#include <gen_cpp/Types_types.h>
+
 #include <cstdint>
-#include <memory>
-#include <sstream>
 
 #include "exec/schema_scanner/schema_helper.h"
-#include "runtime/primitive_type.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
+class RuntimeState;
+
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaColumnsScanner::_s_col_columns = {
         //   name,       type,          size,                     is_null
diff --git a/be/src/exec/schema_scanner/schema_columns_scanner.h b/be/src/exec/schema_scanner/schema_columns_scanner.h
index 634ff08e2c..4165c6b0bc 100644
--- a/be/src/exec/schema_scanner/schema_columns_scanner.h
+++ b/be/src/exec/schema_scanner/schema_columns_scanner.h
@@ -17,12 +17,19 @@
 
 #pragma once
 
+#include <gen_cpp/FrontendService_types.h>
+
 #include <string>
+#include <vector>
 
+#include "common/status.h"
 #include "exec/schema_scanner.h"
-#include "gen_cpp/FrontendService_types.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaColumnsScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_dummy_scanner.cpp b/be/src/exec/schema_scanner/schema_dummy_scanner.cpp
index b858855718..1d5956f390 100644
--- a/be/src/exec/schema_scanner/schema_dummy_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_dummy_scanner.cpp
@@ -17,6 +17,15 @@
 
 #include "schema_dummy_scanner.h"
 
+#include <vector>
+
+namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
+} // namespace doris
+
 namespace {
 std::vector<doris::SchemaScanner::ColumnDesc> DUMMY_COLUMN;
 }
diff --git a/be/src/exec/schema_scanner/schema_dummy_scanner.h b/be/src/exec/schema_scanner/schema_dummy_scanner.h
index d48466e83a..919bae8fbb 100644
--- a/be/src/exec/schema_scanner/schema_dummy_scanner.h
+++ b/be/src/exec/schema_scanner/schema_dummy_scanner.h
@@ -17,9 +17,14 @@
 
 #pragma once
 
+#include "common/status.h"
 #include "exec/schema_scanner.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaDummyScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_files_scanner.cpp b/be/src/exec/schema_scanner/schema_files_scanner.cpp
index 1eb8132668..596d21c3dd 100644
--- a/be/src/exec/schema_scanner/schema_files_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_files_scanner.cpp
@@ -17,12 +17,20 @@
 
 #include "exec/schema_scanner/schema_files_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+#include <stdint.h>
+
 #include "exec/schema_scanner/schema_helper.h"
-#include "runtime/primitive_type.h"
+#include "runtime/define_primitive_type.h"
 #include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaFilesScanner::_s_tbls_columns = {
         //   name,       type,          size,     is_null
diff --git a/be/src/exec/schema_scanner/schema_files_scanner.h b/be/src/exec/schema_scanner/schema_files_scanner.h
index 0e33bf6dd3..7d140bdbe4 100644
--- a/be/src/exec/schema_scanner/schema_files_scanner.h
+++ b/be/src/exec/schema_scanner/schema_files_scanner.h
@@ -17,10 +17,18 @@
 
 #pragma once
 
+#include <gen_cpp/FrontendService_types.h>
+
+#include <vector>
+
+#include "common/status.h"
 #include "exec/schema_scanner.h"
-#include "gen_cpp/FrontendService_types.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaFilesScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_helper.cpp b/be/src/exec/schema_scanner/schema_helper.cpp
index dc4e213d67..9dbc40886e 100644
--- a/be/src/exec/schema_scanner/schema_helper.cpp
+++ b/be/src/exec/schema_scanner/schema_helper.cpp
@@ -17,15 +17,24 @@
 
 #include "exec/schema_scanner/schema_helper.h"
 
-#include <sstream>
-#include <thread>
+#include <gen_cpp/FrontendService.h>
 
-#include "gen_cpp/FrontendService.h"
-#include "gen_cpp/FrontendService_types.h"
 #include "runtime/client_cache.h"
 #include "util/thrift_rpc_helper.h"
 
 namespace doris {
+class TDescribeTableParams;
+class TDescribeTableResult;
+class TDescribeTablesParams;
+class TDescribeTablesResult;
+class TGetDbsParams;
+class TGetDbsResult;
+class TGetTablesParams;
+class TGetTablesResult;
+class TListPrivilegesResult;
+class TListTableStatusResult;
+class TShowVariableRequest;
+class TShowVariableResult;
 
 Status SchemaHelper::get_db_names(const std::string& ip, const int32_t port,
                                   const TGetDbsParams& request, TGetDbsResult* result) {
diff --git a/be/src/exec/schema_scanner/schema_helper.h b/be/src/exec/schema_scanner/schema_helper.h
index a6a2ed976e..d6a340e1e4 100644
--- a/be/src/exec/schema_scanner/schema_helper.h
+++ b/be/src/exec/schema_scanner/schema_helper.h
@@ -17,10 +17,26 @@
 
 #pragma once
 
+#include <gen_cpp/FrontendService_types.h>
+#include <stdint.h>
+
+#include <string>
+
 #include "common/status.h"
-#include "gen_cpp/FrontendService_types.h"
 
 namespace doris {
+class TDescribeTableParams;
+class TDescribeTableResult;
+class TDescribeTablesParams;
+class TDescribeTablesResult;
+class TGetDbsParams;
+class TGetDbsResult;
+class TGetTablesParams;
+class TGetTablesResult;
+class TListPrivilegesResult;
+class TListTableStatusResult;
+class TShowVariableRequest;
+class TShowVariableResult;
 
 // this class is a helper for getting schema info from FE
 class SchemaHelper {
diff --git a/be/src/exec/schema_scanner/schema_partitions_scanner.cpp b/be/src/exec/schema_scanner/schema_partitions_scanner.cpp
index 5aaded0f93..7c6b7827b9 100644
--- a/be/src/exec/schema_scanner/schema_partitions_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_partitions_scanner.cpp
@@ -17,13 +17,21 @@
 
 #include "exec/schema_scanner/schema_partitions_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+#include <stdint.h>
+
 #include "exec/schema_scanner/schema_helper.h"
-#include "runtime/datetime_value.h"
-#include "runtime/primitive_type.h"
+#include "runtime/decimalv2_value.h"
+#include "runtime/define_primitive_type.h"
 #include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaPartitionsScanner::_s_tbls_columns = {
         //   name,       type,          size,     is_null
diff --git a/be/src/exec/schema_scanner/schema_partitions_scanner.h b/be/src/exec/schema_scanner/schema_partitions_scanner.h
index 197f917a1d..1c5a86f145 100644
--- a/be/src/exec/schema_scanner/schema_partitions_scanner.h
+++ b/be/src/exec/schema_scanner/schema_partitions_scanner.h
@@ -17,10 +17,18 @@
 
 #pragma once
 
+#include <gen_cpp/FrontendService_types.h>
+
+#include <vector>
+
+#include "common/status.h"
 #include "exec/schema_scanner.h"
-#include "gen_cpp/FrontendService_types.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaPartitionsScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_rowsets_scanner.cpp b/be/src/exec/schema_scanner/schema_rowsets_scanner.cpp
index 1028812910..49a8d1fe7f 100644
--- a/be/src/exec/schema_scanner/schema_rowsets_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_rowsets_scanner.cpp
@@ -17,20 +17,32 @@
 
 #include "exec/schema_scanner/schema_rowsets_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+
+#include <algorithm>
 #include <cstddef>
+#include <memory>
+#include <shared_mutex>
+#include <string>
+#include <utility>
 
 #include "common/status.h"
-#include "gutil/integral_types.h"
-#include "olap/rowset/beta_rowset.h"
+#include "olap/olap_common.h"
 #include "olap/rowset/rowset.h"
-#include "olap/rowset/segment_v2/segment.h"
-#include "olap/segment_loader.h"
+#include "olap/rowset/rowset_meta.h"
 #include "olap/storage_engine.h"
 #include "olap/tablet.h"
-#include "runtime/descriptors.h"
-#include "runtime/primitive_type.h"
+#include "olap/tablet_manager.h"
+#include "runtime/define_primitive_type.h"
+#include "runtime/runtime_state.h"
+#include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
+
 namespace doris {
+namespace vectorized {
+class Block;
+} // namespace vectorized
+
 std::vector<SchemaScanner::ColumnDesc> SchemaRowsetsScanner::_s_tbls_columns = {
         //   name,       type,          size,     is_null
         {"BACKEND_ID", TYPE_BIGINT, sizeof(int64_t), true},
diff --git a/be/src/exec/schema_scanner/schema_rowsets_scanner.h b/be/src/exec/schema_scanner/schema_rowsets_scanner.h
index 1e8921a78d..0a8dabc270 100644
--- a/be/src/exec/schema_scanner/schema_rowsets_scanner.h
+++ b/be/src/exec/schema_scanner/schema_rowsets_scanner.h
@@ -20,13 +20,21 @@
 #include <cstddef>
 #include <cstdint>
 #include <memory>
+#include <vector>
 
 #include "common/status.h"
 #include "exec/schema_scanner.h"
 #include "olap/rowset/rowset.h"
 #include "olap/rowset/segment_v2/segment.h"
 #include "runtime/runtime_state.h"
+
 namespace doris {
+class RuntimeState;
+
+namespace vectorized {
+class Block;
+} // namespace vectorized
+
 class SchemaRowsetsScanner : public SchemaScanner {
 public:
     SchemaRowsetsScanner();
diff --git a/be/src/exec/schema_scanner/schema_schema_privileges_scanner.cpp b/be/src/exec/schema_scanner/schema_schema_privileges_scanner.cpp
index 17973a1e61..4b210bc5eb 100644
--- a/be/src/exec/schema_scanner/schema_schema_privileges_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_schema_privileges_scanner.cpp
@@ -17,11 +17,21 @@
 
 #include "exec/schema_scanner/schema_schema_privileges_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+
+#include <string>
+
 #include "exec/schema_scanner/schema_helper.h"
-#include "runtime/primitive_type.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaSchemaPrivilegesScanner::_s_tbls_columns = {
         //   name,       type,          size,     is_null
diff --git a/be/src/exec/schema_scanner/schema_schema_privileges_scanner.h b/be/src/exec/schema_scanner/schema_schema_privileges_scanner.h
index a52643afd1..9b34f952f5 100644
--- a/be/src/exec/schema_scanner/schema_schema_privileges_scanner.h
+++ b/be/src/exec/schema_scanner/schema_schema_privileges_scanner.h
@@ -17,10 +17,18 @@
 
 #pragma once
 
+#include <gen_cpp/FrontendService_types.h>
+
+#include <vector>
+
+#include "common/status.h"
 #include "exec/schema_scanner.h"
-#include "gen_cpp/FrontendService_types.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaSchemaPrivilegesScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_schemata_scanner.cpp b/be/src/exec/schema_scanner/schema_schemata_scanner.cpp
index 4c2b05a172..b0d0aca2bb 100644
--- a/be/src/exec/schema_scanner/schema_schemata_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_schemata_scanner.cpp
@@ -17,11 +17,21 @@
 
 #include "exec/schema_scanner/schema_schemata_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+
+#include <string>
+
 #include "exec/schema_scanner/schema_helper.h"
-#include "runtime/primitive_type.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaSchemataScanner::_s_columns = {
         //   name,       type,          size
diff --git a/be/src/exec/schema_scanner/schema_schemata_scanner.h b/be/src/exec/schema_scanner/schema_schemata_scanner.h
index 572f8b741d..90437e0f78 100644
--- a/be/src/exec/schema_scanner/schema_schemata_scanner.h
+++ b/be/src/exec/schema_scanner/schema_schemata_scanner.h
@@ -17,10 +17,18 @@
 
 #pragma once
 
+#include <gen_cpp/FrontendService_types.h>
+
+#include <vector>
+
+#include "common/status.h"
 #include "exec/schema_scanner.h"
-#include "gen_cpp/FrontendService_types.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaSchemataScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_statistics_scanner.cpp b/be/src/exec/schema_scanner/schema_statistics_scanner.cpp
index ed2e713da0..f4f3d5dba8 100644
--- a/be/src/exec/schema_scanner/schema_statistics_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_statistics_scanner.cpp
@@ -17,7 +17,9 @@
 
 #include "exec/schema_scanner/schema_statistics_scanner.h"
 
-#include "runtime/primitive_type.h"
+#include <stdint.h>
+
+#include "runtime/define_primitive_type.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
diff --git a/be/src/exec/schema_scanner/schema_statistics_scanner.h b/be/src/exec/schema_scanner/schema_statistics_scanner.h
index e31c27277b..2d02863bfc 100644
--- a/be/src/exec/schema_scanner/schema_statistics_scanner.h
+++ b/be/src/exec/schema_scanner/schema_statistics_scanner.h
@@ -17,6 +17,8 @@
 
 #pragma once
 
+#include <vector>
+
 #include "exec/schema_scanner.h"
 
 namespace doris {
diff --git a/be/src/exec/schema_scanner/schema_table_privileges_scanner.cpp b/be/src/exec/schema_scanner/schema_table_privileges_scanner.cpp
index d40ce08d18..4a4982bad7 100644
--- a/be/src/exec/schema_scanner/schema_table_privileges_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_table_privileges_scanner.cpp
@@ -17,13 +17,21 @@
 
 #include "exec/schema_scanner/schema_table_privileges_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+
 #include <string>
 
 #include "exec/schema_scanner/schema_helper.h"
-#include "runtime/primitive_type.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaTablePrivilegesScanner::_s_tbls_columns = {
         //   name,       type,          size,     is_null
diff --git a/be/src/exec/schema_scanner/schema_table_privileges_scanner.h b/be/src/exec/schema_scanner/schema_table_privileges_scanner.h
index 00545717c2..3aa0d4c659 100644
--- a/be/src/exec/schema_scanner/schema_table_privileges_scanner.h
+++ b/be/src/exec/schema_scanner/schema_table_privileges_scanner.h
@@ -17,10 +17,18 @@
 
 #pragma once
 
+#include <gen_cpp/FrontendService_types.h>
+
+#include <vector>
+
+#include "common/status.h"
 #include "exec/schema_scanner.h"
-#include "gen_cpp/FrontendService_types.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaTablePrivilegesScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_tables_scanner.cpp b/be/src/exec/schema_scanner/schema_tables_scanner.cpp
index def0ef5540..3acf255df2 100644
--- a/be/src/exec/schema_scanner/schema_tables_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_tables_scanner.cpp
@@ -17,13 +17,27 @@
 
 #include "exec/schema_scanner/schema_tables_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+#include <stdint.h>
+
+#include <string>
+
 #include "common/status.h"
 #include "exec/schema_scanner/schema_helper.h"
-#include "runtime/primitive_type.h"
-#include "vec/columns/column_complex.h"
+#include "runtime/decimalv2_value.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
+#include "util/timezone_utils.h"
 #include "vec/common/string_ref.h"
+#include "vec/runtime/vdatetime_value.h"
 
 namespace doris {
+class RuntimeState;
+
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaTablesScanner::_s_tbls_columns = {
         //   name,       type,          size,     is_null
diff --git a/be/src/exec/schema_scanner/schema_tables_scanner.h b/be/src/exec/schema_scanner/schema_tables_scanner.h
index 18488451cc..c295fc6ba2 100644
--- a/be/src/exec/schema_scanner/schema_tables_scanner.h
+++ b/be/src/exec/schema_scanner/schema_tables_scanner.h
@@ -17,12 +17,20 @@
 
 #pragma once
 
+#include <gen_cpp/FrontendService_types.h>
+
+#include <vector>
+
 #include "common/status.h"
 #include "exec/schema_scanner.h"
-#include "gen_cpp/FrontendService_types.h"
 #include "vec/core/block.h"
 
 namespace doris {
+class RuntimeState;
+
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaTablesScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_user_privileges_scanner.cpp b/be/src/exec/schema_scanner/schema_user_privileges_scanner.cpp
index bd1037941a..f74436d8d1 100644
--- a/be/src/exec/schema_scanner/schema_user_privileges_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_user_privileges_scanner.cpp
@@ -17,11 +17,21 @@
 
 #include "exec/schema_scanner/schema_user_privileges_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+
+#include <string>
+
 #include "exec/schema_scanner/schema_helper.h"
-#include "runtime/primitive_type.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaUserPrivilegesScanner::_s_tbls_columns = {
         //   name,       type,          size,     is_null
diff --git a/be/src/exec/schema_scanner/schema_user_privileges_scanner.h b/be/src/exec/schema_scanner/schema_user_privileges_scanner.h
index 1bef2b340b..66c3501c37 100644
--- a/be/src/exec/schema_scanner/schema_user_privileges_scanner.h
+++ b/be/src/exec/schema_scanner/schema_user_privileges_scanner.h
@@ -17,10 +17,18 @@
 
 #pragma once
 
+#include <gen_cpp/FrontendService_types.h>
+
+#include <vector>
+
+#include "common/status.h"
 #include "exec/schema_scanner.h"
-#include "gen_cpp/FrontendService_types.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaUserPrivilegesScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_variables_scanner.cpp b/be/src/exec/schema_scanner/schema_variables_scanner.cpp
index 7627ecdfe1..5efcee07b7 100644
--- a/be/src/exec/schema_scanner/schema_variables_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_variables_scanner.cpp
@@ -17,12 +17,24 @@
 
 #include "exec/schema_scanner/schema_variables_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+#include <string.h>
+
+#include <map>
+#include <string>
+#include <utility>
+
 #include "exec/schema_scanner/schema_helper.h"
-#include "runtime/primitive_type.h"
-#include "runtime/runtime_state.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaVariablesScanner::_s_vars_columns = {
         //   name,       type,          size
diff --git a/be/src/exec/schema_scanner/schema_variables_scanner.h b/be/src/exec/schema_scanner/schema_variables_scanner.h
index 4befea7bda..2728b70530 100644
--- a/be/src/exec/schema_scanner/schema_variables_scanner.h
+++ b/be/src/exec/schema_scanner/schema_variables_scanner.h
@@ -17,13 +17,22 @@
 
 #pragma once
 
+#include <gen_cpp/FrontendService_types.h>
+#include <gen_cpp/Types_types.h>
+
 #include <map>
 #include <string>
+#include <vector>
 
+#include "common/status.h"
 #include "exec/schema_scanner.h"
-#include "gen_cpp/FrontendService_types.h"
 
 namespace doris {
+class RuntimeState;
+
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaVariablesScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/schema_scanner/schema_views_scanner.cpp b/be/src/exec/schema_scanner/schema_views_scanner.cpp
index 34e9d93dae..abfb1929f3 100644
--- a/be/src/exec/schema_scanner/schema_views_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_views_scanner.cpp
@@ -17,11 +17,21 @@
 
 #include "exec/schema_scanner/schema_views_scanner.h"
 
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+
+#include <string>
+
 #include "exec/schema_scanner/schema_helper.h"
-#include "runtime/primitive_type.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 std::vector<SchemaScanner::ColumnDesc> SchemaViewsScanner::_s_tbls_columns = {
         //   name,       type,          size,     is_null
diff --git a/be/src/exec/schema_scanner/schema_views_scanner.h b/be/src/exec/schema_scanner/schema_views_scanner.h
index 9eeca601f3..65683c4a8b 100644
--- a/be/src/exec/schema_scanner/schema_views_scanner.h
+++ b/be/src/exec/schema_scanner/schema_views_scanner.h
@@ -17,10 +17,18 @@
 
 #pragma once
 
+#include <gen_cpp/FrontendService_types.h>
+
+#include <vector>
+
+#include "common/status.h"
 #include "exec/schema_scanner.h"
-#include "gen_cpp/FrontendService_types.h"
 
 namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
 
 class SchemaViewsScanner : public SchemaScanner {
 public:
diff --git a/be/src/exec/table_connector.cpp b/be/src/exec/table_connector.cpp
index e4af221a2b..ef0b9db8c3 100644
--- a/be/src/exec/table_connector.cpp
+++ b/be/src/exec/table_connector.cpp
@@ -17,22 +17,37 @@
 
 #include "exec/table_connector.h"
 
-#include <fmt/core.h>
+#include <bthread/errno.h>
+#include <errno.h>
+#include <gen_cpp/Metrics_types.h>
 #include <gen_cpp/Types_types.h>
 #include <glog/logging.h>
 #include <iconv.h>
 
+#include <cstdlib>
+#include <memory>
+#include <string_view>
+#include <type_traits>
+
+#include "runtime/decimalv2_value.h"
 #include "runtime/define_primitive_type.h"
-#include "runtime/primitive_type.h"
-#include "util/mysql_global.h"
+#include "util/binary_cast.hpp"
+#include "vec/columns/column.h"
 #include "vec/columns/column_array.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/common/assert_cast.h"
+#include "vec/common/string_ref.h"
 #include "vec/core/block.h"
+#include "vec/core/column_with_type_and_name.h"
 #include "vec/data_types/data_type.h"
 #include "vec/data_types/data_type_array.h"
+#include "vec/data_types/data_type_nullable.h"
 #include "vec/exprs/vexpr.h"
 #include "vec/exprs/vexpr_context.h"
+#include "vec/runtime/vdatetime_value.h"
 
 namespace doris {
+class TupleDescriptor;
 
 // Default max buffer size use in insert to: 50MB, normally a batch is smaller than the size
 static constexpr uint32_t INSERT_BUFFER_SIZE = 1024l * 1024 * 50;
diff --git a/be/src/exec/table_connector.h b/be/src/exec/table_connector.h
index 872aa7fe14..27d3061e75 100644
--- a/be/src/exec/table_connector.h
+++ b/be/src/exec/table_connector.h
@@ -19,6 +19,7 @@
 
 #include <fmt/format.h>
 #include <gen_cpp/Types_types.h>
+#include <stdint.h>
 
 #include <boost/format.hpp>
 #include <cstdlib>
@@ -27,9 +28,20 @@
 
 #include "common/status.h"
 #include "runtime/descriptors.h"
+#include "runtime/types.h"
+#include "util/runtime_profile.h"
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/data_types/data_type.h"
 #include "vec/exprs/vexpr_context.h"
 
 namespace doris {
+class RuntimeState;
+class TupleDescriptor;
+
+namespace vectorized {
+class Block;
+class VExprContext;
+} // namespace vectorized
 
 // Table Connector for scan data from ODBC/JDBC
 class TableConnector {
diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp
index b4583943ea..1b79278d8b 100644
--- a/be/src/exec/tablet_info.cpp
+++ b/be/src/exec/tablet_info.cpp
@@ -17,10 +17,28 @@
 
 #include "exec/tablet_info.h"
 
+#include <butil/fast_rand.h>
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/Exprs_types.h>
+#include <gen_cpp/Types_types.h>
+#include <gen_cpp/descriptors.pb.h>
+#include <glog/logging.h>
+#include <stddef.h>
+
+#include <algorithm>
+#include <ostream>
+
+#include "olap/tablet_schema.h"
+#include "runtime/descriptors.h"
 #include "runtime/large_int_value.h"
+#include "runtime/memory/mem_tracker.h"
 #include "runtime/raw_value.h"
+#include "runtime/types.h"
+#include "util/hash_util.hpp"
 #include "util/string_parser.hpp"
+#include "vec/common/string_ref.h"
 #include "vec/exprs/vexpr.h"
+#include "vec/runtime/vdatetime_value.h"
 
 namespace doris {
 
diff --git a/be/src/exec/tablet_info.h b/be/src/exec/tablet_info.h
index 97dcd2a40d..6c9272787d 100644
--- a/be/src/exec/tablet_info.h
+++ b/be/src/exec/tablet_info.h
@@ -17,22 +17,39 @@
 
 #pragma once
 
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/descriptors.pb.h>
+
 #include <cstdint>
+#include <functional>
+#include <iterator>
 #include <map>
 #include <memory>
+#include <string>
 #include <unordered_map>
+#include <utility>
 #include <vector>
 
 #include "common/object_pool.h"
 #include "common/status.h"
-#include "gen_cpp/Descriptors_types.h"
-#include "gen_cpp/descriptors.pb.h"
 #include "olap/tablet_schema.h"
 #include "runtime/descriptors.h"
+#include "vec/columns/column.h"
 #include "vec/core/block.h"
+#include "vec/core/column_with_type_and_name.h"
 #include "vec/exprs/vexpr_context.h"
 
 namespace doris {
+class MemTracker;
+class SlotDescriptor;
+class TExprNode;
+class TabletColumn;
+class TabletIndex;
+class TupleDescriptor;
+
+namespace vectorized {
+class VExprContext;
+} // namespace vectorized
 
 struct OlapTableIndexSchema {
     int64_t index_id;
diff --git a/be/src/exec/text_converter.cpp b/be/src/exec/text_converter.cpp
index 3859bd3022..8249b42a98 100644
--- a/be/src/exec/text_converter.cpp
+++ b/be/src/exec/text_converter.cpp
@@ -17,15 +17,25 @@
 
 #include "text_converter.h"
 
+#include <glog/logging.h>
 #include <sql.h>
+#include <stdint.h>
 
+#include <ostream>
+
+#include "common/compiler_util.h"
+#include "olap/hll.h"
 #include "runtime/decimalv2_value.h"
+#include "runtime/define_primitive_type.h"
 #include "runtime/descriptors.h"
+#include "runtime/types.h"
+#include "util/slice.h"
 #include "util/string_parser.hpp"
-#include "util/types.h"
 #include "vec/columns/column_complex.h"
 #include "vec/columns/column_nullable.h"
-#include "vec/common/string_ref.h"
+#include "vec/columns/column_string.h"
+#include "vec/columns/column_vector.h"
+#include "vec/core/types.h"
 #include "vec/runtime/vdatetime_value.h"
 
 namespace doris {
diff --git a/be/src/exec/text_converter.h b/be/src/exec/text_converter.h
index d1b32a094e..0bf3b3f16e 100644
--- a/be/src/exec/text_converter.h
+++ b/be/src/exec/text_converter.h
@@ -17,7 +17,10 @@
 
 #pragma once
 
+#include <stddef.h>
+
 #include "vec/columns/column.h"
+
 namespace doris {
 
 class SlotDescriptor;
diff --git a/be/src/exprs/block_bloom_filter_avx_impl.cc b/be/src/exprs/block_bloom_filter_avx_impl.cc
index db8b9156f0..6781a23332 100644
--- a/be/src/exprs/block_bloom_filter_avx_impl.cc
+++ b/be/src/exprs/block_bloom_filter_avx_impl.cc
@@ -20,10 +20,14 @@
 
 #ifdef __AVX2__
 
+#include <glog/logging.h>
 #include <immintrin.h>
+#include <stddef.h>
+#include <stdint.h>
+
+#include <ostream>
 
 #include "exprs/block_bloom_filter.hpp"
-#include "gutil/macros.h"
 
 namespace doris {
 
diff --git a/be/src/exprs/block_bloom_filter_impl.cc b/be/src/exprs/block_bloom_filter_impl.cc
index 07d98e842b..d285edcb31 100644
--- a/be/src/exprs/block_bloom_filter_impl.cc
+++ b/be/src/exprs/block_bloom_filter_impl.cc
@@ -20,15 +20,20 @@
 // and modified by Doris
 
 #include <butil/iobuf.h>
+#include <fmt/format.h>
+#include <glog/logging.h>
+#include <stdint.h>
 
 #include <algorithm>
-#include <climits>
-#include <cmath>
+#include <climits> // IWYU pragma: keep
+#include <cmath>   // IWYU pragma: keep
 #include <cstdlib>
 #include <cstring>
 #include <string>
 
+#include "common/status.h"
 #include "exprs/block_bloom_filter.hpp"
+// IWYU pragma: no_include <emmintrin.h>
 #include "util/sse_util.hpp"
 
 namespace doris {
diff --git a/be/src/exprs/json_functions.cpp b/be/src/exprs/json_functions.cpp
index 5a3cbcc9e0..ebfc4b0ae8 100644
--- a/be/src/exprs/json_functions.cpp
+++ b/be/src/exprs/json_functions.cpp
@@ -17,25 +17,23 @@
 
 #include "exprs/json_functions.h"
 
+#include <rapidjson/allocators.h>
 #include <rapidjson/document.h>
-#include <rapidjson/stringbuffer.h>
-#include <rapidjson/writer.h>
+#include <rapidjson/encodings.h>
+#include <rapidjson/rapidjson.h>
 #include <re2/re2.h>
+#include <simdjson/simdjson.h> // IWYU pragma: keep
 #include <stdlib.h>
-#include <sys/time.h>
 
-#include <boost/algorithm/string.hpp>
-#include <iomanip>
+#include <boost/iterator/iterator_facade.hpp>
+#include <boost/token_functions.hpp>
+#include <boost/tokenizer.hpp>
 #include <sstream>
 #include <string>
-#include <string_view>
 #include <vector>
 
 #include "common/compiler_util.h"
 #include "common/logging.h"
-#include "gutil/strings/stringpiece.h"
-#include "udf/udf.h"
-#include "util/string_util.h"
 
 namespace doris {
 
diff --git a/be/src/exprs/json_functions.h b/be/src/exprs/json_functions.h
index 32a2f93814..2de0857219 100644
--- a/be/src/exprs/json_functions.h
+++ b/be/src/exprs/json_functions.h
@@ -18,14 +18,27 @@
 #pragma once
 
 #include <fmt/core.h>
+#include <fmt/format.h>
 #include <rapidjson/document.h>
 #include <simdjson.h>
 
 #include <sstream>
+#include <string>
+#include <utility>
+#include <vector>
 
 #include "common/status.h"
 #include "udf/udf.h"
 
+namespace simdjson {
+namespace fallback {
+namespace ondemand {
+class object;
+class value;
+} // namespace ondemand
+} // namespace fallback
+} // namespace simdjson
+
 namespace doris {
 
 enum JsonFunctionType {
diff --git a/be/src/exprs/math_functions.cpp b/be/src/exprs/math_functions.cpp
index 04ca6f57c1..e76cdde2a5 100644
--- a/be/src/exprs/math_functions.cpp
+++ b/be/src/exprs/math_functions.cpp
@@ -21,17 +21,14 @@
 #include "exprs/math_functions.h"
 
 #include <stdlib.h>
-
+#include <string.h>
+// IWYU pragma: no_include <bits/std_abs.h>
 #include <cmath>
-#include <iomanip>
-#include <random>
-#include <sstream>
+#include <limits>
 #include <string_view>
 
 #include "common/compiler_util.h"
-#include "runtime/decimalv2_value.h"
-#include "runtime/large_int_value.h"
-#include "util/simd/vstring_function.h"
+#include "udf/udf.h"
 #include "util/string_parser.hpp"
 
 namespace doris {
diff --git a/be/src/exprs/math_functions.h b/be/src/exprs/math_functions.h
index 108a690660..5399a39f42 100644
--- a/be/src/exprs/math_functions.h
+++ b/be/src/exprs/math_functions.h
@@ -26,6 +26,7 @@
 #include "vec/common/string_ref.h"
 
 namespace doris {
+class FunctionContext;
 
 class MathFunctions {
 public:
diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp
index 0a39b25248..30d8f3b988 100644
--- a/be/src/exprs/runtime_filter.cpp
+++ b/be/src/exprs/runtime_filter.cpp
@@ -17,27 +17,46 @@
 
 #include "runtime_filter.h"
 
+#include <gen_cpp/Opcodes_types.h>
+#include <gen_cpp/PaloInternalService_types.h>
+#include <gen_cpp/PlanNodes_types.h>
+#include <gen_cpp/Types_types.h>
+#include <gen_cpp/internal_service.pb.h>
+#include <stddef.h>
+
+#include <algorithm>
+// IWYU pragma: no_include <bits/chrono.h>
+#include <chrono> // IWYU pragma: keep
+#include <map>
 #include <memory>
+#include <mutex>
+#include <ostream>
+#include <utility>
 
+#include "common/logging.h"
 #include "common/object_pool.h"
 #include "common/status.h"
 #include "exprs/bitmapfilter_predicate.h"
+#include "exprs/bloom_filter_func.h"
 #include "exprs/create_predicate_function.h"
 #include "exprs/hybrid_set.h"
 #include "exprs/minmax_predicate.h"
-#include "gen_cpp/internal_service.pb.h"
+#include "gutil/strings/substitute.h"
 #include "runtime/define_primitive_type.h"
 #include "runtime/large_int_value.h"
 #include "runtime/primitive_type.h"
 #include "runtime/runtime_filter_mgr.h"
+#include "util/bitmap_value.h"
 #include "util/runtime_profile.h"
 #include "util/string_parser.hpp"
 #include "vec/columns/column.h"
 #include "vec/columns/column_complex.h"
+#include "vec/common/assert_cast.h"
 #include "vec/exprs/vbitmap_predicate.h"
 #include "vec/exprs/vbloom_predicate.h"
 #include "vec/exprs/vdirect_in_predicate.h"
 #include "vec/exprs/vexpr.h"
+#include "vec/exprs/vexpr_context.h"
 #include "vec/exprs/vliteral.h"
 #include "vec/exprs/vruntimefilter_wrapper.h"
 #include "vec/runtime/shared_hash_table_controller.h"
diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h
index 5050d81c41..141df632c5 100644
--- a/be/src/exprs/runtime_filter.h
+++ b/be/src/exprs/runtime_filter.h
@@ -17,12 +17,31 @@
 
 #pragma once
 
+#include <fmt/format.h>
+#include <gen_cpp/Exprs_types.h>
+#include <stdint.h>
+
+#include <atomic>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "common/status.h"
+#include "runtime/datetime_value.h"
+#include "runtime/decimalv2_value.h"
+#include "runtime/define_primitive_type.h"
 #include "runtime/large_int_value.h"
+#include "runtime/primitive_type.h"
 #include "runtime/runtime_state.h"
+#include "runtime/types.h"
 #include "util/lock.h"
 #include "util/runtime_profile.h"
 #include "util/time.h"
 #include "util/uid_util.h"
+#include "vec/common/string_ref.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type.h"
+#include "vec/runtime/vdatetime_value.h"
 
 namespace butil {
 class IOBufAsZeroCopyInputStream;
@@ -42,6 +61,8 @@ class HashJoinNode;
 class RuntimeProfile;
 class BloomFilterFuncBase;
 class BitmapFilterFuncBase;
+class TNetworkAddress;
+class TQueryOptions;
 
 namespace vectorized {
 class VExpr;
@@ -339,6 +360,7 @@ protected:
     std::vector<doris::vectorized::VExpr*> _push_down_vexprs;
 
     struct rpc_context;
+
     std::shared_ptr<rpc_context> _rpc_context;
 
     // parent profile
diff --git a/be/src/exprs/runtime_filter_rpc.cpp b/be/src/exprs/runtime_filter_rpc.cpp
index faaee410d0..8a5c4acb9b 100644
--- a/be/src/exprs/runtime_filter_rpc.cpp
+++ b/be/src/exprs/runtime_filter_rpc.cpp
@@ -15,14 +15,25 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <brpc/controller.h>
+#include <butil/iobuf.h>
+#include <fmt/format.h>
+#include <gen_cpp/Types_types.h>
+#include <gen_cpp/types.pb.h>
+
+#include <memory>
+#include <ostream>
+#include <string>
+
 #include "common/config.h"
 #include "common/status.h"
 #include "exprs/runtime_filter.h"
 #include "runtime/exec_env.h"
 #include "runtime/runtime_state.h"
-
 // for rpc
-#include "gen_cpp/internal_service.pb.h"
+#include <gen_cpp/internal_service.pb.h>
+
+#include "common/logging.h"
 #include "util/brpc_client_cache.h"
 
 namespace doris {
diff --git a/be/src/exprs/string_functions.cpp b/be/src/exprs/string_functions.cpp
index ac80e21b9b..ce5738dba9 100644
--- a/be/src/exprs/string_functions.cpp
+++ b/be/src/exprs/string_functions.cpp
@@ -21,12 +21,9 @@
 #include "exprs/string_functions.h"
 
 #include <re2/re2.h>
+#include <re2/stringpiece.h>
 
-#include <algorithm>
-
-#include "math_functions.h"
-#include "util/simd/vstring_function.h"
-#include "util/url_parser.h"
+#include <sstream>
 
 // NOTE: be careful not to use string::append.  It is not performant.
 namespace doris {
diff --git a/be/src/exprs/string_functions.h b/be/src/exprs/string_functions.h
index e8527eb614..5a64790032 100644
--- a/be/src/exprs/string_functions.h
+++ b/be/src/exprs/string_functions.h
@@ -24,7 +24,9 @@
 
 #include <iomanip>
 #include <locale>
+#include <memory>
 #include <sstream>
+#include <string>
 #include <string_view>
 
 #include "gutil/strings/numbers.h"
diff --git a/be/src/geo/ByteOrderDataInStream.h b/be/src/geo/ByteOrderDataInStream.h
index 2980fc4b43..f42d685808 100644
--- a/be/src/geo/ByteOrderDataInStream.h
+++ b/be/src/geo/ByteOrderDataInStream.h
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
+
 #include <cstddef>
 
 #include "ByteOrderValues.h"
diff --git a/be/src/geo/ByteOrderValues.cpp b/be/src/geo/ByteOrderValues.cpp
index ba05bb647d..1e78322eaa 100644
--- a/be/src/geo/ByteOrderValues.cpp
+++ b/be/src/geo/ByteOrderValues.cpp
@@ -17,7 +17,8 @@
 
 #include "ByteOrderValues.h"
 
-#include <cassert>
+#include <stdint.h>
+
 #include <cstring>
 
 namespace doris {
diff --git a/be/src/geo/ByteOrderValues.h b/be/src/geo/ByteOrderValues.h
index 79a0bbde8a..c78f6c8856 100644
--- a/be/src/geo/ByteOrderValues.h
+++ b/be/src/geo/ByteOrderValues.h
@@ -15,7 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
+
+#include <sys/types.h>
+
 #include <ctime>
+
 using int64_t = __int64_t;
 using int32_t = __int32_t;
 using uint32_t = __uint32_t;
diff --git a/be/src/geo/geo_tobinary.cpp b/be/src/geo/geo_tobinary.cpp
index f3980075c0..12b36dd3a8 100644
--- a/be/src/geo/geo_tobinary.cpp
+++ b/be/src/geo/geo_tobinary.cpp
@@ -17,10 +17,16 @@
 
 #include "geo_tobinary.h"
 
+#include <cstddef>
+#include <sstream>
+#include <vector>
+
+#include "geo/ByteOrderValues.h"
+#include "geo/geo_common.h"
+#include "geo/machine.h"
+#include "geo/wkt_parse_type.h"
 #include "geo_tobinary_type.h"
 #include "geo_types.h"
-#include "sstream"
-#include "wkb_parse.h"
 
 namespace doris {
 
diff --git a/be/src/geo/geo_tobinary.h b/be/src/geo/geo_tobinary.h
index ee77bea88e..8b861a519a 100644
--- a/be/src/geo/geo_tobinary.h
+++ b/be/src/geo/geo_tobinary.h
@@ -15,18 +15,26 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
+
 //#include "geo_types.h"
 
+#include <string>
+
 #include "geo/geo_common.h"
 #include "geo/geo_tobinary_type.h"
 #include "geo/wkt_parse_type.h"
 
+struct ToBinaryContext;
+
 namespace doris {
 
 class GeoShape;
 class GeoPoint;
 class GeoLine;
 class GeoPolygon;
+struct GeoCoordinate;
+struct GeoCoordinateList;
 
 class toBinary {
 public:
diff --git a/be/src/geo/geo_types.cpp b/be/src/geo/geo_types.cpp
index 10d7f5e0a1..672e594273 100644
--- a/be/src/geo/geo_types.cpp
+++ b/be/src/geo/geo_types.cpp
@@ -17,19 +17,25 @@
 
 #include "geo/geo_types.h"
 
+#include <absl/strings/str_format.h>
+#include <glog/logging.h>
+#include <s2/s1angle.h>
 #include <s2/s2cap.h>
-#include <s2/s2cell.h>
 #include <s2/s2earth.h>
 #include <s2/s2latlng.h>
+#include <s2/s2loop.h>
+#include <s2/s2point.h>
 #include <s2/s2polygon.h>
 #include <s2/s2polyline.h>
 #include <s2/util/coding/coder.h>
 #include <s2/util/units/length-units.h>
-#include <stdio.h>
-
+#include <string.h>
+// IWYU pragma: no_include <bits/std_abs.h>
+#include <cmath>
 #include <iomanip>
 #include <sstream>
-#include <type_traits>
+#include <utility>
+#include <vector>
 
 #include "geo/geo_tobinary.h"
 #include "geo/wkb_parse.h"
diff --git a/be/src/geo/geo_types.h b/be/src/geo/geo_types.h
index 055d5463ae..7416dccb35 100644
--- a/be/src/geo/geo_types.h
+++ b/be/src/geo/geo_types.h
@@ -17,6 +17,8 @@
 
 #pragma once
 
+#include <stddef.h>
+
 #include <memory>
 #include <string>
 #include <vector>
@@ -28,7 +30,6 @@ class S2Polyline;
 class S2Polygon;
 class S2Cap;
 class S2Loop;
-
 template <typename T>
 class Vector3;
 
diff --git a/be/src/geo/machine.h b/be/src/geo/machine.h
index db2a67e14e..f40f7b9529 100644
--- a/be/src/geo/machine.h
+++ b/be/src/geo/machine.h
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
+
 namespace doris {
 /**
  * Check endianness of current machine.
diff --git a/be/src/geo/wkb_parse.cpp b/be/src/geo/wkb_parse.cpp
index 5510fa5250..f792dd9a25 100644
--- a/be/src/geo/wkb_parse.cpp
+++ b/be/src/geo/wkb_parse.cpp
@@ -17,10 +17,16 @@
 
 #include "wkb_parse.h"
 
-#include <string.h>
-
+#include <array>
+#include <cstddef>
+#include <istream>
+#include <vector>
+
+#include "geo/ByteOrderDataInStream.h"
+#include "geo/ByteOrderValues.h"
+#include "geo/geo_types.h"
+#include "geo/wkb_parse_ctx.h"
 #include "geo_tobinary_type.h"
-#include "sstream"
 
 namespace doris {
 
diff --git a/be/src/geo/wkb_parse.h b/be/src/geo/wkb_parse.h
index 312459e0f1..c27537966b 100644
--- a/be/src/geo/wkb_parse.h
+++ b/be/src/geo/wkb_parse.h
@@ -15,13 +15,25 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
+
+#include <stdint.h>
+
+#include <iosfwd>
+
 #include "geo/geo_common.h"
 #include "geo/geo_types.h"
+#include "geo/wkt_parse_type.h"
 #include "wkb_parse_ctx.h"
 
+struct WkbParseContext;
+
 namespace doris {
 
 class GeoShape;
+class GeoLine;
+class GeoPoint;
+class GeoPolygon;
 
 class WkbParse {
 public:
diff --git a/be/src/geo/wkt_parse.cpp b/be/src/geo/wkt_parse.cpp
index 1e69f7036a..5dd8778332 100644
--- a/be/src/geo/wkt_parse.cpp
+++ b/be/src/geo/wkt_parse.cpp
@@ -17,9 +17,10 @@
 
 #include "geo/wkt_parse.h"
 
-#include "geo/geo_types.h"
 #include "geo/wkt_parse_ctx.h"
+#include "geo/wkt_parse_type.h" // IWYU pragma: keep
 #include "geo/wkt_yacc.y.hpp"
+
 #define YYSTYPE WKT_STYPE
 #define YY_EXTRA_TYPE WktParseContext*
 #include "geo/wkt_lex.l.h"
diff --git a/be/src/geo/wkt_parse.h b/be/src/geo/wkt_parse.h
index a121944aef..0bb999af73 100644
--- a/be/src/geo/wkt_parse.h
+++ b/be/src/geo/wkt_parse.h
@@ -15,6 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
+
+#include <stddef.h>
+
 #include <memory>
 
 #include "geo/geo_common.h"
diff --git a/be/src/http/action/check_rpc_channel_action.cpp b/be/src/http/action/check_rpc_channel_action.cpp
index 6a688e836d..d483fc4fa3 100644
--- a/be/src/http/action/check_rpc_channel_action.cpp
+++ b/be/src/http/action/check_rpc_channel_action.cpp
@@ -17,13 +17,21 @@
 
 #include "http/action/check_rpc_channel_action.h"
 
-#include <fmt/core.h>
+#include <brpc/controller.h>
+#include <fmt/format.h>
+#include <gen_cpp/internal_service.pb.h>
+#include <gen_cpp/types.pb.h>
+#include <glog/logging.h>
+#include <stdint.h>
+
+#include <exception>
+#include <memory>
+#include <string>
 
-#include "gen_cpp/internal_service.pb.h"
 #include "http/http_channel.h"
 #include "http/http_request.h"
+#include "http/http_status.h"
 #include "runtime/exec_env.h"
-#include "service/brpc.h"
 #include "util/brpc_client_cache.h"
 #include "util/md5.h"
 
diff --git a/be/src/http/action/check_rpc_channel_action.h b/be/src/http/action/check_rpc_channel_action.h
index d8b456ca8b..6847c51b08 100644
--- a/be/src/http/action/check_rpc_channel_action.h
+++ b/be/src/http/action/check_rpc_channel_action.h
@@ -21,6 +21,8 @@
 
 namespace doris {
 class ExecEnv;
+class HttpRequest;
+
 class CheckRPCChannelAction : public HttpHandler {
 public:
     explicit CheckRPCChannelAction(ExecEnv* exec_env);
diff --git a/be/src/http/action/check_tablet_segment_action.cpp b/be/src/http/action/check_tablet_segment_action.cpp
index 6e822ace7d..4b5362a587 100644
--- a/be/src/http/action/check_tablet_segment_action.cpp
+++ b/be/src/http/action/check_tablet_segment_action.cpp
@@ -17,6 +17,11 @@
 
 #include "http/action/check_tablet_segment_action.h"
 
+#include <glog/logging.h>
+#include <stdint.h>
+
+#include <ostream>
+#include <set>
 #include <string>
 
 #include "http/http_channel.h"
@@ -24,7 +29,9 @@
 #include "http/http_request.h"
 #include "http/http_status.h"
 #include "olap/storage_engine.h"
+#include "olap/tablet_manager.h"
 #include "service/backend_options.h"
+#include "util/easy_json.h"
 
 namespace doris {
 
diff --git a/be/src/http/action/check_tablet_segment_action.h b/be/src/http/action/check_tablet_segment_action.h
index 353c372a13..a4c6f422ee 100644
--- a/be/src/http/action/check_tablet_segment_action.h
+++ b/be/src/http/action/check_tablet_segment_action.h
@@ -23,6 +23,7 @@
 #include "util/easy_json.h"
 
 namespace doris {
+class HttpRequest;
 
 class CheckTabletSegmentAction : public HttpHandler {
 public:
diff --git a/be/src/http/action/checksum_action.cpp b/be/src/http/action/checksum_action.cpp
index 3bbf1dd5a6..6ef7215b95 100644
--- a/be/src/http/action/checksum_action.cpp
+++ b/be/src/http/action/checksum_action.cpp
@@ -17,20 +17,17 @@
 
 #include "http/action/checksum_action.h"
 
+#include <boost/lexical_cast/bad_lexical_cast.hpp>
 #include <sstream>
 #include <string>
 
 #include "boost/lexical_cast.hpp"
 #include "common/logging.h"
+#include "common/status.h"
 #include "http/http_channel.h"
-#include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/http_status.h"
-#include "olap/olap_define.h"
-#include "olap/storage_engine.h"
 #include "olap/task/engine_checksum_task.h"
-#include "runtime/exec_env.h"
 
 namespace doris {
 
diff --git a/be/src/http/action/checksum_action.h b/be/src/http/action/checksum_action.h
index 441c302cb1..5afacbfe6d 100644
--- a/be/src/http/action/checksum_action.h
+++ b/be/src/http/action/checksum_action.h
@@ -24,6 +24,7 @@
 namespace doris {
 
 class ExecEnv;
+class HttpRequest;
 
 class ChecksumAction : public HttpHandler {
 public:
diff --git a/be/src/http/action/compaction_action.cpp b/be/src/http/action/compaction_action.cpp
index 86d2548946..96d450c599 100644
--- a/be/src/http/action/compaction_action.cpp
+++ b/be/src/http/action/compaction_action.cpp
@@ -17,11 +17,16 @@
 
 #include "http/action/compaction_action.h"
 
-#include <sys/syscall.h>
-
+// IWYU pragma: no_include <bits/chrono.h>
+#include <chrono> // IWYU pragma: keep
+#include <exception>
 #include <future>
+#include <memory>
+#include <mutex>
 #include <sstream>
 #include <string>
+#include <thread>
+#include <utility>
 
 #include "common/logging.h"
 #include "gutil/strings/substitute.h"
@@ -31,8 +36,12 @@
 #include "http/http_status.h"
 #include "olap/base_compaction.h"
 #include "olap/cumulative_compaction.h"
+#include "olap/cumulative_compaction_policy.h"
 #include "olap/olap_define.h"
 #include "olap/storage_engine.h"
+#include "olap/tablet_manager.h"
+#include "util/doris_metrics.h"
+#include "util/stopwatch.hpp"
 
 namespace doris {
 using namespace ErrorCode;
diff --git a/be/src/http/action/compaction_action.h b/be/src/http/action/compaction_action.h
index 6def386a44..8fc487d4a0 100644
--- a/be/src/http/action/compaction_action.h
+++ b/be/src/http/action/compaction_action.h
@@ -17,11 +17,16 @@
 
 #pragma once
 
+#include <stdint.h>
+
+#include <string>
+
 #include "common/status.h"
 #include "http/http_handler.h"
 #include "olap/tablet.h"
 
 namespace doris {
+class HttpRequest;
 
 enum class CompactionActionType {
     SHOW_INFO = 1,
diff --git a/be/src/http/action/config_action.cpp b/be/src/http/action/config_action.cpp
index 9c95276db3..60cc21e36b 100644
--- a/be/src/http/action/config_action.cpp
+++ b/be/src/http/action/config_action.cpp
@@ -18,11 +18,16 @@
 #include "http/action/config_action.h"
 
 #include <rapidjson/document.h>
+#include <rapidjson/encodings.h>
 #include <rapidjson/prettywriter.h>
-#include <rapidjson/rapidjson.h>
 #include <rapidjson/stringbuffer.h>
+#include <rapidjson/writer.h>
 
+#include <map>
+#include <ostream>
 #include <string>
+#include <utility>
+#include <vector>
 
 #include "common/configbase.h"
 #include "common/logging.h"
@@ -31,7 +36,6 @@
 #include "http/http_channel.h"
 #include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/http_status.h"
 
 namespace doris {
diff --git a/be/src/http/action/config_action.h b/be/src/http/action/config_action.h
index ca20f3862f..7d9d3e7f2e 100644
--- a/be/src/http/action/config_action.h
+++ b/be/src/http/action/config_action.h
@@ -20,6 +20,7 @@
 #include "http/http_handler.h"
 
 namespace doris {
+class HttpRequest;
 
 enum ConfigActionType {
     UPDATE_CONFIG = 1,
diff --git a/be/src/http/action/download_action.cpp b/be/src/http/action/download_action.cpp
index e56b24c61f..074baf5ec1 100644
--- a/be/src/http/action/download_action.cpp
+++ b/be/src/http/action/download_action.cpp
@@ -17,21 +17,19 @@
 
 #include "http/action/download_action.h"
 
-#include <sys/types.h>
-#include <unistd.h>
-
+#include <algorithm>
+#include <memory>
 #include <sstream>
 #include <string>
+#include <utility>
 
+#include "common/config.h"
+#include "common/logging.h"
 #include "http/http_channel.h"
-#include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
-#include "http/http_status.h"
 #include "http/utils.h"
 #include "io/fs/local_file_system.h"
 #include "runtime/exec_env.h"
-#include "util/path_util.h"
 
 namespace doris {
 
diff --git a/be/src/http/action/download_action.h b/be/src/http/action/download_action.h
index 944c422fc6..86bee3e2b1 100644
--- a/be/src/http/action/download_action.h
+++ b/be/src/http/action/download_action.h
@@ -17,6 +17,10 @@
 
 #pragma once
 
+#include <string>
+#include <vector>
+
+#include "common/status.h"
 #include "exec/scan_node.h"
 #include "http/http_handler.h"
 #include "runtime/descriptors.h"
@@ -24,6 +28,7 @@
 namespace doris {
 
 class ExecEnv;
+class HttpRequest;
 
 // A simple handler that serves incoming HTTP requests of file-download to send their respective HTTP responses.
 //
diff --git a/be/src/http/action/health_action.cpp b/be/src/http/action/health_action.cpp
index 163c782b5f..ef937ab27c 100644
--- a/be/src/http/action/health_action.cpp
+++ b/be/src/http/action/health_action.cpp
@@ -23,7 +23,6 @@
 #include "http/http_channel.h"
 #include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/http_status.h"
 
 namespace doris {
diff --git a/be/src/http/action/health_action.h b/be/src/http/action/health_action.h
index 68d46fa325..cdee0f12b4 100644
--- a/be/src/http/action/health_action.h
+++ b/be/src/http/action/health_action.h
@@ -22,6 +22,7 @@
 namespace doris {
 
 class ExecEnv;
+class HttpRequest;
 
 // Get BE health state from http API.
 class HealthAction : public HttpHandler {
diff --git a/be/src/http/action/jeprofile_actions.cpp b/be/src/http/action/jeprofile_actions.cpp
index c014ef4eba..4faac5b51a 100644
--- a/be/src/http/action/jeprofile_actions.cpp
+++ b/be/src/http/action/jeprofile_actions.cpp
@@ -18,22 +18,25 @@
 #include "http/action/jeprofile_actions.h"
 
 #include <jemalloc/jemalloc.h>
+#include <stdlib.h>
+#include <unistd.h>
 
 #include <ctime>
 #include <fstream>
+#include <memory>
 #include <mutex>
-#include <sstream>
+#include <string>
 
 #include "common/config.h"
 #include "common/object_pool.h"
 #include "http/ev_http_server.h"
 #include "http/http_channel.h"
 #include "http/http_handler.h"
-#include "http/http_headers.h"
-#include "http/http_request.h"
+#include "http/http_method.h"
 #include "io/fs/local_file_system.h"
 
 namespace doris {
+class HttpRequest;
 
 static std::mutex kJeprofileActionMutex;
 class JeHeapAction : public HttpHandler {
diff --git a/be/src/http/action/meta_action.cpp b/be/src/http/action/meta_action.cpp
index bb8300c8d3..ede286e3d9 100644
--- a/be/src/http/action/meta_action.cpp
+++ b/be/src/http/action/meta_action.cpp
@@ -17,22 +17,27 @@
 
 #include "http/action/meta_action.h"
 
+#include <json2pb/pb_to_json.h>
+#include <stdint.h>
+
+#include <cstring>
+#include <exception>
+#include <memory>
+#include <shared_mutex>
 #include <sstream>
 #include <string>
 
 #include "common/logging.h"
-#include "gutil/strings/substitute.h"
 #include "http/http_channel.h"
 #include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/http_status.h"
 #include "olap/olap_define.h"
 #include "olap/storage_engine.h"
 #include "olap/tablet.h"
+#include "olap/tablet_manager.h"
 #include "olap/tablet_meta.h"
-#include "olap/tablet_meta_manager.h"
-#include "util/json_util.h"
+#include "util/easy_json.h"
 
 namespace doris {
 
diff --git a/be/src/http/action/meta_action.h b/be/src/http/action/meta_action.h
index 2349d7ec22..c4809cd3b5 100644
--- a/be/src/http/action/meta_action.h
+++ b/be/src/http/action/meta_action.h
@@ -17,12 +17,15 @@
 
 #pragma once
 
+#include <string>
+
 #include "common/status.h"
 #include "http/http_handler.h"
 
 namespace doris {
 
 class ExecEnv;
+class HttpRequest;
 
 // Get Meta Info
 class MetaAction : public HttpHandler {
diff --git a/be/src/http/action/metrics_action.cpp b/be/src/http/action/metrics_action.cpp
index f2bd8b43cb..d9df65928a 100644
--- a/be/src/http/action/metrics_action.cpp
+++ b/be/src/http/action/metrics_action.cpp
@@ -17,18 +17,11 @@
 
 #include "http/action/metrics_action.h"
 
-#include <rapidjson/document.h>
-#include <rapidjson/rapidjson.h>
-#include <rapidjson/stringbuffer.h>
-#include <rapidjson/writer.h>
-
 #include <string>
 
 #include "http/http_channel.h"
 #include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
-#include "runtime/exec_env.h"
 #include "util/metrics.h"
 
 namespace doris {
diff --git a/be/src/http/action/monitor_action.cpp b/be/src/http/action/monitor_action.cpp
index cf1604623e..22a1a89cd2 100644
--- a/be/src/http/action/monitor_action.cpp
+++ b/be/src/http/action/monitor_action.cpp
@@ -17,12 +17,13 @@
 
 #include "http/action/monitor_action.h"
 
+#include <glog/logging.h>
+
 #include <sstream>
+#include <utility>
 
-#include "common/status.h"
 #include "http/http_channel.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/http_status.h"
 #include "http/rest_monitor_iface.h"
 
diff --git a/be/src/http/action/pad_rowset_action.cpp b/be/src/http/action/pad_rowset_action.cpp
index c353b4fad7..18e0b2568e 100644
--- a/be/src/http/action/pad_rowset_action.cpp
+++ b/be/src/http/action/pad_rowset_action.cpp
@@ -17,14 +17,27 @@
 
 #include "http/action/pad_rowset_action.h"
 
+#include <gen_cpp/olap_file.pb.h>
+#include <glog/logging.h>
+#include <stdint.h>
+
+#include <cstdlib>
 #include <memory>
 #include <mutex>
+#include <ostream>
+#include <string>
+#include <vector>
 
 #include "http/http_channel.h"
+#include "http/http_request.h"
+#include "http/http_status.h"
 #include "olap/olap_common.h"
-#include "olap/rowset/beta_rowset_writer.h"
 #include "olap/rowset/rowset.h"
+#include "olap/rowset/rowset_writer.h"
+#include "olap/rowset/rowset_writer_context.h"
 #include "olap/storage_engine.h"
+#include "olap/tablet_manager.h"
+#include "util/time.h"
 
 namespace doris {
 
diff --git a/be/src/http/action/pad_rowset_action.h b/be/src/http/action/pad_rowset_action.h
index f6036dc9fa..13090dcd8c 100644
--- a/be/src/http/action/pad_rowset_action.h
+++ b/be/src/http/action/pad_rowset_action.h
@@ -23,6 +23,8 @@
 #include "olap/tablet.h"
 
 namespace doris {
+class HttpRequest;
+struct Version;
 
 class PadRowsetAction : public HttpHandler {
 public:
diff --git a/be/src/http/action/pprof_actions.cpp b/be/src/http/action/pprof_actions.cpp
index 0be0e2c708..7696078a4d 100644
--- a/be/src/http/action/pprof_actions.cpp
+++ b/be/src/http/action/pprof_actions.cpp
@@ -17,29 +17,23 @@
 
 #include "http/action/pprof_actions.h"
 
-#include <gperftools/heap-profiler.h>
-#include <gperftools/malloc_extension.h>
-#include <gperftools/profiler.h>
+#include <stdio.h>
 
 #include <fstream>
-#include <iostream>
+#include <memory>
 #include <mutex>
-#include <sstream>
+#include <string>
 
-#include "agent/utils.h"
 #include "common/config.h"
 #include "common/object_pool.h"
-#include "gutil/strings/substitute.h"
 #include "http/ev_http_server.h"
 #include "http/http_channel.h"
 #include "http/http_handler.h"
-#include "http/http_headers.h"
+#include "http/http_method.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "io/fs/local_file_system.h"
 #include "runtime/exec_env.h"
 #include "util/bfd_parser.h"
-#include "util/pprof_utils.h"
 
 namespace doris {
 
diff --git a/be/src/http/action/reload_tablet_action.cpp b/be/src/http/action/reload_tablet_action.cpp
index ee4d6bd22f..30f65fccce 100644
--- a/be/src/http/action/reload_tablet_action.cpp
+++ b/be/src/http/action/reload_tablet_action.cpp
@@ -17,17 +17,18 @@
 
 #include "http/action/reload_tablet_action.h"
 
+#include <gen_cpp/AgentService_types.h>
+
+#include <boost/lexical_cast/bad_lexical_cast.hpp>
 #include <sstream>
 #include <string>
 
 #include "boost/lexical_cast.hpp"
 #include "common/logging.h"
+#include "common/status.h"
 #include "http/http_channel.h"
-#include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/http_status.h"
-#include "olap/olap_define.h"
 #include "olap/storage_engine.h"
 #include "runtime/exec_env.h"
 
diff --git a/be/src/http/action/reload_tablet_action.h b/be/src/http/action/reload_tablet_action.h
index 5338131224..aa36a474dd 100644
--- a/be/src/http/action/reload_tablet_action.h
+++ b/be/src/http/action/reload_tablet_action.h
@@ -17,12 +17,17 @@
 
 #pragma once
 
-#include "gen_cpp/AgentService_types.h"
+#include <gen_cpp/AgentService_types.h>
+#include <stdint.h>
+
+#include <string>
+
 #include "http/http_handler.h"
 
 namespace doris {
 
 class ExecEnv;
+class HttpRequest;
 
 class ReloadTabletAction : public HttpHandler {
 public:
diff --git a/be/src/http/action/reset_rpc_channel_action.cpp b/be/src/http/action/reset_rpc_channel_action.cpp
index 242bfe7a05..95f9ba9e7e 100644
--- a/be/src/http/action/reset_rpc_channel_action.cpp
+++ b/be/src/http/action/reset_rpc_channel_action.cpp
@@ -17,10 +17,16 @@
 
 #include "http/action/reset_rpc_channel_action.h"
 
-#include <fmt/core.h>
+#include <fmt/format.h>
+#include <glog/logging.h>
+
+#include <algorithm>
+#include <string>
+#include <vector>
 
 #include "http/http_channel.h"
 #include "http/http_request.h"
+#include "http/http_status.h"
 #include "runtime/exec_env.h"
 #include "util/brpc_client_cache.h"
 #include "util/string_util.h"
diff --git a/be/src/http/action/reset_rpc_channel_action.h b/be/src/http/action/reset_rpc_channel_action.h
index 912a324217..52df0d6817 100644
--- a/be/src/http/action/reset_rpc_channel_action.h
+++ b/be/src/http/action/reset_rpc_channel_action.h
@@ -21,6 +21,8 @@
 
 namespace doris {
 class ExecEnv;
+class HttpRequest;
+
 class ResetRPCChannelAction : public HttpHandler {
 public:
     explicit ResetRPCChannelAction(ExecEnv* exec_env);
diff --git a/be/src/http/action/restore_tablet_action.cpp b/be/src/http/action/restore_tablet_action.cpp
index 118008d160..d710a2b36b 100644
--- a/be/src/http/action/restore_tablet_action.cpp
+++ b/be/src/http/action/restore_tablet_action.cpp
@@ -17,24 +17,31 @@
 
 #include "http/action/restore_tablet_action.h"
 
-#include <unistd.h>
+#include <fmt/format.h>
+#include <gen_cpp/AgentService_types.h>
+#include <glog/logging.h>
 
+#include <cstdlib>
+#include <filesystem>
+#include <memory>
 #include <sstream>
 #include <string>
+#include <utility>
+#include <vector>
 
-#include "gutil/strings/substitute.h" // for Substitute
 #include "http/http_channel.h"
-#include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/http_status.h"
+#include "io/fs/file_system.h"
+#include "io/fs/local_file_system.h"
+#include "io/fs/path.h"
 #include "olap/data_dir.h"
-#include "olap/olap_define.h"
 #include "olap/storage_engine.h"
+#include "olap/tablet.h"
+#include "olap/tablet_manager.h"
 #include "olap/tablet_meta.h"
 #include "olap/utils.h"
 #include "runtime/exec_env.h"
-#include "util/json_util.h"
 
 using std::filesystem::path;
 
diff --git a/be/src/http/action/restore_tablet_action.h b/be/src/http/action/restore_tablet_action.h
index 7cae5ff70a..a11d7783b7 100644
--- a/be/src/http/action/restore_tablet_action.h
+++ b/be/src/http/action/restore_tablet_action.h
@@ -17,16 +17,20 @@
 
 #pragma once
 
+#include <gen_cpp/AgentService_types.h>
+#include <stdint.h>
+
 #include <map>
 #include <mutex>
+#include <string>
 
 #include "common/status.h"
-#include "gen_cpp/AgentService_types.h"
 #include "http/http_handler.h"
 
 namespace doris {
 
 class ExecEnv;
+class HttpRequest;
 
 class RestoreTabletAction : public HttpHandler {
 public:
diff --git a/be/src/http/action/snapshot_action.cpp b/be/src/http/action/snapshot_action.cpp
index 4862df5037..19e603fa34 100644
--- a/be/src/http/action/snapshot_action.cpp
+++ b/be/src/http/action/snapshot_action.cpp
@@ -17,21 +17,19 @@
 
 #include "http/action/snapshot_action.h"
 
+#include <gen_cpp/AgentService_types.h>
+
 #include <boost/lexical_cast.hpp>
+#include <boost/lexical_cast/bad_lexical_cast.hpp>
 #include <sstream>
 #include <string>
 
 #include "common/logging.h"
-#include "gen_cpp/AgentService_types.h"
+#include "common/status.h"
 #include "http/http_channel.h"
-#include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/http_status.h"
-#include "olap/olap_define.h"
 #include "olap/snapshot_manager.h"
-#include "olap/storage_engine.h"
-#include "runtime/exec_env.h"
 
 namespace doris {
 
diff --git a/be/src/http/action/snapshot_action.h b/be/src/http/action/snapshot_action.h
index 49285bb336..0d1399604d 100644
--- a/be/src/http/action/snapshot_action.h
+++ b/be/src/http/action/snapshot_action.h
@@ -25,6 +25,7 @@
 namespace doris {
 
 class ExecEnv;
+class HttpRequest;
 
 // make snapshot
 // be_host:be_http_port/api/snapshot?tablet_id=123&schema_hash=456
diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp
index 2520b3a5f4..925a8ad8e0 100644
--- a/be/src/http/action/stream_load.cpp
+++ b/be/src/http/action/stream_load.cpp
@@ -17,45 +17,49 @@
 
 #include "http/action/stream_load.h"
 
-#include <deque>
-#include <future>
-#include <sstream>
-
 // use string iequal
 #include <event2/buffer.h>
-#include <event2/bufferevent.h>
 #include <event2/http.h>
-#include <rapidjson/prettywriter.h>
+#include <gen_cpp/FrontendService.h>
+#include <gen_cpp/FrontendService_types.h>
+#include <gen_cpp/HeartbeatService_types.h>
+#include <gen_cpp/PaloInternalService_types.h>
+#include <gen_cpp/PlanNodes_types.h>
+#include <gen_cpp/Types_types.h>
+#include <stdint.h>
+#include <stdlib.h>
+#include <sys/time.h>
 #include <thrift/protocol/TDebugProtocol.h>
+#include <time.h>
+
+#include <future>
+#include <map>
+#include <sstream>
+#include <stdexcept>
+#include <utility>
 
 #include "common/config.h"
 #include "common/consts.h"
 #include "common/logging.h"
+#include "common/status.h"
 #include "common/utils.h"
-#include "gen_cpp/FrontendService.h"
-#include "gen_cpp/FrontendService_types.h"
-#include "gen_cpp/HeartbeatService_types.h"
 #include "http/http_channel.h"
 #include "http/http_common.h"
 #include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/utils.h"
 #include "io/fs/stream_load_pipe.h"
 #include "olap/storage_engine.h"
 #include "runtime/client_cache.h"
 #include "runtime/exec_env.h"
-#include "runtime/fragment_mgr.h"
 #include "runtime/load_path_mgr.h"
-#include "runtime/plan_fragment_executor.h"
+#include "runtime/message_body_sink.h"
 #include "runtime/stream_load/new_load_stream_mgr.h"
 #include "runtime/stream_load/stream_load_context.h"
 #include "runtime/stream_load/stream_load_executor.h"
 #include "runtime/stream_load/stream_load_recorder.h"
 #include "util/byte_buffer.h"
-#include "util/debug_util.h"
 #include "util/doris_metrics.h"
-#include "util/json_util.h"
 #include "util/metrics.h"
 #include "util/string_util.h"
 #include "util/thrift_rpc_helper.h"
diff --git a/be/src/http/action/stream_load.h b/be/src/http/action/stream_load.h
index 2dc3c762ad..3c44a77bb4 100644
--- a/be/src/http/action/stream_load.h
+++ b/be/src/http/action/stream_load.h
@@ -17,18 +17,23 @@
 
 #pragma once
 
+#include <gen_cpp/PlanNodes_types.h>
+
 #include <functional>
+#include <memory>
+#include <string>
 
-#include "gen_cpp/PlanNodes_types.h"
 #include "http/http_handler.h"
 #include "runtime/client_cache.h"
 #include "runtime/message_body_sink.h"
+#include "util/metrics.h"
 
 namespace doris {
 
 class ExecEnv;
 class Status;
 class StreamLoadContext;
+class HttpRequest;
 
 class StreamLoadAction : public HttpHandler {
 public:
diff --git a/be/src/http/action/stream_load_2pc.cpp b/be/src/http/action/stream_load_2pc.cpp
index 3a5690cb8f..ce3a8961b2 100644
--- a/be/src/http/action/stream_load_2pc.cpp
+++ b/be/src/http/action/stream_load_2pc.cpp
@@ -17,18 +17,25 @@
 
 #include "http/action/stream_load_2pc.h"
 
+#include <glog/logging.h>
+#include <rapidjson/encodings.h>
 #include <rapidjson/prettywriter.h>
 #include <rapidjson/stringbuffer.h>
 
+#include <exception>
+#include <memory>
+#include <new>
+#include <ostream>
+
 #include "common/status.h"
 #include "http/http_channel.h"
-#include "http/http_headers.h"
+#include "http/http_common.h"
 #include "http/http_request.h"
 #include "http/http_status.h"
 #include "http/utils.h"
+#include "runtime/exec_env.h"
 #include "runtime/stream_load/stream_load_context.h"
 #include "runtime/stream_load/stream_load_executor.h"
-#include "util/json_util.h"
 
 namespace doris {
 
diff --git a/be/src/http/action/stream_load_2pc.h b/be/src/http/action/stream_load_2pc.h
index 1a1013a12b..fdf39873e2 100644
--- a/be/src/http/action/stream_load_2pc.h
+++ b/be/src/http/action/stream_load_2pc.h
@@ -24,6 +24,7 @@
 namespace doris {
 
 class ExecEnv;
+class HttpRequest;
 
 class StreamLoad2PCAction : public HttpHandler {
 public:
diff --git a/be/src/http/action/tablet_migration_action.cpp b/be/src/http/action/tablet_migration_action.cpp
index 6cb05d13a0..3f0fbed297 100644
--- a/be/src/http/action/tablet_migration_action.cpp
+++ b/be/src/http/action/tablet_migration_action.cpp
@@ -17,17 +17,20 @@
 
 #include "http/action/tablet_migration_action.h"
 
+#include <glog/logging.h>
+
+#include <exception>
 #include <string>
 
-#include "gutil/strings/substitute.h"
+#include "common/config.h"
 #include "http/http_channel.h"
 #include "http/http_headers.h"
 #include "http/http_request.h"
 #include "http/http_status.h"
+#include "olap/data_dir.h"
 #include "olap/storage_engine.h"
 #include "olap/tablet_manager.h"
 #include "olap/task/engine_storage_migration_task.h"
-#include "util/json_util.h"
 
 namespace doris {
 
diff --git a/be/src/http/action/tablet_migration_action.h b/be/src/http/action/tablet_migration_action.h
index 4e5a92b877..aee9a65d12 100644
--- a/be/src/http/action/tablet_migration_action.h
+++ b/be/src/http/action/tablet_migration_action.h
@@ -17,16 +17,28 @@
 
 #pragma once
 
+#include <gen_cpp/Status_types.h>
+#include <stdint.h>
+
+#include <deque>
+#include <map>
+#include <memory>
+#include <mutex>
+#include <ostream>
 #include <string>
+#include <utility>
 
 #include "common/status.h"
-#include "gen_cpp/Status_types.h"
+#include "gutil/stringprintf.h"
+#include "gutil/strings/numbers.h"
 #include "http/http_handler.h"
 #include "olap/data_dir.h"
 #include "olap/tablet.h"
 #include "util/threadpool.h"
 
 namespace doris {
+class DataDir;
+class HttpRequest;
 
 // Migrate a tablet from a disk to another.
 class TabletMigrationAction : public HttpHandler {
diff --git a/be/src/http/action/tablets_distribution_action.cpp b/be/src/http/action/tablets_distribution_action.cpp
index 6bb7b5ab6f..605f900b17 100644
--- a/be/src/http/action/tablets_distribution_action.cpp
+++ b/be/src/http/action/tablets_distribution_action.cpp
@@ -17,7 +17,14 @@
 
 #include "http/action/tablets_distribution_action.h"
 
+#include <glog/logging.h>
+
+#include <exception>
+#include <map>
+#include <ostream>
 #include <string>
+#include <utility>
+#include <vector>
 
 #include "common/status.h"
 #include "gutil/strings/substitute.h"
@@ -25,10 +32,11 @@
 #include "http/http_headers.h"
 #include "http/http_request.h"
 #include "http/http_status.h"
+#include "olap/data_dir.h"
+#include "olap/olap_common.h"
 #include "olap/storage_engine.h"
 #include "olap/tablet_manager.h"
 #include "service/backend_options.h"
-#include "util/json_util.h"
 
 namespace doris {
 
diff --git a/be/src/http/action/tablets_distribution_action.h b/be/src/http/action/tablets_distribution_action.h
index 218882d6c2..0e400f9feb 100644
--- a/be/src/http/action/tablets_distribution_action.h
+++ b/be/src/http/action/tablets_distribution_action.h
@@ -17,12 +17,15 @@
 
 #pragma once
 
+#include <stdint.h>
+
 #include <string>
 
 #include "http/http_handler.h"
 #include "util/easy_json.h"
 
 namespace doris {
+class HttpRequest;
 
 // Get BE tablets distribution info from http API.
 class TabletsDistributionAction : public HttpHandler {
diff --git a/be/src/http/action/tablets_info_action.cpp b/be/src/http/action/tablets_info_action.cpp
index 56ccbc09a8..656fbdf158 100644
--- a/be/src/http/action/tablets_info_action.cpp
+++ b/be/src/http/action/tablets_info_action.cpp
@@ -17,12 +17,21 @@
 
 #include "http/action/tablets_info_action.h"
 
+#include <ctype.h>
+
+#include <algorithm>
+#include <cstdint>
+#include <cstdlib>
+#include <limits>
 #include <string>
+#include <vector>
 
+#include "gutil/strings/numbers.h"
 #include "http/http_channel.h"
 #include "http/http_headers.h"
 #include "http/http_request.h"
 #include "http/http_status.h"
+#include "olap/olap_common.h"
 #include "olap/storage_engine.h"
 #include "olap/tablet_manager.h"
 #include "service/backend_options.h"
diff --git a/be/src/http/action/tablets_info_action.h b/be/src/http/action/tablets_info_action.h
index f21045bfae..22f8634faf 100644
--- a/be/src/http/action/tablets_info_action.h
+++ b/be/src/http/action/tablets_info_action.h
@@ -23,6 +23,7 @@
 #include "util/easy_json.h"
 
 namespace doris {
+class HttpRequest;
 
 // Get BE tablets info from http API.
 class TabletsInfoAction : public HttpHandler {
diff --git a/be/src/http/action/version_action.cpp b/be/src/http/action/version_action.cpp
index c9cb908522..18b05ed3c4 100644
--- a/be/src/http/action/version_action.cpp
+++ b/be/src/http/action/version_action.cpp
@@ -17,14 +17,13 @@
 
 #include "http/action/version_action.h"
 
-#include <sstream>
+#include <gen_cpp/version.h>
+
 #include <string>
 
-#include "gen_cpp/version.h"
 #include "http/http_channel.h"
 #include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/http_status.h"
 #include "util/easy_json.h"
 
diff --git a/be/src/http/action/version_action.h b/be/src/http/action/version_action.h
index 7695ad9fe4..e78e19f37c 100644
--- a/be/src/http/action/version_action.h
+++ b/be/src/http/action/version_action.h
@@ -23,6 +23,7 @@
 namespace doris {
 
 class ExecEnv;
+class HttpRequest;
 
 // Get BE version info from http API.
 class VersionAction : public HttpHandler {
diff --git a/be/src/http/default_path_handlers.cpp b/be/src/http/default_path_handlers.cpp
index b4c837f3d8..539268f693 100644
--- a/be/src/http/default_path_handlers.cpp
+++ b/be/src/http/default_path_handlers.cpp
@@ -17,24 +17,32 @@
 
 #include "http/default_path_handlers.h"
 
-#include <gperftools/heap-profiler.h>
+#include <gen_cpp/Metrics_types.h>
+
+#include <boost/algorithm/string/replace.hpp>
 #ifdef USE_JEMALLOC
 #include "jemalloc/jemalloc.h"
 #else
 #include <gperftools/malloc_extension.h>
 #endif
 
-#include <boost/algorithm/string.hpp>
+#include <functional>
+#include <map>
+#include <memory>
+#include <mutex>
 #include <sstream>
+#include <string>
+#include <utility>
+#include <vector>
 
-#include "agent/utils.h"
-#include "common/configbase.h"
+#include "common/config.h"
 #include "gutil/strings/numbers.h"
 #include "gutil/strings/substitute.h"
 #include "http/action/tablets_info_action.h"
 #include "http/web_page_handler.h"
+#include "runtime/memory/mem_tracker.h"
 #include "runtime/memory/mem_tracker_limiter.h"
-#include "util/debug_util.h"
+#include "util/easy_json.h"
 #include "util/mem_info.h"
 #include "util/perf_counters.h"
 #include "util/pretty_printer.h"
diff --git a/be/src/http/ev_http_server.cpp b/be/src/http/ev_http_server.cpp
index bef87541a3..ccea181a8f 100644
--- a/be/src/http/ev_http_server.cpp
+++ b/be/src/http/ev_http_server.cpp
@@ -17,14 +17,20 @@
 
 #include "http/ev_http_server.h"
 
-#include <event2/buffer.h>
-#include <event2/bufferevent.h>
+#include <arpa/inet.h>
+#include <bthread/errno.h>
+#include <butil/endpoint.h>
+#include <butil/fd_utility.h>
 #include <event2/event.h>
 #include <event2/http.h>
 #include <event2/http_struct.h>
-#include <event2/keyvalq_struct.h>
 #include <event2/thread.h>
+#include <netinet/in.h>
+#include <string.h>
+#include <sys/socket.h>
+#include <unistd.h>
 
+#include <algorithm>
 #include <memory>
 #include <sstream>
 
@@ -33,12 +39,13 @@
 #include "http/http_handler.h"
 #include "http/http_headers.h"
 #include "http/http_request.h"
-#include "runtime/thread_context.h"
+#include "http/http_status.h"
 #include "service/backend_options.h"
-#include "service/brpc.h"
-#include "util/debug_util.h"
 #include "util/threadpool.h"
 
+struct event_base;
+struct evhttp;
+
 namespace doris {
 
 static void on_chunked(struct evhttp_request* ev_req, void* param) {
diff --git a/be/src/http/ev_http_server.h b/be/src/http/ev_http_server.h
index 7263b2f3ef..5ba634b219 100644
--- a/be/src/http/ev_http_server.h
+++ b/be/src/http/ev_http_server.h
@@ -17,6 +17,7 @@
 
 #pragma once
 
+#include <memory>
 #include <mutex>
 #include <string>
 #include <thread>
@@ -26,6 +27,8 @@
 #include "http/http_method.h"
 #include "util/path_trie.hpp"
 
+struct event_base;
+
 namespace doris {
 
 class HttpHandler;
diff --git a/be/src/http/http_channel.cpp b/be/src/http/http_channel.cpp
index 4a06a4e7f5..5727ba3902 100644
--- a/be/src/http/http_channel.cpp
+++ b/be/src/http/http_channel.cpp
@@ -20,16 +20,19 @@
 #include <event2/buffer.h>
 #include <event2/http.h>
 
-#include <mutex>
+#include <algorithm>
 #include <sstream>
 #include <string>
+#include <vector>
 
 #include "common/logging.h"
+#include "common/status.h"
 #include "gutil/strings/split.h"
+#include "gutil/strings/strip.h"
 #include "http/http_headers.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/http_status.h"
+#include "util/slice.h"
 #include "util/zlib.h"
 
 namespace doris {
diff --git a/be/src/http/http_channel.h b/be/src/http/http_channel.h
index 83b696a409..a12eddb535 100644
--- a/be/src/http/http_channel.h
+++ b/be/src/http/http_channel.h
@@ -17,6 +17,8 @@
 
 #pragma once
 
+#include <stddef.h>
+
 #include <cstdint>
 #include <string>
 
diff --git a/be/src/http/http_client.cpp b/be/src/http/http_client.cpp
index 0cb5b97ec4..0919d21d38 100644
--- a/be/src/http/http_client.cpp
+++ b/be/src/http/http_client.cpp
@@ -17,6 +17,12 @@
 
 #include "http/http_client.h"
 
+#include <glog/logging.h>
+#include <unistd.h>
+
+#include <memory>
+#include <ostream>
+
 #include "common/config.h"
 #include "util/stack_util.h"
 
diff --git a/be/src/http/http_client.h b/be/src/http/http_client.h
index b9605afb0f..7ec714587a 100644
--- a/be/src/http/http_client.h
+++ b/be/src/http/http_client.h
@@ -18,8 +18,11 @@
 #pragma once
 
 #include <curl/curl.h>
+#include <curl/system.h>
+#include <stdint.h>
 
 #include <cstdio>
+#include <functional>
 #include <string>
 
 #include "common/status.h"
diff --git a/be/src/http/http_method.cpp b/be/src/http/http_method.cpp
index c7a95905b1..9986f3c653 100644
--- a/be/src/http/http_method.cpp
+++ b/be/src/http/http_method.cpp
@@ -19,6 +19,7 @@
 
 #include <map>
 #include <string>
+#include <utility>
 
 namespace doris {
 
diff --git a/be/src/http/http_request.cpp b/be/src/http/http_request.cpp
index c3228f7a58..14bde591b4 100644
--- a/be/src/http/http_request.cpp
+++ b/be/src/http/http_request.cpp
@@ -18,19 +18,16 @@
 #include "http/http_request.h"
 
 #include <event2/buffer.h>
-#include <event2/bufferevent.h>
 #include <event2/http.h>
 #include <event2/http_struct.h>
 #include <event2/keyvalq_struct.h>
 
-#include <boost/algorithm/string.hpp>
 #include <sstream>
 #include <string>
-#include <vector>
+#include <unordered_map>
+#include <utility>
 
-#include "common/logging.h"
 #include "http/http_handler.h"
-#include "util/url_coding.h"
 
 namespace doris {
 
diff --git a/be/src/http/http_status.cpp b/be/src/http/http_status.cpp
index 415be0b835..b716ff269f 100644
--- a/be/src/http/http_status.cpp
+++ b/be/src/http/http_status.cpp
@@ -19,6 +19,7 @@
 
 #include <map>
 #include <string>
+#include <utility>
 
 namespace doris {
 
diff --git a/be/src/http/utils.cpp b/be/src/http/utils.cpp
index ec793b9c30..fe11738d5a 100644
--- a/be/src/http/utils.cpp
+++ b/be/src/http/utils.cpp
@@ -18,15 +18,23 @@
 #include "http/utils.h"
 
 #include <fcntl.h>
+#include <stdint.h>
 #include <sys/stat.h>
+#include <unistd.h>
+
+#include <memory>
+#include <ostream>
+#include <vector>
 
 #include "common/logging.h"
 #include "common/status.h"
 #include "common/utils.h"
 #include "http/http_channel.h"
-#include "http/http_common.h"
 #include "http/http_headers.h"
+#include "http/http_method.h"
 #include "http/http_request.h"
+#include "http/http_status.h"
+#include "io/fs/file_system.h"
 #include "io/fs/local_file_system.h"
 #include "util/path_util.h"
 #include "util/url_coding.h"
diff --git a/be/src/http/utils.h b/be/src/http/utils.h
index a120c67cbf..f9b09b3017 100644
--- a/be/src/http/utils.h
+++ b/be/src/http/utils.h
@@ -26,6 +26,7 @@
 namespace doris {
 
 class HttpRequest;
+struct AuthInfo;
 
 std::string encode_basic_auth(const std::string& user, const std::string& passwd);
 // parse Basic authorization
diff --git a/be/src/http/web_page_handler.cpp b/be/src/http/web_page_handler.cpp
index 20e6a00185..c6def7ff0f 100644
--- a/be/src/http/web_page_handler.cpp
+++ b/be/src/http/web_page_handler.cpp
@@ -17,22 +17,28 @@
 
 #include "http/web_page_handler.h"
 
+#include <stdlib.h>
+
 #include <functional>
+#include <memory>
 
-#include "common/config.h"
+#include "common/logging.h"
+#include "common/status.h"
 #include "gutil/stl_util.h"
+#include "gutil/strings/numbers.h"
 #include "gutil/strings/substitute.h"
 #include "http/ev_http_server.h"
 #include "http/http_channel.h"
 #include "http/http_headers.h"
+#include "http/http_method.h"
 #include "http/http_request.h"
-#include "http/http_response.h"
 #include "http/http_status.h"
 #include "http/utils.h"
 #include "io/fs/local_file_system.h"
 #include "util/cpu_info.h"
 #include "util/debug_util.h"
 #include "util/disk_info.h"
+#include "util/easy_json.h"
 #include "util/mem_info.h"
 #include "util/mustache/mustache.h"
 
diff --git a/be/src/http/web_page_handler.h b/be/src/http/web_page_handler.h
index e55197a716..9ab304d96f 100644
--- a/be/src/http/web_page_handler.h
+++ b/be/src/http/web_page_handler.h
@@ -22,6 +22,7 @@
 #include <mutex>
 #include <sstream>
 #include <string>
+#include <utility>
 #include <vector>
 
 #include "http/http_handler.h"
@@ -30,6 +31,8 @@
 namespace doris {
 
 class EvHttpServer;
+class EasyJson;
+class HttpRequest;
 
 // This a handler for webpage request
 // and this handler manage all the page handler
diff --git a/be/src/io/fs/file_reader_writer_fwd.h b/be/src/io/fs/file_reader_writer_fwd.h
index a0e3078095..e63c5395de 100644
--- a/be/src/io/fs/file_reader_writer_fwd.h
+++ b/be/src/io/fs/file_reader_writer_fwd.h
@@ -19,6 +19,9 @@
 // and modified by Doris
 
 #pragma once
+
+#include <memory>
+
 namespace doris {
 namespace io {
 
diff --git a/be/src/io/fs/hdfs.h b/be/src/io/fs/hdfs.h
index eb9e1b2c07..c260ff9dea 100644
--- a/be/src/io/fs/hdfs.h
+++ b/be/src/io/fs/hdfs.h
@@ -18,7 +18,7 @@
 #pragma once
 
 #ifdef USE_HADOOP_HDFS
-#include <hadoop_hdfs/hdfs.h>
+#include <hadoop_hdfs/hdfs.h> // IWYU pragma: export
 #else
-#include <hdfs/hdfs.h>
+#include <hdfs/hdfs.h> // IWYU pragma: export
 #endif
diff --git a/be/src/io/fs/hdfs_file_reader.h b/be/src/io/fs/hdfs_file_reader.h
index 2b10baa602..cfb53eae39 100644
--- a/be/src/io/fs/hdfs_file_reader.h
+++ b/be/src/io/fs/hdfs_file_reader.h
@@ -17,11 +17,23 @@
 
 #pragma once
 
+#include <stddef.h>
+
+#include <atomic>
+#include <memory>
+#include <string>
+
+#include "common/status.h"
 #include "io/fs/file_reader.h"
+#include "io/fs/file_system.h"
+#include "io/fs/hdfs.h"
 #include "io/fs/hdfs_file_system.h"
+#include "io/fs/path.h"
+#include "util/slice.h"
 
 namespace doris {
 namespace io {
+class IOContext;
 
 class HdfsFileReader : public FileReader {
 public:
diff --git a/be/src/io/fs/hdfs_file_writer.h b/be/src/io/fs/hdfs_file_writer.h
index 92130a93c2..c9b3ff9def 100644
--- a/be/src/io/fs/hdfs_file_writer.h
+++ b/be/src/io/fs/hdfs_file_writer.h
@@ -17,18 +17,25 @@
 
 #pragma once
 
+#include <stddef.h>
+
 #include <map>
 #include <string>
 
+#include "common/status.h"
 #include "gen_cpp/PlanNodes_types.h"
+#include "io/fs/file_system.h"
 #include "io/fs/file_writer.h"
+#include "io/fs/hdfs.h"
 #include "io/fs/path.h"
 #include "io/hdfs_builder.h"
+#include "util/slice.h"
 
 namespace doris {
 namespace io {
 
 class HdfsFileSystem;
+
 class HdfsFileWriter : public FileWriter {
 public:
     HdfsFileWriter(Path file, FileSystemSPtr fs);
diff --git a/be/src/service/backend_options.cpp b/be/src/service/backend_options.cpp
index bad73b0c82..3b1652ef43 100644
--- a/be/src/service/backend_options.cpp
+++ b/be/src/service/backend_options.cpp
@@ -18,6 +18,7 @@
 #include "service/backend_options.h"
 
 #include <algorithm>
+#include <ostream>
 
 #include "common/config.h"
 #include "common/logging.h"
diff --git a/be/src/service/backend_options.h b/be/src/service/backend_options.h
index 5d28edbf12..87ca36b17a 100644
--- a/be/src/service/backend_options.h
+++ b/be/src/service/backend_options.h
@@ -17,6 +17,7 @@
 
 #pragma once
 
+#include <butil/macros.h>
 #include <gutil/macros.h>
 
 #include <string>
diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp
index 3ec71b8b37..977d9cfc02 100644
--- a/be/src/service/backend_service.cpp
+++ b/be/src/service/backend_service.cpp
@@ -18,43 +18,53 @@
 #include "service/backend_service.h"
 
 #include <arrow/record_batch.h>
-#include <gperftools/heap-profiler.h>
+#include <fmt/format.h>
+#include <gen_cpp/BackendService.h>
+#include <gen_cpp/BackendService_types.h>
+#include <gen_cpp/Data_types.h>
+#include <gen_cpp/DorisExternalService_types.h>
+#include <gen_cpp/PaloInternalService_types.h>
+#include <gen_cpp/Planner_types.h>
+#include <gen_cpp/Status_types.h>
+#include <gen_cpp/Types_types.h>
+#include <sys/types.h>
 #include <thrift/concurrency/ThreadFactory.h>
-#include <thrift/processor/TMultiplexedProcessor.h>
-#include <thrift/protocol/TDebugProtocol.h>
+#include <time.h>
 
 #include <map>
 #include <memory>
+#include <ostream>
+#include <utility>
 
 #include "common/config.h"
 #include "common/logging.h"
 #include "common/status.h"
-#include "gen_cpp/DorisExternalService_types.h"
-#include "gen_cpp/PaloInternalService_types.h"
-#include "gen_cpp/TDorisExternalService.h"
-#include "gen_cpp/Types_types.h"
 #include "gutil/strings/substitute.h"
+#include "olap/olap_common.h"
+#include "olap/olap_define.h"
 #include "olap/storage_engine.h"
-#include "runtime/descriptors.h"
+#include "olap/tablet_manager.h"
 #include "runtime/exec_env.h"
-#include "runtime/export_task_mgr.h"
 #include "runtime/external_scan_context_mgr.h"
 #include "runtime/fragment_mgr.h"
-#include "runtime/primitive_type.h"
-#include "runtime/result_buffer_mgr.h"
 #include "runtime/result_queue_mgr.h"
 #include "runtime/routine_load/routine_load_task_executor.h"
 #include "runtime/stream_load/stream_load_context.h"
-#include "service/backend_options.h"
+#include "runtime/stream_load/stream_load_recorder.h"
 #include "util/arrow/row_batch.h"
-#include "util/blocking_queue.hpp"
-#include "util/debug_util.h"
-#include "util/doris_metrics.h"
-#include "util/network_util.h"
 #include "util/thrift_server.h"
-#include "util/thrift_util.h"
 #include "util/uid_util.h"
-#include "util/url_coding.h"
+
+namespace apache {
+namespace thrift {
+class TException;
+class TMultiplexedProcessor;
+class TProcessor;
+namespace transport {
+class TTransportException;
+} // namespace transport
+} // namespace thrift
+} // namespace apache
 
 namespace doris {
 
diff --git a/be/src/service/backend_service.h b/be/src/service/backend_service.h
index 64a06e34ec..35a9ba8abc 100644
--- a/be/src/service/backend_service.h
+++ b/be/src/service/backend_service.h
@@ -17,17 +17,20 @@
 
 #pragma once
 
+#include <gen_cpp/BackendService.h>
+#include <gen_cpp/DorisExternalService_types.h>
+#include <gen_cpp/TDorisExternalService.h>
+#include <stdint.h>
 #include <thrift/protocol/TDebugProtocol.h>
 #include <time.h>
 
 #include <map>
 #include <memory>
+#include <string>
+#include <vector>
 
 #include "agent/agent_server.h"
 #include "common/status.h"
-#include "gen_cpp/BackendService.h"
-#include "gen_cpp/DorisExternalService_types.h"
-#include "gen_cpp/TDorisExternalService.h"
 
 namespace doris {
 
@@ -57,6 +60,20 @@ class TExportTaskRequest;
 class TExportStatusResult;
 class TStreamLoadRecordResult;
 class TDiskTrashInfo;
+class TCancelPlanFragmentParams;
+class TCheckStorageFormatResult;
+class TRoutineLoadTask;
+class TScanBatchResult;
+class TScanCloseParams;
+class TScanCloseResult;
+class TScanNextBatchParams;
+class TScanOpenParams;
+class TScanOpenResult;
+class TSnapshotRequest;
+class TStatus;
+class TTabletStatResult;
+class TTransmitDataParams;
+class TUniqueId;
 
 // This class just forward rpc for actual handler
 // make this class because we can bind multiple service on single point
diff --git a/be/src/service/brpc_service.cpp b/be/src/service/brpc_service.cpp
index e606fd8c85..35989f29dc 100644
--- a/be/src/service/brpc_service.cpp
+++ b/be/src/service/brpc_service.cpp
@@ -17,11 +17,18 @@
 
 #include "service/brpc_service.h"
 
+#include <brpc/server.h>
+#include <brpc/ssl_options.h>
+#include <bthread/errno.h>
+#include <butil/endpoint.h>
+#include <gflags/gflags_declare.h>
 #include <string.h>
 
+#include <ostream>
+
+#include "common/config.h"
 #include "common/logging.h"
 #include "service/backend_options.h"
-#include "service/brpc.h"
 #include "service/internal_service.h"
 
 namespace brpc {
diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp
index b3fd6e07ac..ab91661b5f 100644
--- a/be/src/service/doris_main.cpp
+++ b/be/src/service/doris_main.cpp
@@ -15,18 +15,28 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <bthread/errno.h>
+#include <butil/macros.h>
 #include <errno.h>
+#include <fcntl.h>
 #include <libgen.h>
 #include <setjmp.h>
-#include <sys/file.h>
+#include <signal.h>
+#include <stdint.h>
+#include <stdio.h>
+#include <stdlib.h>
 #include <unistd.h>
 
-#include <condition_variable>
+#include <algorithm>
 #include <cstring>
-#include <mutex>
-#include <thread>
-#include <unordered_map>
-
+#include <ostream>
+#include <string>
+#include <tuple>
+#include <vector>
+
+#include "olap/tablet_schema_cache.h"
+#include "olap/utils.h"
+#include "runtime/memory/mem_tracker_limiter.h"
 #include "util/jni-util.h"
 
 #if defined(LEAK_SANITIZER)
@@ -34,39 +44,35 @@
 #endif
 
 #include <curl/curl.h>
-#include <gperftools/profiler.h>
 #include <thrift/TOutput.h>
 
 #include "agent/heartbeat_server.h"
-#include "agent/topic_subscriber.h"
 #include "common/config.h"
 #include "common/daemon.h"
 #include "common/logging.h"
 #include "common/resource_tls.h"
 #include "common/signal_handler.h"
 #include "common/status.h"
-#include "common/utils.h"
 #include "io/cache/block/block_file_cache_factory.h"
 #include "olap/options.h"
 #include "olap/storage_engine.h"
 #include "runtime/exec_env.h"
-#include "runtime/heartbeat_flags.h"
 #include "service/backend_options.h"
 #include "service/backend_service.h"
 #include "service/brpc_service.h"
 #include "service/http_service.h"
 #include "service/single_replica_load_download_service.h"
 #include "util/debug_util.h"
-#include "util/doris_metrics.h"
-#include "util/perf_counters.h"
 #include "util/telemetry/telemetry.h"
 #include "util/thrift_rpc_helper.h"
 #include "util/thrift_server.h"
 #include "util/uid_util.h"
 
-static void help(const char*);
+namespace doris {
+class TMasterInfo;
+} // namespace doris
 
-#include <dlfcn.h>
+static void help(const char*);
 
 extern "C" {
 void __lsan_do_leak_check();
diff --git a/be/src/service/http_service.cpp b/be/src/service/http_service.cpp
index 3280ac0e6f..91e2a85044 100644
--- a/be/src/service/http_service.cpp
+++ b/be/src/service/http_service.cpp
@@ -17,6 +17,10 @@
 
 #include "service/http_service.h"
 
+#include <algorithm>
+#include <string>
+#include <vector>
+
 #include "http/action/check_rpc_channel_action.h"
 #include "http/action/check_tablet_segment_action.h"
 #include "http/action/checksum_action.h"
@@ -43,6 +47,7 @@
 #include "http/ev_http_server.h"
 #include "http/http_method.h"
 #include "http/web_page_handler.h"
+#include "olap/options.h"
 #include "runtime/exec_env.h"
 #include "runtime/load_path_mgr.h"
 #include "util/doris_metrics.h"
diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp
index 4a337f0f1e..1cf6323587 100644
--- a/be/src/service/internal_service.cpp
+++ b/be/src/service/internal_service.cpp
@@ -17,54 +17,92 @@
 
 #include "service/internal_service.h"
 
+#include <assert.h>
+#include <brpc/closure_guard.h>
+#include <brpc/controller.h>
+#include <bthread/bthread.h>
+#include <bthread/types.h>
+#include <butil/errno.h>
 #include <butil/iobuf.h>
+#include <fcntl.h>
+#include <gen_cpp/PaloInternalService_types.h>
+#include <gen_cpp/PlanNodes_types.h>
 #include <gen_cpp/Status_types.h>
+#include <gen_cpp/Types_types.h>
+#include <gen_cpp/internal_service.pb.h>
+#include <gen_cpp/olap_file.pb.h>
+#include <gen_cpp/segment_v2.pb.h>
+#include <gen_cpp/types.pb.h>
+#include <google/protobuf/stubs/callback.h>
+#include <stddef.h>
 #include <stdint.h>
+#include <sys/stat.h>
 
 #include <algorithm>
+#include <filesystem>
+#include <memory>
 #include <set>
 #include <sstream>
 #include <string>
+#include <utility>
 #include <vector>
 
 #include "common/config.h"
-#include "common/consts.h"
-#include "gen_cpp/BackendService.h"
-#include "gen_cpp/PaloInternalService_types.h"
-#include "gen_cpp/internal_service.pb.h"
+#include "common/logging.h"
+#include "gutil/integral_types.h"
 #include "http/http_client.h"
+#include "io/fs/stream_load_pipe.h"
+#include "io/io_common.h"
+#include "olap/data_dir.h"
+#include "olap/olap_common.h"
 #include "olap/rowset/beta_rowset.h"
+#include "olap/rowset/rowset.h"
 #include "olap/rowset/rowset_factory.h"
+#include "olap/rowset/rowset_meta.h"
 #include "olap/rowset/segment_v2/column_reader.h"
+#include "olap/rowset/segment_v2/common.h"
+#include "olap/rowset/segment_v2/segment.h"
 #include "olap/segment_loader.h"
 #include "olap/storage_engine.h"
 #include "olap/tablet.h"
+#include "olap/tablet_manager.h"
 #include "olap/tablet_schema.h"
+#include "olap/txn_manager.h"
+#include "olap/utils.h"
 #include "runtime/buffer_control_block.h"
+#include "runtime/cache/result_cache.h"
+#include "runtime/define_primitive_type.h"
+#include "runtime/descriptors.h"
 #include "runtime/exec_env.h"
 #include "runtime/fold_constant_executor.h"
 #include "runtime/fragment_mgr.h"
 #include "runtime/load_channel_mgr.h"
 #include "runtime/result_buffer_mgr.h"
 #include "runtime/routine_load/routine_load_task_executor.h"
-#include "runtime/runtime_state.h"
+#include "runtime/stream_load/new_load_stream_mgr.h"
+#include "runtime/stream_load/stream_load_context.h"
 #include "runtime/thread_context.h"
-#include "service/brpc.h"
+#include "runtime/types.h"
 #include "service/point_query_executor.h"
 #include "util/async_io.h"
 #include "util/brpc_client_cache.h"
-#include "util/defer_op.h"
+#include "util/doris_metrics.h"
 #include "util/md5.h"
+#include "util/metrics.h"
 #include "util/proto_util.h"
 #include "util/ref_count_closure.h"
-#include "util/s3_uri.h"
+#include "util/runtime_profile.h"
+#include "util/stopwatch.hpp"
 #include "util/string_util.h"
 #include "util/telemetry/brpc_carrier.h"
 #include "util/telemetry/telemetry.h"
 #include "util/thrift_util.h"
+#include "util/time.h"
 #include "util/uid_util.h"
+#include "vec/columns/column.h"
 #include "vec/core/block.h"
-#include "vec/data_types/data_type_string.h"
+#include "vec/core/column_with_type_and_name.h"
+#include "vec/data_types/data_type.h"
 #include "vec/exec/format/csv/csv_reader.h"
 #include "vec/exec/format/generic_reader.h"
 #include "vec/exec/format/json/new_json_reader.h"
@@ -72,6 +110,12 @@
 #include "vec/exec/format/parquet/vparquet_reader.h"
 #include "vec/runtime/vdata_stream_mgr.h"
 
+namespace google {
+namespace protobuf {
+class RpcController;
+} // namespace protobuf
+} // namespace google
+
 namespace doris {
 using namespace ErrorCode;
 
diff --git a/be/src/service/internal_service.h b/be/src/service/internal_service.h
index 2730839ffb..1b1e56d0cf 100644
--- a/be/src/service/internal_service.h
+++ b/be/src/service/internal_service.h
@@ -17,11 +17,22 @@
 
 #pragma once
 
+#include <gen_cpp/internal_service.pb.h>
+#include <stdint.h>
+
+#include <string>
+
 #include "common/status.h"
-#include "gen_cpp/internal_service.pb.h"
 #include "runtime/cache/result_cache.h"
 #include "util/priority_thread_pool.hpp"
 
+namespace google {
+namespace protobuf {
+class Closure;
+class RpcController;
+} // namespace protobuf
+} // namespace google
+
 namespace brpc {
 class Controller;
 }
@@ -29,6 +40,8 @@ class Controller;
 namespace doris {
 
 class ExecEnv;
+class PHandShakeRequest;
+class PHandShakeResponse;
 
 class PInternalServiceImpl : public PBackendService {
 public:
diff --git a/be/src/service/point_query_executor.cpp b/be/src/service/point_query_executor.cpp
index f57547d003..ed42c0c9c4 100644
--- a/be/src/service/point_query_executor.cpp
+++ b/be/src/service/point_query_executor.cpp
@@ -17,18 +17,27 @@
 
 #include "service/point_query_executor.h"
 
+#include <fmt/format.h>
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/Exprs_types.h>
+#include <gen_cpp/internal_service.pb.h>
+#include <stdlib.h>
+
 #include "olap/lru_cache.h"
+#include "olap/olap_tuple.h"
 #include "olap/row_cursor.h"
 #include "olap/storage_engine.h"
-#include "service/internal_service.h"
-#include "util/defer_op.h"
+#include "olap/tablet_manager.h"
+#include "olap/tablet_schema.h"
+#include "runtime/runtime_state.h"
 #include "util/key_util.h"
 #include "util/runtime_profile.h"
 #include "util/thrift_util.h"
 #include "vec/exprs/vexpr.h"
-#include "vec/exprs/vliteral.h"
+#include "vec/exprs/vexpr_context.h"
 #include "vec/jsonb/serialize.h"
 #include "vec/sink/vmysql_result_writer.cpp"
+#include "vec/sink/vmysql_result_writer.h"
 
 namespace doris {
 
diff --git a/be/src/service/point_query_executor.h b/be/src/service/point_query_executor.h
index 3f4a0426a1..69fa6850e9 100644
--- a/be/src/service/point_query_executor.h
+++ b/be/src/service/point_query_executor.h
@@ -17,22 +17,53 @@
 
 #pragma once
 
+#include <assert.h>
+#include <butil/macros.h>
+#include <butil/time.h>
+#include <gen_cpp/Metrics_types.h>
+#include <gen_cpp/internal_service.pb.h>
+#include <parallel_hashmap/phmap.h>
+#include <stdint.h>
+#include <string.h>
+
+#include <algorithm>
 #include <memory>
+#include <mutex>
+#include <optional>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
 
 #include "butil/containers/doubly_buffered_data.h"
+#include "common/config.h"
+#include "common/logging.h"
 #include "common/status.h"
-#include "gen_cpp/internal_service.pb.h"
 #include "gutil/int128.h"
+#include "olap/lru_cache.h"
 #include "olap/olap_common.h"
 #include "olap/rowset/rowset.h"
 #include "olap/tablet.h"
+#include "olap/utils.h"
+#include "runtime/descriptors.h"
+#include "util/mysql_global.h"
 #include "util/runtime_profile.h"
+#include "util/slice.h"
 #include "vec/core/block.h"
 
 namespace doris {
 
 class RowCache;
 class Cache;
+class PTabletKeyLookupRequest;
+class PTabletKeyLookupResponse;
+class RuntimeState;
+class TDescriptorTable;
+class TExpr;
+
+namespace vectorized {
+class VExprContext;
+} // namespace vectorized
 
 // For caching point lookup pre allocted blocks and exprs
 class Reusable {
diff --git a/be/src/service/single_replica_load_download_service.cpp b/be/src/service/single_replica_load_download_service.cpp
index 1cee6c677a..0e48c6760f 100644
--- a/be/src/service/single_replica_load_download_service.cpp
+++ b/be/src/service/single_replica_load_download_service.cpp
@@ -17,9 +17,15 @@
 
 #include "service/single_replica_load_download_service.h"
 
+#include <algorithm>
+#include <string>
+#include <vector>
+
 #include "http/action/download_action.h"
 #include "http/ev_http_server.h"
+#include "http/http_method.h"
 #include "io/fs/fs_utils.h"
+#include "olap/options.h"
 #include "runtime/exec_env.h"
 
 namespace doris {
diff --git a/be/src/udf/udf.cpp b/be/src/udf/udf.cpp
index d57d1cc3ae..37c8e72d17 100644
--- a/be/src/udf/udf.cpp
+++ b/be/src/udf/udf.cpp
@@ -20,23 +20,14 @@
 
 #include "udf/udf.h"
 
-#include <assert.h>
-
 #include <iostream>
-#include <sstream>
-
-#include "common/logging.h"
-#include "gen_cpp/types.pb.h"
-#include "olap/hll.h"
-#include "runtime/decimalv2_value.h"
+#include <utility>
 
 // Be careful what this includes since this needs to be linked into the UDF's
 // binary. For example, it would be unfortunate if they had a random dependency
 // on libhdfs.
 #include "runtime/runtime_state.h"
 #include "runtime/types.h"
-#include "udf/udf.h"
-#include "util/debug_util.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
diff --git a/be/src/udf/udf.h b/be/src/udf/udf.h
index 3d7d960038..bec07e8363 100644
--- a/be/src/udf/udf.h
+++ b/be/src/udf/udf.h
@@ -26,6 +26,7 @@
 #include <functional>
 #include <iostream>
 #include <memory>
+#include <string>
 #include <vector>
 
 #include "runtime/types.h"
@@ -38,7 +39,6 @@ class BitmapValue;
 class DecimalV2Value;
 class CollectionValue;
 struct TypeDescriptor;
-
 class RuntimeState;
 // All input and output values will be one of the structs below. The struct is a simple
 // object containing a boolean to store if the value is nullptr and the value itself. The
diff --git a/be/src/util/sse_util.hpp b/be/src/util/sse_util.hpp
index 60f1950a65..1d00a54fba 100644
--- a/be/src/util/sse_util.hpp
+++ b/be/src/util/sse_util.hpp
@@ -21,10 +21,10 @@
 #pragma once
 
 #if defined(__aarch64__)
-#include <sse2neon.h>
+#include <sse2neon.h> // IWYU pragma: export
 #elif defined(__x86_64__)
-#include <immintrin.h>
-#include <mm_malloc.h>
+#include <immintrin.h> // IWYU pragma: export
+#include <mm_malloc.h> // IWYU pragma: export
 #endif
 
 namespace doris {


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


[doris] 09/36: [BugFix](functions) fix multi_search_all_positions #18682

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

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

commit b883b2019a4c8dda2f58149a991bea66522bd19e
Author: zclllyybb <zh...@selectdb.com>
AuthorDate: Mon Apr 17 08:32:57 2023 +0800

    [BugFix](functions) fix multi_search_all_positions #18682
---
 be/src/vec/common/string_searcher.h                |  66 ----------
 .../functions/functions_multi_string_position.cpp  | 140 +++++++++++----------
 .../search/multi_search_all_positions.md           |  16 +--
 .../search/multi_search_all_positions.md           |  16 +--
 .../test_multi_string_position.out                 |  43 ++++---
 .../test_multi_string_position.groovy              |  47 +++++--
 6 files changed, 149 insertions(+), 179 deletions(-)

diff --git a/be/src/vec/common/string_searcher.h b/be/src/vec/common/string_searcher.h
index 97c5570a81..af76f2100d 100644
--- a/be/src/vec/common/string_searcher.h
+++ b/be/src/vec/common/string_searcher.h
@@ -416,70 +416,4 @@ struct LibCASCIICaseInsensitiveStringSearcher : public StringSearcherBase {
         return search(haystack, haystack + haystack_size);
     }
 };
-
-template <typename StringSearcher>
-class MultiStringSearcherBase {
-private:
-    /// needles
-    const std::vector<StringRef>& needles;
-    /// searchers
-    std::vector<StringSearcher> searchers;
-    /// last index of needles that was not processed
-    size_t last;
-
-public:
-    explicit MultiStringSearcherBase(const std::vector<StringRef>& needles_)
-            : needles {needles_}, last {0} {
-        searchers.reserve(needles.size());
-
-        size_t size = needles.size();
-        for (int i = 0; i < size; ++i) {
-            const char* cur_needle_data = needles[i].data;
-            const size_t cur_needle_size = needles[i].size;
-
-            searchers.emplace_back(cur_needle_data, cur_needle_size);
-        }
-    }
-
-    /**
-     * while (hasMoreToSearch())
-     * {
-     *     search inside the haystack with the known needles
-     * }
-     */
-    bool hasMoreToSearch() {
-        if (last >= needles.size()) {
-            return false;
-        }
-
-        return true;
-    }
-
-    bool searchOne(const uint8_t* haystack, const uint8_t* haystack_end) {
-        const size_t size = needles.size();
-        if (last >= size) {
-            return false;
-        }
-
-        if (searchers[++last].search(haystack, haystack_end) != haystack_end) {
-            return true;
-        }
-        return false;
-    }
-
-    template <typename CountCharsCallback, typename AnsType>
-    void searchOneAll(const uint8_t* haystack, const uint8_t* haystack_end, AnsType* answer,
-                      const CountCharsCallback& count_chars) {
-        const size_t size = needles.size();
-        for (; last < size; ++last) {
-            const uint8_t* ptr = searchers[last].search(haystack, haystack_end);
-            if (ptr != haystack_end) {
-                answer[last] = count_chars(haystack, ptr);
-            }
-        }
-    }
-};
-
-using MultiStringSearcher = MultiStringSearcherBase<ASCIICaseSensitiveStringSearcher>;
-
 } // namespace doris
diff --git a/be/src/vec/functions/functions_multi_string_position.cpp b/be/src/vec/functions/functions_multi_string_position.cpp
index a3c3420acb..5b8dd13b9b 100644
--- a/be/src/vec/functions/functions_multi_string_position.cpp
+++ b/be/src/vec/functions/functions_multi_string_position.cpp
@@ -18,6 +18,9 @@
 // https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsMultiStringPosition.h
 // and modified by Doris
 
+#include <cstdint>
+#include <iterator>
+
 #include "function.h"
 #include "function_helpers.h"
 #include "vec/columns/column_array.h"
@@ -103,48 +106,53 @@ public:
     }
 };
 
-template <typename Impl>
 struct FunctionMultiSearchAllPositionsImpl {
+public:
     using ResultType = Int32;
-
+    using SingleSearcher = ASCIICaseSensitiveStringSearcher;
     static constexpr auto name = "multi_search_all_positions";
 
     static Status vector_constant(const ColumnString::Chars& haystack_data,
                                   const ColumnString::Offsets& haystack_offsets,
                                   const Array& needles_arr, PaddedPODArray<Int32>& vec_res,
                                   PaddedPODArray<UInt64>& offsets_res) {
-        if (needles_arr.size() > std::numeric_limits<UInt8>::max())
+        if (needles_arr.size() > std::numeric_limits<UInt8>::max()) {
             return Status::InvalidArgument(
                     "number of arguments for function {} doesn't match: "
                     "passed {}, should be at most 255",
                     name, needles_arr.size());
+        }
 
-        std::vector<StringRef> needles;
-        needles.reserve(needles_arr.size());
-        for (const auto& needle : needles_arr) needles.emplace_back(needle.get<StringRef>());
-
-        auto res_callback = [](const UInt8* start, const UInt8* end) -> Int32 {
-            return 1 + Impl::count_chars(reinterpret_cast<const char*>(start),
-                                         reinterpret_cast<const char*>(end));
-        };
-
-        auto searcher = Impl::create_multi_searcher(needles);
+        const size_t needles_size = needles_arr.size();
+        std::vector<SingleSearcher> searchers;
+        searchers.reserve(needles_size);
+        for (const auto& needle : needles_arr) {
+            searchers.emplace_back(needle.get<StringRef>().data, needle.get<StringRef>().size);
+        }
 
         const size_t haystack_size = haystack_offsets.size();
-        const size_t needles_size = needles.size();
-
-        vec_res.resize(haystack_size * needles.size());
+        vec_res.resize(haystack_size * needles_size);
         offsets_res.resize(haystack_size);
 
         std::fill(vec_res.begin(), vec_res.end(), 0);
 
-        while (searcher.hasMoreToSearch()) {
+        // we traverse to generator answer by Vector's slot of ColumnVector, not by Vector.
+        // TODO: check if the order of loop is best. The large data may make us writing across the line which size out of L2 cache.
+        for (size_t ans_slot_in_row = 0; ans_slot_in_row < searchers.size(); ans_slot_in_row++) {
+            //  is i.e. answer slot index in one Vector(row) of answer
+            auto& searcher = searchers[ans_slot_in_row];
             size_t prev_haystack_offset = 0;
-            for (size_t j = 0, from = 0; j < haystack_size; ++j, from += needles_size) {
+
+            for (size_t haystack_index = 0, res_index = ans_slot_in_row;
+                 haystack_index < haystack_size; ++haystack_index, res_index += needles_size) {
                 const auto* haystack = &haystack_data[prev_haystack_offset];
-                const auto* haystack_end = haystack + haystack_offsets[j] - prev_haystack_offset;
-                searcher.searchOneAll(haystack, haystack_end, &vec_res[from], res_callback);
-                prev_haystack_offset = haystack_offsets[j];
+                const auto* haystack_end =
+                        haystack - prev_haystack_offset + haystack_offsets[haystack_index];
+
+                auto ans_now = searcher.search(haystack, haystack_end);
+                vec_res[res_index] =
+                        ans_now >= haystack_end ? 0 : std::distance(haystack, ans_now) + 1;
+                prev_haystack_offset = haystack_offsets[haystack_index];
             }
         }
 
@@ -166,72 +174,72 @@ struct FunctionMultiSearchAllPositionsImpl {
         size_t prev_haystack_offset = 0;
         size_t prev_needles_offset = 0;
 
-        auto res_callback = [](const UInt8* start, const UInt8* end) -> Int32 {
-            return 1 + Impl::count_chars(reinterpret_cast<const char*>(start),
-                                         reinterpret_cast<const char*>(end));
-        };
-
-        offsets_res.reserve(haystack_offsets.size());
+        offsets_res.reserve(haystack_data.size());
+        uint64_t offset_now = 0;
 
         auto& nested_column =
                 vectorized::check_and_get_column<vectorized::ColumnNullable>(needles_data)
                         ->get_nested_column();
         const ColumnString* needles_data_string = check_and_get_column<ColumnString>(nested_column);
 
-        std::vector<StringRef> needles;
-        for (size_t i = 0; i < haystack_offsets.size(); ++i) {
-            needles.reserve(needles_offsets[i] - prev_needles_offset);
-
-            for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) {
-                needles.emplace_back(needles_data_string->get_data_at(j));
+        std::vector<StringRef> needles_for_row;
+        // haystack first, row by row.
+        for (size_t haystack_index = 0; haystack_index < haystack_offsets.size();
+             ++haystack_index) {
+            // get haystack for this row.
+            const auto* haystack = &haystack_data[prev_haystack_offset];
+            const auto* haystack_end =
+                    haystack - prev_haystack_offset + haystack_offsets[haystack_index];
+
+            // build needles for this row.
+            needles_for_row.reserve(needles_offsets[haystack_index] - prev_needles_offset);
+            for (size_t j = prev_needles_offset; j < needles_offsets[haystack_index]; ++j) {
+                needles_for_row.emplace_back(needles_data_string->get_data_at(j));
             }
-
-            const size_t needles_size = needles.size();
-            if (needles_size > std::numeric_limits<UInt8>::max())
+            const size_t needles_row_size = needles_for_row.size();
+            if (needles_row_size > std::numeric_limits<UInt8>::max()) {
                 return Status::InvalidArgument(
                         "number of arguments for function {} doesn't match: "
                         "passed {}, should be at most 255",
-                        name, needles_size);
-
-            vec_res.resize(vec_res.size() + needles_size);
-
-            auto searcher = Impl::create_multi_searcher(needles);
-
-            std::fill(vec_res.begin() + vec_res.size() - needles_size, vec_res.end(), 0);
+                        name, needles_row_size);
+            }
 
-            while (searcher.hasMoreToSearch()) {
-                const auto* haystack = &haystack_data[prev_haystack_offset];
-                const auto* haystack_end = haystack + haystack_offsets[i] - prev_haystack_offset;
-                searcher.searchOneAll(haystack, haystack_end,
-                                      &vec_res[vec_res.size() - needles_size], res_callback);
+            // each searcher search for one needle.
+            std::vector<SingleSearcher> searchers;
+            searchers.clear();
+            searchers.reserve(needles_row_size);
+            for (auto needle : needles_for_row) {
+                searchers.emplace_back(needle.data, needle.size);
             }
 
-            if (offsets_res.empty())
-                offsets_res.push_back(needles_size);
-            else
-                offsets_res.push_back(offsets_res.back() + needles_size);
+            // search for first so that the ans's size is constant for each row.
+            auto ans_row_begin = vec_res.size();
+            vec_res.resize(vec_res.size() + needles_row_size);
+            offset_now += searchers.size();
+            offsets_res.emplace_back(offset_now);
+
+            //for now haystack, apply needle to search, generator answer by order.
+            for (size_t ans_slot_in_row = 0; ans_slot_in_row < searchers.size();
+                 ans_slot_in_row++) {
+                //  is i.e. answer slot index in one Vector(row) of answer
+                auto& searcher = searchers[ans_slot_in_row];
+
+                auto ans_now = searcher.search(haystack, haystack_end);
+                vec_res[ans_row_begin + ans_slot_in_row] =
+                        ans_now >= haystack_end ? 0 : std::distance(haystack, ans_now) + 1;
+            }
 
-            prev_haystack_offset = haystack_offsets[i];
-            prev_needles_offset = needles_offsets[i];
-            needles.clear();
+            prev_haystack_offset = haystack_offsets[haystack_index];
+            prev_needles_offset = needles_offsets[haystack_index];
+            needles_for_row.clear();
         }
 
         return Status::OK();
     }
 };
 
-struct MultiSearcherImpl {
-    using MultiSearcher = MultiStringSearcher;
-
-    static MultiSearcher create_multi_searcher(const std::vector<StringRef>& needles) {
-        return MultiSearcher(needles);
-    }
-
-    static size_t count_chars(const char* begin, const char* end) { return end - begin; }
-};
-
 using FunctionMultiSearchAllPositions =
-        FunctionMultiStringPosition<FunctionMultiSearchAllPositionsImpl<MultiSearcherImpl>>;
+        FunctionMultiStringPosition<FunctionMultiSearchAllPositionsImpl>;
 
 void register_function_multi_string_position(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionMultiSearchAllPositions>();
diff --git a/docs/en/docs/sql-manual/sql-functions/string-functions/search/multi_search_all_positions.md b/docs/en/docs/sql-manual/sql-functions/string-functions/search/multi_search_all_positions.md
index 9f5c07fdcc..db52923b6a 100644
--- a/docs/en/docs/sql-manual/sql-functions/string-functions/search/multi_search_all_positions.md
+++ b/docs/en/docs/sql-manual/sql-functions/string-functions/search/multi_search_all_positions.md
@@ -30,8 +30,7 @@ under the License.
 
 `ARRAY<INT> multi_search_all_positions(VARCHAR haystack, ARRAY<VARCHAR> needles)`
 
-
-Searches for the substrings `needles` in the string `haystack`, and returns array of positions of the found corresponding substrings in the string. Positions are indexed starting from 1.
+Returns an `ARRAY` where the `i`-th element is the position of the `i`-th element in `needles`(i.e. `needle`)'s **first** occurrence in the string `haystack`. Positions are counted from 1, with 0 meaning the element was not found. **Case-sensitive**.
 
 ### example
 
@@ -43,12 +42,13 @@ mysql> select multi_search_all_positions('Hello, World!', ['hello', '!', 'world'
 | [0,13,0]                                                             |
 +----------------------------------------------------------------------+
 
-mysql> select multi_search_all_positions('abc', ['a', 'bc', 'd']);
-+-----------------------------------------------------+
-| multi_search_all_positions('abc', ['a', 'bc', 'd']) |
-+-----------------------------------------------------+
-| [1,2,0]                                             |
-+-----------------------------------------------------+
+select multi_search_all_positions("Hello, World!", ['hello', '!', 'world', 'Hello', 'World']);
++---------------------------------------------------------------------------------------------+
+| multi_search_all_positions('Hello, World!', ARRAY('hello', '!', 'world', 'Hello', 'World')) |
++---------------------------------------------------------------------------------------------+
+| [0, 13, 0, 1, 8]                                                                            |
++---------------------------------------------------------------------------------------------+
 ```
+
 ### keywords
     MULTI_SEARCH,SEARCH,POSITIONS
diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/string-functions/search/multi_search_all_positions.md b/docs/zh-CN/docs/sql-manual/sql-functions/string-functions/search/multi_search_all_positions.md
index f93466587d..27f60f1b59 100644
--- a/docs/zh-CN/docs/sql-manual/sql-functions/string-functions/search/multi_search_all_positions.md
+++ b/docs/zh-CN/docs/sql-manual/sql-functions/string-functions/search/multi_search_all_positions.md
@@ -30,8 +30,7 @@ under the License.
 
 `ARRAY<INT> multi_search_all_positions(VARCHAR haystack, ARRAY<VARCHAR> needles)`
 
-
-搜索字符串 `haystack` 中的子字符串 `needles`,并返回在 `haystack` 中找到的相应 `needles` 的位置数组。位置的下标从 1 开始。
+返回一个 `ARRAY`,其中第 `i` 个元素为 `needles` 中第 `i` 个元素 `needle`,在字符串 `haystack` 中**首次**出现的位置。位置从1开始计数,0代表未找到该元素。**大小写敏感**。
 
 ### example
 
@@ -43,12 +42,13 @@ mysql> select multi_search_all_positions('Hello, World!', ['hello', '!', 'world'
 | [0,13,0]                                                             |
 +----------------------------------------------------------------------+
 
-mysql> select multi_search_all_positions('abc', ['a', 'bc', 'd']);
-+-----------------------------------------------------+
-| multi_search_all_positions('abc', ['a', 'bc', 'd']) |
-+-----------------------------------------------------+
-| [1,2,0]                                             |
-+-----------------------------------------------------+
+select multi_search_all_positions("Hello, World!", ['hello', '!', 'world', 'Hello', 'World']);
++---------------------------------------------------------------------------------------------+
+| multi_search_all_positions('Hello, World!', ARRAY('hello', '!', 'world', 'Hello', 'World')) |
++---------------------------------------------------------------------------------------------+
+| [0, 13, 0, 1, 8]                                                                            |
++---------------------------------------------------------------------------------------------+
 ```
+
 ### keywords
     MULTI_SEARCH,SEARCH,POSITIONS
diff --git a/regression-test/data/query_p0/sql_functions/search_functions/test_multi_string_position.out b/regression-test/data/query_p0/sql_functions/search_functions/test_multi_string_position.out
index 165a97dea6..017fa620c4 100644
--- a/regression-test/data/query_p0/sql_functions/search_functions/test_multi_string_position.out
+++ b/regression-test/data/query_p0/sql_functions/search_functions/test_multi_string_position.out
@@ -1,25 +1,30 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
--- !select --
+-- !table_select1 --
+[0, 13, 0, 1, 8]
+[0, 13, 0, 1, 8]
+[1, 13, 8, 0, 0]
+[1, 13, 8, 0, 0]
+[0, 6, 0, 0, 0]
+
+-- !table_select2 --
+[0, 0]
+[0, 0, 1, 13]
+[0, 8]
+[1, 8, 0, 13]
+[1, 1, 4, 0]
+
+-- !select1 --
 [4, 1, 1, 2, 6, 1, 1, 0, 4, 1, 14, 0, 10, 0, 16, 6]
--- !select --
+
+-- !select2 --
 [0, 0, 0, 2, 3, 0, 1, 0, 5, 0, 0, 0, 11, 10, 6, 7]
--- !select --
+
+-- !select3 --
 [1, 1, 0, 0, 0, 1, 1, 1, 4, 0, 6, 6, 0, 10, 1, 5]
--- !select --
+
+-- !select4 --
 [1, 0, 0, 8, 6, 0, 7, 1, 3, 0, 0, 0, 0, 12]
--- !select --
+
+-- !select5 --
 [7, 0, 0, 8, 0, 2, 0, 0, 6, 0, 2, 0, 3, 1]
--- !select --
-[0, 0, 0, 19, 14, 22, 10, 0, 0, 13, 0, 8]
--- !select --
-[6, 1, 1, 0, 0, 5, 1, 0, 8, 0, 5, 0, 2, 12, 0, 15, 0, 0]
--- !select --
-[0, 0, 5, 1, 1, 0, 15, 1, 5, 10, 4, 0, 1, 0, 3, 0, 0, 0]
--- !select --
-[0, 1, 11, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4, 1]
--- !select --
-[0, 0, 0, 3, 0, 15, 0, 0, 12, 7, 0, 0, 0, 0, 5, 0]
--- !select --
-[0, 0, 12, 4, 4, 0, 13, 23, 0, 1, 0, 2, 0, 0, 0, 3, 0, 0]
--- !select --
-[6, 8, 1, 4, 0, 10, 0, 1, 14, 0, 1, 0, 5, 0, 0, 0, 0, 15, 0, 1]
\ No newline at end of file
+
diff --git a/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_position.groovy b/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_position.groovy
index 28cc08efb8..fa3ec92b66 100644
--- a/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_position.groovy
+++ b/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_position.groovy
@@ -16,16 +16,39 @@
 // under the License.
 
 suite("test_multi_string_position") {
-    qt_select "select multi_search_all_positions('jmdqwjbrxlbatqeixknricfk', ['qwjbrxlba', 'jmd', '', 'mdqwjbrxlbatqe', 'jbrxlbatqeixknric', 'jmdqwjbrxlbatqeixknri', '', 'fdtmnwtts', 'qwjbrxlba', '', 'qeixknricfk', 'hzjjgrnoilfkvzxaemzhf', 'lb', 'kamz', 'ixknr', 'jbrxlbatq'])"
-    qt_select "select multi_search_all_positions('coxcctuehmzkbrsmodfvx', ['bkhnp', 'nlypjvriuk', 'rkslxwfqjjivcwdexrdtvjdtvuu', 'oxcctuehm', 'xcctuehmzkbrsm', 'kfrieuocovykjmkwxbdlkgwctwvcuh', 'coxc', 'lbwvetgxyndxjqqwthtkgasbafii', 'ctuehmzkbrsmodfvx', 'obzldxjldxowk', 'ngfikgigeyll', 'wdaejjukowgvzijnw', 'zkbr', 'mzkb', 'tuehm', 'ue'])"
-    qt_select "select multi_search_all_positions('mpswgtljbbrmivkcglamemayfn', ['', 'm', 'saejhpnfgfq', 'rzanrkdssmmkanqjpfi', 'oputeneprgoowg', 'mp', '', '', 'wgtljbbrmivkcglamemay', 'cbpthtrgrmgfypizi', 'tl', 'tlj', 'xuhs', 'brmivkcglamemayfn', '', 'gtljb'])"
-    qt_select "select multi_search_all_positions('arbphzbbecypbzsqsljurtddve', ['arbphzb', 'mnrboimjfijnti', 'cikcrd', 'becypbz', 'z', 'uocmqgnczhdcrvtqrnaxdxjjlhakoszuwc', 'bbe', '', 'bp', 'yhltnexlpdijkdzt', 'jkwjmrckvgmccmmrolqvy', 'vdxmicjmfbtsbqqmqcgtnrvdgaucsgspwg', 'witlfqwvhmmyjrnrzttrikhhsrd', 'pbzsqsljurt'])"
-    qt_select "select multi_search_all_positions('aizovxqpzcbbxuhwtiaaqhdqjdei', ['qpzcbbxuhw', 'jugrpglqbm', 'dspwhzpyjohhtizegrnswhjfpdz', 'pzcbbxuh', 'vayzeszlycke', 'i', 'gvrontcpqavsjxtjwzgwxugiyhkhmhq', 'gyzmeroxztgaurmrqwtmsxcqnxaezuoapatvu', 'xqpzc', 'mjiswsvlvlpqrhhptqq', 'iz', 'hmzjxxfjsvcvdpqwtrdrp', 'zovxqpzcbbxuhwtia', 'ai'])"
-    qt_select "select multi_search_all_positions('ydfgiluhyxwqdfiwtzobwzscyxhuov', ['srsoubrgghleyheujsbwwwykerzlqphgejpxvog', 'axchkyleddjwkvbuyhmekpbbbztxdlm', 'zqodzvlkmfe', 'obwz', 'fi', 'zsc', 'xwq', 'pvmurvrd', 'uulcdtexckmrsokmgdpkstlkoavyrmxeaacvydxf', 'dfi', 'mxcngttujzgtlssrmluaflmjuv', 'hyxwqdfiwtzobwzscyxhu'])"
-    qt_select "select multi_search_all_positions('pyepgwainvmwekwhhqxxvzdjw', ['w', '', '', 'gvvkllofjnxvcu', 'kmwwhboplctvzazcyfpxhwtaddfnhekei', 'gwainv', 'pyepgwain', 'ekpnogkzzmbpfynsunwqp', 'invmwe', 'hrxpiplfplqjsstuybksuteoz', 'gwa', 'akfpyduqrwosxcbdemtxrxvundrgse', 'yepgwainvmw', 'wekwhhqxxvzdjw', 'fyimzvedmyriubgoznmcav', 'whhq', 'ozxowbwdqfisuupyzaqynoprgsjhkwlum', 'vpoufrofekajksdp'])"
-    qt_select "select multi_search_all_positions('lqwahffxurkbhhzytequotkfk', ['rwjqudpuaiufle', 'livwgbnflvy', 'hffxurkbhh', '', '', 'xcajwbqbttzfzfowjubmmgnmssat', 'zytequ', 'lq', 'h', 'rkbhh', 'a', 'immejthwgdr', '', 'llhhnlhcvnxxorzzjt', 'w', 'cvjynqxcivmmmvc', 'wexjomdcmursppjtsweybheyxzleuz', 'fzronsnddfxwlkkzidiknhpjipyrcrzel'])"
-    qt_select "select multi_search_all_positions('nkddriylnakicwgdwrfxpodqea', ['izwdpgrgpmjlwkanjrffgela', '', 'kicw', 'hltmfymgmrjckdiylkzjlvvyuleksikdjrg', 'yigveskrbidknjxigwilmkgyizewikh', 'xyvzhsnqmuec', 'odcgzlavzrwesjks', 'oilvfgliktoujukpgzvhmokdgkssqgqot', 'llsfsurvimbahwqtbqbp', 'nxj', 'pimydixeobdxmdkvhcyzcgnbhzsydx', 'couzmvxedobuohibgxwoxvmpote', 'driylnakicwgdwrf', 'nkddr'])"
-    qt_select "select multi_search_all_positions('jnckhtjqwycyihuejibqmddrdxe', ['tajzx', 'vuddoylclxatcjvinusdwt', 'spxkhxvzsljkmnzpeubszjnhqczavgtqopxn', 'ckhtjqwycyi', 'xlbfzdxspldoes', 'u', 'czosfebeznt', 'gzhabdsuyreisxvyfrfrkq', 'yihuejibqmd', 'jqwycyihuejibqm', 'cfbvprgzx', 'hxu', 'vxbhrfpzacgd', 'afoaij', 'htjqwycyihu', 'httzbskqd'])"
-    qt_select "select multi_search_all_positions('dzejajvpoojdkqbnayahygidyrjmb', ['khwxxvtnqhobbvwgwkpusjlhlzifiuclycml', 'nzvuhtwdaivo', 'dkqbnayahygidyr', 'jajvpoo', 'j', 'wdtbvwmeqgyvetu', 'kqbn', 'idyrjmb', 'tsnxuxevsxrxpgpfdgrkhwqpkse', '', 'efsdgzuefhdzkmquxu', 'zejajvpoojdkqbnayahyg', 'ugwfuighbygrxyctop', 'fcbxzbdugc', 'dxmzzrcplob', 'ejaj', 'wmmupyxrylvawsyfccluiiene', 'ohzmsqhpzbafvbzqwzftbvftei'])"
-    qt_select "select multi_search_all_positions('ffaujlverosspbzaqefjzql', ['lvero', 'erossp', 'f', 'ujlverosspbz', 'btfimgklzzxlbkbuqyrmnud', 'osspb', 'muqexvtjuaar', 'f', 'bzaq', 'lprihswhwkdhqciqhfaowarn', 'ffaujlve', 'uhbbjrqjb', 'jlver', 'umucyhbbu', 'pjthtzmgxhvpbdphesnnztuu', 'xfqhfdfsbbazactpastzvzqudgk', 'lvovjfoatc', 'z', 'givejzhoqsd', ''])"
+    def table_name = "strings"
+
+    sql """ DROP TABLE IF EXISTS ${table_name} """
+    sql """ CREATE TABLE IF NOT EXISTS ${table_name}
+            (
+                `col1`      INT NOT NULL,
+                `content`   TEXT NOT NULL,
+                `mode`      ARRAY<TEXT> NOT NULL
+            ) ENGINE=OLAP
+            DUPLICATE KEY(`col1`)
+            COMMENT 'OLAP'
+            DISTRIBUTED BY HASH(`col1`) BUCKETS 3
+            PROPERTIES (
+            "replication_allocation" = "tag.location.default: 1",
+            "in_memory" = "false",
+            "storage_format" = "V2"
+            );
+        """
+
+    sql """ INSERT INTO ${table_name} (col1, content, mode) VALUES
+            (1, 'Hello, World!', ['hello', 'world'] ),
+            (2, 'Hello, World!', ['hello', 'world', 'Hello', '!'] ),
+            (3, 'hello, world!', ['Hello', 'world'] ),
+            (4, 'hello, world!', ['hello', 'world', 'Hello', '!'] ),
+            (5, 'HHHHW!', ['H', 'HHHH', 'HW', 'WH'] );
+        """
+
+    qt_table_select1 "select multi_search_all_positions(content, ['hello', '!', 'world', 'Hello', 'World']) from ${table_name} order by col1"
+    qt_table_select2 "select multi_search_all_positions(content, mode) from ${table_name} order by col1"
+
+    qt_select1 "select multi_search_all_positions('jmdqwjbrxlbatqeixknricfk', ['qwjbrxlba', 'jmd', '', 'mdqwjbrxlbatqe', 'jbrxlbatqeixknric', 'jmdqwjbrxlbatqeixknri', '', 'fdtmnwtts', 'qwjbrxlba', '', 'qeixknricfk', 'hzjjgrnoilfkvzxaemzhf', 'lb', 'kamz', 'ixknr', 'jbrxlbatq'])"
+    qt_select2 "select multi_search_all_positions('coxcctuehmzkbrsmodfvx', ['bkhnp', 'nlypjvriuk', 'rkslxwfqjjivcwdexrdtvjdtvuu', 'oxcctuehm', 'xcctuehmzkbrsm', 'kfrieuocovykjmkwxbdlkgwctwvcuh', 'coxc', 'lbwvetgxyndxjqqwthtkgasbafii', 'ctuehmzkbrsmodfvx', 'obzldxjldxowk', 'ngfikgigeyll', 'wdaejjukowgvzijnw', 'zkbr', 'mzkb', 'tuehm', 'ue'])"
+    qt_select3 "select multi_search_all_positions('mpswgtljbbrmivkcglamemayfn', ['', 'm', 'saejhpnfgfq', 'rzanrkdssmmkanqjpfi', 'oputeneprgoowg', 'mp', '', '', 'wgtljbbrmivkcglamemay', 'cbpthtrgrmgfypizi', 'tl', 'tlj', 'xuhs', 'brmivkcglamemayfn', '', 'gtljb'])"
+    qt_select4 "select multi_search_all_positions('arbphzbbecypbzsqsljurtddve', ['arbphzb', 'mnrboimjfijnti', 'cikcrd', 'becypbz', 'z', 'uocmqgnczhdcrvtqrnaxdxjjlhakoszuwc', 'bbe', '', 'bp', 'yhltnexlpdijkdzt', 'jkwjmrckvgmccmmrolqvy', 'vdxmicjmfbtsbqqmqcgtnrvdgaucsgspwg', 'witlfqwvhmmyjrnrzttrikhhsrd', 'pbzsqsljurt'])"
+    qt_select5 "select multi_search_all_positions('aizovxqpzcbbxuhwtiaaqhdqjdei', ['qpzcbbxuhw', 'jugrpglqbm', 'dspwhzpyjohhtizegrnswhjfpdz', 'pzcbbxuh', 'vayzeszlycke', 'i', 'gvrontcpqavsjxtjwzgwxugiyhkhmhq', 'gyzmeroxztgaurmrqwtmsxcqnxaezuoapatvu', 'xqpzc', 'mjiswsvlvlpqrhhptqq', 'iz', 'hmzjxxfjsvcvdpqwtrdrp', 'zovxqpzcbbxuhwtia', 'ai'])"
 }


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


[doris] 16/36: [fix](build) Fix missing header files (#18740)

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

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

commit cd37185860a1d8378d6f5adbf9908b10c7724760
Author: yongjinhou <10...@users.noreply.github.com>
AuthorDate: Mon Apr 17 21:22:15 2023 +0800

    [fix](build) Fix missing header files (#18740)
---
 be/src/exec/data_sink.cpp | 1 +
 be/src/exec/exec_node.cpp | 1 +
 2 files changed, 2 insertions(+)

diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp
index 197a353124..fd8406a355 100644
--- a/be/src/exec/data_sink.cpp
+++ b/be/src/exec/data_sink.cpp
@@ -34,6 +34,7 @@
 #include "vec/sink/vdata_stream_sender.h"
 #include "vec/sink/vjdbc_table_sink.h"
 #include "vec/sink/vmemory_scratch_sink.h"
+#include "vec/sink/vmysql_table_sink.h" // IWYU pragma: keep
 #include "vec/sink/vodbc_table_sink.h"
 #include "vec/sink/vresult_file_sink.h"
 #include "vec/sink/vresult_sink.h"
diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp
index 6c3a011dc5..aced73a595 100644
--- a/be/src/exec/exec_node.cpp
+++ b/be/src/exec/exec_node.cpp
@@ -61,6 +61,7 @@
 #include "vec/exec/vdata_gen_scan_node.h"
 #include "vec/exec/vempty_set_node.h"
 #include "vec/exec/vexchange_node.h"
+#include "vec/exec/vmysql_scan_node.h" // IWYU pragma: keep
 #include "vec/exec/vrepeat_node.h"
 #include "vec/exec/vschema_scan_node.h"
 #include "vec/exec/vselect_node.h"


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


[doris] 11/36: [Bug](DECIMALV3) report failure if a decimal value is overflow (#18336)

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

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

commit 64bb7c278ecf5b3ffeae7dc8b0932b5c7731a78d
Author: Gabriel <ga...@gmail.com>
AuthorDate: Mon Apr 17 13:18:14 2023 +0800

    [Bug](DECIMALV3) report failure if a decimal value is overflow (#18336)
---
 be/src/runtime/decimalv2_value.cpp                 |    2 +-
 be/src/util/string_parser.hpp                      |    8 +
 be/src/vec/data_types/data_type_decimal.cpp        |   11 +-
 be/src/vec/data_types/data_type_decimal.h          |    2 +-
 be/src/vec/exprs/vexpr.cpp                         |  166 +-
 be/src/vec/exprs/vliteral.cpp                      |   59 +-
 be/src/vec/io/io_helper.h                          |   12 +-
 be/src/vec/sink/vtablet_sink.cpp                   |   69 +
 be/src/vec/sink/vtablet_sink.h                     |   10 +
 .../java/org/apache/doris/catalog/ScalarType.java  |    6 +-
 .../org/apache/doris/analysis/ArithmeticExpr.java  |    9 +-
 .../org/apache/doris/planner/QueryPlanTest.java    |    4 +-
 .../data/datatype_p0/decimalv3/test_overflow.out   |   19 -
 regression-test/data/insert_p0/insert.out          |  182 +-
 regression-test/data/nereids_arith_p0/date.out     | 2968 ------
 regression-test/data/nereids_arith_p0/decimal.out  | 9995 ++------------------
 regression-test/data/nereids_arith_p0/integer.out  | 6096 +-----------
 regression-test/data/nereids_arith_p0/string.out   | 3047 +-----
 .../data/nereids_p0/aggregate/aggregate.out        |    8 +-
 .../operator/test_arithmetic_operators.out         |   10 -
 .../conditional_functions/test_nullif.out          |    2 +-
 .../test_arithmetic_operators.out                  |   18 -
 .../operator/test_arithmetic_operators.out         |  261 +-
 .../test_array_functions_by_literal.out            |    3 -
 .../cast_function/test_cast_function.out           |   16 +-
 .../conditional_functions/test_nullif.out          |    2 +-
 .../decimalv3/test_data/test_overflow.csv          |    2 +
 .../datatype_p0/decimalv3/test_overflow.groovy     |   56 -
 .../suites/nereids_arith_p0/load.groovy            |    4 +-
 .../suites/nereids_p0/aggregate/aggregate.groovy   |    2 +-
 .../operator/test_arithmetic_operators.groovy      |   14 +-
 .../conditional_functions/test_nullif.groovy       |    4 +-
 .../test_arithmetic_operators.groovy               |   14 +-
 .../suites/point_query_p0/test_point_query.groovy  |    2 +-
 .../suites/query_p0/aggregate/aggregate.groovy     |    2 +-
 .../operator/test_arithmetic_operators.groovy      |   22 +-
 .../test_array_functions_by_literal.groovy         |    2 +-
 .../cast_function/test_cast_function.groovy        |    6 +-
 .../conditional_functions/test_nullif.groovy       |    4 +-
 .../unique_with_mow_p0/test_pk_uk_case.groovy      |    8 +-
 40 files changed, 1349 insertions(+), 21778 deletions(-)

diff --git a/be/src/runtime/decimalv2_value.cpp b/be/src/runtime/decimalv2_value.cpp
index 9a3202a4ee..0256f6210f 100644
--- a/be/src/runtime/decimalv2_value.cpp
+++ b/be/src/runtime/decimalv2_value.cpp
@@ -353,7 +353,7 @@ int DecimalV2Value::parse_from_str(const char* decimal_str, int32_t length) {
 
     _value = StringParser::string_to_decimal<__int128>(decimal_str, length, PRECISION, SCALE,
                                                        &result);
-    if (result == StringParser::PARSE_FAILURE) {
+    if (result != StringParser::PARSE_SUCCESS) {
         error = E_DEC_BAD_NUM;
     }
     return error;
diff --git a/be/src/util/string_parser.hpp b/be/src/util/string_parser.hpp
index 5c776029da..4dbca73b23 100644
--- a/be/src/util/string_parser.hpp
+++ b/be/src/util/string_parser.hpp
@@ -32,6 +32,7 @@
 #include "common/compiler_util.h"
 #include "common/status.h"
 #include "runtime/primitive_type.h"
+#include "vec/data_types/data_type_decimal.h"
 
 namespace doris {
 
@@ -614,6 +615,13 @@ T StringParser::string_to_decimal(const char* s, int len, int type_precision, in
             // an exponent will be made later.
             if (LIKELY(type_precision > precision)) {
                 value = (value * 10) + (c - '0'); // Benchmarks are faster with parenthesis...
+            } else {
+                *result = StringParser::PARSE_OVERFLOW;
+                value = is_negative ? vectorized::min_decimal_value<vectorized::Decimal<T>>(
+                                              type_precision)
+                                    : vectorized::max_decimal_value<vectorized::Decimal<T>>(
+                                              type_precision);
+                return value;
             }
             DCHECK(value >= 0); // For some reason //DCHECK_GE doesn't work with __int128.
             ++precision;
diff --git a/be/src/vec/data_types/data_type_decimal.cpp b/be/src/vec/data_types/data_type_decimal.cpp
index d623d358a2..482b32f81a 100644
--- a/be/src/vec/data_types/data_type_decimal.cpp
+++ b/be/src/vec/data_types/data_type_decimal.cpp
@@ -156,14 +156,11 @@ MutableColumnPtr DataTypeDecimal<T>::create_column() const {
 }
 
 template <typename T>
-T DataTypeDecimal<T>::parse_from_string(const std::string& str) const {
+bool DataTypeDecimal<T>::parse_from_string(const std::string& str, T* res) const {
     StringParser::ParseResult result = StringParser::PARSE_SUCCESS;
-    T value = StringParser::string_to_decimal<__int128>(str.c_str(), str.size(), precision, scale,
-                                                        &result);
-    if (result != StringParser::PARSE_SUCCESS) {
-        LOG(WARNING) << "Failed to parse string of decimal";
-    }
-    return value;
+    *res = StringParser::string_to_decimal<__int128>(str.c_str(), str.size(), precision, scale,
+                                                     &result);
+    return result == StringParser::PARSE_SUCCESS;
 }
 
 DataTypePtr create_decimal(UInt64 precision_value, UInt64 scale_value, bool use_v2) {
diff --git a/be/src/vec/data_types/data_type_decimal.h b/be/src/vec/data_types/data_type_decimal.h
index 74f1df9230..ec207055a9 100644
--- a/be/src/vec/data_types/data_type_decimal.h
+++ b/be/src/vec/data_types/data_type_decimal.h
@@ -243,7 +243,7 @@ public:
 
     static T get_scale_multiplier(UInt32 scale);
 
-    T parse_from_string(const std::string& str) const;
+    bool parse_from_string(const std::string& str, T* res) const;
 
 private:
     const UInt32 precision;
diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp
index 46297a724f..b1724c9af8 100644
--- a/be/src/vec/exprs/vexpr.cpp
+++ b/be/src/vec/exprs/vexpr.cpp
@@ -120,88 +120,92 @@ void VExpr::close(doris::RuntimeState* state, VExprContext* context,
 
 Status VExpr::create_expr(doris::ObjectPool* pool, const doris::TExprNode& texpr_node,
                           VExpr** expr) {
-    switch (texpr_node.node_type) {
-    case TExprNodeType::BOOL_LITERAL:
-    case TExprNodeType::INT_LITERAL:
-    case TExprNodeType::LARGE_INT_LITERAL:
-    case TExprNodeType::FLOAT_LITERAL:
-    case TExprNodeType::DECIMAL_LITERAL:
-    case TExprNodeType::DATE_LITERAL:
-    case TExprNodeType::STRING_LITERAL:
-    case TExprNodeType::JSON_LITERAL:
-    case TExprNodeType::NULL_LITERAL: {
-        *expr = pool->add(new VLiteral(texpr_node));
-        break;
-    }
-    case TExprNodeType::ARRAY_LITERAL: {
-        *expr = pool->add(new VArrayLiteral(texpr_node));
-        break;
-    }
-    case TExprNodeType::MAP_LITERAL: {
-        *expr = pool->add(new VMapLiteral(texpr_node));
-        break;
-    }
-    case TExprNodeType::STRUCT_LITERAL: {
-        *expr = pool->add(new VStructLiteral(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::SLOT_REF: {
-        *expr = pool->add(new VSlotRef(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::COLUMN_REF: {
-        *expr = pool->add(new VColumnRef(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::COMPOUND_PRED: {
-        *expr = pool->add(new VcompoundPred(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::LAMBDA_FUNCTION_EXPR: {
-        *expr = pool->add(new VLambdaFunctionExpr(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::LAMBDA_FUNCTION_CALL_EXPR: {
-        *expr = pool->add(new VLambdaFunctionCallExpr(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::ARITHMETIC_EXPR:
-    case doris::TExprNodeType::BINARY_PRED:
-    case doris::TExprNodeType::FUNCTION_CALL:
-    case doris::TExprNodeType::COMPUTE_FUNCTION_CALL:
-    case doris::TExprNodeType::MATCH_PRED: {
-        *expr = pool->add(new VectorizedFnCall(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::CAST_EXPR: {
-        *expr = pool->add(new VCastExpr(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::IN_PRED: {
-        *expr = pool->add(new VInPredicate(texpr_node));
-        break;
-    }
-    case doris::TExprNodeType::CASE_EXPR: {
-        if (!texpr_node.__isset.case_expr) {
-            return Status::InternalError("Case expression not set in thrift node");
+    try {
+        switch (texpr_node.node_type) {
+        case TExprNodeType::BOOL_LITERAL:
+        case TExprNodeType::INT_LITERAL:
+        case TExprNodeType::LARGE_INT_LITERAL:
+        case TExprNodeType::FLOAT_LITERAL:
+        case TExprNodeType::DECIMAL_LITERAL:
+        case TExprNodeType::DATE_LITERAL:
+        case TExprNodeType::STRING_LITERAL:
+        case TExprNodeType::JSON_LITERAL:
+        case TExprNodeType::NULL_LITERAL: {
+            *expr = pool->add(new VLiteral(texpr_node));
+            break;
         }
-        *expr = pool->add(new VCaseExpr(texpr_node));
-        break;
-    }
-    case TExprNodeType::INFO_FUNC: {
-        *expr = pool->add(new VInfoFunc(texpr_node));
-        break;
-    }
-    case TExprNodeType::TUPLE_IS_NULL_PRED: {
-        *expr = pool->add(new VTupleIsNullPredicate(texpr_node));
-        break;
-    }
-    case TExprNodeType::SCHEMA_CHANGE_EXPR: {
-        *expr = pool->add(new VSchemaChangeExpr(texpr_node));
-        break;
-    }
-    default:
-        return Status::InternalError("Unknown expr node type: {}", texpr_node.node_type);
+        case TExprNodeType::ARRAY_LITERAL: {
+            *expr = pool->add(new VArrayLiteral(texpr_node));
+            break;
+        }
+        case TExprNodeType::MAP_LITERAL: {
+            *expr = pool->add(new VMapLiteral(texpr_node));
+            break;
+        }
+        case TExprNodeType::STRUCT_LITERAL: {
+            *expr = pool->add(new VStructLiteral(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::SLOT_REF: {
+            *expr = pool->add(new VSlotRef(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::COLUMN_REF: {
+            *expr = pool->add(new VColumnRef(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::COMPOUND_PRED: {
+            *expr = pool->add(new VcompoundPred(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::LAMBDA_FUNCTION_EXPR: {
+            *expr = pool->add(new VLambdaFunctionExpr(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::LAMBDA_FUNCTION_CALL_EXPR: {
+            *expr = pool->add(new VLambdaFunctionCallExpr(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::ARITHMETIC_EXPR:
+        case doris::TExprNodeType::BINARY_PRED:
+        case doris::TExprNodeType::FUNCTION_CALL:
+        case doris::TExprNodeType::COMPUTE_FUNCTION_CALL:
+        case doris::TExprNodeType::MATCH_PRED: {
+            *expr = pool->add(new VectorizedFnCall(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::CAST_EXPR: {
+            *expr = pool->add(new VCastExpr(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::IN_PRED: {
+            *expr = pool->add(new VInPredicate(texpr_node));
+            break;
+        }
+        case doris::TExprNodeType::CASE_EXPR: {
+            if (!texpr_node.__isset.case_expr) {
+                return Status::InternalError("Case expression not set in thrift node");
+            }
+            *expr = pool->add(new VCaseExpr(texpr_node));
+            break;
+        }
+        case TExprNodeType::INFO_FUNC: {
+            *expr = pool->add(new VInfoFunc(texpr_node));
+            break;
+        }
+        case TExprNodeType::TUPLE_IS_NULL_PRED: {
+            *expr = pool->add(new VTupleIsNullPredicate(texpr_node));
+            break;
+        }
+        case TExprNodeType::SCHEMA_CHANGE_EXPR: {
+            *expr = pool->add(new VSchemaChangeExpr(texpr_node));
+            break;
+        }
+        default:
+            return Status::InternalError("Unknown expr node type: {}", texpr_node.node_type);
+        }
+    } catch (const doris::Exception& e) {
+        return Status::Error(e.code(), e.to_string());
     }
     if (!(*expr)->data_type()) {
         return Status::InvalidArgument("Unknown expr type: {}", texpr_node.node_type);
diff --git a/be/src/vec/exprs/vliteral.cpp b/be/src/vec/exprs/vliteral.cpp
index 4f3921b278..69db0e2456 100644
--- a/be/src/vec/exprs/vliteral.cpp
+++ b/be/src/vec/exprs/vliteral.cpp
@@ -143,8 +143,15 @@ void VLiteral::init(const TExprNode& node) {
         case TYPE_DECIMALV2: {
             DCHECK_EQ(node.node_type, TExprNodeType::DECIMAL_LITERAL);
             DCHECK(node.__isset.decimal_literal);
-            DecimalV2Value value(node.decimal_literal.value);
-            field = DecimalField<Decimal128>(value.value(), value.scale());
+            DecimalV2Value value;
+            if (value.parse_from_str(node.decimal_literal.value.c_str(),
+                                     node.decimal_literal.value.size()) == E_DEC_OK) {
+                field = DecimalField<Decimal128>(value.value(), value.scale());
+            } else {
+                throw doris::Exception(doris::ErrorCode::INVALID_ARGUMENT,
+                                       "Invalid decimal(scale: {}) value: {}", value.scale(),
+                                       node.decimal_literal.value);
+            }
             break;
         }
         case TYPE_DECIMAL32: {
@@ -152,11 +159,17 @@ void VLiteral::init(const TExprNode& node) {
             DCHECK(node.__isset.decimal_literal);
             DataTypePtr type_ptr = create_decimal(node.type.types[0].scalar_type.precision,
                                                   node.type.types[0].scalar_type.scale, false);
-            auto val = typeid_cast<const DataTypeDecimal<Decimal32>*>(type_ptr.get())
-                               ->parse_from_string(node.decimal_literal.value);
-            auto scale =
-                    typeid_cast<const DataTypeDecimal<Decimal32>*>(type_ptr.get())->get_scale();
-            field = DecimalField<Decimal32>(val, scale);
+            Decimal32 val;
+            if (typeid_cast<const DataTypeDecimal<Decimal32>*>(type_ptr.get())
+                        ->parse_from_string(node.decimal_literal.value, &val)) {
+                auto scale =
+                        typeid_cast<const DataTypeDecimal<Decimal32>*>(type_ptr.get())->get_scale();
+                field = DecimalField<Decimal32>(val, scale);
+            } else {
+                throw doris::Exception(doris::ErrorCode::INVALID_ARGUMENT,
+                                       "Invalid value: {} for type {}", node.decimal_literal.value,
+                                       type_ptr->get_name());
+            }
             break;
         }
         case TYPE_DECIMAL64: {
@@ -164,11 +177,17 @@ void VLiteral::init(const TExprNode& node) {
             DCHECK(node.__isset.decimal_literal);
             DataTypePtr type_ptr = create_decimal(node.type.types[0].scalar_type.precision,
                                                   node.type.types[0].scalar_type.scale, false);
-            auto val = typeid_cast<const DataTypeDecimal<Decimal64>*>(type_ptr.get())
-                               ->parse_from_string(node.decimal_literal.value);
-            auto scale =
-                    typeid_cast<const DataTypeDecimal<Decimal64>*>(type_ptr.get())->get_scale();
-            field = DecimalField<Decimal64>(val, scale);
+            Decimal64 val;
+            if (typeid_cast<const DataTypeDecimal<Decimal64>*>(type_ptr.get())
+                        ->parse_from_string(node.decimal_literal.value, &val)) {
+                auto scale =
+                        typeid_cast<const DataTypeDecimal<Decimal64>*>(type_ptr.get())->get_scale();
+                field = DecimalField<Decimal64>(val, scale);
+            } else {
+                throw doris::Exception(doris::ErrorCode::INVALID_ARGUMENT,
+                                       "Invalid value: {} for type {}", node.decimal_literal.value,
+                                       type_ptr->get_name());
+            }
             break;
         }
         case TYPE_DECIMAL128I: {
@@ -176,11 +195,17 @@ void VLiteral::init(const TExprNode& node) {
             DCHECK(node.__isset.decimal_literal);
             DataTypePtr type_ptr = create_decimal(node.type.types[0].scalar_type.precision,
                                                   node.type.types[0].scalar_type.scale, false);
-            auto val = typeid_cast<const DataTypeDecimal<Decimal128I>*>(type_ptr.get())
-                               ->parse_from_string(node.decimal_literal.value);
-            auto scale =
-                    typeid_cast<const DataTypeDecimal<Decimal128I>*>(type_ptr.get())->get_scale();
-            field = DecimalField<Decimal128I>(val, scale);
+            Decimal128I val;
+            if (typeid_cast<const DataTypeDecimal<Decimal128I>*>(type_ptr.get())
+                        ->parse_from_string(node.decimal_literal.value, &val)) {
+                auto scale = typeid_cast<const DataTypeDecimal<Decimal128I>*>(type_ptr.get())
+                                     ->get_scale();
+                field = DecimalField<Decimal128I>(val, scale);
+            } else {
+                throw doris::Exception(doris::ErrorCode::INVALID_ARGUMENT,
+                                       "Invalid value: {} for type {}", node.decimal_literal.value,
+                                       type_ptr->get_name());
+            }
             break;
         }
         default: {
diff --git a/be/src/vec/io/io_helper.h b/be/src/vec/io/io_helper.h
index c24ba2aa41..003124630f 100644
--- a/be/src/vec/io/io_helper.h
+++ b/be/src/vec/io/io_helper.h
@@ -324,16 +324,18 @@ bool read_decimal_text_impl(T& x, ReadBuffer& buf, UInt32 precision, UInt32 scal
                 (const char*)buf.position(), buf.count(), precision, scale, &result);
         // only to match the is_all_read() check to prevent return null
         buf.position() = buf.end();
-        return result != StringParser::PARSE_FAILURE;
+        return result == StringParser::PARSE_SUCCESS || result == StringParser::PARSE_UNDERFLOW;
     } else {
-        auto dv = binary_cast<Int128, DecimalV2Value>(x.value);
-        auto ans = dv.parse_from_str((const char*)buf.position(), buf.count()) == 0;
+        StringParser::ParseResult result = StringParser::PARSE_SUCCESS;
+
+        x.value = StringParser::string_to_decimal<__int128>(buf.position(), buf.count(),
+                                                            DecimalV2Value::PRECISION,
+                                                            DecimalV2Value::SCALE, &result);
 
         // only to match the is_all_read() check to prevent return null
         buf.position() = buf.end();
 
-        x.value = dv.value();
-        return ans;
+        return result == StringParser::PARSE_SUCCESS || result == StringParser::PARSE_UNDERFLOW;
     }
 }
 
diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp
index 50d21552aa..ad3796477d 100644
--- a/be/src/vec/sink/vtablet_sink.cpp
+++ b/be/src/vec/sink/vtablet_sink.cpp
@@ -48,6 +48,7 @@
 #include "vec/columns/columns_number.h"
 #include "vec/core/block.h"
 #include "vec/core/types.h"
+#include "vec/data_types/data_type_decimal.h"
 #include "vec/exprs/vexpr.h"
 #include "vec/exprs/vexpr_context.h"
 
@@ -1375,6 +1376,33 @@ DecimalV2Value VOlapTableSink::_get_decimalv2_min_or_max(const TypeDescriptor& t
     return value;
 }
 
+template <typename DecimalType, bool IsMin>
+DecimalType VOlapTableSink::_get_decimalv3_min_or_max(const TypeDescriptor& type) {
+    std::map<int, typename DecimalType::NativeType>* pmap = nullptr;
+    if constexpr (std::is_same_v<DecimalType, vectorized::Decimal32>) {
+        pmap = IsMin ? &_min_decimal32_val : &_max_decimal32_val;
+    } else if constexpr (std::is_same_v<DecimalType, vectorized::Decimal64>) {
+        pmap = IsMin ? &_min_decimal64_val : &_max_decimal64_val;
+    } else {
+        pmap = IsMin ? &_min_decimal128_val : &_max_decimal128_val;
+    }
+
+    // found
+    auto iter = pmap->find(type.precision);
+    if (iter != pmap->end()) {
+        return iter->second;
+    }
+
+    typename DecimalType::NativeType value;
+    if constexpr (IsMin) {
+        value = vectorized::min_decimal_value<DecimalType>(type.precision);
+    } else {
+        value = vectorized::max_decimal_value<DecimalType>(type.precision);
+    }
+    pmap->emplace(type.precision, value);
+    return value;
+}
+
 Status VOlapTableSink::_validate_column(RuntimeState* state, const TypeDescriptor& type,
                                         bool is_nullable, vectorized::ColumnPtr column,
                                         size_t slot_index, Bitmap* filter_bitmap,
@@ -1510,6 +1538,47 @@ Status VOlapTableSink::_validate_column(RuntimeState* state, const TypeDescripto
         }
         break;
     }
+    case TYPE_DECIMAL32: {
+#define CHECK_VALIDATION_FOR_DECIMALV3(ColumnDecimalType, DecimalType)                             \
+    auto column_decimal = const_cast<vectorized::ColumnDecimal<vectorized::ColumnDecimalType>*>(   \
+            assert_cast<const vectorized::ColumnDecimal<vectorized::ColumnDecimalType>*>(          \
+                    real_column_ptr.get()));                                                       \
+    for (size_t j = 0; j < column->size(); ++j) {                                                  \
+        auto row = rows ? (*rows)[j] : j;                                                          \
+        if (row == last_invalid_row) {                                                             \
+            continue;                                                                              \
+        }                                                                                          \
+        if (need_to_validate(j, row)) {                                                            \
+            auto dec_val = column_decimal->get_data()[j];                                          \
+            bool invalid = false;                                                                  \
+            const auto& max_decimal =                                                              \
+                    _get_decimalv3_min_or_max<vectorized::DecimalType, false>(type);               \
+            const auto& min_decimal =                                                              \
+                    _get_decimalv3_min_or_max<vectorized::DecimalType, true>(type);                \
+            if (dec_val > max_decimal || dec_val < min_decimal) {                                  \
+                fmt::format_to(error_msg, "{}", "decimal value is not valid for definition");      \
+                fmt::format_to(error_msg, ", value={}", dec_val);                                  \
+                fmt::format_to(error_msg, ", precision={}, scale={}", type.precision, type.scale); \
+                fmt::format_to(error_msg, ", min={}, max={}; ", min_decimal, max_decimal);         \
+                invalid = true;                                                                    \
+            }                                                                                      \
+            if (invalid) {                                                                         \
+                last_invalid_row = row;                                                            \
+                RETURN_IF_ERROR(set_invalid_and_append_error_msg(row));                            \
+            }                                                                                      \
+        }                                                                                          \
+    }
+        CHECK_VALIDATION_FOR_DECIMALV3(Decimal32, Decimal32);
+        break;
+    }
+    case TYPE_DECIMAL64: {
+        CHECK_VALIDATION_FOR_DECIMALV3(Decimal64, Decimal64);
+        break;
+    }
+    case TYPE_DECIMAL128I: {
+        CHECK_VALIDATION_FOR_DECIMALV3(Decimal128I, Decimal128);
+        break;
+    }
     case TYPE_ARRAY: {
         const auto column_array =
                 assert_cast<const vectorized::ColumnArray*>(real_column_ptr.get());
diff --git a/be/src/vec/sink/vtablet_sink.h b/be/src/vec/sink/vtablet_sink.h
index 97fbaeac21..6f4492a37b 100644
--- a/be/src/vec/sink/vtablet_sink.h
+++ b/be/src/vec/sink/vtablet_sink.h
@@ -445,6 +445,9 @@ private:
     template <bool is_min>
     DecimalV2Value _get_decimalv2_min_or_max(const TypeDescriptor& type);
 
+    template <typename DecimalType, bool IsMin>
+    DecimalType _get_decimalv3_min_or_max(const TypeDescriptor& type);
+
     Status _validate_column(RuntimeState* state, const TypeDescriptor& type, bool is_nullable,
                             vectorized::ColumnPtr column, size_t slot_index, Bitmap* filter_bitmap,
                             bool* stop_processing, fmt::memory_buffer& error_prefix,
@@ -505,6 +508,13 @@ private:
     std::map<std::pair<int, int>, DecimalV2Value> _max_decimalv2_val;
     std::map<std::pair<int, int>, DecimalV2Value> _min_decimalv2_val;
 
+    std::map<int, int32_t> _max_decimal32_val;
+    std::map<int, int32_t> _min_decimal32_val;
+    std::map<int, int64_t> _max_decimal64_val;
+    std::map<int, int64_t> _min_decimal64_val;
+    std::map<int, int128_t> _max_decimal128_val;
+    std::map<int, int128_t> _min_decimal128_val;
+
     // Stats for this
     int64_t _validate_data_ns = 0;
     int64_t _send_data_ns = 0;
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 02ce00093e..42cc886e8d 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
@@ -1092,15 +1092,17 @@ public class ScalarType extends Type {
     }
 
     public static ScalarType getAssignmentCompatibleDecimalV2Type(ScalarType t1, ScalarType t2) {
-        int targetPrecision = Math.max(t1.decimalPrecision(), t2.decimalPrecision());
         int targetScale = Math.max(t1.decimalScale(), t2.decimalScale());
+        int targetPrecision = Math.max(t1.decimalPrecision() - t1.decimalScale(), t2.decimalPrecision()
+                - t2.decimalScale()) + targetScale;
         return ScalarType.createDecimalType(PrimitiveType.DECIMALV2,
                 targetPrecision, targetScale);
     }
 
     public static ScalarType getAssignmentCompatibleDecimalV3Type(ScalarType t1, ScalarType t2) {
-        int targetPrecision = Math.max(t1.decimalPrecision(), t2.decimalPrecision());
         int targetScale = Math.max(t1.decimalScale(), t2.decimalScale());
+        int targetPrecision = Math.max(t1.decimalPrecision() - t1.decimalScale(), t2.decimalPrecision()
+                - t2.decimalScale()) + targetScale;
         return ScalarType.createDecimalV3Type(targetPrecision, targetScale);
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
index 383556fc32..152df98ae1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
@@ -373,13 +373,14 @@ public class ArithmeticExpr extends Expr {
         if (pt1 == PrimitiveType.DOUBLE || pt2 == PrimitiveType.DOUBLE) {
             return Type.DOUBLE;
         } else if (pt1 == PrimitiveType.DECIMALV2 || pt2 == PrimitiveType.DECIMALV2) {
-            return Type.MAX_DECIMALV2_TYPE;
+            return pt1 == PrimitiveType.DECIMALV2 && pt2 == PrimitiveType.DECIMALV2
+                    ? Type.MAX_DECIMALV2_TYPE : Type.DOUBLE;
         } else if (pt1 == PrimitiveType.DECIMAL32 || pt2 == PrimitiveType.DECIMAL32) {
-            return Type.DECIMAL32;
+            return pt1 == PrimitiveType.DECIMAL32 && pt2 == PrimitiveType.DECIMAL32 ? Type.DECIMAL32 : Type.DOUBLE;
         } else if (pt1 == PrimitiveType.DECIMAL64 || pt2 == PrimitiveType.DECIMAL64) {
-            return Type.DECIMAL64;
+            return pt1 == PrimitiveType.DECIMAL64 && pt2 == PrimitiveType.DECIMAL64 ? Type.DECIMAL64 : Type.DOUBLE;
         } else if (pt1 == PrimitiveType.DECIMAL128 || pt2 == PrimitiveType.DECIMAL128) {
-            return Type.DECIMAL128;
+            return pt1 == PrimitiveType.DECIMAL128 && pt2 == PrimitiveType.DECIMAL128 ? Type.DECIMAL128 : Type.DOUBLE;
         } else if (pt1 == PrimitiveType.LARGEINT || pt2 == PrimitiveType.LARGEINT) {
             return Type.LARGEINT;
         } else {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java
index 45f1436e4e..04db33005a 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java
@@ -2057,9 +2057,9 @@ public class QueryPlanTest extends TestWithFeService {
                 + "PROPERTIES (\n"
                 + "\"replication_allocation\" = \"tag.location.default: 1\"\n"
                 + ")");
-        String sql = "explain insert into test.decimal_tb select 1, 10, 1, 1, 1;";
+        String sql = "explain insert into test.decimal_tb select 1, 1, 1, 1, 1;";
         String explainString = getSQLPlanOrErrorMsg(sql);
-        Assert.assertTrue(explainString.contains("1 | 10 | 1 | 1 | 1"));
+        Assert.assertTrue(explainString.contains("1 | 1 | 1 | 1 | 1"));
     }
 
     @Test
diff --git a/regression-test/data/datatype_p0/decimalv3/test_overflow.out b/regression-test/data/datatype_p0/decimalv3/test_overflow.out
deleted file mode 100644
index c9b9873cd7..0000000000
--- a/regression-test/data/datatype_p0/decimalv3/test_overflow.out
+++ /dev/null
@@ -1,19 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !select_all --
-11111111111111111111.100000000000000000	11111111111111111111.200000000000000000	11111111111111111111.300000000000000000	1.1000000000000000000000000000000000000	1.2000000000000000000000000000000000000	1.3000000000000000000000000000000000000	9
-
--- !select_check_overflow1 --
-\N	\N	\N	99999999999999999999.900000000000000000	\N
-
--- !select_check_overflow2 --
-1.1000000000000000000000000000000000000	111111111111111111111.000000000000000000	\N
-
--- !select_check_overflow3 --
-11111111111111111111.100000000000000000	\N
-
--- !select_not_check_overflow1 --
-99.999999999999999999999999999999999999	99.999999999999999999999999999999999999	1.1111111111111111E21	99999999999999999999.900000000000000000	99999999999999999999.999999999999999999
-
--- !select_not_check_overflow2 --
-1.1000000000000000000000000000000000000	111111111111111111111.000000000000000000	-15.9141183460469231731687303715884105728
-
diff --git a/regression-test/data/insert_p0/insert.out b/regression-test/data/insert_p0/insert.out
index 39756a4193..2ef1dbbbb0 100644
--- a/regression-test/data/insert_p0/insert.out
+++ b/regression-test/data/insert_p0/insert.out
@@ -97,108 +97,108 @@
 4	1.1	a	2016-01-01	2015-01-01T03:15:16	false	-123.22	-123456789012345678.012345678
 4	1.1	a	2016-01-01	2015-01-01T03:15:16	false	-123.22	-123456789012345678.012345678
 4	1.1	a	2016-01-01	2015-01-01T03:15:16	false	-123.22	-123456789012345678.012345678
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	123456789012345678.900123456
-25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	-123456789012345678.900123456
-25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	-123456789012345678.900123456
-25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	-123456789012345678.900123456
-25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	-123456789012345678.900123456
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+12	12.25	String1	1999-01-08	1999-01-08T02:05:06	true	123.22	\N
+25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	\N
+25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	\N
+25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	\N
+25	55.52	test	1952-01-05	1989-01-08T04:05:06	false	321.21	\N
 50	9.1	abc	2014-01-01	2015-01-01T03:15:16	true	123.22	123456789012345678.012345678
-60	27.25	String1	1999-01-08	1999-01-08T02:05:06	true	61.61	0
-60	27.25	String1	1999-01-08	1999-01-08T02:05:06	true	61.61	0
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	-987654321098765432.109876543
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	-999999999999999999.999999999
-100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-125	70.52000000000001	test	1952-01-05	1989-01-08T04:05:06	false	160.61	0
-125	70.52000000000001	test	1952-01-05	1989-01-08T04:05:06	false	160.61	0
-500	24.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	5.01	2
-500	24.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	5.01	2
-500	27.25	testing	1949-07-08	2002-01-07T01:05:06	true	-196.61	0
-500	27.25	testing	1949-07-08	2002-01-07T01:05:06	true	-196.61	0
-500	27.8788	string1	1922-04-02	2010-01-02T02:05:06	true	500	0
-500	27.8788	string1	1922-04-02	2010-01-02T02:05:06	true	500	0
-500	114.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-500	114.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0	0
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	987654321098765432.109876543
-4820	15.245	Again	1936-02-08	2005-01-09T04:05:06	false	166.91	0
-4820	15.245	Again	1936-02-08	2005-01-09T04:05:06	false	166.91	0
+60	27.25	String1	1999-01-08	1999-01-08T02:05:06	true	61.61	\N
+60	27.25	String1	1999-01-08	1999-01-08T02:05:06	true	61.61	\N
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	9.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	10.01	2.000000000
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.25	testing	1949-07-08	2002-01-07T01:05:06	true	-393.22	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	12.8788	string1	1922-04-02	2010-01-02T02:05:06	true	999.99	\N
+100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+100	99.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+125	70.52000000000001	test	1952-01-05	1989-01-08T04:05:06	false	160.61	\N
+125	70.52000000000001	test	1952-01-05	1989-01-08T04:05:06	false	160.61	\N
+500	24.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	5.01	2.000000000
+500	24.8777	STRING1	1923-04-08	2010-01-02T05:09:06	true	5.01	2.000000000
+500	27.25	testing	1949-07-08	2002-01-07T01:05:06	true	-196.61	\N
+500	27.25	testing	1949-07-08	2002-01-07T01:05:06	true	-196.61	\N
+500	27.8788	string1	1922-04-02	2010-01-02T02:05:06	true	500.00	\N
+500	27.8788	string1	1922-04-02	2010-01-02T02:05:06	true	500.00	\N
+500	114.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+500	114.8777	AGAIN	1987-04-09	2010-01-02T04:03:06	true	0.00	0E-9
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+964	0.245	Again	1936-02-08	2005-01-09T04:05:06	false	333.82	\N
+4820	15.245	Again	1936-02-08	2005-01-09T04:05:06	false	166.91	\N
+4820	15.245	Again	1936-02-08	2005-01-09T04:05:06	false	166.91	\N
 5000	67.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
 5000	67.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
 5000	67.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
 5000	67.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123	1
-5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0
-5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0
-5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0
-5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
-5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	123456789012345678.900123456
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5252	12.25	sample	1987-04-09	2010-01-02T04:03:06	true	123.00	1.000000000
+5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0E-9
+5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0E-9
+5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0E-9
+5748	67.87	Sample	1987-04-06	2010-01-02T04:03:06	true	181.18	0E-9
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0E-9
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0E-9
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0E-9
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	-999.99	0E-9
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
+5748	67.87	sample	1987-04-06	2010-01-02T04:03:06	true	181.18	\N
 6000	\N	\N	1987-04-06	\N	true	\N	\N
 6000	\N	\N	1987-04-06	\N	true	\N	\N
 6000	\N	\N	1987-04-06	\N	true	\N	\N
 6000	\N	\N	1987-04-06	\N	true	\N	\N
-8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	999999999999999999.999999999
-8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	999999999999999999.999999999
-8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	999999999999999999.999999999
-8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	999999999999999999.999999999
+8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
+8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
+8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
+8996	98.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
 25000	82.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
 25000	82.87	testing	\N	2010-01-02T04:03:06	\N	\N	\N
-26260	27.25	sample	1987-04-09	2010-01-02T04:03:06	true	61.5	1
-26260	27.25	sample	1987-04-09	2010-01-02T04:03:06	true	61.5	1
-28740	82.87	Sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
-28740	82.87	Sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	-500	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	-500	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
-28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0
+26260	27.25	sample	1987-04-09	2010-01-02T04:03:06	true	61.50	1.000000000
+26260	27.25	sample	1987-04-09	2010-01-02T04:03:06	true	61.50	1.000000000
+28740	82.87	Sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0E-9
+28740	82.87	Sample	1987-04-06	2010-01-02T04:03:06	true	90.59	0E-9
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	-500.00	0E-9
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	-500.00	0E-9
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	\N
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	\N
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	\N
+28740	82.87	sample	1987-04-06	2010-01-02T04:03:06	true	90.59	\N
 30000	\N	\N	1987-04-06	\N	true	\N	\N
 30000	\N	\N	1987-04-06	\N	true	\N	\N
-44980	113.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	0
-44980	113.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	0
+44980	113.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
+44980	113.8777	again	1987-04-09	2010-01-02T04:03:06	false	-0.01	\N
 
diff --git a/regression-test/data/nereids_arith_p0/date.out b/regression-test/data/nereids_arith_p0/date.out
index f79784b077..7327a4acda 100644
--- a/regression-test/data/nereids_arith_p0/date.out
+++ b/regression-test/data/nereids_arith_p0/date.out
@@ -1165,59 +1165,6 @@
 23	2201971174684465595	1.8384750871896913E-4	20120311
 24	4403938229404681240	9.192384949324504E-5	20120312
 
--- !sql_test_Date_LargeInt_2 --
-\N	\N
-1	0
-2	0
-3	0
-4	0
-5	0
-6	0
-7	0
-8	0
-9	0
-10	0
-11	0
-12	140173221323848800754997356878603747328
-13	6
-14	10181
-15	139847696493216
-16	371154043233002365088956416
-17	371154080126490512528179950
-18	371154117019978659947283184
-19	371154153913466807366386418
-20	371154190806954954785489652
-21	371154227700443102204592886
-22	371154043233002365109076728
-23	371154080126490512528179950
-24	371154117019978659947283184
-
--- !sql_test_Date_LargeInt_notn_2 --
-1	0
-2	0
-3	0
-4	0
-5	0
-6	0
-7	0
-8	0
-9	0
-10	0
-11	0
-12	0
-13	1742613662948936925785779951809024
-14	1742613662948936925645932255510528
-15	1742613662948936925785770762804000
-16	0
-17	0
-18	0
-19	0
-20	0
-21	0
-22	0
-23	0
-24	0
-
 -- !sql_test_Date_LargeInt_3 --
 \N	\N	\N	\N
 1	2228933	124982013	122753080
@@ -1377,59 +1324,6 @@
 23	2.0120312100000024E7	2.0120309899999976E7
 24	2.0120313200000048E7	2.0120310799999952E7
 
--- !sql_test_Date_Float_1 --
-\N	\N	\N	\N
-1	2012030.1299815848	2.0120300700184155E8	1.8415600061416626E-4
-2	4024060.4599631727	1.006015085009207E8	0.10018414258956909
-3	6036091.139852703	6.706767400163674E7	4.91023063659668E-4
-4	8048121.719926357	5.0300759250460275E7	0.1001841127872467
-5	1.00601525E7	4.024061E7	0.0
-6	1.2072184079705477E7	3.3533842000818174E7	4.909038543701172E-4
-7	1.408421466014725E7	2.874329620378113E7	0.14264678955078125
-8	1.6096246639852762E7	2.5150384625230063E7	0.5001840591430664
-9	1.810827762029445E7	2.2355899481117975E7	0.4330061674118042
-10	2.012031E7	2.012031E7	0.0
-11	2.2132342579705596E7	1.829119142173092E7	0.4639040231704712
-12	2.414437535941124E7	1.6766926000408888E7	4.906654357910156E-4
-13	2012030.1299815848	2.0120300700184155E8	1.8415600061416626E-4
-14	4024060.4599631727	1.006015085009207E8	0.10018414258956909
-15	6036091.139852703	6.706767400163674E7	4.91023063659668E-4
-16	8048121.719926357	5.0300759250460275E7	0.1001841127872467
-17	1.00601525E7	4.024061E7	0.0
-18	1.2072184079705477E7	3.3533842000818174E7	4.909038543701172E-4
-19	1.408421466014725E7	2.874329620378113E7	0.14264678955078125
-20	1.6096246639852762E7	2.5150384625230063E7	0.5001840591430664
-21	1.810827762029445E7	2.2355899481117975E7	0.4330061674118042
-22	2.012031E7	2.012031E7	0.0
-23	2.2132342579705596E7	1.829119142173092E7	0.4639040231704712
-24	2.414437535941124E7	1.6766926000408888E7	4.906654357910156E-4
-
--- !sql_test_Date_Float_notn_1 --
-1	2012030.1299815848	2.0120300700184155E8	1.8415600061416626E-4
-2	4024060.4599631727	1.006015085009207E8	0.10018414258956909
-3	6036091.139852703	6.706767400163674E7	4.91023063659668E-4
-4	8048121.719926357	5.0300759250460275E7	0.1001841127872467
-5	1.00601525E7	4.024061E7	0.0
-6	1.2072184079705477E7	3.3533842000818174E7	4.909038543701172E-4
-7	1.408421466014725E7	2.874329620378113E7	0.14264678955078125
-8	1.6096246639852762E7	2.5150384625230063E7	0.5001840591430664
-9	1.810827762029445E7	2.2355899481117975E7	0.4330061674118042
-10	2.012031E7	2.012031E7	0.0
-11	2.2132342579705596E7	1.829119142173092E7	0.4639040231704712
-12	2.414437535941124E7	1.6766926000408888E7	4.906654357910156E-4
-13	2012030.1299815848	2.0120300700184155E8	1.8415600061416626E-4
-14	4024060.4599631727	1.006015085009207E8	0.10018414258956909
-15	6036091.139852703	6.706767400163674E7	4.91023063659668E-4
-16	8048121.719926357	5.0300759250460275E7	0.1001841127872467
-17	1.00601525E7	4.024061E7	0.0
-18	1.2072184079705477E7	3.3533842000818174E7	4.909038543701172E-4
-19	1.408421466014725E7	2.874329620378113E7	0.14264678955078125
-20	1.6096246639852762E7	2.5150384625230063E7	0.5001840591430664
-21	1.810827762029445E7	2.2355899481117975E7	0.4330061674118042
-22	2.012031E7	2.012031E7	0.0
-23	2.2132342579705596E7	1.829119142173092E7	0.4639040231704712
-24	2.414437535941124E7	1.6766926000408888E7	4.906654357910156E-4
-
 -- !sql_test_Date_Float_2 --
 \N	\N
 1	\N
@@ -1536,59 +1430,6 @@
 23	1	20120311	20120310
 24	0	20120313	20120313
 
--- !sql_test_Date_Float_4 --
-\N	\N	\N	\N
-1	0	-19	-19
-2	0	-18	-18
-3	0	-17	-17
-4	0	-16	-16
-5	0	-15	-15
-6	0	-14	-14
-7	0	-13	-13
-8	0	-12	-12
-9	0	-11	-11
-10	0	-9	-9
-11	1	-9	-10
-12	0	-7	-7
-13	0	-19	-19
-14	0	-18	-18
-15	0	-17	-17
-16	0	-16	-16
-17	0	-15	-15
-18	0	-14	-14
-19	0	-13	-13
-20	0	-12	-12
-21	0	-11	-11
-22	0	-9	-9
-23	1	-9	-10
-24	0	-7	-7
-
--- !sql_test_Date_Float_notn_4 --
-1	0	-19	-19
-2	0	-18	-18
-3	0	-17	-17
-4	0	-16	-16
-5	0	-15	-15
-6	0	-14	-14
-7	0	-13	-13
-8	0	-12	-12
-9	0	-11	-11
-10	0	-9	-9
-11	1	-9	-10
-12	0	-7	-7
-13	0	-19	-19
-14	0	-18	-18
-15	0	-17	-17
-16	0	-16	-16
-17	0	-15	-15
-18	0	-14	-14
-19	0	-13	-13
-20	0	-12	-12
-21	0	-11	-11
-22	0	-9	-9
-23	1	-9	-10
-24	0	-7	-7
-
 -- !sql_test_Date_Double_0 --
 \N	\N	\N
 1	2.01203015244E7	2.01203004756E7
@@ -1801,59 +1642,6 @@
 23	16	20120311	20120295
 24	16	20120318	20120302
 
--- !sql_test_Date_Double_4 --
-\N	\N	\N	\N
-1	0	-19	-19
-2	0	-18	-18
-3	1	-17	-18
-4	0	-15	-15
-5	0	-13	-13
-6	2	-14	-16
-7	0	-9	-9
-8	4	-11	-15
-9	0	-3	-3
-10	2	-1	-3
-11	16	-9	-25
-12	16	-2	-18
-13	0	-19	-19
-14	0	-18	-18
-15	1	-17	-18
-16	0	-15	-15
-17	0	-13	-13
-18	2	-14	-16
-19	0	-9	-9
-20	4	-11	-15
-21	0	-3	-3
-22	2	-1	-3
-23	16	-9	-25
-24	16	-2	-18
-
--- !sql_test_Date_Double_notn_4 --
-1	0	-19	-19
-2	0	-18	-18
-3	1	-17	-18
-4	0	-15	-15
-5	0	-13	-13
-6	2	-14	-16
-7	0	-9	-9
-8	4	-11	-15
-9	0	-3	-3
-10	2	-1	-3
-11	16	-9	-25
-12	16	-2	-18
-13	0	-19	-19
-14	0	-18	-18
-15	1	-17	-18
-16	0	-15	-15
-17	0	-13	-13
-18	2	-14	-16
-19	0	-9	-9
-20	4	-11	-15
-21	0	-3	-3
-22	2	-1	-3
-23	16	-9	-25
-24	16	-2	-18
-
 -- !sql_test_Date_DecimalV2_0 --
 \N	\N	\N
 1	20120325.395000000	20120276.605000000
@@ -2066,59 +1854,6 @@
 23	515	20120575	20120060
 24	72	20121342	20121270
 
--- !sql_test_Date_DecimalV2_4 --
-\N	\N	\N	\N
-1	8	-3	-11
-2	34	-18	-52
-3	32	-1	-33
-4	64	-12	-76
-5	97	-15	-112
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	8	-3	-11
-14	34	-18	-52
-15	32	-1	-33
-16	64	-12	-76
-17	97	-15	-112
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
--- !sql_test_Date_DecimalV2_notn_4 --
-1	8	-3	-11
-2	34	-18	-52
-3	32	-1	-33
-4	64	-12	-76
-5	97	-15	-112
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	8	-3	-11
-14	34	-18	-52
-15	32	-1	-33
-16	64	-12	-76
-17	97	-15	-112
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
 -- !sql_test_Date_Decimal32V3_0 --
 \N	\N	\N
 1	20120313.012	20120288.988
@@ -2172,59 +1907,6 @@
 23	20120567.454	20120054.546
 24	20120579.565	20120044.435
 
--- !sql_test_Date_Decimal32V3_1 --
-\N	\N	\N	\N
-1	241685055.612	1675	0.000
-2	465241743.146	870	20120.302
-3	688798452.902	587	0.000
-4	912355184.880	443	20120.303
-5	1135911939.080	356	0.000
-6	1359468715.502	297	20120.304
-7	1583025514.146	255	0.000
-8	1806582335.012	224	20120.305
-9	2030139178.100	199	0.000
-10	2253696043.410	179	20120.306
-11	2477252930.942	163	0.000
-12	2700809840.696	149	20120.307
-13	2924365028.544	138	0.000
-14	3147921849.410	128	20120.308
-15	3371478692.498	120	0.000
-16	3595035557.808	112	20120.309
-17	3818592445.340	106	0.000
-18	4042149355.094	100	20120.310
-19	4265706287.070	94	0.000
-20	4489263241.268	90	20120.311
-21	4712820217.688	85	0.000
-22	4936377216.330	82	20120.312
-23	5159934237.194	78	0.000
-24	5383491280.280	75	20120.301
-
--- !sql_test_Date_Decimal32V3_notn_1 --
-1	241685055.612	1675	232021.504
-2	465241743.146	870	32.561
-3	688798452.902	587	1260615.93
-4	912355184.880	443	-485796.965
-5	1135911939.080	356	232021.504
-6	1359468715.502	297	32.561
-7	1583025514.146	255	231671.040
-8	1806582335.012	224	32.561
-9	2030139178.100	199	-1009846.52
-10	2253696043.410	179	32.558
-11	2477252930.942	163	231671.041
-12	2700809840.696	149	32.561
-13	2924365028.544	138	232480.512
-14	3147921849.410	128	32.561
-15	3371478692.498	120	1260615.93
-16	3595035557.808	112	-485796.965
-17	3818592445.340	106	-842817.152
-18	4042149355.094	100	32.558
-19	4265706287.070	94	-529716.912
-20	4489263241.268	90	21.994
-21	4712820217.688	85	-842816.992
-22	4936377216.330	82	32.558
-23	5159934237.194	78	-529716.912
-24	5383491280.280	75	21.994
-
 -- !sql_test_Date_Decimal32V3_2 --
 \N	\N
 1	1676691
@@ -2331,59 +2013,6 @@
 23	0	20120567	20120567
 24	8	20120571	20120563
 
--- !sql_test_Date_Decimal32V3_4 --
-\N	\N	\N	\N
-1	12	-19	-31
-2	6	-1	-7
-3	34	-17	-51
-4	32	-3	-35
-5	48	-7	-55
-6	66	-13	-79
-7	66	-1	-67
-8	80	-3	-83
-9	100	-11	-111
-10	112	-10	-122
-11	115	-1	-116
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
--- !sql_test_Date_Decimal32V3_notn_4 --
-1	12	-19	-31
-2	6	-1	-7
-3	34	-17	-51
-4	32	-3	-35
-5	48	-7	-55
-6	66	-13	-79
-7	66	-1	-67
-8	80	-3	-83
-9	100	-11	-111
-10	112	-10	-122
-11	115	-1	-116
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
 -- !sql_test_Date_Decimal64V3_0 --
 \N	\N	\N
 1	20121535.01234	20119066.98766
@@ -2437,59 +2066,6 @@
 23	20145989.45676	20094632.54324
 24	20147101.56787	20093522.43213
 
--- !sql_test_Date_Decimal64V3_1 --
-\N	\N	\N	\N
-1	24828699718.51434	0	128849.01890
-2	47184592041.28190	0	214748.36484
-3	69540486586.27168	0	300647.71078
-4	91896383353.48368	0	386547.05672
-5	114252282342.91790	0	472446.40266
-6	136608183554.57434	0	558345.74860
-7	158964086988.45300	0	644245.09454
-8	181319992644.55388	0	730144.44048
-9	203675900522.87698	0	816043.78642
-10	226031810623.42230	0	901943.13236
-11	248387722946.18984	0	987842.47830
-12	270743637491.17960	0	0.00024
-13	293099379450.24366	0	0.00000
-14	315455285106.34454	0	0.00000
-15	337811192984.66764	0	0.00000
-16	360167103085.21296	0	0.00000
-17	382523015407.98050	0	0.00000
-18	404878929952.97026	0	0.00000
-19	427234846720.18224	0	0.00000
-20	449590765709.61644	0	0.00000
-21	471946686921.27286	0	0.00000
-22	494302610355.15150	0	0.00000
-23	516658536011.25236	0	0.00000
-24	539014463889.57544	0	0.00000
-
--- !sql_test_Date_Decimal64V3_notn_1 --
-1	24828699718.51434	0	1234.01234
-2	47184592041.28190	0	2345.12345
-3	69540486586.27168	0	3456.23456
-4	91896383353.48368	0	4567.34567
-5	114252282342.91790	0	5678.45678
-6	136608183554.57434	0	6789.56789
-7	158964086988.45300	0	7900.67900
-8	181319992644.55388	0	9011.79011
-9	203675900522.87698	0	10122.90122
-10	226031810623.42230	0	11234.01233
-11	248387722946.18984	0	12345.12344
-12	270743637491.17960	0	13456.23455
-13	293099379450.24366	0	14567.34566
-14	315455285106.34454	0	15678.45677
-15	337811192984.66764	0	16789.56788
-16	360167103085.21296	0	17900.67899
-17	382523015407.98050	0	19011.79010
-18	404878929952.97026	0	20122.90121
-19	427234846720.18224	0	21234.01232
-20	449590765709.61644	0	22345.12343
-21	471946686921.27286	0	23456.23454
-22	494302610355.15150	0	24567.34565
-23	516658536011.25236	0	25678.45676
-24	539014463889.57544	0	26789.56787
-
 -- !sql_test_Date_Decimal64V3_2 --
 \N	\N
 1	16304
@@ -2596,59 +2172,6 @@
 23	70	20145919	20145849
 24	160	20146941	20146781
 
--- !sql_test_Date_Decimal64V3_4 --
-\N	\N	\N	\N
-1	109	-1	-110
-2	110	-1	-111
-3	111	-1	-112
-4	112	-1	-113
-5	113	-1	-114
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
--- !sql_test_Date_Decimal64V3_notn_4 --
-1	109	-1	-110
-2	110	-1	-111
-3	111	-1	-112
-4	112	-1	-113
-5	113	-1	-114
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
 -- !sql_test_Date_Decimal128V3_0 --
 \N	\N	\N
 1	32465979.01234500	7774622.98765500
@@ -2702,59 +2225,6 @@
 23	276910433.45678700	-236669811.45678700
 24	288021545.56789800	-247780921.56789800
 
--- !sql_test_Date_Decimal128V3_1 --
-\N	\N	\N	\N
-1	248398757657463.11584500	0	45679011.34567800
-2	471957681114250.00371200	0	90123455.79012200
-3	695516626793259.11380100	0	156790122.45678800
-4	919075594694490.44611200	0	179012344.67901000
-5	1142634584817944.00064500	0	245679011.34567600
-6	1366193597163619.77740000	0	12345678.01234500
-7	1589752631731517.77637700	0	79012344.67901100
-8	1813311688521637.99757600	0	123456789.12345500
-9	2036870767533980.44099700	0	145679011.34567700
-10	2260429868768545.10664000	0	212345678.01234300
-11	2483988992225331.99450500	0	234567900.23456500
-12	2707548137904341.10459200	0	67901233.56790000
-13	2931105557657436.28877700	0	112345678.01234400
-14	3154664614447556.50997600	0	134567900.23456600
-15	3378223693459898.95339700	0	201234566.90123200
-16	3601782794694463.61904000	0	223456789.12345400
-17	3825341918151250.50690500	0	267901233.56789800
-18	4048901063830259.61699200	0	34567900.23456700
-19	4272460231731490.94930100	0	56790122.45678900
-20	4496019421854944.50383200	0	101234566.90123300
-21	4719578634200620.28058500	0	167901233.56789900
-22	4943137868768518.27956000	0	190123455.79012100
-23	5166697125558638.50075700	0	256790122.45678700
-24	5390256404570980.94417600	0	0E-8
-
--- !sql_test_Date_Decimal128V3_notn_1 --
-1	248398757657463.11584500	0	0E-8
-2	471957681114250.00371200	0	12345678.01234500
-3	695516626793259.11380100	0	23456789.12345600
-4	919075594694490.44611200	0	34567900.23456700
-5	1142634584817944.00064500	0	45679011.34567800
-6	1366193597163619.77740000	0	56790122.45678900
-7	1589752631731517.77637700	0	67901233.56790000
-8	1813311688521637.99757600	0	79012344.67901100
-9	2036870767533980.44099700	0	90123455.79012200
-10	2260429868768545.10664000	0	101234566.90123300
-11	2483988992225331.99450500	0	112345678.01234400
-12	2707548137904341.10459200	0	123456789.12345500
-13	2931105557657436.28877700	0	134567900.23456600
-14	3154664614447556.50997600	0	145679011.34567700
-15	3378223693459898.95339700	0	156790122.45678800
-16	3601782794694463.61904000	0	167901233.56789900
-17	3825341918151250.50690500	0	179012344.67901000
-18	4048901063830259.61699200	0	190123455.79012100
-19	4272460231731490.94930100	0	201234566.90123200
-20	4496019421854944.50383200	0	212345678.01234300
-21	4719578634200620.28058500	0	223456789.12345400
-22	4943137868768518.27956000	0	234567900.23456500
-23	5166697125558638.50075700	0	245679011.34567600
-24	5390256404570980.94417600	0	256790122.45678700
-
 -- !sql_test_Date_Decimal128V3_2 --
 \N	\N
 1	1
@@ -2861,59 +2331,6 @@
 23	16908898	260001535	243092637
 24	20119600	267901945	247782345
 
--- !sql_test_Date_Decimal128V3_4 --
-\N	\N	\N	\N
-1	109	-1	-110
-2	110	-1	-111
-3	111	-1	-112
-4	112	-1	-113
-5	113	-1	-114
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
--- !sql_test_Date_Decimal128V3_notn_4 --
-1	109	-1	-110
-2	110	-1	-111
-3	111	-1	-112
-4	112	-1	-113
-5	113	-1	-114
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
 -- !sql_test_Date_Char_0 --
 \N	\N	\N
 1	\N	\N
@@ -3126,59 +2543,6 @@
 23	580	20124663	20124083
 24	568	20126719	20126151
 
--- !sql_test_Date_Char_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_Date_Char_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_Date_Varchar_0 --
 \N	\N	\N
 1	\N	\N
@@ -3391,59 +2755,6 @@
 23	65681	20128767	20063086
 24	65672	20159485	20093813
 
--- !sql_test_Date_Varchar_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_Date_Varchar_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_Date_String_0 --
 \N	\N	\N
 1	\N	\N
@@ -3656,59 +2967,6 @@
 23	65571	20393727	20328156
 24	196768	20402943	20206175
 
--- !sql_test_Date_String_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_Date_String_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_Date_Date_0 --
 \N	\N	\N
 1	40240602	0
@@ -6200,59 +5458,6 @@
 23	2201971186833533373967095	183.84750973332203	92751647976
 24	4403938255913701917197240	91.92385004657052	202212497417
 
--- !sql_test_DateTime_LargeInt_2 --
-\N	\N
-1	867753287971371017662291
-2	217007497283119165234150
-3	54251874320779791308538
-4	13558356894176520439230
-5	3375754165488847946095
-6	3465792723312624267165787
-7	867753287971371017662291
-8	217007497283119165234150
-9	54251874320779791308538
-10	13558356894176520439230
-11	3375754165488847946095
-12	2579573034850483451839117615169627
-13	139838934423496
-14	1742613662948936925785771179872288
-15	139838934243456
-16	371154043417488252570125641711616
-17	371154080683637523786429537478150
-18	371154117949786794982613133244784
-19	371154155215936066178796729011418
-20	371154192482085337374980324778052
-21	371154229748234608571163920544686
-22	371154043417488252590245953832728
-23	371154080683637523786429537478150
-24	371154117949786794982613133244784
-
--- !sql_test_DateTime_LargeInt_notn_2 --
-1	1734639578971277686398441
-2	433978101078090911365057
-3	108503748641559582617076
-4	27116713788353040878461
-5	6769955075051405443807
-6	1678653710707569197239
-7	1734639578971277686398441
-8	433978101078090911365057
-9	108503748641559582617076
-10	27116713788353040878461
-11	6769955075051405443807
-12	1678653710707569197239
-13	1742613662948936925785771189986848
-14	1742613662948936925645932255510528
-15	1742613662948936925785779161844960
-16	139846906492928
-17	371154062050562888188337739594833
-18	371154099316712159384521335361467
-19	371154136582861430580704931128101
-20	371154173849010701776888526894735
-21	371154211115159972973072122661369
-22	371154248381309244169255718428003
-23	371154062050562888188337739594833
-24	371154099316712159384521335361467
-
 -- !sql_test_DateTime_LargeInt_3 --
 \N	\N	\N	\N
 1	106958929	20120301141717	20120194182788
@@ -6412,59 +5617,6 @@
 23	2.01203111110121E13	2.01203111110099E13
 24	2.01203121211132E13	2.01203121211108E13
 
--- !sql_test_DateTime_Float_1 --
-\N	\N	\N	\N
-1	2.0120301309816848E12	2.0120300710185156E14	0.05584884434938431
-2	4.0240604639835728E12	1.006015086014307E14	0.14079716801643372
-3	6.036091148913603E12	6.706767410231341E13	0.12184399366378784
-4	8.048121736047957E12	5.030075935122027E13	0.11069381237030029
-5	1.00601525252025E13	4.024061010081E13	0.0
-6	1.2072184116009078E13	3.3533842101661504E13	0.3014305830001831
-7	1.4084214709572148E13	2.8743296304648277E13	0.19318246841430664
-8	1.6096246704419164E13	2.5150384726115062E13	0.05048710107803345
-9	1.810827770202255E13	2.2355899582016867E13	0.7812881469726562
-10	2.012031010091E13	2.012031010091E13	0.0
-11	2.21323427018177E13	1.8291191522650008E13	0.010709524154663086
-12	2.4144375504745645E13	1.676692610133555E13	0.6606037616729736
-13	2.0120301309816848E12	2.0120300710185156E14	0.05584884434938431
-14	4.0240604639835728E12	1.006015086014307E14	0.14079716801643372
-15	6.036091148913603E12	6.706767410231341E13	0.12184399366378784
-16	8.048121736047957E12	5.030075935122027E13	0.11069381237030029
-17	1.00601525252025E13	4.024061010081E13	0.0
-18	1.2072184116009078E13	3.3533842101661504E13	0.3014305830001831
-19	1.4084214709572148E13	2.8743296304648277E13	0.19318246841430664
-20	1.6096246704419164E13	2.5150384726115062E13	0.05048710107803345
-21	1.810827770202255E13	2.2355899582016867E13	0.7812881469726562
-22	2.012031010091E13	2.012031010091E13	0.0
-23	2.21323427018177E13	1.8291191522650008E13	0.010709524154663086
-24	2.4144375504745645E13	1.676692610133555E13	0.6606037616729736
-
--- !sql_test_DateTime_Float_notn_1 --
-1	2.0120301309816848E12	2.0120300710185156E14	0.05584884434938431
-2	4.0240604639835728E12	1.006015086014307E14	0.14079716801643372
-3	6.036091148913603E12	6.706767410231341E13	0.12184399366378784
-4	8.048121736047957E12	5.030075935122027E13	0.11069381237030029
-5	1.00601525252025E13	4.024061010081E13	0.0
-6	1.2072184116009078E13	3.3533842101661504E13	0.3014305830001831
-7	1.4084214709572148E13	2.8743296304648277E13	0.19318246841430664
-8	1.6096246704419164E13	2.5150384726115062E13	0.05048710107803345
-9	1.810827770202255E13	2.2355899582016867E13	0.7812881469726562
-10	2.012031010091E13	2.012031010091E13	0.0
-11	2.21323427018177E13	1.8291191522650008E13	0.010709524154663086
-12	2.4144375504745645E13	1.676692610133555E13	0.6606037616729736
-13	2.0120301309816848E12	2.0120300710185156E14	0.05584884434938431
-14	4.0240604639835728E12	1.006015086014307E14	0.14079716801643372
-15	6.036091148913603E12	6.706767410231341E13	0.12184399366378784
-16	8.048121736047957E12	5.030075935122027E13	0.11069381237030029
-17	1.00601525252025E13	4.024061010081E13	0.0
-18	1.2072184116009078E13	3.3533842101661504E13	0.3014305830001831
-19	1.4084214709572148E13	2.8743296304648277E13	0.19318246841430664
-20	1.6096246704419164E13	2.5150384726115062E13	0.05048710107803345
-21	1.810827770202255E13	2.2355899582016867E13	0.7812881469726562
-22	2.012031010091E13	2.012031010091E13	0.0
-23	2.21323427018177E13	1.8291191522650008E13	0.010709524154663086
-24	2.4144375504745645E13	1.676692610133555E13	0.6606037616729736
-
 -- !sql_test_DateTime_Float_2 --
 \N	\N
 1	\N
@@ -6571,59 +5723,6 @@
 23	1	20120311111011	20120311111010
 24	0	20120312121113	20120312121113
 
--- !sql_test_DateTime_Float_4 --
-\N	\N	\N	\N
-1	0	81	81
-2	0	6	6
-3	0	-69	-69
-4	0	112	112
-5	0	37	37
-6	0	-38	-38
-7	0	-113	-113
-8	0	68	68
-9	0	-7	-7
-10	0	-81	-81
-11	1	99	98
-12	0	25	25
-13	0	81	81
-14	0	6	6
-15	0	-69	-69
-16	0	112	112
-17	0	37	37
-18	0	-38	-38
-19	0	-113	-113
-20	0	68	68
-21	0	-7	-7
-22	0	-81	-81
-23	1	99	98
-24	0	25	25
-
--- !sql_test_DateTime_Float_notn_4 --
-1	0	81	81
-2	0	6	6
-3	0	-69	-69
-4	0	112	112
-5	0	37	37
-6	0	-38	-38
-7	0	-113	-113
-8	0	68	68
-9	0	-7	-7
-10	0	-81	-81
-11	1	99	98
-12	0	25	25
-13	0	81	81
-14	0	6	6
-15	0	-69	-69
-16	0	112	112
-17	0	37	37
-18	0	-38	-38
-19	0	-113	-113
-20	0	68	68
-21	0	-7	-7
-22	0	-81	-81
-23	1	99	98
-24	0	25	25
-
 -- !sql_test_DateTime_Double_0 --
 \N	\N	\N
 1	2.0120301010001523E13	2.0120301010000477E13
@@ -6836,59 +5935,6 @@
 23	0	20120311111027	20120311111027
 24	16	20120312121118	20120312121102
 
--- !sql_test_DateTime_Double_4 --
-\N	\N	\N	\N
-1	0	81	81
-2	0	6	6
-3	1	-69	-70
-4	0	113	113
-5	0	39	39
-6	2	-38	-40
-7	4	-113	-117
-8	4	69	65
-9	8	-7	-15
-10	10	-81	-91
-11	0	115	115
-12	16	30	14
-13	0	81	81
-14	0	6	6
-15	1	-69	-70
-16	0	113	113
-17	0	39	39
-18	2	-38	-40
-19	4	-113	-117
-20	4	69	65
-21	8	-7	-15
-22	10	-81	-91
-23	0	115	115
-24	16	30	14
-
--- !sql_test_DateTime_Double_notn_4 --
-1	0	81	81
-2	0	6	6
-3	1	-69	-70
-4	0	113	113
-5	0	39	39
-6	2	-38	-40
-7	4	-113	-117
-8	4	69	65
-9	8	-7	-15
-10	10	-81	-91
-11	0	115	115
-12	16	30	14
-13	0	81	81
-14	0	6	6
-15	1	-69	-70
-16	0	113	113
-17	0	39	39
-18	2	-38	-40
-19	4	-113	-117
-20	4	69	65
-21	8	-7	-15
-22	10	-81	-91
-23	0	115	115
-24	16	30	14
-
 -- !sql_test_DateTime_DecimalV2_0 --
 \N	\N	\N
 1	20120301010025.395000000	20120301009976.605000000
@@ -7101,59 +6147,6 @@
 23	259	20120311111531	20120311111272
 24	1032	20120312121182	20120312120150
 
--- !sql_test_DateTime_DecimalV2_4 --
-\N	\N	\N	\N
-1	16	89	73
-2	2	38	36
-3	48	-69	-117
-4	64	116	52
-5	33	101	68
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	16	89	73
-14	2	38	36
-15	48	-69	-117
-16	64	116	52
-17	33	101	68
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
--- !sql_test_DateTime_DecimalV2_notn_4 --
-1	16	89	73
-2	2	38	36
-3	48	-69	-117
-4	64	116	52
-5	33	101	68
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	16	89	73
-14	2	38	36
-15	48	-69	-117
-16	64	116	52
-17	33	101	68
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
 -- !sql_test_DateTime_Decimal32V3_0 --
 \N	\N	\N
 1	20120301010013.012	20120301009988.988
@@ -7207,59 +6200,6 @@
 23	20120311111267.454	20120311110754.546
 24	20120312121379.565	20120312120844.435
 
--- !sql_test_DateTime_Decimal32V3_1 --
-\N	\N	\N	\N
-1	241685055732132.012	1675016734	0.000
-2	465241743610818.546	870142369	0.000
-3	688798453935969.502	587728662	0.000
-4	912355186707584.880	443716044	0.000
-5	1135911941925664.680	356389135	0.000
-6	1359468719590208.902	297783031	0.000
-7	1583025519701217.546	255729772	0.000
-8	1806582342258690.612	224084331	0.000
-9	2030139187262628.100	199408415	0.000
-10	2253696054713030.010	179627983	0.000
-11	2477252944609896.342	163417676	0.000
-12	2700809856953227.096	149890951	0.000
-13	2924365029997585.344	138432277	0.000
-14	3147921852555058.410	128601208	0.000
-15	3371478697558995.898	120073899	0.000
-16	3595035565009397.808	112607129	0.000
-17	3818592454906264.140	106014632	0.000
-18	4042149367249594.894	100151349	0.001
-19	4265706302039390.070	94902632	0.000
-20	4489263259275649.668	90176666	0.000
-21	4712820238958373.688	85899062	0.000
-22	4936377241087562.130	82008902	0.001
-23	5159934265663214.994	78455828	0.000
-24	5383491312685332.280	75197847	0.000
-
--- !sql_test_DateTime_Decimal32V3_notn_1 --
-1	241685055732132.012	1675016734	-374022.048
-2	465241743610818.546	870142369	21.994
-3	688798453935969.502	587728662	0.006
-4	912355186707584.880	443716044	0.000
-5	1135911941925664.680	356389135	0.000
-6	1359468719590208.902	297783031	0.000
-7	1583025519701217.546	255729772	0.000
-8	1806582342258690.612	224084331	0.010
-9	2030139187262628.100	199408415	0.000
-10	2253696054713030.010	179627983	0.000
-11	2477252944609896.342	163417676	0.000
-12	2700809856953227.096	149890951	0.000
-13	2924365029997585.344	138432277	0.000
-14	3147921852555058.410	128601208	0.000
-15	3371478697558995.898	120073899	0.000
-16	3595035565009397.808	112607129	0.000
-17	3818592454906264.140	106014632	0.000
-18	4042149367249594.894	100151349	0.000
-19	4265706302039390.070	94902632	0.000
-20	4489263259275649.668	90176666	0.000
-21	4712820238958373.688	85899062	0.000
-22	4936377241087562.130	82008902	0.000
-23	5159934265663214.994	78455828	0.000
-24	5383491312685332.280	75197847	0.000
-
 -- !sql_test_DateTime_Decimal32V3_2 --
 \N	\N
 1	1676691750833
@@ -7366,59 +6306,6 @@
 23	256	20120311111011	20120311110755
 24	264	20120312121115	20120312120851
 
--- !sql_test_DateTime_Decimal32V3_4 --
-\N	\N	\N	\N
-1	0	93	93
-2	6	23	17
-3	34	-69	-103
-4	32	125	93
-5	32	61	29
-6	66	-37	-103
-7	14	-49	-63
-8	64	93	29
-9	96	-3	-99
-10	32	-2	-34
-11	99	123	24
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
--- !sql_test_DateTime_Decimal32V3_notn_4 --
-1	0	93	93
-2	6	23	17
-3	34	-69	-103
-4	32	125	93
-5	32	61	29
-6	66	-37	-103
-7	14	-49	-63
-8	64	93	29
-9	96	-3	-99
-10	32	-2	-34
-11	99	123	24
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
 -- !sql_test_DateTime_Decimal64V3_0 --
 \N	\N	\N
 1	20120301011235.01234	20120301008766.98766
@@ -7472,59 +6359,6 @@
 23	20120311136689.45676	20120311085332.54324
 24	20120312147901.56787	20120312094322.43213
 
--- !sql_test_DateTime_Decimal64V3_1 --
-\N	\N	\N	\N
-1	24828699730855697.41234	163047	0.00000
-2	47184592088423571.59190	85796	0.00000
-3	69540486690660332.41568	58214	0.00000
-4	91896383537565979.88368	44052	0.00000
-5	114252282629140513.99590	35432	0.00000
-6	136608183965383934.75234	29634	0.00000
-7	158964087546296242.15300	25466	0.00000
-8	181319993371877436.19788	22326	0.00000
-9	203675901442127516.88698	19876	0.00000
-10	226031811757046484.22030	17910	0.00000
-11	248387724316634338.19784	16298	0.00000
-12	270743639120891078.81960	14952	0.00000
-13	293099379595931683.94566	13811	0.00000
-14	315455285421512877.99054	12833	0.00000
-15	337811193491762958.67964	11983	0.00000
-16	360167103806681926.01296	11239	0.00000
-17	382523016366269779.99050	10583	0.00000
-18	404878931170526520.61226	9998	0.00000
-19	427234848219452147.87824	9475	0.00000
-20	449590767513046661.78844	9004	0.00000
-21	471946689051310062.34286	8577	0.00000
-22	494302612834242349.54150	8189	0.00000
-23	516658538861843523.38436	7835	0.00000
-24	539014467134113583.87144	7510	0.00000
-
--- !sql_test_DateTime_Decimal64V3_notn_1 --
-1	24828699730855697.41234	163047	0.00000
-2	47184592088423571.59190	85796	201.44096
-3	69540486690660332.41568	58214	201.67847
-4	91896383537565979.88368	44052	202.15348
-5	114252282629140513.99590	35432	203.10349
-6	136608183965383934.75234	29634	205.00350
-7	158964087546296242.15300	25466	208.80351
-8	181319993371877436.19788	22326	216.40352
-9	203675901442127516.88698	19876	231.60353
-10	226031811757046484.22030	17910	262.00354
-11	248387724316634338.19784	16298	322.80355
-12	270743639120891078.81960	14952	444.40356
-13	293099379595931683.94566	13811	687.60357
-14	315455285421512877.99054	12833	201.44096
-15	337811193491762958.67964	11983	201.67847
-16	360167103806681926.01296	11239	202.15348
-17	382523016366269779.99050	10583	203.10349
-18	404878931170526520.61226	9998	205.00350
-19	427234848219452147.87824	9475	208.80351
-20	449590767513046661.78844	9004	216.40352
-21	471946689051310062.34286	8577	231.60353
-22	494302612834242349.54150	8189	262.00354
-23	516658538861843523.38436	7835	322.80355
-24	539014467134113583.87144	7510	444.40356
-
 -- !sql_test_DateTime_Decimal64V3_2 --
 \N	\N
 1	16304944092
@@ -7631,59 +6465,6 @@
 23	1090	20120311135599	20120311134509
 24	0	20120312147901	20120312147901
 
--- !sql_test_DateTime_Decimal64V3_4 --
-\N	\N	\N	\N
-1	81	127	46
-2	6	127	121
-3	59	-1	-60
-4	112	127	15
-5	37	127	90
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
--- !sql_test_DateTime_Decimal64V3_notn_4 --
-1	81	127	46
-2	6	127	121
-3	59	-1	-60
-4	112	127	15
-5	37	127	90
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
 -- !sql_test_DateTime_Decimal128V3_0 --
 \N	\N	\N
 1	20120313355679.01234500	20120288664322.98765500
@@ -7737,59 +6518,6 @@
 23	20120567901133.45678700	20120054320888.54321300
 24	20120580022345.56789800	20120044219878.43210200
 
--- !sql_test_DateTime_Decimal128V3_1 --
-\N	\N	\N	\N
-1	248398757780932241646.46234500	0	20120278563312.87654400
-2	471957681585778378671.71251200	0	20120268462302.76543300
-3	695516627837313404585.62710100	0	20120258361292.65432200
-4	919075596535537319388.20611200	0	20120248260282.54321100
-5	1142634587680450123079.44954500	0	20120238159272.43210000
-6	1366193601272051815659.35740000	0	20120228058262.32098900
-7	1589752637310342397127.92967700	0	20120217957252.20987800
-8	1813311695795321867485.16637600	0	20120207856242.09876700
-9	2036870776726990226731.06749700	0	20120197755231.98765600
-10	2260429880105347474865.63304000	0	20120187654221.87654500
-11	2483989005930393611888.86300500	0	20120177553211.76543400
-12	2707548154202128637800.75739200	0	20120155330989.65432300
-13	2931105559114372081245.11567700	0	20120145229979.54321200
-14	3154664617599351551602.35237600	0	20120135128969.43210100
-15	3378223698531019910848.25349700	0	20120125027959.32099000
-16	3601782801909377158982.81904000	0	20120114926949.20987900
-17	3825341927734423296006.04900500	0	20120104825939.09876800
-18	4048901076006158321917.94339200	0	20120094724928.98765700
-19	4272460246724582236718.50220100	0	20120084623918.87654600
-20	4496019439889695040407.72543200	0	20120074522908.76543500
-21	4719578655501496732985.61308500	0	20120064421898.65432400
-22	4943137893559987314452.16516000	0	20120054320888.54321300
-23	5166697154065166784807.38165700	0	20120044219878.43210200
-24	5390256437017035144051.26257600	0	20120044219878.43210200
-
--- !sql_test_DateTime_Decimal128V3_notn_1 --
-1	248398757780932241646.46234500	0	20120288664322.98765500
-2	471957681585778378671.71251200	0	20120278563312.87654400
-3	695516627837313404585.62710100	0	20120268462302.76543300
-4	919075596535537319388.20611200	0	20120258361292.65432200
-5	1142634587680450123079.44954500	0	20120248260282.54321100
-6	1366193601272051815659.35740000	0	20120238159272.43210000
-7	1589752637310342397127.92967700	0	20120228058262.32098900
-8	1813311695795321867485.16637600	0	20120217957252.20987800
-9	2036870776726990226731.06749700	0	20120207856242.09876700
-10	2260429880105347474865.63304000	0	20120197755231.98765600
-11	2483989005930393611888.86300500	0	20120187654221.87654500
-12	2707548154202128637800.75739200	0	20120177553211.76543400
-13	2931105559114372081245.11567700	0	20120155330989.65432300
-14	3154664617599351551602.35237600	0	20120145229979.54321200
-15	3378223698531019910848.25349700	0	20120135128969.43210100
-16	3601782801909377158982.81904000	0	20120125027959.32099000
-17	3825341927734423296006.04900500	0	20120114926949.20987900
-18	4048901076006158321917.94339200	0	20120104825939.09876800
-19	4272460246724582236718.50220100	0	20120094724928.98765700
-20	4496019439889695040407.72543200	0	20120084623918.87654600
-21	4719578655501496732985.61308500	0	20120074522908.76543500
-22	4943137893559987314452.16516000	0	20120064421898.65432400
-23	5166697154065166784807.38165700	0	20120054320888.54321300
-24	5390256437017035144051.26257600	0	20120044219878.43210200
-
 -- !sql_test_DateTime_Decimal128V3_2 --
 \N	\N
 1	1629744
@@ -7896,59 +6624,6 @@
 23	256773218	20120311127915	20120054354697
 24	426256	20120579596089	20120579169833
 
--- !sql_test_DateTime_Decimal128V3_4 --
-\N	\N	\N	\N
-1	81	127	46
-2	6	127	121
-3	59	-1	-60
-4	112	127	15
-5	37	127	90
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
--- !sql_test_DateTime_Decimal128V3_notn_4 --
-1	81	127	46
-2	6	127	121
-3	59	-1	-60
-4	112	127	15
-5	37	127	90
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
 -- !sql_test_DateTime_Char_0 --
 \N	\N	\N
 1	\N	\N
@@ -8161,59 +6836,6 @@
 23	4416	20120311111527	20120311107111
 24	792	20120312127295	20120312126503
 
--- !sql_test_DateTime_Char_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DateTime_Char_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DateTime_Varchar_0 --
 \N	\N	\N
 1	\N	\N
@@ -8426,59 +7048,6 @@
 23	65793	20120311119355	20120311053562
 24	33032	20120312192925	20120312159893
 
--- !sql_test_DateTime_Varchar_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DateTime_Varchar_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DateTime_String_0 --
 \N	\N	\N
 1	\N	\N
@@ -8691,59 +7260,6 @@
 23	330787	20120311119211	20120310788424
 24	393216	20120312207295	20120311814079
 
--- !sql_test_DateTime_String_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DateTime_String_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DateTime_Date_0 --
 \N	\N	\N
 1	20120321130302	20120280889700
@@ -11235,59 +9751,6 @@
 23	2201971174684465595	1.8384750871896913E-4	20120311
 24	4403938229404681240	9.192384949324504E-5	20120312
 
--- !sql_test_DateV2_LargeInt_2 --
-\N	\N
-1	0
-2	0
-3	0
-4	0
-5	0
-6	0
-7	0
-8	0
-9	0
-10	0
-11	0
-12	1742614626213415178628831358484480
-13	1742614626200428670940778567205888
-14	1742614626200428670940778567014400
-15	0
-16	0
-17	0
-18	0
-19	0
-20	0
-21	0
-22	0
-23	0
-24	0
-
--- !sql_test_DateV2_LargeInt_notn_2 --
-1	0
-2	0
-3	0
-4	0
-5	0
-6	0
-7	0
-8	0
-9	0
-10	0
-11	0
-12	0
-13	1742613662948936925785780900199648
-14	1742613662948936925645932255510528
-15	1742613662948936925785771182616928
-16	2579572686983111875537539243928576
-17	0
-18	0
-19	0
-20	0
-21	0
-22	0
-23	0
-24	0
-
 -- !sql_test_DateV2_LargeInt_3 --
 \N	\N	\N	\N
 1	2228933	124982013	122753080
@@ -11447,59 +9910,6 @@
 23	2.0120312100000024E7	2.0120309899999976E7
 24	2.0120313200000048E7	2.0120310799999952E7
 
--- !sql_test_DateV2_Float_1 --
-\N	\N	\N	\N
-1	2012030.1299815848	2.0120300700184155E8	1.8415600061416626E-4
-2	4024060.4599631727	1.006015085009207E8	0.10018414258956909
-3	6036091.139852703	6.706767400163674E7	4.91023063659668E-4
-4	8048121.719926357	5.0300759250460275E7	0.1001841127872467
-5	1.00601525E7	4.024061E7	0.0
-6	1.2072184079705477E7	3.3533842000818174E7	4.909038543701172E-4
-7	1.408421466014725E7	2.874329620378113E7	0.14264678955078125
-8	1.6096246639852762E7	2.5150384625230063E7	0.5001840591430664
-9	1.810827762029445E7	2.2355899481117975E7	0.4330061674118042
-10	2.012031E7	2.012031E7	0.0
-11	2.2132342579705596E7	1.829119142173092E7	0.4639040231704712
-12	2.414437535941124E7	1.6766926000408888E7	4.906654357910156E-4
-13	2012030.1299815848	2.0120300700184155E8	1.8415600061416626E-4
-14	4024060.4599631727	1.006015085009207E8	0.10018414258956909
-15	6036091.139852703	6.706767400163674E7	4.91023063659668E-4
-16	8048121.719926357	5.0300759250460275E7	0.1001841127872467
-17	1.00601525E7	4.024061E7	0.0
-18	1.2072184079705477E7	3.3533842000818174E7	4.909038543701172E-4
-19	1.408421466014725E7	2.874329620378113E7	0.14264678955078125
-20	1.6096246639852762E7	2.5150384625230063E7	0.5001840591430664
-21	1.810827762029445E7	2.2355899481117975E7	0.4330061674118042
-22	2.012031E7	2.012031E7	0.0
-23	2.2132342579705596E7	1.829119142173092E7	0.4639040231704712
-24	2.414437535941124E7	1.6766926000408888E7	4.906654357910156E-4
-
--- !sql_test_DateV2_Float_notn_1 --
-1	2012030.1299815848	2.0120300700184155E8	1.8415600061416626E-4
-2	4024060.4599631727	1.006015085009207E8	0.10018414258956909
-3	6036091.139852703	6.706767400163674E7	4.91023063659668E-4
-4	8048121.719926357	5.0300759250460275E7	0.1001841127872467
-5	1.00601525E7	4.024061E7	0.0
-6	1.2072184079705477E7	3.3533842000818174E7	4.909038543701172E-4
-7	1.408421466014725E7	2.874329620378113E7	0.14264678955078125
-8	1.6096246639852762E7	2.5150384625230063E7	0.5001840591430664
-9	1.810827762029445E7	2.2355899481117975E7	0.4330061674118042
-10	2.012031E7	2.012031E7	0.0
-11	2.2132342579705596E7	1.829119142173092E7	0.4639040231704712
-12	2.414437535941124E7	1.6766926000408888E7	4.906654357910156E-4
-13	2012030.1299815848	2.0120300700184155E8	1.8415600061416626E-4
-14	4024060.4599631727	1.006015085009207E8	0.10018414258956909
-15	6036091.139852703	6.706767400163674E7	4.91023063659668E-4
-16	8048121.719926357	5.0300759250460275E7	0.1001841127872467
-17	1.00601525E7	4.024061E7	0.0
-18	1.2072184079705477E7	3.3533842000818174E7	4.909038543701172E-4
-19	1.408421466014725E7	2.874329620378113E7	0.14264678955078125
-20	1.6096246639852762E7	2.5150384625230063E7	0.5001840591430664
-21	1.810827762029445E7	2.2355899481117975E7	0.4330061674118042
-22	2.012031E7	2.012031E7	0.0
-23	2.2132342579705596E7	1.829119142173092E7	0.4639040231704712
-24	2.414437535941124E7	1.6766926000408888E7	4.906654357910156E-4
-
 -- !sql_test_DateV2_Float_2 --
 \N	\N
 1	\N
@@ -11606,59 +10016,6 @@
 23	1	20120311	20120310
 24	0	20120313	20120313
 
--- !sql_test_DateV2_Float_4 --
-\N	\N	\N	\N
-1	0	-19	-19
-2	0	-18	-18
-3	0	-17	-17
-4	0	-16	-16
-5	0	-15	-15
-6	0	-14	-14
-7	0	-13	-13
-8	0	-12	-12
-9	0	-11	-11
-10	0	-9	-9
-11	1	-9	-10
-12	0	-7	-7
-13	0	-19	-19
-14	0	-18	-18
-15	0	-17	-17
-16	0	-16	-16
-17	0	-15	-15
-18	0	-14	-14
-19	0	-13	-13
-20	0	-12	-12
-21	0	-11	-11
-22	0	-9	-9
-23	1	-9	-10
-24	0	-7	-7
-
--- !sql_test_DateV2_Float_notn_4 --
-1	0	-19	-19
-2	0	-18	-18
-3	0	-17	-17
-4	0	-16	-16
-5	0	-15	-15
-6	0	-14	-14
-7	0	-13	-13
-8	0	-12	-12
-9	0	-11	-11
-10	0	-9	-9
-11	1	-9	-10
-12	0	-7	-7
-13	0	-19	-19
-14	0	-18	-18
-15	0	-17	-17
-16	0	-16	-16
-17	0	-15	-15
-18	0	-14	-14
-19	0	-13	-13
-20	0	-12	-12
-21	0	-11	-11
-22	0	-9	-9
-23	1	-9	-10
-24	0	-7	-7
-
 -- !sql_test_DateV2_Double_0 --
 \N	\N	\N
 1	2.01203015244E7	2.01203004756E7
@@ -11871,59 +10228,6 @@
 23	16	20120311	20120295
 24	16	20120318	20120302
 
--- !sql_test_DateV2_Double_4 --
-\N	\N	\N	\N
-1	0	-19	-19
-2	0	-18	-18
-3	1	-17	-18
-4	0	-15	-15
-5	0	-13	-13
-6	2	-14	-16
-7	0	-9	-9
-8	4	-11	-15
-9	0	-3	-3
-10	2	-1	-3
-11	16	-9	-25
-12	16	-2	-18
-13	0	-19	-19
-14	0	-18	-18
-15	1	-17	-18
-16	0	-15	-15
-17	0	-13	-13
-18	2	-14	-16
-19	0	-9	-9
-20	4	-11	-15
-21	0	-3	-3
-22	2	-1	-3
-23	16	-9	-25
-24	16	-2	-18
-
--- !sql_test_DateV2_Double_notn_4 --
-1	0	-19	-19
-2	0	-18	-18
-3	1	-17	-18
-4	0	-15	-15
-5	0	-13	-13
-6	2	-14	-16
-7	0	-9	-9
-8	4	-11	-15
-9	0	-3	-3
-10	2	-1	-3
-11	16	-9	-25
-12	16	-2	-18
-13	0	-19	-19
-14	0	-18	-18
-15	1	-17	-18
-16	0	-15	-15
-17	0	-13	-13
-18	2	-14	-16
-19	0	-9	-9
-20	4	-11	-15
-21	0	-3	-3
-22	2	-1	-3
-23	16	-9	-25
-24	16	-2	-18
-
 -- !sql_test_DateV2_DecimalV2_0 --
 \N	\N	\N
 1	20120325.395000000	20120276.605000000
@@ -12136,59 +10440,6 @@
 23	515	20120575	20120060
 24	72	20121342	20121270
 
--- !sql_test_DateV2_DecimalV2_4 --
-\N	\N	\N	\N
-1	8	-3	-11
-2	34	-18	-52
-3	32	-1	-33
-4	64	-12	-76
-5	97	-15	-112
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	8	-3	-11
-14	34	-18	-52
-15	32	-1	-33
-16	64	-12	-76
-17	97	-15	-112
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
--- !sql_test_DateV2_DecimalV2_notn_4 --
-1	8	-3	-11
-2	34	-18	-52
-3	32	-1	-33
-4	64	-12	-76
-5	97	-15	-112
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	8	-3	-11
-14	34	-18	-52
-15	32	-1	-33
-16	64	-12	-76
-17	97	-15	-112
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
 -- !sql_test_DateV2_Decimal32V3_0 --
 \N	\N	\N
 1	20120313.012	20120288.988
@@ -12242,59 +10493,6 @@
 23	20120567.454	20120054.546
 24	20120579.565	20120044.435
 
--- !sql_test_DateV2_Decimal32V3_1 --
-\N	\N	\N	\N
-1	241685055.612	1675	1030.241
-2	465241743.146	870	1030.242
-3	688798452.902	587	1030.243
-4	912355184.880	443	1030.244
-5	1135911939.080	356	1030.245
-6	1359468715.502	297	1030.246
-7	1583025514.146	255	1030.247
-8	1806582335.012	224	1030.248
-9	2030139178.100	199	1030.249
-10	2253696043.410	179	1030.250
-11	2477252930.942	163	1030.251
-12	2700809840.696	149	1030.252
-13	2924365028.544	138	1030.241
-14	3147921849.410	128	1030.242
-15	3371478692.498	120	1030.243
-16	3595035557.808	112	1030.244
-17	3818592445.340	106	1030.245
-18	4042149355.094	100	1030.246
-19	4265706287.070	94	1030.247
-20	4489263241.268	90	1030.248
-21	4712820217.688	85	1030.249
-22	4936377216.330	82	1030.250
-23	5159934237.194	78	1030.251
-24	5383491280.280	75	1030.252
-
--- !sql_test_DateV2_Decimal32V3_notn_1 --
-1	241685055.612	1675	-374022.048
-2	465241743.146	870	21.994
-3	688798452.902	587	0.006
-4	912355184.880	443	0.000
-5	1135911939.080	356	0.000
-6	1359468715.502	297	0.000
-7	1583025514.146	255	486408.704
-8	1806582335.012	224	32.559
-9	2030139178.100	199	0.000
-10	2253696043.410	179	0.000
-11	2477252930.942	163	0.000
-12	2700809840.696	149	0.000
-13	2924365028.544	138	0.000
-14	3147921849.410	128	0.000
-15	3371478692.498	120	0.000
-16	3595035557.808	112	0.000
-17	3818592445.340	106	0.000
-18	4042149355.094	100	32.561
-19	4265706287.070	94	0.000
-20	4489263241.268	90	0.000
-21	4712820217.688	85	0.000
-22	4936377216.330	82	0.000
-23	5159934237.194	78	0.000
-24	5383491280.280	75	0.000
-
 -- !sql_test_DateV2_Decimal32V3_2 --
 \N	\N
 1	1676691
@@ -12401,59 +10599,6 @@
 23	0	20120567	20120567
 24	8	20120571	20120563
 
--- !sql_test_DateV2_Decimal32V3_4 --
-\N	\N	\N	\N
-1	12	-19	-31
-2	6	-1	-7
-3	34	-17	-51
-4	32	-3	-35
-5	48	-7	-55
-6	66	-13	-79
-7	66	-1	-67
-8	80	-3	-83
-9	100	-11	-111
-10	112	-10	-122
-11	115	-1	-116
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
--- !sql_test_DateV2_Decimal32V3_notn_4 --
-1	12	-19	-31
-2	6	-1	-7
-3	34	-17	-51
-4	32	-3	-35
-5	48	-7	-55
-6	66	-13	-79
-7	66	-1	-67
-8	80	-3	-83
-9	100	-11	-111
-10	112	-10	-122
-11	115	-1	-116
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
 -- !sql_test_DateV2_Decimal64V3_0 --
 \N	\N	\N
 1	20121535.01234	20119066.98766
@@ -12507,59 +10652,6 @@
 23	20145989.45676	20094632.54324
 24	20147101.56787	20093522.43213
 
--- !sql_test_DateV2_Decimal64V3_1 --
-\N	\N	\N	\N
-1	24828699718.51434	0	201203010.10001
-2	47184592041.28190	0	201203020.20102
-3	69540486586.27168	0	201203030.30203
-4	91896383353.48368	0	201203040.40304
-5	114252282342.91790	0	201203050.50405
-6	136608183554.57434	0	201203060.60506
-7	158964086988.45300	0	201203070.70607
-8	181319992644.55388	0	201203080.80708
-9	203675900522.87698	0	201203090.90809
-10	226031810623.42230	0	201203101.00910
-11	248387722946.18984	0	201203111.11011
-12	270743637491.17960	0	201203121.21112
-13	293099379450.24366	0	201203010.10001
-14	315455285106.34454	0	201203020.20102
-15	337811192984.66764	0	201203030.30203
-16	360167103085.21296	0	201203040.40304
-17	382523015407.98050	0	201203050.50405
-18	404878929952.97026	0	201203060.60506
-19	427234846720.18224	0	201203070.70607
-20	449590765709.61644	0	201203080.80708
-21	471946686921.27286	0	201203090.90809
-22	494302610355.15150	0	201203101.00910
-23	516658536011.25236	0	201203111.11011
-24	539014463889.57544	0	201203121.21112
-
--- !sql_test_DateV2_Decimal64V3_notn_1 --
-1	24828699718.51434	0	0.00000
-2	47184592041.28190	0	20120301.00000
-3	69540486586.27168	0	20120302.00000
-4	91896383353.48368	0	20120303.00000
-5	114252282342.91790	0	20120304.00000
-6	136608183554.57434	0	20120305.00000
-7	158964086988.45300	0	20120306.00000
-8	181319992644.55388	0	20120307.00000
-9	203675900522.87698	0	20120308.00000
-10	226031810623.42230	0	20120309.00000
-11	248387722946.18984	0	20120310.00000
-12	270743637491.17960	0	20120311.00000
-13	293099379450.24366	0	20120312.00000
-14	315455285106.34454	0	20120301.00000
-15	337811192984.66764	0	20120302.00000
-16	360167103085.21296	0	20120303.00000
-17	382523015407.98050	0	20120304.00000
-18	404878929952.97026	0	20120305.00000
-19	427234846720.18224	0	20120306.00000
-20	449590765709.61644	0	20120307.00000
-21	471946686921.27286	0	20120308.00000
-22	494302610355.15150	0	20120309.00000
-23	516658536011.25236	0	20120310.00000
-24	539014463889.57544	0	20120311.00000
-
 -- !sql_test_DateV2_Decimal64V3_2 --
 \N	\N
 1	16304
@@ -12666,59 +10758,6 @@
 23	70	20145919	20145849
 24	160	20146941	20146781
 
--- !sql_test_DateV2_Decimal64V3_4 --
-\N	\N	\N	\N
-1	109	-1	-110
-2	110	-1	-111
-3	111	-1	-112
-4	112	-1	-113
-5	113	-1	-114
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
--- !sql_test_DateV2_Decimal64V3_notn_4 --
-1	109	-1	-110
-2	110	-1	-111
-3	111	-1	-112
-4	112	-1	-113
-5	113	-1	-114
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
 -- !sql_test_DateV2_Decimal128V3_0 --
 \N	\N	\N
 1	32465979.01234500	7774622.98765500
@@ -12772,59 +10811,6 @@
 23	276910433.45678700	-236669811.45678700
 24	288021545.56789800	-247780921.56789800
 
--- !sql_test_DateV2_Decimal128V3_1 --
-\N	\N	\N	\N
-1	248398757657463.11584500	0	-3336487.12345600
-2	471957681114250.00371200	0	-14447597.23456700
-3	695516626793259.11380100	0	-25558707.34567800
-4	919075594694490.44611200	0	-36669817.45678900
-5	1142634584817944.00064500	0	-47780927.56790000
-6	1366193597163619.77740000	0	-58892037.67901100
-7	1589752631731517.77637700	0	-70003147.79012200
-8	1813311688521637.99757600	0	-81114257.90123300
-9	2036870767533980.44099700	0	-92225368.01234400
-10	2260429868768545.10664000	0	-103336478.12345500
-11	2483988992225331.99450500	0	-114447588.23456600
-12	2707548137904341.10459200	0	-125558710.34567700
-13	2931105557657436.28877700	0	-136669820.45678800
-14	3154664614447556.50997600	0	-147780930.56789900
-15	3378223693459898.95339700	0	-158892040.67901000
-16	3601782794694463.61904000	0	-170003150.79012100
-17	3825341918151250.50690500	0	-181114260.90123200
-18	4048901063830259.61699200	0	-192225371.01234300
-19	4272460231731490.94930100	0	-203336481.12345400
-20	4496019421854944.50383200	0	-214447591.23456500
-21	4719578634200620.28058500	0	-225558701.34567600
-22	4943137868768518.27956000	0	-236669811.45678700
-23	5166697125558638.50075700	0	-247780921.56789800
-24	5390256404570980.94417600	0	-247780921.56789800
-
--- !sql_test_DateV2_Decimal128V3_notn_1 --
-1	248398757657463.11584500	0	7774622.98765500
-2	471957681114250.00371200	0	-3336487.12345600
-3	695516626793259.11380100	0	-14447597.23456700
-4	919075594694490.44611200	0	-25558707.34567800
-5	1142634584817944.00064500	0	-36669817.45678900
-6	1366193597163619.77740000	0	-47780927.56790000
-7	1589752631731517.77637700	0	-58892037.67901100
-8	1813311688521637.99757600	0	-70003147.79012200
-9	2036870767533980.44099700	0	-81114257.90123300
-10	2260429868768545.10664000	0	-92225368.01234400
-11	2483988992225331.99450500	0	-103336478.12345500
-12	2707548137904341.10459200	0	-114447588.23456600
-13	2931105557657436.28877700	0	-125558710.34567700
-14	3154664614447556.50997600	0	-136669820.45678800
-15	3378223693459898.95339700	0	-147780930.56789900
-16	3601782794694463.61904000	0	-158892040.67901000
-17	3825341918151250.50690500	0	-170003150.79012100
-18	4048901063830259.61699200	0	-181114260.90123200
-19	4272460231731490.94930100	0	-192225371.01234300
-20	4496019421854944.50383200	0	-203336481.12345400
-21	4719578634200620.28058500	0	-214447591.23456500
-22	4943137868768518.27956000	0	-225558701.34567600
-23	5166697125558638.50075700	0	-236669811.45678700
-24	5390256404570980.94417600	0	-247780921.56789800
-
 -- !sql_test_DateV2_Decimal128V3_2 --
 \N	\N
 1	1
@@ -12931,59 +10917,6 @@
 23	16908898	260001535	243092637
 24	20119600	267901945	247782345
 
--- !sql_test_DateV2_Decimal128V3_4 --
-\N	\N	\N	\N
-1	109	-1	-110
-2	110	-1	-111
-3	111	-1	-112
-4	112	-1	-113
-5	113	-1	-114
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
--- !sql_test_DateV2_Decimal128V3_notn_4 --
-1	109	-1	-110
-2	110	-1	-111
-3	111	-1	-112
-4	112	-1	-113
-5	113	-1	-114
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	109	-1	-110
-14	110	-1	-111
-15	111	-1	-112
-16	112	-1	-113
-17	113	-1	-114
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
 -- !sql_test_DateV2_Char_0 --
 \N	\N	\N
 1	\N	\N
@@ -13196,59 +11129,6 @@
 23	580	20124663	20124083
 24	568	20126719	20126151
 
--- !sql_test_DateV2_Char_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DateV2_Char_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DateV2_Varchar_0 --
 \N	\N	\N
 1	\N	\N
@@ -13461,59 +11341,6 @@
 23	65681	20128767	20063086
 24	65672	20159485	20093813
 
--- !sql_test_DateV2_Varchar_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DateV2_Varchar_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DateV2_String_0 --
 \N	\N	\N
 1	\N	\N
@@ -13726,59 +11553,6 @@
 23	65571	20393727	20328156
 24	196768	20402943	20206175
 
--- !sql_test_DateV2_String_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DateV2_String_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DateV2_Date_0 --
 \N	\N	\N
 1	40240602	0
@@ -16270,59 +14044,6 @@
 23	2201971186833533373967095	183.84750973332203	92751647976
 24	4403938255913701917197240	91.92385004657052	202212497417
 
--- !sql_test_DateTimeV2_LargeInt_2 --
-\N	\N
-1	867753287971371017662291
-2	217007497283119165234150
-3	54251874320779791308538
-4	13558356894176520439230
-5	3375754165488847946095
-6	3465792723312624267165787
-7	867753287971371017662291
-8	217007497283119165234150
-9	54251874320779791308538
-10	13558356894176520439230
-11	3375754165488847946095
-12	2360475650127473278569014804611163
-13	1742613662948936925645932255510528
-14	1742613662948936925785780278928384
-15	1742613662948936925785780278927648
-16	1742613662948936925785780278928128
-17	1742613662948936925785780278859264
-18	1742613662948936925785780278928160
-19	1742613662948936925785780278928512
-20	1742613662948936925785780278928544
-21	1742613662948936925785780278928768
-22	1742613662948936925785780278928896
-23	1742613662948936925785780278929024
-24	1742613662948936925785780278929280
-
--- !sql_test_DateTimeV2_LargeInt_notn_2 --
-1	1734639578971277686398441
-2	433978101078090911365057
-3	108503748641559582617076
-4	27116713788353040878461
-5	6769955075051405443807
-6	1678653710707569197239
-7	1734639578971277686398441
-8	433978101078090911365057
-9	108503748641559582617076
-10	27116713788353040878461
-11	6769955075051405443807
-12	1678653710707569197239
-13	1742613662948936925785780278931232
-14	1742613619535041257004566394699776
-15	1742613619535041257144415043291520
-16	139838742151168
-17	371154062050562888188337739594833
-18	371154099316712159384521335361467
-19	371154136582861430580704931128101
-20	371154173849010701776888526894735
-21	371154211115159972973072122661369
-22	371154248381309244169255718428003
-23	371154062050562888188337739594833
-24	371154099316712159384521335361467
-
 -- !sql_test_DateTimeV2_LargeInt_3 --
 \N	\N	\N	\N
 1	106958929	20120301141717	20120194182788
@@ -16482,59 +14203,6 @@
 23	2.01203111110121E13	2.01203111110099E13
 24	2.01203121211132E13	2.01203121211108E13
 
--- !sql_test_DateTimeV2_Float_1 --
-\N	\N	\N	\N
-1	2.0120301309816848E12	2.0120300710185156E14	0.05584884434938431
-2	4.0240604639835728E12	1.006015086014307E14	0.14079716801643372
-3	6.036091148913603E12	6.706767410231341E13	0.12184399366378784
-4	8.048121736047957E12	5.030075935122027E13	0.11069381237030029
-5	1.00601525252025E13	4.024061010081E13	0.0
-6	1.2072184116009078E13	3.3533842101661504E13	0.3014305830001831
-7	1.4084214709572148E13	2.8743296304648277E13	0.19318246841430664
-8	1.6096246704419164E13	2.5150384726115062E13	0.05048710107803345
-9	1.810827770202255E13	2.2355899582016867E13	0.7812881469726562
-10	2.012031010091E13	2.012031010091E13	0.0
-11	2.21323427018177E13	1.8291191522650008E13	0.010709524154663086
-12	2.4144375504745645E13	1.676692610133555E13	0.6606037616729736
-13	2.0120301309816848E12	2.0120300710185156E14	0.05584884434938431
-14	4.0240604639835728E12	1.006015086014307E14	0.14079716801643372
-15	6.036091148913603E12	6.706767410231341E13	0.12184399366378784
-16	8.048121736047957E12	5.030075935122027E13	0.11069381237030029
-17	1.00601525252025E13	4.024061010081E13	0.0
-18	1.2072184116009078E13	3.3533842101661504E13	0.3014305830001831
-19	1.4084214709572148E13	2.8743296304648277E13	0.19318246841430664
-20	1.6096246704419164E13	2.5150384726115062E13	0.05048710107803345
-21	1.810827770202255E13	2.2355899582016867E13	0.7812881469726562
-22	2.012031010091E13	2.012031010091E13	0.0
-23	2.21323427018177E13	1.8291191522650008E13	0.010709524154663086
-24	2.4144375504745645E13	1.676692610133555E13	0.6606037616729736
-
--- !sql_test_DateTimeV2_Float_notn_1 --
-1	2.0120301309816848E12	2.0120300710185156E14	0.05584884434938431
-2	4.0240604639835728E12	1.006015086014307E14	0.14079716801643372
-3	6.036091148913603E12	6.706767410231341E13	0.12184399366378784
-4	8.048121736047957E12	5.030075935122027E13	0.11069381237030029
-5	1.00601525252025E13	4.024061010081E13	0.0
-6	1.2072184116009078E13	3.3533842101661504E13	0.3014305830001831
-7	1.4084214709572148E13	2.8743296304648277E13	0.19318246841430664
-8	1.6096246704419164E13	2.5150384726115062E13	0.05048710107803345
-9	1.810827770202255E13	2.2355899582016867E13	0.7812881469726562
-10	2.012031010091E13	2.012031010091E13	0.0
-11	2.21323427018177E13	1.8291191522650008E13	0.010709524154663086
-12	2.4144375504745645E13	1.676692610133555E13	0.6606037616729736
-13	2.0120301309816848E12	2.0120300710185156E14	0.05584884434938431
-14	4.0240604639835728E12	1.006015086014307E14	0.14079716801643372
-15	6.036091148913603E12	6.706767410231341E13	0.12184399366378784
-16	8.048121736047957E12	5.030075935122027E13	0.11069381237030029
-17	1.00601525252025E13	4.024061010081E13	0.0
-18	1.2072184116009078E13	3.3533842101661504E13	0.3014305830001831
-19	1.4084214709572148E13	2.8743296304648277E13	0.19318246841430664
-20	1.6096246704419164E13	2.5150384726115062E13	0.05048710107803345
-21	1.810827770202255E13	2.2355899582016867E13	0.7812881469726562
-22	2.012031010091E13	2.012031010091E13	0.0
-23	2.21323427018177E13	1.8291191522650008E13	0.010709524154663086
-24	2.4144375504745645E13	1.676692610133555E13	0.6606037616729736
-
 -- !sql_test_DateTimeV2_Float_2 --
 \N	\N
 1	\N
@@ -16641,59 +14309,6 @@
 23	1	20120311111011	20120311111010
 24	0	20120312121113	20120312121113
 
--- !sql_test_DateTimeV2_Float_4 --
-\N	\N	\N	\N
-1	0	81	81
-2	0	6	6
-3	0	-69	-69
-4	0	112	112
-5	0	37	37
-6	0	-38	-38
-7	0	-113	-113
-8	0	68	68
-9	0	-7	-7
-10	0	-81	-81
-11	1	99	98
-12	0	25	25
-13	0	81	81
-14	0	6	6
-15	0	-69	-69
-16	0	112	112
-17	0	37	37
-18	0	-38	-38
-19	0	-113	-113
-20	0	68	68
-21	0	-7	-7
-22	0	-81	-81
-23	1	99	98
-24	0	25	25
-
--- !sql_test_DateTimeV2_Float_notn_4 --
-1	0	81	81
-2	0	6	6
-3	0	-69	-69
-4	0	112	112
-5	0	37	37
-6	0	-38	-38
-7	0	-113	-113
-8	0	68	68
-9	0	-7	-7
-10	0	-81	-81
-11	1	99	98
-12	0	25	25
-13	0	81	81
-14	0	6	6
-15	0	-69	-69
-16	0	112	112
-17	0	37	37
-18	0	-38	-38
-19	0	-113	-113
-20	0	68	68
-21	0	-7	-7
-22	0	-81	-81
-23	1	99	98
-24	0	25	25
-
 -- !sql_test_DateTimeV2_Double_0 --
 \N	\N	\N
 1	2.0120301010001523E13	2.0120301010000477E13
@@ -16906,59 +14521,6 @@
 23	0	20120311111027	20120311111027
 24	16	20120312121118	20120312121102
 
--- !sql_test_DateTimeV2_Double_4 --
-\N	\N	\N	\N
-1	0	81	81
-2	0	6	6
-3	1	-69	-70
-4	0	113	113
-5	0	39	39
-6	2	-38	-40
-7	4	-113	-117
-8	4	69	65
-9	8	-7	-15
-10	10	-81	-91
-11	0	115	115
-12	16	30	14
-13	0	81	81
-14	0	6	6
-15	1	-69	-70
-16	0	113	113
-17	0	39	39
-18	2	-38	-40
-19	4	-113	-117
-20	4	69	65
-21	8	-7	-15
-22	10	-81	-91
-23	0	115	115
-24	16	30	14
-
--- !sql_test_DateTimeV2_Double_notn_4 --
-1	0	81	81
-2	0	6	6
-3	1	-69	-70
-4	0	113	113
-5	0	39	39
-6	2	-38	-40
-7	4	-113	-117
-8	4	69	65
-9	8	-7	-15
-10	10	-81	-91
-11	0	115	115
-12	16	30	14
-13	0	81	81
-14	0	6	6
-15	1	-69	-70
-16	0	113	113
-17	0	39	39
-18	2	-38	-40
-19	4	-113	-117
-20	4	69	65
-21	8	-7	-15
-22	10	-81	-91
-23	0	115	115
-24	16	30	14
-
 -- !sql_test_DateTimeV2_DecimalV2_0 --
 \N	\N	\N
 1	20120301010025.395000000	20120301009976.605000000
@@ -17171,59 +14733,6 @@
 23	259	20120311111531	20120311111272
 24	1032	20120312121182	20120312120150
 
--- !sql_test_DateTimeV2_DecimalV2_4 --
-\N	\N	\N	\N
-1	16	89	73
-2	2	38	36
-3	48	-69	-117
-4	64	116	52
-5	33	101	68
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	16	89	73
-14	2	38	36
-15	48	-69	-117
-16	64	116	52
-17	33	101	68
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
--- !sql_test_DateTimeV2_DecimalV2_notn_4 --
-1	16	89	73
-2	2	38	36
-3	48	-69	-117
-4	64	116	52
-5	33	101	68
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	16	89	73
-14	2	38	36
-15	48	-69	-117
-16	64	116	52
-17	33	101	68
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
 -- !sql_test_DateTimeV2_Decimal32V3_0 --
 \N	\N	\N
 1	20120301010013.012	20120301009988.988
@@ -17277,59 +14786,6 @@
 23	20120311111267.454	20120311110754.546
 24	20120312121379.565	20120312120844.435
 
--- !sql_test_DateTimeV2_Decimal32V3_1 --
-\N	\N	\N	\N
-1	241685055732132.012	1675016734	4.684
-2	465241743610818.546	870142369	-1619761.65
-3	688798453935969.502	587728662	4.684
-4	912355186707584.880	443716044	-1618751.55
-5	1135911941925664.680	356389135	4.684
-6	1359468719590208.902	297783031	-1617741.45
-7	1583025519701217.546	255729772	4.684
-8	1806582342258690.612	224084331	-1616731.35
-9	2030139187262628.100	199408415	4.684
-10	2253696054713030.010	179627983	-1615721.25
-11	2477252944609896.342	163417676	4.684
-12	2700809856953227.096	149890951	-1614711.15
-13	2924365029997585.344	138432277	4.684
-14	3147921852555058.410	128601208	-1613701.05
-15	3371478697558995.898	120073899	4.684
-16	3595035565009397.808	112607129	-1612690.95
-17	3818592454906264.140	106014632	4.684
-18	4042149367249594.894	100151349	-1611680.85
-19	4265706302039390.070	94902632	4.684
-20	4489263259275649.668	90176666	-1610670.74
-21	4712820238958373.688	85899062	4.684
-22	4936377241087562.130	82008902	-1609660.64
-23	5159934265663214.994	78455828	4.684
-24	5383491312685332.280	75197847	-1620771.75
-
--- !sql_test_DateTimeV2_Decimal32V3_notn_1 --
-1	241685055732132.012	1675016734	98001.408
-2	465241743610818.546	870142369	32.559
-3	688798453935969.502	587728662	1260615.93
-4	912355186707584.880	443716044	-485796.965
-5	1135911941925664.680	356389135	98001.408
-6	1359468719590208.902	297783031	32.559
-7	1583025519701217.546	255729772	239690.240
-8	1806582342258690.612	224084331	32.561
-9	2030139187262628.100	199408415	-1123039.10
-10	2253696054713030.010	179627983	32.558
-11	2477252944609896.342	163417676	239690.241
-12	2700809856953227.096	149890951	32.561
-13	2924365029997585.344	138432277	98083.456
-14	3147921852555058.410	128601208	32.559
-15	3371478697558995.898	120073899	1260615.93
-16	3595035565009397.808	112607129	-485796.965
-17	3818592454906264.140	106014632	241106.880
-18	4042149367249594.894	100151349	32.561
-19	4265706302039390.070	94902632	-529716.912
-20	4489263259275649.668	90176666	21.994
-21	4712820238958373.688	85899062	241106.912
-22	4936377241087562.130	82008902	32.561
-23	5159934265663214.994	78455828	-529716.912
-24	5383491312685332.280	75197847	21.994
-
 -- !sql_test_DateTimeV2_Decimal32V3_2 --
 \N	\N
 1	1676691750833
@@ -17436,59 +14892,6 @@
 23	256	20120311111011	20120311110755
 24	264	20120312121115	20120312120851
 
--- !sql_test_DateTimeV2_Decimal32V3_4 --
-\N	\N	\N	\N
-1	0	93	93
-2	6	23	17
-3	34	-69	-103
-4	32	125	93
-5	32	61	29
-6	66	-37	-103
-7	14	-49	-63
-8	64	93	29
-9	96	-3	-99
-10	32	-2	-34
-11	99	123	24
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
--- !sql_test_DateTimeV2_Decimal32V3_notn_4 --
-1	0	93	93
-2	6	23	17
-3	34	-69	-103
-4	32	125	93
-5	32	61	29
-6	66	-37	-103
-7	14	-49	-63
-8	64	93	29
-9	96	-3	-99
-10	32	-2	-34
-11	99	123	24
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
 -- !sql_test_DateTimeV2_Decimal64V3_0 --
 \N	\N	\N
 1	20120301011235.01234	20120301008766.98766
@@ -17542,59 +14945,6 @@
 23	20120311136689.45676	20120311085332.54324
 24	20120312147901.56787	20120312094322.43213
 
--- !sql_test_DateTimeV2_Decimal64V3_1 --
-\N	\N	\N	\N
-1	24828699730855697.41234	163047	201203020.20102
-2	47184592088423571.59190	85796	201203030.30203
-3	69540486690660332.41568	58214	201203040.40304
-4	91896383537565979.88368	44052	201203050.50405
-5	114252282629140513.99590	35432	201203060.60506
-6	136608183965383934.75234	29634	201203070.70607
-7	158964087546296242.15300	25466	201203080.80708
-8	181319993371877436.19788	22326	201203090.90809
-9	203675901442127516.88698	19876	201203101.00910
-10	226031811757046484.22030	17910	201203111.11011
-11	248387724316634338.19784	16298	201203121.21112
-12	270743639120891078.81960	14952	201203010.10001
-13	293099379595931683.94566	13811	201203020.20102
-14	315455285421512877.99054	12833	201203030.30203
-15	337811193491762958.67964	11983	201203040.40304
-16	360167103806681926.01296	11239	201203050.50405
-17	382523016366269779.99050	10583	201203060.60506
-18	404878931170526520.61226	9998	201203070.70607
-19	427234848219452147.87824	9475	201203080.80708
-20	449590767513046661.78844	9004	201203090.90809
-21	471946689051310062.34286	8577	201203101.00910
-22	494302612834242349.54150	8189	201203111.11011
-23	516658538861843523.38436	7835	201203121.21112
-24	539014467134113583.87144	7510	70898515052585.4213
-
--- !sql_test_DateTimeV2_Decimal64V3_notn_1 --
-1	24828699730855697.41234	163047	42949.67296
-2	47184592088423571.59190	85796	128849.01890
-3	69540486690660332.41568	58214	214748.36484
-4	91896383537565979.88368	44052	300647.71078
-5	114252282629140513.99590	35432	386547.05672
-6	136608183965383934.75234	29634	472446.40266
-7	158964087546296242.15300	25466	558345.74860
-8	181319993371877436.19788	22326	644245.09454
-9	203675901442127516.88698	19876	730144.44048
-10	226031811757046484.22030	17910	816043.78642
-11	248387724316634338.19784	16298	901943.13236
-12	270743639120891078.81960	14952	987842.47830
-13	293099379595931683.94566	13811	0.00024
-14	315455285421512877.99054	12833	201.20302
-15	337811193491762958.67964	11983	201.20303
-16	360167103806681926.01296	11239	201.20304
-17	382523016366269779.99050	10583	201.20305
-18	404878931170526520.61226	9998	201.20306
-19	427234848219452147.87824	9475	201.20307
-20	449590767513046661.78844	9004	201.20308
-21	471946689051310062.34286	8577	201.20309
-22	494302612834242349.54150	8189	201.20310
-23	516658538861843523.38436	7835	201.20311
-24	539014467134113583.87144	7510	201.20312
-
 -- !sql_test_DateTimeV2_Decimal64V3_2 --
 \N	\N
 1	16304944092
@@ -17701,59 +15051,6 @@
 23	1090	20120311135599	20120311134509
 24	0	20120312147901	20120312147901
 
--- !sql_test_DateTimeV2_Decimal64V3_4 --
-\N	\N	\N	\N
-1	81	127	46
-2	6	127	121
-3	59	-1	-60
-4	112	127	15
-5	37	127	90
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
--- !sql_test_DateTimeV2_Decimal64V3_notn_4 --
-1	81	127	46
-2	6	127	121
-3	59	-1	-60
-4	112	127	15
-5	37	127	90
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
 -- !sql_test_DateTimeV2_Decimal128V3_0 --
 \N	\N	\N
 1	20120313355679.01234500	20120288664322.98765500
@@ -17807,59 +15104,6 @@
 23	20120567901133.45678700	20120054320888.54321300
 24	20120580022345.56789800	20120044219878.43210200
 
--- !sql_test_DateTimeV2_Decimal128V3_1 --
-\N	\N	\N	\N
-1	248398757780932241646.46234500	0	68.40215645
-2	471957681585778378671.71251200	0	1094.40215645
-3	695516627837313404585.62710100	0	1.07090645
-4	919075596535537319388.20611200	0	68.40215645
-5	1142634587680450123079.44954500	0	273.60215645
-6	1366193601272051815659.35740000	0	2.13965645
-7	1589752637310342397127.92967700	0	8.55215645
-8	1813311695795321867485.16637600	0	136.80215645
-9	2036870776726990226731.06749700	0	2.13965645
-10	2260429880105347474865.63304000	0	8.55215645
-11	2483989005930393611888.86300500	0	547.20215645
-12	2707548154202128637800.75739200	0	4.27715645
-13	2931105559114372081245.11567700	0	17.10215645
-14	3154664617599351551602.35237600	0	273.60215645
-15	3378223698531019910848.25349700	0	4.27715645
-16	3601782801909377158982.81904000	0	17.10215645
-17	3825341927734423296006.04900500	0	1094.40215645
-18	4048901076006158321917.94339200	0	34.20215645
-19	4272460246724582236718.50220100	0	547.20215645
-20	4496019439889695040407.72543200	0	2188.80215645
-21	4719578655501496732985.61308500	0	34.20215645
-22	4943137893559987314452.16516000	0	136.80215645
-23	5166697154065166784807.38165700	0	2188.80215645
-24	5390256437017035144051.26257600	0	25795636703660211998970995.40856941
-
--- !sql_test_DateTimeV2_Decimal128V3_notn_1 --
-1	248398757780932241646.46234500	0	1.07090645
-2	471957681585778378671.71251200	0	68.40215645
-3	695516627837313404585.62710100	0	1094.40215645
-4	919075596535537319388.20611200	0	1.07090645
-5	1142634587680450123079.44954500	0	68.40215645
-6	1366193601272051815659.35740000	0	273.60215645
-7	1589752637310342397127.92967700	0	2.13965645
-8	1813311695795321867485.16637600	0	8.55215645
-9	2036870776726990226731.06749700	0	136.80215645
-10	2260429880105347474865.63304000	0	2.13965645
-11	2483989005930393611888.86300500	0	8.55215645
-12	2707548154202128637800.75739200	0	547.20215645
-13	2931105559114372081245.11567700	0	4.27715645
-14	3154664617599351551602.35237600	0	17.10215645
-15	3378223698531019910848.25349700	0	273.60215645
-16	3601782801909377158982.81904000	0	4.27715645
-17	3825341927734423296006.04900500	0	17.10215645
-18	4048901076006158321917.94339200	0	1094.40215645
-19	4272460246724582236718.50220100	0	34.20215645
-20	4496019439889695040407.72543200	0	547.20215645
-21	4719578655501496732985.61308500	0	2188.80215645
-22	4943137893559987314452.16516000	0	34.20215645
-23	5166697154065166784807.38165700	0	136.80215645
-24	5390256437017035144051.26257600	0	2188.80215645
-
 -- !sql_test_DateTimeV2_Decimal128V3_2 --
 \N	\N
 1	1629744
@@ -17966,59 +15210,6 @@
 23	256773218	20120311127915	20120054354697
 24	426256	20120579596089	20120579169833
 
--- !sql_test_DateTimeV2_Decimal128V3_4 --
-\N	\N	\N	\N
-1	81	127	46
-2	6	127	121
-3	59	-1	-60
-4	112	127	15
-5	37	127	90
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
--- !sql_test_DateTimeV2_Decimal128V3_notn_4 --
-1	81	127	46
-2	6	127	121
-3	59	-1	-60
-4	112	127	15
-5	37	127	90
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	81	127	46
-14	6	127	121
-15	59	-1	-60
-16	112	127	15
-17	37	127	90
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
 -- !sql_test_DateTimeV2_Char_0 --
 \N	\N	\N
 1	\N	\N
@@ -18231,59 +15422,6 @@
 23	4416	20120311111527	20120311107111
 24	792	20120312127295	20120312126503
 
--- !sql_test_DateTimeV2_Char_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DateTimeV2_Char_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DateTimeV2_Varchar_0 --
 \N	\N	\N
 1	\N	\N
@@ -18496,59 +15634,6 @@
 23	65793	20120311119355	20120311053562
 24	33032	20120312192925	20120312159893
 
--- !sql_test_DateTimeV2_Varchar_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DateTimeV2_Varchar_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DateTimeV2_String_0 --
 \N	\N	\N
 1	\N	\N
@@ -18761,59 +15846,6 @@
 23	330787	20120311119211	20120310788424
 24	393216	20120312207295	20120311814079
 
--- !sql_test_DateTimeV2_String_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DateTimeV2_String_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DateTimeV2_Date_0 --
 \N	\N	\N
 1	20120321130302	20120280889700
diff --git a/regression-test/data/nereids_arith_p0/decimal.out b/regression-test/data/nereids_arith_p0/decimal.out
index abfff1c26a..be6e000550 100644
--- a/regression-test/data/nereids_arith_p0/decimal.out
+++ b/regression-test/data/nereids_arith_p0/decimal.out
@@ -54,56 +54,56 @@
 
 -- !sql_test_Float_TinyInt_1 --
 \N	\N	\N	\N
-1	0.10000000149011612	0.10000000149011612	0.10000000149011612
-2	0.4000000059604645	0.10000000149011612	0.20000000298023224
-3	0.9000000357627869	0.10000000397364299	0.30000001192092896
-4	1.600000023841858	0.10000000149011612	0.4000000059604645
+1	0.10000000149011612	0.10000000149011612	0.1
+2	0.4000000059604645	0.10000000149011612	0.2
+3	0.9000000357627869	0.10000000397364299	0.3
+4	1.600000023841858	0.10000000149011612	0.4
 5	2.5	0.1	0.5
-6	3.6000001430511475	0.10000000397364299	0.6000000238418579
-7	4.899999916553497	0.09999999829701015	0.699999988079071
-8	6.400000095367432	0.10000000149011612	0.800000011920929
-9	8.099999785423279	0.09999999735090467	0.8999999761581421
+6	3.6000001430511475	0.10000000397364299	0.6
+7	4.899999916553497	0.09999999829701015	0.7
+8	6.400000095367432	0.10000000149011612	0.8
+9	8.099999785423279	0.09999999735090467	0.9
 10	10.0	0.1	1.0
-11	12.100000262260437	0.10000000216744163	1.100000023841858
-12	14.40000057220459	0.10000000397364299	1.2000000476837158
-13	0.10000000149011612	0.10000000149011612	0.10000000149011612
-14	0.4000000059604645	0.10000000149011612	0.20000000298023224
-15	0.9000000357627869	0.10000000397364299	0.30000001192092896
-16	1.600000023841858	0.10000000149011612	0.4000000059604645
+11	12.100000262260437	0.10000000216744163	1.1
+12	14.40000057220459	0.10000000397364299	1.2
+13	0.10000000149011612	0.10000000149011612	0.1
+14	0.4000000059604645	0.10000000149011612	0.2
+15	0.9000000357627869	0.10000000397364299	0.3
+16	1.600000023841858	0.10000000149011612	0.4
 17	2.5	0.1	0.5
-18	3.6000001430511475	0.10000000397364299	0.6000000238418579
-19	4.899999916553497	0.09999999829701015	0.699999988079071
-20	6.400000095367432	0.10000000149011612	0.800000011920929
-21	8.099999785423279	0.09999999735090467	0.8999999761581421
+18	3.6000001430511475	0.10000000397364299	0.6
+19	4.899999916553497	0.09999999829701015	0.7
+20	6.400000095367432	0.10000000149011612	0.8
+21	8.099999785423279	0.09999999735090467	0.9
 22	10.0	0.1	1.0
-23	12.100000262260437	0.10000000216744163	1.100000023841858
-24	14.40000057220459	0.10000000397364299	1.2000000476837158
+23	12.100000262260437	0.10000000216744163	1.1
+24	14.40000057220459	0.10000000397364299	1.2
 
 -- !sql_test_Float_TinyInt_notn_1 --
-1	0.10000000149011612	0.10000000149011612	0.10000000149011612
-2	0.4000000059604645	0.10000000149011612	0.20000000298023224
-3	0.9000000357627869	0.10000000397364299	0.30000001192092896
-4	1.600000023841858	0.10000000149011612	0.4000000059604645
+1	0.10000000149011612	0.10000000149011612	0.1
+2	0.4000000059604645	0.10000000149011612	0.2
+3	0.9000000357627869	0.10000000397364299	0.3
+4	1.600000023841858	0.10000000149011612	0.4
 5	2.5	0.1	0.5
-6	3.6000001430511475	0.10000000397364299	0.6000000238418579
-7	4.899999916553497	0.09999999829701015	0.699999988079071
-8	6.400000095367432	0.10000000149011612	0.800000011920929
-9	8.099999785423279	0.09999999735090467	0.8999999761581421
+6	3.6000001430511475	0.10000000397364299	0.6
+7	4.899999916553497	0.09999999829701015	0.7
+8	6.400000095367432	0.10000000149011612	0.8
+9	8.099999785423279	0.09999999735090467	0.9
 10	10.0	0.1	1.0
-11	12.100000262260437	0.10000000216744163	1.100000023841858
-12	14.40000057220459	0.10000000397364299	1.2000000476837158
-13	0.10000000149011612	0.10000000149011612	0.10000000149011612
-14	0.4000000059604645	0.10000000149011612	0.20000000298023224
-15	0.9000000357627869	0.10000000397364299	0.30000001192092896
-16	1.600000023841858	0.10000000149011612	0.4000000059604645
+11	12.100000262260437	0.10000000216744163	1.1
+12	14.40000057220459	0.10000000397364299	1.2
+13	0.10000000149011612	0.10000000149011612	0.1
+14	0.4000000059604645	0.10000000149011612	0.2
+15	0.9000000357627869	0.10000000397364299	0.3
+16	1.600000023841858	0.10000000149011612	0.4
 17	2.5	0.1	0.5
-18	3.6000001430511475	0.10000000397364299	0.6000000238418579
-19	4.899999916553497	0.09999999829701015	0.699999988079071
-20	6.400000095367432	0.10000000149011612	0.800000011920929
-21	8.099999785423279	0.09999999735090467	0.8999999761581421
+18	3.6000001430511475	0.10000000397364299	0.6
+19	4.899999916553497	0.09999999829701015	0.7
+20	6.400000095367432	0.10000000149011612	0.8
+21	8.099999785423279	0.09999999735090467	0.9
 22	10.0	0.1	1.0
-23	12.100000262260437	0.10000000216744163	1.100000023841858
-24	14.40000057220459	0.10000000397364299	1.2000000476837158
+23	12.100000262260437	0.10000000216744163	1.1
+24	14.40000057220459	0.10000000397364299	1.2
 
 -- !sql_test_Float_TinyInt_2 --
 \N	\N
@@ -319,56 +319,56 @@
 
 -- !sql_test_Float_SmallInt_1 --
 \N	\N	\N	\N
-1	1.0000000149011612	0.010000000149011612	0.10000000149011612
-2	4.000000059604645	0.010000000149011612	0.20000000298023224
-3	12.000000476837158	0.007500000298023224	0.30000001192092896
-4	32.00000047683716	0.005000000074505806	0.4000000059604645
+1	1.0000000149011612	0.010000000149011612	0.1
+2	4.000000059604645	0.010000000149011612	0.2
+3	12.000000476837158	0.007500000298023224	0.3
+4	32.00000047683716	0.005000000074505806	0.4
 5	80.0	0.003125	0.5
-6	192.00000762939453	0.001875000074505806	0.6000000238418579
-7	447.99999237060547	0.0010937499813735486	0.699999988079071
-8	1024.000015258789	6.250000093132258E-4	0.800000011920929
-9	2303.9999389648438	3.5156249068677423E-4	0.8999999761581421
+6	192.00000762939453	0.001875000074505806	0.6
+7	447.99999237060547	0.0010937499813735486	0.7
+8	1024.000015258789	6.250000093132258E-4	0.8
+9	2303.9999389648438	3.5156249068677423E-4	0.9
 10	5120.0	1.953125E-4	1.0
-11	11264.000244140625	1.0742187732830644E-4	1.100000023841858
-12	24576.0009765625	5.8593752328306437E-5	1.2000000476837158
-13	1.0000000149011612	0.010000000149011612	0.10000000149011612
-14	4.000000059604645	0.010000000149011612	0.20000000298023224
-15	12.000000476837158	0.007500000298023224	0.30000001192092896
-16	32.00000047683716	0.005000000074505806	0.4000000059604645
+11	11264.000244140625	1.0742187732830644E-4	1.1
+12	24576.0009765625	5.8593752328306437E-5	1.2
+13	1.0000000149011612	0.010000000149011612	0.1
+14	4.000000059604645	0.010000000149011612	0.2
+15	12.000000476837158	0.007500000298023224	0.3
+16	32.00000047683716	0.005000000074505806	0.4
 17	80.0	0.003125	0.5
-18	192.00000762939453	0.001875000074505806	0.6000000238418579
-19	447.99999237060547	0.0010937499813735486	0.699999988079071
-20	1024.000015258789	6.250000093132258E-4	0.800000011920929
-21	2303.9999389648438	3.5156249068677423E-4	0.8999999761581421
+18	192.00000762939453	0.001875000074505806	0.6
+19	447.99999237060547	0.0010937499813735486	0.7
+20	1024.000015258789	6.250000093132258E-4	0.8
+21	2303.9999389648438	3.5156249068677423E-4	0.9
 22	5120.0	1.953125E-4	1.0
-23	11264.000244140625	1.0742187732830644E-4	1.100000023841858
-24	24576.0009765625	5.8593752328306437E-5	1.2000000476837158
+23	11264.000244140625	1.0742187732830644E-4	1.1
+24	24576.0009765625	5.8593752328306437E-5	1.2
 
 -- !sql_test_Float_SmallInt_notn_1 --
-1	1.0000000149011612	0.010000000149011612	0.10000000149011612
-2	4.000000059604645	0.010000000149011612	0.20000000298023224
-3	12.000000476837158	0.007500000298023224	0.30000001192092896
-4	32.00000047683716	0.005000000074505806	0.4000000059604645
+1	1.0000000149011612	0.010000000149011612	0.1
+2	4.000000059604645	0.010000000149011612	0.2
+3	12.000000476837158	0.007500000298023224	0.3
+4	32.00000047683716	0.005000000074505806	0.4
 5	80.0	0.003125	0.5
-6	192.00000762939453	0.001875000074505806	0.6000000238418579
-7	447.99999237060547	0.0010937499813735486	0.699999988079071
-8	1024.000015258789	6.250000093132258E-4	0.800000011920929
-9	2303.9999389648438	3.5156249068677423E-4	0.8999999761581421
+6	192.00000762939453	0.001875000074505806	0.6
+7	447.99999237060547	0.0010937499813735486	0.7
+8	1024.000015258789	6.250000093132258E-4	0.8
+9	2303.9999389648438	3.5156249068677423E-4	0.9
 10	5120.0	1.953125E-4	1.0
-11	11264.000244140625	1.0742187732830644E-4	1.100000023841858
-12	24576.0009765625	5.8593752328306437E-5	1.2000000476837158
-13	1.0000000149011612	0.010000000149011612	0.10000000149011612
-14	4.000000059604645	0.010000000149011612	0.20000000298023224
-15	12.000000476837158	0.007500000298023224	0.30000001192092896
-16	32.00000047683716	0.005000000074505806	0.4000000059604645
+11	11264.000244140625	1.0742187732830644E-4	1.1
+12	24576.0009765625	5.8593752328306437E-5	1.2
+13	1.0000000149011612	0.010000000149011612	0.1
+14	4.000000059604645	0.010000000149011612	0.2
+15	12.000000476837158	0.007500000298023224	0.3
+16	32.00000047683716	0.005000000074505806	0.4
 17	80.0	0.003125	0.5
-18	192.00000762939453	0.001875000074505806	0.6000000238418579
-19	447.99999237060547	0.0010937499813735486	0.699999988079071
-20	1024.000015258789	6.250000093132258E-4	0.800000011920929
-21	2303.9999389648438	3.5156249068677423E-4	0.8999999761581421
+18	192.00000762939453	0.001875000074505806	0.6
+19	447.99999237060547	0.0010937499813735486	0.7
+20	1024.000015258789	6.250000093132258E-4	0.8
+21	2303.9999389648438	3.5156249068677423E-4	0.9
 22	5120.0	1.953125E-4	1.0
-23	11264.000244140625	1.0742187732830644E-4	1.100000023841858
-24	24576.0009765625	5.8593752328306437E-5	1.2000000476837158
+23	11264.000244140625	1.0742187732830644E-4	1.1
+24	24576.0009765625	5.8593752328306437E-5	1.2
 
 -- !sql_test_Float_SmallInt_2 --
 \N	\N
@@ -476,59 +476,6 @@
 23	0	10241	10241
 24	0	20481	20481
 
--- !sql_test_Float_SmallInt_4 --
-\N	\N	\N	\N
-1	0	10	10
-2	0	20	20
-3	0	40	40
-4	0	80	80
-5	0	-96	-96
-6	0	64	64
-7	0	-128	-128
-8	0	0	0
-9	0	0	0
-10	0	1	1
-11	0	1	1
-12	0	1	1
-13	0	10	10
-14	0	20	20
-15	0	40	40
-16	0	80	80
-17	0	-96	-96
-18	0	64	64
-19	0	-128	-128
-20	0	0	0
-21	0	0	0
-22	0	1	1
-23	0	1	1
-24	0	1	1
-
--- !sql_test_Float_SmallInt_notn_4 --
-1	0	10	10
-2	0	20	20
-3	0	40	40
-4	0	80	80
-5	0	-96	-96
-6	0	64	64
-7	0	-128	-128
-8	0	0	0
-9	0	0	0
-10	0	1	1
-11	0	1	1
-12	0	1	1
-13	0	10	10
-14	0	20	20
-15	0	40	40
-16	0	80	80
-17	0	-96	-96
-18	0	64	64
-19	0	-128	-128
-20	0	0	0
-21	0	0	0
-22	0	1	1
-23	0	1	1
-24	0	1	1
-
 -- !sql_test_Float_Integer_0 --
 \N	\N	\N
 1	23795.10000000149	-23794.89999999851
@@ -584,56 +531,56 @@
 
 -- !sql_test_Float_Integer_1 --
 \N	\N	\N	\N
-1	2379.500035457313	4.202563626396979E-6	0.10000000149011612
-2	9509.000141695142	4.206541234204064E-6	0.20000000298023224
-3	28513.501133024693	3.1563997256134354E-6	0.30000001192092896
-4	76018.00113275647	2.1047646923647794E-6	0.4000000059604645
+1	2379.500035457313	4.202563626396979E-6	0.1
+2	9509.000141695142	4.206541234204064E-6	0.2
+3	28513.501133024693	3.1563997256134354E-6	0.3
+4	76018.00113275647	2.1047646923647794E-6	0.4
 5	190022.5	1.3156336749595442E-6	0.5
-6	456027.0181208849	7.894269731948212E-7	0.6000000238418579
-7	1064031.4818796515	4.6051267434784567E-7	0.699999988079071
-8	2432036.0362401605	2.631540032864411E-7	0.800000011920929
-9	5472040.355040431	1.4802521628674494E-7	0.8999999761581421
+6	456027.0181208849	7.894269731948212E-7	0.6
+7	1064031.4818796515	4.6051267434784567E-7	0.7
+8	2432036.0362401605	2.631540032864411E-7	0.8
+9	5472040.355040431	1.4802521628674494E-7	0.9
 10	1.2160045E7	8.22365377759704E-8	1.0
-11	2.6752050079835057E7	4.5230180447522115E-8	1.100000023841858
-12	5.836805631933808E7	2.4671030787157286E-8	1.2000000476837158
-13	2379.500035457313	4.202563626396979E-6	0.10000000149011612
-14	9509.000141695142	4.206541234204064E-6	0.20000000298023224
-15	28513.501133024693	3.1563997256134354E-6	0.30000001192092896
-16	76018.00113275647	2.1047646923647794E-6	0.4000000059604645
+11	2.6752050079835057E7	4.5230180447522115E-8	1.1
+12	5.836805631933808E7	2.4671030787157286E-8	1.2
+13	2379.500035457313	4.202563626396979E-6	0.1
+14	9509.000141695142	4.206541234204064E-6	0.2
+15	28513.501133024693	3.1563997256134354E-6	0.3
+16	76018.00113275647	2.1047646923647794E-6	0.4
 17	190022.5	1.3156336749595442E-6	0.5
-18	456027.0181208849	7.894269731948212E-7	0.6000000238418579
-19	1064031.4818796515	4.6051267434784567E-7	0.699999988079071
-20	2432036.0362401605	2.631540032864411E-7	0.800000011920929
-21	5472040.355040431	1.4802521628674494E-7	0.8999999761581421
+18	456027.0181208849	7.894269731948212E-7	0.6
+19	1064031.4818796515	4.6051267434784567E-7	0.7
+20	2432036.0362401605	2.631540032864411E-7	0.8
+21	5472040.355040431	1.4802521628674494E-7	0.9
 22	1.2160045E7	8.22365377759704E-8	1.0
-23	2.6752050079835057E7	4.5230180447522115E-8	1.100000023841858
-24	5.836805631933808E7	2.4671030787157286E-8	1.2000000476837158
+23	2.6752050079835057E7	4.5230180447522115E-8	1.1
+24	5.836805631933808E7	2.4671030787157286E-8	1.2
 
 -- !sql_test_Float_Integer_notn_1 --
-1	2379.500035457313	4.202563626396979E-6	0.10000000149011612
-2	9509.000141695142	4.206541234204064E-6	0.20000000298023224
-3	28513.501133024693	3.1563997256134354E-6	0.30000001192092896
-4	76018.00113275647	2.1047646923647794E-6	0.4000000059604645
+1	2379.500035457313	4.202563626396979E-6	0.1
+2	9509.000141695142	4.206541234204064E-6	0.2
+3	28513.501133024693	3.1563997256134354E-6	0.3
+4	76018.00113275647	2.1047646923647794E-6	0.4
 5	190022.5	1.3156336749595442E-6	0.5
-6	456027.0181208849	7.894269731948212E-7	0.6000000238418579
-7	1064031.4818796515	4.6051267434784567E-7	0.699999988079071
-8	2432036.0362401605	2.631540032864411E-7	0.800000011920929
-9	5472040.355040431	1.4802521628674494E-7	0.8999999761581421
+6	456027.0181208849	7.894269731948212E-7	0.6
+7	1064031.4818796515	4.6051267434784567E-7	0.7
+8	2432036.0362401605	2.631540032864411E-7	0.8
+9	5472040.355040431	1.4802521628674494E-7	0.9
 10	1.2160045E7	8.22365377759704E-8	1.0
-11	2.6752050079835057E7	4.5230180447522115E-8	1.100000023841858
-12	5.836805631933808E7	2.4671030787157286E-8	1.2000000476837158
-13	2379.500035457313	4.202563626396979E-6	0.10000000149011612
-14	9509.000141695142	4.206541234204064E-6	0.20000000298023224
-15	28513.501133024693	3.1563997256134354E-6	0.30000001192092896
-16	76018.00113275647	2.1047646923647794E-6	0.4000000059604645
+11	2.6752050079835057E7	4.5230180447522115E-8	1.1
+12	5.836805631933808E7	2.4671030787157286E-8	1.2
+13	2379.500035457313	4.202563626396979E-6	0.1
+14	9509.000141695142	4.206541234204064E-6	0.2
+15	28513.501133024693	3.1563997256134354E-6	0.3
+16	76018.00113275647	2.1047646923647794E-6	0.4
 17	190022.5	1.3156336749595442E-6	0.5
-18	456027.0181208849	7.894269731948212E-7	0.6000000238418579
-19	1064031.4818796515	4.6051267434784567E-7	0.699999988079071
-20	2432036.0362401605	2.631540032864411E-7	0.800000011920929
-21	5472040.355040431	1.4802521628674494E-7	0.8999999761581421
+18	456027.0181208849	7.894269731948212E-7	0.6
+19	1064031.4818796515	4.6051267434784567E-7	0.7
+20	2432036.0362401605	2.631540032864411E-7	0.8
+21	5472040.355040431	1.4802521628674494E-7	0.9
 22	1.2160045E7	8.22365377759704E-8	1.0
-23	2.6752050079835057E7	4.5230180447522115E-8	1.100000023841858
-24	5.836805631933808E7	2.4671030787157286E-8	1.2000000476837158
+23	2.6752050079835057E7	4.5230180447522115E-8	1.1
+24	5.836805631933808E7	2.4671030787157286E-8	1.2
 
 -- !sql_test_Float_Integer_2 --
 \N	\N
@@ -741,59 +688,6 @@
 23	1	24320045	24320044
 24	1	48640045	48640044
 
--- !sql_test_Float_Integer_4 --
-\N	\N	\N	\N
-1	0	-13	-13
-2	0	-71	-71
-3	0	69	69
-4	0	93	93
-5	0	-115	-115
-6	0	-19	-19
-7	0	-83	-83
-8	0	45	45
-9	0	45	45
-10	1	45	44
-11	1	45	44
-12	1	45	44
-13	0	-13	-13
-14	0	-71	-71
-15	0	69	69
-16	0	93	93
-17	0	-115	-115
-18	0	-19	-19
-19	0	-83	-83
-20	0	45	45
-21	0	45	45
-22	1	45	44
-23	1	45	44
-24	1	45	44
-
--- !sql_test_Float_Integer_notn_4 --
-1	0	-13	-13
-2	0	-71	-71
-3	0	69	69
-4	0	93	93
-5	0	-115	-115
-6	0	-19	-19
-7	0	-83	-83
-8	0	45	45
-9	0	45	45
-10	1	45	44
-11	1	45	44
-12	1	45	44
-13	0	-13	-13
-14	0	-71	-71
-15	0	69	69
-16	0	93	93
-17	0	-115	-115
-18	0	-19	-19
-19	0	-83	-83
-20	0	45	45
-21	0	45	45
-22	1	45	44
-23	1	45	44
-24	1	45	44
-
 -- !sql_test_Float_BigInt_0 --
 \N	\N	\N
 1	5354529.1000000015	-5354528.8999999985
@@ -847,59 +741,6 @@
 23	5.4720107801E9	-5.4720107779E9
 24	1.09440107802E10	-1.09440107778E10
 
--- !sql_test_Float_BigInt_1 --
-\N	\N	\N	\N
-1	535452.90797887	1.8675779231024078E-8	0.10000000149011612
-2	2139655.831883356	1.869459592334732E-8	0.20000000298023224
-3	6415733.954938352	1.4028014220147368E-8	0.30000001192092896
-4	1.7104311854874104E7	9.354366672329905E-9	0.4000000059604645
-5	4.27553895E7	5.847216056820158E-9	0.5
-6	1.026064714772147E8	3.508550907436413E-9	0.6000000238418579
-7	2.394075412229138E8	2.0467190833159998E-9	0.699999988079071
-8	5.47208631354044E8	1.1695722296810895E-9	0.800000011920929
-9	1.2312096684840813E9	6.578895356482729E-10	0.8999999761581421
-10	2.736010779E9	3.654956360827992E-10	1.0
-11	6.019211987362904E9	2.010230001854786E-10	1.100000023841858
-12	1.31328134566511E10	1.0964901916821438E-10	1.2000000476837158
-13	535452.90797887	1.8675779231024078E-8	0.10000000149011612
-14	2139655.831883356	1.869459592334732E-8	0.20000000298023224
-15	6415733.954938352	1.4028014220147368E-8	0.30000001192092896
-16	1.7104311854874104E7	9.354366672329905E-9	0.4000000059604645
-17	4.27553895E7	5.847216056820158E-9	0.5
-18	1.026064714772147E8	3.508550907436413E-9	0.6000000238418579
-19	2.394075412229138E8	2.0467190833159998E-9	0.699999988079071
-20	5.47208631354044E8	1.1695722296810895E-9	0.800000011920929
-21	1.2312096684840813E9	6.578895356482729E-10	0.8999999761581421
-22	2.736010779E9	3.654956360827992E-10	1.0
-23	6.019211987362904E9	2.010230001854786E-10	1.100000023841858
-24	1.31328134566511E10	1.0964901916821438E-10	1.2000000476837158
-
--- !sql_test_Float_BigInt_notn_1 --
-1	535452.90797887	1.8675779231024078E-8	0.10000000149011612
-2	2139655.831883356	1.869459592334732E-8	0.20000000298023224
-3	6415733.954938352	1.4028014220147368E-8	0.30000001192092896
-4	1.7104311854874104E7	9.354366672329905E-9	0.4000000059604645
-5	4.27553895E7	5.847216056820158E-9	0.5
-6	1.026064714772147E8	3.508550907436413E-9	0.6000000238418579
-7	2.394075412229138E8	2.0467190833159998E-9	0.699999988079071
-8	5.47208631354044E8	1.1695722296810895E-9	0.800000011920929
-9	1.2312096684840813E9	6.578895356482729E-10	0.8999999761581421
-10	2.736010779E9	3.654956360827992E-10	1.0
-11	6.019211987362904E9	2.010230001854786E-10	1.100000023841858
-12	1.31328134566511E10	1.0964901916821438E-10	1.2000000476837158
-13	535452.90797887	1.8675779231024078E-8	0.10000000149011612
-14	2139655.831883356	1.869459592334732E-8	0.20000000298023224
-15	6415733.954938352	1.4028014220147368E-8	0.30000001192092896
-16	1.7104311854874104E7	9.354366672329905E-9	0.4000000059604645
-17	4.27553895E7	5.847216056820158E-9	0.5
-18	1.026064714772147E8	3.508550907436413E-9	0.6000000238418579
-19	2.394075412229138E8	2.0467190833159998E-9	0.699999988079071
-20	5.47208631354044E8	1.1695722296810895E-9	0.800000011920929
-21	1.2312096684840813E9	6.578895356482729E-10	0.8999999761581421
-22	2.736010779E9	3.654956360827992E-10	1.0
-23	6.019211987362904E9	2.010230001854786E-10	1.100000023841858
-24	1.31328134566511E10	1.0964901916821438E-10	1.2000000476837158
-
 -- !sql_test_Float_BigInt_2 --
 \N	\N
 1	0
@@ -1006,59 +847,6 @@
 23	1	5472010779	5472010778
 24	1	10944010779	10944010778
 
--- !sql_test_Float_BigInt_4 --
-\N	\N	\N	\N
-1	0	33	33
-2	0	39	39
-3	0	51	51
-4	0	75	75
-5	0	123	123
-6	0	-37	-37
-7	0	-101	-101
-8	0	27	27
-9	0	27	27
-10	1	27	26
-11	1	27	26
-12	1	27	26
-13	0	33	33
-14	0	39	39
-15	0	51	51
-16	0	75	75
-17	0	123	123
-18	0	-37	-37
-19	0	-101	-101
-20	0	27	27
-21	0	27	27
-22	1	27	26
-23	1	27	26
-24	1	27	26
-
--- !sql_test_Float_BigInt_notn_4 --
-1	0	33	33
-2	0	39	39
-3	0	51	51
-4	0	75	75
-5	0	123	123
-6	0	-37	-37
-7	0	-101	-101
-8	0	27	27
-9	0	27	27
-10	1	27	26
-11	1	27	26
-12	1	27	26
-13	0	33	33
-14	0	39	39
-15	0	51	51
-16	0	75	75
-17	0	123	123
-18	0	-37	-37
-19	0	-101	-101
-20	0	27	27
-21	0	27	27
-22	1	27	26
-23	1	27	26
-24	1	27	26
-
 -- !sql_test_Float_LargeInt_0 --
 \N	\N	\N
 1	1.070906451E8	-1.070906449E8
@@ -1112,59 +900,6 @@
 23	1.094402156461E11	-1.094402156439E11
 24	2.188802156462E11	-2.188802156438E11
 
--- !sql_test_Float_LargeInt_1 --
-\N	\N	\N	\N
-1	1.0709064659577496E7	9.337883947763702E-10	0.10000000149011612
-2	4.279312963766731E7	9.34729512208524E-10	0.20000000298023224
-3	1.2831469859876782E8	7.014006044154147E-10	0.30000001192092896
-4	3.4208626309748244E8	4.67718298067927E-10	0.4000000059604645
-5	8.551078225E8	2.9236079172927925E-10	0.5
-6	2.0521294685442953E9	1.7542754203788161E-10	0.6000000238418579
-7	4.788150869958275E9	1.0233595319333986E-10	0.699999988079071
-8	1.094417267908088E10	5.847861120619995E-11	0.800000011920929
-9	2.4624193428181625E10	3.2894476704265833E-11	0.8999999761581421
-10	5.4720215645E10	1.8274781782432063E-11	1.0
-11	1.2038423981875807E11	1.0051150003304234E-11	1.100000023841858
-12	2.62656269211022E11	5.482450956782617E-12	1.2000000476837158
-13	1.0709064659577496E7	9.337883947763702E-10	0.10000000149011612
-14	4.279312963766731E7	9.34729512208524E-10	0.20000000298023224
-15	1.2831469859876782E8	7.014006044154147E-10	0.30000001192092896
-16	3.4208626309748244E8	4.67718298067927E-10	0.4000000059604645
-17	8.551078225E8	2.9236079172927925E-10	0.5
-18	2.0521294685442953E9	1.7542754203788161E-10	0.6000000238418579
-19	4.788150869958275E9	1.0233595319333986E-10	0.699999988079071
-20	1.094417267908088E10	5.847861120619995E-11	0.800000011920929
-21	2.4624193428181625E10	3.2894476704265833E-11	0.8999999761581421
-22	5.4720215645E10	1.8274781782432063E-11	1.0
-23	1.2038423981875807E11	1.0051150003304234E-11	1.100000023841858
-24	2.62656269211022E11	5.482450956782617E-12	1.2000000476837158
-
--- !sql_test_Float_LargeInt_notn_1 --
-1	1.0709064659577496E7	9.337883947763702E-10	0.10000000149011612
-2	4.279312963766731E7	9.34729512208524E-10	0.20000000298023224
-3	1.2831469859876782E8	7.014006044154147E-10	0.30000001192092896
-4	3.4208626309748244E8	4.67718298067927E-10	0.4000000059604645
-5	8.551078225E8	2.9236079172927925E-10	0.5
-6	2.0521294685442953E9	1.7542754203788161E-10	0.6000000238418579
-7	4.788150869958275E9	1.0233595319333986E-10	0.699999988079071
-8	1.094417267908088E10	5.847861120619995E-11	0.800000011920929
-9	2.4624193428181625E10	3.2894476704265833E-11	0.8999999761581421
-10	5.4720215645E10	1.8274781782432063E-11	1.0
-11	1.2038423981875807E11	1.0051150003304234E-11	1.100000023841858
-12	2.62656269211022E11	5.482450956782617E-12	1.2000000476837158
-13	1.0709064659577496E7	9.337883947763702E-10	0.10000000149011612
-14	4.279312963766731E7	9.34729512208524E-10	0.20000000298023224
-15	1.2831469859876782E8	7.014006044154147E-10	0.30000001192092896
-16	3.4208626309748244E8	4.67718298067927E-10	0.4000000059604645
-17	8.551078225E8	2.9236079172927925E-10	0.5
-18	2.0521294685442953E9	1.7542754203788161E-10	0.6000000238418579
-19	4.788150869958275E9	1.0233595319333986E-10	0.699999988079071
-20	1.094417267908088E10	5.847861120619995E-11	0.800000011920929
-21	2.4624193428181625E10	3.2894476704265833E-11	0.8999999761581421
-22	5.4720215645E10	1.8274781782432063E-11	1.0
-23	1.2038423981875807E11	1.0051150003304234E-11	1.100000023841858
-24	2.62656269211022E11	5.482450956782617E-12	1.2000000476837158
-
 -- !sql_test_Float_LargeInt_2 --
 \N	\N
 1	0
@@ -1271,59 +1006,6 @@
 23	1	109440215645	109440215644
 24	1	218880215645	218880215644
 
--- !sql_test_Float_LargeInt_4 --
-\N	\N	\N	\N
-1	0	-43	-43
-2	0	77	77
-3	0	61	61
-4	0	29	29
-5	0	-35	-35
-6	0	93	93
-7	0	93	93
-8	0	93	93
-9	0	93	93
-10	1	93	92
-11	1	93	92
-12	1	93	92
-13	0	-43	-43
-14	0	77	77
-15	0	61	61
-16	0	29	29
-17	0	-35	-35
-18	0	93	93
-19	0	93	93
-20	0	93	93
-21	0	93	93
-22	1	93	92
-23	1	93	92
-24	1	93	92
-
--- !sql_test_Float_LargeInt_notn_4 --
-1	0	-43	-43
-2	0	77	77
-3	0	61	61
-4	0	29	29
-5	0	-35	-35
-6	0	93	93
-7	0	93	93
-8	0	93	93
-9	0	93	93
-10	1	93	92
-11	1	93	92
-12	1	93	92
-13	0	-43	-43
-14	0	77	77
-15	0	61	61
-16	0	29	29
-17	0	-35	-35
-18	0	93	93
-19	0	93	93
-20	0	93	93
-21	0	93	93
-22	1	93	92
-23	1	93	92
-24	1	93	92
-
 -- !sql_test_Float_Float_0 --
 \N	\N	\N
 1	0.20000000298023224	0.0
@@ -2066,164 +1748,58 @@
 23	1	779	778
 24	0	1103	1103
 
--- !sql_test_Float_DecimalV2_4 --
-\N	\N	\N	\N
-1	0	24	24
-2	0	34	34
-3	0	48	48
-4	0	68	68
-5	0	97	97
-6	0	127	127
-7	0	127	127
-8	0	127	127
-9	0	127	127
-10	1	127	126
-11	1	127	126
-12	1	127	126
-13	0	24	24
-14	0	34	34
-15	0	48	48
-16	0	68	68
-17	0	97	97
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
--- !sql_test_Float_DecimalV2_notn_4 --
-1	0	24	24
-2	0	34	34
-3	0	48	48
-4	0	68	68
-5	0	97	97
-6	0	127	127
-7	0	127	127
-8	0	127	127
-9	0	127	127
-10	1	127	126
-11	1	127	126
-12	1	127	126
-13	0	24	24
-14	0	34	34
-15	0	48	48
-16	0	68	68
-17	0	97	97
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
 -- !sql_test_Float_Decimal32V3_0 --
 \N	\N	\N
-1	12.112000001490117	-11.911999998509884
-2	23.323000002980233	-22.92299999701977
-3	34.53400001192093	-33.93399998807907
-4	45.74500000596046	-44.944999994039534
-5	56.956	-55.956
-6	68.16700002384185	-66.96699997615814
-7	79.37799998807907	-77.97800001192093
-8	90.58900001192093	-88.98899998807907
-9	101.79999997615815	-100.00000002384186
-10	113.011	-111.011
-11	124.22200002384186	-122.02199997615814
-12	135.43300004768372	-133.0329999523163
-13	145.4440000014901	-145.24399999850988
-14	156.65500000298024	-156.25499999701978
-15	167.86600001192093	-167.26599998807907
-16	179.07700000596046	-178.27699999403953
-17	190.288	-189.288
-18	201.49900002384186	-200.29899997615814
-19	212.70999998807906	-211.31000001192092
-20	223.92100001192094	-222.32099998807908
-21	235.13199997615814	-233.33200002384186
-22	246.343	-244.343
-23	257.55400002384187	-255.35399997615815
-24	268.7650000476837	-266.3649999523163
+1	12.1120000	-11.9120000
+2	23.3230000	-22.9230000
+3	34.5340000	-33.9340000
+4	45.7450000	-44.9450000
+5	56.9560000	-55.9560000
+6	68.1670000	-66.9670000
+7	79.3780000	-77.9780000
+8	90.5890000	-88.9890000
+9	101.8000000	-100.0000000
+10	113.0110000	-111.0110000
+11	124.2220000	-122.0220000
+12	135.4330000	-133.0330000
+13	145.4440000	-145.2440000
+14	156.6550000	-156.2550000
+15	167.8660000	-167.2660000
+16	179.0770000	-178.2770000
+17	190.2880000	-189.2880000
+18	201.4990000	-200.2990000
+19	212.7100000	-211.3100000
+20	223.9210000	-222.3210000
+21	235.1320000	-233.3320000
+22	246.3430000	-244.3430000
+23	257.5540000	-255.3540000
+24	268.7650000	-266.3650000
 
 -- !sql_test_Float_Decimal32V3_notn_0 --
-1	12.112000001490117	-11.911999998509884
-2	23.323000002980233	-22.92299999701977
-3	34.53400001192093	-33.93399998807907
-4	45.74500000596046	-44.944999994039534
-5	56.956	-55.956
-6	68.16700002384185	-66.96699997615814
-7	79.37799998807907	-77.97800001192093
-8	90.58900001192093	-88.98899998807907
-9	101.79999997615815	-100.00000002384186
-10	113.011	-111.011
-11	124.22200002384186	-122.02199997615814
-12	135.43300004768372	-133.0329999523163
-13	145.4440000014901	-145.24399999850988
-14	156.65500000298024	-156.25499999701978
-15	167.86600001192093	-167.26599998807907
-16	179.07700000596046	-178.27699999403953
-17	190.288	-189.288
-18	201.49900002384186	-200.29899997615814
-19	212.70999998807906	-211.31000001192092
-20	223.92100001192094	-222.32099998807908
-21	235.13199997615814	-233.33200002384186
-22	246.343	-244.343
-23	257.55400002384187	-255.35399997615815
-24	268.7650000476837	-266.3649999523163
-
--- !sql_test_Float_Decimal32V3_1 --
-\N	\N	\N	\N
-1	1.2012000178992748	0.008325008449060615	0.10000000149011612
-2	4.62460006891191	0.00864939683346591	0.20000000298023224
-3	10.270200408101083	0.008763218201814832	0.30000001192092896
-4	18.13800027027726	0.008821259366202767	0.4000000059604645
-5	28.228	0.008856454584100892	0.5
-6	40.54020161092281	0.00888007494548904	0.6000000238418579
-7	55.07459906208515	0.00889702315868567	0.699999988079071
-8	71.8312010703683	0.008909777499704072	0.800000011920929
-9	90.80999759435655	0.008919722261230348	0.8999999761581421
-10	112.011	0.008927694601423074	1.0
-11	135.43420293545722	0.00893422803269812	1.100000023841858
-12	161.07960640072824	0.008939679867720424	1.2000000476837158
-13	14.534400216579437	6.880229076543657E-4	0.10000000149011612
-14	31.291000466272237	0.001278322859481846	0.20000000298023224
-15	50.26980199754238	0.0017903394001225125	0.30000001192092896
-16	71.47080106499791	0.0022386765278153567	0.4000000059604645
-17	94.894	0.002634518515396126	0.5
-18	120.53940478980542	0.0029865754625053283	0.6000000238418579
-19	148.40699747264384	0.0033017309941940054	0.699999988079071
-20	178.4968026598096	0.0035854985049409466	0.800000011920929
-21	210.80879441547393	0.003842344240574055	0.8999999761581421
-22	245.343	0.004075926356162597	1.0
-23	282.09940611433984	0.0042892683438037925	1.100000023841858
-24	321.0780127584934	0.004484891699899896	1.2000000476837158
-
--- !sql_test_Float_Decimal32V3_notn_1 --
-1	1.2012000178992748	0.008325008449060615	0.10000000149011612
-2	4.62460006891191	0.00864939683346591	0.20000000298023224
-3	10.270200408101083	0.008763218201814832	0.30000001192092896
-4	18.13800027027726	0.008821259366202767	0.4000000059604645
-5	28.228	0.008856454584100892	0.5
-6	40.54020161092281	0.00888007494548904	0.6000000238418579
-7	55.07459906208515	0.00889702315868567	0.699999988079071
-8	71.8312010703683	0.008909777499704072	0.800000011920929
-9	90.80999759435655	0.008919722261230348	0.8999999761581421
-10	112.011	0.008927694601423074	1.0
-11	135.43420293545722	0.00893422803269812	1.100000023841858
-12	161.07960640072824	0.008939679867720424	1.2000000476837158
-13	14.534400216579437	6.880229076543657E-4	0.10000000149011612
-14	31.291000466272237	0.001278322859481846	0.20000000298023224
-15	50.26980199754238	0.0017903394001225125	0.30000001192092896
-16	71.47080106499791	0.0022386765278153567	0.4000000059604645
-17	94.894	0.002634518515396126	0.5
-18	120.53940478980542	0.0029865754625053283	0.6000000238418579
-19	148.40699747264384	0.0033017309941940054	0.699999988079071
-20	178.4968026598096	0.0035854985049409466	0.800000011920929
-21	210.80879441547393	0.003842344240574055	0.8999999761581421
-22	245.343	0.004075926356162597	1.0
-23	282.09940611433984	0.0042892683438037925	1.100000023841858
-24	321.0780127584934	0.004484891699899896	1.2000000476837158
+1	12.1120000	-11.9120000
+2	23.3230000	-22.9230000
+3	34.5340000	-33.9340000
+4	45.7450000	-44.9450000
+5	56.9560000	-55.9560000
+6	68.1670000	-66.9670000
+7	79.3780000	-77.9780000
+8	90.5890000	-88.9890000
+9	101.8000000	-100.0000000
+10	113.0110000	-111.0110000
+11	124.2220000	-122.0220000
+12	135.4330000	-133.0330000
+13	145.4440000	-145.2440000
+14	156.6550000	-156.2550000
+15	167.8660000	-167.2660000
+16	179.0770000	-178.2770000
+17	190.2880000	-189.2880000
+18	201.4990000	-200.2990000
+19	212.7100000	-211.3100000
+20	223.9210000	-222.3210000
+21	235.1320000	-233.3320000
+22	246.3430000	-244.3430000
+23	257.5540000	-255.3540000
+24	268.7650000	-266.3650000
 
 -- !sql_test_Float_Decimal32V3_2 --
 \N	\N
@@ -2331,164 +1907,58 @@
 23	0	257	257
 24	1	267	266
 
--- !sql_test_Float_Decimal32V3_4 --
-\N	\N	\N	\N
-1	0	12	12
-2	0	23	23
-3	0	34	34
-4	0	45	45
-5	0	56	56
-6	0	67	67
-7	0	78	78
-8	0	89	89
-9	0	100	100
-10	0	113	113
-11	1	123	122
-12	1	127	126
-13	0	127	127
-14	0	127	127
-15	0	127	127
-16	0	127	127
-17	0	127	127
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
--- !sql_test_Float_Decimal32V3_notn_4 --
-1	0	12	12
-2	0	23	23
-3	0	34	34
-4	0	45	45
-5	0	56	56
-6	0	67	67
-7	0	78	78
-8	0	89	89
-9	0	100	100
-10	0	113	113
-11	1	123	122
-12	1	127	126
-13	0	127	127
-14	0	127	127
-15	0	127	127
-16	0	127	127
-17	0	127	127
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
 -- !sql_test_Float_Decimal64V3_0 --
 \N	\N	\N
-1	1234.11234000149	-1233.9123399985099
-2	2345.3234500029803	-2344.92344999702
-3	3456.534560011921	-3455.934559988079
-4	4567.74567000596	-4566.945669994039
-5	5678.95678	-5677.95678
-6	6790.167890023842	-6788.967889976158
-7	7901.378999988079	-7899.979000011921
-8	9012.59011001192	-9010.990109988079
-9	10123.801219976158	-10122.001220023842
-10	11235.01233	-11233.01233
-11	12346.223440023841	-12344.023439976158
-12	13457.434550047683	-13455.034549952316
-13	14567.445660001491	-14567.24565999851
-14	15678.656770002981	-15678.25676999702
-15	16789.86788001192	-16789.267879988078
-16	17901.07899000596	-17900.27898999404
-17	19012.2901	-19011.2901
-18	20123.501210023842	-20122.301209976158
-19	21234.71231998808	-21233.312320011923
-20	22345.92343001192	-22344.32342998808
-21	23457.13453997616	-23455.334540023843
-22	24568.34565	-24566.34565
-23	25679.556760023843	-25677.35675997616
-24	26790.767870047683	-26788.367869952315
+1	1234.1123400	-1233.9123400
+2	2345.3234500	-2344.9234500
+3	3456.5345600	-3455.9345600
+4	4567.7456700	-4566.9456700
+5	5678.9567800	-5677.9567800
+6	6790.1678900	-6788.9678900
+7	7901.3790000	-7899.9790000
+8	9012.5901100	-9010.9901100
+9	10123.8012200	-10122.0012200
+10	11235.0123300	-11233.0123300
+11	12346.2234400	-12344.0234400
+12	13457.4345500	-13455.0345500
+13	14567.4456600	-14567.2456600
+14	15678.6567700	-15678.2567700
+15	16789.8678800	-16789.2678800
+16	17901.0789900	-17900.2789900
+17	19012.2901000	-19011.2901000
+18	20123.5012100	-20122.3012100
+19	21234.7123200	-21233.3123200
+20	22345.9234300	-22344.3234300
+21	23457.1345400	-23455.3345400
+22	24568.3456500	-24566.3456500
+23	25679.5567600	-25677.3567600
+24	26790.7678700	-26788.3678700
 
 -- !sql_test_Float_Decimal64V3_notn_0 --
-1	1234.11234000149	-1233.9123399985099
-2	2345.3234500029803	-2344.92344999702
-3	3456.534560011921	-3455.934559988079
-4	4567.74567000596	-4566.945669994039
-5	5678.95678	-5677.95678
-6	6790.167890023842	-6788.967889976158
-7	7901.378999988079	-7899.979000011921
-8	9012.59011001192	-9010.990109988079
-9	10123.801219976158	-10122.001220023842
-10	11235.01233	-11233.01233
-11	12346.223440023841	-12344.023439976158
-12	13457.434550047683	-13455.034549952316
-13	14567.445660001491	-14567.24565999851
-14	15678.656770002981	-15678.25676999702
-15	16789.86788001192	-16789.267879988078
-16	17901.07899000596	-17900.27898999404
-17	19012.2901	-19011.2901
-18	20123.501210023842	-20122.301209976158
-19	21234.71231998808	-21233.312320011923
-20	22345.92343001192	-22344.32342998808
-21	23457.13453997616	-23455.334540023843
-22	24568.34565	-24566.34565
-23	25679.556760023843	-25677.35675997616
-24	26790.767870047683	-26788.367869952315
-
--- !sql_test_Float_Decimal64V3_1 --
-\N	\N	\N	\N
-1	123.40123583882168	8.103646799035747E-5	0.10000000149011612
-2	469.0246969890125	8.528335810220662E-5	0.20000000298023224
-3	1036.8704092015266	8.679966787929143E-5	0.30000001192092896
-4	1826.9382952235014	8.757822045040539E-5	0.4000000059604645
-5	2839.22839	8.805209220946117E-5	0.5
-6	4073.740895875913	8.837087036504438E-5	0.6000000238418579
-7	5530.475205816567	8.859997831567021E-5	0.699999988079071
-8	7209.43219542891	8.877259702633364E-5	0.800000011920929
-9	9110.610856651227	8.890731585723625E-5	0.8999999761581421
-10	11234.01233	8.901539099521356E-5	1.0
-11	13579.636078330679	8.910401173289994E-5	1.100000023841858
-12	16147.482101643263	8.917799724914247E-5	1.2000000476837158
-13	1456.7345877070368	6.864668679119962E-6	0.10000000149011612
-14	3135.6914007254422	1.2756357715188077E-5	0.20000000298023224
-15	5036.870564147245	1.7868239019915086E-5	0.30000001192092896
-16	7160.271702696361	2.2345521428763662E-5	0.4000000059604645
-17	9505.89505	2.6299469822149996E-5	0.5
-18	12073.741205767352	2.9816775303935307E-5	0.6000000238418579
-19	14863.80837087085	3.296597823953184E-5	0.699999988079071
-20	17876.09901037463	3.580199565364088E-5	0.800000011920929
-21	21110.61052675979	3.8369328829116576E-5	0.8999999761581421
-22	24567.34565	4.070443808812533E-5	1.0
-23	28246.30304822212	4.283746621235263E-5	1.100000023841858
-24	32147.48272142614	4.47935574588914E-5	1.2000000476837158
-
--- !sql_test_Float_Decimal64V3_notn_1 --
-1	123.40123583882168	8.103646799035747E-5	0.10000000149011612
-2	469.0246969890125	8.528335810220662E-5	0.20000000298023224
-3	1036.8704092015266	8.679966787929143E-5	0.30000001192092896
-4	1826.9382952235014	8.757822045040539E-5	0.4000000059604645
-5	2839.22839	8.805209220946117E-5	0.5
-6	4073.740895875913	8.837087036504438E-5	0.6000000238418579
-7	5530.475205816567	8.859997831567021E-5	0.699999988079071
-8	7209.43219542891	8.877259702633364E-5	0.800000011920929
-9	9110.610856651227	8.890731585723625E-5	0.8999999761581421
-10	11234.01233	8.901539099521356E-5	1.0
-11	13579.636078330679	8.910401173289994E-5	1.100000023841858
-12	16147.482101643263	8.917799724914247E-5	1.2000000476837158
-13	1456.7345877070368	6.864668679119962E-6	0.10000000149011612
-14	3135.6914007254422	1.2756357715188077E-5	0.20000000298023224
-15	5036.870564147245	1.7868239019915086E-5	0.30000001192092896
-16	7160.271702696361	2.2345521428763662E-5	0.4000000059604645
-17	9505.89505	2.6299469822149996E-5	0.5
-18	12073.741205767352	2.9816775303935307E-5	0.6000000238418579
-19	14863.80837087085	3.296597823953184E-5	0.699999988079071
-20	17876.09901037463	3.580199565364088E-5	0.800000011920929
-21	21110.61052675979	3.8369328829116576E-5	0.8999999761581421
-22	24567.34565	4.070443808812533E-5	1.0
-23	28246.30304822212	4.283746621235263E-5	1.100000023841858
-24	32147.48272142614	4.47935574588914E-5	1.2000000476837158
+1	1234.1123400	-1233.9123400
+2	2345.3234500	-2344.9234500
+3	3456.5345600	-3455.9345600
+4	4567.7456700	-4566.9456700
+5	5678.9567800	-5677.9567800
+6	6790.1678900	-6788.9678900
+7	7901.3790000	-7899.9790000
+8	9012.5901100	-9010.9901100
+9	10123.8012200	-10122.0012200
+10	11235.0123300	-11233.0123300
+11	12346.2234400	-12344.0234400
+12	13457.4345500	-13455.0345500
+13	14567.4456600	-14567.2456600
+14	15678.6567700	-15678.2567700
+15	16789.8678800	-16789.2678800
+16	17901.0789900	-17900.2789900
+17	19012.2901000	-19011.2901000
+18	20123.5012100	-20122.3012100
+19	21234.7123200	-21233.3123200
+20	22345.9234300	-22344.3234300
+21	23457.1345400	-23455.3345400
+22	24568.3456500	-24566.3456500
+23	25679.5567600	-25677.3567600
+24	26790.7678700	-26788.3678700
 
 -- !sql_test_Float_Decimal64V3_2 --
 \N	\N
@@ -2596,164 +2066,58 @@
 23	0	25679	25679
 24	1	26789	26788
 
--- !sql_test_Float_Decimal64V3_4 --
-\N	\N	\N	\N
-1	0	127	127
-2	0	127	127
-3	0	127	127
-4	0	127	127
-5	0	127	127
-6	0	127	127
-7	0	127	127
-8	0	127	127
-9	0	127	127
-10	1	127	126
-11	1	127	126
-12	1	127	126
-13	0	127	127
-14	0	127	127
-15	0	127	127
-16	0	127	127
-17	0	127	127
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
--- !sql_test_Float_Decimal64V3_notn_4 --
-1	0	127	127
-2	0	127	127
-3	0	127	127
-4	0	127	127
-5	0	127	127
-6	0	127	127
-7	0	127	127
-8	0	127	127
-9	0	127	127
-10	1	127	126
-11	1	127	126
-12	1	127	126
-13	0	127	127
-14	0	127	127
-15	0	127	127
-16	0	127	127
-17	0	127	127
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
 -- !sql_test_Float_Decimal128V3_0 --
 \N	\N	\N
-1	1.2345678112345E7	-1.2345677912344998E7
-2	2.3456789323456004E7	-2.3456788923456E7
-3	3.456790053456701E7	-3.456789993456699E7
-4	4.567901174567801E7	-4.5679010945677996E7
-5	5.6790122956789E7	-5.6790121956789E7
-6	6.790123416790003E7	-6.790123296789998E7
-7	7.901234537901099E7	-7.901234397901101E7
-8	9.012345659012201E7	-9.012345499012199E7
-9	1.0123456780123298E8	-1.0123456600123303E8
-10	1.12345679012344E8	-1.12345677012344E8
-11	1.2345679022345503E8	-1.2345678802345498E8
-12	1.3456790143456605E8	-1.3456789903456596E8
-13	1.4567901144567698E8	-1.45679011245677E8
-14	1.56790122656788E8	-1.5679012225678802E8
-15	1.67901233867899E8	-1.6790123326789898E8
-16	1.7901234507901E8	-1.7901234427901E8
-17	1.90123456290121E8	-1.90123455290121E8
-18	2.0123456750123203E8	-2.0123456630123198E8
-19	2.1234567871234298E8	-2.12345677312343E8
-20	2.2345678992345402E8	-2.23456788323454E8
-21	2.3456790113456497E8	-2.34567899334565E8
-22	2.45679012345676E8	-2.45679010345676E8
-23	2.56790123556787E8	-2.5679012135678697E8
-24	2.6790123476789805E8	-2.6790123236789796E8
+1	12345678.11234500	-12345677.91234500
+2	23456789.32345600	-23456788.92345600
+3	34567900.53456702	-34567899.93456698
+4	45679011.74567800	-45679010.94567800
+5	56790122.95678900	-56790121.95678900
+6	67901234.16790004	-67901232.96789996
+7	79012345.37901100	-79012343.97901100
+8	90123456.59012200	-90123454.99012200
+9	101234567.80123300	-101234566.00123300
+10	112345679.01234400	-112345677.01234400
+11	123456790.22345500	-123456788.02345500
+12	134567901.43456608	-134567899.03456592
+13	145679011.44567700	-145679011.24567700
+14	156790122.65678800	-156790122.25678800
+15	167901233.86789902	-167901233.26789898
+16	179012345.07901000	-179012344.27901000
+17	190123456.29012100	-190123455.29012100
+18	201234567.50123204	-201234566.30123196
+19	212345678.71234300	-212345677.31234300
+20	223456789.92345400	-223456788.32345400
+21	234567901.13456500	-234567899.33456500
+22	245679012.34567600	-245679010.34567600
+23	256790123.55678700	-256790121.35678700
+24	267901234.76789808	-267901232.36789792
 
 -- !sql_test_Float_Decimal128V3_notn_0 --
-1	1.2345678112345E7	-1.2345677912344998E7
-2	2.3456789323456004E7	-2.3456788923456E7
-3	3.456790053456701E7	-3.456789993456699E7
-4	4.567901174567801E7	-4.5679010945677996E7
-5	5.6790122956789E7	-5.6790121956789E7
-6	6.790123416790003E7	-6.790123296789998E7
-7	7.901234537901099E7	-7.901234397901101E7
-8	9.012345659012201E7	-9.012345499012199E7
-9	1.0123456780123298E8	-1.0123456600123303E8
-10	1.12345679012344E8	-1.12345677012344E8
-11	1.2345679022345503E8	-1.2345678802345498E8
-12	1.3456790143456605E8	-1.3456789903456596E8
-13	1.4567901144567698E8	-1.45679011245677E8
-14	1.56790122656788E8	-1.5679012225678802E8
-15	1.67901233867899E8	-1.6790123326789898E8
-16	1.7901234507901E8	-1.7901234427901E8
-17	1.90123456290121E8	-1.90123455290121E8
-18	2.0123456750123203E8	-2.0123456630123198E8
-19	2.1234567871234298E8	-2.12345677312343E8
-20	2.2345678992345402E8	-2.23456788323454E8
-21	2.3456790113456497E8	-2.34567899334565E8
-22	2.45679012345676E8	-2.45679010345676E8
-23	2.56790123556787E8	-2.5679012135678697E8
-24	2.6790123476789805E8	-2.6790123236789796E8
-
--- !sql_test_Float_Decimal128V3_1 --
-\N	\N	\N	\N
-1	1234567.8196309938	8.100000776799915E-9	0.10000000149011612
-2	4691357.89459788	8.526316280016302E-9	0.20000000298023224
-3	1.0370370482451584E7	8.67857202448579E-9	0.30000001192092896
-4	1.8271604810539324E7	8.756757078944773E-9	0.4000000059604645
-5	2.83950612283945E7	8.804347981120214E-9	0.5
-6	4.074074175963157E7	8.836364117624885E-9	0.6000000238418579
-7	5.5308640333407156E7	8.859374961252358E-9	0.699999988079071
-8	7.209876570645292E7	8.876712559535618E-9	0.800000011920929
-9	9.111110779748954E7	8.890243754746388E-9	0.8999999761581421
-10	1.12345678012344E8	8.901098980328596E-9	1.0
-11	1.3580247097923973E8	8.91000026529018E-9	1.100000023841858
-12	1.614814866981767E8	8.917431613274707E-9	1.2000000476837158
-13	1.4567901351646341E7	6.864406929068826E-10	0.10000000149011612
-14	3.135802495862858E7	1.2755905783245565E-9	0.20000000298023224
-15	5.037037207190838E7	1.786764787523788E-9	0.30000001192092896
-16	7.160493893860072E7	2.23448280439939E-9	0.4000000059604645
-17	9.50617278950605E7	2.6298701437025977E-9	0.5
-18	1.2074074493854515E8	2.981595225319038E-9	0.6000000238418579
-19	1.4864197207728234E8	3.2965115872920295E-9	0.699999988079071
-20	1.787654339625757E8	3.5801105666069066E-9	0.800000011920929
-21	2.1111110461857393E8	3.836842019978664E-9	0.8999999761581421
-22	2.45679011345676E8	4.0703517753617834E-9	1.0
-23	2.824691408248193E8	4.283653955681132E-9	1.100000023841858
-24	3.2148149305600387E8	4.47926286752085E-9	1.2000000476837158
-
--- !sql_test_Float_Decimal128V3_notn_1 --
-1	1234567.8196309938	8.100000776799915E-9	0.10000000149011612
-2	4691357.89459788	8.526316280016302E-9	0.20000000298023224
-3	1.0370370482451584E7	8.67857202448579E-9	0.30000001192092896
-4	1.8271604810539324E7	8.756757078944773E-9	0.4000000059604645
-5	2.83950612283945E7	8.804347981120214E-9	0.5
-6	4.074074175963157E7	8.836364117624885E-9	0.6000000238418579
-7	5.5308640333407156E7	8.859374961252358E-9	0.699999988079071
-8	7.209876570645292E7	8.876712559535618E-9	0.800000011920929
-9	9.111110779748954E7	8.890243754746388E-9	0.8999999761581421
-10	1.12345678012344E8	8.901098980328596E-9	1.0
-11	1.3580247097923973E8	8.91000026529018E-9	1.100000023841858
-12	1.614814866981767E8	8.917431613274707E-9	1.2000000476837158
-13	1.4567901351646341E7	6.864406929068826E-10	0.10000000149011612
-14	3.135802495862858E7	1.2755905783245565E-9	0.20000000298023224
-15	5.037037207190838E7	1.786764787523788E-9	0.30000001192092896
-16	7.160493893860072E7	2.23448280439939E-9	0.4000000059604645
-17	9.50617278950605E7	2.6298701437025977E-9	0.5
-18	1.2074074493854515E8	2.981595225319038E-9	0.6000000238418579
-19	1.4864197207728234E8	3.2965115872920295E-9	0.699999988079071
-20	1.787654339625757E8	3.5801105666069066E-9	0.800000011920929
-21	2.1111110461857393E8	3.836842019978664E-9	0.8999999761581421
-22	2.45679011345676E8	4.0703517753617834E-9	1.0
-23	2.824691408248193E8	4.283653955681132E-9	1.100000023841858
-24	3.2148149305600387E8	4.47926286752085E-9	1.2000000476837158
+1	12345678.11234500	-12345677.91234500
+2	23456789.32345600	-23456788.92345600
+3	34567900.53456702	-34567899.93456698
+4	45679011.74567800	-45679010.94567800
+5	56790122.95678900	-56790121.95678900
+6	67901234.16790004	-67901232.96789996
+7	79012345.37901100	-79012343.97901100
+8	90123456.59012200	-90123454.99012200
+9	101234567.80123300	-101234566.00123300
+10	112345679.01234400	-112345677.01234400
+11	123456790.22345500	-123456788.02345500
+12	134567901.43456608	-134567899.03456592
+13	145679011.44567700	-145679011.24567700
+14	156790122.65678800	-156790122.25678800
+15	167901233.86789902	-167901233.26789898
+16	179012345.07901000	-179012344.27901000
+17	190123456.29012100	-190123455.29012100
+18	201234567.50123204	-201234566.30123196
+19	212345678.71234300	-212345677.31234300
+20	223456789.92345400	-223456788.32345400
+21	234567901.13456500	-234567899.33456500
+22	245679012.34567600	-245679010.34567600
+23	256790123.55678700	-256790121.35678700
+24	267901234.76789808	-267901232.36789792
 
 -- !sql_test_Float_Decimal128V3_2 --
 \N	\N
@@ -2861,59 +2225,6 @@
 23	0	256790123	256790123
 24	1	267901233	267901232
 
--- !sql_test_Float_Decimal128V3_4 --
-\N	\N	\N	\N
-1	0	127	127
-2	0	127	127
-3	0	127	127
-4	0	127	127
-5	0	127	127
-6	0	127	127
-7	0	127	127
-8	0	127	127
-9	0	127	127
-10	1	127	126
-11	1	127	126
-12	1	127	126
-13	0	127	127
-14	0	127	127
-15	0	127	127
-16	0	127	127
-17	0	127	127
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
--- !sql_test_Float_Decimal128V3_notn_4 --
-1	0	127	127
-2	0	127	127
-3	0	127	127
-4	0	127	127
-5	0	127	127
-6	0	127	127
-7	0	127	127
-8	0	127	127
-9	0	127	127
-10	1	127	126
-11	1	127	126
-12	1	127	126
-13	0	127	127
-14	0	127	127
-15	0	127	127
-16	0	127	127
-17	0	127	127
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
 -- !sql_test_Float_Char_0 --
 \N	\N	\N
 1	\N	\N
@@ -3126,59 +2437,6 @@
 23	0	4933	4933
 24	1	6975	6974
 
--- !sql_test_Float_Char_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_Float_Char_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_Float_Varchar_0 --
 \N	\N	\N
 1	\N	\N
@@ -3391,59 +2649,6 @@
 23	1	74137	74136
 24	1	104845	104844
 
--- !sql_test_Float_Varchar_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_Float_Varchar_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_Float_String_0 --
 \N	\N	\N
 1	\N	\N
@@ -3656,59 +2861,6 @@
 23	1	338987	338986
 24	1	479399	479398
 
--- !sql_test_Float_String_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_Float_String_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_Float_Date_0 --
 \N	\N	\N
 1	2.01203011E7	-2.01203009E7
@@ -3762,59 +2914,6 @@
 23	2.0120312100000024E7	-2.0120309899999976E7
 24	2.0120313200000048E7	-2.0120310799999952E7
 
--- !sql_test_Float_Date_1 --
-\N	\N	\N	\N
-1	2012030.1299815848	4.970104646551566E-9	0.10000000149011612
-2	4024060.4599631727	9.94020879906436E-9	0.20000000298023224
-3	6036091.139852703	1.4910312827840067E-8	0.30000001192092896
-4	8048121.719926357	1.9880415621973927E-8	0.4000000059604645
-5	1.00601525E7	2.4850517922069274E-8	0.5
-6	1.2072184079705477E7	2.9820621209332395E-8	0.6000000238418579
-7	1.408421466014725E7	3.479072104014472E-8	0.699999988079071
-8	1.6096246639852762E7	3.9760823339331035E-8	0.800000011920929
-9	1.810827762029445E7	4.473092218206699E-8	0.8999999761581421
-10	2.012031E7	4.9701023493176794E-8	1.0
-11	2.2132342579705596E7	5.467112431024838E-8	1.100000023841858
-12	2.414437535941124E7	5.964122463328182E-8	1.2000000476837158
-13	2012030.1299815848	4.970104646551566E-9	0.10000000149011612
-14	4024060.4599631727	9.94020879906436E-9	0.20000000298023224
-15	6036091.139852703	1.4910312827840067E-8	0.30000001192092896
-16	8048121.719926357	1.9880415621973927E-8	0.4000000059604645
-17	1.00601525E7	2.4850517922069274E-8	0.5
-18	1.2072184079705477E7	2.9820621209332395E-8	0.6000000238418579
-19	1.408421466014725E7	3.479072104014472E-8	0.699999988079071
-20	1.6096246639852762E7	3.9760823339331035E-8	0.800000011920929
-21	1.810827762029445E7	4.473092218206699E-8	0.8999999761581421
-22	2.012031E7	4.9701023493176794E-8	1.0
-23	2.2132342579705596E7	5.467112431024838E-8	1.100000023841858
-24	2.414437535941124E7	5.964122463328182E-8	1.2000000476837158
-
--- !sql_test_Float_Date_notn_1 --
-1	2012030.1299815848	4.970104646551566E-9	0.10000000149011612
-2	4024060.4599631727	9.94020879906436E-9	0.20000000298023224
-3	6036091.139852703	1.4910312827840067E-8	0.30000001192092896
-4	8048121.719926357	1.9880415621973927E-8	0.4000000059604645
-5	1.00601525E7	2.4850517922069274E-8	0.5
-6	1.2072184079705477E7	2.9820621209332395E-8	0.6000000238418579
-7	1.408421466014725E7	3.479072104014472E-8	0.699999988079071
-8	1.6096246639852762E7	3.9760823339331035E-8	0.800000011920929
-9	1.810827762029445E7	4.473092218206699E-8	0.8999999761581421
-10	2.012031E7	4.9701023493176794E-8	1.0
-11	2.2132342579705596E7	5.467112431024838E-8	1.100000023841858
-12	2.414437535941124E7	5.964122463328182E-8	1.2000000476837158
-13	2012030.1299815848	4.970104646551566E-9	0.10000000149011612
-14	4024060.4599631727	9.94020879906436E-9	0.20000000298023224
-15	6036091.139852703	1.4910312827840067E-8	0.30000001192092896
-16	8048121.719926357	1.9880415621973927E-8	0.4000000059604645
-17	1.00601525E7	2.4850517922069274E-8	0.5
-18	1.2072184079705477E7	2.9820621209332395E-8	0.6000000238418579
-19	1.408421466014725E7	3.479072104014472E-8	0.699999988079071
-20	1.6096246639852762E7	3.9760823339331035E-8	0.800000011920929
-21	1.810827762029445E7	4.473092218206699E-8	0.8999999761581421
-22	2.012031E7	4.9701023493176794E-8	1.0
-23	2.2132342579705596E7	5.467112431024838E-8	1.100000023841858
-24	2.414437535941124E7	5.964122463328182E-8	1.2000000476837158
-
 -- !sql_test_Float_Date_2 --
 \N	\N
 1	0
@@ -3921,59 +3020,6 @@
 23	1	20120311	20120310
 24	0	20120313	20120313
 
--- !sql_test_Float_Date_4 --
-\N	\N	\N	\N
-1	0	-19	-19
-2	0	-18	-18
-3	0	-17	-17
-4	0	-16	-16
-5	0	-15	-15
-6	0	-14	-14
-7	0	-13	-13
-8	0	-12	-12
-9	0	-11	-11
-10	0	-9	-9
-11	1	-9	-10
-12	0	-7	-7
-13	0	-19	-19
-14	0	-18	-18
-15	0	-17	-17
-16	0	-16	-16
-17	0	-15	-15
-18	0	-14	-14
-19	0	-13	-13
-20	0	-12	-12
-21	0	-11	-11
-22	0	-9	-9
-23	1	-9	-10
-24	0	-7	-7
-
--- !sql_test_Float_Date_notn_4 --
-1	0	-19	-19
-2	0	-18	-18
-3	0	-17	-17
-4	0	-16	-16
-5	0	-15	-15
-6	0	-14	-14
-7	0	-13	-13
-8	0	-12	-12
-9	0	-11	-11
-10	0	-9	-9
-11	1	-9	-10
-12	0	-7	-7
-13	0	-19	-19
-14	0	-18	-18
-15	0	-17	-17
-16	0	-16	-16
-17	0	-15	-15
-18	0	-14	-14
-19	0	-13	-13
-20	0	-12	-12
-21	0	-11	-11
-22	0	-9	-9
-23	1	-9	-10
-24	0	-7	-7
-
 -- !sql_test_Float_DateTime_0 --
 \N	\N	\N
 1	2.01203010100011E13	-2.01203010100009E13
@@ -4027,59 +3073,6 @@
 23	2.01203111110121E13	-2.01203111110099E13
 24	2.01203121211132E13	-2.01203121211108E13
 
--- !sql_test_Float_DateTime_1 --
-\N	\N	\N	\N
-1	2.0120301309816848E12	4.970104644081125E-15	0.10000000149011612
-2	4.0240604639835728E12	9.940208789133193E-15	0.20000000298023224
-3	6.036091148913603E12	1.491031280545789E-14	0.30000001192092896
-4	8.048121736047957E12	1.9880415582150457E-14	0.4000000059604645
-5	1.00601525252025E13	2.4850517859814234E-14	0.5
-6	1.2072184116009078E13	2.982062111965551E-14	0.6000000238418579
-7	1.4084214709572148E13	3.47907209180557E-14	0.699999988079071
-8	1.6096246704419164E13	3.9760823179839614E-14	0.800000011920929
-9	1.810827770202255E13	4.47309219801829E-14	0.8999999761581421
-10	2.012031010091E13	4.970102324390975E-14	1.0
-11	2.21323427018177E13	5.467112400860811E-14	1.100000023841858
-12	2.4144375504745645E13	5.964122427427804E-14	1.2000000476837158
-13	2.0120301309816848E12	4.970104644081125E-15	0.10000000149011612
-14	4.0240604639835728E12	9.940208789133193E-15	0.20000000298023224
-15	6.036091148913603E12	1.491031280545789E-14	0.30000001192092896
-16	8.048121736047957E12	1.9880415582150457E-14	0.4000000059604645
-17	1.00601525252025E13	2.4850517859814234E-14	0.5
-18	1.2072184116009078E13	2.982062111965551E-14	0.6000000238418579
-19	1.4084214709572148E13	3.47907209180557E-14	0.699999988079071
-20	1.6096246704419164E13	3.9760823179839614E-14	0.800000011920929
-21	1.810827770202255E13	4.47309219801829E-14	0.8999999761581421
-22	2.012031010091E13	4.970102324390975E-14	1.0
-23	2.21323427018177E13	5.467112400860811E-14	1.100000023841858
-24	2.4144375504745645E13	5.964122427427804E-14	1.2000000476837158
-
--- !sql_test_Float_DateTime_notn_1 --
-1	2.0120301309816848E12	4.970104644081125E-15	0.10000000149011612
-2	4.0240604639835728E12	9.940208789133193E-15	0.20000000298023224
-3	6.036091148913603E12	1.491031280545789E-14	0.30000001192092896
-4	8.048121736047957E12	1.9880415582150457E-14	0.4000000059604645
-5	1.00601525252025E13	2.4850517859814234E-14	0.5
-6	1.2072184116009078E13	2.982062111965551E-14	0.6000000238418579
-7	1.4084214709572148E13	3.47907209180557E-14	0.699999988079071
-8	1.6096246704419164E13	3.9760823179839614E-14	0.800000011920929
-9	1.810827770202255E13	4.47309219801829E-14	0.8999999761581421
-10	2.012031010091E13	4.970102324390975E-14	1.0
-11	2.21323427018177E13	5.467112400860811E-14	1.100000023841858
-12	2.4144375504745645E13	5.964122427427804E-14	1.2000000476837158
-13	2.0120301309816848E12	4.970104644081125E-15	0.10000000149011612
-14	4.0240604639835728E12	9.940208789133193E-15	0.20000000298023224
-15	6.036091148913603E12	1.491031280545789E-14	0.30000001192092896
-16	8.048121736047957E12	1.9880415582150457E-14	0.4000000059604645
-17	1.00601525252025E13	2.4850517859814234E-14	0.5
-18	1.2072184116009078E13	2.982062111965551E-14	0.6000000238418579
-19	1.4084214709572148E13	3.47907209180557E-14	0.699999988079071
-20	1.6096246704419164E13	3.9760823179839614E-14	0.800000011920929
-21	1.810827770202255E13	4.47309219801829E-14	0.8999999761581421
-22	2.012031010091E13	4.970102324390975E-14	1.0
-23	2.21323427018177E13	5.467112400860811E-14	1.100000023841858
-24	2.4144375504745645E13	5.964122427427804E-14	1.2000000476837158
-
 -- !sql_test_Float_DateTime_2 --
 \N	\N
 1	0
@@ -4186,59 +3179,6 @@
 23	1	20120311111011	20120311111010
 24	0	20120312121113	20120312121113
 
--- !sql_test_Float_DateTime_4 --
-\N	\N	\N	\N
-1	0	81	81
-2	0	6	6
-3	0	-69	-69
-4	0	112	112
-5	0	37	37
-6	0	-38	-38
-7	0	-113	-113
-8	0	68	68
-9	0	-7	-7
-10	0	-81	-81
-11	1	99	98
-12	0	25	25
-13	0	81	81
-14	0	6	6
-15	0	-69	-69
-16	0	112	112
-17	0	37	37
-18	0	-38	-38
-19	0	-113	-113
-20	0	68	68
-21	0	-7	-7
-22	0	-81	-81
-23	1	99	98
-24	0	25	25
-
--- !sql_test_Float_DateTime_notn_4 --
-1	0	81	81
-2	0	6	6
-3	0	-69	-69
-4	0	112	112
-5	0	37	37
-6	0	-38	-38
-7	0	-113	-113
-8	0	68	68
-9	0	-7	-7
-10	0	-81	-81
-11	1	99	98
-12	0	25	25
-13	0	81	81
-14	0	6	6
-15	0	-69	-69
-16	0	112	112
-17	0	37	37
-18	0	-38	-38
-19	0	-113	-113
-20	0	68	68
-21	0	-7	-7
-22	0	-81	-81
-23	1	99	98
-24	0	25	25
-
 -- !sql_test_Float_DateV2_0 --
 \N	\N	\N
 1	2.01203011E7	-2.01203009E7
@@ -4292,59 +3232,6 @@
 23	2.0120312100000024E7	-2.0120309899999976E7
 24	2.0120313200000048E7	-2.0120310799999952E7
 
--- !sql_test_Float_DateV2_1 --
-\N	\N	\N	\N
-1	2012030.1299815848	4.970104646551566E-9	0.10000000149011612
-2	4024060.4599631727	9.94020879906436E-9	0.20000000298023224
-3	6036091.139852703	1.4910312827840067E-8	0.30000001192092896
-4	8048121.719926357	1.9880415621973927E-8	0.4000000059604645
-5	1.00601525E7	2.4850517922069274E-8	0.5
-6	1.2072184079705477E7	2.9820621209332395E-8	0.6000000238418579
-7	1.408421466014725E7	3.479072104014472E-8	0.699999988079071
-8	1.6096246639852762E7	3.9760823339331035E-8	0.800000011920929
-9	1.810827762029445E7	4.473092218206699E-8	0.8999999761581421
-10	2.012031E7	4.9701023493176794E-8	1.0
-11	2.2132342579705596E7	5.467112431024838E-8	1.100000023841858
-12	2.414437535941124E7	5.964122463328182E-8	1.2000000476837158
-13	2012030.1299815848	4.970104646551566E-9	0.10000000149011612
-14	4024060.4599631727	9.94020879906436E-9	0.20000000298023224
-15	6036091.139852703	1.4910312827840067E-8	0.30000001192092896
-16	8048121.719926357	1.9880415621973927E-8	0.4000000059604645
-17	1.00601525E7	2.4850517922069274E-8	0.5
-18	1.2072184079705477E7	2.9820621209332395E-8	0.6000000238418579
-19	1.408421466014725E7	3.479072104014472E-8	0.699999988079071
-20	1.6096246639852762E7	3.9760823339331035E-8	0.800000011920929
-21	1.810827762029445E7	4.473092218206699E-8	0.8999999761581421
-22	2.012031E7	4.9701023493176794E-8	1.0
-23	2.2132342579705596E7	5.467112431024838E-8	1.100000023841858
-24	2.414437535941124E7	5.964122463328182E-8	1.2000000476837158
-
--- !sql_test_Float_DateV2_notn_1 --
-1	2012030.1299815848	4.970104646551566E-9	0.10000000149011612
-2	4024060.4599631727	9.94020879906436E-9	0.20000000298023224
-3	6036091.139852703	1.4910312827840067E-8	0.30000001192092896
-4	8048121.719926357	1.9880415621973927E-8	0.4000000059604645
-5	1.00601525E7	2.4850517922069274E-8	0.5
-6	1.2072184079705477E7	2.9820621209332395E-8	0.6000000238418579
-7	1.408421466014725E7	3.479072104014472E-8	0.699999988079071
-8	1.6096246639852762E7	3.9760823339331035E-8	0.800000011920929
-9	1.810827762029445E7	4.473092218206699E-8	0.8999999761581421
-10	2.012031E7	4.9701023493176794E-8	1.0
-11	2.2132342579705596E7	5.467112431024838E-8	1.100000023841858
-12	2.414437535941124E7	5.964122463328182E-8	1.2000000476837158
-13	2012030.1299815848	4.970104646551566E-9	0.10000000149011612
-14	4024060.4599631727	9.94020879906436E-9	0.20000000298023224
-15	6036091.139852703	1.4910312827840067E-8	0.30000001192092896
-16	8048121.719926357	1.9880415621973927E-8	0.4000000059604645
-17	1.00601525E7	2.4850517922069274E-8	0.5
-18	1.2072184079705477E7	2.9820621209332395E-8	0.6000000238418579
-19	1.408421466014725E7	3.479072104014472E-8	0.699999988079071
-20	1.6096246639852762E7	3.9760823339331035E-8	0.800000011920929
-21	1.810827762029445E7	4.473092218206699E-8	0.8999999761581421
-22	2.012031E7	4.9701023493176794E-8	1.0
-23	2.2132342579705596E7	5.467112431024838E-8	1.100000023841858
-24	2.414437535941124E7	5.964122463328182E-8	1.2000000476837158
-
 -- !sql_test_Float_DateV2_2 --
 \N	\N
 1	0
@@ -4451,59 +3338,6 @@
 23	1	20120311	20120310
 24	0	20120313	20120313
 
--- !sql_test_Float_DateV2_4 --
-\N	\N	\N	\N
-1	0	-19	-19
-2	0	-18	-18
-3	0	-17	-17
-4	0	-16	-16
-5	0	-15	-15
-6	0	-14	-14
-7	0	-13	-13
-8	0	-12	-12
-9	0	-11	-11
-10	0	-9	-9
-11	1	-9	-10
-12	0	-7	-7
-13	0	-19	-19
-14	0	-18	-18
-15	0	-17	-17
-16	0	-16	-16
-17	0	-15	-15
-18	0	-14	-14
-19	0	-13	-13
-20	0	-12	-12
-21	0	-11	-11
-22	0	-9	-9
-23	1	-9	-10
-24	0	-7	-7
-
--- !sql_test_Float_DateV2_notn_4 --
-1	0	-19	-19
-2	0	-18	-18
-3	0	-17	-17
-4	0	-16	-16
-5	0	-15	-15
-6	0	-14	-14
-7	0	-13	-13
-8	0	-12	-12
-9	0	-11	-11
-10	0	-9	-9
-11	1	-9	-10
-12	0	-7	-7
-13	0	-19	-19
-14	0	-18	-18
-15	0	-17	-17
-16	0	-16	-16
-17	0	-15	-15
-18	0	-14	-14
-19	0	-13	-13
-20	0	-12	-12
-21	0	-11	-11
-22	0	-9	-9
-23	1	-9	-10
-24	0	-7	-7
-
 -- !sql_test_Float_DateTimeV2_0 --
 \N	\N	\N
 1	2.01203010100011E13	-2.01203010100009E13
@@ -4557,59 +3391,6 @@
 23	2.01203111110121E13	-2.01203111110099E13
 24	2.01203121211132E13	-2.01203121211108E13
 
--- !sql_test_Float_DateTimeV2_1 --
-\N	\N	\N	\N
-1	2.0120301309816848E12	4.970104644081125E-15	0.10000000149011612
-2	4.0240604639835728E12	9.940208789133193E-15	0.20000000298023224
-3	6.036091148913603E12	1.491031280545789E-14	0.30000001192092896
-4	8.048121736047957E12	1.9880415582150457E-14	0.4000000059604645
-5	1.00601525252025E13	2.4850517859814234E-14	0.5
-6	1.2072184116009078E13	2.982062111965551E-14	0.6000000238418579
-7	1.4084214709572148E13	3.47907209180557E-14	0.699999988079071
-8	1.6096246704419164E13	3.9760823179839614E-14	0.800000011920929
-9	1.810827770202255E13	4.47309219801829E-14	0.8999999761581421
-10	2.012031010091E13	4.970102324390975E-14	1.0
-11	2.21323427018177E13	5.467112400860811E-14	1.100000023841858
-12	2.4144375504745645E13	5.964122427427804E-14	1.2000000476837158
-13	2.0120301309816848E12	4.970104644081125E-15	0.10000000149011612
-14	4.0240604639835728E12	9.940208789133193E-15	0.20000000298023224
-15	6.036091148913603E12	1.491031280545789E-14	0.30000001192092896
-16	8.048121736047957E12	1.9880415582150457E-14	0.4000000059604645
-17	1.00601525252025E13	2.4850517859814234E-14	0.5
-18	1.2072184116009078E13	2.982062111965551E-14	0.6000000238418579
-19	1.4084214709572148E13	3.47907209180557E-14	0.699999988079071
-20	1.6096246704419164E13	3.9760823179839614E-14	0.800000011920929
-21	1.810827770202255E13	4.47309219801829E-14	0.8999999761581421
-22	2.012031010091E13	4.970102324390975E-14	1.0
-23	2.21323427018177E13	5.467112400860811E-14	1.100000023841858
-24	2.4144375504745645E13	5.964122427427804E-14	1.2000000476837158
-
--- !sql_test_Float_DateTimeV2_notn_1 --
-1	2.0120301309816848E12	4.970104644081125E-15	0.10000000149011612
-2	4.0240604639835728E12	9.940208789133193E-15	0.20000000298023224
-3	6.036091148913603E12	1.491031280545789E-14	0.30000001192092896
-4	8.048121736047957E12	1.9880415582150457E-14	0.4000000059604645
-5	1.00601525252025E13	2.4850517859814234E-14	0.5
-6	1.2072184116009078E13	2.982062111965551E-14	0.6000000238418579
-7	1.4084214709572148E13	3.47907209180557E-14	0.699999988079071
-8	1.6096246704419164E13	3.9760823179839614E-14	0.800000011920929
-9	1.810827770202255E13	4.47309219801829E-14	0.8999999761581421
-10	2.012031010091E13	4.970102324390975E-14	1.0
-11	2.21323427018177E13	5.467112400860811E-14	1.100000023841858
-12	2.4144375504745645E13	5.964122427427804E-14	1.2000000476837158
-13	2.0120301309816848E12	4.970104644081125E-15	0.10000000149011612
-14	4.0240604639835728E12	9.940208789133193E-15	0.20000000298023224
-15	6.036091148913603E12	1.491031280545789E-14	0.30000001192092896
-16	8.048121736047957E12	1.9880415582150457E-14	0.4000000059604645
-17	1.00601525252025E13	2.4850517859814234E-14	0.5
-18	1.2072184116009078E13	2.982062111965551E-14	0.6000000238418579
-19	1.4084214709572148E13	3.47907209180557E-14	0.699999988079071
-20	1.6096246704419164E13	3.9760823179839614E-14	0.800000011920929
-21	1.810827770202255E13	4.47309219801829E-14	0.8999999761581421
-22	2.012031010091E13	4.970102324390975E-14	1.0
-23	2.21323427018177E13	5.467112400860811E-14	1.100000023841858
-24	2.4144375504745645E13	5.964122427427804E-14	1.2000000476837158
-
 -- !sql_test_Float_DateTimeV2_2 --
 \N	\N
 1	0
@@ -4716,59 +3497,6 @@
 23	1	20120311111011	20120311111010
 24	0	20120312121113	20120312121113
 
--- !sql_test_Float_DateTimeV2_4 --
-\N	\N	\N	\N
-1	0	81	81
-2	0	6	6
-3	0	-69	-69
-4	0	112	112
-5	0	37	37
-6	0	-38	-38
-7	0	-113	-113
-8	0	68	68
-9	0	-7	-7
-10	0	-81	-81
-11	1	99	98
-12	0	25	25
-13	0	81	81
-14	0	6	6
-15	0	-69	-69
-16	0	112	112
-17	0	37	37
-18	0	-38	-38
-19	0	-113	-113
-20	0	68	68
-21	0	-7	-7
-22	0	-81	-81
-23	1	99	98
-24	0	25	25
-
--- !sql_test_Float_DateTimeV2_notn_4 --
-1	0	81	81
-2	0	6	6
-3	0	-69	-69
-4	0	112	112
-5	0	37	37
-6	0	-38	-38
-7	0	-113	-113
-8	0	68	68
-9	0	-7	-7
-10	0	-81	-81
-11	1	99	98
-12	0	25	25
-13	0	81	81
-14	0	6	6
-15	0	-69	-69
-16	0	112	112
-17	0	37	37
-18	0	-38	-38
-19	0	-113	-113
-20	0	68	68
-21	0	-7	-7
-22	0	-81	-81
-23	1	99	98
-24	0	25	25
-
 -- !sql_test_Float_Boolean_0 --
 \N	\N	\N
 1	0.10000000149011612	0.10000000149011612
@@ -4822,59 +3550,6 @@
 23	2.100000023841858	0.10000002384185791
 24	2.200000047683716	0.20000004768371582
 
--- !sql_test_Float_Boolean_1 --
-\N	\N	\N	\N
-1	0.0	\N	\N
-2	0.0	\N	\N
-3	0.0	\N	\N
-4	0.0	\N	\N
-5	0.0	\N	\N
-6	0.0	\N	\N
-7	0.0	\N	\N
-8	0.800000011920929	0.800000011920929	0.800000011920929
-9	0.8999999761581421	0.8999999761581421	0.8999999761581421
-10	1.0	1.0	0.0
-11	1.100000023841858	1.100000023841858	0.10000002384185791
-12	1.2000000476837158	1.2000000476837158	0.20000004768371582
-13	0.0	\N	\N
-14	0.0	\N	\N
-15	0.0	\N	\N
-16	0.0	\N	\N
-17	0.0	\N	\N
-18	0.0	\N	\N
-19	0.0	\N	\N
-20	0.800000011920929	0.800000011920929	0.800000011920929
-21	0.8999999761581421	0.8999999761581421	0.8999999761581421
-22	1.0	1.0	0.0
-23	1.100000023841858	1.100000023841858	0.10000002384185791
-24	1.2000000476837158	1.2000000476837158	0.20000004768371582
-
--- !sql_test_Float_Boolean_notn_1 --
-1	0.0	\N	\N
-2	0.0	\N	\N
-3	0.0	\N	\N
-4	0.0	\N	\N
-5	0.0	\N	\N
-6	0.0	\N	\N
-7	0.0	\N	\N
-8	0.800000011920929	0.800000011920929	0.800000011920929
-9	0.8999999761581421	0.8999999761581421	0.8999999761581421
-10	1.0	1.0	0.0
-11	1.100000023841858	1.100000023841858	0.10000002384185791
-12	1.2000000476837158	1.2000000476837158	0.20000004768371582
-13	0.0	\N	\N
-14	0.0	\N	\N
-15	0.0	\N	\N
-16	0.0	\N	\N
-17	0.0	\N	\N
-18	0.0	\N	\N
-19	0.0	\N	\N
-20	0.800000011920929	0.800000011920929	0.800000011920929
-21	0.8999999761581421	0.8999999761581421	0.8999999761581421
-22	1.0	1.0	0.0
-23	1.100000023841858	1.100000023841858	0.10000002384185791
-24	1.2000000476837158	1.2000000476837158	0.20000004768371582
-
 -- !sql_test_Float_Boolean_2 --
 \N	\N
 1	\N
@@ -4981,59 +3656,6 @@
 23	1	1	0
 24	1	1	0
 
--- !sql_test_Float_Boolean_4 --
-\N	\N	\N	\N
-1	0	0	0
-2	0	0	0
-3	0	0	0
-4	0	0	0
-5	0	0	0
-6	0	0	0
-7	0	0	0
-8	0	1	1
-9	0	1	1
-10	1	1	0
-11	1	1	0
-12	1	1	0
-13	0	0	0
-14	0	0	0
-15	0	0	0
-16	0	0	0
-17	0	0	0
-18	0	0	0
-19	0	0	0
-20	0	1	1
-21	0	1	1
-22	1	1	0
-23	1	1	0
-24	1	1	0
-
--- !sql_test_Float_Boolean_notn_4 --
-1	0	0	0
-2	0	0	0
-3	0	0	0
-4	0	0	0
-5	0	0	0
-6	0	0	0
-7	0	0	0
-8	0	1	1
-9	0	1	1
-10	1	1	0
-11	1	1	0
-12	1	1	0
-13	0	0	0
-14	0	0	0
-15	0	0	0
-16	0	0	0
-17	0	0	0
-18	0	0	0
-19	0	0	0
-20	0	1	1
-21	0	1	1
-22	1	1	0
-23	1	1	0
-24	1	1	0
-
 -- !sql_test_Double_TinyInt_0 --
 \N	\N	\N
 1	1.5244	-0.4756
@@ -5490,79 +4112,26 @@
 2	0	20	20
 3	0	41	41
 4	0	81	81
-5	0	162	162
-6	0	322	322
-7	0	644	644
-8	0	1285	1285
-9	0	2568	2568
-10	0	5131	5131
-11	0	10256	10256
-12	0	20502	20502
-13	0	10	10
-14	0	20	20
-15	0	41	41
-16	0	81	81
-17	0	162	162
-18	0	322	322
-19	0	644	644
-20	0	1285	1285
-21	0	2568	2568
-22	0	5131	5131
-23	0	10256	10256
-24	0	20502	20502
-
--- !sql_test_Double_SmallInt_4 --
-\N	\N	\N	\N
-1	0	10	10
-2	0	20	20
-3	0	41	41
-4	0	81	81
-5	0	-94	-94
-6	0	66	66
-7	0	-124	-124
-8	0	5	5
-9	0	8	8
-10	0	11	11
-11	0	16	16
-12	0	22	22
-13	0	10	10
-14	0	20	20
-15	0	41	41
-16	0	81	81
-17	0	-94	-94
-18	0	66	66
-19	0	-124	-124
-20	0	5	5
-21	0	8	8
-22	0	11	11
-23	0	16	16
-24	0	22	22
-
--- !sql_test_Double_SmallInt_notn_4 --
-1	0	10	10
-2	0	20	20
-3	0	41	41
-4	0	81	81
-5	0	-94	-94
-6	0	66	66
-7	0	-124	-124
-8	0	5	5
-9	0	8	8
-10	0	11	11
-11	0	16	16
-12	0	22	22
+5	0	162	162
+6	0	322	322
+7	0	644	644
+8	0	1285	1285
+9	0	2568	2568
+10	0	5131	5131
+11	0	10256	10256
+12	0	20502	20502
 13	0	10	10
 14	0	20	20
 15	0	41	41
 16	0	81	81
-17	0	-94	-94
-18	0	66	66
-19	0	-124	-124
-20	0	5	5
-21	0	8	8
-22	0	11	11
-23	0	16	16
-24	0	22	22
+17	0	162	162
+18	0	322	322
+19	0	644	644
+20	0	1285	1285
+21	0	2568	2568
+22	0	5131	5131
+23	0	10256	10256
+24	0	20502	20502
 
 -- !sql_test_Double_Integer_0 --
 \N	\N	\N
@@ -5776,59 +4345,6 @@
 23	0	24320061	24320061
 24	4	48640063	48640059
 
--- !sql_test_Double_Integer_4 --
-\N	\N	\N	\N
-1	0	-13	-13
-2	0	-71	-71
-3	1	69	68
-4	1	93	92
-5	0	-113	-113
-6	0	-17	-17
-7	4	-83	-87
-8	5	45	40
-9	8	45	37
-10	9	47	38
-11	0	61	61
-12	4	63	59
-13	0	-13	-13
-14	0	-71	-71
-15	1	69	68
-16	1	93	92
-17	0	-113	-113
-18	0	-17	-17
-19	4	-83	-87
-20	5	45	40
-21	8	45	37
-22	9	47	38
-23	0	61	61
-24	4	63	59
-
--- !sql_test_Double_Integer_notn_4 --
-1	0	-13	-13
-2	0	-71	-71
-3	1	69	68
-4	1	93	92
-5	0	-113	-113
-6	0	-17	-17
-7	4	-83	-87
-8	5	45	40
-9	8	45	37
-10	9	47	38
-11	0	61	61
-12	4	63	59
-13	0	-13	-13
-14	0	-71	-71
-15	1	69	68
-16	1	93	92
-17	0	-113	-113
-18	0	-17	-17
-19	4	-83	-87
-20	5	45	40
-21	8	45	37
-22	9	47	38
-23	0	61	61
-24	4	63	59
-
 -- !sql_test_Double_BigInt_0 --
 \N	\N	\N
 1	5354529.5244	-5354528.4756
@@ -6041,59 +4557,6 @@
 23	16	5472010779	5472010763
 24	18	10944010783	10944010765
 
--- !sql_test_Double_BigInt_4 --
-\N	\N	\N	\N
-1	0	33	33
-2	0	39	39
-3	1	51	50
-4	1	75	74
-5	2	123	121
-6	2	-37	-39
-7	0	-97	-97
-8	1	31	30
-9	8	27	19
-10	11	27	16
-11	16	27	11
-12	18	31	13
-13	0	33	33
-14	0	39	39
-15	1	51	50
-16	1	75	74
-17	2	123	121
-18	2	-37	-39
-19	0	-97	-97
-20	1	31	30
-21	8	27	19
-22	11	27	16
-23	16	27	11
-24	18	31	13
-
--- !sql_test_Double_BigInt_notn_4 --
-1	0	33	33
-2	0	39	39
-3	1	51	50
-4	1	75	74
-5	2	123	121
-6	2	-37	-39
-7	0	-97	-97
-8	1	31	30
-9	8	27	19
-10	11	27	16
-11	16	27	11
-12	18	31	13
-13	0	33	33
-14	0	39	39
-15	1	51	50
-16	1	75	74
-17	2	123	121
-18	2	-37	-39
-19	0	-97	-97
-20	1	31	30
-21	8	27	19
-22	11	27	16
-23	16	27	11
-24	18	31	13
-
 -- !sql_test_Double_LargeInt_0 --
 \N	\N	\N
 1	1.070906455244E8	-1.070906444756E8
@@ -6306,59 +4769,6 @@
 23	16	109440215645	109440215629
 24	20	218880215647	218880215627
 
--- !sql_test_Double_LargeInt_4 --
-\N	\N	\N	\N
-1	0	-43	-43
-2	0	77	77
-3	1	61	60
-4	1	29	28
-5	0	-33	-33
-6	0	95	95
-7	4	93	89
-8	5	93	88
-9	8	93	85
-10	9	95	86
-11	16	93	77
-12	20	95	75
-13	0	-43	-43
-14	0	77	77
-15	1	61	60
-16	1	29	28
-17	0	-33	-33
-18	0	95	95
-19	4	93	89
-20	5	93	88
-21	8	93	85
-22	9	95	86
-23	16	93	77
-24	20	95	75
-
--- !sql_test_Double_LargeInt_notn_4 --
-1	0	-43	-43
-2	0	77	77
-3	1	61	60
-4	1	29	28
-5	0	-33	-33
-6	0	95	95
-7	4	93	89
-8	5	93	88
-9	8	93	85
-10	9	95	86
-11	16	93	77
-12	20	95	75
-13	0	-43	-43
-14	0	77	77
-15	1	61	60
-16	1	29	28
-17	0	-33	-33
-18	0	95	95
-19	4	93	89
-20	5	93	88
-21	8	93	85
-22	9	95	86
-23	16	93	77
-24	20	95	75
-
 -- !sql_test_Double_Float_0 --
 \N	\N	\N
 1	0.6244000014901161	0.42439999850988386
@@ -7101,59 +5511,6 @@
 23	0	795	795
 24	6	1118	1112
 
--- !sql_test_Double_DecimalV2_4 --
-\N	\N	\N	\N
-1	0	24	24
-2	0	34	34
-3	0	49	49
-4	0	69	69
-5	0	99	99
-6	2	127	125
-7	4	127	123
-8	5	127	122
-9	8	127	119
-10	11	127	116
-11	16	127	111
-12	22	127	105
-13	0	24	24
-14	0	34	34
-15	0	49	49
-16	0	69	69
-17	0	99	99
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
--- !sql_test_Double_DecimalV2_notn_4 --
-1	0	24	24
-2	0	34	34
-3	0	49	49
-4	0	69	69
-5	0	99	99
-6	2	127	125
-7	4	127	123
-8	5	127	122
-9	8	127	119
-10	11	127	116
-11	16	127	111
-12	22	127	105
-13	0	24	24
-14	0	34	34
-15	0	49	49
-16	0	69	69
-17	0	99	99
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
 -- !sql_test_Double_Decimal32V3_0 --
 \N	\N	\N
 1	12.5364	-11.4876
@@ -7366,59 +5723,6 @@
 23	0	272	272
 24	2	287	285
 
--- !sql_test_Double_Decimal32V3_4 --
-\N	\N	\N	\N
-1	0	12	12
-2	0	23	23
-3	0	35	35
-4	1	45	44
-5	0	58	58
-6	2	67	65
-7	4	78	74
-8	1	93	92
-9	0	108	108
-10	0	123	123
-11	16	123	107
-12	22	127	105
-13	0	127	127
-14	0	127	127
-15	1	127	126
-16	1	127	126
-17	2	127	125
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
--- !sql_test_Double_Decimal32V3_notn_4 --
-1	0	12	12
-2	0	23	23
-3	0	35	35
-4	1	45	44
-5	0	58	58
-6	2	67	65
-7	4	78	74
-8	1	93	92
-9	0	108	108
-10	0	123	123
-11	16	123	107
-12	22	127	105
-13	0	127	127
-14	0	127	127
-15	1	127	126
-16	1	127	126
-17	2	127	125
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
 -- !sql_test_Double_Decimal64V3_0 --
 \N	\N	\N
 1	1234.53674	-1233.48794
@@ -7631,59 +5935,6 @@
 23	0	25694	25694
 24	4	26807	26803
 
--- !sql_test_Double_Decimal64V3_4 --
-\N	\N	\N	\N
-1	0	127	127
-2	0	127	127
-3	1	127	126
-4	1	127	126
-5	2	127	125
-6	2	127	125
-7	4	127	123
-8	5	127	122
-9	8	127	119
-10	11	127	116
-11	16	127	111
-12	22	127	105
-13	0	127	127
-14	0	127	127
-15	1	127	126
-16	1	127	126
-17	2	127	125
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
--- !sql_test_Double_Decimal64V3_notn_4 --
-1	0	127	127
-2	0	127	127
-3	1	127	126
-4	1	127	126
-5	2	127	125
-6	2	127	125
-7	4	127	123
-8	5	127	122
-9	8	127	119
-10	11	127	116
-11	16	127	111
-12	22	127	105
-13	0	127	127
-14	0	127	127
-15	1	127	126
-16	1	127	126
-17	2	127	125
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
 -- !sql_test_Double_Decimal128V3_0 --
 \N	\N	\N
 1	1.2345678536744999E7	-1.2345677487945E7
@@ -7896,59 +6147,6 @@
 23	0	256790138	256790138
 24	16	267901239	267901223
 
--- !sql_test_Double_Decimal128V3_4 --
-\N	\N	\N	\N
-1	0	127	127
-2	0	127	127
-3	1	127	126
-4	1	127	126
-5	2	127	125
-6	2	127	125
-7	4	127	123
-8	5	127	122
-9	8	127	119
-10	11	127	116
-11	16	127	111
-12	22	127	105
-13	0	127	127
-14	0	127	127
-15	1	127	126
-16	1	127	126
-17	2	127	125
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
--- !sql_test_Double_Decimal128V3_notn_4 --
-1	0	127	127
-2	0	127	127
-3	1	127	126
-4	1	127	126
-5	2	127	125
-6	2	127	125
-7	4	127	123
-8	5	127	122
-9	8	127	119
-10	11	127	116
-11	16	127	111
-12	22	127	105
-13	0	127	127
-14	0	127	127
-15	1	127	126
-16	1	127	126
-17	2	127	125
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
 -- !sql_test_Double_Char_0 --
 \N	\N	\N
 1	\N	\N
@@ -8161,59 +6359,6 @@
 23	0	4948	4948
 24	22	6975	6953
 
--- !sql_test_Double_Char_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_Double_Char_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_Double_Varchar_0 --
 \N	\N	\N
 1	\N	\N
@@ -8426,59 +6571,6 @@
 23	16	74137	74121
 24	4	104863	104859
 
--- !sql_test_Double_Varchar_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_Double_Varchar_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_Double_String_0 --
 \N	\N	\N
 1	\N	\N
@@ -8691,59 +6783,6 @@
 23	0	339003	339003
 24	6	479415	479409
 
--- !sql_test_Double_String_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_Double_String_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_Double_Date_0 --
 \N	\N	\N
 1	2.01203015244E7	-2.01203004756E7
@@ -8956,59 +6995,6 @@
 23	16	20120311	20120295
 24	16	20120318	20120302
 
--- !sql_test_Double_Date_4 --
-\N	\N	\N	\N
-1	0	-19	-19
-2	0	-18	-18
-3	1	-17	-18
-4	0	-15	-15
-5	0	-13	-13
-6	2	-14	-16
-7	0	-9	-9
-8	4	-11	-15
-9	0	-3	-3
-10	2	-1	-3
-11	16	-9	-25
-12	16	-2	-18
-13	0	-19	-19
-14	0	-18	-18
-15	1	-17	-18
-16	0	-15	-15
-17	0	-13	-13
-18	2	-14	-16
-19	0	-9	-9
-20	4	-11	-15
-21	0	-3	-3
-22	2	-1	-3
-23	16	-9	-25
-24	16	-2	-18
-
--- !sql_test_Double_Date_notn_4 --
-1	0	-19	-19
-2	0	-18	-18
-3	1	-17	-18
-4	0	-15	-15
-5	0	-13	-13
-6	2	-14	-16
-7	0	-9	-9
-8	4	-11	-15
-9	0	-3	-3
-10	2	-1	-3
-11	16	-9	-25
-12	16	-2	-18
-13	0	-19	-19
-14	0	-18	-18
-15	1	-17	-18
-16	0	-15	-15
-17	0	-13	-13
-18	2	-14	-16
-19	0	-9	-9
-20	4	-11	-15
-21	0	-3	-3
-22	2	-1	-3
-23	16	-9	-25
-24	16	-2	-18
-
 -- !sql_test_Double_DateTime_0 --
 \N	\N	\N
 1	2.0120301010001523E13	-2.0120301010000477E13
@@ -9221,59 +7207,6 @@
 23	0	20120311111027	20120311111027
 24	16	20120312121118	20120312121102
 
--- !sql_test_Double_DateTime_4 --
-\N	\N	\N	\N
-1	0	81	81
-2	0	6	6
-3	1	-69	-70
-4	0	113	113
-5	0	39	39
-6	2	-38	-40
-7	4	-113	-117
-8	4	69	65
-9	8	-7	-15
-10	10	-81	-91
-11	0	115	115
-12	16	30	14
-13	0	81	81
-14	0	6	6
-15	1	-69	-70
-16	0	113	113
-17	0	39	39
-18	2	-38	-40
-19	4	-113	-117
-20	4	69	65
-21	8	-7	-15
-22	10	-81	-91
-23	0	115	115
-24	16	30	14
-
--- !sql_test_Double_DateTime_notn_4 --
-1	0	81	81
-2	0	6	6
-3	1	-69	-70
-4	0	113	113
-5	0	39	39
-6	2	-38	-40
-7	4	-113	-117
-8	4	69	65
-9	8	-7	-15
-10	10	-81	-91
-11	0	115	115
-12	16	30	14
-13	0	81	81
-14	0	6	6
-15	1	-69	-70
-16	0	113	113
-17	0	39	39
-18	2	-38	-40
-19	4	-113	-117
-20	4	69	65
-21	8	-7	-15
-22	10	-81	-91
-23	0	115	115
-24	16	30	14
-
 -- !sql_test_Double_DateV2_0 --
 \N	\N	\N
 1	2.01203015244E7	-2.01203004756E7
@@ -9486,59 +7419,6 @@
 23	16	20120311	20120295
 24	16	20120318	20120302
 
--- !sql_test_Double_DateV2_4 --
-\N	\N	\N	\N
-1	0	-19	-19
-2	0	-18	-18
-3	1	-17	-18
-4	0	-15	-15
-5	0	-13	-13
-6	2	-14	-16
-7	0	-9	-9
-8	4	-11	-15
-9	0	-3	-3
-10	2	-1	-3
-11	16	-9	-25
-12	16	-2	-18
-13	0	-19	-19
-14	0	-18	-18
-15	1	-17	-18
-16	0	-15	-15
-17	0	-13	-13
-18	2	-14	-16
-19	0	-9	-9
-20	4	-11	-15
-21	0	-3	-3
-22	2	-1	-3
-23	16	-9	-25
-24	16	-2	-18
-
--- !sql_test_Double_DateV2_notn_4 --
-1	0	-19	-19
-2	0	-18	-18
-3	1	-17	-18
-4	0	-15	-15
-5	0	-13	-13
-6	2	-14	-16
-7	0	-9	-9
-8	4	-11	-15
-9	0	-3	-3
-10	2	-1	-3
-11	16	-9	-25
-12	16	-2	-18
-13	0	-19	-19
-14	0	-18	-18
-15	1	-17	-18
-16	0	-15	-15
-17	0	-13	-13
-18	2	-14	-16
-19	0	-9	-9
-20	4	-11	-15
-21	0	-3	-3
-22	2	-1	-3
-23	16	-9	-25
-24	16	-2	-18
-
 -- !sql_test_Double_DateTimeV2_0 --
 \N	\N	\N
 1	2.0120301010001523E13	-2.0120301010000477E13
@@ -9751,59 +7631,6 @@
 23	0	20120311111027	20120311111027
 24	16	20120312121118	20120312121102
 
--- !sql_test_Double_DateTimeV2_4 --
-\N	\N	\N	\N
-1	0	81	81
-2	0	6	6
-3	1	-69	-70
-4	0	113	113
-5	0	39	39
-6	2	-38	-40
-7	4	-113	-117
-8	4	69	65
-9	8	-7	-15
-10	10	-81	-91
-11	0	115	115
-12	16	30	14
-13	0	81	81
-14	0	6	6
-15	1	-69	-70
-16	0	113	113
-17	0	39	39
-18	2	-38	-40
-19	4	-113	-117
-20	4	69	65
-21	8	-7	-15
-22	10	-81	-91
-23	0	115	115
-24	16	30	14
-
--- !sql_test_Double_DateTimeV2_notn_4 --
-1	0	81	81
-2	0	6	6
-3	1	-69	-70
-4	0	113	113
-5	0	39	39
-6	2	-38	-40
-7	4	-113	-117
-8	4	69	65
-9	8	-7	-15
-10	10	-81	-91
-11	0	115	115
-12	16	30	14
-13	0	81	81
-14	0	6	6
-15	1	-69	-70
-16	0	113	113
-17	0	39	39
-18	2	-38	-40
-19	4	-113	-117
-20	4	69	65
-21	8	-7	-15
-22	10	-81	-91
-23	0	115	115
-24	16	30	14
-
 -- !sql_test_Double_Boolean_0 --
 \N	\N	\N
 1	0.5244	0.5244
@@ -10016,59 +7843,6 @@
 23	0	17	17
 24	0	23	23
 
--- !sql_test_Double_Boolean_4 --
-\N	\N	\N	\N
-1	0	0	0
-2	0	0	0
-3	0	1	1
-4	0	1	1
-5	0	2	2
-6	0	2	2
-7	0	4	4
-8	1	5	4
-9	0	9	9
-10	1	11	10
-11	0	17	17
-12	0	23	23
-13	0	0	0
-14	0	0	0
-15	0	1	1
-16	0	1	1
-17	0	2	2
-18	0	2	2
-19	0	4	4
-20	1	5	4
-21	0	9	9
-22	1	11	10
-23	0	17	17
-24	0	23	23
-
--- !sql_test_Double_Boolean_notn_4 --
-1	0	0	0
-2	0	0	0
-3	0	1	1
-4	0	1	1
-5	0	2	2
-6	0	2	2
-7	0	4	4
-8	1	5	4
-9	0	9	9
-10	1	11	10
-11	0	17	17
-12	0	23	23
-13	0	0	0
-14	0	0	0
-15	0	1	1
-16	0	1	1
-17	0	2	2
-18	0	2	2
-19	0	4	4
-20	1	5	4
-21	0	9	9
-22	1	11	10
-23	0	17	17
-24	0	23	23
-
 -- !sql_test_DecimalV2_TinyInt_0 --
 \N	\N	\N
 1	25.395000000	23.395000000
@@ -10281,59 +8055,6 @@
 23	11	779	768
 24	12	1102	1090
 
--- !sql_test_DecimalV2_TinyInt_4 --
-\N	\N	\N	\N
-1	0	25	25
-2	2	34	32
-3	0	51	51
-4	4	68	64
-5	1	101	100
-6	6	127	121
-7	7	127	120
-8	8	127	119
-9	9	127	118
-10	10	127	117
-11	11	127	116
-12	12	127	115
-13	0	25	25
-14	2	34	32
-15	0	51	51
-16	4	68	64
-17	1	101	100
-18	6	127	121
-19	7	127	120
-20	8	127	119
-21	9	127	118
-22	10	127	117
-23	11	127	116
-24	12	127	115
-
--- !sql_test_DecimalV2_TinyInt_notn_4 --
-1	0	25	25
-2	2	34	32
-3	0	51	51
-4	4	68	64
-5	1	101	100
-6	6	127	121
-7	7	127	120
-8	8	127	119
-9	9	127	118
-10	10	127	117
-11	11	127	116
-12	12	127	115
-13	0	25	25
-14	2	34	32
-15	0	51	51
-16	4	68	64
-17	1	101	100
-18	6	127	121
-19	7	127	120
-20	8	127	119
-21	9	127	118
-22	10	127	117
-23	11	127	116
-24	12	127	115
-
 -- !sql_test_DecimalV2_SmallInt_0 --
 \N	\N	\N
 1	34.395000000	14.395000000
@@ -10541,63 +8262,10 @@
 18	0	457	457
 19	128	706	578
 20	256	1299	1043
-21	0	2949	2949
-22	0	5671	5671
-23	0	11019	11019
-24	0	21582	21582
-
--- !sql_test_DecimalV2_SmallInt_4 --
-\N	\N	\N	\N
-1	8	26	18
-2	0	54	54
-3	32	56	24
-4	64	84	20
-5	32	-31	-63
-6	64	127	63
-7	0	-1	-1
-8	0	127	127
-9	0	127	127
-10	0	127	127
-11	0	127	127
-12	0	127	127
-13	8	26	18
-14	0	54	54
-15	32	56	24
-16	64	84	20
-17	32	-31	-63
-18	64	127	63
-19	0	-1	-1
-20	0	127	127
-21	0	127	127
-22	0	127	127
-23	0	127	127
-24	0	127	127
-
--- !sql_test_DecimalV2_SmallInt_notn_4 --
-1	8	26	18
-2	0	54	54
-3	32	56	24
-4	64	84	20
-5	32	-31	-63
-6	64	127	63
-7	0	-1	-1
-8	0	127	127
-9	0	127	127
-10	0	127	127
-11	0	127	127
-12	0	127	127
-13	8	26	18
-14	0	54	54
-15	32	56	24
-16	64	84	20
-17	32	-31	-63
-18	64	127	63
-19	0	-1	-1
-20	0	127	127
-21	0	127	127
-22	0	127	127
-23	0	127	127
-24	0	127	127
+21	0	2949	2949
+22	0	5671	5671
+23	0	11019	11019
+24	0	21582	21582
 
 -- !sql_test_DecimalV2_Integer_0 --
 \N	\N	\N
@@ -10811,59 +8479,6 @@
 23	9	24320815	24320806
 24	12	48641135	48641123
 
--- !sql_test_DecimalV2_Integer_4 --
-\N	\N	\N	\N
-1	16	-5	-21
-2	32	-69	-101
-3	0	117	117
-4	68	93	25
-5	1	-19	-20
-6	109	-1	-110
-7	45	-1	-46
-8	45	127	82
-9	45	127	82
-10	45	127	82
-11	45	127	82
-12	45	127	82
-13	16	-5	-21
-14	32	-69	-101
-15	0	117	117
-16	68	93	25
-17	1	-19	-20
-18	109	-1	-110
-19	45	-1	-46
-20	45	127	82
-21	45	127	82
-22	45	127	82
-23	45	127	82
-24	45	127	82
-
--- !sql_test_DecimalV2_Integer_notn_4 --
-1	16	-5	-21
-2	32	-69	-101
-3	0	117	117
-4	68	93	25
-5	1	-19	-20
-6	109	-1	-110
-7	45	-1	-46
-8	45	127	82
-9	45	127	82
-10	45	127	82
-11	45	127	82
-12	45	127	82
-13	16	-5	-21
-14	32	-69	-101
-15	0	117	117
-16	68	93	25
-17	1	-19	-20
-18	109	-1	-110
-19	45	-1	-46
-20	45	127	82
-21	45	127	82
-22	45	127	82
-23	45	127	82
-24	45	127	82
-
 -- !sql_test_DecimalV2_BigInt_0 --
 \N	\N	\N
 1	5354553.395000000	-5354504.605000000
@@ -11076,59 +8691,6 @@
 23	523	5472011035	5472010512
 24	10	10944011871	10944011861
 
--- !sql_test_DecimalV2_BigInt_4 --
-\N	\N	\N	\N
-1	0	57	57
-2	34	39	5
-3	48	51	3
-4	64	79	15
-5	97	123	26
-6	91	-1	-92
-7	27	-1	-28
-8	27	127	100
-9	27	127	100
-10	27	127	100
-11	27	127	100
-12	27	127	100
-13	0	57	57
-14	34	39	5
-15	48	51	3
-16	64	79	15
-17	97	123	26
-18	91	-1	-92
-19	27	-1	-28
-20	27	127	100
-21	27	127	100
-22	27	127	100
-23	27	127	100
-24	27	127	100
-
--- !sql_test_DecimalV2_BigInt_notn_4 --
-1	0	57	57
-2	34	39	5
-3	48	51	3
-4	64	79	15
-5	97	123	26
-6	91	-1	-92
-7	27	-1	-28
-8	27	127	100
-9	27	127	100
-10	27	127	100
-11	27	127	100
-12	27	127	100
-13	0	57	57
-14	34	39	5
-15	48	51	3
-16	64	79	15
-17	97	123	26
-18	91	-1	-92
-19	27	-1	-28
-20	27	127	100
-21	27	127	100
-22	27	127	100
-23	27	127	100
-24	27	127	100
-
 -- !sql_test_DecimalV2_LargeInt_0 --
 \N	\N	\N
 1	107090669.395000000	-107090620.605000000
@@ -11182,59 +8744,6 @@
 23	109440216424.908000000	-109440214865.092000000
 24	218880216747.957000000	-218880214542.043000000
 
--- !sql_test_DecimalV2_LargeInt_1 --
-\N	\N	\N	\N
-1	2612476284.775000000	2.28E-7	24.395000000
-2	7378391302.180000000	1.61E-7	34.484000000
-3	20853703987.620000000	1.14E-7	48.756000000
-4	58961132213.235000000	8.1E-8	68.943000000
-5	166735764093.630000000	5.7E-8	97.494000000
-6	471558811838.730000000	4.0E-8	137.874000000
-7	1333705246462.100000000	2.9E-8	194.980000000
-8	3772196342167.945000000	2.0E-8	275.741000000
-9	10669252891845.975000000	1.4E-8	389.955000000
-10	30177049803688.955000000	1.0E-8	551.479000000
-11	85353299703260.660000000	7E-9	779.908000000
-12	241415466007162.265000000	5E-9	1102.957000000
-13	2612476284.775000000	2.28E-7	24.395000000
-14	7378391302.180000000	1.61E-7	34.484000000
-15	20853703987.620000000	1.14E-7	48.756000000
-16	58961132213.235000000	8.1E-8	68.943000000
-17	166735764093.630000000	5.7E-8	97.494000000
-18	471558811838.730000000	4.0E-8	137.874000000
-19	1333705246462.100000000	2.9E-8	194.980000000
-20	3772196342167.945000000	2.0E-8	275.741000000
-21	10669252891845.975000000	1.4E-8	389.955000000
-22	30177049803688.955000000	1.0E-8	551.479000000
-23	85353299703260.660000000	7E-9	779.908000000
-24	241415466007162.265000000	5E-9	1102.957000000
-
--- !sql_test_DecimalV2_LargeInt_notn_1 --
-1	2612476284.775000000	2.28E-7	24.395000000
-2	7378391302.180000000	1.61E-7	34.484000000
-3	20853703987.620000000	1.14E-7	48.756000000
-4	58961132213.235000000	8.1E-8	68.943000000
-5	166735764093.630000000	5.7E-8	97.494000000
-6	471558811838.730000000	4.0E-8	137.874000000
-7	1333705246462.100000000	2.9E-8	194.980000000
-8	3772196342167.945000000	2.0E-8	275.741000000
-9	10669252891845.975000000	1.4E-8	389.955000000
-10	30177049803688.955000000	1.0E-8	551.479000000
-11	85353299703260.660000000	7E-9	779.908000000
-12	241415466007162.265000000	5E-9	1102.957000000
-13	2612476284.775000000	2.28E-7	24.395000000
-14	7378391302.180000000	1.61E-7	34.484000000
-15	20853703987.620000000	1.14E-7	48.756000000
-16	58961132213.235000000	8.1E-8	68.943000000
-17	166735764093.630000000	5.7E-8	97.494000000
-18	471558811838.730000000	4.0E-8	137.874000000
-19	1333705246462.100000000	2.9E-8	194.980000000
-20	3772196342167.945000000	2.0E-8	275.741000000
-21	10669252891845.975000000	1.4E-8	389.955000000
-22	30177049803688.955000000	1.0E-8	551.479000000
-23	85353299703260.660000000	7E-9	779.908000000
-24	241415466007162.265000000	5E-9	1102.957000000
-
 -- !sql_test_DecimalV2_LargeInt_2 --
 \N	\N
 1	0
@@ -11341,59 +8850,6 @@
 23	521	109440215903	109440215382
 24	76	218880216671	218880216595
 
--- !sql_test_DecimalV2_LargeInt_4 --
-\N	\N	\N	\N
-1	16	-35	-51
-2	0	111	111
-3	48	61	13
-4	4	93	89
-5	65	-3	-68
-6	93	127	34
-7	93	127	34
-8	93	127	34
-9	93	127	34
-10	93	127	34
-11	93	127	34
-12	93	127	34
-13	16	-35	-51
-14	0	111	111
-15	48	61	13
-16	4	93	89
-17	65	-3	-68
-18	93	127	34
-19	93	127	34
-20	93	127	34
-21	93	127	34
-22	93	127	34
-23	93	127	34
-24	93	127	34
-
--- !sql_test_DecimalV2_LargeInt_notn_4 --
-1	16	-35	-51
-2	0	111	111
-3	48	61	13
-4	4	93	89
-5	65	-3	-68
-6	93	127	34
-7	93	127	34
-8	93	127	34
-9	93	127	34
-10	93	127	34
-11	93	127	34
-12	93	127	34
-13	16	-35	-51
-14	0	111	111
-15	48	61	13
-16	4	93	89
-17	65	-3	-68
-18	93	127	34
-19	93	127	34
-20	93	127	34
-21	93	127	34
-22	93	127	34
-23	93	127	34
-24	93	127	34
-
 -- !sql_test_DecimalV2_Float_0 --
 \N	\N	\N
 1	24.495000000	24.295000000
@@ -11606,59 +9062,6 @@
 23	1	779	778
 24	0	1103	1103
 
--- !sql_test_DecimalV2_Float_4 --
-\N	\N	\N	\N
-1	0	24	24
-2	0	34	34
-3	0	48	48
-4	0	68	68
-5	0	97	97
-6	0	127	127
-7	0	127	127
-8	0	127	127
-9	0	127	127
-10	1	127	126
-11	1	127	126
-12	1	127	126
-13	0	24	24
-14	0	34	34
-15	0	48	48
-16	0	68	68
-17	0	97	97
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
--- !sql_test_DecimalV2_Float_notn_4 --
-1	0	24	24
-2	0	34	34
-3	0	48	48
-4	0	68	68
-5	0	97	97
-6	0	127	127
-7	0	127	127
-8	0	127	127
-9	0	127	127
-10	1	127	126
-11	1	127	126
-12	1	127	126
-13	0	24	24
-14	0	34	34
-15	0	48	48
-16	0	68	68
-17	0	97	97
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
 -- !sql_test_DecimalV2_Double_0 --
 \N	\N	\N
 1	24.9194	23.8706
@@ -11871,59 +9274,6 @@
 23	0	795	795
 24	6	1118	1112
 
--- !sql_test_DecimalV2_Double_4 --
-\N	\N	\N	\N
-1	0	24	24
-2	0	34	34
-3	0	49	49
-4	0	69	69
-5	0	99	99
-6	2	127	125
-7	4	127	123
-8	5	127	122
-9	8	127	119
-10	11	127	116
-11	16	127	111
-12	22	127	105
-13	0	24	24
-14	0	34	34
-15	0	49	49
-16	0	69	69
-17	0	99	99
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
--- !sql_test_DecimalV2_Double_notn_4 --
-1	0	24	24
-2	0	34	34
-3	0	49	49
-4	0	69	69
-5	0	99	99
-6	2	127	125
-7	4	127	123
-8	5	127	122
-9	8	127	119
-10	11	127	116
-11	16	127	111
-12	22	127	105
-13	0	24	24
-14	0	34	34
-15	0	49	49
-16	0	69	69
-17	0	99	99
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
 -- !sql_test_DecimalV2_DecimalV2_0 --
 \N	\N	\N
 1	48.790000000	0E-9
@@ -12136,59 +9486,6 @@
 23	779	779	0
 24	1102	1102	0
 
--- !sql_test_DecimalV2_DecimalV2_4 --
-\N	\N	\N	\N
-1	24	24	0
-2	34	34	0
-3	48	48	0
-4	68	68	0
-5	97	97	0
-6	127	127	0
-7	127	127	0
-8	127	127	0
-9	127	127	0
-10	127	127	0
-11	127	127	0
-12	127	127	0
-13	24	24	0
-14	34	34	0
-15	48	48	0
-16	68	68	0
-17	97	97	0
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
--- !sql_test_DecimalV2_DecimalV2_notn_4 --
-1	24	24	0
-2	34	34	0
-3	48	48	0
-4	68	68	0
-5	97	97	0
-6	127	127	0
-7	127	127	0
-8	127	127	0
-9	127	127	0
-10	127	127	0
-11	127	127	0
-12	127	127	0
-13	24	24	0
-14	34	34	0
-15	48	48	0
-16	68	68	0
-17	97	97	0
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
 -- !sql_test_DecimalV2_Decimal32V3_0 --
 \N	\N	\N
 1	36.407	12.383
@@ -12242,59 +9539,6 @@
 23	1036.362	523.454
 24	1370.522	835.392
 
--- !sql_test_DecimalV2_Decimal32V3_1 --
-\N	\N	\N	\N
-1	293.032740	0.002	0.001
-2	797.373532	0.001	0.002
-3	1669.112904	0.001	0.003
-4	3126.220335	0.001	0.004
-5	5504.121264	0.001	0.005
-6	9315.732558	0.002	0.006
-7	15340.636440	0.002	0.007
-8	24758.508649	0.003	0.008
-9	39346.459500	0.003	0.009
-10	61771.714269	0.004	0.010
-11	96023.832776	0.006	0.011
-12	148053.226981	0.008	0.012
-13	3545.666880	0.000	0.013
-14	5395.194220	0.000	0.014
-15	8169.847896	0.000	0.015
-16	12318.528411	0.000	0.016
-17	18503.191272	0.000	0.017
-18	27698.748726	0.000	0.018
-19	41337.709800	0.000	0.019
-20	61523.607661	0.001	0.020
-21	91339.939560	0.001	0.021
-22	135301.512297	0.002	0.022
-23	200010.526232	0.003	0.023
-24	295112.689705	0.004	0.024
-
--- !sql_test_DecimalV2_Decimal32V3_notn_1 --
-1	293.032740	0.002	-374022.048
-2	797.373532	0.001	21.994
-3	1669.112904	0.001	0.007
-4	3126.220335	0.001	0.000
-5	5504.121264	0.001	0.000
-6	9315.732558	0.002	0.000
-7	15340.636440	0.002	-903942.144
-8	24758.508649	0.003	32.558
-9	39346.459500	0.003	0.000
-10	61771.714269	0.004	0.000
-11	96023.832776	0.006	0.000
-12	148053.226981	0.008	0.000
-13	3545.666880	0.000	0.000
-14	5395.194220	0.000	0.000
-15	8169.847896	0.000	0.000
-16	12318.528411	0.000	0.000
-17	18503.191272	0.000	0.000
-18	27698.748726	0.000	32.558
-19	41337.709800	0.000	0.000
-20	61523.607661	0.001	0.000
-21	91339.939560	0.001	0.000
-22	135301.512297	0.002	21.994
-23	200010.526232	0.003	0.000
-24	295112.689705	0.004	0.000
-
 -- !sql_test_DecimalV2_Decimal32V3_2 --
 \N	\N
 1	2
@@ -12401,59 +9645,6 @@
 23	256	779	523
 24	10	1359	1349
 
--- !sql_test_DecimalV2_Decimal32V3_4 --
-\N	\N	\N	\N
-1	8	28	20
-2	2	55	53
-3	32	50	18
-4	4	109	105
-5	32	121	89
-6	67	127	60
-7	78	127	49
-8	89	127	38
-9	100	127	27
-10	112	127	15
-11	123	127	4
-12	127	127	0
-13	24	127	103
-14	34	127	93
-15	48	127	79
-16	68	127	59
-17	97	127	30
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
--- !sql_test_DecimalV2_Decimal32V3_notn_4 --
-1	8	28	20
-2	2	55	53
-3	32	50	18
-4	4	109	105
-5	32	121	89
-6	67	127	60
-7	78	127	49
-8	89	127	38
-9	100	127	27
-10	112	127	15
-11	123	127	4
-12	127	127	0
-13	24	127	103
-14	34	127	93
-15	48	127	79
-16	68	127	59
-17	97	127	30
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
 -- !sql_test_DecimalV2_Decimal64V3_0 --
 \N	\N	\N
 1	1258.40734	-1209.61734
@@ -12507,59 +9698,6 @@
 23	26458.36476	-24898.54876
 24	27892.52487	-25686.61087
 
--- !sql_test_DecimalV2_Decimal64V3_1 --
-\N	\N	\N	\N
-1	30103.73103430	0.000	2379.60745
-2	80869.23704980	0.000	3504.99056
-3	168512.17220736	0.000	4636.28867
-4	314886.51252681	0.000	5775.95078
-5	553615.46530932	0.000	6927.44189
-6	936104.88326586	0.000	8095.65900
-7	1540474.39142000	0.000	9287.53111
-8	2484920.01672151	0.000	10512.85622
-9	3947475.94524510	0.000	11785.49133
-10	6195321.88573607	0.000	13125.03144
-11	9628060.53184352	0.000	14559.19155
-12	14841648.09056435	0.000	14591.74066
-13	355370.39737570	0.000	15712.94077
-14	540655.90325668	0.000	16838.32388
-15	818592.17155728	0.000	17969.62199
-16	1234126.51160757	0.000	19109.28410
-17	1853535.46400940	0.000	20260.77521
-18	2774424.88142754	0.000	21428.99232
-19	4140207.72215360	0.000	22620.86443
-20	6161466.67971163	0.000	23846.18954
-21	9146875.94004570	0.000	25118.82465
-22	13548375.21171635	0.000	26458.36476
-23	20026833.85477808	0.000	27892.52487
-24	29547741.40919159	0.000	70898515052585.4213
-
--- !sql_test_DecimalV2_Decimal64V3_notn_1 --
-1	30103.73103430	0.000	0.00000
-2	80869.23704980	0.000	0.00000
-3	168512.17220736	0.000	0.00000
-4	314886.51252681	0.000	0.00000
-5	553615.46530932	0.000	0.00000
-6	936104.88326586	0.000	0.00000
-7	1540474.39142000	0.000	0.00000
-8	2484920.01672151	0.000	0.00000
-9	3947475.94524510	0.000	171798.69184
-10	6195321.88573607	0.000	0.00000
-11	9628060.53184352	0.000	0.00000
-12	14841648.09056435	0.000	0.00000
-13	355370.39737570	0.000	0.00000
-14	540655.90325668	0.000	1398476912.97128
-15	818592.17155728	0.000	1398476912.97128
-16	1234126.51160757	0.000	0.00000
-17	1853535.46400940	0.000	0.00000
-18	2774424.88142754	0.000	0.00000
-19	4140207.72215360	0.000	0.00000
-20	6161466.67971163	0.000	1398476912.97176
-21	9146875.94004570	0.000	1398476912.97176
-22	13548375.21171635	0.000	0.00000
-23	20026833.85477808	0.000	0.00000
-24	29547741.40919159	0.000	0.00000
-
 -- !sql_test_DecimalV2_Decimal64V3_2 --
 \N	\N
 1	0
@@ -12666,59 +9804,6 @@
 23	10	26447	26437
 24	4	27887	27883
 
--- !sql_test_DecimalV2_Decimal64V3_4 --
-\N	\N	\N	\N
-1	24	127	103
-2	34	127	93
-3	48	127	79
-4	68	127	59
-5	97	127	30
-6	127	127	0
-7	127	127	0
-8	127	127	0
-9	127	127	0
-10	127	127	0
-11	127	127	0
-12	127	127	0
-13	24	127	103
-14	34	127	93
-15	48	127	79
-16	68	127	59
-17	97	127	30
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
--- !sql_test_DecimalV2_Decimal64V3_notn_4 --
-1	24	127	103
-2	34	127	93
-3	48	127	79
-4	68	127	59
-5	97	127	30
-6	127	127	0
-7	127	127	0
-8	127	127	0
-9	127	127	0
-10	127	127	0
-11	127	127	0
-12	127	127	0
-13	24	127	103
-14	34	127	93
-15	48	127	79
-16	68	127	59
-17	97	127	30
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
 -- !sql_test_DecimalV2_Decimal128V3_0 --
 \N	\N	\N
 1	12345702.40734500	-12345653.61734500
@@ -12772,59 +9857,6 @@
 23	256790902.36478700	-256789342.54878700
 24	267902336.52489800	-267900130.61089800
 
--- !sql_test_DecimalV2_Decimal128V3_1 --
-\N	\N	\N	\N
-1	301172815.11115627500	0.000	23456789.12345600
-2	808883916.13325670400	0.000	34567900.23456700
-3	1685392543.83654865200	0.000	45679011.34567800
-4	3149248079.20507835400	0.000	56790122.45678900
-5	5536696198.80218676600	0.000	67901233.56790000
-6	9361814676.94064460000	0.000	79012344.67901100
-7	15405826965.51356478000	0.000	90123455.79012200
-8	24850731823.02403040200	0.000	101234566.90123300
-9	39476925535.97031451500	0.000	112345678.01234400
-10	61956282164.56945677600	0.000	123456789.12345500
-11	96284937491.69554214000	0.000	134567900.23456600
-12	148422607539.01621166200	0.000	145679011.34567700
-13	3553839481.77779041500	0.000	156790122.45678800
-14	5406750582.79987739200	0.000	167901233.56789900
-15	8186192543.83648364400	0.000	179012344.67901000
-16	12341648079.20498643000	0.000	190123455.79012100
-17	18535896198.80205677400	0.000	201234566.90123200
-18	27745014676.94046076800	0.000	212345678.01234300
-19	41403160298.84663814000	0.000	223456789.12345400
-20	61616198489.69032941400	0.000	234567900.23456500
-21	91470925535.96979457500	0.000	245679011.34567600
-22	135486815497.90205480400	0.000	256790122.45678700
-23	200272670825.02783559600	0.000	267901233.56789800
-24	295483540872.34807438600	0.000	17426146262004286709407884.99594240
-
--- !sql_test_DecimalV2_Decimal128V3_notn_1 --
-1	301172815.11115627500	0.000	0E-8
-2	808883916.13325670400	0.000	12345678.01234500
-3	1685392543.83654865200	0.000	23456789.12345600
-4	3149248079.20507835400	0.000	34567900.23456700
-5	5536696198.80218676600	0.000	45679011.34567800
-6	9361814676.94064460000	0.000	56790122.45678900
-7	15405826965.51356478000	0.000	67901233.56790000
-8	24850731823.02403040200	0.000	79012344.67901100
-9	39476925535.97031451500	0.000	90123455.79012200
-10	61956282164.56945677600	0.000	101234566.90123300
-11	96284937491.69554214000	0.000	112345678.01234400
-12	148422607539.01621166200	0.000	123456789.12345500
-13	3553839481.77779041500	0.000	134567900.23456600
-14	5406750582.79987739200	0.000	145679011.34567700
-15	8186192543.83648364400	0.000	156790122.45678800
-16	12341648079.20498643000	0.000	167901233.56789900
-17	18535896198.80205677400	0.000	179012344.67901000
-18	27745014676.94046076800	0.000	190123455.79012100
-19	41403160298.84663814000	0.000	201234566.90123200
-20	61616198489.69032941400	0.000	212345678.01234300
-21	91470925535.96979457500	0.000	223456789.12345400
-22	135486815497.90205480400	0.000	234567900.23456500
-23	200272670825.02783559600	0.000	245679011.34567600
-24	295483540872.34807438600	0.000	256790122.45678700
-
 -- !sql_test_DecimalV2_Decimal128V3_2 --
 \N	\N
 1	0
@@ -12931,59 +9963,6 @@
 23	522	256790379	256789857
 24	0	267902335	267902335
 
--- !sql_test_DecimalV2_Decimal128V3_4 --
-\N	\N	\N	\N
-1	24	127	103
-2	34	127	93
-3	48	127	79
-4	68	127	59
-5	97	127	30
-6	127	127	0
-7	127	127	0
-8	127	127	0
-9	127	127	0
-10	127	127	0
-11	127	127	0
-12	127	127	0
-13	24	127	103
-14	34	127	93
-15	48	127	79
-16	68	127	59
-17	97	127	30
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
--- !sql_test_DecimalV2_Decimal128V3_notn_4 --
-1	24	127	103
-2	34	127	93
-3	48	127	79
-4	68	127	59
-5	97	127	30
-6	127	127	0
-7	127	127	0
-8	127	127	0
-9	127	127	0
-10	127	127	0
-11	127	127	0
-12	127	127	0
-13	24	127	103
-14	34	127	93
-15	48	127	79
-16	68	127	59
-17	97	127	30
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
 -- !sql_test_DecimalV2_Char_0 --
 \N	\N	\N
 1	\N	\N
@@ -13196,59 +10175,6 @@
 23	768	4943	4175
 24	14	8063	8049
 
--- !sql_test_DecimalV2_Char_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DecimalV2_Char_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DecimalV2_Varchar_0 --
 \N	\N	\N
 1	\N	\N
@@ -13461,59 +10387,6 @@
 23	265	74651	74386
 24	12	105935	105923
 
--- !sql_test_DecimalV2_Varchar_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DecimalV2_Varchar_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DecimalV2_String_0 --
 \N	\N	\N
 1	\N	\N
@@ -13726,59 +10599,6 @@
 23	11	339755	339744
 24	6	480495	480489
 
--- !sql_test_DecimalV2_String_4 --
-\N	\N	\N	\N
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
--- !sql_test_DecimalV2_String_notn_4 --
-1	\N	\N	\N
-2	\N	\N	\N
-3	\N	\N	\N
-4	\N	\N	\N
-5	\N	\N	\N
-6	\N	\N	\N
-7	\N	\N	\N
-8	\N	\N	\N
-9	\N	\N	\N
-10	\N	\N	\N
-11	\N	\N	\N
-12	\N	\N	\N
-13	\N	\N	\N
-14	\N	\N	\N
-15	\N	\N	\N
-16	\N	\N	\N
-17	\N	\N	\N
-18	\N	\N	\N
-19	\N	\N	\N
-20	\N	\N	\N
-21	\N	\N	\N
-22	\N	\N	\N
-23	\N	\N	\N
-24	\N	\N	\N
-
 -- !sql_test_DecimalV2_Date_0 --
 \N	\N	\N
 1	20120325.395000000	-20120276.605000000
@@ -13991,59 +10811,6 @@
 23	515	20120575	20120060
 24	72	20121342	20121270
 
--- !sql_test_DecimalV2_Date_4 --
-\N	\N	\N	\N
-1	8	-3	-11
-2	34	-18	-52
-3	32	-1	-33
-4	64	-12	-76
-5	97	-15	-112
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	8	-3	-11
-14	34	-18	-52
-15	32	-1	-33
-16	64	-12	-76
-17	97	-15	-112
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
--- !sql_test_DecimalV2_Date_notn_4 --
-1	8	-3	-11
-2	34	-18	-52
-3	32	-1	-33
-4	64	-12	-76
-5	97	-15	-112
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	8	-3	-11
-14	34	-18	-52
-15	32	-1	-33
-16	64	-12	-76
-17	97	-15	-112
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
 -- !sql_test_DecimalV2_DateTime_0 --
 \N	\N	\N
 1	20120301010025.395000000	-20120301009976.605000000
@@ -14256,59 +11023,6 @@
 23	259	20120311111531	20120311111272
 24	1032	20120312121182	20120312120150
 
--- !sql_test_DecimalV2_DateTime_4 --
-\N	\N	\N	\N
-1	16	89	73
-2	2	38	36
-3	48	-69	-117
-4	64	116	52
-5	33	101	68
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	16	89	73
-14	2	38	36
-15	48	-69	-117
-16	64	116	52
-17	33	101	68
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
--- !sql_test_DecimalV2_DateTime_notn_4 --
-1	16	89	73
-2	2	38	36
-3	48	-69	-117
-4	64	116	52
-5	33	101	68
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	16	89	73
-14	2	38	36
-15	48	-69	-117
-16	64	116	52
-17	33	101	68
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
 -- !sql_test_DecimalV2_DateV2_0 --
 \N	\N	\N
 1	20120325.395000000	-20120276.605000000
@@ -14521,59 +11235,6 @@
 23	515	20120575	20120060
 24	72	20121342	20121270
 
--- !sql_test_DecimalV2_DateV2_4 --
-\N	\N	\N	\N
-1	8	-3	-11
-2	34	-18	-52
-3	32	-1	-33
-4	64	-12	-76
-5	97	-15	-112
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	8	-3	-11
-14	34	-18	-52
-15	32	-1	-33
-16	64	-12	-76
-17	97	-15	-112
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
--- !sql_test_DecimalV2_DateV2_notn_4 --
-1	8	-3	-11
-2	34	-18	-52
-3	32	-1	-33
-4	64	-12	-76
-5	97	-15	-112
-6	114	-1	-115
-7	115	-1	-116
-8	116	-1	-117
-9	117	-1	-118
-10	118	-1	-119
-11	119	-1	-120
-12	120	-1	-121
-13	8	-3	-11
-14	34	-18	-52
-15	32	-1	-33
-16	64	-12	-76
-17	97	-15	-112
-18	114	-1	-115
-19	115	-1	-116
-20	116	-1	-117
-21	117	-1	-118
-22	118	-1	-119
-23	119	-1	-120
-24	120	-1	-121
-
 -- !sql_test_DecimalV2_DateTimeV2_0 --
 \N	\N	\N
 1	20120301010025.395000000	-20120301009976.605000000
@@ -14786,59 +11447,6 @@
 23	259	20120311111531	20120311111272
 24	1032	20120312121182	20120312120150
 
--- !sql_test_DecimalV2_DateTimeV2_4 --
-\N	\N	\N	\N
-1	16	89	73
-2	2	38	36
-3	48	-69	-117
-4	64	116	52
-5	33	101	68
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	16	89	73
-14	2	38	36
-15	48	-69	-117
-16	64	116	52
-17	33	101	68
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
--- !sql_test_DecimalV2_DateTimeV2_notn_4 --
-1	16	89	73
-2	2	38	36
-3	48	-69	-117
-4	64	116	52
-5	33	101	68
-6	90	-1	-91
-7	15	-1	-16
-8	68	127	59
-9	121	-1	-122
-10	46	-1	-47
-11	99	127	28
-12	24	127	103
-13	16	89	73
-14	2	38	36
-15	48	-69	-117
-16	64	116	52
-17	33	101	68
-18	90	-1	-91
-19	15	-1	-16
-20	68	127	59
-21	121	-1	-122
-22	46	-1	-47
-23	99	127	28
-24	24	127	103
-
 -- !sql_test_DecimalV2_Boolean_0 --
 \N	\N	\N
 1	24.395000000	24.395000000
@@ -15051,59 +11659,6 @@
 23	1	779	778
 24	0	1103	1103
 
--- !sql_test_DecimalV2_Boolean_4 --
-\N	\N	\N	\N
-1	0	24	24
-2	0	34	34
-3	0	48	48
-4	0	68	68
-5	0	97	97
-6	0	127	127
-7	0	127	127
-8	1	127	126
-9	1	127	126
-10	1	127	126
-11	1	127	126
-12	1	127	126
-13	0	24	24
-14	0	34	34
-15	0	48	48
-16	0	68	68
-17	0	97	97
-18	0	127	127
-19	0	127	127
-20	1	127	126
-21	1	127	126
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
--- !sql_test_DecimalV2_Boolean_notn_4 --
-1	0	24	24
-2	0	34	34
-3	0	48	48
-4	0	68	68
-5	0	97	97
-6	0	127	127
-7	0	127	127
-8	1	127	126
-9	1	127	126
-10	1	127	126
-11	1	127	126
-12	1	127	126
-13	0	24	24
-14	0	34	34
-15	0	48	48
-16	0	68	68
-17	0	97	97
-18	0	127	127
-19	0	127	127
-20	1	127	126
-21	1	127	126
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
 -- !sql_test_Decimal32V3_TinyInt_0 --
 \N	\N	\N
 1	13.012	11.012
@@ -15157,59 +11712,6 @@
 23	267.454	245.454
 24	279.565	255.565
 
--- !sql_test_Decimal32V3_TinyInt_1 --
-\N	\N	\N	\N
-1	12.012	12.012	2.000
-2	46.246	11.561	3.000
-3	102.702	11.411	4.000
-4	181.380	11.336	5.000
-5	282.280	11.291	6.000
-6	405.402	11.261	7.000
-7	550.746	11.239	8.000
-8	718.312	11.223	9.000
-9	908.100	11.211	10.000
-10	1120.110	11.201	11.000
-11	1354.342	11.192	12.000
-12	1610.796	11.186	1.000
-13	145.344	145.344	2.000
-14	312.910	78.227	3.000
-15	502.698	55.855	4.000
-16	714.708	44.669	5.000
-17	948.940	37.957	6.000
-18	1205.394	33.483	7.000
-19	1484.070	30.287	8.000
-20	1784.968	27.890	9.000
-21	2108.088	26.025	10.000
-22	2453.430	24.534	11.000
-23	2820.994	23.314	12.000
-24	3210.780	22.297	0.024
-
--- !sql_test_Decimal32V3_TinyInt_notn_1 --
-1	12.012	12.012	51591147159.552
-2	46.246	11.561	147033910434.387
-3	102.702	11.411	242476673708.321
-4	181.380	11.336	337919436982.255
-5	282.280	11.291	433362200256.189
-6	405.402	11.261	528804963530.123
-7	550.746	11.239	624247726804.057
-8	718.312	11.223	719690490077.991
-9	908.100	11.211	815133253351.925
-10	1120.110	11.201	910576016625.859
-11	1354.342	11.192	1006018779899.793
-12	1610.796	11.186	1101461543173.727
-13	145.344	145.344	267.565
-14	312.910	78.227	0.000
-15	502.698	55.855	0.000
-16	714.708	44.669	0.000
-17	948.940	37.957	0.000
-18	1205.394	33.483	0.000
-19	1484.070	30.287	0.000
-20	1784.968	27.890	0.000
-21	2108.088	26.025	0.000
-22	2453.430	24.534	0.000
-23	2820.994	23.314	0.000
-24	3210.780	22.297	0.000
-
 -- !sql_test_Decimal32V3_TinyInt_2 --
 \N	\N
 1	12
@@ -15316,59 +11818,6 @@
 23	0	267	267
 24	8	271	263
 
--- !sql_test_Decimal32V3_TinyInt_4 --
-\N	\N	\N	\N
-1	0	13	13
-2	2	23	21
-3	2	35	33
-4	4	45	41
-5	0	61	61
-6	2	71	69
-7	6	79	73
-8	8	89	81
-9	0	109	109
-10	0	122	122
-11	11	123	112
-12	12	127	115
-13	1	127	126
-14	2	127	125
-15	3	127	124
-16	4	127	123
-17	5	127	122
-18	6	127	121
-19	7	127	120
-20	8	127	119
-21	9	127	118
-22	10	127	117
-23	11	127	116
-24	12	127	115
-
--- !sql_test_Decimal32V3_TinyInt_notn_4 --
-1	0	13	13
-2	2	23	21
-3	2	35	33
-4	4	45	41
-5	0	61	61
-6	2	71	69
-7	6	79	73
-8	8	89	81
-9	0	109	109
-10	0	122	122
-11	11	123	112
-12	12	127	115
-13	1	127	126
-14	2	127	125
-15	3	127	124
-16	4	127	123
-17	5	127	122
-18	6	127	121
-19	7	127	120
-20	8	127	119
-21	9	127	118
-22	10	127	117
-23	11	127	116
-24	12	127	115
-
 -- !sql_test_Decimal32V3_SmallInt_0 --
 \N	\N	\N
 1	22.012	2.012
@@ -15422,59 +11871,6 @@
 23	10496.454	-9983.546
 24	20747.565	-20212.435
 
--- !sql_test_Decimal32V3_SmallInt_1 --
-\N	\N	\N	\N
-1	120.120	1.201	20.000
-2	462.460	1.156	40.000
-3	1369.360	0.855	80.000
-4	3627.600	0.566	160.000
-5	9032.960	0.352	320.000
-6	21621.440	0.211	640.000
-7	50353.920	0.122	1280.000
-8	114929.920	0.070	2560.000
-9	258304.000	0.039	5120.000
-10	573496.320	0.021	10240.000
-11	1260769.280	0.012	20480.000
-12	2749091.840	0.006	10.000
-13	1453.440	14.534	20.000
-14	3129.100	7.822	40.000
-15	6702.640	4.189	80.000
-16	14294.160	2.233	160.000
-17	30366.080	1.186	320.000
-18	64287.680	0.627	640.000
-19	135686.400	0.331	1280.000
-20	285594.880	0.174	2560.000
-21	599633.920	0.091	5120.000
-22	1256156.160	0.047	10240.000
-23	2626088.960	0.025	20480.000
-24	5479731.200	0.013	0.024
-
--- !sql_test_Decimal32V3_SmallInt_notn_1 --
-1	120.120	1.201	51591147159.552
-2	462.460	1.156	147033910434.387
-3	1369.360	0.855	242476673708.321
-4	3627.600	0.566	337919436982.255
-5	9032.960	0.352	433362200256.189
-6	21621.440	0.211	528804963530.123
-7	50353.920	0.122	624247726804.057
-8	114929.920	0.070	719690490077.991
-9	258304.000	0.039	815133253351.925
-10	573496.320	0.021	910576016625.859
-11	1260769.280	0.012	1006018779899.793
-12	2749091.840	0.006	1101461543173.727
-13	1453.440	14.534	267.565
-14	3129.100	7.822	0.000
-15	6702.640	4.189	0.000
-16	14294.160	2.233	0.000
-17	30366.080	1.186	0.000
-18	64287.680	0.627	0.000
-19	135686.400	0.331	0.000
-20	285594.880	0.174	0.000
-21	599633.920	0.091	0.000
-22	1256156.160	0.047	0.000
-23	2626088.960	0.025	0.000
-24	5479731.200	0.013	0.000
-
 -- !sql_test_Decimal32V3_SmallInt_2 --
 \N	\N
 1	1
@@ -15581,59 +11977,6 @@
 23	0	10496	10496
 24	0	20747	20747
 
--- !sql_test_Decimal32V3_SmallInt_4 --
-\N	\N	\N	\N
-1	8	14	6
-2	20	23	3
-3	32	42	10
-4	0	125	125
-5	32	-72	-104
-6	64	67	3
-7	0	-50	-50
-8	0	89	89
-9	0	100	100
-10	0	112	112
-11	0	123	123
-12	0	127	127
-13	10	127	117
-14	20	127	107
-15	40	127	87
-16	80	127	47
-17	32	-1	-33
-18	64	127	63
-19	0	-1	-1
-20	0	127	127
-21	0	127	127
-22	0	127	127
-23	0	127	127
-24	0	127	127
-
--- !sql_test_Decimal32V3_SmallInt_notn_4 --
-1	8	14	6
-2	20	23	3
-3	32	42	10
-4	0	125	125
-5	32	-72	-104
-6	64	67	3
-7	0	-50	-50
-8	0	89	89
-9	0	100	100
-10	0	112	112
-11	0	123	123
-12	0	127	127
-13	10	127	117
-14	20	127	107
-15	40	127	87
-16	80	127	47
-17	32	-1	-33
-18	64	127	63
-19	0	-1	-1
-20	0	127	127
-21	0	127	127
-22	0	127	127
-23	0	127	127
-24	0	127	127
-
 -- !sql_test_Decimal32V3_Integer_0 --
 \N	\N	\N
 1	23807.012	-23782.988
@@ -15687,59 +12030,6 @@
 23	24320301.454	-24319788.546
 24	48640312.565	-48639777.435
 
--- !sql_test_Decimal32V3_Integer_1 --
-\N	\N	\N	\N
-1	285825.540	0.000	47545.000
-2	1099383.035	0.000	95045.000
-3	3253770.530	0.000	190045.000
-4	8617590.525	0.000	380045.000
-5	21455820.520	0.000	760045.000
-6	51353960.515	0.000	1520045.000
-7	119594100.510	0.000	3040045.000
-8	272962600.505	0.000	6080045.000
-9	613476540.500	0.000	12160045.000
-10	1362058800.495	0.000	24320045.000
-11	2994332580.490	0.000	48640045.000
-12	6529099160.485	0.000	23795.000
-13	3458460.480	0.006	47545.000
-14	7438652.975	0.003	95045.000
-15	15926310.470	0.001	190045.000
-16	33956670.465	0.000	380045.000
-17	72127980.460	0.000	760045.000
-18	152692280.455	0.000	1520045.000
-19	322264740.450	0.000	3040045.000
-20	678297880.445	0.000	6080045.000
-21	1424141100.440	0.000	12160045.000
-22	2983381920.435	0.000	24320045.000
-23	6236972820.430	0.000	48640045.000
-24	13014373640.425	0.000	139848849110.592
-
--- !sql_test_Decimal32V3_Integer_notn_1 --
-1	285825.540	0.000	102198746808.320
-2	1099383.035	0.000	408215166695.865
-3	3253770.530	0.000	1632280846198.365
-4	8617590.525	0.000	6528543564208.365
-5	21455820.520	0.000	26113594436248.365
-6	51353960.515	0.000	104453797924408.365
-7	119594100.510	0.000	102198795448.365
-8	272962600.505	0.000	408215166695.865
-9	613476540.500	0.000	1632280846198.365
-10	1362058800.495	0.000	6528543564208.365
-11	2994332580.490	0.000	26113594436248.365
-12	6529099160.485	0.000	104453797924408.365
-13	3458460.480	0.006	48640.045
-14	7438652.975	0.003	0.000
-15	15926310.470	0.001	0.000
-16	33956670.465	0.000	0.000
-17	72127980.460	0.000	0.000
-18	152692280.455	0.000	0.000
-19	322264740.450	0.000	0.000
-20	678297880.445	0.000	0.000
-21	1424141100.440	0.000	0.000
-22	2983381920.435	0.000	0.000
-23	6236972820.430	0.000	0.000
-24	13014373640.425	0.000	0.000
-
 -- !sql_test_Decimal32V3_Integer_2 --
 \N	\N
 1	0
@@ -15846,59 +12136,6 @@
 23	0	24320301	24320301
 24	9	48640303	48640294
 
--- !sql_test_Decimal32V3_Integer_4 --
-\N	\N	\N	\N
-1	0	-1	-1
-2	17	-65	-82
-3	0	103	103
-4	13	125	112
-5	8	-67	-75
-6	65	-17	-82
-7	12	-17	-29
-8	9	125	116
-9	36	109	73
-10	32	125	93
-11	41	127	86
-12	45	127	82
-13	115	-1	-116
-14	57	-1	-58
-15	69	127	58
-16	93	127	34
-17	13	-1	-14
-18	109	-1	-110
-19	45	-1	-46
-20	45	127	82
-21	45	127	82
-22	45	127	82
-23	45	127	82
-24	45	127	82
-
--- !sql_test_Decimal32V3_Integer_notn_4 --
-1	0	-1	-1
-2	17	-65	-82
-3	0	103	103
-4	13	125	112
-5	8	-67	-75
-6	65	-17	-82
-7	12	-17	-29
-8	9	125	116
-9	36	109	73
-10	32	125	93
-11	41	127	86
-12	45	127	82
-13	115	-1	-116
-14	57	-1	-58
-15	69	127	58
-16	93	127	34
-17	13	-1	-14
-18	109	-1	-110
-19	45	-1	-46
-20	45	127	82
-21	45	127	82
-22	45	127	82
-23	45	127	82
-24	45	127	82
-
 -- !sql_test_Decimal32V3_BigInt_0 --
 \N	\N	\N
 1	5354541.012	-5354516.988
@@ -15952,59 +12189,6 @@
 23	5472011035.454	-5472010522.546
 24	10944011046.565	-10944010511.435
 
--- !sql_test_Decimal32V3_BigInt_1 --
-\N	\N	\N	\N
-1	64318602.348	0.000	10698302.123
-2	247376305.317	0.000	21385813.234
-3	732120758.286	0.000	42760824.345
-4	1938987523.755	0.000	85510835.456
-5	4827596539.224	0.000	171010846.567
-6	11554685304.693	0.000	342010857.678
-7	26908724070.162	0.000	684010868.789
-8	61416643835.631	0.000	1368010879.900
-9	138032287601.100	0.000	2736010891.011
-10	306463303366.569	0.000	5472010902.122
-11	673724911132.038	0.000	10944010913.233
-12	1469047398897.507	0.000	5354674.344
-13	778248662.976	0.000	10698435.455
-14	1673799240.945	0.000	21385946.566
-15	3583529443.914	0.000	42760957.677
-16	7640367709.383	0.000	85510968.788
-17	16228919724.852	0.000	171010979.899
-18	34355894490.321	0.000	342010991.010
-19	72509705255.790	0.000	684011002.121
-20	152617169021.259	0.000	1368011013.232
-21	320431900786.728	0.000	2736011024.343
-22	671261092552.197	0.000	5472011035.454
-23	1403319052317.666	0.000	10944011046.565
-24	2928234244083.135	0.000	1742613662948936925785770690644.576
-
--- !sql_test_Decimal32V3_BigInt_notn_1 --
-1	64318602.348	0.000	0.000
-2	247376305.317	0.000	64318602.348
-3	732120758.286	0.000	247376305.317
-4	1938987523.755	0.000	732120758.286
-5	4827596539.224	0.000	1938987523.755
-6	11554685304.693	0.000	4827596539.224
-7	26908724070.162	0.000	11554685304.693
-8	61416643835.631	0.000	26908724070.162
-9	138032287601.100	0.000	61416643835.631
-10	306463303366.569	0.000	138032287601.100
-11	673724911132.038	0.000	306463303366.569
-12	1469047398897.507	0.000	673724911132.038
-13	778248662.976	0.000	1469047398897.507
-14	1673799240.945	0.000	778248662.976
-15	3583529443.914	0.000	1673799240.945
-16	7640367709.383	0.000	3583529443.914
-17	16228919724.852	0.000	7640367709.383
-18	34355894490.321	0.000	16228919724.852
-19	72509705255.790	0.000	34355894490.321
-20	152617169021.259	0.000	72509705255.790
-21	320431900786.728	0.000	152617169021.259
-22	671261092552.197	0.000	320431900786.728
-23	1403319052317.666	0.000	671261092552.197
-24	2928234244083.135	0.000	1403319052317.666
-
 -- !sql_test_Decimal32V3_BigInt_2 --
 \N	\N
 1	0
@@ -16111,164 +12295,58 @@
 23	0	5472011035	5472011035
 24	11	10944011035	10944011024
 
--- !sql_test_Decimal32V3_BigInt_4 --
-\N	\N	\N	\N
-1	0	45	45
-2	7	55	48
-3	34	51	17
-4	9	111	102
-5	56	123	67
-6	67	-37	-104
-7	10	-33	-43
-8	25	91	66
-9	0	127	127
-10	16	123	107
-11	27	123	96
-12	27	127	100
-13	33	127	94
-14	39	127	88
-15	51	127	76
-16	75	127	52
-17	123	127	4
-18	91	-1	-92
-19	27	-1	-28
-20	27	127	100
-21	27	127	100
-22	27	127	100
-23	27	127	100
-24	27	127	100
-
--- !sql_test_Decimal32V3_BigInt_notn_4 --
-1	0	45	45
-2	7	55	48
-3	34	51	17
-4	9	111	102
-5	56	123	67
-6	67	-37	-104
-7	10	-33	-43
-8	25	91	66
-9	0	127	127
-10	16	123	107
-11	27	123	96
-12	27	127	100
-13	33	127	94
-14	39	127	88
-15	51	127	76
-16	75	127	52
-17	123	127	4
-18	91	-1	-92
-19	27	-1	-28
-20	27	127	100
-21	27	127	100
-22	27	127	100
-23	27	127	100
-24	27	127	100
-
 -- !sql_test_Decimal32V3_LargeInt_0 --
 \N	\N	\N
-1	107090657.012	-107090632.988
-2	213965668.123	-213965621.877
-3	427715679.234	-427715610.766
-4	855215690.345	-855215599.655
-5	1710215701.456	-1710215588.544
-6	3420215712.567	-3420215577.433
-7	6840215723.678	-6840215566.322
-8	13680215734.789	-13680215555.211
-9	27360215745.900	-27360215544.100
-10	54720215757.011	-54720215532.989
-11	109440215768.122	-109440215521.878
-12	218880215779.233	-218880215510.767
-13	107090790.344	-107090499.656
-14	213965801.455	-213965488.545
-15	427715812.566	-427715477.434
-16	855215823.677	-855215466.323
-17	1710215834.788	-1710215455.212
-18	3420215845.899	-3420215444.101
-19	6840215857.010	-6840215432.990
-20	13680215868.121	-13680215421.879
-21	27360215879.232	-27360215410.768
-22	54720215890.343	-54720215399.657
-23	109440215901.454	-109440215388.546
-24	218880215912.565	-218880215377.435
+1	1.07090657012E8	-1.07090632988E8
+2	2.13965668123E8	-2.13965621877E8
+3	4.27715679234E8	-4.27715610766E8
+4	8.55215690345E8	-8.55215599655E8
+5	1.710215701456E9	-1.710215588544E9
+6	3.420215712567E9	-3.420215577433E9
+7	6.840215723678E9	-6.840215566322E9
+8	1.3680215734789E10	-1.3680215555211E10
+9	2.73602157459E10	-2.73602155441E10
+10	5.4720215757011E10	-5.4720215532989E10
+11	1.09440215768122E11	-1.09440215521878E11
+12	2.18880215779233E11	-2.18880215510767E11
+13	1.07090790344E8	-1.07090499656E8
+14	2.13965801455E8	-2.13965488545E8
+15	4.27715812566E8	-4.27715477434E8
+16	8.55215823677E8	-8.55215466323E8
+17	1.710215834788E9	-1.710215455212E9
+18	3.420215845899E9	-3.420215444101E9
+19	6.84021585701E9	-6.84021543299E9
+20	1.3680215868121E10	-1.3680215421879E10
+21	2.7360215879232E10	-2.7360215410768E10
+22	5.4720215890343E10	-5.4720215399657E10
+23	1.09440215901454E11	-1.09440215388546E11
+24	2.18880215912565E11	-2.18880215377435E11
 
 -- !sql_test_Decimal32V3_LargeInt_notn_0 --
-1	107090657.012	-107090632.988
-2	213965668.123	-213965621.877
-3	427715679.234	-427715610.766
-4	855215690.345	-855215599.655
-5	1710215701.456	-1710215588.544
-6	3420215712.567	-3420215577.433
-7	6840215723.678	-6840215566.322
-8	13680215734.789	-13680215555.211
-9	27360215745.900	-27360215544.100
-10	54720215757.011	-54720215532.989
-11	109440215768.122	-109440215521.878
-12	218880215779.233	-218880215510.767
-13	107090790.344	-107090499.656
-14	213965801.455	-213965488.545
-15	427715812.566	-427715477.434
-16	855215823.677	-855215466.323
-17	1710215834.788	-1710215455.212
-18	3420215845.899	-3420215444.101
-19	6840215857.010	-6840215432.990
-20	13680215868.121	-13680215421.879
-21	27360215879.232	-27360215410.768
-22	54720215890.343	-54720215399.657
-23	109440215901.454	-109440215388.546
-24	218880215912.565	-218880215377.435
-
--- !sql_test_Decimal32V3_LargeInt_1 --
-\N	\N	\N	\N
-1	1286372827.740	0.000	213965.645
-2	4947527609.335	0.000	427715.645
-3	14642417390.930	0.000	855215.645
-4	38779753422.525	0.000	1710215.645
-5	96551934454.120	0.000	3420215.645
-6	231093710485.715	0.000	6840215.645
-7	538174486517.310	0.000	13680215.645
-8	1228332882548.905	0.000	27360215.645
-9	2760645758580.500	0.000	54720215.645
-10	6129266074612.095	0.000	109440215.645
-11	13474498230643.690	0.000	218880215.645
-12	29380947986675.285	0.000	107090.645
-13	15564982706.880	0.000	213965.645
-14	33475994988.475	0.000	427715.645
-15	71670599770.070	0.000	855215.645
-16	152807365801.665	0.000	1710215.645
-17	324578406833.260	0.000	3420215.645
-18	687117902864.855	0.000	6840215.645
-19	1450194118896.450	0.000	13680215.645
-20	3052343394928.045	0.000	27360215.645
-21	6408638030959.640	0.000	54720215.645
-22	13425221866991.235	0.000	109440215.645
-23	28066381063022.830	0.000	218880215.645
-24	58564684899054.425	0.000	218880215645.000
-
--- !sql_test_Decimal32V3_LargeInt_notn_1 --
-1	1286372827.740	0.000	0.000
-2	4947527609.335	0.000	107090.645
-3	14642417390.930	0.000	213965.645
-4	38779753422.525	0.000	427715.645
-5	96551934454.120	0.000	855215.645
-6	231093710485.715	0.000	1710215.645
-7	538174486517.310	0.000	3420215.645
-8	1228332882548.905	0.000	6840215.645
-9	2760645758580.500	0.000	13680215.645
-10	6129266074612.095	0.000	27360215.645
-11	13474498230643.690	0.000	54720215.645
-12	29380947986675.285	0.000	109440215.645
-13	15564982706.880	0.000	218880215.645
-14	33475994988.475	0.000	107090.645
-15	71670599770.070	0.000	213965.645
-16	152807365801.665	0.000	427715.645
-17	324578406833.260	0.000	855215.645
-18	687117902864.855	0.000	1710215.645
-19	1450194118896.450	0.000	3420215.645
-20	3052343394928.045	0.000	6840215.645
-21	6408638030959.640	0.000	13680215.645
-22	13425221866991.235	0.000	27360215.645
-23	28066381063022.830	0.000	54720215.645
-24	58564684899054.425	0.000	109440215.645
+1	1.07090657012E8	-1.07090632988E8
+2	2.13965668123E8	-2.13965621877E8
+3	4.27715679234E8	-4.27715610766E8
+4	8.55215690345E8	-8.55215599655E8
+5	1.710215701456E9	-1.710215588544E9
+6	3.420215712567E9	-3.420215577433E9
+7	6.840215723678E9	-6.840215566322E9
+8	1.3680215734789E10	-1.3680215555211E10
+9	2.73602157459E10	-2.73602155441E10
+10	5.4720215757011E10	-5.4720215532989E10
+11	1.09440215768122E11	-1.09440215521878E11
+12	2.18880215779233E11	-2.18880215510767E11
+13	1.07090790344E8	-1.07090499656E8
+14	2.13965801455E8	-2.13965488545E8
+15	4.27715812566E8	-4.27715477434E8
+16	8.55215823677E8	-8.55215466323E8
+17	1.710215834788E9	-1.710215455212E9
+18	3.420215845899E9	-3.420215444101E9
+19	6.84021585701E9	-6.84021543299E9
+20	1.3680215868121E10	-1.3680215421879E10
+21	2.7360215879232E10	-2.7360215410768E10
+22	5.4720215890343E10	-5.4720215399657E10
+23	1.09440215901454E11	-1.09440215388546E11
+24	2.18880215912565E11	-2.18880215377435E11
 
 -- !sql_test_Decimal32V3_LargeInt_2 --
 \N	\N
@@ -16376,164 +12454,58 @@
 23	0	109440215901	109440215901
 24	9	218880215903	218880215894
 
--- !sql_test_Decimal32V3_LargeInt_4 --
-\N	\N	\N	\N
-1	4	-35	-39
-2	5	95	90
-3	32	63	31
-4	13	61	48
-5	24	-3	-27
-6	65	95	30
-7	76	95	19
-8	89	93	4
-9	68	125	57
-10	80	125	45
-11	89	127	38
-12	93	127	34
-13	85	-1	-86
-14	77	127	50
-15	61	127	66
-16	29	127	98
-17	93	-1	-94
-18	93	127	34
-19	93	127	34
-20	93	127	34
-21	93	127	34
-22	93	127	34
-23	93	127	34
-24	93	127	34
-
--- !sql_test_Decimal32V3_LargeInt_notn_4 --
-1	4	-35	-39
-2	5	95	90
-3	32	63	31
-4	13	61	48
-5	24	-3	-27
-6	65	95	30
-7	76	95	19
-8	89	93	4
-9	68	125	57
-10	80	125	45
-11	89	127	38
-12	93	127	34
-13	85	-1	-86
-14	77	127	50
-15	61	127	66
-16	29	127	98
-17	93	-1	-94
-18	93	127	34
-19	93	127	34
-20	93	127	34
-21	93	127	34
-22	93	127	34
-23	93	127	34
-24	93	127	34
-
 -- !sql_test_Decimal32V3_Float_0 --
 \N	\N	\N
-1	12.112000001490117	11.911999998509884
-2	23.323000002980233	22.92299999701977
-3	34.53400001192093	33.93399998807907
-4	45.74500000596046	44.944999994039534
-5	56.956	55.956
-6	68.16700002384185	66.96699997615814
-7	79.37799998807907	77.97800001192093
-8	90.58900001192093	88.98899998807907
-9	101.79999997615815	100.00000002384186
-10	113.011	111.011
-11	124.22200002384186	122.02199997615814
-12	135.43300004768372	133.0329999523163
-13	145.4440000014901	145.24399999850988
-14	156.65500000298024	156.25499999701978
-15	167.86600001192093	167.26599998807907
-16	179.07700000596046	178.27699999403953
-17	190.288	189.288
-18	201.49900002384186	200.29899997615814
-19	212.70999998807906	211.31000001192092
-20	223.92100001192094	222.32099998807908
-21	235.13199997615814	233.33200002384186
-22	246.343	244.343
-23	257.55400002384187	255.35399997615815
-24	268.7650000476837	266.3649999523163
+1	12.1120000	11.9120000
+2	23.3230000	22.9230000
+3	34.5340000	33.9340000
+4	45.7450000	44.9450000
+5	56.9560000	55.9560000
+6	68.1670000	66.9670000
+7	79.3780000	77.9780000
+8	90.5890000	88.9890000
+9	101.8000000	100.0000000
+10	113.0110000	111.0110000
+11	124.2220000	122.0220000
+12	135.4330000	133.0330000
+13	145.4440000	145.2440000
+14	156.6550000	156.2550000
+15	167.8660000	167.2660000
+16	179.0770000	178.2770000
+17	190.2880000	189.2880000
+18	201.4990000	200.2990000
+19	212.7100000	211.3100000
+20	223.9210000	222.3210000
+21	235.1320000	233.3320000
+22	246.3430000	244.3430000
+23	257.5540000	255.3540000
+24	268.7650000	266.3650000
 
 -- !sql_test_Decimal32V3_Float_notn_0 --
-1	12.112000001490117	11.911999998509884
-2	23.323000002980233	22.92299999701977
-3	34.53400001192093	33.93399998807907
-4	45.74500000596046	44.944999994039534
-5	56.956	55.956
-6	68.16700002384185	66.96699997615814
-7	79.37799998807907	77.97800001192093
-8	90.58900001192093	88.98899998807907
-9	101.79999997615815	100.00000002384186
-10	113.011	111.011
-11	124.22200002384186	122.02199997615814
-12	135.43300004768372	133.0329999523163
-13	145.4440000014901	145.24399999850988
-14	156.65500000298024	156.25499999701978
-15	167.86600001192093	167.26599998807907
-16	179.07700000596046	178.27699999403953
-17	190.288	189.288
-18	201.49900002384186	200.29899997615814
-19	212.70999998807906	211.31000001192092
-20	223.92100001192094	222.32099998807908
-21	235.13199997615814	233.33200002384186
-22	246.343	244.343
-23	257.55400002384187	255.35399997615815
-24	268.7650000476837	266.3649999523163
-
--- !sql_test_Decimal32V3_Float_1 --
-\N	\N	\N	\N
-1	1.2012000178992748	120.11999821007255	0.011999821186066129
-2	4.62460006891191	115.61499827720228	0.12299965727329365
-3	10.270200408101083	114.11332879887705	0.03399864101410088
-4	18.13800027027726	113.36249831076714	0.1449993264675129
-5	28.228	112.912	0.45600000000000307
-6	40.54020161092281	112.61166219188124	0.3669973297119071
-7	55.07459906208515	112.39714477125482	0.27800133514404024
-8	71.8312010703683	112.23624832754957	0.1889986648559585
-9	90.80999759435655	112.11111408104139	0.10000267028809162
-10	112.011	112.011	0.01099999999999568
-11	135.43420293545722	111.92908848309325	1.0219973535537719
-12	161.07960640072824	111.86082888838335	1.032994707107548
-13	14.534400216579437	1453.4399783420565	0.04399783486127262
-14	31.291000466272237	782.2749883431943	0.05499766945840179
-15	50.26980199754238	558.5533111384188	0.16599334812164557
-16	71.47080106499791	446.69249334376315	0.2769973416328355
-17	94.894	379.576	0.2880000000000109
-18	120.53940478980542	334.8316533616522	0.4989920368194589
-19	148.40699747264384	302.87143372929836	0.6100036001205353
-20	178.4968026598096	278.9012458440476	0.7209966859817598
-21	210.80879441547393	260.25778467225456	0.23200619888305596
-22	245.343	245.343	0.3429999999999893
-23	282.09940611433984	233.1399949468267	0.15399444484711466
-24	321.0780127584934	222.9708244732688	1.1649894142150856
-
--- !sql_test_Decimal32V3_Float_notn_1 --
-1	1.2012000178992748	120.11999821007255	0.011999821186066129
-2	4.62460006891191	115.61499827720228	0.12299965727329365
-3	10.270200408101083	114.11332879887705	0.03399864101410088
-4	18.13800027027726	113.36249831076714	0.1449993264675129
-5	28.228	112.912	0.45600000000000307
-6	40.54020161092281	112.61166219188124	0.3669973297119071
-7	55.07459906208515	112.39714477125482	0.27800133514404024
-8	71.8312010703683	112.23624832754957	0.1889986648559585
-9	90.80999759435655	112.11111408104139	0.10000267028809162
-10	112.011	112.011	0.01099999999999568
-11	135.43420293545722	111.92908848309325	1.0219973535537719
-12	161.07960640072824	111.86082888838335	1.032994707107548
-13	14.534400216579437	1453.4399783420565	0.04399783486127262
-14	31.291000466272237	782.2749883431943	0.05499766945840179
-15	50.26980199754238	558.5533111384188	0.16599334812164557
-16	71.47080106499791	446.69249334376315	0.2769973416328355
-17	94.894	379.576	0.2880000000000109
-18	120.53940478980542	334.8316533616522	0.4989920368194589
-19	148.40699747264384	302.87143372929836	0.6100036001205353
-20	178.4968026598096	278.9012458440476	0.7209966859817598
-21	210.80879441547393	260.25778467225456	0.23200619888305596
-22	245.343	245.343	0.3429999999999893
-23	282.09940611433984	233.1399949468267	0.15399444484711466
-24	321.0780127584934	222.9708244732688	1.1649894142150856
+1	12.1120000	11.9120000
+2	23.3230000	22.9230000
+3	34.5340000	33.9340000
+4	45.7450000	44.9450000
+5	56.9560000	55.9560000
+6	68.1670000	66.9670000
+7	79.3780000	77.9780000
+8	90.5890000	88.9890000
+9	101.8000000	100.0000000
+10	113.0110000	111.0110000
+11	124.2220000	122.0220000
+12	135.4330000	133.0330000
+13	145.4440000	145.2440000
+14	156.6550000	156.2550000
+15	167.8660000	167.2660000
+16	179.0770000	178.2770000
+17	190.2880000	189.2880000
+18	201.4990000	200.2990000
+19	212.7100000	211.3100000
+20	223.9210000	222.3210000
+21	235.1320000	233.3320000
+22	246.3430000	244.3430000
+23	257.5540000	255.3540000
+24	268.7650000	266.3650000
 
 -- !sql_test_Decimal32V3_Float_2 --
 \N	\N
@@ -16641,59 +12613,6 @@
 23	0	257	257
 24	1	267	266
 
--- !sql_test_Decimal32V3_Float_4 --
-\N	\N	\N	\N
-1	0	12	12
-2	0	23	23
-3	0	34	34
-4	0	45	45
-5	0	56	56
-6	0	67	67
-7	0	78	78
-8	0	89	89
-9	0	100	100
-10	0	113	113
-11	1	123	122
-12	1	127	126
-13	0	127	127
-14	0	127	127
-15	0	127	127
-16	0	127	127
-17	0	127	127
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
--- !sql_test_Decimal32V3_Float_notn_4 --
-1	0	12	12
-2	0	23	23
-3	0	34	34
-4	0	45	45
-5	0	56	56
-6	0	67	67
-7	0	78	78
-8	0	89	89
-9	0	100	100
-10	0	113	113
-11	1	123	122
-12	1	127	126
-13	0	127	127
-14	0	127	127
-15	0	127	127
-16	0	127	127
-17	0	127	127
-18	0	127	127
-19	0	127	127
-20	0	127	127
-21	0	127	127
-22	1	127	126
-23	1	127	126
-24	1	127	126
-
 -- !sql_test_Decimal32V3_Double_0 --
 \N	\N	\N
 1	12.5364	11.4876
@@ -16906,59 +12825,6 @@
 23	0	272	272
 24	2	287	285
 
--- !sql_test_Decimal32V3_Double_4 --
-\N	\N	\N	\N
-1	0	12	12
-2	0	23	23
-3	0	35	35
-4	1	45	44
-5	0	58	58
-6	2	67	65
-7	4	78	74
-8	1	93	92
-9	0	108	108
-10	0	123	123
-11	16	123	107
-12	22	127	105
-13	0	127	127
-14	0	127	127
-15	1	127	126
-16	1	127	126
-17	2	127	125
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
--- !sql_test_Decimal32V3_Double_notn_4 --
-1	0	12	12
-2	0	23	23
-3	0	35	35
-4	1	45	44
-5	0	58	58
-6	2	67	65
-7	4	78	74
-8	1	93	92
-9	0	108	108
-10	0	123	123
-11	16	123	107
-12	22	127	105
-13	0	127	127
-14	0	127	127
-15	1	127	126
-16	1	127	126
-17	2	127	125
-18	2	127	125
-19	4	127	123
-20	5	127	122
-21	8	127	119
-22	11	127	116
-23	16	127	111
-24	22	127	105
-
 -- !sql_test_Decimal32V3_DecimalV2_0 --
 \N	\N	\N
 1	36.407	-12.383
@@ -17012,59 +12878,6 @@
 23	1036.362	-523.454
 24	1370.522	-835.392
 
--- !sql_test_Decimal32V3_DecimalV2_1 --
-\N	\N	\N	\N
-1	293.032740	0.492	12.012
-2	797.373532	0.670	23.123
-3	1669.112904	0.702	34.234
-4	3126.220335	0.657	45.345
-5	5504.121264	0.579	56.456
-6	9315.732558	0.490	67.567
-7	15340.636440	0.403	78.678
-8	24758.508649	0.325	89.789
-9	39346.459500	0.258	100.900
-10	61771.714269	0.203	112.011
-11	96023.832776	0.157	123.122
-12	148053.226981	0.121	134.233
-13	3545.666880	5.957	145.344
-14	5395.194220	4.537	156.455
-15	8169.847896	3.436	167.566
-16	12318.528411	2.591	178.677
-17	18503.191272	1.946	189.788
-18	27698.748726	1.457	200.899
-19	41337.709800	1.087	212.010
-20	61523.607661	0.809	223.121
-21	91339.939560	0.600	234.232
-22	135301.512297	0.444	245.343
-23	200010.526232	0.328	256.454
-24	295112.689705	0.242	267.565
-
--- !sql_test_Decimal32V3_DecimalV2_notn_1 --
-1	293.032740	0.492	-374022.048
-2	797.373532	0.670	21.994
-3	1669.112904	0.702	0.029
-4	3126.220335	0.657	0.000
-5	5504.121264	0.579	0.009
-6	9315.732558	0.490	0.003
-7	15340.636440	0.403	486408.870
-8	24758.508649	0.325	32.559
-9	39346.459500	0.258	0.000
-10	61771.714269	0.203	0.000
-11	96023.832776	0.157	0.000
-12	148053.226981	0.121	0.000
-13	3545.666880	5.957	0.000
-14	5395.194220	4.537	0.000
-15	8169.847896	3.436	0.000
-16	12318.528411	2.591	0.000
-17	18503.191272	1.946	0.000
-18	27698.748726	1.457	32.561
-19	41337.709800	1.087	0.000
-20	61523.607661	0.809	0.000
-21	91339.939560	0.600	0.000
-22	135301.512297	0.444	0.000
-23	200010.526232	0.328	0.000
-24	295112.689705	0.242	0.000
-
 -- !sql_test_Decimal32V3_DecimalV2_2 --
 \N	\N
 1	0
@@ -17171,59 +12984,6 @@
 23	256	779	523
 24	10	1359	1349
 
--- !sql_test_Decimal32V3_DecimalV2_4 --
-\N	\N	\N	\N
-1	8	28	20
-2	2	55	53
-3	32	50	18
-4	4	109	105
-5	32	121	89
-6	67	127	60
-7	78	127	49
-8	89	127	38
-9	100	127	27
-10	112	127	15
-11	123	127	4
-12	127	127	0
-13	24	127	103
-14	34	127	93
-15	48	127	79
-16	68	127	59
-17	97	127	30
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
--- !sql_test_Decimal32V3_DecimalV2_notn_4 --
-1	8	28	20
-2	2	55	53
-3	32	50	18
-4	4	109	105
-5	32	121	89
-6	67	127	60
-7	78	127	49
-8	89	127	38
-9	100	127	27
-10	112	127	15
-11	123	127	4
-12	127	127	0
-13	24	127	103
-14	34	127	93
-15	48	127	79
-16	68	127	59
-17	97	127	30
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
 -- !sql_test_Decimal32V3_Decimal32V3_0 --
 \N	\N	\N
 1	24.024	0.000
@@ -17277,59 +13037,6 @@
 23	512.908	0.000
 24	535.130	0.000
 
--- !sql_test_Decimal32V3_Decimal32V3_1 --
-\N	\N	\N	\N
-1	144.288144	1.000	32.561
-2	534.673129	1.000	-477498.944
-3	1171.966756	1.000	21.994
-4	2056.169025	1.000	291677.440
-5	3187.279936	1.000	32.561
-6	4565.299489	1.000	-477506.112
-7	6190.227684	1.000	21.994
-8	8062.064521	1.000	-985816.320
-9	10180.810000	1.000	32.558
-10	12546.464121	1.000	-529711.200
-11	15159.026884	1.000	21.994
-12	18018.498289	1.000	292723.712
-13	21124.878336	1.000	32.561
-14	24478.167025	1.000	-477498.944
-15	28078.364356	1.000	21.994
-16	31925.470329	1.000	291678.352
-17	36019.484944	1.000	32.561
-18	40360.408201	1.000	-477506.112
-19	44948.240100	1.000	21.994
-20	49782.980641	1.000	-131607.552
-21	54864.629824	1.000	32.558
-22	60193.187649	1.000	-477506.224
-23	65768.654116	1.000	21.994
-24	71591.029225	1.000	291680.656
-
--- !sql_test_Decimal32V3_Decimal32V3_notn_1 --
-1	144.288144	1.000	-374022.048
-2	534.673129	1.000	21.994
-3	1171.966756	1.000	0.029
-4	2056.169025	1.000	0.000
-5	3187.279936	1.000	0.005
-6	4565.299489	1.000	0.003
-7	6190.227684	1.000	0.038
-8	8062.064521	1.000	0.008
-9	10180.810000	1.000	0.000
-10	12546.464121	1.000	0.000
-11	15159.026884	1.000	0.000
-12	18018.498289	1.000	0.000
-13	21124.878336	1.000	0.000
-14	24478.167025	1.000	0.000
-15	28078.364356	1.000	0.000
-16	31925.470329	1.000	0.000
-17	36019.484944	1.000	0.000
-18	40360.408201	1.000	0.006
-19	44948.240100	1.000	0.000
-20	49782.980641	1.000	0.000
-21	54864.629824	1.000	0.000
-22	60193.187649	1.000	0.010
-23	65768.654116	1.000	0.000
-24	71591.029225	1.000	0.000
-
 -- !sql_test_Decimal32V3_Decimal32V3_2 --
 \N	\N
 1	1
@@ -17436,59 +13143,6 @@
 23	256	256	0
 24	267	267	0
 
--- !sql_test_Decimal32V3_Decimal32V3_4 --
-\N	\N	\N	\N
-1	12	12	0
-2	23	23	0
-3	34	34	0
-4	45	45	0
-5	56	56	0
-6	67	67	0
-7	78	78	0
-8	89	89	0
-9	100	100	0
-10	112	112	0
-11	123	123	0
-12	127	127	0
-13	127	127	0
-14	127	127	0
-15	127	127	0
-16	127	127	0
-17	127	127	0
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
--- !sql_test_Decimal32V3_Decimal32V3_notn_4 --
-1	12	12	0
-2	23	23	0
-3	34	34	0
-4	45	45	0
-5	56	56	0
-6	67	67	0
-7	78	78	0
-8	89	89	0
-9	100	100	0
-10	112	112	0
-11	123	123	0
-12	127	127	0
-13	127	127	0
-14	127	127	0
-15	127	127	0
-16	127	127	0
-17	127	127	0
-18	127	127	0
-19	127	127	0
-20	127	127	0
-21	127	127	0
-22	127	127	0
-23	127	127	0
-24	127	127	0
-
 -- !sql_test_Decimal32V3_Decimal64V3_0 --
 \N	\N	\N
 1	1246.02434	-1222.00034
@@ -17542,59 +13196,6 @@
 23	25934.91076	-25422.00276
 24	27057.13287	-26522.00287
 
--- !sql_test_Decimal32V3_Decimal64V3_1 --
-\N	\N	\N	\N
-1	14822.95622808	0.009	-2322.00045
-2	54226.28953435	0.009	-3422.00056
-3	118320.73392704	0.009	-4522.00067
-4	207106.28940615	0.009	-5622.00078
-5	320582.95597168	0.009	-6722.00089
-6	458750.73362363	0.009	-7822.00100
-7	621609.62236200	0.009	-8922.00111
-8	809159.62218679	0.009	-10022.00122
-9	1021400.73309800	0.009	-11122.00133
-10	1258332.95509563	0.009	-12222.00144
-11	1519956.28817968	0.009	-13322.00155
-12	1806270.73235015	0.009	-14422.00166
-13	2117276.28760704	0.009	-15522.00177
-14	2452972.95395035	0.009	-16622.00188
-15	2813360.73138008	0.009	-17722.00199
-16	3198439.61989623	0.009	-18822.00210
-17	3608209.61949880	0.009	-19922.00221
-18	4042670.73018779	0.009	-21022.00232
-19	4501822.95196320	0.009	-22122.00243
-20	4985666.28482503	0.009	-23222.00254
-21	5494200.72877328	0.009	-24322.00265
-22	6027426.28380795	0.009	-25422.00276
-23	6585342.94992904	0.009	-26522.00287
-24	7167950.72713655	0.009	0.00000
-
--- !sql_test_Decimal32V3_Decimal64V3_notn_1 --
-1	14822.95622808	0.009	515911471.59552
-2	54226.28953435	0.009	1470339104.34387
-3	118320.73392704	0.009	2424766737.08321
-4	207106.28940615	0.009	3379194369.82255
-5	320582.95597168	0.009	4333622002.56189
-6	458750.73362363	0.009	5288049635.30123
-7	621609.62236200	0.009	6242477268.04057
-8	809159.62218679	0.009	7196904900.77991
-9	1021400.73309800	0.009	8151332533.51925
-10	1258332.95509563	0.009	9105760166.25859
-11	1519956.28817968	0.009	10060187798.99793
-12	1806270.73235015	0.009	11014615431.73727
-13	2117276.28760704	0.009	2.67565
-14	2452972.95395035	0.009	0.00000
-15	2813360.73138008	0.009	0.00000
-16	3198439.61989623	0.009	0.00000
-17	3608209.61949880	0.009	0.00000
-18	4042670.73018779	0.009	0.00000
-19	4501822.95196320	0.009	0.00000
-20	4985666.28482503	0.009	0.00000
-21	5494200.72877328	0.009	0.00000
-22	6027426.28380795	0.009	0.00000
-23	6585342.94992904	0.009	0.00000
-24	7167950.72713655	0.009	0.00000
-
 -- !sql_test_Decimal32V3_Decimal64V3_2 --
 \N	\N
 1	0
@@ -17701,59 +13302,6 @@
 23	0	25934	25934
 24	1	27055	27054
 
--- !sql_test_Decimal32V3_Decimal64V3_4 --
-\N	\N	\N	\N
-1	12	127	115
-2	23	127	104
-3	34	127	93
-4	45	127	82
-5	56	127	71
-6	67	127	60
-7	78	127	49
-8	89	127	38
-9	100	127	27
-10	112	127	15
-11	123	127	4
-12	127	127	0
... 16540 lines suppressed ...


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


[doris] 20/36: [fix](multi-catalog)fix old s3 properties check (#18430)

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

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

commit 47b652b13d35766787724adb3e0613017a769e9d
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Tue Apr 18 09:58:13 2023 +0800

    [fix](multi-catalog)fix old s3 properties check  (#18430)
    
    fix old s3 properties check
    fix for #18005 (comment)
---
 .../java/org/apache/doris/backup/Repository.java   |   3 +
 .../java/org/apache/doris/backup/S3Storage.java    |   7 -
 .../java/org/apache/doris/catalog/S3Resource.java  |   5 +-
 .../datasource/property/PropertyConverter.java     |   8 +-
 .../property/constants/GlueProperties.java         |  10 +
 .../property/constants/S3Properties.java           |  15 +-
 .../ExternalFileTableValuedFunction.java           |   3 +
 .../doris/tablefunction/S3TableValuedFunction.java |  16 +-
 .../datasource/property/PropertyConverterTest.java | 360 +++++++++++++++++++++
 .../apache/doris/utframe/TestWithFeService.java    |   5 +
 10 files changed, 414 insertions(+), 18 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java b/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java
index ba25f892c0..f46bd7b94d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java
@@ -205,6 +205,9 @@ public class Repository implements Writable {
 
     // create repository dir and repo info file
     public Status initRepository() {
+        if (FeConstants.runningUnitTest) {
+            return Status.OK;
+        }
         String repoInfoFilePath = assembleRepoInfoFilePath();
         // check if the repo is already exist in remote
         List<RemoteFile> remoteFiles = Lists.newArrayList();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/S3Storage.java b/fe/fe-core/src/main/java/org/apache/doris/backup/S3Storage.java
index 76470b2e41..39b62458da 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/S3Storage.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/S3Storage.java
@@ -97,11 +97,6 @@ public class S3Storage extends BlobStorage {
     public void setProperties(Map<String, String> properties) {
         super.setProperties(properties);
         caseInsensitiveProperties.putAll(properties);
-        if (!caseInsensitiveProperties.containsKey(S3Properties.ENDPOINT)) {
-            // try to get new properties from old version
-            // compatible with old version
-            S3Properties.convertToStdProperties(caseInsensitiveProperties);
-        }
         try {
             S3Properties.requiredS3Properties(caseInsensitiveProperties);
         } catch (DdlException e) {
@@ -136,7 +131,6 @@ public class S3Storage extends BlobStorage {
     @Override
     public FileSystem getFileSystem(String remotePath) throws UserException {
         if (dfsFileSystem == null) {
-            S3Properties.requiredS3Properties(caseInsensitiveProperties);
             Configuration conf = new Configuration();
             System.setProperty("com.amazonaws.services.s3.enableV4", "true");
             PropertyConverter.convertToHadoopFSProperties(caseInsensitiveProperties).forEach(conf::set);
@@ -151,7 +145,6 @@ public class S3Storage extends BlobStorage {
 
     private S3Client getClient(String bucket) throws UserException {
         if (client == null) {
-            S3Properties.requiredS3Properties(caseInsensitiveProperties);
             URI tmpEndpoint = URI.create(caseInsensitiveProperties.get(S3Properties.ENDPOINT));
             StaticCredentialsProvider scp;
             if (!caseInsensitiveProperties.containsKey(S3Properties.SESSION_TOKEN)) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/S3Resource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/S3Resource.java
index 28a6ca2276..76cba51a22 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/S3Resource.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/S3Resource.java
@@ -86,9 +86,6 @@ public class S3Resource extends Resource {
     @Override
     protected void setProperties(Map<String, String> properties) throws DdlException {
         Preconditions.checkState(properties != null);
-
-        // compatible with old version
-        S3Properties.convertToStdProperties(properties);
         // check properties
         S3Properties.requiredS3PingProperties(properties);
         // default need check resource conf valid, so need fix ut and regression case
@@ -164,7 +161,7 @@ public class S3Resource extends Resource {
                 throw new DdlException("current not support modify property : " + any.get());
             }
         }
-        // compatible with old version
+        // compatible with old version, Need convert if modified properties map uses old properties.
         S3Properties.convertToStdProperties(properties);
         boolean needCheck = isNeedCheck(properties);
         LOG.debug("s3 info need check validity : {}", needCheck);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/PropertyConverter.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/PropertyConverter.java
index 6c7e246ed4..fb130d8a01 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/PropertyConverter.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/PropertyConverter.java
@@ -129,7 +129,9 @@ public class PropertyConverter {
         properties.put(S3Properties.REGION, credential.getRegion());
         properties.put(S3Properties.ACCESS_KEY, credential.getAccessKey());
         properties.put(S3Properties.SECRET_KEY, credential.getSecretKey());
-        properties.put(S3Properties.SESSION_TOKEN, credential.getSessionToken());
+        if (properties.containsKey(S3Properties.Env.TOKEN)) {
+            properties.put(S3Properties.SESSION_TOKEN, credential.getSessionToken());
+        }
         if (properties.containsKey(S3Properties.Env.MAX_CONNECTIONS)) {
             properties.put(S3Properties.MAX_CONNECTIONS, properties.get(S3Properties.Env.MAX_CONNECTIONS));
         }
@@ -158,6 +160,7 @@ public class PropertyConverter {
             s3Properties.put(Constants.SOCKET_TIMEOUT, properties.get(S3Properties.CONNECTION_TIMEOUT_MS));
         }
         setS3FsAccess(s3Properties, properties, credential);
+        s3Properties.putAll(properties);
         return s3Properties;
     }
 
@@ -315,7 +318,6 @@ public class PropertyConverter {
             String endpoint = props.get(GlueProperties.ENDPOINT);
             props.put(AWSGlueConfig.AWS_GLUE_ENDPOINT, endpoint);
             String region = S3Properties.getRegionOfEndpoint(endpoint);
-            props.put(GlueProperties.REGION, region);
             props.put(AWSGlueConfig.AWS_REGION, region);
             if (credential.isWhole()) {
                 props.put(AWSGlueConfig.AWS_GLUE_ACCESS_KEY, credential.getAccessKey());
@@ -357,7 +359,7 @@ public class PropertyConverter {
         //  "s3.secret_key" = "yy"
         // )
         String endpoint = props.get(GlueProperties.ENDPOINT);
-        String region = props.getOrDefault(GlueProperties.REGION, S3Properties.getRegionOfEndpoint(endpoint));
+        String region = S3Properties.getRegionOfEndpoint(endpoint);
         if (!Strings.isNullOrEmpty(region)) {
             props.put(S3Properties.REGION, region);
             String s3Endpoint = "s3." + region + ".amazonaws.com";
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/GlueProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/GlueProperties.java
index 68936369fb..bf30a05099 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/GlueProperties.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/GlueProperties.java
@@ -39,4 +39,14 @@ public class GlueProperties extends BaseProperties {
     public static CloudCredential getCredential(Map<String, String> props) {
         return getCloudCredential(props, ACCESS_KEY, SECRET_KEY, SESSION_TOKEN);
     }
+
+    public static CloudCredential getCompatibleCredential(Map<String, String> props) {
+        // Compatible with older versions.
+        CloudCredential credential = getCloudCredential(props, AWSGlueConfig.AWS_GLUE_ACCESS_KEY,
+                    AWSGlueConfig.AWS_GLUE_SECRET_KEY, AWSGlueConfig.AWS_GLUE_SESSION_TOKEN);
+        if (!credential.isWhole()) {
+            credential = BaseProperties.getCompatibleCredential(props);
+        }
+        return credential;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/S3Properties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/S3Properties.java
index 309f469e43..3927e23644 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/S3Properties.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/S3Properties.java
@@ -97,7 +97,7 @@ public class S3Properties extends BaseProperties {
         public static final String DEFAULT_MAX_CONNECTIONS = "50";
         public static final String DEFAULT_REQUEST_TIMEOUT_MS = "3000";
         public static final String DEFAULT_CONNECTION_TIMEOUT_MS = "1000";
-        public static final List<String> REQUIRED_FIELDS = Arrays.asList(ENDPOINT, REGION, ACCESS_KEY, SECRET_KEY);
+        public static final List<String> REQUIRED_FIELDS = Arrays.asList(ENDPOINT, ACCESS_KEY, SECRET_KEY);
         public static final List<String> FS_KEYS = Arrays.asList(ENDPOINT, REGION, ACCESS_KEY, SECRET_KEY, TOKEN,
                 ROOT_PATH, BUCKET, MAX_CONNECTIONS, REQUEST_TIMEOUT_MS, CONNECTION_TIMEOUT_MS);
     }
@@ -155,8 +155,17 @@ public class S3Properties extends BaseProperties {
     }
 
     public static void requiredS3Properties(Map<String, String> properties) throws DdlException {
-        for (String field : S3Properties.REQUIRED_FIELDS) {
-            checkRequiredProperty(properties, field);
+        // Try to convert env properties to uniform properties
+        // compatible with old version
+        S3Properties.convertToStdProperties(properties);
+        if (properties.containsKey(S3Properties.Env.ENDPOINT)) {
+            for (String field : S3Properties.Env.REQUIRED_FIELDS) {
+                checkRequiredProperty(properties, field);
+            }
+        } else {
+            for (String field : S3Properties.REQUIRED_FIELDS) {
+                checkRequiredProperty(properties, field);
+            }
         }
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java
index 6f56ede3d9..8d64ed476c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java
@@ -315,6 +315,9 @@ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctio
 
     @Override
     public List<Column> getTableColumns() throws AnalysisException {
+        if (FeConstants.runningUnitTest) {
+            return Lists.newArrayList();
+        }
         if (!csvSchema.isEmpty()) {
             return csvSchema;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java
index 09e9c6ab72..193566dee8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java
@@ -18,8 +18,11 @@
 package org.apache.doris.tablefunction;
 
 import org.apache.doris.analysis.BrokerDesc;
+import org.apache.doris.analysis.StorageBackend;
 import org.apache.doris.analysis.StorageBackend.StorageType;
+import org.apache.doris.backup.BlobStorage;
 import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.S3URI;
 import org.apache.doris.datasource.credentials.CloudCredentialWithEndpoint;
@@ -27,6 +30,7 @@ import org.apache.doris.datasource.property.PropertyConverter;
 import org.apache.doris.datasource.property.constants.S3Properties;
 import org.apache.doris.thrift.TFileType;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
 
 import java.util.HashMap;
@@ -83,7 +87,17 @@ public class S3TableValuedFunction extends ExternalFileTableValuedFunction {
         locationProperties = S3Properties.credentialToMap(credential);
         String usePathStyle = tvfParams.getOrDefault(PropertyConverter.USE_PATH_STYLE, "false");
         locationProperties.put(PropertyConverter.USE_PATH_STYLE, usePathStyle);
-        parseFile();
+        if (FeConstants.runningUnitTest) {
+            // Just check
+            BlobStorage.create(null, StorageBackend.StorageType.S3, locationProperties);
+        } else {
+            parseFile();
+        }
+    }
+
+    @VisibleForTesting
+    public static Map<String, String> getParams(Map<String, String> params) throws AnalysisException {
+        return getValidParams(params);
     }
 
     private static Map<String, String> getValidParams(Map<String, String> params) throws AnalysisException {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java
new file mode 100644
index 0000000000..e136afa098
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java
@@ -0,0 +1,360 @@
+// 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.datasource.property;
+
+import org.apache.doris.analysis.CreateCatalogStmt;
+import org.apache.doris.analysis.CreateRepositoryStmt;
+import org.apache.doris.analysis.CreateResourceStmt;
+import org.apache.doris.analysis.OutFileClause;
+import org.apache.doris.analysis.QueryStmt;
+import org.apache.doris.analysis.SelectStmt;
+import org.apache.doris.analysis.TableValuedFunctionRef;
+import org.apache.doris.backup.Repository;
+import org.apache.doris.catalog.Env;
+import org.apache.doris.catalog.Resource;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.FeConstants;
+import org.apache.doris.common.Pair;
+import org.apache.doris.common.UserException;
+import org.apache.doris.common.jmockit.Deencapsulation;
+import org.apache.doris.datasource.HMSExternalCatalog;
+import org.apache.doris.tablefunction.S3TableValuedFunction;
+import org.apache.doris.thrift.TFileFormatType;
+import org.apache.doris.utframe.TestWithFeService;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Map;
+
+public class PropertyConverterTest extends TestWithFeService {
+
+    @Override
+    protected void runBeforeAll() throws Exception {
+        createDorisCluster();
+        createDatabase("mock_db");
+        useDatabase("mock_db");
+        createTable("create table mock_tbl1 \n" + "(k1 int, k2 int) distributed by hash(k1) buckets 1\n"
+                + "properties('replication_num' = '1');");
+    }
+
+    @Test
+    public void testOutFileS3PropertiesConverter() throws Exception {
+        String query = "select * from mock_tbl1 \n"
+                + "into outfile 's3://bucket/mock_dir'\n"
+                + "format as csv\n"
+                + "properties(\n"
+                + "    'AWS_ENDPOINT' = 'http://127.0.0.1:9000',\n"
+                + "    'AWS_ACCESS_KEY' = 'akk',\n"
+                + "    'AWS_SECRET_KEY'='akk',\n"
+                + "    'AWS_REGION' = 'mock',\n"
+                + "    'use_path_style' = 'true'\n"
+                + ");";
+        QueryStmt analyzedOutStmt = createStmt(query);
+        Assertions.assertTrue(analyzedOutStmt.hasOutFileClause());
+
+        OutFileClause outFileClause = analyzedOutStmt.getOutFileClause();
+        boolean isOutFileClauseAnalyzed = Deencapsulation.getField(outFileClause, "isAnalyzed");
+        Assertions.assertTrue(isOutFileClauseAnalyzed);
+
+        Assertions.assertEquals(outFileClause.getFileFormatType(), TFileFormatType.FORMAT_CSV_PLAIN);
+
+        String queryNew = "select * from mock_tbl1 \n"
+                + "into outfile 's3://bucket/mock_dir'\n"
+                + "format as csv\n"
+                + "properties(\n"
+                + "    's3.endpoint' = 'http://127.0.0.1:9000',\n"
+                + "    's3.access_key' = 'akk',\n"
+                + "    's3.secret_key'='akk',\n"
+                + "    'use_path_style' = 'true'\n"
+                + ");";
+        QueryStmt analyzedOutStmtNew = createStmt(queryNew);
+        Assertions.assertTrue(analyzedOutStmtNew.hasOutFileClause());
+
+        OutFileClause outFileClauseNew = analyzedOutStmtNew.getOutFileClause();
+        boolean isNewAnalyzed = Deencapsulation.getField(outFileClauseNew, "isAnalyzed");
+        Assertions.assertTrue(isNewAnalyzed);
+    }
+
+    @Test
+    public void testS3SourcePropertiesConverter() throws Exception {
+        String queryOld = "CREATE RESOURCE 'remote_s3'\n"
+                + "PROPERTIES\n"
+                + "(\n"
+                + "   'type' = 's3',\n"
+                + "   'AWS_ENDPOINT' = 's3.us-east-1.amazonaws.com',\n"
+                + "   'AWS_REGION' = 'us-east-1',\n"
+                + "   'AWS_ACCESS_KEY' = 'akk',\n"
+                + "   'AWS_SECRET_KEY' = 'skk',\n"
+                + "   'AWS_ROOT_PATH' = '/',\n"
+                + "   'AWS_BUCKET' = 'bucket',\n"
+                + "   's3_validity_check' = 'false'"
+                + ");";
+        CreateResourceStmt analyzedResourceStmt = createStmt(queryOld);
+        Assertions.assertEquals(analyzedResourceStmt.getProperties().size(), 8);
+        Resource resource = Resource.fromStmt(analyzedResourceStmt);
+        // will add converted properties
+        Assertions.assertEquals(resource.getCopiedProperties().size(), 20);
+
+        String queryNew = "CREATE RESOURCE 'remote_new_s3'\n"
+                + "PROPERTIES\n"
+                + "(\n"
+                + "   'type' = 's3',\n"
+                + "   's3.endpoint' = 'http://s3.us-east-1.amazonaws.com',\n"
+                + "   's3.region' = 'us-east-1',\n"
+                + "   's3.access_key' = 'akk',\n"
+                + "   's3.secret_key' = 'skk',\n"
+                + "   's3.root.path' = '/',\n"
+                + "   's3.bucket' = 'bucket',\n"
+                + "   's3_validity_check' = 'false'"
+                + ");";
+        CreateResourceStmt analyzedResourceStmtNew = createStmt(queryNew);
+        Assertions.assertEquals(analyzedResourceStmtNew.getProperties().size(), 8);
+        Resource newResource = Resource.fromStmt(analyzedResourceStmtNew);
+        // will add converted properties
+        Assertions.assertEquals(newResource.getCopiedProperties().size(), 14);
+    }
+
+    @Test
+    public void testS3RepositoryPropertiesConverter() throws Exception {
+        FeConstants.runningUnitTest = true;
+        String s3Repo = "CREATE REPOSITORY `s3_repo`\n"
+                + "WITH S3\n"
+                + "ON LOCATION 's3://s3-repo'\n"
+                + "PROPERTIES\n"
+                + "(\n"
+                + "    'AWS_ENDPOINT' = 'http://s3.us-east-1.amazonaws.com',\n"
+                + "    'AWS_ACCESS_KEY' = 'akk',\n"
+                + "    'AWS_SECRET_KEY'='skk',\n"
+                + "    'AWS_REGION' = 'us-east-1'\n"
+                + ");";
+        CreateRepositoryStmt analyzedStmt = createStmt(s3Repo);
+        Assertions.assertEquals(analyzedStmt.getProperties().size(), 4);
+        Repository repository = getRepository(analyzedStmt, "s3_repo");
+        Assertions.assertEquals(repository.getStorage().getProperties().size(), 5);
+
+        String s3RepoNew = "CREATE REPOSITORY `s3_repo_new`\n"
+                + "WITH S3\n"
+                + "ON LOCATION 's3://s3-repo'\n"
+                + "PROPERTIES\n"
+                + "(\n"
+                + "    's3.endpoint' = 'http://s3.us-east-1.amazonaws.com',\n"
+                + "    's3.access_key' = 'akk',\n"
+                + "    's3.secret_key' = 'skk'\n"
+                + ");";
+        CreateRepositoryStmt analyzedStmtNew = createStmt(s3RepoNew);
+        Assertions.assertEquals(analyzedStmtNew.getProperties().size(), 3);
+        Repository repositoryNew = getRepository(analyzedStmtNew, "s3_repo_new");
+        Assertions.assertEquals(repositoryNew.getStorage().getProperties().size(), 4);
+    }
+
+    private static Repository getRepository(CreateRepositoryStmt analyzedStmt, String name) throws DdlException {
+        Env.getCurrentEnv().getBackupHandler().createRepository(analyzedStmt);
+        return Env.getCurrentEnv().getBackupHandler().getRepoMgr().getRepo(name);
+    }
+
+    @Test
+    public void testBosBrokerRepositoryPropertiesConverter() throws Exception {
+        FeConstants.runningUnitTest = true;
+        String bosBroker = "CREATE REPOSITORY `bos_broker_repo`\n"
+                + "WITH BROKER `bos_broker`\n"
+                + "ON LOCATION 'bos://backup'\n"
+                + "PROPERTIES\n"
+                + "(\n"
+                + "    'bos_endpoint' = 'http://gz.bcebos.com',\n"
+                + "    'bos_accesskey' = 'akk',\n"
+                + "    'bos_secret_accesskey'='skk'\n"
+                + ");";
+        CreateRepositoryStmt analyzedStmt = createStmt(bosBroker);
+        analyzedStmt.getProperties();
+        Assertions.assertEquals(analyzedStmt.getProperties().size(), 3);
+
+        List<Pair<String, Integer>> brokers = ImmutableList.of(Pair.of("127.0.0.1", 9999));
+        Env.getCurrentEnv().getBrokerMgr().addBrokers("bos_broker", brokers);
+
+        Repository repositoryNew = getRepository(analyzedStmt, "bos_broker_repo");
+        Assertions.assertEquals(repositoryNew.getStorage().getProperties().size(), 4);
+    }
+
+    @Test
+    public void testS3TVFPropertiesConverter() throws Exception {
+        FeConstants.runningUnitTest = true;
+        String queryOld = "select * from s3(\n"
+                    + "  'uri' = 'http://s3.us-east-1.amazonaws.com/test.parquet',\n"
+                    + "  'access_key' = 'akk',\n"
+                    + "  'secret_key' = 'skk',\n"
+                    + "  'region' = 'us-east-1',\n"
+                    + "  'format' = 'parquet',\n"
+                    + "  'use_path_style' = 'true'\n"
+                    + ") limit 10;";
+        SelectStmt analyzedStmt = createStmt(queryOld);
+        Assertions.assertEquals(analyzedStmt.getTableRefs().size(), 1);
+        TableValuedFunctionRef oldFuncTable = (TableValuedFunctionRef) analyzedStmt.getTableRefs().get(0);
+        S3TableValuedFunction s3Tvf = (S3TableValuedFunction) oldFuncTable.getTableFunction();
+        Assertions.assertEquals(s3Tvf.getBrokerDesc().getProperties().size(), 9);
+
+        String queryNew = "select * from s3(\n"
+                    + "  'uri' = 'http://s3.us-east-1.amazonaws.com/test.parquet',\n"
+                    + "  's3.access_key' = 'akk',\n"
+                    + "  's3.secret_key' = 'skk',\n"
+                    + "  'format' = 'parquet',\n"
+                    + "  'use_path_style' = 'true'\n"
+                    + ") limit 10;";
+        SelectStmt analyzedStmtNew = createStmt(queryNew);
+        Assertions.assertEquals(analyzedStmtNew.getTableRefs().size(), 1);
+        TableValuedFunctionRef newFuncTable = (TableValuedFunctionRef) analyzedStmt.getTableRefs().get(0);
+        S3TableValuedFunction newS3Tvf = (S3TableValuedFunction) newFuncTable.getTableFunction();
+        Assertions.assertEquals(newS3Tvf.getBrokerDesc().getProperties().size(), 9);
+    }
+
+    @Test
+    public void testAWSOldCatalogPropertiesConverter() throws Exception {
+        String queryOld = "create catalog hms_s3_old properties (\n"
+                    + "    'type'='hms',\n"
+                    + "    'hive.metastore.uris' = 'thrift://172.21.0.44:7004',\n"
+                    + "    'AWS_ENDPOINT' = 's3.us-east-1.amazonaws.com',\n"
+                    + "    'AWS_REGION' = 'us-east-1',\n"
+                    + "    'AWS_ACCESS_KEY' = 'akk',\n"
+                    + "    'AWS_SECRET_KEY' = 'skk'\n"
+                    + ");";
+        CreateCatalogStmt analyzedStmt = createStmt(queryOld);
+        HMSExternalCatalog catalog = createAndGetCatalog(analyzedStmt, "hms_s3_old");
+        Map<String, String> properties = catalog.getCatalogProperty().getProperties();
+        Assertions.assertEquals(properties.size(), 6);
+
+        Map<String, String> hdProps = catalog.getCatalogProperty().getHadoopProperties();
+        Assertions.assertEquals(hdProps.size(), 19);
+    }
+
+    @Test
+    public void testS3CatalogPropertiesConverter() throws Exception {
+        String query = "create catalog hms_s3 properties (\n"
+                    + "    'type'='hms',\n"
+                    + "    'hive.metastore.uris' = 'thrift://172.21.0.1:7004',\n"
+                    + "    's3.endpoint' = 's3.us-east-1.amazonaws.com',\n"
+                    + "    's3.access_key' = 'akk',\n"
+                    + "    's3.secret_key' = 'skk'\n"
+                    + ");";
+        CreateCatalogStmt analyzedStmt = createStmt(query);
+        HMSExternalCatalog catalog = createAndGetCatalog(analyzedStmt, "hms_s3");
+        Map<String, String> properties = catalog.getCatalogProperty().getProperties();
+        Assertions.assertEquals(properties.size(), 9);
+
+        Map<String, String> hdProps = catalog.getCatalogProperty().getHadoopProperties();
+        Assertions.assertEquals(hdProps.size(), 18);
+    }
+
+    @Test
+    public void testGlueCatalogPropertiesConverter() throws Exception {
+        String queryOld = "create catalog hms_glue_old properties (\n"
+                + "    'type'='hms',\n"
+                + "    'hive.metastore.type'='glue',\n"
+                + "    'hive.metastore.uris' = 'thrift://172.21.0.1:7004',\n"
+                + "    'aws.glue.endpoint' = 'glue.us-east-1.amazonaws.com',\n"
+                + "    'aws.glue.access-key' = 'akk',\n"
+                + "    'aws.glue.secret-key' = 'skk',\n"
+                + "    'aws.region' = 'us-east-1'\n"
+                + ");";
+        CreateCatalogStmt analyzedStmt = createStmt(queryOld);
+        HMSExternalCatalog catalog = createAndGetCatalog(analyzedStmt, "hms_glue_old");
+        Map<String, String> properties = catalog.getCatalogProperty().getProperties();
+        Assertions.assertEquals(properties.size(), 18);
+
+        Map<String, String> hdProps = catalog.getCatalogProperty().getHadoopProperties();
+        Assertions.assertEquals(hdProps.size(), 27);
+
+        String query = "create catalog hms_glue properties (\n"
+                    + "    'type'='hms',\n"
+                    + "    'hive.metastore.type'='glue',\n"
+                    + "    'hive.metastore.uris' = 'thrift://172.21.0.1:7004',\n"
+                    + "    'glue.endpoint' = 'glue.us-east-1.amazonaws.com',\n"
+                    + "    'glue.access_key' = 'akk',\n"
+                    + "    'glue.secret_key' = 'skk'\n"
+                    + ");";
+        CreateCatalogStmt analyzedStmtNew = createStmt(query);
+        HMSExternalCatalog catalogNew = createAndGetCatalog(analyzedStmtNew, "hms_glue");
+        Map<String, String> propertiesNew = catalogNew.getCatalogProperty().getProperties();
+        Assertions.assertEquals(propertiesNew.size(), 18);
+
+        Map<String, String> hdPropsNew = catalogNew.getCatalogProperty().getHadoopProperties();
+        Assertions.assertEquals(hdPropsNew.size(), 27);
+    }
+
+    @Test
+    public void testOBSCatalogPropertiesConverter() throws Exception {
+        String query = "create catalog hms_obs properties (\n"
+                    + "    'type'='hms',\n"
+                    + "    'hive.metastore.uris' = 'thrift://172.21.0.1:7004',\n"
+                    + "    'obs.endpoint' = 'obs.cn-north-4.myhuaweicloud.com',\n"
+                    + "    'obs.access_key' = 'akk',\n"
+                    + "    'obs.secret_key' = 'skk'\n"
+                    + ");";
+        CreateCatalogStmt analyzedStmt = createStmt(query);
+        HMSExternalCatalog catalog = createAndGetCatalog(analyzedStmt, "hms_obs");
+        Map<String, String> properties = catalog.getCatalogProperty().getProperties();
+        Assertions.assertEquals(properties.size(), 9);
+
+        Map<String, String> hdProps = catalog.getCatalogProperty().getHadoopProperties();
+        Assertions.assertEquals(hdProps.size(), 13);
+    }
+
+    @Test
+    public void testCOSCatalogPropertiesConverter() throws Exception {
+        String query = "create catalog hms_cos properties (\n"
+                    + "    'type'='hms',\n"
+                    + "    'hive.metastore.uris' = 'thrift://172.21.0.1:7004',\n"
+                    + "    'cos.endpoint' = 'cos.ap-beijing.myqcloud.com',\n"
+                    + "    'cos.access_key' = 'akk',\n"
+                    + "    'cos.secret_key' = 'skk',\n"
+                    + "    'enable.self.splitter'='true'\n"
+                    + ");";
+        CreateCatalogStmt analyzedStmt = createStmt(query);
+        HMSExternalCatalog catalog = createAndGetCatalog(analyzedStmt, "hms_cos");
+        Map<String, String> properties = catalog.getCatalogProperty().getProperties();
+        Assertions.assertEquals(properties.size(), 10);
+
+        Map<String, String> hdProps = catalog.getCatalogProperty().getHadoopProperties();
+        Assertions.assertEquals(hdProps.size(), 22);
+    }
+
+    @Test
+    public void testOSSCatalogPropertiesConverter() throws Exception {
+        String query = "create catalog hms_oss properties (\n"
+                    + "    'type'='hms',\n"
+                    + "    'hive.metastore.uris' = 'thrift://172.21.0.1:7004',\n"
+                    + "    'oss.endpoint' = 'oss.oss-cn-beijing.aliyuncs.com',\n"
+                    + "    'oss.access_key' = 'akk',\n"
+                    + "    'oss.secret_key' = 'skk'\n"
+                    + ");";
+        CreateCatalogStmt analyzedStmt = createStmt(query);
+        HMSExternalCatalog catalog = createAndGetCatalog(analyzedStmt, "hms_oss");
+        Map<String, String> properties = catalog.getCatalogProperty().getProperties();
+        Assertions.assertEquals(properties.size(), 9);
+
+        Map<String, String> hdProps = catalog.getCatalogProperty().getHadoopProperties();
+        Assertions.assertEquals(hdProps.size(), 21);
+    }
+
+    private static HMSExternalCatalog createAndGetCatalog(CreateCatalogStmt analyzedStmt, String name)
+            throws UserException {
+        Env.getCurrentEnv().getCatalogMgr().createCatalog(analyzedStmt);
+        return (HMSExternalCatalog) Env.getCurrentEnv().getCatalogMgr().getCatalog(name);
+    }
+}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java
index 72ac00d60e..5fa5946e83 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java
@@ -177,6 +177,11 @@ public abstract class TestWithFeService {
         return statementContext;
     }
 
+    protected  <T extends StatementBase> T createStmt(String showSql)
+            throws Exception {
+        return (T) parseAndAnalyzeStmt(showSql, connectContext);
+    }
+
     protected CascadesContext createCascadesContext(String sql) {
         StatementContext statementCtx = createStatementCtx(sql);
         return MemoTestUtils.createCascadesContext(statementCtx, sql);


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


[doris] 23/36: [fix](const column) fix coredump caused by const column for some functions (#18737)

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

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

commit 941f4655fa41e9ddf25e7eb4033de15a5665815f
Author: TengJianPing <18...@users.noreply.github.com>
AuthorDate: Tue Apr 18 13:57:55 2023 +0800

    [fix](const column) fix coredump caused by const column for some functions (#18737)
---
 be/src/vec/columns/column_nullable.cpp             |  6 ++-
 be/src/vec/columns/column_nullable.h               |  3 +-
 be/src/vec/functions/function_bitmap.cpp           |  2 +-
 be/src/vec/functions/function_conv.cpp             |  2 +-
 be/src/vec/functions/function_convert_tz.h         |  2 +-
 be/src/vec/functions/function_jsonb.cpp            |  2 +-
 be/src/vec/functions/function_regexp.cpp           |  2 +-
 be/src/vec/functions/function_string.h             |  2 +-
 be/src/vec/functions/function_timestamp.cpp        |  8 ++--
 be/src/vec/functions/function_totype.h             |  4 +-
 be/src/vec/utils/util.hpp                          | 13 +++++--
 .../bitmap_functions/test_bitmap_function.out      | 19 +++++++++
 .../datetime_functions/test_date_function.out      | 20 ++++++++++
 .../sql_functions/math_functions/test_conv.out     | 19 +++++++++
 .../string_functions/test_string_function.out      | 19 +++++++++
 .../test_string_function_regexp.out                | 19 +++++++++
 .../bitmap_functions/test_bitmap_function.groovy   | 45 ++++++++++++++++++++++
 .../datetime_functions/test_date_function.groovy   | 22 +++++++++++
 .../sql_functions/math_functions/test_conv.groovy  | 29 ++++++++++++++
 .../string_functions/test_string_function.groovy   | 29 ++++++++++++++
 .../test_string_function_regexp.groovy             | 22 ++++++++++-
 21 files changed, 270 insertions(+), 19 deletions(-)

diff --git a/be/src/vec/columns/column_nullable.cpp b/be/src/vec/columns/column_nullable.cpp
index 64db3e6455..959ae55472 100644
--- a/be/src/vec/columns/column_nullable.cpp
+++ b/be/src/vec/columns/column_nullable.cpp
@@ -634,12 +634,14 @@ ColumnPtr ColumnNullable::index(const IColumn& indexes, size_t limit) const {
     return ColumnNullable::create(indexed_data, indexed_null_map);
 }
 
-void check_set_nullable(ColumnPtr& argument_column, ColumnVector<UInt8>::MutablePtr& null_map) {
+void check_set_nullable(ColumnPtr& argument_column, ColumnVector<UInt8>::MutablePtr& null_map,
+                        bool is_single) {
     if (auto* nullable = check_and_get_column<ColumnNullable>(*argument_column)) {
         // Danger: Here must dispose the null map data first! Because
         // argument_columns[i]=nullable->get_nested_column_ptr(); will release the mem
         // of column nullable mem of null map
-        VectorizedUtils::update_null_map(null_map->get_data(), nullable->get_null_map_data());
+        VectorizedUtils::update_null_map(null_map->get_data(), nullable->get_null_map_data(),
+                                         is_single);
         argument_column = nullable->get_nested_column_ptr();
     }
 }
diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h
index 5200531b90..b815121e03 100644
--- a/be/src/vec/columns/column_nullable.h
+++ b/be/src/vec/columns/column_nullable.h
@@ -369,5 +369,6 @@ ColumnPtr make_nullable(const ColumnPtr& column, bool is_nullable = false);
 ColumnPtr remove_nullable(const ColumnPtr& column);
 // check if argument column is nullable. If so, extract its concrete column and set null_map.
 //TODO: use this to replace inner usages.
-void check_set_nullable(ColumnPtr&, ColumnVector<UInt8>::MutablePtr&);
+// is_single: whether null_map is null map of a ColumnConst
+void check_set_nullable(ColumnPtr&, ColumnVector<UInt8>::MutablePtr& null_map, bool is_single);
 } // namespace doris::vectorized
diff --git a/be/src/vec/functions/function_bitmap.cpp b/be/src/vec/functions/function_bitmap.cpp
index 7d340f6937..8f81e8c2a6 100644
--- a/be/src/vec/functions/function_bitmap.cpp
+++ b/be/src/vec/functions/function_bitmap.cpp
@@ -1001,7 +1001,7 @@ public:
         default_preprocess_parameter_columns(argument_columns, col_const, {1, 2}, block, arguments);
 
         for (int i = 0; i < 3; i++) {
-            check_set_nullable(argument_columns[i], res_null_map);
+            check_set_nullable(argument_columns[i], res_null_map, col_const[i]);
         }
 
         auto bitmap_column = assert_cast<const ColumnBitmap*>(argument_columns[0].get());
diff --git a/be/src/vec/functions/function_conv.cpp b/be/src/vec/functions/function_conv.cpp
index 027a7e6a17..68f2070bea 100644
--- a/be/src/vec/functions/function_conv.cpp
+++ b/be/src/vec/functions/function_conv.cpp
@@ -65,7 +65,7 @@ public:
         default_preprocess_parameter_columns(argument_columns, col_const, {1, 2}, block, arguments);
 
         for (int i = 0; i < 3; i++) {
-            check_set_nullable(argument_columns[i], result_null_map_column);
+            check_set_nullable(argument_columns[i], result_null_map_column, col_const[i]);
         }
 
         if (col_const[1] && col_const[2]) {
diff --git a/be/src/vec/functions/function_convert_tz.h b/be/src/vec/functions/function_convert_tz.h
index 9857417653..4a1d66b459 100644
--- a/be/src/vec/functions/function_convert_tz.h
+++ b/be/src/vec/functions/function_convert_tz.h
@@ -194,7 +194,7 @@ public:
         default_preprocess_parameter_columns(argument_columns, col_const, {1, 2}, block, arguments);
 
         for (int i = 0; i < 3; i++) {
-            check_set_nullable(argument_columns[i], result_null_map_column);
+            check_set_nullable(argument_columns[i], result_null_map_column, col_const[i]);
         }
 
         if (col_const[1] && col_const[2]) {
diff --git a/be/src/vec/functions/function_jsonb.cpp b/be/src/vec/functions/function_jsonb.cpp
index 750f11bca5..024c390072 100644
--- a/be/src/vec/functions/function_jsonb.cpp
+++ b/be/src/vec/functions/function_jsonb.cpp
@@ -316,7 +316,7 @@ public:
         for (int i = 0; i < 2; ++i) {
             std::tie(argument_columns[i], col_const[i]) =
                     unpack_if_const(block.get_by_position(arguments[i]).column);
-            check_set_nullable(argument_columns[i], null_map);
+            check_set_nullable(argument_columns[i], null_map, col_const[i]);
         }
 
         auto res = Impl::ColumnType::create();
diff --git a/be/src/vec/functions/function_regexp.cpp b/be/src/vec/functions/function_regexp.cpp
index 8f646a82f6..c9061d3c61 100644
--- a/be/src/vec/functions/function_regexp.cpp
+++ b/be/src/vec/functions/function_regexp.cpp
@@ -429,7 +429,7 @@ public:
                                                  arguments);
         }
         for (int i = 0; i < argument_size; i++) {
-            check_set_nullable(argument_columns[i], result_null_map);
+            check_set_nullable(argument_columns[i], result_null_map, col_const[i]);
         }
 
         if constexpr (std::is_same_v<Impl, RegexpExtractAllImpl>) {
diff --git a/be/src/vec/functions/function_string.h b/be/src/vec/functions/function_string.h
index b02f379781..cb28fa7b82 100644
--- a/be/src/vec/functions/function_string.h
+++ b/be/src/vec/functions/function_string.h
@@ -140,7 +140,7 @@ struct SubstringUtil {
         default_preprocess_parameter_columns(argument_columns, col_const, {1, 2}, block, arguments);
 
         for (int i = 0; i < 3; i++) {
-            check_set_nullable(argument_columns[i], null_map);
+            check_set_nullable(argument_columns[i], null_map, col_const[i]);
         }
 
         auto specific_str_column = assert_cast<const ColumnString*>(argument_columns[0].get());
diff --git a/be/src/vec/functions/function_timestamp.cpp b/be/src/vec/functions/function_timestamp.cpp
index 4ef50490ab..2122c2b09d 100644
--- a/be/src/vec/functions/function_timestamp.cpp
+++ b/be/src/vec/functions/function_timestamp.cpp
@@ -52,12 +52,12 @@ struct StrToDate {
         ColumnPtr argument_columns[2] = {
                 col_const[0] ? static_cast<const ColumnConst&>(*col0).convert_to_full_column()
                              : col0};
-        check_set_nullable(argument_columns[0], null_map);
+        check_set_nullable(argument_columns[0], null_map, col_const[0]);
         //TODO: when we set default implementation for nullable, the check_set_nullable for arguments is useless. consider to remove it.
 
         std::tie(argument_columns[1], col_const[1]) =
                 unpack_if_const(block.get_by_position(arguments[1]).column);
-        check_set_nullable(argument_columns[1], null_map);
+        check_set_nullable(argument_columns[1], null_map, col_const[1]);
 
         auto specific_str_column = assert_cast<const ColumnString*>(argument_columns[0].get());
         auto specific_char_column = assert_cast<const ColumnString*>(argument_columns[1].get());
@@ -190,11 +190,11 @@ struct MakeDateImpl {
         ColumnPtr argument_columns[2] = {
                 col_const[0] ? static_cast<const ColumnConst&>(*col0).convert_to_full_column()
                              : col0};
-        check_set_nullable(argument_columns[0], null_map);
+        check_set_nullable(argument_columns[0], null_map, col_const[0]);
 
         std::tie(argument_columns[1], col_const[1]) =
                 unpack_if_const(block.get_by_position(arguments[1]).column);
-        check_set_nullable(argument_columns[1], null_map);
+        check_set_nullable(argument_columns[1], null_map, col_const[1]);
 
         ColumnPtr res = nullptr;
         WhichDataType which(remove_nullable(block.get_by_position(result).type));
diff --git a/be/src/vec/functions/function_totype.h b/be/src/vec/functions/function_totype.h
index 0fa2724a22..2ead9ab704 100644
--- a/be/src/vec/functions/function_totype.h
+++ b/be/src/vec/functions/function_totype.h
@@ -330,7 +330,7 @@ public:
         for (int i = 0; i < 2; ++i) {
             std::tie(argument_columns[i], col_const[i]) =
                     unpack_if_const(block.get_by_position(arguments[i]).column);
-            check_set_nullable(argument_columns[i], null_map);
+            check_set_nullable(argument_columns[i], null_map, col_const[i]);
         }
 
         using ResultDataType = typename Impl<LeftDataType, RightDataType, ResultDateType,
@@ -402,7 +402,7 @@ public:
         for (int i = 0; i < 2; ++i) {
             std::tie(argument_columns[i], col_const[i]) =
                     unpack_if_const(block.get_by_position(arguments[i]).column);
-            check_set_nullable(argument_columns[i], null_map);
+            check_set_nullable(argument_columns[i], null_map, col_const[i]);
         }
 
         auto res = Impl::ColumnType::create();
diff --git a/be/src/vec/utils/util.hpp b/be/src/vec/utils/util.hpp
index 67a7489d2f..cb01c36eb3 100644
--- a/be/src/vec/utils/util.hpp
+++ b/be/src/vec/utils/util.hpp
@@ -65,12 +65,19 @@ public:
         return columns_with_type_and_name;
     }
 
-    static void update_null_map(NullMap& dst, const NullMap& src) {
+    // is_single: whether src is null map of a ColumnConst
+    static void update_null_map(NullMap& dst, const NullMap& src, bool is_single = false) {
         size_t size = dst.size();
         auto* __restrict l = dst.data();
         auto* __restrict r = src.data();
-        for (size_t i = 0; i < size; ++i) {
-            l[i] |= r[i];
+        if (is_single && r[0]) {
+            for (size_t i = 0; i < size; ++i) {
+                l[i] = 1;
+            }
+        } else {
+            for (size_t i = 0; i < size; ++i) {
+                l[i] |= r[i];
+            }
         }
     }
 
diff --git a/regression-test/data/query_p0/sql_functions/bitmap_functions/test_bitmap_function.out b/regression-test/data/query_p0/sql_functions/bitmap_functions/test_bitmap_function.out
index 53f5f731db..8dc638b78a 100644
--- a/regression-test/data/query_p0/sql_functions/bitmap_functions/test_bitmap_function.out
+++ b/regression-test/data/query_p0/sql_functions/bitmap_functions/test_bitmap_function.out
@@ -473,3 +473,22 @@ true
 -- !sql --
 \N
 
+-- !sql_bitmap_subset_in_range --
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+
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 fe55f5432e..d32137d1b2 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
@@ -23,6 +23,26 @@
 -- !sql --
 \N
 
+-- !sql_convert_tz_null --
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+
 -- !sql1 --
 1	2019-08-01T13:21:03	Asia/Shanghai	Asia/Shanghai	2019-08-01T13:21:03
 2	2019-08-01T13:21:03	Asia/Singapore	Asia/Shanghai	2019-08-01T13:21:03
diff --git a/regression-test/data/query_p0/sql_functions/math_functions/test_conv.out b/regression-test/data/query_p0/sql_functions/math_functions/test_conv.out
index cb2f1e9d4a..ff6e6b72e9 100644
--- a/regression-test/data/query_p0/sql_functions/math_functions/test_conv.out
+++ b/regression-test/data/query_p0/sql_functions/math_functions/test_conv.out
@@ -2,3 +2,22 @@
 -- !select --
 1111
 
+-- !sql_conv1 --
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+
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 5e3bba66ba..ff04cdd167 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
@@ -293,6 +293,25 @@ c1
 -- !sql --
 bc
 
+-- !sql_substring1 --
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+
 -- !sql --
 
 
diff --git a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function_regexp.out b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function_regexp.out
index 8d27c997bb..5480d6e270 100644
--- a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function_regexp.out
+++ b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function_regexp.out
@@ -51,6 +51,25 @@ a-b c
 -- !sql --
 a <b> b
 
+-- !sql_regexp_null --
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+\N
+
 -- !sql --
 false	1	1989	1001	11011902	123.123	true	1989-03-21	1989-03-21T13:00	wangjuoo4	0.1	6.333	string12345	170141183460469231731687303715884105727
 
diff --git a/regression-test/suites/query_p0/sql_functions/bitmap_functions/test_bitmap_function.groovy b/regression-test/suites/query_p0/sql_functions/bitmap_functions/test_bitmap_function.groovy
index b4cf3cdc3b..a14267a395 100644
--- a/regression-test/suites/query_p0/sql_functions/bitmap_functions/test_bitmap_function.groovy
+++ b/regression-test/suites/query_p0/sql_functions/bitmap_functions/test_bitmap_function.groovy
@@ -645,4 +645,49 @@ suite("test_bitmap_function") {
     sql "insert into d_table select -4,-4,-4,'d';"
     try_sql "select bitmap_union(to_bitmap_with_check(k2)) from d_table;"
     qt_sql "select bitmap_union(to_bitmap(k2)) from d_table;"
+
+    // bug fix
+    sql """ DROP TABLE IF EXISTS test_bitmap1 """
+    sql """
+        CREATE TABLE test_bitmap1 (
+          dt INT(11) NULL,
+          id bitmap BITMAP_UNION NULL
+        ) ENGINE=OLAP
+        AGGREGATE KEY(dt)
+        DISTRIBUTED BY HASH(dt) BUCKETS 1
+        properties (
+            "replication_num" = "1"
+        );
+    """
+    sql """
+        insert into
+            test_bitmap1
+        values
+            (1, to_bitmap(11)),
+            (2, to_bitmap(22)),
+            (3, to_bitmap(33)),
+            (4, to_bitmap(44)),
+            (5, to_bitmap(44)),
+            (6, to_bitmap(44)),
+            (7, to_bitmap(44)),
+            (8, to_bitmap(44)),
+            (9, to_bitmap(44)),
+            (10, to_bitmap(44)),
+            (11, to_bitmap(44)),
+            (12, to_bitmap(44)),
+            (13, to_bitmap(44)),
+            (14, to_bitmap(44)),
+            (15, to_bitmap(44)),
+            (16, to_bitmap(44)),
+            (17, to_bitmap(44));
+    """
+    qt_sql_bitmap_subset_in_range """
+        select /*+SET_VAR(parallel_fragment_exec_instance_num=1)*/
+            bitmap_to_string(
+                bitmap_subset_in_range(id, cast(null as bigint), cast(null as bigint))
+            )
+        from
+            test_bitmap1;
+    """
+
 }
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 ebd2f98b88..b344198391 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
@@ -49,6 +49,28 @@ suite("test_date_function") {
     qt_sql """ SELECT convert_tz('2022-02-29 13:21:03', '+08:00', 'America/London') result; """
     qt_sql """ SELECT convert_tz('1900-00-00 13:21:03', '+08:00', 'America/London') result; """
 
+    // bug fix
+    sql """ insert into ${tableName} values 
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03"),
+                ("2019-08-01 13:21:03");
+    """
+    qt_sql_convert_tz_null """ SELECT /*+SET_VAR(parallel_fragment_exec_instance_num=1)*/ convert_tz(test_datetime, cast(null as varchar), cast(null as varchar)) result from test_date_function; """
+
     sql """ truncate table ${tableName} """
 
     def timezoneCachedTableName = "test_convert_tz_with_timezone_cache"
diff --git a/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy b/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy
index f0bd110ab8..6c4867174d 100644
--- a/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy
+++ b/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy
@@ -17,5 +17,34 @@
 
 suite("test_conv") {
     qt_select "SELECT CONV(15,10,2)"
+
+    sql """ drop table if exists test_conv; """
+    sql """ create table test_conv(
+        k1 varchar(16),
+        v1 int
+    ) distributed by hash (k1) buckets 1
+    properties ("replication_num"="1");
+    """
+    sql """ insert into test_conv values
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1),
+        ("100", 1)
+    """
+
+    qt_sql_conv1 """ select /*+SET_VAR(parallel_fragment_exec_instance_num=1)*/conv(k1, cast(null as bigint), cast(null as bigint)) from test_conv; """
 }
 
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 13d217dd68..73a9222aa1 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
@@ -144,6 +144,35 @@ suite("test_string_function") {
     qt_sql "select substring('abc1', 5);"
     qt_sql "select substring('abc1def', 2, 2);"
 
+    sql """ drop table if exists test_string_function; """
+    sql """ create table test_string_function (
+        k1 varchar(16),
+        v1 int
+    ) distributed by hash (k1) buckets 1
+    properties ("replication_num"="1");
+    """
+    sql """ insert into test_string_function values
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1),
+        ("aaaaaaaa", 1)
+    """
+    // bug fix
+    qt_sql_substring1 """ select /*+SET_VAR(parallel_fragment_exec_instance_num=1)*/ substring(k1, cast(null as int), cast(null as int)) from test_string_function; """
+
     qt_sql "select substr('a',3,1);"
     qt_sql "select substr('a',2,1);"
     qt_sql "select substr('a',1,1);"
diff --git a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function_regexp.groovy b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function_regexp.groovy
index c56690337e..fba213ef98 100644
--- a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function_regexp.groovy
+++ b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function_regexp.groovy
@@ -24,7 +24,7 @@ suite("test_string_function_regexp") {
             CREATE TABLE IF NOT EXISTS ${tbName} (
                 k varchar(32)
             )
-            DISTRIBUTED BY HASH(k) BUCKETS 5 properties("replication_num" = "1");
+            DISTRIBUTED BY HASH(k) BUCKETS 1 properties("replication_num" = "1");
         """
     sql """
         INSERT INTO ${tbName} VALUES 
@@ -55,6 +55,26 @@ suite("test_string_function_regexp") {
     qt_sql "SELECT regexp_replace_one('a b c', \" \", \"-\");"
     qt_sql "SELECT regexp_replace_one('a b b','(b)','<\\\\1>');"
 
+    // bug fix
+    sql """
+        INSERT INTO ${tbName} VALUES
+            ("billie eillish"),
+            ("billie eillish"),
+            ("billie eillish"),
+            ("billie eillish"),
+            ("billie eillish"),
+            ("billie eillish"),
+            ("billie eillish"),
+            ("billie eillish"),
+            ("billie eillish"),
+            ("billie eillish"),
+            ("billie eillish"),
+            ("billie eillish"),
+            ("billie eillish")
+        """
+    qt_sql_regexp_null "SELECT /*+SET_VAR(parallel_fragment_exec_instance_num=1)*/regexp_extract(k, cast(null as varchar), 1) from test_string_function_regexp;"
+    // end bug fix
+
     sql "DROP TABLE ${tbName};"
 
     def tableName= "test"


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


[doris] 29/36: [Fix](vertical compaction) Preserve _segment_num_rows during final segment flush (#18779)

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

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

commit 588e3623f1ff93e84174230d428487fe920a8d47
Author: airborne12 <ai...@gmail.com>
AuthorDate: Tue Apr 18 20:58:23 2023 +0800

    [Fix](vertical compaction) Preserve _segment_num_rows during final segment flush (#18779)
---
 be/src/olap/rowset/vertical_beta_rowset_writer.cpp | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/be/src/olap/rowset/vertical_beta_rowset_writer.cpp b/be/src/olap/rowset/vertical_beta_rowset_writer.cpp
index 769be401e9..e1e3ab46de 100644
--- a/be/src/olap/rowset/vertical_beta_rowset_writer.cpp
+++ b/be/src/olap/rowset/vertical_beta_rowset_writer.cpp
@@ -67,9 +67,6 @@ Status VerticalBetaRowsetWriter::add_columns(const vectorized::Block* block,
             // segment is full, need flush columns and create new segment writer
             RETURN_IF_ERROR(_flush_columns(&_segment_writers[_cur_writer_idx], true));
 
-            _segment_num_rows.resize(_cur_writer_idx + 1);
-            _segment_num_rows[_cur_writer_idx] = _segment_writers[_cur_writer_idx]->row_count();
-
             std::unique_ptr<segment_v2::SegmentWriter> writer;
             RETURN_IF_ERROR(_create_segment_writer(col_ids, is_key, &writer));
             _segment_writers.emplace_back(std::move(writer));
@@ -116,6 +113,8 @@ Status VerticalBetaRowsetWriter::_flush_columns(
         key_bounds.set_min_key(min_key.to_string());
         key_bounds.set_max_key(max_key.to_string());
         _segments_encoded_key_bounds.emplace_back(key_bounds);
+        _segment_num_rows.resize(_cur_writer_idx + 1);
+        _segment_num_rows[_cur_writer_idx] = _segment_writers[_cur_writer_idx]->row_count();
     }
     _total_index_size += static_cast<int64_t>(index_size);
     return Status::OK();


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


[doris] 32/36: [Doc] Fix error test example (#18764)

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

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

commit a0f1cbe023907066a08fc5a20686290ada672183
Author: HHoflittlefish777 <77...@users.noreply.github.com>
AuthorDate: Wed Apr 19 09:06:27 2023 +0800

    [Doc] Fix error test example (#18764)
---
 run-fe-ut.sh | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/run-fe-ut.sh b/run-fe-ut.sh
index b7df8c046e..ca1dafacac 100755
--- a/run-fe-ut.sh
+++ b/run-fe-ut.sh
@@ -112,8 +112,8 @@ else
     if [[ "${RUN}" -eq 1 ]]; then
         echo "Run the specified class: $1"
         # eg:
-        # sh run-fe-ut.sh --run org.apache.doris.utframe.Demo
-        # sh run-fe-ut.sh --run org.apache.doris.utframe.Demo#testCreateDbAndTable+test2
+        # sh run-fe-ut.sh --run org.apache.doris.utframe.DemoTest
+        # sh run-fe-ut.sh --run org.apache.doris.utframe.DemoTest#testCreateDbAndTable+test2
         "${MVN_CMD}" test -Dcheckstyle.skip=true -DfailIfNoTests=false -D test="$1"
     else
         echo "Run Frontend UT"


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


[doris] 31/36: [fix](compile) Fix block.cpp compilation failure (#18797)

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

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

commit 8a3fc223feb90e2ef611f00e6ec9a658e2e3d700
Author: Xinyi Zou <zo...@gmail.com>
AuthorDate: Wed Apr 19 08:49:23 2023 +0800

    [fix](compile) Fix block.cpp compilation failure (#18797)
---
 be/src/vec/core/block.cpp | 1 +
 1 file changed, 1 insertion(+)

diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp
index 63b79d7965..dc7cef55c1 100644
--- a/be/src/vec/core/block.cpp
+++ b/be/src/vec/core/block.cpp
@@ -27,6 +27,7 @@
 #include "agent/be_exec_version_manager.h"
 #include "common/status.h"
 #include "runtime/descriptors.h"
+#include "runtime/thread_context.h"
 #include "udf/udf.h"
 #include "util/block_compression.h"
 #include "util/faststring.h"


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


[doris] 08/36: [chore](build) add apache-orc git submodule path (#18695)

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

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

commit 982e4374704d95799ebca8baa50407084c054afe
Author: ZhangYu0123 <67...@users.noreply.github.com>
AuthorDate: Mon Apr 17 00:08:25 2023 +0800

    [chore](build) add apache-orc git submodule path (#18695)
    
    1. Add apache-orc git submodule update path, not update all modules
    When sh build.sh, update all modules will fails serveral times because of unstable github network.
    It wastes many time.
    
    2. Add gitignore for  be/src/apache-orc/   to avoid mistake commits.
---
 .gitignore | 1 +
 build.sh   | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)

diff --git a/.gitignore b/.gitignore
index ff1ab08882..f4532ce110 100644
--- a/.gitignore
+++ b/.gitignore
@@ -84,6 +84,7 @@ be/src/gen_cpp/opcode
 be/tags
 be/test/olap/test_data/tablet_meta_test.hdr
 be/.devcontainer/
+be/src/apache-orc/
 
 ## tools
 tools/ssb-tools/ssb-data/
diff --git a/build.sh b/build.sh
index 8c4de60e34..e9701fcac0 100755
--- a/build.sh
+++ b/build.sh
@@ -251,7 +251,7 @@ if [[ ! -f "${DORIS_HOME}/be/src/apache-orc/README.md" ]]; then
     echo "apache-orc not exists, need to update submodules ..."
     set +e
     cd "${DORIS_HOME}"
-    git submodule update --init --recursive
+    git submodule update --init --recursive be/src/apache-orc
     exit_code=$?
     set -e
     if [[ "${exit_code}" -ne 0 ]]; then


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


[doris] 21/36: [chore](hashtable) Use doris' Allocator to replace std::allocator in phmap (#18735)

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

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

commit 353f5fc24692729dd258442c2624806abd7a1fd7
Author: Jerry Hu <mr...@gmail.com>
AuthorDate: Tue Apr 18 09:58:28 2023 +0800

    [chore](hashtable) Use doris' Allocator to replace std::allocator in phmap (#18735)
---
 be/src/exprs/hybrid_set.h                          |  7 ++-
 be/src/olap/hll.cpp                                |  2 +-
 be/src/olap/hll.h                                  |  4 +-
 be/src/olap/in_list_predicate.h                    |  2 -
 be/src/olap/reader.cpp                             |  2 -
 be/src/olap/rowset/segment_v2/binary_dict_page.h   |  2 -
 .../vec/aggregate_functions/aggregate_function.h   |  5 +-
 .../aggregate_functions/aggregate_function_topn.h  |  4 +-
 .../aggregate_functions/aggregate_function_uniq.h  |  5 +-
 be/src/vec/common/hash_table/ph_hash_map.h         |  3 +-
 be/src/vec/common/hash_table/phmap_fwd_decl.h      | 54 ++++++++++++++++++++++
 11 files changed, 68 insertions(+), 22 deletions(-)

diff --git a/be/src/exprs/hybrid_set.h b/be/src/exprs/hybrid_set.h
index 67c8f43099..54a30cd343 100644
--- a/be/src/exprs/hybrid_set.h
+++ b/be/src/exprs/hybrid_set.h
@@ -17,14 +17,13 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
-
 #include "common/object_pool.h"
 #include "runtime/decimalv2_value.h"
 #include "runtime/define_primitive_type.h"
 #include "runtime/primitive_type.h"
 #include "vec/columns/column_nullable.h"
 #include "vec/columns/column_string.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
@@ -149,7 +148,7 @@ template <typename T>
 class DynamicContainer {
 public:
     using Self = DynamicContainer;
-    using Iterator = typename phmap::flat_hash_set<T>::iterator;
+    using Iterator = typename vectorized::flat_hash_set<T>::iterator;
     using ElementType = T;
 
     DynamicContainer() = default;
@@ -168,7 +167,7 @@ public:
     size_t size() const { return _set.size(); }
 
 private:
-    phmap::flat_hash_set<T> _set;
+    vectorized::flat_hash_set<T> _set;
 };
 
 // TODO Maybe change void* parameter to template parameter better.
diff --git a/be/src/olap/hll.cpp b/be/src/olap/hll.cpp
index 7c234d1703..8f7a61e712 100644
--- a/be/src/olap/hll.cpp
+++ b/be/src/olap/hll.cpp
@@ -49,7 +49,7 @@ void HyperLogLog::_convert_explicit_to_register() {
         _update_registers(value);
     }
     // clear _hash_set
-    phmap::flat_hash_set<uint64_t>().swap(_hash_set);
+    vectorized::flat_hash_set<uint64_t>().swap(_hash_set);
 }
 
 // Change HLL_DATA_EXPLICIT to HLL_DATA_FULL directly, because HLL_DATA_SPARSE
diff --git a/be/src/olap/hll.h b/be/src/olap/hll.h
index a3347f4626..f1ad81100b 100644
--- a/be/src/olap/hll.h
+++ b/be/src/olap/hll.h
@@ -18,7 +18,6 @@
 #pragma once
 
 #include <math.h>
-#include <parallel_hashmap/phmap.h>
 #include <stdio.h>
 
 #include <map>
@@ -30,6 +29,7 @@
 #endif
 
 #include "gutil/macros.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 
 namespace doris {
 
@@ -268,7 +268,7 @@ public:
 
 private:
     HllDataType _type = HLL_DATA_EMPTY;
-    phmap::flat_hash_set<uint64_t> _hash_set;
+    vectorized::flat_hash_set<uint64_t> _hash_set;
 
     // This field is much space consuming(HLL_REGISTERS_COUNT), we create
     // it only when it is really needed.
diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h
index 7d0001cf1c..e207bf28d5 100644
--- a/be/src/olap/in_list_predicate.h
+++ b/be/src/olap/in_list_predicate.h
@@ -17,8 +17,6 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
-
 #include <cstdint>
 #include <roaring/roaring.hh>
 
diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp
index f3ffd44d32..b21691af2d 100644
--- a/be/src/olap/reader.cpp
+++ b/be/src/olap/reader.cpp
@@ -17,8 +17,6 @@
 
 #include "olap/reader.h"
 
-#include <parallel_hashmap/phmap.h>
-
 #include "common/status.h"
 #include "exprs/create_predicate_function.h"
 #include "exprs/hybrid_set.h"
diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.h b/be/src/olap/rowset/segment_v2/binary_dict_page.h
index 26bf127a1f..b570b042dc 100644
--- a/be/src/olap/rowset/segment_v2/binary_dict_page.h
+++ b/be/src/olap/rowset/segment_v2/binary_dict_page.h
@@ -17,8 +17,6 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
-
 #include <functional>
 #include <memory>
 #include <string>
diff --git a/be/src/vec/aggregate_functions/aggregate_function.h b/be/src/vec/aggregate_functions/aggregate_function.h
index c55612ec15..2d6b07e403 100644
--- a/be/src/vec/aggregate_functions/aggregate_function.h
+++ b/be/src/vec/aggregate_functions/aggregate_function.h
@@ -20,9 +20,8 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
-
 #include "vec/columns/column_complex.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 #include "vec/core/block.h"
 #include "vec/core/column_numbers.h"
 #include "vec/core/field.h"
@@ -221,7 +220,7 @@ public:
                       std::is_same_v<Derived,
                                      AggregateFunctionBitmapOp<AggregateFunctionBitmapUnionOp>>) {
             if (agg_many) {
-                phmap::flat_hash_map<AggregateDataPtr, std::vector<int>> place_rows;
+                flat_hash_map<AggregateDataPtr, std::vector<int>> place_rows;
                 for (int i = 0; i < batch_size; ++i) {
                     auto iter = place_rows.find(places[i] + place_offset);
                     if (iter == place_rows.end()) {
diff --git a/be/src/vec/aggregate_functions/aggregate_function_topn.h b/be/src/vec/aggregate_functions/aggregate_function_topn.h
index 99ec3dadd9..7bfacf7977 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_topn.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_topn.h
@@ -17,7 +17,6 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
 #include <rapidjson/stringbuffer.h>
 #include <rapidjson/writer.h>
 
@@ -29,6 +28,7 @@
 #include "vec/columns/column_array.h"
 #include "vec/columns/column_string.h"
 #include "vec/columns/column_vector.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 #include "vec/core/types.h"
 #include "vec/data_types/data_type_array.h"
 #include "vec/data_types/data_type_string.h"
@@ -183,7 +183,7 @@ struct AggregateFunctionTopNData {
 
     int top_num = 0;
     uint64_t capacity = 0;
-    phmap::flat_hash_map<T, uint64_t> counter_map;
+    flat_hash_map<T, uint64_t> counter_map;
 };
 
 struct AggregateFunctionTopNImplInt {
diff --git a/be/src/vec/aggregate_functions/aggregate_function_uniq.h b/be/src/vec/aggregate_functions/aggregate_function_uniq.h
index ff17082752..592cb5a925 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_uniq.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_uniq.h
@@ -20,8 +20,6 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
-
 #include <type_traits>
 
 #include "gutil/hash/city.h"
@@ -31,6 +29,7 @@
 #include "vec/common/assert_cast.h"
 #include "vec/common/bit_cast.h"
 #include "vec/common/hash_table/hash_set.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 #include "vec/common/typeid_cast.h"
 #include "vec/data_types/data_type_number.h"
 
@@ -47,7 +46,7 @@ struct AggregateFunctionUniqExactData {
     using Key = std::conditional_t<is_string_key, UInt128, T>;
     using Hash = std::conditional_t<is_string_key, UInt128TrivialHash, HashCRC32<Key>>;
 
-    using Set = phmap::flat_hash_set<Key, Hash>;
+    using Set = flat_hash_set<Key, Hash>;
 
     static UInt128 ALWAYS_INLINE get_key(const StringRef& value) {
         UInt128 key;
diff --git a/be/src/vec/common/hash_table/ph_hash_map.h b/be/src/vec/common/hash_table/ph_hash_map.h
index 2a847eda75..da51f31cf9 100644
--- a/be/src/vec/common/hash_table/ph_hash_map.h
+++ b/be/src/vec/common/hash_table/ph_hash_map.h
@@ -23,6 +23,7 @@
 
 #include "vec/common/hash_table/hash.h"
 #include "vec/common/hash_table/hash_table_utils.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 
 template <typename Key, typename Mapped>
 ALWAYS_INLINE inline auto lookup_result_get_mapped(std::pair<const Key, Mapped>* it) {
@@ -36,7 +37,7 @@ public:
     using Self = PHHashMap;
     using Hash = HashMethod;
     using cell_type = std::pair<const Key, Mapped>;
-    using HashMapImpl = phmap::flat_hash_map<Key, Mapped, Hash>;
+    using HashMapImpl = doris::vectorized::flat_hash_map<Key, Mapped, Hash>;
 
     using key_type = Key;
     using mapped_type = Mapped;
diff --git a/be/src/vec/common/hash_table/phmap_fwd_decl.h b/be/src/vec/common/hash_table/phmap_fwd_decl.h
new file mode 100644
index 0000000000..d37eb898e1
--- /dev/null
+++ b/be/src/vec/common/hash_table/phmap_fwd_decl.h
@@ -0,0 +1,54 @@
+// 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.
+
+#pragma once
+
+#include <parallel_hashmap/phmap.h>
+
+#include "vec/common/allocator.h"
+
+namespace doris::vectorized {
+
+/// `Allocator_` implements several interfaces of `std::allocator`
+/// which `phmap::flat_hash_map` will use.
+template <typename T>
+class Allocator_ : private Allocator<false, true> {
+public:
+    using value_type = T;
+    using pointer = T*;
+
+    Allocator_() = default;
+
+    template <typename T_>
+    Allocator_(const Allocator_<T_>&) {};
+
+    constexpr T* allocate(size_t n) { return static_cast<T*>(Allocator::alloc(n * sizeof(T))); }
+
+    void deallocate(pointer p, size_t n) { Allocator::free(p, n * sizeof(T)); }
+
+    friend bool operator==(const Allocator_&, const Allocator_&) { return true; }
+};
+
+template <typename K, typename V, typename Hash = phmap::Hash<K>, typename Eq = phmap::EqualTo<K>,
+          typename Alloc = Allocator_<phmap::Pair<const K, V>>>
+using flat_hash_map = phmap::flat_hash_map<K, V, Hash, Eq, Alloc>;
+
+template <typename K, typename Hash = phmap::Hash<K>, typename Eq = phmap::EqualTo<K>,
+          typename Alloc = Allocator_<K>>
+using flat_hash_set = phmap::flat_hash_set<K, Hash, Eq, Alloc>;
+
+} // namespace doris::vectorized


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


[doris] 15/36: [feature](segcompaction) enable segcompaction by default (#18722)

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

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

commit 7e5790c995aec6ea48365f9833199a61bbf290c9
Author: zhengyu <fr...@gmail.com>
AuthorDate: Mon Apr 17 18:00:49 2023 +0800

    [feature](segcompaction) enable segcompaction by default (#18722)
    
    Signed-off-by: freemandealer <fr...@gmail.com>
---
 be/src/common/config.h                           |  2 +-
 docs/en/docs/admin-manual/config/be-config.md    | 10 +++++-----
 docs/en/docs/faq/data-faq.md                     |  4 ++--
 docs/zh-CN/docs/admin-manual/config/be-config.md | 10 +++++-----
 docs/zh-CN/docs/faq/data-faq.md                  |  6 +++---
 5 files changed, 16 insertions(+), 16 deletions(-)

diff --git a/be/src/common/config.h b/be/src/common/config.h
index 3f2034191c..976dfd3149 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -875,7 +875,7 @@ CONF_String(be_node_role, "mix");
 // Hide the be config page for webserver.
 CONF_Bool(hide_webserver_config_page, "false");
 
-CONF_Bool(enable_segcompaction, "false"); // currently only support vectorized storage
+CONF_Bool(enable_segcompaction, "true");
 
 // Trigger segcompaction if the num of segments in a rowset exceeds this threshold.
 CONF_Int32(segcompaction_threshold_segment_num, "10");
diff --git a/docs/en/docs/admin-manual/config/be-config.md b/docs/en/docs/admin-manual/config/be-config.md
index 821f81b03e..c9443394cf 100644
--- a/docs/en/docs/admin-manual/config/be-config.md
+++ b/docs/en/docs/admin-manual/config/be-config.md
@@ -7,7 +7,7 @@
 }
 ---
 
-<!-- 
+<!--
 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
@@ -246,7 +246,7 @@ There are two ways to configure BE configuration items:
 
 * Description: This configuration is mainly used to modify the parameter `socket_max_unwritten_bytes` of brpc.
   - Sometimes the query fails and an error message of `The server is overcrowded` will appear in the BE log. This means there are too many messages to buffer at the sender side, which may happen when the SQL needs to send large bitmap value. You can avoid this error by increasing the configuration.
-    
+
 #### `transfer_large_data_by_brpc`
 
 * Type: bool
@@ -279,7 +279,7 @@ There are two ways to configure BE configuration items:
 
 * Type: string
 * Description:This configuration indicates the service model used by FE's Thrift service. The type is string and is case-insensitive. This parameter needs to be consistent with the setting of fe's thrift_server_type parameter. Currently there are two values for this parameter, `THREADED` and `THREAD_POOL`.
-  
+
     - If the parameter is `THREADED`, the model is a non-blocking I/O model.
 
     - If the parameter is `THREAD_POOL`, the model is a blocking I/O model.
@@ -628,8 +628,8 @@ Metrics: {"filtered_rows":0,"input_row_num":3346807,"input_rowsets_count":42,"in
 #### `enable_segcompaction`
 
 * Type: bool
-* Description: Enable to use segment compaction during loading
-* Default value: false
+* Description: Enable to use segment compaction during loading to avoid -238 error
+* Default value: true
 
 #### `segcompaction_threshold_segment_num`
 
diff --git a/docs/en/docs/faq/data-faq.md b/docs/en/docs/faq/data-faq.md
index e8fd0b30b5..e5926c65c4 100644
--- a/docs/en/docs/faq/data-faq.md
+++ b/docs/en/docs/faq/data-faq.md
@@ -60,7 +60,7 @@ This error usually occurs during data import operations. The error code is -235.
 
 This error is usually caused by the import frequency being too high, which is greater than the compaction speed of the backend data, causing versions to pile up and eventually exceed the limit. At this point, we can first pass the show tablet 27306172 statement, and then execute the show proc statement in the result to check the status of each copy of the tablet. The versionCount in the result represents the number of versions. If you find that a copy has too many versions, you need to r [...]
 
-The -238 error usually occurs when the same batch of imported data is too large, resulting in too many Segment files for a tablet (default is 200, controlled by the BE parameter `max_segment_num_per_rowset`). At this time, it is recommended to reduce the amount of data imported in one batch, or appropriately increase the BE configuration parameter value to solve the problem.
+The -238 error usually occurs when the same batch of imported data is too large, resulting in too many Segment files for a tablet (default is 200, controlled by the BE parameter `max_segment_num_per_rowset`). At this time, it is recommended to reduce the amount of data imported in one batch, or appropriately increase the BE configuration parameter value to solve the problem. Since version 2.0, users can enable segment compaction feature to reduce segment file number by setting `enable_se [...]
 
 ### Q5. tablet 110309738 has few replicas: 1, alive backends: [10003]
 
@@ -152,7 +152,7 @@ broker_timeout_ms = 10000
 
 Adding parameters here requires restarting the FE service.
 
-### Q11. [ Routine load ] ReasonOfStateChanged: ErrorReason{code=errCode = 104, msg='be 10004 abort task with reason: fetch failed due to requested offset not available on the broker: Broker: Offset out of range'} 
+### Q11. [ Routine load ] ReasonOfStateChanged: ErrorReason{code=errCode = 104, msg='be 10004 abort task with reason: fetch failed due to requested offset not available on the broker: Broker: Offset out of range'}
 
 The reason for this problem is that Kafka's cleanup policy defaults to 7 days. When a routine load task is suspended for some reason and the task is not restored for a long time, when the task is resumed, the routine load records the consumption offset, and This problem occurs when kafka has cleaned up the corresponding offset
 
diff --git a/docs/zh-CN/docs/admin-manual/config/be-config.md b/docs/zh-CN/docs/admin-manual/config/be-config.md
index d0aeec1020..0a009d4ce3 100644
--- a/docs/zh-CN/docs/admin-manual/config/be-config.md
+++ b/docs/zh-CN/docs/admin-manual/config/be-config.md
@@ -7,7 +7,7 @@
 }
 ---
 
-<!-- 
+<!--
 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
@@ -190,7 +190,7 @@ BE 重启后该配置将失效。如果想持久化修改结果,使用如下
 * 描述:当BE启动时,会检查``storage_root_path`` 配置下的所有路径。
 
   - `ignore_broken_disk=true`
-  
+
   如果路径不存在或路径下无法进行读写文件(坏盘),将忽略此路径,如果有其他可用路径则不中断启动。
 
   - `ignore_broken_disk=false`
@@ -642,8 +642,8 @@ Metrics: {"filtered_rows":0,"input_row_num":3346807,"input_rowsets_count":42,"in
 #### `enable_segcompaction`
 
 * 类型:bool
-* 描述:在导入时进行 segment compaction 来减少 segment 数量
-* 默认值:false
+* 描述:在导入时进行 segment compaction 来减少 segment 数量, 以避免出现写入时的 -238 错误
+* 默认值:true
 
 #### `segcompaction_threshold_segment_num`
 
@@ -1303,7 +1303,7 @@ load tablets from header failed, failed tablets size: xxx, path=xxx
 #### `jvm_max_heap_size`
 
 * 类型:string
-* 描述:BE 使用 JVM 堆内存的最大值,即 JVM 的 -Xmx 参数 
+* 描述:BE 使用 JVM 堆内存的最大值,即 JVM 的 -Xmx 参数
 * 默认值:1024M
 
 </version>
diff --git a/docs/zh-CN/docs/faq/data-faq.md b/docs/zh-CN/docs/faq/data-faq.md
index 8c9cbd3d62..45c6629ff6 100644
--- a/docs/zh-CN/docs/faq/data-faq.md
+++ b/docs/zh-CN/docs/faq/data-faq.md
@@ -60,7 +60,7 @@ Unique Key模型的表是一个对业务比较友好的表,因为其特有的
 
 这个错误通常是因为导入的频率过高,大于后台数据的compaction速度,导致版本堆积并最终超过了限制。此时,我们可以先通过show tablet 27306172 语句,然后执行结果中的 show proc 语句,查看tablet各个副本的情况。结果中的 versionCount即表示版本数量。如果发现某个副本的版本数量过多,则需要降低导入频率或停止导入,并观察版本数是否有下降。如果停止导入后,版本数依然没有下降,则需要去对应的BE节点查看be.INFO日志,搜索tablet id以及 compaction关键词,检查compaction是否正常运行。关于compaction调优相关,可以参阅 ApacheDoris 公众号文章:Doris 最佳实践-Compaction调优(3)
 
--238 错误通常出现在同一批导入数据量过大的情况,从而导致某一个 tablet 的 Segment 文件过多(默认是 200,由 BE 参数 `max_segment_num_per_rowset` 控制)。此时建议减少一批次导入的数据量,或者适当提高 BE 配置参数值来解决。
+-238 错误通常出现在同一批导入数据量过大的情况,从而导致某一个 tablet 的 Segment 文件过多(默认是 200,由 BE 参数 `max_segment_num_per_rowset` 控制)。此时建议减少一批次导入的数据量,或者适当提高 BE 配置参数值来解决。在2.0版本及以后,可以通过打开 segment compaction 功能来减少 Segment 文件数量(BE config 中 enable_segcompaction=true)。
 
 ### Q5. tablet 110309738 has few replicas: 1, alive backends: [10003]
 
@@ -92,7 +92,7 @@ Unique Key模型的表是一个对业务比较友好的表,因为其特有的
 
    可以升级到 Doris 0.15 及之后的版本,已修复这个问题。
 
-### Q8. 执行导入、查询时报错-214 
+### Q8. 执行导入、查询时报错-214
 
 在执行导入、查询等操作时,可能会遇到如下错误:
 
@@ -150,7 +150,7 @@ broker_timeout_ms = 10000
 
 这里添加参数,需要重启 FE 服务。
 
-### Q11.[ Routine load ] ReasonOfStateChanged: ErrorReason{code=errCode = 104, msg='be 10004 abort task with reason: fetch failed due to requested offset not available on the broker: Broker: Offset out of range'} 
+### Q11.[ Routine load ] ReasonOfStateChanged: ErrorReason{code=errCode = 104, msg='be 10004 abort task with reason: fetch failed due to requested offset not available on the broker: Broker: Offset out of range'}
 
 出现这个问题的原因是因为kafka的清理策略默认为7天,当某个routine load任务因为某种原因导致任务暂停,长时间没有恢复,当重新恢复任务的时候routine load记录了消费的offset,而kafka的清理策略已经清理了对应的offset,就会出现这个问题
 


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


[doris] 12/36: [fix](bitmap) fix bitmap_count errors to set nullable to non-nullable bitmap col (#18689)

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

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

commit b0014fb2a9ab634f27a62583ba681ecc48d84bfd
Author: luozenglin <lu...@baidu.com>
AuthorDate: Mon Apr 17 13:23:27 2023 +0800

    [fix](bitmap) fix bitmap_count errors to set nullable to non-nullable bitmap col (#18689)
---
 be/src/vec/functions/function_bitmap.cpp                       | 2 +-
 be/src/vec/functions/function_bitmap_min_or_max.h              | 2 +-
 regression-test/data/query_p0/join/test_bitmap_filter.out      | 9 +++++++++
 regression-test/suites/query_p0/join/test_bitmap_filter.groovy | 2 ++
 4 files changed, 13 insertions(+), 2 deletions(-)

diff --git a/be/src/vec/functions/function_bitmap.cpp b/be/src/vec/functions/function_bitmap.cpp
index d6cf1652db..7d340f6937 100644
--- a/be/src/vec/functions/function_bitmap.cpp
+++ b/be/src/vec/functions/function_bitmap.cpp
@@ -412,7 +412,7 @@ public:
         auto data_null_map = ColumnUInt8::create(input_rows_count, 0);
         auto& null_map = data_null_map->get_data();
 
-        auto& column = block.get_by_position(arguments[0]).column;
+        auto column = block.get_by_position(arguments[0]).column;
         if (auto* nullable = check_and_get_column<const ColumnNullable>(*column)) {
             VectorizedUtils::update_null_map(null_map, nullable->get_null_map_data());
             column = nullable->get_nested_column_ptr();
diff --git a/be/src/vec/functions/function_bitmap_min_or_max.h b/be/src/vec/functions/function_bitmap_min_or_max.h
index 788f8222a5..4f7018292d 100644
--- a/be/src/vec/functions/function_bitmap_min_or_max.h
+++ b/be/src/vec/functions/function_bitmap_min_or_max.h
@@ -56,7 +56,7 @@ public:
         auto result_column = ColumnInt64::create();
         auto result_null_map_column = ColumnUInt8::create(input_rows_count, 0);
 
-        ColumnPtr& argument_column = block.get_by_position(arguments[0]).column;
+        ColumnPtr argument_column = block.get_by_position(arguments[0]).column;
         if (auto* nullable = check_and_get_column<ColumnNullable>(*argument_column)) {
             // Danger: Here must dispose the null map data first! Because
             // argument_columns[i]=nullable->get_nested_column_ptr(); will release the mem
diff --git a/regression-test/data/query_p0/join/test_bitmap_filter.out b/regression-test/data/query_p0/join/test_bitmap_filter.out
index 1ad7849124..925421cbe2 100644
--- a/regression-test/data/query_p0/join/test_bitmap_filter.out
+++ b/regression-test/data/query_p0/join/test_bitmap_filter.out
@@ -138,3 +138,12 @@
 14	255
 15	1992
 
+-- !sql19 --
+1
+3
+5
+7
+9
+11
+13
+
diff --git a/regression-test/suites/query_p0/join/test_bitmap_filter.groovy b/regression-test/suites/query_p0/join/test_bitmap_filter.groovy
index fe18451978..d6286e91bd 100644
--- a/regression-test/suites/query_p0/join/test_bitmap_filter.groovy
+++ b/regression-test/suites/query_p0/join/test_bitmap_filter.groovy
@@ -77,6 +77,8 @@ suite("test_bitmap_filter", "query_p0") {
 
     qt_sql18 "select k1, k2 from ${tbl1} t where 100 not in (select k2 from ${tbl2}) order by 1, 2;"
 
+    qt_sql19 "select k1 from ${tbl1} t where k1 in (select k2 from ${tbl2} where bitmap_count(k2) > 6) order by 1;"
+
     test {
         sql "select k1, k2 from ${tbl1} b1 where k1 in (select k2 from ${tbl2} b2 where b1.k2 = b2.k1) order by k1;"
         exception "In bitmap does not support correlated subquery"


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


[doris] 18/36: [Compile](BE) Fix compile failed with tcmalloc (#18748)

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

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

commit cf224e932d3e89ec08e10d522907baf0c55412ab
Author: HappenLee <ha...@hotmail.com>
AuthorDate: Tue Apr 18 09:26:45 2023 +0800

    [Compile](BE) Fix compile failed with tcmalloc (#18748)
---
 be/src/common/daemon.cpp             | 4 +---
 be/src/exec/data_sink.cpp            | 8 ++++----
 be/src/http/action/pprof_actions.cpp | 5 ++++-
 be/src/service/doris_main.cpp        | 2 +-
 4 files changed, 10 insertions(+), 9 deletions(-)

diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp
index 4a64e67053..6ef0d751f3 100644
--- a/be/src/common/daemon.cpp
+++ b/be/src/common/daemon.cpp
@@ -19,6 +19,7 @@
 
 #include <bthread/errno.h>
 #include <gflags/gflags.h>
+#include <gperftools/malloc_extension.h> // IWYU pragma: keep
 // IWYU pragma: no_include <bits/std_abs.h>
 #include <math.h>
 #include <signal.h>
@@ -40,7 +41,6 @@
 #include "common/status.h"
 #include "olap/options.h"
 #include "olap/storage_engine.h"
-#include "olap/tablet_manager.h"
 #include "runtime/block_spill_manager.h"
 #include "runtime/exec_env.h"
 #include "runtime/load_channel_mgr.h"
@@ -53,10 +53,8 @@
 #include "util/disk_info.h"
 #include "util/doris_metrics.h"
 #include "util/mem_info.h"
-#include "util/metrics.h"
 #include "util/network_util.h"
 #include "util/perf_counters.h"
-#include "util/system_metrics.h"
 #include "util/thrift_util.h"
 #include "util/time.h"
 
diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp
index fd8406a355..f93aa7bb6f 100644
--- a/be/src/exec/data_sink.cpp
+++ b/be/src/exec/data_sink.cpp
@@ -119,8 +119,8 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink
         if (!thrift_sink.__isset.mysql_table_sink) {
             return Status::InternalError("Missing data buffer sink.");
         }
-        doris::vectorized::VMysqlTableSink* vmysql_tbl_sink =
-                new doris::vectorized::VMysqlTableSink(pool, row_desc, output_exprs);
+        vectorized::VMysqlTableSink* vmysql_tbl_sink =
+                new vectorized::VMysqlTableSink(pool, row_desc, output_exprs);
         sink->reset(vmysql_tbl_sink);
         break;
 #else
@@ -260,8 +260,8 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink
         if (!thrift_sink.__isset.mysql_table_sink) {
             return Status::InternalError("Missing data buffer sink.");
         }
-        doris::vectorized::VMysqlTableSink* vmysql_tbl_sink =
-                new doris::vectorized::VMysqlTableSink(pool, row_desc, output_exprs);
+        vectorized::VMysqlTableSink* vmysql_tbl_sink =
+                new vectorized::VMysqlTableSink(pool, row_desc, output_exprs);
         sink->reset(vmysql_tbl_sink);
         break;
 #else
diff --git a/be/src/http/action/pprof_actions.cpp b/be/src/http/action/pprof_actions.cpp
index 7696078a4d..56ae5e49e9 100644
--- a/be/src/http/action/pprof_actions.cpp
+++ b/be/src/http/action/pprof_actions.cpp
@@ -17,12 +17,14 @@
 
 #include "http/action/pprof_actions.h"
 
+#include <gperftools/heap-profiler.h>    // IWYU pragma: keep
+#include <gperftools/malloc_extension.h> // IWYU pragma: keep
+#include <gperftools/profiler.h>         // IWYU pragma: keep
 #include <stdio.h>
 
 #include <fstream>
 #include <memory>
 #include <mutex>
-#include <string>
 
 #include "common/config.h"
 #include "common/object_pool.h"
@@ -34,6 +36,7 @@
 #include "io/fs/local_file_system.h"
 #include "runtime/exec_env.h"
 #include "util/bfd_parser.h"
+#include "util/pprof_utils.h" // IWYU pragma: keep
 
 namespace doris {
 
diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp
index ab91661b5f..be10089eae 100644
--- a/be/src/service/doris_main.cpp
+++ b/be/src/service/doris_main.cpp
@@ -19,6 +19,7 @@
 #include <butil/macros.h>
 #include <errno.h>
 #include <fcntl.h>
+#include <gperftools/malloc_extension.h> // IWYU pragma: keep
 #include <libgen.h>
 #include <setjmp.h>
 #include <signal.h>
@@ -27,7 +28,6 @@
 #include <stdlib.h>
 #include <unistd.h>
 
-#include <algorithm>
 #include <cstring>
 #include <ostream>
 #include <string>


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


[doris] 30/36: [typo](docs) fix insert load doc error (#18773)

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

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

commit 4f1e13620671a039a80c490845e628f1fffed3f5
Author: Jiwen liu <61...@users.noreply.github.com>
AuthorDate: Tue Apr 18 21:13:23 2023 +0800

    [typo](docs) fix insert load doc error (#18773)
---
 docs/zh-CN/docs/data-operate/import/import-scenes/jdbc-load.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/zh-CN/docs/data-operate/import/import-scenes/jdbc-load.md b/docs/zh-CN/docs/data-operate/import/import-scenes/jdbc-load.md
index c5470cc318..3c4e04ce18 100644
--- a/docs/zh-CN/docs/data-operate/import/import-scenes/jdbc-load.md
+++ b/docs/zh-CN/docs/data-operate/import/import-scenes/jdbc-load.md
@@ -150,7 +150,7 @@ public class DorisJDBCDemo {
 
 2. 批次大小
 
-   因为是在客户端进行批量处理,因此一批次如果过大的话,话占用客户端的内存资源,需关注。
+   因为是在客户端进行批量处理,所以一批次过大的话,会占用客户端的内存资源,需关注。
 
    Doris 后续会支持服务端的 PrepareStatemnt,敬请期待。
 


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


[doris] 26/36: [fix](merge-on-write) enable_unique_key_merge_on_write property should only be used for unique table (#18734)

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

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

commit 23a3229d9adcc88a40d1b77126074ddc0b171058
Author: Xin Liao <li...@126.com>
AuthorDate: Tue Apr 18 18:40:01 2023 +0800

    [fix](merge-on-write) enable_unique_key_merge_on_write property should only be used for unique table (#18734)
---
 .../apache/doris/datasource/InternalCatalog.java   | 12 +--
 .../suites/ddl_p0/test_create_table.groovy         |  3 +-
 .../unique_with_mow_p0/test_create_table.groovy    | 97 ++++++++++++++++++++++
 3 files changed, 105 insertions(+), 7 deletions(-)

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 292723e091..290eddf9c1 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
@@ -1956,11 +1956,13 @@ public class InternalCatalog implements CatalogIf<Database> {
                 keysDesc.keysColumnSize(), storageFormat);
         olapTable.setDataSortInfo(dataSortInfo);
 
-        boolean enableUniqueKeyMergeOnWrite;
-        try {
-            enableUniqueKeyMergeOnWrite = PropertyAnalyzer.analyzeUniqueKeyMergeOnWrite(properties);
-        } catch (AnalysisException e) {
-            throw new DdlException(e.getMessage());
+        boolean enableUniqueKeyMergeOnWrite = false;
+        if (keysType == KeysType.UNIQUE_KEYS) {
+            try {
+                enableUniqueKeyMergeOnWrite = PropertyAnalyzer.analyzeUniqueKeyMergeOnWrite(properties);
+            } catch (AnalysisException e) {
+                throw new DdlException(e.getMessage());
+            }
         }
         olapTable.setEnableUniqueKeyMergeOnWrite(enableUniqueKeyMergeOnWrite);
 
diff --git a/regression-test/suites/ddl_p0/test_create_table.groovy b/regression-test/suites/ddl_p0/test_create_table.groovy
index dd7340cbbb..a022f51e52 100644
--- a/regression-test/suites/ddl_p0/test_create_table.groovy
+++ b/regression-test/suites/ddl_p0/test_create_table.groovy
@@ -42,8 +42,7 @@ suite("sql_create_time_range_table") {
 	properties(
 		"replication_num"="1",
 		"light_schema_change"="true",
-		"compression"="zstd",
-		"enable_unique_key_merge_on_write" = "true"
+		"compression"="zstd"
 	);
 		"""
 
diff --git a/regression-test/suites/unique_with_mow_p0/test_create_table.groovy b/regression-test/suites/unique_with_mow_p0/test_create_table.groovy
new file mode 100644
index 0000000000..9a3ad4b2a5
--- /dev/null
+++ b/regression-test/suites/unique_with_mow_p0/test_create_table.groovy
@@ -0,0 +1,97 @@
+// 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_create_table") {
+    def tableName = "unique_mow_create_table"
+    sql """ DROP TABLE IF EXISTS ${tableName} """
+
+    // duplicate table with enable_unique_key_merge_on_write property
+    test {
+        sql """
+            CREATE TABLE `$tableName` (
+                    `c_custkey` int(11) NOT NULL COMMENT "",
+                    `c_name` varchar(26) NOT NULL COMMENT "",
+                    `c_address` varchar(41) NOT NULL COMMENT "",
+                    `c_city` varchar(11) NOT NULL COMMENT ""
+            )
+            DUPLICATE KEY (`c_custkey`)
+            DISTRIBUTED BY HASH(`c_custkey`) BUCKETS 1
+            PROPERTIES (
+                    "replication_num" = "1",
+                    "enable_unique_key_merge_on_write" = "true"
+             );
+        """
+        exception "Unknown properties"
+    }
+
+    // duplicate table with enable_unique_key_merge_on_write property
+    test {
+        sql """
+            CREATE TABLE `$tableName` (
+                    `c_custkey` int(11) NOT NULL COMMENT "",
+                    `c_name` varchar(26) NOT NULL COMMENT "",
+                    `c_address` varchar(41) NOT NULL COMMENT "",
+                    `c_city` varchar(11) NOT NULL COMMENT ""
+            )
+            DUPLICATE KEY (`c_custkey`)
+            DISTRIBUTED BY HASH(`c_custkey`) BUCKETS 1
+            PROPERTIES (
+                    "replication_num" = "1",
+                    "enable_unique_key_merge_on_write" = "false"
+             );
+        """
+        exception "Unknown properties"
+    }
+
+    // agg table with enable_unique_key_merge_on_write property
+    test {
+        sql """
+            CREATE TABLE `$tableName` (
+                    `c_custkey` int(11) NOT NULL COMMENT "",
+                    `c_name` varchar(26) max NOT NULL COMMENT "",
+                    `c_address` varchar(41) max NOT NULL COMMENT "",
+                    `c_city` varchar(11) max NOT NULL COMMENT ""
+            )
+            AGGREGATE KEY (`c_custkey`)
+            DISTRIBUTED BY HASH(`c_custkey`) BUCKETS 1
+            PROPERTIES (
+                    "replication_num" = "1",
+                    "enable_unique_key_merge_on_write" = "true"
+             );
+        """
+        exception "Unknown properties"
+    }
+
+    // agg table with enable_unique_key_merge_on_write property
+    test {
+        sql """
+            CREATE TABLE `$tableName` (
+                    `c_custkey` int(11) NOT NULL COMMENT "",
+                    `c_name` varchar(26) max NOT NULL COMMENT "",
+                    `c_address` varchar(41) max NOT NULL COMMENT "",
+                    `c_city` varchar(11) max NOT NULL COMMENT ""
+            )
+            AGGREGATE KEY (`c_custkey`)
+            DISTRIBUTED BY HASH(`c_custkey`) BUCKETS 1
+            PROPERTIES (
+                    "replication_num" = "1",
+                    "enable_unique_key_merge_on_write" = "false"
+             );
+        """
+        exception "Unknown properties"
+    }
+}
\ 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] 04/36: [fix](executor)Fix scanner's _max_thread_num may == 0 #18465

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

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

commit e4d11ca3ad74dc49abaf57699cda9d8e9ddc412a
Author: wangbo <wa...@apache.org>
AuthorDate: Sun Apr 16 18:17:18 2023 +0800

    [fix](executor)Fix scanner's _max_thread_num may == 0 #18465
---
 be/src/vec/exec/scan/scanner_context.cpp | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp
index a42044f4c1..1c1cd3f73f 100644
--- a/be/src/vec/exec/scan/scanner_context.cpp
+++ b/be/src/vec/exec/scan/scanner_context.cpp
@@ -57,6 +57,8 @@ Status ScannerContext::init() {
     // should find a more reasonable value.
     _max_thread_num = _state->shared_scan_opt() ? config::doris_scanner_thread_pool_thread_num
                                                 : config::doris_scanner_thread_pool_thread_num / 4;
+    _max_thread_num = _max_thread_num == 0 ? 1 : _max_thread_num;
+    DCHECK(_max_thread_num > 0);
     _max_thread_num = std::min(_max_thread_num, (int32_t)_scanners.size());
     // For select * from table limit 10; should just use one thread.
     if (_parent->should_run_serial()) {


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


[doris] 25/36: [fix](stats) Stats still in cache after user dropped it (#18720)

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

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

commit 1f08b772008edd8010df117c487924fecd14738d
Author: AKIRA <33...@users.noreply.github.com>
AuthorDate: Tue Apr 18 17:41:10 2023 +0900

    [fix](stats) Stats still in cache after user dropped it (#18720)
    
    1. Evict the dropped stats from cache
    2. Remove codes for the partition level stats collection
    3. Disable analyze whole database directly
    4. Fix the potential death loop in the stats cleaner
    5. Sleep thread in each loop when scanning stats table to avoid excessive IO usage by this task.
---
 fe/fe-core/src/main/cup/sql_parser.cup             |  14 +--
 .../org/apache/doris/analysis/AnalyzeStmt.java     |  40 --------
 .../org/apache/doris/analysis/DropStatsStmt.java   | 114 +++++++--------------
 .../apache/doris/statistics/AnalysisManager.java   |  36 +++----
 .../apache/doris/statistics/AnalysisTaskInfo.java  |   7 +-
 .../doris/statistics/AnalysisTaskInfoBuilder.java  |  10 +-
 .../org/apache/doris/statistics/HistogramTask.java |   2 +-
 .../apache/doris/statistics/MVAnalysisTask.java    |   2 +-
 .../apache/doris/statistics/OlapAnalysisTask.java  |   2 +-
 .../doris/statistics/StatisticConstants.java       |   1 +
 .../apache/doris/statistics/StatisticsCache.java   |   3 +-
 .../apache/doris/statistics/StatisticsCleaner.java |   8 +-
 .../doris/statistics/StatisticsRepository.java     |  63 +++---------
 .../apache/doris/statistics/AnalysisJobTest.java   |   2 -
 .../doris/statistics/AnalysisTaskExecutorTest.java |   2 -
 .../apache/doris/statistics/HistogramTaskTest.java |   3 -
 regression-test/data/statistics/analyze_test.out   |   3 +
 .../suites/statistics/analyze_test.groovy          |  43 ++++++--
 18 files changed, 117 insertions(+), 238 deletions(-)

diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index a860b74d4f..90a254302d 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -2812,30 +2812,30 @@ show_create_reporitory_stmt ::=
 
 // analyze statment
 analyze_stmt ::=
-    KW_ANALYZE opt_sync:sync KW_TABLE table_name:tbl opt_col_list:cols opt_partition_names:partitionNames opt_properties:properties
+    KW_ANALYZE opt_sync:sync KW_TABLE table_name:tbl opt_col_list:cols opt_properties:properties
     {:
         boolean is_whole_tbl = (cols == null);
         boolean is_histogram = false;
         boolean is_increment = false;
-        RESULT = new AnalyzeStmt(tbl, sync, cols, partitionNames, properties, is_whole_tbl, is_histogram, is_increment);
+        RESULT = new AnalyzeStmt(tbl, sync, cols, properties, is_whole_tbl, is_histogram, is_increment);
     :}
     | KW_ANALYZE opt_sync:sync KW_INCREMENTAL KW_TABLE table_name:tbl opt_col_list:cols opt_partition_names:partitionNames opt_properties:properties
     {:
         boolean is_whole_tbl = (cols == null);
         boolean is_histogram = false;
         boolean is_increment = true;
-        RESULT = new AnalyzeStmt(tbl, sync, cols, partitionNames, properties, is_whole_tbl, is_histogram, is_increment);
+        RESULT = new AnalyzeStmt(tbl, sync, cols, properties, is_whole_tbl, is_histogram, is_increment);
     :}
     | KW_ANALYZE opt_sync:sync KW_TABLE table_name:tbl KW_UPDATE KW_HISTOGRAM KW_ON ident_list:cols opt_partition_names:partitionNames opt_properties:properties
     {:
         boolean is_whole_tbl = false;
         boolean is_histogram = true;
         boolean is_increment = false;
-        RESULT = new AnalyzeStmt(tbl, sync, cols, partitionNames, properties, is_whole_tbl, is_histogram, is_increment);
+        RESULT = new AnalyzeStmt(tbl, sync, cols, properties, is_whole_tbl, is_histogram, is_increment);
     :}
     | KW_ANALYZE opt_sync:sync KW_TABLE table_name:tbl KW_UPDATE KW_HISTOGRAM
     {:
-        RESULT = new AnalyzeStmt(tbl, sync, null, null, new HashMap<>(), true, true, false);
+        RESULT = new AnalyzeStmt(tbl, sync, null, new HashMap<>(), true, true, false);
     :}
     ;
 
@@ -3014,9 +3014,9 @@ drop_stmt ::=
         RESULT = new DropPolicyStmt(PolicyTypeEnum.STORAGE, ifExists, policyName, null, null);
     :}
     /* statistics */
-    | KW_DROP KW_STATS opt_table_name:tbl opt_col_list:cols opt_partition_names:partitionNames
+    | KW_DROP KW_STATS table_name:tbl opt_col_list:cols
     {:
-        RESULT = new DropStatsStmt(tbl, partitionNames, cols);
+        RESULT = new DropStatsStmt(tbl, cols);
     :}
     | KW_DROP KW_EXPIRED KW_STATS
     {:
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeStmt.java
index 61a45682d6..485aec8d37 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeStmt.java
@@ -21,8 +21,6 @@ import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.DatabaseIf;
 import org.apache.doris.catalog.Env;
-import org.apache.doris.catalog.OlapTable;
-import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.TableIf;
 import org.apache.doris.catalog.View;
 import org.apache.doris.common.AnalysisException;
@@ -75,7 +73,6 @@ public class AnalyzeStmt extends DdlStmt {
     private final TableName tableName;
 
     private final boolean sync;
-    private final PartitionNames partitionNames;
     private final List<String> columnNames;
     private final Map<String, String> properties;
 
@@ -86,7 +83,6 @@ public class AnalyzeStmt extends DdlStmt {
     public AnalyzeStmt(TableName tableName,
                        boolean sync,
                        List<String> columnNames,
-                       PartitionNames partitionNames,
                        Map<String, String> properties,
                        Boolean isWholeTbl,
                        Boolean isHistogram,
@@ -94,7 +90,6 @@ public class AnalyzeStmt extends DdlStmt {
         this.tableName = tableName;
         this.sync = sync;
         this.columnNames = columnNames;
-        this.partitionNames = partitionNames;
         this.properties = properties;
         this.isWholeTbl = isWholeTbl;
         this.isHistogram = isHistogram;
@@ -137,8 +132,6 @@ public class AnalyzeStmt extends DdlStmt {
             }
         }
 
-        checkPartitionNames();
-
         checkProperties();
     }
 
@@ -159,29 +152,6 @@ public class AnalyzeStmt extends DdlStmt {
         }
     }
 
-    private void checkPartitionNames() throws AnalysisException {
-        if (partitionNames != null) {
-            partitionNames.analyze(analyzer);
-            Database db = analyzer.getEnv().getInternalCatalog()
-                    .getDbOrAnalysisException(tableName.getDb());
-            OlapTable olapTable = (OlapTable) db.getTableOrAnalysisException(tableName.getTbl());
-            if (!olapTable.isPartitioned()) {
-                throw new AnalysisException("Not a partitioned table: " + olapTable.getName());
-            }
-            List<String> names = partitionNames.getPartitionNames();
-            Set<String> olapPartitionNames = olapTable.getPartitionNames();
-            List<String> tempPartitionNames = olapTable.getTempPartitions().stream()
-                    .map(Partition::getName).collect(Collectors.toList());
-            Optional<String> illegalPartitionName = names.stream()
-                    .filter(name -> (tempPartitionNames.contains(name)
-                            || !olapPartitionNames.contains(name)))
-                    .findFirst();
-            if (illegalPartitionName.isPresent()) {
-                throw new AnalysisException("Temporary partition or partition does not exist");
-            }
-        }
-    }
-
     private void checkProperties() throws UserException {
         if (properties != null) {
             Optional<String> optional = properties.keySet().stream().filter(
@@ -227,11 +197,6 @@ public class AnalyzeStmt extends DdlStmt {
                 .stream().map(Column::getName).collect(Collectors.toSet()) : Sets.newHashSet(columnNames);
     }
 
-    public Set<String> getPartitionNames() {
-        return partitionNames == null ? Sets.newHashSet(table.getPartitionNames())
-                : Sets.newHashSet(partitionNames.getPartitionNames());
-    }
-
     public Map<String, String> getProperties() {
         // TODO add default properties
         return properties != null ? properties : Maps.newHashMap();
@@ -263,11 +228,6 @@ public class AnalyzeStmt extends DdlStmt {
             sb.append(")");
         }
 
-        if (partitionNames != null) {
-            sb.append(" ");
-            sb.append(partitionNames.toSql());
-        }
-
         if (properties != null) {
             sb.append(" ");
             sb.append("PROPERTIES(");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropStatsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropStatsStmt.java
index da67165bf8..137db5e310 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropStatsStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropStatsStmt.java
@@ -17,10 +17,9 @@
 
 package org.apache.doris.analysis;
 
-import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.DatabaseIf;
 import org.apache.doris.catalog.Env;
-import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.TableIf;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.ErrorCode;
@@ -31,9 +30,9 @@ import org.apache.doris.datasource.CatalogIf;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.qe.ConnectContext;
 
-import com.google.common.collect.Sets;
-import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.StringUtils;
 
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
@@ -51,26 +50,20 @@ public class DropStatsStmt extends DdlStmt {
     public final boolean dropExpired;
 
     private final TableName tableName;
-    private final PartitionNames partitionNames;
-    private final List<String> columnNames;
+    private Set<String> columnNames;
 
-    // after analyzed
-    private long dbId;
-    private final Set<Long> tbIds = Sets.newHashSet();
-    private final Set<Long> partitionIds = Sets.newHashSet();
+    private long tblId;
 
     public DropStatsStmt(boolean dropExpired) {
         this.dropExpired = dropExpired;
         this.tableName = null;
-        this.partitionNames = null;
         this.columnNames = null;
     }
 
     public DropStatsStmt(TableName tableName,
-            PartitionNames partitionNames, List<String> columnNames) {
+            List<String> columnNames) {
         this.tableName = tableName;
-        this.partitionNames = partitionNames;
-        this.columnNames = columnNames;
+        this.columnNames = new HashSet<>(columnNames);
         dropExpired = false;
     }
 
@@ -80,75 +73,43 @@ public class DropStatsStmt extends DdlStmt {
         if (dropExpired) {
             return;
         }
-        if (tableName != null) {
-            tableName.analyze(analyzer);
-
-            String catalogName = tableName.getCtl();
-            String dbName = tableName.getDb();
-            String tblName = tableName.getTbl();
-            CatalogIf catalog = analyzer.getEnv().getCatalogMgr()
-                    .getCatalogOrAnalysisException(catalogName);
-            DatabaseIf db = catalog.getDbOrAnalysisException(dbName);
-            TableIf table = db.getTableOrAnalysisException(tblName);
-
-            dbId = db.getId();
-            tbIds.add(table.getId());
-
-            // disallow external catalog
-            Util.prohibitExternalCatalog(tableName.getCtl(),
-                    this.getClass().getSimpleName());
-
-            // check permission
-            checkAnalyzePriv(db.getFullName(), table.getName());
-
-            // check partitionNames
-            if (partitionNames != null) {
-                partitionNames.analyze(analyzer);
-                partitionIds.addAll(partitionNames.getPartitionNames().stream()
-                        .map(name -> table.getPartition(name).getId())
-                        .collect(Collectors.toList()));
-            }
-
-            // check columnNames
-            if (columnNames != null) {
-                for (String cName : columnNames) {
-                    if (table.getColumn(cName) == null) {
-                        ErrorReport.reportAnalysisException(
-                                ErrorCode.ERR_WRONG_COLUMN_NAME,
-                                "DROP",
-                                ConnectContext.get().getQualifiedUser(),
-                                ConnectContext.get().getRemoteIP(),
-                                cName);
-                    }
+        tableName.analyze(analyzer);
+        String catalogName = tableName.getCtl();
+        String dbName = tableName.getDb();
+        String tblName = tableName.getTbl();
+        CatalogIf catalog = analyzer.getEnv().getCatalogMgr()
+                .getCatalogOrAnalysisException(catalogName);
+        DatabaseIf db = catalog.getDbOrAnalysisException(dbName);
+        TableIf table = db.getTableOrAnalysisException(tblName);
+        tblId = table.getId();
+        // disallow external catalog
+        Util.prohibitExternalCatalog(tableName.getCtl(),
+                this.getClass().getSimpleName());
+        // check permission
+        checkAnalyzePriv(db.getFullName(), table.getName());
+        // check columnNames
+        if (columnNames != null) {
+            for (String cName : columnNames) {
+                if (table.getColumn(cName) == null) {
+                    ErrorReport.reportAnalysisException(
+                            ErrorCode.ERR_WRONG_COLUMN_NAME,
+                            "DROP",
+                            ConnectContext.get().getQualifiedUser(),
+                            ConnectContext.get().getRemoteIP(),
+                            cName);
                 }
             }
         } else {
-            Database db = analyzer.getEnv().getInternalCatalog()
-                    .getDbOrAnalysisException(analyzer.getDefaultDb());
-            List<Table> tables = db.getTables();
-            for (Table table : tables) {
-                checkAnalyzePriv(db.getFullName(), table.getName());
-            }
-
-            dbId = db.getId();
-            tbIds.addAll(tables.stream().map(Table::getId).collect(Collectors.toList()));
+            columnNames = table.getColumns().stream().map(Column::getName).collect(Collectors.toSet());
         }
     }
 
-    public long getDbId() {
-        return dbId;
-    }
-
-    public Set<Long> getTbIds() {
-        return tbIds;
-    }
-
-    public Set<Long> getPartitionIds() {
-        return partitionIds;
+    public long getTblId() {
+        return tblId;
     }
 
     public Set<String> getColumnNames() {
-        return columnNames != null ? Sets.newHashSet(columnNames) : Sets.newHashSet();
+        return columnNames;
     }
 
     @Override
@@ -166,11 +127,6 @@ public class DropStatsStmt extends DdlStmt {
             sb.append(")");
         }
 
-        if (partitionNames != null) {
-            sb.append(" ");
-            sb.append(partitionNames.toSql());
-        }
-
         return sb.toString();
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java
index a0171cc3c0..433135d5c8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java
@@ -37,7 +37,6 @@ import org.apache.doris.statistics.util.StatisticsUtil;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.text.StringSubstitutor;
 import org.apache.logging.log4j.LogManager;
@@ -53,7 +52,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.stream.Collectors;
 
 public class AnalysisManager {
 
@@ -98,23 +96,10 @@ public class AnalysisManager {
         TableName tbl = analyzeStmt.getTblName();
         StatisticsUtil.convertTableNameToObjects(tbl);
         Set<String> colNames = analyzeStmt.getColumnNames();
-        Set<String> partitionNames = analyzeStmt.getPartitionNames();
         Map<Long, AnalysisTaskInfo> analysisTaskInfos = new HashMap<>();
         long jobId = Env.getCurrentEnv().getNextId();
-        // If the analysis is not incremental, need to delete existing statistics.
-        // we cannot collect histograms incrementally and do not support it
-        if (!analyzeStmt.isIncrement && !analyzeStmt.isHistogram) {
-            long dbId = analyzeStmt.getDbId();
-            TableIf table = analyzeStmt.getTable();
-            Set<Long> tblIds = Sets.newHashSet(table.getId());
-            Set<Long> partIds = partitionNames.stream()
-                    .map(p -> table.getPartition(p).getId())
-                    .collect(Collectors.toSet());
-            StatisticsRepository.dropStatistics(dbId, tblIds, colNames, partIds);
-        }
-
-        createTaskForEachColumns(analyzeStmt, catalogName, db, tbl, colNames, partitionNames, analysisTaskInfos, jobId);
-        createTaskForMVIdx(analyzeStmt, catalogName, db, tbl, partitionNames, analysisTaskInfos, jobId);
+        createTaskForEachColumns(analyzeStmt, catalogName, db, tbl, colNames, analysisTaskInfos, jobId);
+        createTaskForMVIdx(analyzeStmt, catalogName, db, tbl, analysisTaskInfos, jobId);
         persistAnalysisJob(catalogName, db, tbl, jobId);
 
         if (analyzeStmt.isSync()) {
@@ -143,7 +128,7 @@ public class AnalysisManager {
     }
 
     private void createTaskForMVIdx(AnalyzeStmt analyzeStmt, String catalogName, String db, TableName tbl,
-            Set<String> partitionNames, Map<Long, AnalysisTaskInfo> analysisTaskInfos, long jobId) throws DdlException {
+            Map<Long, AnalysisTaskInfo> analysisTaskInfos, long jobId) throws DdlException {
         if (!(analyzeStmt.isWholeTbl && analyzeStmt.getTable().getType().equals(TableType.OLAP))) {
             return;
         }
@@ -158,7 +143,7 @@ public class AnalysisManager {
                 AnalysisTaskInfo analysisTaskInfo = new AnalysisTaskInfoBuilder().setJobId(
                                 jobId).setTaskId(taskId)
                         .setCatalogName(catalogName).setDbName(db)
-                        .setTblName(tbl.getTbl()).setPartitionNames(partitionNames)
+                        .setTblName(tbl.getTbl())
                         .setIndexId(meta.getIndexId()).setJobType(JobType.MANUAL)
                         .setAnalysisMethod(AnalysisMethod.FULL).setAnalysisType(AnalysisType.INDEX)
                         .setScheduleType(ScheduleType.ONCE).build();
@@ -175,7 +160,7 @@ public class AnalysisManager {
     }
 
     private void createTaskForEachColumns(AnalyzeStmt analyzeStmt, String catalogName, String db, TableName tbl,
-            Set<String> colNames, Set<String> partitionNames, Map<Long, AnalysisTaskInfo> analysisTaskInfos,
+            Set<String> colNames, Map<Long, AnalysisTaskInfo> analysisTaskInfos,
             long jobId) throws DdlException {
         for (String colName : colNames) {
             long taskId = Env.getCurrentEnv().getNextId();
@@ -183,7 +168,7 @@ public class AnalysisManager {
             AnalysisTaskInfo analysisTaskInfo = new AnalysisTaskInfoBuilder().setJobId(jobId)
                     .setTaskId(taskId).setCatalogName(catalogName).setDbName(db)
                     .setTblName(tbl.getTbl()).setColName(colName)
-                    .setPartitionNames(partitionNames).setJobType(JobType.MANUAL)
+                    .setJobType(JobType.MANUAL)
                     .setAnalysisMethod(AnalysisMethod.FULL).setAnalysisType(analType)
                     .setState(AnalysisState.PENDING)
                     .setScheduleType(ScheduleType.ONCE).build();
@@ -271,12 +256,17 @@ public class AnalysisManager {
         }
     }
 
-    public void dropStats(DropStatsStmt dropStatsStmt) {
+    public void dropStats(DropStatsStmt dropStatsStmt) throws DdlException {
         if (dropStatsStmt.dropExpired) {
             Env.getCurrentEnv().getStatisticsCleaner().clear();
             return;
         }
-        StatisticsRepository.dropTableStatistics(dropStatsStmt);
+        Set<String> cols = dropStatsStmt.getColumnNames();
+        long tblId = dropStatsStmt.getTblId();
+        StatisticsRepository.dropStatistics(tblId, cols);
+        for (String col : cols) {
+            Env.getCurrentEnv().getStatisticsCache().invidate(tblId, -1L, col);
+        }
     }
 
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfo.java
index def16de41c..004d51b55e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfo.java
@@ -20,7 +20,6 @@ package org.apache.doris.statistics;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import java.util.Set;
 import java.util.StringJoiner;
 
 public class AnalysisTaskInfo {
@@ -63,8 +62,6 @@ public class AnalysisTaskInfo {
 
     public final String colName;
 
-    public final Set<String> partitionNames;
-
     public final Long indexId;
 
     public final JobType jobType;
@@ -87,7 +84,7 @@ public class AnalysisTaskInfo {
     public final ScheduleType scheduleType;
 
     public AnalysisTaskInfo(long jobId, long taskId, String catalogName, String dbName, String tblName,
-            String colName, Set<String> partitionNames, Long indexId, JobType jobType,
+            String colName, Long indexId, JobType jobType,
             AnalysisMethod analysisMethod, AnalysisType analysisType, String message,
             int lastExecTimeInMs, AnalysisState state, ScheduleType scheduleType) {
         this.jobId = jobId;
@@ -96,7 +93,6 @@ public class AnalysisTaskInfo {
         this.dbName = dbName;
         this.tblName = tblName;
         this.colName = colName;
-        this.partitionNames = partitionNames;
         this.indexId = indexId;
         this.jobType = jobType;
         this.analysisMethod = analysisMethod;
@@ -115,7 +111,6 @@ public class AnalysisTaskInfo {
         sj.add("DBName: " + dbName);
         sj.add("TableName: " + tblName);
         sj.add("ColumnName: " + colName);
-        sj.add("PartitionNames: " + partitionNames);
         sj.add("TaskType: " + analysisType.toString());
         sj.add("TaskMethod: " + analysisMethod.toString());
         sj.add("Message: " + message);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfoBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfoBuilder.java
index e687d6fe93..e804388153 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfoBuilder.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisTaskInfoBuilder.java
@@ -22,8 +22,6 @@ import org.apache.doris.statistics.AnalysisTaskInfo.AnalysisType;
 import org.apache.doris.statistics.AnalysisTaskInfo.JobType;
 import org.apache.doris.statistics.AnalysisTaskInfo.ScheduleType;
 
-import java.util.Set;
-
 public class AnalysisTaskInfoBuilder {
     private long jobId;
     private long taskId;
@@ -31,7 +29,6 @@ public class AnalysisTaskInfoBuilder {
     private String dbName;
     private String tblName;
     private String colName;
-    private Set<String> partitionNames;
     private Long indexId;
     private JobType jobType;
     private AnalysisMethod analysisMethod;
@@ -71,11 +68,6 @@ public class AnalysisTaskInfoBuilder {
         return this;
     }
 
-    public AnalysisTaskInfoBuilder setPartitionNames(Set<String> partitionNames) {
-        this.partitionNames = partitionNames;
-        return this;
-    }
-
     public AnalysisTaskInfoBuilder setIndexId(Long indexId) {
         this.indexId = indexId;
         return this;
@@ -117,7 +109,7 @@ public class AnalysisTaskInfoBuilder {
     }
 
     public AnalysisTaskInfo build() {
-        return new AnalysisTaskInfo(jobId, taskId, catalogName, dbName, tblName, colName, partitionNames,
+        return new AnalysisTaskInfo(jobId, taskId, catalogName, dbName, tblName, colName,
                 indexId, jobType, analysisMethod, analysisType, message, lastExecTimeInMs, state, scheduleType);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java
index 0cb83169ee..5adb34d4b0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/HistogramTask.java
@@ -81,7 +81,7 @@ public class HistogramTask extends BaseAnalysisTask {
         params.put("percentValue", String.valueOf((int) (info.sampleRate * 100)));
 
         String histogramSql;
-        Set<String> partitionNames = info.partitionNames;
+        Set<String> partitionNames = tbl.getPartitionNames();
 
         if (partitionNames.isEmpty()) {
             StringSubstitutor stringSubstitutor = new StringSubstitutor(params);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/MVAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/MVAnalysisTask.java
index ce73dbea87..62eb72ec9e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/MVAnalysisTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/MVAnalysisTask.java
@@ -97,7 +97,7 @@ public class MVAnalysisTask extends BaseAnalysisTask {
                     .get();
             selectItem.setAlias(column.getName());
             Map<String, String> params = new HashMap<>();
-            for (String partName : info.partitionNames) {
+            for (String partName : tbl.getPartitionNames()) {
                 PartitionNames partitionName = new PartitionNames(false,
                         Collections.singletonList(partName));
                 tableRef.setPartitionNames(partitionName);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java
index c3a01fb5b5..de848a2592 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java
@@ -73,7 +73,7 @@ public class OlapAnalysisTask extends BaseAnalysisTask {
         List<String> partitionAnalysisSQLs = new ArrayList<>();
         try {
             tbl.readLock();
-            Set<String> partNames = info.partitionNames;
+            Set<String> partNames = tbl.getPartitionNames();
             for (String partName : partNames) {
                 Partition part = tbl.getPartition(partName);
                 if (part == null) {
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 19df103e11..d5d473eac6 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
@@ -66,5 +66,6 @@ public class StatisticConstants {
     public static final int ANALYSIS_JOB_INFO_EXPIRATION_TIME_IN_DAYS = 7;
 
     public static final int FETCH_LIMIT = 10000;
+    public static final int FETCH_INTERVAL_IN_MS = 500;
 
 }
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 6ee30bb90a..ccac80a5f2 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
@@ -123,8 +123,7 @@ public class StatisticsCache {
         return Optional.empty();
     }
 
-    // TODO: finish this method.
-    public void eraseExpiredCache(long tblId, long idxId, String colName) {
+    public void invidate(long tblId, long idxId, String colName) {
         columnStatisticsCache.synchronous().invalidate(new StatisticsCacheKey(tblId, idxId, colName));
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCleaner.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCleaner.java
index 4ef5f35727..3bccdff3b2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCleaner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsCleaner.java
@@ -174,9 +174,8 @@ public class StatisticsCleaner extends MasterDaemon {
 
     public ExpiredStats findExpiredStats(OlapTable statsTbl) {
         ExpiredStats expiredStats = new ExpiredStats();
-        long rowCount = statsTbl.getRowCount();
         long pos = 0;
-        while (pos < rowCount
+        while (pos < statsTbl.getRowCount()
                 && !expiredStats.isFull()) {
             List<ResultRow> rows = StatisticsRepository.fetchStatsFullName(StatisticConstants.FETCH_LIMIT, pos);
             pos += StatisticConstants.FETCH_LIMIT;
@@ -227,6 +226,11 @@ public class StatisticsCleaner extends MasterDaemon {
                     LOG.warn("Error occurred when retrieving expired stats", e);
                 }
             }
+            try {
+                Thread.sleep(StatisticConstants.FETCH_INTERVAL_IN_MS);
+            } catch (InterruptedException t) {
+                // IGNORE
+            }
         }
         return expiredStats;
     }
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 4ce673a063..80187edec2 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
@@ -18,12 +18,12 @@
 package org.apache.doris.statistics;
 
 import org.apache.doris.analysis.AlterColumnStatsStmt;
-import org.apache.doris.analysis.DropStatsStmt;
 import org.apache.doris.analysis.TableName;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.statistics.util.DBObjects;
 import org.apache.doris.statistics.util.InternalQueryResult.ResultRow;
@@ -85,10 +85,7 @@ public class StatisticsRepository {
             + "'${colId}', ${partId}, ${count}, ${ndv}, ${nullCount}, '${min}', '${max}', ${dataSize}, NOW())";
 
     private static final String DROP_TABLE_STATISTICS_TEMPLATE = "DELETE FROM " + FeConstants.INTERNAL_DB_NAME
-            + "." + StatisticConstants.STATISTIC_TBL_NAME + " WHERE ${condition}";
-
-    private static final String DROP_TABLE_HISTOGRAM_TEMPLATE = "DELETE FROM " + FeConstants.INTERNAL_DB_NAME
-            + "." + StatisticConstants.HISTOGRAM_TBL_NAME + " WHERE ${condition}";
+            + "." + "${tblName}" + " WHERE ${condition}";
 
     private static final String FETCH_RECENT_STATS_UPDATED_COL =
             "SELECT * FROM "
@@ -175,48 +172,21 @@ public class StatisticsRepository {
         return stringJoiner.toString();
     }
 
-    public static void dropStatistics(Long dbId,
-            Set<Long> tblIds, Set<String> colNames, Set<Long> partIds) {
-        dropStatistics(dbId, tblIds, colNames, partIds, false);
-    }
-
-    public static void dropHistogram(Long dbId,
-            Set<Long> tblIds, Set<String> colNames, Set<Long> partIds) {
-        dropStatistics(dbId, tblIds, colNames, partIds, true);
+    public static void dropStatistics(long tblId, Set<String> colNames) throws DdlException {
+        dropStatistics(tblId, colNames, StatisticConstants.STATISTIC_TBL_NAME);
+        dropStatistics(tblId, colNames, StatisticConstants.HISTOGRAM_TBL_NAME);
     }
 
-    private static void dropStatistics(Long dbId,
-            Set<Long> tblIds, Set<String> colNames, Set<Long> partIds, boolean isHistogram) {
-        if (dbId <= 0) {
-            throw new IllegalArgumentException("Database id is not specified.");
-        }
-
-        StringBuilder predicate = new StringBuilder();
-        predicate.append(String.format("db_id = '%d'", dbId));
-
-        if (!tblIds.isEmpty()) {
-            buildPredicate("tbl_id", tblIds, predicate);
-        }
-
-        if (!colNames.isEmpty()) {
-            buildPredicate("col_id", colNames, predicate);
-        }
-
-        if (!partIds.isEmpty() && !isHistogram) {
-            // Histogram is not collected and deleted by partition
-            buildPredicate("part_id", partIds, predicate);
-        }
-
+    public static void dropStatistics(long tblId, Set<String> colNames, String statsTblName) throws DdlException {
         Map<String, String> params = new HashMap<>();
-        params.put("condition", predicate.toString());
-        StringSubstitutor stringSubstitutor = new StringSubstitutor(params);
-
+        String right = colNames.stream().map(s -> "'" + s + "'").collect(Collectors.joining(","));
+        String inPredicate = String.format("tbl_id = %s AND %s IN (%s)", tblId, "col_id", right);
+        params.put("tblName", statsTblName);
+        params.put("condition", inPredicate);
         try {
-            String statement = isHistogram ? stringSubstitutor.replace(DROP_TABLE_HISTOGRAM_TEMPLATE) :
-                    stringSubstitutor.replace(DROP_TABLE_STATISTICS_TEMPLATE);
-            StatisticsUtil.execUpdate(statement);
+            StatisticsUtil.execUpdate(new StringSubstitutor(params).replace(DROP_TABLE_STATISTICS_TEMPLATE));
         } catch (Exception e) {
-            LOG.warn("Drop statistics failed", e);
+            throw new DdlException(e.getMessage(), e);
         }
     }
 
@@ -302,15 +272,6 @@ public class StatisticsRepository {
                 .updateColStatsCache(objects.table.getId(), -1, colName, builder.build());
     }
 
-    public static void dropTableStatistics(DropStatsStmt dropTableStatsStmt) {
-        Long dbId = dropTableStatsStmt.getDbId();
-        Set<Long> tbIds = dropTableStatsStmt.getTbIds();
-        Set<String> cols = dropTableStatsStmt.getColumnNames();
-        Set<Long> partIds = dropTableStatsStmt.getPartitionIds();
-        dropHistogram(dbId, tbIds, cols, partIds);
-        dropStatistics(dbId, tbIds, cols, partIds);
-    }
-
     public static List<ResultRow> fetchRecentStatsUpdatedCol() {
         return StatisticsUtil.execStatisticQuery(FETCH_RECENT_STATS_UPDATED_COL);
     }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java
index dcfb475c24..09f64621ce 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java
@@ -28,7 +28,6 @@ import org.apache.doris.statistics.AnalysisTaskInfo.JobType;
 import org.apache.doris.statistics.util.StatisticsUtil;
 import org.apache.doris.utframe.TestWithFeService;
 
-import com.google.common.collect.Sets;
 import mockit.Expectations;
 import mockit.Mock;
 import mockit.MockUp;
@@ -111,7 +110,6 @@ public class AnalysisJobTest extends TestWithFeService {
                 .setCatalogName("internal").setDbName("default_cluster:analysis_job_test").setTblName("t1")
                 .setColName("col1").setJobType(JobType.MANUAL).setAnalysisMethod(AnalysisMethod.FULL).setAnalysisType(
                         AnalysisType.COLUMN)
-                .setPartitionNames(Sets.newHashSet("t1"))
                 .build();
         new OlapAnalysisTask(analysisJobInfo).execute();
     }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java
index 22e4d531a1..10e1635560 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java
@@ -26,7 +26,6 @@ import org.apache.doris.statistics.AnalysisTaskInfo.JobType;
 import org.apache.doris.statistics.util.BlockingCounter;
 import org.apache.doris.utframe.TestWithFeService;
 
-import com.google.common.collect.Sets;
 import mockit.Expectations;
 import mockit.Mock;
 import mockit.MockUp;
@@ -96,7 +95,6 @@ public class AnalysisTaskExecutorTest extends TestWithFeService {
                 .setCatalogName("internal").setDbName("default_cluster:analysis_job_test").setTblName("t1")
                 .setColName("col1").setJobType(JobType.MANUAL).setAnalysisMethod(AnalysisMethod.FULL).setAnalysisType(
                         AnalysisType.COLUMN)
-                .setPartitionNames(Sets.newHashSet("t1"))
                 .build();
         OlapAnalysisTask task = new OlapAnalysisTask(analysisTaskInfo);
         new MockUp<AnalysisTaskScheduler>() {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/HistogramTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/HistogramTaskTest.java
index 3184634815..caf316429f 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/statistics/HistogramTaskTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/HistogramTaskTest.java
@@ -26,7 +26,6 @@ import org.apache.doris.statistics.AnalysisTaskInfo.JobType;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.utframe.TestWithFeService;
 
-import com.google.common.collect.Sets;
 import mockit.Expectations;
 import mockit.Mock;
 import mockit.MockUp;
@@ -92,7 +91,6 @@ public class HistogramTaskTest extends TestWithFeService {
                 Assertions.assertEquals(AnalysisType.HISTOGRAM, info.analysisType);
                 Assertions.assertEquals("t1", info.tblName);
                 Assertions.assertEquals("col1", info.colName);
-                Assertions.assertEquals("p_201701", info.partitionNames.iterator().next());
             }
         }
     }
@@ -105,7 +103,6 @@ public class HistogramTaskTest extends TestWithFeService {
                 .setDbName(SystemInfoService.DEFAULT_CLUSTER + ":" + "histogram_task_test").setTblName("t1")
                 .setColName("col1").setJobType(JobType.MANUAL).setAnalysisMethod(AnalysisMethod.FULL)
                 .setAnalysisType(AnalysisType.HISTOGRAM)
-                 .setPartitionNames(Sets.newHashSet("t"))
                 .build();
         HistogramTask task = new HistogramTask(analysisTaskInfo);
 
diff --git a/regression-test/data/statistics/analyze_test.out b/regression-test/data/statistics/analyze_test.out
index 0123d30de7..048b06a7e2 100644
--- a/regression-test/data/statistics/analyze_test.out
+++ b/regression-test/data/statistics/analyze_test.out
@@ -23,3 +23,6 @@
 5	5	0	1	7	5
 5	5	0	1	7	5
 
+-- !sql --
+0
+
diff --git a/regression-test/suites/statistics/analyze_test.groovy b/regression-test/suites/statistics/analyze_test.groovy
index fb2c7598d6..f8414d9f68 100644
--- a/regression-test/suites/statistics/analyze_test.groovy
+++ b/regression-test/suites/statistics/analyze_test.groovy
@@ -59,7 +59,7 @@ suite("analyze_test") {
     sql """
         DROP TABLE IF EXISTS ${tblName1}
     """
-    
+
     sql """CREATE TABLE ${tblName1} (analyze_test_col1 varchar(11451) not null, analyze_test_col2 int not null, analyze_test_col3 int not null)
     UNIQUE KEY(analyze_test_col1)
     DISTRIBUTED BY HASH(analyze_test_col1)
@@ -68,11 +68,11 @@ suite("analyze_test") {
             "replication_num"="1",
             "enable_unique_key_merge_on_write"="true"
     );"""
-    
+
     sql """
         DROP TABLE IF EXISTS ${tblName2}
     """
-    
+
     sql """CREATE TABLE ${tblName2} (analyze_test_col1 varchar(11451) not null, analyze_test_col2 int not null, analyze_test_col3 int not null)
     UNIQUE KEY(analyze_test_col1)
     DISTRIBUTED BY HASH(analyze_test_col1)
@@ -81,11 +81,11 @@ suite("analyze_test") {
             "replication_num"="1",
             "enable_unique_key_merge_on_write"="true"
     );"""
-    
+
     sql """
         DROP TABLE IF EXISTS ${tblName3}
     """
-    
+
     sql """CREATE TABLE ${tblName3} (analyze_test_col1 varchar(11451) not null, analyze_test_col2 int not null, analyze_test_col3 int not null)
     UNIQUE KEY(analyze_test_col1)
     DISTRIBUTED BY HASH(analyze_test_col1)
@@ -100,14 +100,14 @@ suite("analyze_test") {
     sql """insert into ${tblName1} values(4, 5, 6);"""
     sql """insert into ${tblName1} values(7, 1, 9);"""
     sql """insert into ${tblName1} values(3, 8, 2);"""
-    sql """insert into ${tblName1} values(5, 2, 1);""" 
-    
+    sql """insert into ${tblName1} values(5, 2, 1);"""
+
     sql """insert into ${tblName2} values(1, 2, 3);"""
     sql """insert into ${tblName2} values(4, 5, 6);"""
     sql """insert into ${tblName2} values(7, 1, 9);"""
     sql """insert into ${tblName2} values(3, 8, 2);"""
     sql """insert into ${tblName2} values(5, 2, 1);"""
-    
+
     sql """insert into ${tblName3} values(1, 2, 3);"""
     sql """insert into ${tblName3} values(4, 5, 6);"""
     sql """insert into ${tblName3} values(7, 1, 9);"""
@@ -158,4 +158,29 @@ suite("analyze_test") {
         select count, ndv, null_count, min, max, data_size_in_bytes from __internal_schema.column_statistics where
             col_id in ('analyze_test_col1', 'analyze_test_col2', 'analyze_test_col3') order by col_id
     """
-}
\ No newline at end of file
+
+    sql """
+        DROP STATS ${tblName3} (analyze_test_col1);
+    """
+
+    qt_sql """
+        SELECT COUNT(*) FROM __internal_schema.column_statistics  where
+            col_id in ('analyze_test_col1', 'analyze_test_col2', 'analyze_test_col3') 
+    """
+    // Below test would failed on community pipeline for unknown reason, comment it temporarily
+    // sql """
+    //     SET enable_nereids_planner=true;
+    //
+    // """
+    // sql """
+    //     SET forbid_unknown_col_stats=true;
+    // """
+    //
+    //test {
+    //    sql """
+    //        SELECT analyze_test_col1 FROM ${tblName3}
+    //    """
+    //    exception """errCode = 2, detailMessage = Unexpected exception: column stats for analyze_test_col1 is unknown"""
+    //}
+
+}


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


[doris] 13/36: [fix](Nereids): fix sum func in eager agg (#18675)

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

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

commit 2c67b8de3420a6993b8321cb4218afe1d0be5cea
Author: jakevin <ja...@gmail.com>
AuthorDate: Mon Apr 17 15:06:28 2023 +0800

    [fix](Nereids): fix sum func in eager agg (#18675)
---
 .../doris/nereids/rules/exploration/CBOUtils.java  | 12 -------
 .../nereids/rules/exploration/EagerCount.java      | 10 +++---
 .../nereids/rules/exploration/EagerGroupBy.java    |  2 +-
 .../rules/exploration/EagerGroupByCount.java       |  8 ++---
 .../nereids/rules/exploration/EagerSplit.java      | 18 +++++-----
 .../exploration/join/InnerJoinLAsscomProject.java  |  2 +-
 .../join/InnerJoinLeftAssociateProject.java        |  2 +-
 .../join/InnerJoinRightAssociateProject.java       |  2 +-
 .../exploration/join/JoinExchangeBothProject.java  |  3 +-
 .../exploration/join/JoinExchangeLeftProject.java  |  2 +-
 .../exploration/join/JoinExchangeRightProject.java |  2 +-
 .../join/LogicalJoinSemiJoinTransposeProject.java  |  4 +--
 .../exploration/join/OuterJoinAssocProject.java    |  2 +-
 .../exploration/join/OuterJoinLAsscomProject.java  |  2 +-
 .../join/PushdownProjectThroughInnerJoin.java      |  5 +--
 .../join/PushdownProjectThroughSemiJoin.java       |  5 +--
 .../join/SemiJoinSemiJoinTransposeProject.java     |  2 +-
 .../trees/plans/logical/LogicalProject.java        |  4 +++
 .../nereids/rules/exploration/EagerCountTest.java  |  9 ++---
 .../rules/exploration/EagerGroupByCountTest.java   |  8 ++---
 .../nereids/rules/exploration/EagerSplitTest.java  | 12 +++----
 .../join/InnerJoinLAsscomProjectTest.java          | 16 +++++----
 .../join/JoinExchangeBothProjectTest.java          | 24 +++++++-------
 .../join/JoinExchangeLeftProjectTest.java          | 16 +++++----
 .../join/JoinExchangeRightProjectTest.java         | 24 ++++++++------
 .../join/OuterJoinLAsscomProjectTest.java          | 38 ++++++++++++----------
 26 files changed, 120 insertions(+), 114 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/CBOUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/CBOUtils.java
index 4ab61cfbee..82c6ef309c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/CBOUtils.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/CBOUtils.java
@@ -35,10 +35,6 @@ import java.util.stream.Collectors;
  * Common
  */
 public class CBOUtils {
-    public static boolean isAllSlotProject(LogicalProject<? extends Plan> project) {
-        return project.getProjects().stream().allMatch(expr -> expr instanceof Slot);
-    }
-
     /**
      * Split project according to whether namedExpr contains by splitChildExprIds.
      * Notice: projects must all be Slot.
@@ -56,14 +52,6 @@ public class CBOUtils {
      * If projects is empty or project output equal plan output, return the original plan.
      */
     public static Plan projectOrSelf(List<NamedExpression> projects, Plan plan) {
-        Set<Slot> outputSet = plan.getOutputSet();
-        if (projects.isEmpty() || (outputSet.size() == projects.size() && outputSet.containsAll(projects))) {
-            return plan;
-        }
-        return new LogicalProject<>(projects, plan);
-    }
-
-    public static Plan projectOrSelfInOrder(List<NamedExpression> projects, Plan plan) {
         if (projects.isEmpty() || projects.equals(plan.getOutput())) {
             return plan;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerCount.java
index 09f0f79a9a..50d19bac1e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerCount.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerCount.java
@@ -26,7 +26,6 @@ import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.expressions.SlotReference;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Count;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
-import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.trees.plans.GroupPlan;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
@@ -50,7 +49,7 @@ import java.util.Set;
  * |    *
  * (x)
  * ->
- * aggregate: SUM(x) * cnt
+ * aggregate: SUM(x * cnt)
  * |
  * join
  * |   \
@@ -73,7 +72,7 @@ public class EagerCount implements ExplorationRuleFactory {
                         .then(agg -> eagerCount(agg, agg.child(), ImmutableList.of()))
                         .toRule(RuleType.EAGER_COUNT),
                 logicalAggregate(logicalProject(innerLogicalJoin()))
-                        .when(agg -> CBOUtils.isAllSlotProject(agg.child()))
+                        .when(agg -> agg.child().isAllSlots())
                         .when(agg -> agg.child().child().getOtherJoinConjuncts().size() == 0)
                         .when(agg -> agg.getGroupByExpressions().stream().allMatch(e -> e instanceof Slot))
                         .when(agg -> agg.getAggregateFunctions().stream()
@@ -98,7 +97,7 @@ public class EagerCount implements ExplorationRuleFactory {
                 cntAggGroupBy.add(slot);
             }
         }));
-        Alias cnt = new Alias(new Count(Literal.of(1)), "cnt");
+        Alias cnt = new Alias(new Count(), "cnt");
         List<NamedExpression> cntAggOutput = ImmutableList.<NamedExpression>builder()
                 .addAll(cntAggGroupBy).add(cnt).build();
         LogicalAggregate<GroupPlan> cntAgg = new LogicalAggregate<>(
@@ -116,7 +115,8 @@ public class EagerCount implements ExplorationRuleFactory {
         }
         for (Alias oldSum : sumOutputExprs) {
             Sum oldSumFunc = (Sum) oldSum.child();
-            newOutputExprs.add(new Alias(oldSum.getExprId(), new Multiply(oldSumFunc, cnt.toSlot()),
+            Slot slot = (Slot) oldSumFunc.child();
+            newOutputExprs.add(new Alias(oldSum.getExprId(), new Sum(new Multiply(slot, cnt.toSlot())),
                     oldSum.getName()));
         }
         Plan child = PlanUtils.projectOrSelf(projects, newJoin);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerGroupBy.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerGroupBy.java
index 22e7d5194e..4dcce3d0e9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerGroupBy.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerGroupBy.java
@@ -72,7 +72,7 @@ public class EagerGroupBy implements ExplorationRuleFactory {
                         .then(agg -> eagerGroupBy(agg, agg.child(), ImmutableList.of()))
                         .toRule(RuleType.EAGER_GROUP_BY),
                 logicalAggregate(logicalProject(innerLogicalJoin()))
-                        .when(agg -> CBOUtils.isAllSlotProject(agg.child()))
+                        .when(agg -> agg.child().isAllSlots())
                         .when(agg -> agg.child().child().getOtherJoinConjuncts().size() == 0)
                         .when(agg -> agg.getGroupByExpressions().stream().allMatch(e -> e instanceof Slot))
                         .when(agg -> agg.getAggregateFunctions().stream()
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerGroupByCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerGroupByCount.java
index c538250538..582e84f6b5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerGroupByCount.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerGroupByCount.java
@@ -26,7 +26,6 @@ import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Count;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
-import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.trees.plans.GroupPlan;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
@@ -49,7 +48,7 @@ import java.util.Set;
  * |   (y)
  * (x)
  * ->
- * aggregate: SUM(sum1), SUM(y) * cnt
+ * aggregate: SUM(sum1), SUM(y  * cnt)
  * |
  * join
  * |   \
@@ -96,7 +95,7 @@ public class EagerGroupByCount extends OneExplorationRuleFactory {
                     for (int i = 0; i < leftSums.size(); i++) {
                         bottomSums.add(new Alias(new Sum(leftSums.get(i).child()), "sum" + i));
                     }
-                    Alias cnt = new Alias(new Count(Literal.of(1)), "cnt");
+                    Alias cnt = new Alias(new Count(), "cnt");
                     List<NamedExpression> bottomAggOutput = ImmutableList.<NamedExpression>builder()
                             .addAll(bottomAggGroupBy).addAll(bottomSums).add(cnt).build();
                     LogicalAggregate<GroupPlan> bottomAgg = new LogicalAggregate<>(
@@ -129,7 +128,8 @@ public class EagerGroupByCount extends OneExplorationRuleFactory {
                     }
                     for (Alias oldSum : rightSumOutputExprs) {
                         Sum oldSumFunc = (Sum) oldSum.child();
-                        newOutputExprs.add(new Alias(oldSum.getExprId(), new Multiply(oldSumFunc, cnt.toSlot()),
+                        Slot slot = (Slot) oldSumFunc.child();
+                        newOutputExprs.add(new Alias(oldSum.getExprId(), new Sum(new Multiply(slot, cnt.toSlot())),
                                 oldSum.getName()));
                     }
                     return agg.withAggOutput(newOutputExprs).withChildren(newJoin);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerSplit.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerSplit.java
index abf6dabad8..89023ca69f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerSplit.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/EagerSplit.java
@@ -27,7 +27,6 @@ import org.apache.doris.nereids.trees.expressions.SlotReference;
 import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Count;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
-import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.trees.plans.GroupPlan;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
@@ -51,7 +50,7 @@ import java.util.Set;
  * |   (y)
  * (x)
  * ->
- * aggregate: SUM(sum1) * cnt2, SUM(sum2) * cnt1
+ * aggregate: SUM(sum1 * cnt2), SUM(sum2 * cnt1)
  * |
  * join
  * |   \
@@ -98,7 +97,7 @@ public class EagerSplit extends OneExplorationRuleFactory {
                     for (int i = 0; i < leftSums.size(); i++) {
                         leftBottomSums.add(new Alias(new Sum(leftSums.get(i).child()), "left_sum" + i));
                     }
-                    Alias leftCnt = new Alias(new Count(Literal.of(1)), "left_cnt");
+                    Alias leftCnt = new Alias(new Count(), "left_cnt");
                     List<NamedExpression> leftBottomAggOutput = ImmutableList.<NamedExpression>builder()
                             .addAll(leftBottomAggGroupBy).addAll(leftBottomSums).add(leftCnt).build();
                     LogicalAggregate<GroupPlan> leftBottomAgg = new LogicalAggregate<>(
@@ -117,7 +116,7 @@ public class EagerSplit extends OneExplorationRuleFactory {
                     for (int i = 0; i < rightSums.size(); i++) {
                         rightBottomSums.add(new Alias(new Sum(rightSums.get(i).child()), "right_sum" + i));
                     }
-                    Alias rightCnt = new Alias(new Count(Literal.of(1)), "right_cnt");
+                    Alias rightCnt = new Alias(new Count(), "right_cnt");
                     List<NamedExpression> rightBottomAggOutput = ImmutableList.<NamedExpression>builder()
                             .addAll(rightBottomAggGroupBy).addAll(rightBottomSums).add(rightCnt).build();
                     LogicalAggregate<GroupPlan> rightBottomAgg = new LogicalAggregate<>(
@@ -146,16 +145,15 @@ public class EagerSplit extends OneExplorationRuleFactory {
                     Preconditions.checkState(rightSumOutputExprs.size() == rightBottomSums.size());
                     for (int i = 0; i < leftSumOutputExprs.size(); i++) {
                         Alias oldSum = leftSumOutputExprs.get(i);
-                        Slot bottomSum = leftBottomSums.get(i).toSlot();
-                        Alias newSum = new Alias(oldSum.getExprId(),
-                                new Multiply(new Sum(bottomSum), rightCnt.toSlot()), oldSum.getName());
-                        newOutputExprs.add(newSum);
+                        Slot slot = leftBottomSums.get(i).toSlot();
+                        newOutputExprs.add(new Alias(oldSum.getExprId(), new Sum(new Multiply(slot, rightCnt.toSlot())),
+                                oldSum.getName()));
                     }
                     for (int i = 0; i < rightSumOutputExprs.size(); i++) {
                         Alias oldSum = rightSumOutputExprs.get(i);
                         Slot bottomSum = rightBottomSums.get(i).toSlot();
-                        Alias newSum = new Alias(oldSum.getExprId(),
-                                new Multiply(new Sum(bottomSum), leftCnt.toSlot()), oldSum.getName());
+                        Alias newSum = new Alias(oldSum.getExprId(), new Sum(new Multiply(bottomSum, leftCnt.toSlot())),
+                                oldSum.getName());
                         newOutputExprs.add(newSum);
                     }
                     return agg.withAggOutput(newOutputExprs).withChildren(newJoin);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLAsscomProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLAsscomProject.java
index 4c6147ae00..3304278ab1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLAsscomProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLAsscomProject.java
@@ -56,7 +56,7 @@ public class InnerJoinLAsscomProject extends OneExplorationRuleFactory {
                 .when(topJoin -> InnerJoinLAsscom.checkReorder(topJoin, topJoin.left().child()))
                 .whenNot(join -> join.hasJoinHint() || join.left().child().hasJoinHint())
                 .whenNot(join -> join.isMarkJoin() || join.left().child().isMarkJoin())
-                .when(join -> CBOUtils.isAllSlotProject(join.left()))
+                .when(join -> join.left().isAllSlots())
                 .then(topJoin -> {
                     /* ********** init ********** */
                     LogicalJoin<GroupPlan, GroupPlan> bottomJoin = topJoin.left().child();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLeftAssociateProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLeftAssociateProject.java
index 4dd425d0ab..12bb8bb530 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLeftAssociateProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLeftAssociateProject.java
@@ -52,7 +52,7 @@ public class InnerJoinLeftAssociateProject extends OneExplorationRuleFactory {
                 .when(InnerJoinLeftAssociate::checkReorder)
                 .whenNot(join -> join.hasJoinHint() || join.right().child().hasJoinHint())
                 .whenNot(join -> join.isMarkJoin() || join.right().child().isMarkJoin())
-                .when(join -> CBOUtils.isAllSlotProject(join.right()))
+                .when(join -> join.right().isAllSlots())
                 .then(topJoin -> {
                     LogicalJoin<GroupPlan, GroupPlan> bottomJoin = topJoin.right().child();
                     GroupPlan a = topJoin.left();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinRightAssociateProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinRightAssociateProject.java
index cfc4364a38..77183ed4f7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinRightAssociateProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinRightAssociateProject.java
@@ -50,7 +50,7 @@ public class InnerJoinRightAssociateProject extends OneExplorationRuleFactory {
                 .when(InnerJoinRightAssociate::checkReorder)
                 .whenNot(join -> join.hasJoinHint() || join.left().child().hasJoinHint())
                 .whenNot(join -> join.isMarkJoin() || join.left().child().isMarkJoin())
-                .when(join -> CBOUtils.isAllSlotProject(join.left()))
+                .when(join -> join.left().isAllSlots())
                 .then(topJoin -> {
                     LogicalJoin<GroupPlan, GroupPlan> bottomJoin = topJoin.left().child();
                     GroupPlan a = bottomJoin.left();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeBothProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeBothProject.java
index 0543fcefb7..87d352237c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeBothProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeBothProject.java
@@ -54,8 +54,7 @@ public class JoinExchangeBothProject extends OneExplorationRuleFactory {
     public Rule build() {
         return innerLogicalJoin(logicalProject(innerLogicalJoin()), logicalProject(innerLogicalJoin()))
             .when(JoinExchange::checkReorder)
-            .when(join -> CBOUtils.isAllSlotProject(join.left())
-                    && CBOUtils.isAllSlotProject(join.right()))
+            .when(join -> join.left().isAllSlots() && join.right().isAllSlots())
             .whenNot(join -> join.hasJoinHint()
                     || join.left().child().hasJoinHint() || join.right().child().hasJoinHint())
             .whenNot(join -> join.isMarkJoin() || join.left().child().isMarkJoin() || join.right().child().isMarkJoin())
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeLeftProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeLeftProject.java
index 9f8013f1d8..3d54f5731c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeLeftProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeLeftProject.java
@@ -54,7 +54,7 @@ public class JoinExchangeLeftProject extends OneExplorationRuleFactory {
     public Rule build() {
         return innerLogicalJoin(logicalProject(innerLogicalJoin()), innerLogicalJoin())
                 .when(JoinExchange::checkReorder)
-                .when(join -> CBOUtils.isAllSlotProject(join.left()))
+                .when(join -> join.left().isAllSlots())
                 .whenNot(join -> join.hasJoinHint()
                         || join.left().child().hasJoinHint() || join.right().hasJoinHint())
                 .whenNot(join -> join.isMarkJoin() || join.left().child().isMarkJoin() || join.right().isMarkJoin())
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeRightProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeRightProject.java
index f5df8917c4..6403c48efe 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeRightProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeRightProject.java
@@ -54,7 +54,7 @@ public class JoinExchangeRightProject extends OneExplorationRuleFactory {
     public Rule build() {
         return innerLogicalJoin(innerLogicalJoin(), logicalProject(innerLogicalJoin()))
                 .when(JoinExchange::checkReorder)
-                .when(join -> CBOUtils.isAllSlotProject(join.right()))
+                .when(join -> join.right().isAllSlots())
                 .whenNot(join -> join.hasJoinHint()
                         || join.left().hasJoinHint() || join.right().child().hasJoinHint())
                 .whenNot(join -> join.isMarkJoin() || join.left().isMarkJoin() || join.right().child().isMarkJoin())
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/LogicalJoinSemiJoinTransposeProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/LogicalJoinSemiJoinTransposeProject.java
index bcb66436b5..30c6e4b1ee 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/LogicalJoinSemiJoinTransposeProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/LogicalJoinSemiJoinTransposeProject.java
@@ -46,7 +46,7 @@ public class LogicalJoinSemiJoinTransposeProject implements ExplorationRuleFacto
                                 || topJoin.getJoinType().isLeftOuterJoin())))
                         .whenNot(topJoin -> topJoin.hasJoinHint() || topJoin.left().child().hasJoinHint())
                         .whenNot(LogicalJoin::isMarkJoin)
-                        .when(join -> CBOUtils.isAllSlotProject(join.left()))
+                        .when(join -> join.left().isAllSlots())
                         .then(topJoin -> {
                             LogicalJoin<GroupPlan, GroupPlan> bottomJoin = topJoin.left().child();
                             GroupPlan a = bottomJoin.left();
@@ -64,7 +64,7 @@ public class LogicalJoinSemiJoinTransposeProject implements ExplorationRuleFacto
                         .when(topJoin -> (topJoin.right().child().getJoinType().isLeftSemiOrAntiJoin()
                                 && (topJoin.getJoinType().isInnerJoin()
                                 || topJoin.getJoinType().isRightOuterJoin())))
-                        .when(join -> CBOUtils.isAllSlotProject(join.right()))
+                        .when(join -> join.right().isAllSlots())
                         .then(topJoin -> {
                             LogicalJoin<GroupPlan, GroupPlan> bottomJoin = topJoin.right().child();
                             GroupPlan a = topJoin.left();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinAssocProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinAssocProject.java
index 286d03c505..308e1db583 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinAssocProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinAssocProject.java
@@ -59,7 +59,7 @@ public class OuterJoinAssocProject extends OneExplorationRuleFactory {
                 .whenNot(join -> join.hasJoinHint() || join.left().child().hasJoinHint())
                 .whenNot(join -> join.isMarkJoin() || join.left().child().isMarkJoin())
                 .when(join -> OuterJoinAssoc.checkCondition(join, join.left().child().left().getOutputSet()))
-                .when(join -> CBOUtils.isAllSlotProject(join.left()))
+                .when(join -> join.left().isAllSlots())
                 .then(topJoin -> {
                     /* ********** init ********** */
                     List<NamedExpression> projects = topJoin.left().getProjects();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinLAsscomProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinLAsscomProject.java
index 72d0982edd..2e6007fe54 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinLAsscomProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinLAsscomProject.java
@@ -61,7 +61,7 @@ public class OuterJoinLAsscomProject extends OneExplorationRuleFactory {
                 .when(topJoin -> OuterJoinLAsscom.checkReorder(topJoin, topJoin.left().child()))
                 .whenNot(join -> join.hasJoinHint() || join.left().child().hasJoinHint())
                 .whenNot(join -> join.isMarkJoin() || join.left().child().isMarkJoin())
-                .when(join -> CBOUtils.isAllSlotProject(join.left()))
+                .when(join -> join.left().isAllSlots())
                 .then(topJoin -> {
                     /* ********** init ********** */
                     List<NamedExpression> projects = topJoin.left().getProjects();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/PushdownProjectThroughInnerJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/PushdownProjectThroughInnerJoin.java
index bc1c45f243..6761153e17 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/PushdownProjectThroughInnerJoin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/PushdownProjectThroughInnerJoin.java
@@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.plans.GroupPlan;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList.Builder;
@@ -52,7 +53,7 @@ public class PushdownProjectThroughInnerJoin extends OneExplorationRuleFactory {
     @Override
     public Rule build() {
         return logicalProject(logicalJoin())
-            .whenNot(CBOUtils::isAllSlotProject)
+            .whenNot(LogicalProject::isAllSlots)
             .when(project -> project.child().getJoinType().isInnerJoin())
             .whenNot(project -> project.child().hasJoinHint())
             .then(project -> {
@@ -105,7 +106,7 @@ public class PushdownProjectThroughInnerJoin extends OneExplorationRuleFactory {
                 Plan newRight = CBOUtils.projectOrSelf(newBProject.build(), join.right());
 
                 Plan newJoin = join.withChildrenNoContext(newLeft, newRight);
-                return CBOUtils.projectOrSelfInOrder(new ArrayList<>(project.getOutput()), newJoin);
+                return CBOUtils.projectOrSelf(new ArrayList<>(project.getOutput()), newJoin);
             }).toRule(RuleType.PUSH_DOWN_PROJECT_THROUGH_INNER_JOIN);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/PushdownProjectThroughSemiJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/PushdownProjectThroughSemiJoin.java
index c248874589..79b2047af0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/PushdownProjectThroughSemiJoin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/PushdownProjectThroughSemiJoin.java
@@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.plans.GroupPlan;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -52,7 +53,7 @@ public class PushdownProjectThroughSemiJoin extends OneExplorationRuleFactory {
         return logicalProject(logicalJoin())
             .when(project -> project.child().getJoinType().isLeftSemiOrAntiJoin())
             // Just pushdown project with non-column expr like (t.id + 1)
-            .whenNot(CBOUtils::isAllSlotProject)
+            .whenNot(LogicalProject::isAllSlots)
             .whenNot(project -> project.child().hasJoinHint())
             .then(project -> {
                 LogicalJoin<GroupPlan, GroupPlan> join = project.child();
@@ -65,7 +66,7 @@ public class PushdownProjectThroughSemiJoin extends OneExplorationRuleFactory {
                 Plan newLeft = CBOUtils.projectOrSelf(newProject, join.left());
 
                 Plan newJoin = join.withChildrenNoContext(newLeft, join.right());
-                return CBOUtils.projectOrSelfInOrder(new ArrayList<>(project.getOutput()), newJoin);
+                return CBOUtils.projectOrSelf(new ArrayList<>(project.getOutput()), newJoin);
             }).toRule(RuleType.PUSH_DOWN_PROJECT_THROUGH_SEMI_JOIN);
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/SemiJoinSemiJoinTransposeProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/SemiJoinSemiJoinTransposeProject.java
index 85be57370d..f28f427786 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/SemiJoinSemiJoinTransposeProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/join/SemiJoinSemiJoinTransposeProject.java
@@ -56,7 +56,7 @@ public class SemiJoinSemiJoinTransposeProject extends OneExplorationRuleFactory
                 .when(topSemi -> InnerJoinLAsscom.checkReorder(topSemi, topSemi.left().child()))
                 .whenNot(join -> join.hasJoinHint() || join.left().child().hasJoinHint())
                 .whenNot(join -> join.isMarkJoin() || join.left().child().isMarkJoin())
-                .when(join -> CBOUtils.isAllSlotProject(join.left()))
+                .when(join -> join.left().isAllSlots())
                 .then(topSemi -> {
                     LogicalJoin<GroupPlan, GroupPlan> bottomSemi = topSemi.left().child();
                     LogicalProject abProject = topSemi.left();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java
index 514fc74340..c13ffa518c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java
@@ -102,6 +102,10 @@ public class LogicalProject<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_
         return excepts;
     }
 
+    public boolean isAllSlots() {
+        return projects.stream().allMatch(NamedExpression::isSlot);
+    }
+
     @Override
     public List<Slot> computeOutput() {
         return projects.stream()
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerCountTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerCountTest.java
index ca65e4be78..c3ad243966 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerCountTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerCountTest.java
@@ -49,14 +49,15 @@ class EagerCountTest implements MemoPatternMatchSupported {
                 .build();
         PlanChecker.from(MemoTestUtils.createConnectContext(), agg)
                 .applyExploration(new EagerCount().buildRules())
+                .printlnExploration()
                 .matchesExploration(
                     logicalAggregate(
                         logicalJoin(
                           logicalOlapScan(),
-                          logicalAggregate().when(cntAgg -> cntAgg.getOutputExprsSql().equals("sid, count(1) AS `cnt`"))
+                          logicalAggregate().when(cntAgg -> cntAgg.getOutputExprsSql().equals("sid, count(*) AS `cnt`"))
                         )
                     ).when(newAgg -> newAgg.getGroupByExpressions().equals(((Aggregate) agg).getGroupByExpressions())
-                                        && newAgg.getOutputExprsSql().equals("(sum(gender) * cnt) AS `sum`"))
+                                        && newAgg.getOutputExprsSql().equals("sum((gender * cnt)) AS `sum`"))
                 );
     }
 
@@ -78,11 +79,11 @@ class EagerCountTest implements MemoPatternMatchSupported {
                     logicalAggregate(
                         logicalJoin(
                             logicalOlapScan(),
-                            logicalAggregate().when(cntAgg -> cntAgg.getOutputExprsSql().equals("sid, count(1) AS `cnt`"))
+                            logicalAggregate().when(cntAgg -> cntAgg.getOutputExprsSql().equals("sid, count(*) AS `cnt`"))
                         )
                     ).when(newAgg ->
                         newAgg.getGroupByExpressions().equals(((Aggregate) agg).getGroupByExpressions())
-                            && newAgg.getOutputExprsSql().equals("(sum(gender) * cnt) AS `sum0`, (sum(name) * cnt) AS `sum1`, (sum(age) * cnt) AS `sum2`"))
+                            && newAgg.getOutputExprsSql().equals("sum((gender * cnt)) AS `sum0`, sum((name * cnt)) AS `sum1`, sum((age * cnt)) AS `sum2`"))
                 );
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerGroupByCountTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerGroupByCountTest.java
index de132d22d2..8115525de1 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerGroupByCountTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerGroupByCountTest.java
@@ -58,12 +58,12 @@ class EagerGroupByCountTest implements MemoPatternMatchSupported {
                         logicalAggregate(
                                 logicalJoin(
                                         logicalAggregate().when(
-                                                bottomAgg -> bottomAgg.getOutputExprsSql().equals("id, sum(age) AS `sum0`, count(1) AS `cnt`")),
+                                                bottomAgg -> bottomAgg.getOutputExprsSql().equals("id, sum(age) AS `sum0`, count(*) AS `cnt`")),
                                         logicalOlapScan()
                                 )
                         ).when(newAgg ->
                                 newAgg.getGroupByExpressions().equals(((Aggregate) agg).getGroupByExpressions())
-                                        && newAgg.getOutputExprsSql().equals("sum(sum0) AS `lsum0`, (sum(grade) * cnt) AS `rsum0`"))
+                                        && newAgg.getOutputExprsSql().equals("sum(sum0) AS `lsum0`, sum((grade * cnt)) AS `rsum0`"))
                 );
     }
 
@@ -89,13 +89,13 @@ class EagerGroupByCountTest implements MemoPatternMatchSupported {
                         logicalAggregate(
                                 logicalJoin(
                                         logicalAggregate().when(cntAgg -> cntAgg.getOutputExprsSql()
-                                                .equals("id, sum(gender) AS `sum0`, sum(name) AS `sum1`, sum(age) AS `sum2`, count(1) AS `cnt`")),
+                                                .equals("id, sum(gender) AS `sum0`, sum(name) AS `sum1`, sum(age) AS `sum2`, count(*) AS `cnt`")),
                                         logicalOlapScan()
                                 )
                         ).when(newAgg ->
                                 newAgg.getGroupByExpressions().equals(((Aggregate) agg).getGroupByExpressions())
                                         && newAgg.getOutputExprsSql()
-                                        .equals("sum(sum0) AS `lsum0`, sum(sum1) AS `lsum1`, sum(sum2) AS `lsum2`, (sum(cid) * cnt) AS `rsum0`, (sum(grade) * cnt) AS `rsum1`"))
+                                        .equals("sum(sum0) AS `lsum0`, sum(sum1) AS `lsum1`, sum(sum2) AS `lsum2`, sum((cid * cnt)) AS `rsum0`, sum((grade * cnt)) AS `rsum1`"))
                 );
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerSplitTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerSplitTest.java
index 37e347894f..4ffeb923f5 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerSplitTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/EagerSplitTest.java
@@ -58,13 +58,13 @@ class EagerSplitTest implements MemoPatternMatchSupported {
                         logicalAggregate(
                                 logicalJoin(
                                         logicalAggregate().when(
-                                                a -> a.getOutputExprsSql().equals("id, sum(age) AS `left_sum0`, count(1) AS `left_cnt`")),
+                                                a -> a.getOutputExprsSql().equals("id, sum(age) AS `left_sum0`, count(*) AS `left_cnt`")),
                                         logicalAggregate().when(
-                                                a -> a.getOutputExprsSql().equals("sid, sum(grade) AS `right_sum0`, count(1) AS `right_cnt`"))
+                                                a -> a.getOutputExprsSql().equals("sid, sum(grade) AS `right_sum0`, count(*) AS `right_cnt`"))
                                 )
                         ).when(newAgg ->
                                 newAgg.getGroupByExpressions().equals(((Aggregate) agg).getGroupByExpressions())
-                                        && newAgg.getOutputExprsSql().equals("(sum(left_sum0) * right_cnt) AS `lsum0`, (sum(right_sum0) * left_cnt) AS `rsum0`"))
+                                        && newAgg.getOutputExprsSql().equals("sum((left_sum0 * right_cnt)) AS `lsum0`, sum((right_sum0 * left_cnt)) AS `rsum0`"))
                 );
     }
 
@@ -89,14 +89,14 @@ class EagerSplitTest implements MemoPatternMatchSupported {
                         logicalAggregate(
                                 logicalJoin(
                                         logicalAggregate().when(a -> a.getOutputExprsSql()
-                                                .equals("id, sum(gender) AS `left_sum0`, sum(name) AS `left_sum1`, sum(age) AS `left_sum2`, count(1) AS `left_cnt`")),
+                                                .equals("id, sum(gender) AS `left_sum0`, sum(name) AS `left_sum1`, sum(age) AS `left_sum2`, count(*) AS `left_cnt`")),
                                         logicalAggregate().when(a -> a.getOutputExprsSql()
-                                                .equals("sid, sum(cid) AS `right_sum0`, sum(grade) AS `right_sum1`, count(1) AS `right_cnt`"))
+                                                .equals("sid, sum(cid) AS `right_sum0`, sum(grade) AS `right_sum1`, count(*) AS `right_cnt`"))
                                 )
                         ).when(newAgg ->
                                 newAgg.getGroupByExpressions().equals(((Aggregate) agg).getGroupByExpressions())
                                         && newAgg.getOutputExprsSql()
-                                        .equals("(sum(left_sum0) * right_cnt) AS `lsum0`, (sum(left_sum1) * right_cnt) AS `lsum1`, (sum(left_sum2) * right_cnt) AS `lsum2`, (sum(right_sum0) * left_cnt) AS `rsum0`, (sum(right_sum1) * left_cnt) AS `rsum1`"))
+                                        .equals("sum((left_sum0 * right_cnt)) AS `lsum0`, sum((left_sum1 * right_cnt)) AS `lsum1`, sum((left_sum2 * right_cnt)) AS `lsum2`, sum((right_sum0 * left_cnt)) AS `rsum0`, sum((right_sum1 * left_cnt)) AS `rsum1`"))
                 );
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLAsscomProjectTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLAsscomProjectTest.java
index e7c34c1cfb..f1c73f4edf 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLAsscomProjectTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/InnerJoinLAsscomProjectTest.java
@@ -78,15 +78,17 @@ class InnerJoinLAsscomProjectTest implements MemoPatternMatchSupported {
                 .applyExploration(InnerJoinLAsscomProject.INSTANCE.build())
                 .printlnExploration()
                 .matchesExploration(
+                    logicalProject(
                         logicalJoin(
-                                logicalJoin(
-                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1")),
-                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
-                                ),
-                                logicalProject(
-                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t2"))
-                                ).when(project -> project.getProjects().size() == 1)
+                            logicalJoin(
+                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1")),
+                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
+                            ),
+                            logicalProject(
+                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t2"))
+                            ).when(project -> project.getProjects().size() == 1)
                         )
+                    )
                 );
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeBothProjectTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeBothProjectTest.java
index 0e7a390c17..d48bbff1c1 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeBothProjectTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeBothProjectTest.java
@@ -54,17 +54,19 @@ class JoinExchangeBothProjectTest implements MemoPatternMatchSupported {
         PlanChecker.from(MemoTestUtils.createConnectContext(), plan)
                 .applyExploration(JoinExchangeBothProject.INSTANCE.build())
                 .matchesExploration(
-                    logicalJoin(
-                        logicalProject(
-                            logicalJoin(
-                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1")),
-                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
-                            )
-                        ),
-                        logicalProject(
-                            logicalJoin(
-                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t2")),
-                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t4"))
+                    logicalProject(
+                        logicalJoin(
+                            logicalProject(
+                                logicalJoin(
+                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1")),
+                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
+                                )
+                            ),
+                            logicalProject(
+                                logicalJoin(
+                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t2")),
+                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t4"))
+                                )
                             )
                         )
                     )
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeLeftProjectTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeLeftProjectTest.java
index 7a1a2998e8..95d9286647 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeLeftProjectTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeLeftProjectTest.java
@@ -54,15 +54,17 @@ class JoinExchangeLeftProjectTest implements MemoPatternMatchSupported {
                 .applyExploration(JoinExchangeLeftProject.INSTANCE.build())
                 .printlnExploration()
                 .matchesExploration(
-                    logicalJoin(
+                    logicalProject(
                         logicalJoin(
-                                logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1")),
-                                logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
-                        ),
-                        logicalProject(
                             logicalJoin(
-                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t2")),
-                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t4"))
+                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1")),
+                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
+                            ),
+                            logicalProject(
+                                logicalJoin(
+                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t2")),
+                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t4"))
+                                )
                             )
                         )
                     )
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeRightProjectTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeRightProjectTest.java
index 4b8d038624..113facaa5d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeRightProjectTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/JoinExchangeRightProjectTest.java
@@ -54,18 +54,22 @@ class JoinExchangeRightProjectTest implements MemoPatternMatchSupported {
                 .applyExploration(JoinExchangeRightProject.INSTANCE.build())
                 .printlnExploration()
                 .matchesExploration(
-                    logicalJoin(
-                        logicalJoin(
-                                logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1")),
-                                logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
-                        ),
                         logicalProject(
-                            logicalJoin(
-                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t2")),
-                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t4"))
-                            )
+                                logicalJoin(
+                                        logicalJoin(
+                                                logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1")),
+                                                logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
+                                        ),
+                                        logicalProject(
+                                                logicalJoin(
+                                                        logicalOlapScan().when(
+                                                                scan -> scan.getTable().getName().equals("t2")),
+                                                        logicalOlapScan().when(
+                                                                scan -> scan.getTable().getName().equals("t4"))
+                                                )
+                                        )
+                                )
                         )
-                    )
                 );
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinLAsscomProjectTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinLAsscomProjectTest.java
index c94dcb12ca..9f69966bd9 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinLAsscomProjectTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/join/OuterJoinLAsscomProjectTest.java
@@ -56,15 +56,17 @@ class OuterJoinLAsscomProjectTest implements MemoPatternMatchSupported {
                 .applyExploration(OuterJoinLAsscomProject.INSTANCE.build())
                 .printlnExploration()
                 .matchesExploration(
+                    logicalProject(
                         logicalJoin(
-                                logicalJoin(
-                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1")),
-                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
-                                ),
-                                logicalProject(
-                                        logicalOlapScan().when(scan -> scan.getTable().getName().equals("t2"))
-                                ).when(project -> project.getProjects().size() == 1)
+                            logicalJoin(
+                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1")),
+                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
+                            ),
+                            logicalProject(
+                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t2"))
+                            ).when(project -> project.getProjects().size() == 1)
                         )
+                    )
                 );
     }
 
@@ -82,16 +84,18 @@ class OuterJoinLAsscomProjectTest implements MemoPatternMatchSupported {
                 .applyExploration(OuterJoinLAsscomProject.INSTANCE.build())
                 .printlnExploration()
                 .matchesExploration(
-                    logicalJoin(
-                        logicalProject(
-                            logicalJoin(
-                                logicalProject(logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1"))),
-                                logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
-                            )
-                        ).when(project -> project.getProjects().size() == 3), // t1.id Add t3.id, t3.name
-                        logicalProject(
-                            logicalProject(logicalOlapScan().when(scan -> scan.getTable().getName().equals("t2")))
-                        ).when(project -> project.getProjects().size() == 1)
+                    logicalProject(
+                        logicalJoin(
+                            logicalProject(
+                                logicalJoin(
+                                    logicalProject(logicalOlapScan().when(scan -> scan.getTable().getName().equals("t1"))),
+                                    logicalOlapScan().when(scan -> scan.getTable().getName().equals("t3"))
+                                )
+                            ).when(project -> project.getProjects().size() == 3), // t1.id Add t3.id, t3.name
+                            logicalProject(
+                                logicalProject(logicalOlapScan().when(scan -> scan.getTable().getName().equals("t2")))
+                            ).when(project -> project.getProjects().size() == 1)
+                        )
                     )
                 );
     }


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


[doris] 22/36: [vectorzied](function) fix array_map function analyzed failed with order by clause (#18676)

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

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

commit 9d199e38688f541bc293c01dda75161723c8d55c
Author: zhangstar333 <87...@users.noreply.github.com>
AuthorDate: Tue Apr 18 12:01:44 2023 +0800

    [vectorzied](function) fix array_map function analyzed failed with order by clause (#18676)
    
    * [vectorzied](function) fix array_map function analyzed failed with order by clause
    
    * add test
---
 .../main/java/org/apache/doris/analysis/LambdaFunctionExpr.java    | 7 +++++--
 .../sql_functions/array_functions/test_array_map_function.out      | 6 ++++++
 .../sql_functions/array_functions/test_array_map_function.groovy   | 2 ++
 3 files changed, 13 insertions(+), 2 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/LambdaFunctionExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/LambdaFunctionExpr.java
index 1983b4f3e1..15c37ad850 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/LambdaFunctionExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/LambdaFunctionExpr.java
@@ -82,11 +82,14 @@ public class LambdaFunctionExpr extends Expr {
                 nameSet.add(names.get(i));
             }
             Expr param = params.get(i);
+            if (!param.isAnalyzed()) {
+                param.analyze(analyzer);
+            }
             Type paramType = param.getType();
             if (!paramType.isArrayType()) {
                 throw new AnalysisException(
-                        "The lambda function of params must be array type, now " + (i + 1) + "th is "
-                                + paramType.toString());
+                        "The lambda function of params must be array type, now the param of "
+                                + param.toColumnLabel() + " is " + paramType.toString());
             }
             // this ColumnRefExpr record the unique columnId, which is used for BE
             // so could insert nested column by order.
diff --git a/regression-test/data/query_p0/sql_functions/array_functions/test_array_map_function.out b/regression-test/data/query_p0/sql_functions/array_functions/test_array_map_function.out
index ab77b819a4..470d4a3ca5 100644
--- a/regression-test/data/query_p0/sql_functions/array_functions/test_array_map_function.out
+++ b/regression-test/data/query_p0/sql_functions/array_functions/test_array_map_function.out
@@ -110,3 +110,9 @@
 3	[1]	[-100]	[-100]
 4	\N	\N	\N
 
+-- !select_24 --
+4	\N	\N
+3	[1]	[-100]
+1	[1, 2, 3, 4, 5]	[10, 20, -40, 80, -100]
+2	[6, 7, 8]	[10, 12, 13]
+
diff --git a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function.groovy b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function.groovy
index e516a12b03..b5d4fbbe9d 100644
--- a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function.groovy
+++ b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function.groovy
@@ -68,6 +68,8 @@ suite("test_array_map_function") {
         qt_select_21 "select array_filter(x->abs(x), [1,2]);"
         qt_select_22 "select *,array_filter(x->x%2=0,c_array1) from array_test2 order by id;"
         qt_select_23 "select *,array_filter(x->x%2=0,c_array2) from array_test2 order by id;"
+
+        qt_select_24 "select * from array_test2 order by array_max(array_map(x->x,c_array1));"
         
         sql "DROP TABLE IF EXISTS ${tableName}"
 }
\ 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] 36/36: [fix](Nereids): need update parentExpression after replace child. (#18771)

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

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

commit 701397fb50a643fd8786d97e98409874c0671788
Author: jakevin <ja...@gmail.com>
AuthorDate: Wed Apr 19 15:13:42 2023 +0800

    [fix](Nereids): need update parentExpression after replace child. (#18771)
---
 fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
index 4ad1b118dc..bc376b1a37 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
@@ -38,7 +38,6 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
 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.physical.PhysicalPlan;
-import org.apache.doris.nereids.util.Utils;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.statistics.Statistics;
 
@@ -472,7 +471,7 @@ public class Memo {
         for (GroupExpression reinsertGroupExpr : needReplaceChild) {
             // After change GroupExpression children, hashcode will change, so need to reinsert into map.
             groupExpressions.remove(reinsertGroupExpr);
-            Utils.replaceList(reinsertGroupExpr.children(), source, destination);
+            reinsertGroupExpr.replaceChild(source, destination);
 
             GroupExpression existGroupExpr = groupExpressions.get(reinsertGroupExpr);
             if (existGroupExpr != null) {


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


[doris] 33/36: [minor](decimal) forbid to create table with decimal type exceeds 18 (#18763)

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

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

commit 2f6f3ff208ca1ab8ebf915fd10e24365693dd052
Author: Gabriel <ga...@gmail.com>
AuthorDate: Wed Apr 19 11:34:27 2023 +0800

    [minor](decimal) forbid to create table with decimal type exceeds 18 (#18763)
    
    * [minor](decimal) forbid to create table with decimal type exceeds 18
    
    * update
---
 .../java/org/apache/doris/analysis/TypeDef.java    |  8 ++++++--
 .../account_p0/test_information_schema.groovy      |  6 +++---
 .../test_outer_join_with_subquery.groovy           | 24 +++++++++++-----------
 .../test_outer_join_with_inline_view.groovy        |  2 +-
 .../stream_load/test_parquet_orc_case.groovy       |  4 ++--
 .../suites/mysql_ssl_p0/test_ssl_wild.groovy       | 10 ++++-----
 .../test_array_aggregation_functions.groovy        |  2 +-
 .../test_array_aggregation_functions.groovy        |  2 +-
 8 files changed, 31 insertions(+), 27 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java
index c75259050d..5eae5d57ba 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java
@@ -194,15 +194,19 @@ public class TypeDef implements ParseNode {
                 int precision = scalarType.decimalPrecision();
                 int scale = scalarType.decimalScale();
                 // precision: [1, 27]
-                if (precision < 1 || precision > 27) {
+                if (precision < 1 || precision > ScalarType.MAX_DECIMALV2_PRECISION) {
                     throw new AnalysisException("Precision of decimal must between 1 and 27."
                             + " Precision was set to: " + precision + ".");
                 }
                 // scale: [0, 9]
-                if (scale < 0 || scale > 9) {
+                if (scale < 0 || scale > ScalarType.MAX_DECIMALV2_SCALE) {
                     throw new AnalysisException(
                             "Scale of decimal must between 0 and 9." + " Scale was set to: " + scale + ".");
                 }
+                if (precision - scale > ScalarType.MAX_DECIMALV2_PRECISION - ScalarType.MAX_DECIMALV2_SCALE) {
+                    throw new AnalysisException("Invalid decimal type with precision = " + precision + ", scale = "
+                            + scale);
+                }
                 // scale < precision
                 if (scale > precision) {
                     throw new AnalysisException("Scale of decimal must be smaller than precision."
diff --git a/regression-test/suites/account_p0/test_information_schema.groovy b/regression-test/suites/account_p0/test_information_schema.groovy
index 49f9fffb3e..dcbc0c3532 100644
--- a/regression-test/suites/account_p0/test_information_schema.groovy
+++ b/regression-test/suites/account_p0/test_information_schema.groovy
@@ -45,9 +45,9 @@ suite("test_information_schema") {
                 `qqq` varchar(130) NULL COMMENT "",
                 `rrr` bigint(20) NULL COMMENT "",
                 `sss` bigint(20) NULL COMMENT "",
-                `ttt` decimal(24, 2) NULL COMMENT "",
-                `uuu` decimal(24, 2) NULL COMMENT "",
-                `vvv` decimal(24, 2) NULL COMMENT "",
+                `ttt` decimal(20, 2) NULL COMMENT "",
+                `uuu` decimal(20, 2) NULL COMMENT "",
+                `vvv` decimal(20, 2) NULL COMMENT "",
                 `www` varchar(50) NULL COMMENT "",
                 `xxx` varchar(190) NULL COMMENT "",
                 `yyy` varchar(190) NULL COMMENT "",
diff --git a/regression-test/suites/correctness/test_outer_join_with_subquery.groovy b/regression-test/suites/correctness/test_outer_join_with_subquery.groovy
index 1471f6965a..fd2ce0b7e8 100644
--- a/regression-test/suites/correctness/test_outer_join_with_subquery.groovy
+++ b/regression-test/suites/correctness/test_outer_join_with_subquery.groovy
@@ -44,21 +44,21 @@
             `datatimev2_2`           datetimev2(3) NOT NULL COMMENT "",
             `datatimev2_3`           datetimev2(6) NOT NULL COMMENT "",
             `employee`               int(11) NULL COMMENT "",
-            `oper_rev`               decimal(27, 2) NULL COMMENT "",
-            `net_profit`             decimal(27, 2) NULL COMMENT "",
-            `roe_diluted`            decimal(27, 2) NULL COMMENT "",
-            `roe_forecast1`          decimal(27, 2) NULL COMMENT "",
-            `roe_forecast2`          decimal(27, 2) NULL COMMENT "",
-            `roe_forecast3`          decimal(27, 2) NULL COMMENT "",
+            `oper_rev`               decimal(20, 2) NULL COMMENT "",
+            `net_profit`             decimal(20, 2) NULL COMMENT "",
+            `roe_diluted`            decimal(20, 2) NULL COMMENT "",
+            `roe_forecast1`          decimal(20, 2) NULL COMMENT "",
+            `roe_forecast2`          decimal(20, 2) NULL COMMENT "",
+            `roe_forecast3`          decimal(20, 2) NULL COMMENT "",
             `segment_sales_industry` varchar(2000) NULL COMMENT "",
             `segment_sales_product`  varchar(2000) NULL COMMENT "",
             `segment_sales_region`   varchar(2000) NULL COMMENT "",
-            `cont_liab`              decimal(27, 2) NULL COMMENT "",
-            `rd_exp`                 decimal(27, 2) NULL COMMENT "",
-            `cash_end_bal_cf`        decimal(27, 2) NULL COMMENT "",
-            `deductedprofit`         decimal(27, 2) NULL COMMENT "",
-            `extraordinary`          decimal(27, 2) NULL COMMENT "",
-            `capex`                  decimal(27, 2) NULL COMMENT "",
+            `cont_liab`              decimal(20, 2) NULL COMMENT "",
+            `rd_exp`                 decimal(20, 2) NULL COMMENT "",
+            `cash_end_bal_cf`        decimal(20, 2) NULL COMMENT "",
+            `deductedprofit`         decimal(20, 2) NULL COMMENT "",
+            `extraordinary`          decimal(20, 2) NULL COMMENT "",
+            `capex`                  decimal(20, 2) NULL COMMENT "",
             `update_time`            datetime NULL COMMENT ""
         ) ENGINE=OLAP
         UNIQUE KEY(`stock_code`, `data_time`, `datev2`, `datatimev2_1`, `datatimev2_2`, `datatimev2_3`)
diff --git a/regression-test/suites/correctness_p0/test_outer_join_with_inline_view.groovy b/regression-test/suites/correctness_p0/test_outer_join_with_inline_view.groovy
index 16f02a39bf..3d8c563f0d 100644
--- a/regression-test/suites/correctness_p0/test_outer_join_with_inline_view.groovy
+++ b/regression-test/suites/correctness_p0/test_outer_join_with_inline_view.groovy
@@ -115,7 +115,7 @@ suite("test_outer_join_with_inline_view") {
     sql """
         CREATE TABLE `subquery_table_3` (
         `org_code` varchar(96) NULL, 
-        `bo_ql_in_advance` decimal(27, 6) NULL
+        `bo_ql_in_advance` decimal(24, 6) NULL
         ) ENGINE=OLAP
         DUPLICATE KEY(`org_code`)
         COMMENT 'OLAP'
diff --git a/regression-test/suites/load_p0/stream_load/test_parquet_orc_case.groovy b/regression-test/suites/load_p0/stream_load/test_parquet_orc_case.groovy
index fac1ddabc4..22e61893ed 100644
--- a/regression-test/suites/load_p0/stream_load/test_parquet_orc_case.groovy
+++ b/regression-test/suites/load_p0/stream_load/test_parquet_orc_case.groovy
@@ -211,7 +211,7 @@ suite("test_parquet_orc_case", "p0") {
     sql """ DROP TABLE IF EXISTS ${arrayParquetTbl} """
 
     sql """
-    CREATE TABLE ${arrayParquetTbl} ( 
+    CREATE TABLE ${arrayParquetTbl} (
         k1 int NULL, 
         a1 array<boolean> NULL, 
         a2 array<tinyint> NULL, 
@@ -219,7 +219,7 @@ suite("test_parquet_orc_case", "p0") {
         a4 array<int> NULL, 
         a5 array<bigint> NULL, 
         a6 array<largeint> NULL,
-        a7 array<decimal(27, 7)> NULL, 
+        a7 array<decimal(25, 7)> NULL,
         a8 array<float> NULL, 
         a9 array<double> NULL, 
         a10 array<date> NULL, 
diff --git a/regression-test/suites/mysql_ssl_p0/test_ssl_wild.groovy b/regression-test/suites/mysql_ssl_p0/test_ssl_wild.groovy
index 3e7147698c..a9c3620b1d 100644
--- a/regression-test/suites/mysql_ssl_p0/test_ssl_wild.groovy
+++ b/regression-test/suites/mysql_ssl_p0/test_ssl_wild.groovy
@@ -46,7 +46,7 @@ suite("test_ssl_wild") {
                 col22 text NULL,
                 col23 text NULL,
                 col24 int(11) NULL,
-                col25 decimal(22, 2) NULL,
+                col25 decimal(20, 2) NULL,
                 col26 decimal(22, 4) NULL,
                 col27 decimal(16, 8) NULL,
                 col28 decimal(16, 8) NULL,
@@ -70,10 +70,10 @@ suite("test_ssl_wild") {
                 col46 decimal(16, 2) NULL,
                 col47 int(11) NULL,
                 col48 decimal(16, 2) NULL,
-                col49 decimal(22, 2) NULL,
-                col50 decimal(22, 2) NULL,
-                col51 decimal(22, 2) NULL,
-                col52 decimal(22, 2) NULL,
+                col49 decimal(20, 2) NULL,
+                col50 decimal(20, 2) NULL,
+                col51 decimal(20, 2) NULL,
+                col52 decimal(20, 2) NULL,
                 col53 decimal(16, 2) NULL,
                 col54 int(11) NULL,
                 col55 int(11) NULL,
diff --git a/regression-test/suites/nereids_p0/sql_functions/array_functions/test_array_aggregation_functions.groovy b/regression-test/suites/nereids_p0/sql_functions/array_functions/test_array_aggregation_functions.groovy
index e904c7002b..f1ba1d0bfd 100644
--- a/regression-test/suites/nereids_p0/sql_functions/array_functions/test_array_aggregation_functions.groovy
+++ b/regression-test/suites/nereids_p0/sql_functions/array_functions/test_array_aggregation_functions.groovy
@@ -29,7 +29,7 @@ suite("test_array_aggregation_functions") {
               `a3` array<int(11)> NULL COMMENT "",
               `a4` array<bigint(20)> NULL COMMENT "",
               `a5` array<largeint(40)> NULL COMMENT "",
-              `a6` array<decimal(27, 7)> NULL COMMENT "",
+              `a6` array<decimal(25, 7)> NULL COMMENT "",
               `a7` array<float> NULL COMMENT "",
               `a8` array<double> NULL COMMENT "",
               `a9` array<date> NULL COMMENT "",
diff --git a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_aggregation_functions.groovy b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_aggregation_functions.groovy
index de827daa43..d6fe42b659 100644
--- a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_aggregation_functions.groovy
+++ b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_aggregation_functions.groovy
@@ -27,7 +27,7 @@ suite("test_array_aggregation_functions") {
               `a3` array<int(11)> NULL COMMENT "",
               `a4` array<bigint(20)> NULL COMMENT "",
               `a5` array<largeint(40)> NULL COMMENT "",
-              `a6` array<decimal(27, 7)> NULL COMMENT "",
+              `a6` array<decimal(25, 7)> NULL COMMENT "",
               `a7` array<float> NULL COMMENT "",
               `a8` array<double> NULL COMMENT "",
               `a9` array<date> NULL COMMENT "",


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


[doris] 14/36: [Bug](DECIMAL) Fix bug for arithmatic expr DECIMALV2 / DECIMALV3 (#18723)

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

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

commit bb4a28d8065feb10a549e7055d791d6917257c97
Author: Gabriel <ga...@gmail.com>
AuthorDate: Mon Apr 17 16:43:36 2023 +0800

    [Bug](DECIMAL) Fix bug for arithmatic expr DECIMALV2 / DECIMALV3 (#18723)
---
 .../org/apache/doris/analysis/ArithmeticExpr.java  |  2 +-
 .../decimalv3/test_arithmetic_expressions.out      |  6 ++++++
 .../decimalv3/test_arithmetic_expressions.groovy   | 22 ++++++++++++++++++++++
 3 files changed, 29 insertions(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
index 152df98ae1..0efc993d1e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
@@ -566,7 +566,7 @@ public class ArithmeticExpr extends Expr {
                     if (((ScalarType) type).getScalarScale() != ((ScalarType) children.get(1).type).getScalarScale()) {
                         castChild(type, 1);
                     }
-                } else if (op == Operator.DIVIDE && (t2Scale != 0) && t1.isDecimalV3()) {
+                } else if (op == Operator.DIVIDE && (t2Scale != 0) && t1TargetType.isDecimalV3()) {
                     int targetScale = t1Scale + t2Scale;
                     if (precision < targetScale) {
                         type = castBinaryOp(Type.DOUBLE);
diff --git a/regression-test/data/datatype_p0/decimalv3/test_arithmetic_expressions.out b/regression-test/data/datatype_p0/decimalv3/test_arithmetic_expressions.out
index 085b844d7c..7f29407e10 100644
--- a/regression-test/data/datatype_p0/decimalv3/test_arithmetic_expressions.out
+++ b/regression-test/data/datatype_p0/decimalv3/test_arithmetic_expressions.out
@@ -47,3 +47,9 @@
 -- !select --
 10999999.989
 
+-- !select --
+1	629.028702933	629.028702933
+2	722.810212429	722.810212429
+3	724.291976000	724.291976000
+4	688.890183155	688.890183155
+
diff --git a/regression-test/suites/datatype_p0/decimalv3/test_arithmetic_expressions.groovy b/regression-test/suites/datatype_p0/decimalv3/test_arithmetic_expressions.groovy
index 284cf482e4..b077e4c362 100644
--- a/regression-test/suites/datatype_p0/decimalv3/test_arithmetic_expressions.groovy
+++ b/regression-test/suites/datatype_p0/decimalv3/test_arithmetic_expressions.groovy
@@ -65,4 +65,26 @@ suite("test_arithmetic_expressions") {
     qt_select "select (a + b + c) / d from ${table1};"
     qt_select "select a + b + c + d + e + f + g + h + i + j + k from ${table1};"
     sql "drop table if exists ${table1}"
+
+    def table2 = "test_arithmetic_expressions"
+
+    sql "drop table if exists ${table2}"
+    sql """ create table ${table2} (
+            id smallint,
+            fz decimal(27,9),
+            fzv3 decimalv3(27,9),
+            fm decimalv3(38,10))
+            DISTRIBUTED BY HASH(`id`) BUCKETS auto
+            PROPERTIES
+            (
+                "replication_num" = "1"
+            ); """
+
+    sql """ insert into ${table2} values (1,92594283.129196000,92594283.129196000,147202.0000000000); """
+    sql """ insert into ${table2} values (2,107684988.257976000,107684988.257976000,148981.0000000000); """
+    sql """ insert into ${table2} values (3,76891560.464178000,76891560.464178000,106161.0000000000); """
+    sql """ insert into ${table2} values (4,277170831.851350000,277170831.851350000,402344.0000000000); """
+
+    qt_select """ select id, fz/fm as dec,fzv3/fm as decv3 from ${table2} ORDER BY id; """
+    sql "drop table if exists ${table2}"
 }


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


[doris] 01/36: [Bug](pipeline) regression heap use after free (#18701)

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

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

commit d64e4831634633074871015318aaffa469bbdc31
Author: HappenLee <ha...@hotmail.com>
AuthorDate: Sun Apr 16 16:22:41 2023 +0800

    [Bug](pipeline) regression heap use after free (#18701)
---
 be/src/vec/exec/scan/scanner_context.cpp | 1 -
 1 file changed, 1 deletion(-)

diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp
index eb7b300724..a42044f4c1 100644
--- a/be/src/vec/exec/scan/scanner_context.cpp
+++ b/be/src/vec/exec/scan/scanner_context.cpp
@@ -272,7 +272,6 @@ void ScannerContext::clear_and_join(VScanNode* node, RuntimeState* state) {
     _close_and_clear_scanners(node, state);
 
     _blocks_queue.clear();
-    _free_blocks.clear();
 }
 
 bool ScannerContext::no_schedule() {


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


[doris] 34/36: [Fix](planner)Fix TupleDescriptor include not materialized slot bug (#18783)

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

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

commit 8bc6413866d59affa24d43c48816e30e0b38884c
Author: Jibing-Li <64...@users.noreply.github.com>
AuthorDate: Wed Apr 19 14:08:09 2023 +0800

    [Fix](planner)Fix TupleDescriptor include not materialized slot bug (#18783)
    
    setOutputSmap function in ScanNode may include not materialized to outputTupleDesc. This PR is to fix this.
---
 .../src/main/java/org/apache/doris/planner/ScanNode.java     | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)

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 14cba2c3e0..13dd8638ba 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
@@ -516,7 +516,7 @@ public abstract class ScanNode extends PlanNode {
             // create a tmpSmap for the later setOutputSmap call
             ExprSubstitutionMap tmpSmap = new ExprSubstitutionMap(
                     Lists.newArrayList(outputTupleDesc.getSlots().stream().map(slot -> new SlotRef(slot)).collect(
-                            Collectors.toList())), Lists.newArrayList(projectList));
+                    Collectors.toList())), Lists.newArrayList(projectList));
             Set<SlotId> allOutputSlotIds = outputTupleDesc.getSlots().stream().map(slot -> slot.getId())
                     .collect(Collectors.toSet());
             List<Expr> newRhs = Lists.newArrayList();
@@ -530,10 +530,14 @@ public abstract class ScanNode extends PlanNode {
                         slotDesc.initFromExpr(rhsExpr);
                         if (rhsExpr instanceof SlotRef) {
                             slotDesc.setSrcColumn(((SlotRef) rhsExpr).getColumn());
+                            slotDesc.setIsMaterialized(((SlotRef) rhsExpr).getDesc().isMaterialized());
+                        } else {
+                            slotDesc.setIsMaterialized(true);
+                        }
+                        if (slotDesc.isMaterialized()) {
+                            slotDesc.materializeSrcExpr();
+                            projectList.add(rhsExpr);
                         }
-                        slotDesc.setIsMaterialized(true);
-                        slotDesc.materializeSrcExpr();
-                        projectList.add(rhsExpr);
                         newRhs.add(new SlotRef(slotDesc));
                         allOutputSlotIds.add(slotDesc.getId());
                     } else {


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


[doris] 35/36: [regression-test](iceberg)add iceberg in regression case (#18792)

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

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

commit 563900e23dad149f9b0d022475b862855d41c404
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Wed Apr 19 15:09:20 2023 +0800

    [regression-test](iceberg)add iceberg in regression case (#18792)
    
    add iceberg 'in' clause regression case
    for #18226
---
 .../iceberg/test_external_catalog_icebergv2.out                   | 8 ++++++++
 .../iceberg/test_external_catalog_icebergv2.groovy                | 6 ++++++
 2 files changed, 14 insertions(+)

diff --git a/regression-test/data/external_table_emr_p2/iceberg/test_external_catalog_icebergv2.out b/regression-test/data/external_table_emr_p2/iceberg/test_external_catalog_icebergv2.out
index e7158ffd36..cbabc8ecec 100644
--- a/regression-test/data/external_table_emr_p2/iceberg/test_external_catalog_icebergv2.out
+++ b/regression-test/data/external_table_emr_p2/iceberg/test_external_catalog_icebergv2.out
@@ -52,3 +52,11 @@
 149999999
 149999996
 
+-- !q13 --
+1
+4
+7
+
+-- !q14 --
+Customer#000000004
+Customer#000000007
diff --git a/regression-test/suites/external_table_emr_p2/iceberg/test_external_catalog_icebergv2.groovy b/regression-test/suites/external_table_emr_p2/iceberg/test_external_catalog_icebergv2.groovy
index 3b70f1936e..c4e18d4b99 100644
--- a/regression-test/suites/external_table_emr_p2/iceberg/test_external_catalog_icebergv2.groovy
+++ b/regression-test/suites/external_table_emr_p2/iceberg/test_external_catalog_icebergv2.groovy
@@ -49,8 +49,14 @@ suite("test_external_catalog_icebergv2", "p2") {
             qt_q11 """ select c_custkey from customer for version as of 906874575350293177 order by c_custkey limit 3 """
             qt_q12 """ select c_custkey from customer for version as of 6352416983354893547 order by c_custkey desc limit 3 """
         }
+        // in predicate
+        def q03 = {
+            qt_q13 """ select c_custkey from customer_small where c_custkey in (1, 2, 4, 7) order by c_custkey """
+            qt_q14 """ select c_name from customer_small where c_name in ('Customer#000000004', 'Customer#000000007') order by c_custkey """
+        }
         sql """ use `tpch_1000_icebergv2`; """
         q01()
         q02()
+        q03()
     }
 }


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


[doris] 05/36: [regression-test] add some suites (#18286)

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

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

commit f10f9c640e1e18c0ce80cf095219dcb63f0f6978
Author: yagagagaga <zh...@gmail.com>
AuthorDate: Sun Apr 16 18:19:52 2023 +0800

    [regression-test] add some suites (#18286)
    
    test_join_result_count.groovy
---
 .../query_p0/join/test_join_result_count.groovy    | 198 +++++++++++++++++++++
 1 file changed, 198 insertions(+)

diff --git a/regression-test/suites/query_p0/join/test_join_result_count.groovy b/regression-test/suites/query_p0/join/test_join_result_count.groovy
new file mode 100644
index 0000000000..775d9b37dd
--- /dev/null
+++ b/regression-test/suites/query_p0/join/test_join_result_count.groovy
@@ -0,0 +1,198 @@
+// 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_join_result_count", "query,p0") {
+
+    sql "drop table if exists t3;"
+
+    sql """
+    CREATE TABLE `t3` (
+`c0` datev2 NOT NULL ,
+`c1` varchar(50) NOT NULL ,
+`c2` varchar(50) NOT NULL ,
+`c3` varchar(20) NOT NULL ,
+`c4` varchar(50) NOT NULL ,
+`c5` varchar(20) NOT NULL ,
+`c6` varchar(100) NOT NULL ,
+`c7` decimalv3(38, 18) NOT NULL ,
+`c8` decimalv3(38, 18) NOT NULL ,
+`c9` decimalv3(38, 18) NOT NULL ,
+`c10` decimalv3(38, 18) NOT NULL ,
+`c11` decimalv3(38, 18) NOT NULL ,
+`c12` decimalv3(38, 9) NOT NULL ,
+`c13` decimalv3(38, 18) NOT NULL ,
+`c14` decimalv3(10, 0) NOT NULL ,
+`c15` decimalv3(38, 18) NOT NULL ,
+`c16` decimalv3(38, 18) NOT NULL ,
+`c17` decimalv3(38, 18) NOT NULL ,
+`c18` varchar(200) NOT NULL ,
+`c19` varchar(60) NOT NULL ,
+`c20` varchar(60) NOT NULL ,
+`c21` smallint(6) NOT NULL ,
+`c22` char(2) NOT NULL ,
+`c23` datev2 NOT NULL
+) ENGINE=OLAP
+DUPLICATE KEY(`c0`, `c1`, `c2`, `c3`)
+PARTITION BY RANGE(`c0`)
+(
+PARTITION p_2000 VALUES [('2000-01-01'), ('2099-01-01'))
+)
+DISTRIBUTED BY HASH(`c0`, `c1`, `c2`, `c3`) BUCKETS AUTO
+PROPERTIES (
+"replication_allocation" = "tag.location.default: 1",
+"in_memory" = "false",
+"storage_format" = "V2",
+"light_schema_change" = "true",
+"disable_auto_compaction" = "false"
+);
+"""
+
+    sql "drop table if exists t4;"
+
+    sql """
+CREATE TABLE `t4`
+(
+`d0` varchar(50) NULL ,
+`d1` varchar(80) NULL ,
+`d2` varchar(40) NULL ,
+`d3` varchar(30) NULL ,
+`d4` varchar(50) NULL ,
+`d5` varchar(8) NULL ,
+`d6` varchar(100) NULL ,
+`d7` varchar(40) NULL ,
+`d8` varchar(200) NULL ,
+`d9` varchar(3) NULL ,
+`d10` varchar(10) NULL ,
+`d11` varchar(20) NULL ,
+`d12` varchar(10) NULL ,
+`d13` varchar(20) NULL ,
+`d14` varchar(200) NULL ,
+`d15` varchar(100) NULL ,
+`d16` varchar(100) NULL ,
+`d17` varchar(50) NULL ,
+`d18` datev2 NULL ,
+`d19` datev2 NULL ,
+`d20` varchar(2) NULL ,
+`d21` varchar(20) NULL ,
+`d22` varchar(50) NULL ,
+`d23` varchar(50) NULL ,
+`d24` varchar(50) NULL ,
+`d25` varchar(50) NULL ,
+`d26` varchar(50) NULL ,
+`d27` varchar(50) NULL ,
+`d28` varchar(50) NULL ,
+`d29` varchar(50) NULL ,
+`d30` varchar(8) NULL ,
+`d31` varchar(100) NULL ,
+`d32` datev2 NULL ,
+`d33` datev2 NULL
+) ENGINE=OLAP
+DUPLICATE KEY(`d0`)
+DISTRIBUTED BY HASH(`d0`) BUCKETS AUTO
+PROPERTIES (
+"replication_allocation" = "tag.location.default: 1",
+"in_memory" = "false",
+"storage_format" = "V2",
+"light_schema_change" = "true",
+"disable_auto_compaction" = "false"
+);
+"""
+
+    sql """
+INSERT INTO `t3` VALUES ('2017-01-01','73','1','xy','XYZ','zQ','0N',78.326461744024730726,78.148863384057232844,62.576725243619092236,40.441486336294099290,31.469270014735277558,28.782245553,33.829453793315923236,39,89.415314680352419216,48.277770337435450545,39.771029459737232017,'A3','OGK_345_DF3G','zZ',54,'Sc','2010-01-01'),
+('2017-01-01','C7','1','qR','CDE','LV','hN',81.216425648502760142,28.925014306186767820,52.402422792994751075,5.812755708462516834,53.223879424390418910,13.433030608,1.481331564085870790,10,24.769926557473260971,88.539161343479310865,24.541794550608993058,'rW','MNH_34S_REKK','Gh',45,'Hu','2010-01-01'),
+('2017-01-01','58','1','Yk','CDE','aH','XQ',78.392219414044635220,17.836054009716235976,75.521757437152712728,88.114569368422740929,62.297035646689863337,80.231758806,17.353573976555423210,18,79.688952848768112904,71.647546361197629529,44.315339148367524439,'n2','OGK_345_DF3G','hF',18,'Or','2010-01-01'),
+('2017-01-01','Dg','1','tx','PLM','AZ','Uq',34.250098363937564556,79.587227354840741959,31.814047037176354965,59.917666377146970604,13.834441510087425615,0.365281261,85.497153321232268530,36,77.963919462338067371,32.149317431964937960,17.429028523835712229,'LT','MNH_34S_REKK','LS',15,'Vl','2010-01-01'),
+('2018-01-01','If','1','np','CDE','wb','so',68.279310735621318674,36.879973446170546149,65.513020649488707476,49.847202243832249046,86.194752199665599617,50.454361838,43.347186613370912871,3,49.790111820361468719,74.289699744482439840,66.638144755346800134,'28','MNH_34S_REKK','73',96,'6C','2010-01-01'),
+('2018-01-01','i3','1','jh','ABC','t6','XI',5.138878908729890884,84.930864324459359970,64.637955032208415597,54.938326563130267689,33.923818238453330567,26.320989671,12.294925863562896321,19,12.457533760912251474,61.360584268926811330,88.976385655941290403,'TE','AFD_GFFD_234DF','Ox',14,'fZ','2010-01-01'),
+('2018-01-01','qe','1','kb','XYZ','IL','aT',84.433049408592361382,25.208840938251258951,7.515469155188074817,71.508519492161355211,4.440055388630681670,81.855700857,0.767694961897832592,53,70.124435774461415165,6.105987429865320820,0.993866026878107445,'wt','AFD_GFFD_234DF','MY',41,'mI','2010-01-01'),
+('2017-01-01','cm','1','21','XYZ','H7','Df',12.177693662735298656,61.752358034653687543,79.822770795436730436,14.882751629037019505,70.928139172708888073,53.543966255,41.841997784000600470,35,6.767427047172696420,33.907540824720293700,23.183909440666088700,'bD','MNH_34S_REKK','ra',15,'af','2010-01-01'),
+('2018-01-01','iR','1','C0','ABC','dw','Tk',55.633829470415250135,59.719272511914569753,11.808129171772744906,25.101202231834535087,9.607696585312472635,75.722867715,65.379149305517739900,36,36.564316122969582638,30.299285714344106376,46.966155570132298566,'oZ','MNH_34S_REKK','2n',81,'sk','2010-01-01'),
+('2018-01-01','jT','1','wl','CDE','Wd','oy',70.617199985608908954,68.327012036528644761,23.199845522983460080,77.263245236932631416,13.412770480640487497,22.716667783,32.602714815263571998,77,53.321220219577682622,75.939119586784967887,34.188018831298959993,'Ce','AFD_GFFD_234DF','No',42,'Ki','2010-01-01'),
+('2018-01-01','Pk','1','Lt','PLM','YZ','OZ',2.350255364296265998,11.625233757619484026,35.169240131797152394,5.277131874113814564,30.263333772552171727,20.564065177,67.735031914847684541,75,83.386604866065732868,80.167835905015604074,10.797712531398997108,'dX','MNH_34S_REKK','L8',11,'z5','2010-01-01'),
+('2018-01-01','FT','1','Ej','CDE','1d','Ei',61.332057782663242036,31.402224136695730958,7.906350886877649168,89.782195394735600976,70.969055142406905184,44.520140148,76.315077292503175735,47,57.177942058642784670,4.166306008337098985,52.310617728176608418,'1D','OGK_345_DF3G','7M',32,'5T','2010-01-01'),
+('2018-01-01','l7','1','Hw','XYZ','2w','yW',29.229702085581495287,42.869940413826561091,61.100646671045591915,21.242820597861463490,17.550717841445098757,3.116980760,10.426251340837983888,32,0.844693797947813335,33.770096134014698518,88.114100062344658926,'1F','OGK_345_DF3G','gJ',22,'Yv','2010-01-01'),
+('2017-01-01','bw','1','9W','PLM','DE','Rn',6.665548240305814678,65.867289932367603820,87.515469708592898440,29.137475673560345400,23.581006750858248039,19.827642528,5.331209584153457481,86,74.978579652457219227,42.531435691822733006,66.316141574245043058,'WV','OGK_345_DF3G','ED',87,'hY','2010-01-01'),
+('2017-01-01','lJ','1','zH','ABC','ZH','bB',71.643453760727270970,11.104257350875398143,27.181730008078343707,15.918035675856784986,41.190128258548230995,28.773219100,62.534434037399076284,79,51.370128418839532597,48.411691292347722825,24.353513847677076092,'1d','MNH_34S_REKK','FI',40,'Ka','2010-01-01'),
+('2017-01-01','NT','1','jx','PLM','hX','C3',79.419242678302615841,1.751866939232817946,90.584881785853143175,17.383802609241839689,38.124354795655609389,39.959859245,64.824394171184935885,15,25.207455534423336448,83.738023834531470686,31.736267149099521231,'jP','AFD_GFFD_234DF','J9',64,'yd','2010-01-01'),
+('2017-01-01','iP','1','ti','ABC','ma','Bc',25.475019320992927219,19.800921759804310891,46.728343199913874520,69.444029483172017716,17.449813589090254197,11.487363501,71.394627195137949046,73,44.499383143299315225,71.769214206702996328,25.758292436698892000,'8j','OGK_345_DF3G','V4',83,'Di','2010-01-01'),
+('2018-01-01','4r','1','sy','XYZ','Z9','BL',81.469765872134634393,36.585506940599653559,76.780193929080580285,71.588580352270616740,54.363836157836653350,5.821007704,44.261248744523421346,23,39.678999169479683211,11.918334409614122400,38.409104983585979992,'yd','MNH_34S_REKK','NP',21,'B6','2010-01-01'),
+('2018-01-01','1b','1','LB','PLM','tF','0L',22.968282723551851720,4.694502179661971135,13.604949793483747676,89.642556141327328669,80.332132691245881875,24.369953840,42.722950814067730171,53,68.136452671337581430,49.680663165585190430,43.994169430634118873,'2h','OGK_345_DF3G','dn',53,'tV','2010-01-01'),
+('2018-01-01','DA','1','De','PLM','1q','vE',78.929890910221051612,57.448260906845065691,41.710661063478039380,46.680683333626305567,62.602229366062085466,44.209749163,2.904019766920477559,73,60.235917448111860609,70.862527329647133511,31.636736114626720026,'7U','AFD_GFFD_234DF','8p',12,'So','2010-01-01'),
+('2017-01-01','YT','1','MP','ABC','oX','yr',49.956518890640005315,37.957476083415637945,88.263561377769121760,38.453515337978225849,79.825383407768070345,90.561833806,34.242907580080859872,31,18.809502615686008996,4.232083780384746761,7.936532248786515038,'nc','AFD_GFFD_234DF','oW',53,'18','2010-01-01'),
+('2017-01-01','i3','1','67','CDE','kK','3l',17.600011795822952986,1.785512178350065206,7.329570249035338382,11.978802709459287245,17.519415048345563747,76.201670796,63.330001040764342975,32,56.455032520773707978,69.500928788023164024,46.453886654269917419,'Pa','AFD_GFFD_234DF','Je',80,'xd','2010-01-01'),
+('2018-01-01','hT','1','np','ABC','7j','uD',87.782639454391268394,64.784308112165164957,64.768052876040344764,48.476538057833478220,68.275842269683556270,40.788627052,33.645915656090134178,26,57.363241951015704537,54.593769007141402501,38.281041982557567754,'N9','OGK_345_DF3G','bX',84,'dY','2010-01-01'),
+('2017-01-01','c4','1','4z','XYZ','m2','3x',45.641858931001769490,78.434525445976652527,45.900952883254343920,5.513576956457100970,73.709033426570321194,15.259848945,42.373420393218204258,18,3.666100558275775415,25.135890572510077029,59.910840447327477497,'se','AFD_GFFD_234DF','z9',58,'08','2010-01-01');
+"""
+
+    sql """
+INSERT INTO `t4` VALUES (NULL,'4M','DORIS','td','1','kQ','6G','XM','k3','Jp','02','xA','xj','yK','v3','GR','es','CO','2010-01-01','2010-01-01','9y','Yi','l6','AR','VC','Yp','MK','m7','Kq','da','XYZ','gU','2010-01-01','2010-01-01'),
+(NULL,'Pq','DORIS','Ys','1','yR','FV','kA','z9','1e','02','j2','WB','2g','bW','ve','Md','CO','2010-01-01','2010-01-01','xp','pz','QR','ET','BB','qX','U5','tu','zp','UW','CDE','1L','2010-01-01','2010-01-01'),
+(NULL,'Iz','DORIS','7G','1','pl','Tb','ga','gJ','7W','02','Lu','SI','pi','P5','IP','DW','CO','2010-01-01','2010-01-01','pe','di','EU','TW','SU','N5','7v','cP','jn','Hq','ABC','Ub','2010-01-01','2010-01-01'),
+(NULL,'xz','DORIS','am','1','d2','4N','Us','wt','yN','02','Al','3d','k8','GR','rD','De','CO','2010-01-01','2010-01-01','LG','bM','Cy','lQ','8E','8p','0b','Xz','Ed','c0','PLM','zz','2010-01-01','2010-01-01'),
+(NULL,'WP','DORIS','Xm','1','hr','ee','xv','N7','9K','08','fO','NS','sy','DD','wR','O5','CO','2010-01-01','2010-01-01','Xe','bd','pf','J6','wB','rj','0V','fh','oF','zc','XYZ','W8','2010-01-01','2010-01-01'),
+(NULL,'Ae','DORIS','5t','1','qW','tZ','2Q','JX','7w','08','qG','GJ','gd','3s','Z3','wT','CO','2010-01-01','2010-01-01','UM','6v','Q3','Vh','AA','3F','AU','4i','Wt','4A','CDE','Cn','2010-01-01','2010-01-01'),
+(NULL,'Qd','DORIS','tw','1','S3','Kq','aU','Ks','Oc','08','Dz','AD','s2','ei','g3','qJ','CO','2010-01-01','2010-01-01','63','Aj','7C','Gw','YL','zf','Cq','gs','mT','to','ABC','Qu','2010-01-01','2010-01-01'),
+(NULL,'fy','DORIS','sq','1','RE','PO','eV','Cn','2j','08','Ah','Tr','KT','Ho','6q','YD','CO','2010-01-01','2010-01-01','9q','05','KC','1a','Bp','zD','om','mp','zQ','qC','PLM','1c','2010-01-01','2010-01-01');
+"""
+
+    sql """
+set parallel_fragment_exec_instance_num = 1;
+"""
+    sql """
+set enable_pipeline_engine = 0;
+"""
+    sql """
+set enable_profile = 1;
+"""
+
+    test {
+        sql """
+SELECT DISTINCT
+A.c0 e0 
+,B.d0 e1 
+,B.d2 e2
+,B.d14 e3 
+,A.c2 d4
+,A.c5 d9 
+,A.c4 e6
+,A.c7 e7 
+,A.c8 e8 
+,A.c10 e9 
+,A.c11 e10 
+,A.c9 e11 
+,A.c9-A.c7 e12 
+,A.c15/100 e13 
+,A.c12 e14 
+,A.c13 e15 
+,NULL e16 
+,NULL AS e17
+,NULL AS e18
+,A.c4 d30 
+,NULL d31 
+FROM t3 A
+INNER JOIN t4 B
+ON A.c2=B.d4
+AND B.d30='XYZ'
+WHERE
+A.c19 ='AFD_GFFD_234DF'
+AND A.c0 BETWEEN '2000-01-01' AND '2022-01-01'
+;
+"""
+        rowNum 16
+    }
+}


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


[doris] 06/36: [feature](config) support "experimental" prefix for FE config (#18699)

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

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

commit 6976f3ae32454fd573405dbd854d717ec9097673
Author: Mingyu Chen <mo...@163.com>
AuthorDate: Sun Apr 16 18:32:10 2023 +0800

    [feature](config) support "experimental" prefix for FE config (#18699)
    
    For each release of Doris, there are some experimental features.
    These feature may not stable or qualified enough, and user need to use it by setting config or session variables,
    eg, set enable_mtmv = true, otherwise, these feature is disable by default.
    
    We should explicitly tell user which features are experimental, so that user will notice that and decide whether to
    use it.
    
    Changes
    In this PR, I support the experimental_ prefix for FE config and session variables.
    
    Session Variable
    
    Given enable_nereids_planner as an example.
    
    The Nereids planner is an experimental feature in Doris, so there is an EXPERIMENTAL annotation for it:
    
    @VariableMgr.VarAttr(..., expType = ExperimentalType.EXPERIMENTAL)
    private boolean enableNereidsPlanner = false;
    And for compatibility, user can set it by:
    
    set enable_nereids_planner = true;
    set experimental_enable_nereids_planner = true;
    And for show variables, it will only show experimental_enable_nereids_planner entry.
    
    And you can also see all experimental session variables by:
    
    show variables like "%experimental%"
    Config
    
    Same as session variable, give enable_mtmv as an example.
    
    @ConfField(..., expType = ExperimentalType.EXPERIMENTAL)
    public static boolean enable_mtmv = false;
    User can set it in fe.conf or ADMIN SET FRONTEND CONFIG stmt with both names:
    
    enable_mtmv
    experimental_enable_mtmv
    And user can see all experimental FE configs by:
    
    ADMIN SHOW FRONTEND CONFIG LIKE "%experimental%";
    TODO
    Support this feature for BE config
    
    Only add experimental for:
    
    enable_pipeline_engine
    enable_nereids_planner
    enable_single_replica_insert
    and FE config:
    
    enable_mtmv
    enabel_ssl
    enable_fqdn_mode
    Should modify other config and session vars
---
 .../main/java/org/apache/doris/common/Config.java  |  8 ++-
 .../java/org/apache/doris/common/ConfigBase.java   | 82 ++++++++++++++++++----
 .../org/apache/doris/common/ExperimentalUtil.java  | 36 ++++++++++
 .../java/org/apache/doris/qe/SessionVariable.java  | 30 ++++++--
 .../main/java/org/apache/doris/qe/VariableMgr.java | 45 +++++++++++-
 .../doris/analysis/AdminSetConfigStmtTest.java     | 39 +++++++++-
 .../org/apache/doris/qe/SessionVariablesTest.java  | 77 ++++++++++++++++++++
 .../datev2/tpch_sf0.1_p1/sql/pipeline_q11.sql      |  2 +-
 .../nereids_function_p0/agg_function/agg.groovy    |  4 +-
 run-fe-ut.sh                                       | 12 ++--
 10 files changed, 299 insertions(+), 36 deletions(-)

diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
index 97bf1c0e0b..2636c7650e 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
@@ -17,6 +17,8 @@
 
 package org.apache.doris.common;
 
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
+
 public class Config extends ConfigBase {
 
     /**
@@ -1827,7 +1829,7 @@ public class Config extends ConfigBase {
     /*
      * mtmv is still under dev, remove this config when it is graduate.
      */
-    @ConfField(mutable = true, masterOnly = true)
+    @ConfField(mutable = true, masterOnly = true, expType = ExperimentalType.EXPERIMENTAL)
     public static boolean enable_mtmv = false;
 
     /* Max running task num at the same time, otherwise the submitted task will still be keep in pending poll*/
@@ -1999,7 +2001,7 @@ public class Config extends ConfigBase {
      * When enable_fqdn_mode is true, the name of the pod where be is located will remain unchanged
      * after reconstruction, while the ip can be changed.
      */
-    @ConfField(mutable = false, masterOnly = true)
+    @ConfField(mutable = false, masterOnly = true, expType = ExperimentalType.EXPERIMENTAL)
     public static boolean enable_fqdn_mode = false;
 
     /**
@@ -2036,7 +2038,7 @@ public class Config extends ConfigBase {
     /**
      * If set to ture, doris will establish an encrypted channel based on the SSL protocol with mysql.
      */
-    @ConfField(mutable = false, masterOnly = false)
+    @ConfField(mutable = false, masterOnly = false, expType = ExperimentalType.EXPERIMENTAL)
     public static boolean enable_ssl = true;
 
     /**
diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java b/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java
index 7b1ed7e0d5..eac6973d52 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java
@@ -17,6 +17,8 @@
 
 package org.apache.doris.common;
 
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
+
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -46,10 +48,14 @@ public class ConfigBase {
 
     @Retention(RetentionPolicy.RUNTIME)
     public @interface ConfField {
-        String value() default "";
         boolean mutable() default false;
+
         boolean masterOnly() default false;
+
         String comment() default "";
+
+        ExperimentalType expType() default ExperimentalType.NONE;
+
         Class<? extends ConfHandler> callback() default DefaultConfHandler.class;
     }
 
@@ -87,7 +93,11 @@ public class ConfigBase {
                 if (confField == null) {
                     continue;
                 }
-                confFields.put(confField.value().equals("") ? field.getName() : confField.value(), field);
+                confFields.put(field.getName(), field);
+                if (confField.expType() == ExperimentalType.EXPERIMENTAL
+                        || confField.expType() == ExperimentalType.EXPERIMENTAL_ONLINE) {
+                    confFields.put(ExperimentalUtil.EXPERIMENTAL_PREFIX + field.getName(), field);
+                }
             }
 
             initConf(confFile);
@@ -100,7 +110,11 @@ public class ConfigBase {
                 if (confField == null) {
                     continue;
                 }
-                ldapConfFields.put(confField.value().equals("") ? field.getName() : confField.value(), field);
+                ldapConfFields.put(field.getName(), field);
+                if (confField.expType() == ExperimentalType.EXPERIMENTAL
+                        || confField.expType() == ExperimentalType.EXPERIMENTAL_ONLINE) {
+                    ldapConfFields.put(ExperimentalUtil.EXPERIMENTAL_PREFIX + field.getName(), field);
+                }
             }
             initConf(ldapConfFile);
         }
@@ -129,7 +143,7 @@ public class ConfigBase {
         for (Field f : fields) {
             ConfField anno = f.getAnnotation(ConfField.class);
             if (anno != null) {
-                map.put(anno.value().isEmpty() ? f.getName() : anno.value(), getConfValue(f));
+                map.put(f.getName(), getConfValue(f));
             }
         }
         return map;
@@ -199,8 +213,9 @@ public class ConfigBase {
             }
 
             // ensure that field has property string
-            String confKey = anno.value().equals("") ? f.getName() : anno.value();
-            String confVal = props.getProperty(confKey);
+            String confKey = f.getName();
+            String confVal = props.getProperty(confKey,
+                    props.getProperty(ExperimentalUtil.EXPERIMENTAL_PREFIX + confKey));
             if (Strings.isNullOrEmpty(confVal)) {
                 continue;
             }
@@ -214,7 +229,7 @@ public class ConfigBase {
         }
     }
 
-    public static void setConfigField(Field f, String confVal) throws Exception {
+    private static void setConfigField(Field f, String confVal) throws Exception {
         confVal = confVal.trim();
 
         String[] sa = confVal.split(",");
@@ -320,22 +335,49 @@ public class ConfigBase {
         LOG.info("set config {} to {}", key, value);
     }
 
+    /**
+     * Get display name of experimental configs.
+     * For an experimental config, the given "configsToFilter" contains both config w/o "experimental_" prefix.
+     * We need to return the right display name for these configs, by following rules:
+     * 1. If this config is EXPERIMENTAL, only return the config with "experimental_" prefix.
+     * 2. If this config is not EXPERIMENTAL, only return the config without "experimental_" prefix.
+     *
+     * @param configsToFilter
+     * @param allConfigs
+     */
+    private static void getDisplayConfigInfo(Map<String, Field> configsToFilter, Map<String, Field> allConfigs) {
+        for (Map.Entry<String, Field> e : configsToFilter.entrySet()) {
+            Field f = e.getValue();
+            ConfField confField = f.getAnnotation(ConfField.class);
+            boolean isExperimental = e.getKey().startsWith(ExperimentalUtil.EXPERIMENTAL_PREFIX);
+
+            if (isExperimental && confField.expType() != ExperimentalType.EXPERIMENTAL) {
+                continue;
+            }
+            if (!isExperimental && confField.expType() == ExperimentalType.EXPERIMENTAL) {
+                continue;
+            }
+            allConfigs.put(e.getKey(), f);
+        }
+    }
+
     public static synchronized List<List<String>> getConfigInfo(PatternMatcher matcher) {
         Map<String, Field> allConfFields = Maps.newHashMap();
-        allConfFields.putAll(confFields);
-        allConfFields.putAll(ldapConfFields);
+        getDisplayConfigInfo(confFields, allConfFields);
+        getDisplayConfigInfo(ldapConfFields, allConfFields);
+
         return allConfFields.entrySet().stream().sorted(Map.Entry.comparingByKey()).flatMap(e -> {
             String confKey = e.getKey();
             Field f = e.getValue();
-            ConfField anno = f.getAnnotation(ConfField.class);
+            ConfField confField = f.getAnnotation(ConfField.class);
             if (matcher == null || matcher.match(confKey)) {
                 List<String> config = Lists.newArrayList();
                 config.add(confKey);
                 config.add(getConfValue(f));
                 config.add(f.getType().getSimpleName());
-                config.add(String.valueOf(anno.mutable()));
-                config.add(String.valueOf(anno.masterOnly()));
-                config.add(anno.comment());
+                config.add(String.valueOf(confField.mutable()));
+                config.add(String.valueOf(confField.masterOnly()));
+                config.add(confField.comment());
                 return Stream.of(config);
             } else {
                 return Stream.empty();
@@ -379,4 +421,18 @@ public class ConfigBase {
                     + "will overwrite the configurations in fe.conf");
         }
     }
+
+    public static int getConfigNumByExperimentalType(ExperimentalType type) {
+        int num = 0;
+        for (Field field : Config.class.getFields()) {
+            ConfField confField = field.getAnnotation(ConfField.class);
+            if (confField == null) {
+                continue;
+            }
+            if (confField.expType() == type) {
+                ++num;
+            }
+        }
+        return num;
+    }
 }
diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/ExperimentalUtil.java b/fe/fe-common/src/main/java/org/apache/doris/common/ExperimentalUtil.java
new file mode 100644
index 0000000000..9b164de6b7
--- /dev/null
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/ExperimentalUtil.java
@@ -0,0 +1,36 @@
+// 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.common;
+
+// Currently, this is for FE config and session variable.
+public class ExperimentalUtil {
+    public static final String EXPERIMENTAL_PREFIX = "experimental_";
+
+    public enum ExperimentalType {
+        // Not an experimental item
+        NONE,
+        // An experimental item, it will be shown with `experimental_` prefix
+        // And user can set it with or without `experimental_` prefix.
+        EXPERIMENTAL,
+        // A previous experimental item but now it is GA.
+        // it will be shown without `experimental_` prefix.
+        // But user can set it with or without `experimental_` prefix, for compatibility.
+        EXPERIMENTAL_ONLINE
+    }
+}
+
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 2b03d7797d..b3a253103a 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
@@ -21,6 +21,7 @@ import org.apache.doris.analysis.SetVar;
 import org.apache.doris.analysis.StringLiteral;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.common.util.TimeUtils;
@@ -459,7 +460,7 @@ public class SessionVariable implements Serializable, Writable {
     @VariableMgr.VarAttr(name = ENABLE_COLOCATE_SCAN)
     public boolean enableColocateScan = false;
 
-    @VariableMgr.VarAttr(name = ENABLE_BUCKET_SHUFFLE_JOIN)
+    @VariableMgr.VarAttr(name = ENABLE_BUCKET_SHUFFLE_JOIN, expType = ExperimentalType.EXPERIMENTAL_ONLINE)
     public boolean enableBucketShuffleJoin = true;
 
     @VariableMgr.VarAttr(name = PREFER_JOIN_METHOD)
@@ -529,7 +530,7 @@ public class SessionVariable implements Serializable, Writable {
     public boolean extractWideRangeExpr = true;
 
 
-    @VariableMgr.VarAttr(name = ENABLE_PIPELINE_ENGINE, fuzzy = true)
+    @VariableMgr.VarAttr(name = ENABLE_PIPELINE_ENGINE, fuzzy = true, expType = ExperimentalType.EXPERIMENTAL)
     public boolean enablePipelineEngine = false;
 
     @VariableMgr.VarAttr(name = ENABLE_PARALLEL_OUTFILE)
@@ -620,7 +621,7 @@ public class SessionVariable implements Serializable, Writable {
      * the new optimizer is fully developed. I hope that day
      * would be coming soon.
      */
-    @VariableMgr.VarAttr(name = ENABLE_NEREIDS_PLANNER, needForward = true)
+    @VariableMgr.VarAttr(name = ENABLE_NEREIDS_PLANNER, needForward = true, expType = ExperimentalType.EXPERIMENTAL)
     private boolean enableNereidsPlanner = false;
 
     @VariableMgr.VarAttr(name = DISABLE_NEREIDS_RULES, needForward = true)
@@ -661,7 +662,8 @@ public class SessionVariable implements Serializable, Writable {
     @VariableMgr.VarAttr(name = SESSION_CONTEXT, needForward = true)
     public String sessionContext = "";
 
-    @VariableMgr.VarAttr(name = ENABLE_SINGLE_REPLICA_INSERT, needForward = true)
+    @VariableMgr.VarAttr(name = ENABLE_SINGLE_REPLICA_INSERT,
+            needForward = true, expType = ExperimentalType.EXPERIMENTAL)
     public boolean enableSingleReplicaInsert = false;
 
     @VariableMgr.VarAttr(name = ENABLE_FUNCTION_PUSHDOWN)
@@ -743,7 +745,6 @@ public class SessionVariable implements Serializable, Writable {
     public boolean enableUnicodeNameSupport = false;
 
     @VariableMgr.VarAttr(name = REPEAT_MAX_NUM, needForward = true)
-
     public int repeatMaxNum = 10000;
 
     @VariableMgr.VarAttr(name = GROUP_CONCAT_MAX_LEN)
@@ -1922,7 +1923,22 @@ public class SessionVariable implements Serializable, Writable {
             return;
         }
         setIsSingleSetVar(true);
-        VariableMgr.setVar(this,
-                new SetVar(SessionVariable.ENABLE_NEREIDS_PLANNER, new StringLiteral("false")));
+        VariableMgr.setVar(this, new SetVar(SessionVariable.ENABLE_NEREIDS_PLANNER, new StringLiteral("false")));
+    }
+
+    // return number of variables by given experimental type
+    public int getVariableNumByExperimentalType(ExperimentalType type) {
+        int num = 0;
+        Field[] fields = SessionVariable.class.getDeclaredFields();
+        for (Field f : fields) {
+            VarAttr varAttr = f.getAnnotation(VarAttr.class);
+            if (varAttr == null) {
+                continue;
+            }
+            if (varAttr.expType() == type) {
+                ++num;
+            }
+        }
+        return num;
     }
 }
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 660a5c5f89..6041a5c83b 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
@@ -26,6 +26,8 @@ import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.ErrorCode;
 import org.apache.doris.common.ErrorReport;
+import org.apache.doris.common.ExperimentalUtil;
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
 import org.apache.doris.common.PatternMatcher;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.persist.GlobalVarPersistInfo;
@@ -34,6 +36,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import org.apache.commons.lang.SerializationUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -108,6 +111,10 @@ public class VariableMgr {
     // Map variable name to variable context which have enough information to change variable value.
     // This map contains info of all session and global variables.
     private static ImmutableMap<String, VarContext> ctxByVarName;
+    // Built from ctxByVarName.
+    // If a session variable "foo" is an experimental variable,
+    // its display name is "experimental_foo"
+    private static ImmutableMap<String, VarContext> ctxByDisplayVarName;
 
     // 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.
@@ -131,6 +138,7 @@ public class VariableMgr {
         defaultSessionVariable = new SessionVariable();
         ImmutableSortedMap.Builder<String, VarContext> builder = getStringVarContextBuilder(defaultSessionVariable);
         ctxByVarName = builder.build();
+        ctxByDisplayVarName = getDisplaySessionVars();
     }
 
     public static SessionVariable getDefaultSessionVariable() {
@@ -243,10 +251,20 @@ public class VariableMgr {
     //      setVar: variable information that needs to be set
     public static void setVar(SessionVariable sessionVariable, SetVar setVar)
             throws DdlException {
-        VarContext ctx = ctxByVarName.get(setVar.getVariable());
+        String varName = setVar.getVariable();
+        boolean hasExpPrefix = false;
+        if (varName.startsWith(ExperimentalUtil.EXPERIMENTAL_PREFIX)) {
+            varName = varName.substring(ExperimentalUtil.EXPERIMENTAL_PREFIX.length());
+            hasExpPrefix = true;
+        }
+        VarContext ctx = ctxByVarName.get(varName);
         if (ctx == null) {
             ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_SYSTEM_VARIABLE, setVar.getVariable());
         }
+        // for non experimental variables, can not set it with "experimental_" prefix
+        if (hasExpPrefix && ctx.getField().getAnnotation(VarAttr.class).expType() == ExperimentalType.NONE) {
+            ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_SYSTEM_VARIABLE, setVar.getVariable());
+        }
         // Check variable attribute and setVar
         checkUpdate(setVar, ctx.getFlag());
 
@@ -518,13 +536,34 @@ public class VariableMgr {
         return "";
     }
 
+    /**
+     * return the VarContext map with display var name.
+     * For example, if a session variable "foo" is an experimental variable,
+     * its display name is "experimental_foo"
+     *
+     * @return
+     */
+    private static ImmutableMap<String, VarContext> getDisplaySessionVars() {
+        Map<String, VarContext> result = Maps.newHashMap();
+        for (Map.Entry<String, VarContext> entry : ctxByVarName.entrySet()) {
+            VarContext varContext = entry.getValue();
+            VarAttr varAttr = varContext.getField().getAnnotation(VarAttr.class);
+            if (varAttr.expType() == ExperimentalType.EXPERIMENTAL) {
+                result.put(ExperimentalUtil.EXPERIMENTAL_PREFIX + entry.getKey(), varContext);
+            } else {
+                result.put(entry.getKey(), varContext);
+            }
+        }
+        return ImmutableMap.copyOf(result);
+    }
+
     // Dump all fields. Used for `show variables`
     public static List<List<String>> dump(SetType type, SessionVariable sessionVar, PatternMatcher matcher) {
         List<List<String>> rows = Lists.newArrayList();
         // Hold the read lock when session dump, because this option need to access global variable.
         rlock.lock();
         try {
-            for (Map.Entry<String, VarContext> entry : ctxByVarName.entrySet()) {
+            for (Map.Entry<String, VarContext> entry : ctxByDisplayVarName.entrySet()) {
                 // Filter variable not match to the regex.
                 if (matcher != null && !matcher.match(entry.getKey())) {
                     continue;
@@ -589,6 +628,8 @@ public class VariableMgr {
 
         // Set to true if this variable is fuzzy
         boolean fuzzy() default false;
+
+        ExperimentalType expType() default ExperimentalType.NONE;
     }
 
     private static class VarContext {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java
index 4fc186d5a5..b33b1ccb33 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java
@@ -19,12 +19,21 @@ package org.apache.doris.analysis;
 
 import org.apache.doris.catalog.Env;
 import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.CaseSensibility;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.ConfigBase;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
+import org.apache.doris.common.PatternMatcher;
+import org.apache.doris.common.PatternMatcherWrapper;
 import org.apache.doris.utframe.TestWithFeService;
 
+import org.junit.Assert;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
+import java.util.List;
+
 public class AdminSetConfigStmtTest extends TestWithFeService {
     @Test
     public void testNormal() throws Exception {
@@ -45,10 +54,34 @@ public class AdminSetConfigStmtTest extends TestWithFeService {
 
     @Test
     public void testEmptyConfig() {
-        AnalysisException exception =
-                Assertions.assertThrows(AnalysisException.class,
-                        () -> parseAndAnalyzeStmt("admin set frontend config;"));
+        AnalysisException exception = Assertions.assertThrows(AnalysisException.class,
+                () -> parseAndAnalyzeStmt("admin set frontend config;"));
         Assertions.assertEquals("errCode = 2, detailMessage = config parameter size is not equal to 1",
                 exception.getMessage());
     }
+
+    @Test
+    public void testExperimentalConfig() throws Exception {
+        // 1. set without experimental
+        boolean enableMtmv = Config.enable_mtmv;
+        String stmt = "admin set frontend config('enable_mtmv' = '" + String.valueOf(!enableMtmv) + "');";
+        AdminSetConfigStmt adminSetConfigStmt = (AdminSetConfigStmt) parseAndAnalyzeStmt(stmt);
+        Env.getCurrentEnv().setConfig(adminSetConfigStmt);
+        Assert.assertNotEquals(enableMtmv, Config.enable_mtmv);
+
+        // 2. set with experimental
+        enableMtmv = Config.enable_mtmv;
+        stmt = "admin set frontend config('experimental_enable_mtmv' = '" + String.valueOf(!enableMtmv) + "');";
+        adminSetConfigStmt = (AdminSetConfigStmt) parseAndAnalyzeStmt(stmt);
+        Env.getCurrentEnv().setConfig(adminSetConfigStmt);
+        Assert.assertNotEquals(enableMtmv, Config.enable_mtmv);
+
+        // 3. show config
+        int num = ConfigBase.getConfigNumByExperimentalType(ExperimentalType.EXPERIMENTAL);
+        PatternMatcher matcher = PatternMatcherWrapper.createMysqlPattern("%experimental%",
+                CaseSensibility.CONFIG.getCaseSensibility());
+        List<List<String>> results = ConfigBase.getConfigInfo(matcher);
+        Assert.assertEquals(num, results.size());
+    }
 }
+
diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/SessionVariablesTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/SessionVariablesTest.java
index db5f5c03d5..4bce75b92a 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/qe/SessionVariablesTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/qe/SessionVariablesTest.java
@@ -19,7 +19,14 @@ package org.apache.doris.qe;
 
 import org.apache.doris.analysis.ExportStmt;
 import org.apache.doris.analysis.SetStmt;
+import org.apache.doris.analysis.ShowVariablesStmt;
+import org.apache.doris.common.CaseSensibility;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ExceptionChecker;
+import org.apache.doris.common.ExperimentalUtil.ExperimentalType;
 import org.apache.doris.common.FeConstants;
+import org.apache.doris.common.PatternMatcher;
+import org.apache.doris.common.PatternMatcherWrapper;
 import org.apache.doris.common.util.ProfileManager;
 import org.apache.doris.common.util.RuntimeProfile;
 import org.apache.doris.load.ExportJob;
@@ -29,10 +36,12 @@ import org.apache.doris.utframe.TestWithFeService;
 
 import com.google.common.collect.Lists;
 import mockit.Expectations;
+import org.junit.Assert;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
 import java.lang.reflect.Field;
+import java.util.List;
 import java.util.Map;
 
 public class SessionVariablesTest extends TestWithFeService {
@@ -60,6 +69,74 @@ public class SessionVariablesTest extends TestWithFeService {
         }
     }
 
+    @Test
+    public void testExperimentalSessionVariables() throws Exception {
+        // 1. set without experimental
+        SessionVariable sessionVar = connectContext.getSessionVariable();
+        boolean enableNereids = sessionVar.isEnableNereidsPlanner();
+        String sql = "set enable_nereids_planner=" + (enableNereids ? "false" : "true");
+        SetStmt setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        SetExecutor setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableNereidsPlanner(), enableNereids);
+        // 2. set with experimental
+        enableNereids = sessionVar.isEnableNereidsPlanner();
+        sql = "set experimental_enable_nereids_planner=" + (enableNereids ? "false" : "true");
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableNereidsPlanner(), enableNereids);
+        // 3. set global without experimental
+        enableNereids = sessionVar.isEnableNereidsPlanner();
+        sql = "set global enable_nereids_planner=" + (enableNereids ? "false" : "true");
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableNereidsPlanner(), enableNereids);
+        // 4. set global with experimental
+        enableNereids = sessionVar.isEnableNereidsPlanner();
+        sql = "set global experimental_enable_nereids_planner=" + (enableNereids ? "false" : "true");
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableNereidsPlanner(), enableNereids);
+
+        // 5. set experimental for EXPERIMENTAL_ONLINE var
+        boolean bucketShuffle = sessionVar.isEnableBucketShuffleJoin();
+        sql = "set global experimental_enable_bucket_shuffle_join=" + (bucketShuffle ? "false" : "true");
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableBucketShuffleJoin(), bucketShuffle);
+
+        // 6. set non experimental for EXPERIMENTAL_ONLINE var
+        bucketShuffle = sessionVar.isEnableBucketShuffleJoin();
+        sql = "set global enable_bucket_shuffle_join=" + (bucketShuffle ? "false" : "true");
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        setExecutor = new SetExecutor(connectContext, setStmt);
+        setExecutor.execute();
+        Assert.assertNotEquals(sessionVar.isEnableBucketShuffleJoin(), bucketShuffle);
+
+        // 4. set experimental for none experimental var
+        sql = "set experimental_repeat_max_num=5";
+        setStmt = (SetStmt) parseAndAnalyzeStmt(sql, connectContext);
+        SetExecutor setExecutor2 = new SetExecutor(connectContext, setStmt);
+        ExceptionChecker.expectThrowsWithMsg(DdlException.class, "Unknown system variable",
+                () -> setExecutor2.execute());
+
+        // 5. show variables
+        String showSql = "show variables like '%experimental%'";
+        ShowVariablesStmt showStmt = (ShowVariablesStmt) parseAndAnalyzeStmt(showSql, connectContext);
+        PatternMatcher matcher = null;
+        if (showStmt.getPattern() != null) {
+            matcher = PatternMatcherWrapper.createMysqlPattern(showStmt.getPattern(),
+                    CaseSensibility.VARIABLES.getCaseSensibility());
+        }
+        int num = sessionVar.getVariableNumByExperimentalType(ExperimentalType.EXPERIMENTAL);
+        List<List<String>> result = VariableMgr.dump(showStmt.getType(), sessionVar, matcher);
+        Assert.assertEquals(num, result.size());
+    }
+
     @Test
     public void testForwardSessionVariables() {
         Map<String, String> vars = sessionVariable.getForwardVariables();
diff --git a/regression-test/suites/datev2/tpch_sf0.1_p1/sql/pipeline_q11.sql b/regression-test/suites/datev2/tpch_sf0.1_p1/sql/pipeline_q11.sql
index 39d0795c6b..2e99bf1b52 100644
--- a/regression-test/suites/datev2/tpch_sf0.1_p1/sql/pipeline_q11.sql
+++ b/regression-test/suites/datev2/tpch_sf0.1_p1/sql/pipeline_q11.sql
@@ -1,5 +1,5 @@
 -- tables: partsupp,supplier,nation
-SELECT /*+SET_VAR(enable_pipeline_engine=true) */
+SELECT /*+SET_VAR(experimental_enable_pipeline_engine=true) */
   ps_partkey,
   sum(ps_supplycost * ps_availqty) AS value
 FROM
diff --git a/regression-test/suites/nereids_function_p0/agg_function/agg.groovy b/regression-test/suites/nereids_function_p0/agg_function/agg.groovy
index aa4f8b1ca4..70a262d8fd 100644
--- a/regression-test/suites/nereids_function_p0/agg_function/agg.groovy
+++ b/regression-test/suites/nereids_function_p0/agg_function/agg.groovy
@@ -17,7 +17,7 @@
 
 suite("nereids_agg_fn") {
 	sql 'use regression_test_nereids_function_p0'
-	sql 'set enable_nereids_planner=true'
+	sql 'set experimental_enable_nereids_planner=true'
 	sql 'set enable_fallback_to_original_planner=false'
 	sql '''
 		select any_value(kint) from fn_test group by kbool order by kbool'''
@@ -2733,4 +2733,4 @@ suite("nereids_agg_fn") {
 	qt_sql_window_funnel_BigInt_String_DateTimeV2_Boolean_agg_phase_4_notnull '''
 		select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test'''
 
-}
\ No newline at end of file
+}
diff --git a/run-fe-ut.sh b/run-fe-ut.sh
index 4d2b786932..b7df8c046e 100755
--- a/run-fe-ut.sh
+++ b/run-fe-ut.sh
@@ -33,9 +33,11 @@ Usage: $0 <options>
      --run                build and run ut
 
   Eg.
-    $0                      build and run ut
-    $0 --coverage           build and run coverage statistic
-    $0 --run xxx            build and run the specified class
+    $0                                                                      build and run ut
+    $0 --coverage                                                           build and run coverage statistic
+    $0 --run org.apache.doris.utframe.Demo                                  build and run the test named Demo
+    $0 --run org.apache.doris.utframe.Demo#testCreateDbAndTable+test2       build and run testCreateDbAndTable in Demo test
+    $0 --run org.apache.doris.Demo,org.apache.doris.Demo2                   build and run Demo and Demo2 test
   "
     exit 1
 }
@@ -112,9 +114,9 @@ else
         # eg:
         # sh run-fe-ut.sh --run org.apache.doris.utframe.Demo
         # sh run-fe-ut.sh --run org.apache.doris.utframe.Demo#testCreateDbAndTable+test2
-        "${MVN_CMD}" test -DfailIfNoTests=false -D test="$1"
+        "${MVN_CMD}" test -Dcheckstyle.skip=true -DfailIfNoTests=false -D test="$1"
     else
         echo "Run Frontend UT"
-        "${MVN_CMD}" test -DfailIfNoTests=false
+        "${MVN_CMD}" test -Dcheckstyle.skip=true -DfailIfNoTests=false
     fi
 fi


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


[doris] 27/36: [enhancement](exception) Column filter/replicate supports exception safety (#18503)

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

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

commit fb1f54ddd50663659bbb5bf85918df8d7776f4e8
Author: Xinyi Zou <zo...@gmail.com>
AuthorDate: Tue Apr 18 19:23:09 2023 +0800

    [enhancement](exception) Column filter/replicate supports exception safety (#18503)
---
 be/src/common/status.h                             |  2 +
 be/src/exec/base_scanner.cpp                       |  2 +
 be/src/olap/rowset/segment_v2/segment_iterator.cpp | 15 +++----
 be/src/runtime/thread_context.h                    | 48 +++++++++-------------
 be/src/vec/columns/column_array.cpp                | 24 ++++-------
 be/src/vec/columns/column_complex.h                | 13 ++----
 be/src/vec/columns/column_const.cpp                | 15 ++-----
 be/src/vec/columns/column_decimal.cpp              | 12 ++----
 be/src/vec/columns/column_dummy.h                  |  4 +-
 be/src/vec/columns/column_string.cpp               |  4 +-
 be/src/vec/columns/column_vector.cpp               | 14 ++-----
 be/src/vec/columns/columns_common.cpp              |  8 +---
 be/src/vec/columns/columns_common.h                | 18 ++++++++
 be/src/vec/common/sort/heap_sorter.cpp             |  3 +-
 be/src/vec/common/sort/sorter.cpp                  | 10 ++---
 be/src/vec/core/block.cpp                          |  4 +-
 be/src/vec/core/block.h                            |  3 ++
 be/src/vec/core/block_spill_writer.cpp             |  6 +--
 be/src/vec/core/sort_cursor.h                      |  1 +
 .../exec/format/parquet/vparquet_group_reader.cpp  | 18 ++++----
 .../vec/exec/join/process_hash_table_probe_impl.h  |  4 +-
 be/src/vec/exec/join/vhash_join_node.cpp           | 10 ++---
 be/src/vec/exec/join/vnested_loop_join_node.cpp    |  2 +
 be/src/vec/exec/join/vnested_loop_join_node.h      | 18 ++++----
 be/src/vec/exec/vaggregation_node.cpp              |  2 +-
 be/src/vec/exec/varrow_scanner.cpp                 |  2 +-
 be/src/vec/exprs/vexpr_context.cpp                 |  6 +--
 .../vec/functions/array/function_array_apply.cpp   |  8 +++-
 .../array/function_array_with_constant.cpp         |  3 +-
 be/src/vec/sink/vtablet_sink.cpp                   |  3 +-
 30 files changed, 128 insertions(+), 154 deletions(-)

diff --git a/be/src/common/status.h b/be/src/common/status.h
index ec8fbba4fe..d989ba12c3 100644
--- a/be/src/common/status.h
+++ b/be/src/common/status.h
@@ -209,6 +209,8 @@ E(COLUMN_READ_STREAM, -1706);
 E(COLUMN_STREAM_NOT_EXIST, -1716);
 E(COLUMN_VALUE_NULL, -1717);
 E(COLUMN_SEEK_ERROR, -1719);
+E(COLUMN_NO_MATCH_OFFSETS_SIZE, -1720);
+E(COLUMN_NO_MATCH_FILTER_SIZE, -1721);
 E(DELETE_INVALID_CONDITION, -1900);
 E(DELETE_UPDATE_HEADER_FAILED, -1901);
 E(DELETE_SAVE_HEADER_FAILED, -1902);
diff --git a/be/src/exec/base_scanner.cpp b/be/src/exec/base_scanner.cpp
index 043913a5f8..20633d2c48 100644
--- a/be/src/exec/base_scanner.cpp
+++ b/be/src/exec/base_scanner.cpp
@@ -194,6 +194,7 @@ Status BaseScanner::init_expr_ctxes() {
     return Status::OK();
 }
 
+// need exception safety
 Status BaseScanner::_filter_src_block() {
     auto origin_column_num = _src_block.columns();
     // filter block
@@ -348,6 +349,7 @@ Status BaseScanner::_init_src_block() {
     return Status::OK();
 }
 
+// need exception safety
 Status BaseScanner::_fill_dest_block(vectorized::Block* dest_block, bool* eof) {
     *eof = _scanner_eof;
     _fill_columns_from_path();
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index 4d4aa4ae56..5d3c375f54 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -1601,13 +1601,8 @@ Status SegmentIterator::_read_columns_by_rowids(std::vector<ColumnId>& read_colu
 }
 
 Status SegmentIterator::next_batch(vectorized::Block* block) {
-    Status st;
-    try {
-        st = _next_batch_internal(block);
-    } catch (const doris::Exception& e) {
-        st = Status::Error(e.code(), e.to_string());
-    }
-    return st;
+    RETURN_IF_CATCH_EXCEPTION({ return _next_batch_internal(block); });
+    return Status::OK();
 }
 
 Status SegmentIterator::_next_batch_internal(vectorized::Block* block) {
@@ -1882,7 +1877,8 @@ Status SegmentIterator::_execute_common_expr(uint16_t* sel_rowid_idx, uint16_t&
         }
 
         selected_size = _evaluate_common_expr_filter(sel_rowid_idx, selected_size, filter);
-        vectorized::Block::filter_block_internal(block, _columns_to_filter, filter);
+        RETURN_IF_CATCH_EXCEPTION(
+                vectorized::Block::filter_block_internal(block, _columns_to_filter, filter));
     } else if (auto* const_column =
                        vectorized::check_and_get_column<vectorized::ColumnConst>(*filter_column)) {
         bool ret = const_column->get_bool(0);
@@ -1898,7 +1894,8 @@ Status SegmentIterator::_execute_common_expr(uint16_t* sel_rowid_idx, uint16_t&
                         *filter_column)
                         .get_data();
         selected_size = _evaluate_common_expr_filter(sel_rowid_idx, selected_size, filter);
-        vectorized::Block::filter_block_internal(block, _columns_to_filter, filter);
+        RETURN_IF_CATCH_EXCEPTION(
+                vectorized::Block::filter_block_internal(block, _columns_to_filter, filter));
     }
     return Status::OK();
 }
diff --git a/be/src/runtime/thread_context.h b/be/src/runtime/thread_context.h
index c86691a062..04e8915378 100644
--- a/be/src/runtime/thread_context.h
+++ b/be/src/runtime/thread_context.h
@@ -110,7 +110,7 @@ public:
 };
 
 inline thread_local ThreadContextPtr thread_context_ptr;
-inline thread_local bool enable_thread_catch_bad_alloc = false;
+inline thread_local int enable_thread_catch_bad_alloc = 0;
 
 // To avoid performance problems caused by frequently calling `bthread_getspecific` to obtain bthread TLS
 // in tcmalloc hook, cache the key and value of bthread TLS in pthread TLS.
@@ -292,33 +292,22 @@ private:
     tracker->transfer_to(                               \
             size, doris::thread_context()->thread_mem_tracker_mgr->limiter_mem_tracker_raw())
 
-// Consider catching other memory errors, such as memset failure, etc.
-#define RETURN_IF_CATCH_BAD_ALLOC(stmt)                                                            \
-    do {                                                                                           \
-        doris::thread_context()->thread_mem_tracker_mgr->clear_exceed_mem_limit_msg();             \
-        if (doris::enable_thread_catch_bad_alloc) {                                                \
-            try {                                                                                  \
-                { stmt; }                                                                          \
-            } catch (std::bad_alloc const& e) {                                                    \
-                doris::thread_context()->thread_mem_tracker()->print_log_usage(                    \
-                        doris::thread_context()->thread_mem_tracker_mgr->exceed_mem_limit_msg());  \
-                return Status::MemoryLimitExceeded(fmt::format(                                    \
-                        "PreCatch {}, {}", e.what(),                                               \
-                        doris::thread_context()->thread_mem_tracker_mgr->exceed_mem_limit_msg())); \
-            }                                                                                      \
-        } else {                                                                                   \
-            try {                                                                                  \
-                doris::enable_thread_catch_bad_alloc = true;                                       \
-                Defer defer {[&]() { doris::enable_thread_catch_bad_alloc = false; }};             \
-                { stmt; }                                                                          \
-            } catch (std::bad_alloc const& e) {                                                    \
-                doris::thread_context()->thread_mem_tracker()->print_log_usage(                    \
-                        doris::thread_context()->thread_mem_tracker_mgr->exceed_mem_limit_msg());  \
-                return Status::MemoryLimitExceeded(fmt::format(                                    \
-                        "PreCatch {}, {}", e.what(),                                               \
-                        doris::thread_context()->thread_mem_tracker_mgr->exceed_mem_limit_msg())); \
-            }                                                                                      \
-        }                                                                                          \
+#define RETURN_IF_CATCH_EXCEPTION(stmt)                                                        \
+    do {                                                                                       \
+        try {                                                                                  \
+            doris::thread_context()->thread_mem_tracker_mgr->clear_exceed_mem_limit_msg();     \
+            doris::enable_thread_catch_bad_alloc++;                                            \
+            Defer defer {[&]() { doris::enable_thread_catch_bad_alloc--; }};                   \
+            { stmt; }                                                                          \
+        } catch (std::bad_alloc const& e) {                                                    \
+            doris::thread_context()->thread_mem_tracker()->print_log_usage(                    \
+                    doris::thread_context()->thread_mem_tracker_mgr->exceed_mem_limit_msg());  \
+            return Status::MemoryLimitExceeded(fmt::format(                                    \
+                    "PreCatch {}, {}", e.what(),                                               \
+                    doris::thread_context()->thread_mem_tracker_mgr->exceed_mem_limit_msg())); \
+        } catch (const doris::Exception& e) {                                                  \
+            return Status::Error(e.code(), e.to_string());                                     \
+        }                                                                                      \
     } while (0)
 
 // Mem Hook to consume thread mem tracker
@@ -350,6 +339,7 @@ private:
 #define THREAD_MEM_TRACKER_TRANSFER_FROM(size, tracker) (void)0
 #define CONSUME_MEM_TRACKER(size) (void)0
 #define RELEASE_MEM_TRACKER(size) (void)0
-#define RETURN_IF_CATCH_BAD_ALLOC(stmt) (stmt)
+#define RETURN_IF_CATCH_EXCEPTION(stmt) \
+    { stmt; }
 #endif
 } // namespace doris
diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp
index 35f016d210..79b1aa7914 100644
--- a/be/src/vec/columns/column_array.cpp
+++ b/be/src/vec/columns/column_array.cpp
@@ -440,7 +440,7 @@ size_t ColumnArray::filter_number(const Filter& filter) {
 
 ColumnPtr ColumnArray::filter_string(const Filter& filt, ssize_t result_size_hint) const {
     size_t col_size = get_offsets().size();
-    if (col_size != filt.size()) LOG(FATAL) << "Size of filter doesn't match size of column.";
+    column_match_filter_size(col_size, filt.size());
 
     if (0 == col_size) return ColumnArray::create(data);
 
@@ -503,9 +503,7 @@ ColumnPtr ColumnArray::filter_string(const Filter& filt, ssize_t result_size_hin
 
 size_t ColumnArray::filter_string(const Filter& filter) {
     size_t col_size = get_offsets().size();
-    if (col_size != filter.size()) {
-        LOG(FATAL) << "Size of filter doesn't match size of column.";
-    }
+    column_match_filter_size(col_size, filter.size());
 
     if (0 == col_size) {
         return ColumnArray::create(data);
@@ -567,7 +565,7 @@ size_t ColumnArray::filter_string(const Filter& filter) {
 
 ColumnPtr ColumnArray::filter_generic(const Filter& filt, ssize_t result_size_hint) const {
     size_t size = get_offsets().size();
-    if (size != filt.size()) LOG(FATAL) << "Size of filter doesn't match size of column.";
+    column_match_filter_size(size, filt.size());
 
     if (size == 0) return ColumnArray::create(data);
 
@@ -606,9 +604,7 @@ ColumnPtr ColumnArray::filter_generic(const Filter& filt, ssize_t result_size_hi
 
 size_t ColumnArray::filter_generic(const Filter& filter) {
     size_t size = get_offsets().size();
-    if (size != filter.size()) {
-        LOG(FATAL) << "Size of filter doesn't match size of column.";
-    }
+    column_match_filter_size(size, filter.size());
 
     if (size == 0) {
         return 0;
@@ -788,8 +784,7 @@ void ColumnArray::replicate(const uint32_t* counts, size_t target_size, IColumn&
 template <typename T>
 ColumnPtr ColumnArray::replicate_number(const IColumn::Offsets& replicate_offsets) const {
     size_t col_size = size();
-    if (col_size != replicate_offsets.size())
-        LOG(FATAL) << "Size of offsets doesn't match size of column.";
+    column_match_offsets_size(col_size, replicate_offsets.size());
 
     MutableColumnPtr res = clone_empty();
 
@@ -836,8 +831,7 @@ ColumnPtr ColumnArray::replicate_number(const IColumn::Offsets& replicate_offset
 
 ColumnPtr ColumnArray::replicate_string(const IColumn::Offsets& replicate_offsets) const {
     size_t col_size = size();
-    if (col_size != replicate_offsets.size())
-        LOG(FATAL) << "Size of offsets doesn't match size of column.";
+    column_match_offsets_size(col_size, replicate_offsets.size());
 
     MutableColumnPtr res = clone_empty();
 
@@ -910,8 +904,7 @@ ColumnPtr ColumnArray::replicate_string(const IColumn::Offsets& replicate_offset
 
 ColumnPtr ColumnArray::replicate_const(const IColumn::Offsets& replicate_offsets) const {
     size_t col_size = size();
-    if (col_size != replicate_offsets.size())
-        LOG(FATAL) << "Size of offsets doesn't match size of column.";
+    column_match_offsets_size(col_size, replicate_offsets.size());
 
     if (0 == col_size) return clone_empty();
 
@@ -944,8 +937,7 @@ ColumnPtr ColumnArray::replicate_const(const IColumn::Offsets& replicate_offsets
 
 ColumnPtr ColumnArray::replicate_generic(const IColumn::Offsets& replicate_offsets) const {
     size_t col_size = size();
-    if (col_size != replicate_offsets.size())
-        LOG(FATAL) << "Size of offsets doesn't match size of column.";
+    column_match_offsets_size(col_size, replicate_offsets.size());
 
     MutableColumnPtr res = clone_empty();
     ColumnArray& res_concrete = assert_cast<ColumnArray&>(*res);
diff --git a/be/src/vec/columns/column_complex.h b/be/src/vec/columns/column_complex.h
index 95a6793853..138f3d0fb3 100644
--- a/be/src/vec/columns/column_complex.h
+++ b/be/src/vec/columns/column_complex.h
@@ -29,6 +29,7 @@
 #include "vec/columns/column_impl.h"
 #include "vec/columns/column_string.h"
 #include "vec/columns/column_vector.h"
+#include "vec/columns/columns_common.h"
 #include "vec/core/types.h"
 
 namespace doris::vectorized {
@@ -313,9 +314,7 @@ template <typename T>
 ColumnPtr ColumnComplexType<T>::filter(const IColumn::Filter& filt,
                                        ssize_t result_size_hint) const {
     size_t size = data.size();
-    if (size != filt.size()) {
-        LOG(FATAL) << "Size of filter doesn't match size of column.";
-    }
+    column_match_filter_size(size, filt.size());
 
     if (data.size() == 0) return this->create();
     auto res = this->create();
@@ -340,9 +339,7 @@ ColumnPtr ColumnComplexType<T>::filter(const IColumn::Filter& filt,
 template <typename T>
 size_t ColumnComplexType<T>::filter(const IColumn::Filter& filter) {
     size_t size = data.size();
-    if (size != filter.size()) {
-        LOG(FATAL) << "Size of filter doesn't match size of column.";
-    }
+    column_match_filter_size(size, filter.size());
 
     if (data.size() == 0) {
         return 0;
@@ -391,9 +388,7 @@ ColumnPtr ColumnComplexType<T>::permute(const IColumn::Permutation& perm, size_t
 template <typename T>
 ColumnPtr ColumnComplexType<T>::replicate(const IColumn::Offsets& offsets) const {
     size_t size = data.size();
-    if (size != offsets.size()) {
-        LOG(FATAL) << "Size of offsets doesn't match size of column.";
-    }
+    column_match_offsets_size(size, offsets.size());
 
     if (0 == size) return this->create();
 
diff --git a/be/src/vec/columns/column_const.cpp b/be/src/vec/columns/column_const.cpp
index 96d0c013b1..0beec35355 100644
--- a/be/src/vec/columns/column_const.cpp
+++ b/be/src/vec/columns/column_const.cpp
@@ -55,19 +55,13 @@ ColumnPtr ColumnConst::remove_low_cardinality() const {
 }
 
 ColumnPtr ColumnConst::filter(const Filter& filt, ssize_t /*result_size_hint*/) const {
-    if (s != filt.size()) {
-        LOG(FATAL) << fmt::format("Size of filter ({}) doesn't match size of column ({})",
-                                  filt.size(), s);
-    }
+    column_match_filter_size(s, filt.size());
 
     return ColumnConst::create(data, count_bytes_in_filter(filt));
 }
 
 size_t ColumnConst::filter(const Filter& filter) {
-    if (s != filter.size()) {
-        LOG(FATAL) << fmt::format("Size of filter ({}) doesn't match size of column ({})",
-                                  filter.size(), s);
-    }
+    column_match_filter_size(s, filter.size());
 
     const auto result_size = count_bytes_in_filter(filter);
     resize(result_size);
@@ -75,10 +69,7 @@ size_t ColumnConst::filter(const Filter& filter) {
 }
 
 ColumnPtr ColumnConst::replicate(const Offsets& offsets) const {
-    if (s != offsets.size()) {
-        LOG(FATAL) << fmt::format("Size of offsets ({}) doesn't match size of column ({})",
-                                  offsets.size(), s);
-    }
+    column_match_offsets_size(s, offsets.size());
 
     size_t replicated_size = 0 == s ? 0 : offsets.back();
     return ColumnConst::create(data, replicated_size);
diff --git a/be/src/vec/columns/column_decimal.cpp b/be/src/vec/columns/column_decimal.cpp
index 7731e05389..ab375d1d0d 100644
--- a/be/src/vec/columns/column_decimal.cpp
+++ b/be/src/vec/columns/column_decimal.cpp
@@ -276,9 +276,7 @@ void ColumnDecimal<T>::insert_range_from(const IColumn& src, size_t start, size_
 template <typename T>
 ColumnPtr ColumnDecimal<T>::filter(const IColumn::Filter& filt, ssize_t result_size_hint) const {
     size_t size = data.size();
-    if (size != filt.size()) {
-        LOG(FATAL) << "Size of filter doesn't match size of column.";
-    }
+    column_match_filter_size(size, filt.size());
 
     auto res = this->create(0, scale);
     Container& res_data = res->get_data();
@@ -327,9 +325,7 @@ ColumnPtr ColumnDecimal<T>::filter(const IColumn::Filter& filt, ssize_t result_s
 template <typename T>
 size_t ColumnDecimal<T>::filter(const IColumn::Filter& filter) {
     size_t size = data.size();
-    if (size != filter.size()) {
-        LOG(FATAL) << "Size of filter doesn't match size of column.";
-    }
+    column_match_filter_size(size, filter.size());
 
     const UInt8* filter_pos = filter.data();
     const UInt8* filter_end = filter_pos + size;
@@ -382,9 +378,7 @@ size_t ColumnDecimal<T>::filter(const IColumn::Filter& filter) {
 template <typename T>
 ColumnPtr ColumnDecimal<T>::replicate(const IColumn::Offsets& offsets) const {
     size_t size = data.size();
-    if (size != offsets.size()) {
-        LOG(FATAL) << "Size of offsets doesn't match size of column.";
-    }
+    column_match_offsets_size(size, offsets.size());
 
     auto res = this->create(0, scale);
     if (0 == size) return res;
diff --git a/be/src/vec/columns/column_dummy.h b/be/src/vec/columns/column_dummy.h
index b66b284e3f..f6b36ae80a 100644
--- a/be/src/vec/columns/column_dummy.h
+++ b/be/src/vec/columns/column_dummy.h
@@ -108,9 +108,7 @@ public:
     }
 
     ColumnPtr replicate(const Offsets& offsets) const override {
-        if (s != offsets.size()) {
-            LOG(FATAL) << "Size of offsets doesn't match size of column.";
-        }
+        column_match_offsets_size(s, offsets.size());
 
         return clone_dummy(offsets.back());
     }
diff --git a/be/src/vec/columns/column_string.cpp b/be/src/vec/columns/column_string.cpp
index eff29d17a4..c6317e7453 100644
--- a/be/src/vec/columns/column_string.cpp
+++ b/be/src/vec/columns/column_string.cpp
@@ -377,9 +377,7 @@ void ColumnString::get_permutation(bool reverse, size_t limit, int /*nan_directi
 
 ColumnPtr ColumnString::replicate(const Offsets& replicate_offsets) const {
     size_t col_size = size();
-    if (col_size != replicate_offsets.size()) {
-        LOG(FATAL) << "Size of offsets doesn't match size of column.";
-    }
+    column_match_offsets_size(col_size, replicate_offsets.size());
 
     auto res = ColumnString::create();
 
diff --git a/be/src/vec/columns/column_vector.cpp b/be/src/vec/columns/column_vector.cpp
index 4deaa46b69..45c8b94ddb 100644
--- a/be/src/vec/columns/column_vector.cpp
+++ b/be/src/vec/columns/column_vector.cpp
@@ -387,10 +387,7 @@ void ColumnVector<T>::insert_indices_from(const IColumn& src, const int* indices
 template <typename T>
 ColumnPtr ColumnVector<T>::filter(const IColumn::Filter& filt, ssize_t result_size_hint) const {
     size_t size = data.size();
-    if (size != filt.size()) {
-        LOG(FATAL) << "Size of filter doesn't match size of column. data size: " << size
-                   << ", filter size: " << filt.size() << get_stack_trace();
-    }
+    column_match_filter_size(size, filt.size());
 
     auto res = this->create();
     if constexpr (std::is_same_v<T, vectorized::Int64>) {
@@ -444,10 +441,7 @@ ColumnPtr ColumnVector<T>::filter(const IColumn::Filter& filt, ssize_t result_si
 template <typename T>
 size_t ColumnVector<T>::filter(const IColumn::Filter& filter) {
     size_t size = data.size();
-    if (size != filter.size()) {
-        LOG(FATAL) << "Size of filter doesn't match size of column. data size: " << size
-                   << ", filter size: " << filter.size() << get_stack_trace();
-    }
+    column_match_filter_size(size, filter.size());
 
     const UInt8* filter_pos = filter.data();
     const UInt8* filter_end = filter_pos + size;
@@ -523,9 +517,7 @@ ColumnPtr ColumnVector<T>::permute(const IColumn::Permutation& perm, size_t limi
 template <typename T>
 ColumnPtr ColumnVector<T>::replicate(const IColumn::Offsets& offsets) const {
     size_t size = data.size();
-    if (size != offsets.size()) {
-        LOG(FATAL) << "Size of offsets doesn't match size of column.";
-    }
+    column_match_offsets_size(size, offsets.size());
 
     auto res = this->create();
     if constexpr (std::is_same_v<T, vectorized::Int64>) {
diff --git a/be/src/vec/columns/columns_common.cpp b/be/src/vec/columns/columns_common.cpp
index 5665fd2998..4a4200a002 100644
--- a/be/src/vec/columns/columns_common.cpp
+++ b/be/src/vec/columns/columns_common.cpp
@@ -153,9 +153,7 @@ void filter_arrays_impl_generic(const PaddedPODArray<T>& src_elems,
                                 PaddedPODArray<OT>* res_offsets, const IColumn::Filter& filt,
                                 ssize_t result_size_hint) {
     const size_t size = src_offsets.size();
-    if (size != filt.size()) {
-        LOG(FATAL) << "Size of filter doesn't match size of column.";
-    }
+    column_match_filter_size(size, filt.size());
 
     constexpr int ASSUME_STRING_LENGTH = 5;
     ResultOffsetsBuilder result_offsets_builder(res_offsets);
@@ -233,9 +231,7 @@ size_t filter_arrays_impl_generic_without_reserving(PaddedPODArray<T>& elems,
                                                     PaddedPODArray<OT>& offsets,
                                                     const IColumn::Filter& filter) {
     const size_t size = offsets.size();
-    if (offsets.size() != filter.size()) {
-        LOG(FATAL) << "Size of filter doesn't match size of column.";
-    }
+    column_match_filter_size(size, filter.size());
 
     /// If no need to filter the `offsets`, here do not reset the end ptr of `offsets`
     if constexpr (!std::is_same_v<ResultOffsetsBuilder, NoResultOffsetsBuilder<OT>>) {
diff --git a/be/src/vec/columns/columns_common.h b/be/src/vec/columns/columns_common.h
index 3bae20f0ba..6102cef86b 100644
--- a/be/src/vec/columns/columns_common.h
+++ b/be/src/vec/columns/columns_common.h
@@ -59,6 +59,24 @@ template <typename T, typename OT>
 size_t filter_arrays_impl_only_data(PaddedPODArray<T>& data, PaddedPODArray<OT>& offsets,
                                     const IColumn::Filter& filter);
 
+inline void column_match_offsets_size(size_t size, size_t offsets_size) {
+    if (size != offsets_size) {
+        throw doris::Exception(
+                ErrorCode::COLUMN_NO_MATCH_OFFSETS_SIZE,
+                "Size of offsets doesn't match size of column: size={}, offsets.size={}", size,
+                offsets_size);
+    }
+}
+
+inline void column_match_filter_size(size_t size, size_t filter_size) {
+    if (size != filter_size) {
+        throw doris::Exception(
+                ErrorCode::COLUMN_NO_MATCH_FILTER_SIZE,
+                "Size of filter doesn't match size of column: size={}, filter.size={}", size,
+                filter_size);
+    }
+}
+
 namespace detail {
 template <typename T>
 const PaddedPODArray<T>* get_indexes_data(const IColumn& indexes);
diff --git a/be/src/vec/common/sort/heap_sorter.cpp b/be/src/vec/common/sort/heap_sorter.cpp
index de36223915..406176ff51 100644
--- a/be/src/vec/common/sort/heap_sorter.cpp
+++ b/be/src/vec/common/sort/heap_sorter.cpp
@@ -74,7 +74,7 @@ Status HeapSorter::append_block(Block* block) {
     if (_heap_size == _heap->size()) {
         {
             SCOPED_TIMER(_topn_filter_timer);
-            _do_filter(block_view->value(), num_rows);
+            RETURN_IF_CATCH_EXCEPTION(_do_filter(block_view->value(), num_rows));
         }
         size_t remain_rows = block_view->value().block.rows();
         _topn_filter_rows += (num_rows - remain_rows);
@@ -155,6 +155,7 @@ Field HeapSorter::get_top_value() {
     return field;
 }
 
+// need exception safety
 void HeapSorter::_do_filter(HeapSortCursorBlockView& block_view, size_t num_rows) {
     const auto& top_cursor = _heap->top();
     const int cursor_rid = top_cursor.row_id();
diff --git a/be/src/vec/common/sort/sorter.cpp b/be/src/vec/common/sort/sorter.cpp
index b5315a1489..2e630fb44c 100644
--- a/be/src/vec/common/sort/sorter.cpp
+++ b/be/src/vec/common/sort/sorter.cpp
@@ -305,13 +305,9 @@ Status FullSorter::append_block(Block* block) {
             DCHECK(data[i].type->equals(*(arrival_data[i].type)))
                     << " type1: " << data[i].type->get_name()
                     << " type2: " << arrival_data[i].type->get_name();
-            try {
-                //TODO: to eliminate unnecessary expansion, we need a `insert_range_from_const` for every column type.
-                RETURN_IF_CATCH_BAD_ALLOC(data[i].column->assume_mutable()->insert_range_from(
-                        *arrival_data[i].column->convert_to_full_column_if_const(), 0, sz));
-            } catch (const doris::Exception& e) {
-                return Status::Error(e.code(), e.to_string());
-            }
+            //TODO: to eliminate unnecessary expansion, we need a `insert_range_from_const` for every column type.
+            RETURN_IF_CATCH_EXCEPTION(data[i].column->assume_mutable()->insert_range_from(
+                    *arrival_data[i].column->convert_to_full_column_if_const(), 0, sz));
         }
         block->clear_column_data();
     }
diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp
index 0625718767..63b79d7965 100644
--- a/be/src/vec/core/block.cpp
+++ b/be/src/vec/core/block.cpp
@@ -738,7 +738,7 @@ Status Block::filter_block(Block* block, const std::vector<uint32_t>& columns_to
         for (size_t i = 0; i < size; ++i) {
             filter_data[i] &= !null_map[i];
         }
-        filter_block_internal(block, columns_to_filter, filter);
+        RETURN_IF_CATCH_EXCEPTION(filter_block_internal(block, columns_to_filter, filter));
     } else if (auto* const_column = check_and_get_column<ColumnConst>(*filter_column)) {
         bool ret = const_column->get_bool(0);
         if (!ret) {
@@ -750,7 +750,7 @@ Status Block::filter_block(Block* block, const std::vector<uint32_t>& columns_to
         const IColumn::Filter& filter =
                 assert_cast<const doris::vectorized::ColumnVector<UInt8>&>(*filter_column)
                         .get_data();
-        filter_block_internal(block, columns_to_filter, filter);
+        RETURN_IF_CATCH_EXCEPTION(filter_block_internal(block, columns_to_filter, filter));
     }
 
     erase_useless_column(block, column_to_keep);
diff --git a/be/src/vec/core/block.h b/be/src/vec/core/block.h
index 36810edd28..dba2e62ea3 100644
--- a/be/src/vec/core/block.h
+++ b/be/src/vec/core/block.h
@@ -107,6 +107,7 @@ public:
     ColumnWithTypeAndName& get_by_position(size_t position) { return data[position]; }
     const ColumnWithTypeAndName& get_by_position(size_t position) const { return data[position]; }
 
+    // need exception safety
     Status copy_column_data_to_block(doris::vectorized::IColumn* input_col_ptr,
                                      uint16_t* sel_rowid_idx, uint16_t select_size, int block_cid,
                                      size_t batch_size) {
@@ -264,9 +265,11 @@ public:
 
     void append_block_by_selector(MutableBlock* dst, const IColumn::Selector& selector) const;
 
+    // need exception safety
     static void filter_block_internal(Block* block, const std::vector<uint32_t>& columns_to_filter,
                                       const IColumn::Filter& filter);
 
+    // need exception safety
     static void filter_block_internal(Block* block, const IColumn::Filter& filter,
                                       uint32_t column_to_keep);
 
diff --git a/be/src/vec/core/block_spill_writer.cpp b/be/src/vec/core/block_spill_writer.cpp
index b1cca3a851..a501253c63 100644
--- a/be/src/vec/core/block_spill_writer.cpp
+++ b/be/src/vec/core/block_spill_writer.cpp
@@ -87,14 +87,12 @@ Status BlockSpillWriter::write(const Block& block) {
             auto& dst_data = tmp_block_.get_columns_with_type_and_name();
 
             size_t block_rows = std::min(rows - row_idx, batch_size_);
-            try {
+            RETURN_IF_CATCH_EXCEPTION({
                 for (size_t col_idx = 0; col_idx < block.columns(); ++col_idx) {
                     dst_data[col_idx].column->assume_mutable()->insert_range_from(
                             *src_data[col_idx].column, row_idx, block_rows);
                 }
-            } catch (const doris::Exception& e) {
-                return Status::Error(e.code(), e.to_string());
-            }
+            });
 
             RETURN_IF_ERROR(_write_internal(tmp_block_));
 
diff --git a/be/src/vec/core/sort_cursor.h b/be/src/vec/core/sort_cursor.h
index 01834f8345..aced2c022d 100644
--- a/be/src/vec/core/sort_cursor.h
+++ b/be/src/vec/core/sort_cursor.h
@@ -38,6 +38,7 @@ public:
         _reset();
     }
 
+    // need exception safety
     void filter_block(IColumn::Filter& filter) {
         Block::filter_block_internal(&block, filter, block.columns());
         _reset();
diff --git a/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp
index 0604106b03..29c2f488a5 100644
--- a/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp
+++ b/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp
@@ -272,11 +272,12 @@ Status RowGroupReader::next_batch(Block* block, size_t batch_size, size_t* read_
             if (_position_delete_ctx.has_filter) {
                 filters.push_back(_pos_delete_filter_ptr.get());
             }
-            RETURN_IF_ERROR(_execute_conjuncts_and_filter_block(_filter_conjuncts, filters, block,
-                                                                columns_to_filter, column_to_keep));
+            RETURN_IF_CATCH_EXCEPTION(RETURN_IF_ERROR(_execute_conjuncts_and_filter_block(
+                    _filter_conjuncts, filters, block, columns_to_filter, column_to_keep)));
             _convert_dict_cols_to_string_cols(block);
         } else {
-            RETURN_IF_ERROR(_filter_block(block, column_to_keep, columns_to_filter));
+            RETURN_IF_CATCH_EXCEPTION(
+                    RETURN_IF_ERROR(_filter_block(block, column_to_keep, columns_to_filter)));
         }
 
         *read_rows = block->rows();
@@ -440,8 +441,8 @@ Status RowGroupReader::_do_lazy_read(Block* block, size_t batch_size, size_t* re
             // generated from next batch, so the filter column is removed ahead.
             DCHECK_EQ(block->rows(), 0);
         } else {
-            RETURN_IF_ERROR(_filter_block_internal(block, _lazy_read_ctx.all_predicate_col_ids,
-                                                   result_filter));
+            RETURN_IF_CATCH_EXCEPTION(RETURN_IF_ERROR(_filter_block_internal(
+                    block, _lazy_read_ctx.all_predicate_col_ids, result_filter)));
             Block::erase_useless_column(block, origin_column_num);
         }
     } else {
@@ -633,6 +634,7 @@ Status RowGroupReader::_build_pos_delete_filter(size_t read_rows) {
     return Status::OK();
 }
 
+// need exception safety
 Status RowGroupReader::_filter_block(Block* block, int column_to_keep,
                                      const std::vector<uint32_t>& columns_to_filter) {
     if (_pos_delete_filter_ptr) {
@@ -644,6 +646,7 @@ Status RowGroupReader::_filter_block(Block* block, int column_to_keep,
     return Status::OK();
 }
 
+// need exception safety
 Status RowGroupReader::_filter_block_internal(Block* block,
                                               const std::vector<uint32_t>& columns_to_filter,
                                               const IColumn::Filter& filter) {
@@ -734,8 +737,8 @@ Status RowGroupReader::_rewrite_dict_predicates() {
             temp_block.get_by_position(0).column->assume_mutable()->resize(dict_value_column_size);
         }
         std::vector<IColumn::Filter*> filters;
-        RETURN_IF_ERROR(_execute_conjuncts_and_filter_block(*ctxs, filters, &temp_block,
-                                                            columns_to_filter, column_to_keep));
+        RETURN_IF_CATCH_EXCEPTION(RETURN_IF_ERROR(_execute_conjuncts_and_filter_block(
+                *ctxs, filters, &temp_block, columns_to_filter, column_to_keep)));
         if (dict_pos != 0) {
             // We have to clean the first column to insert right data.
             temp_block.get_by_position(0).column->assume_mutable()->clear();
@@ -978,6 +981,7 @@ Status RowGroupReader::_execute_conjuncts(const std::vector<VExprContext*>& ctxs
 }
 
 // TODO Performance Optimization
+// need exception safety
 Status RowGroupReader::_execute_conjuncts_and_filter_block(
         const std::vector<VExprContext*>& ctxs, const std::vector<IColumn::Filter*>& filters,
         Block* block, std::vector<uint32_t>& columns_to_filter, int column_to_keep) {
diff --git a/be/src/vec/exec/join/process_hash_table_probe_impl.h b/be/src/vec/exec/join/process_hash_table_probe_impl.h
index ba49134c03..c5fbc34b01 100644
--- a/be/src/vec/exec/join/process_hash_table_probe_impl.h
+++ b/be/src/vec/exec/join/process_hash_table_probe_impl.h
@@ -392,7 +392,7 @@ Status ProcessHashTableProbe<JoinOpType>::do_process(HashTableType& hash_table_c
     if constexpr (JoinOpType != TJoinOp::RIGHT_SEMI_JOIN &&
                   JoinOpType != TJoinOp::RIGHT_ANTI_JOIN) {
         SCOPED_TIMER(_probe_side_output_timer);
-        RETURN_IF_CATCH_BAD_ALLOC(
+        RETURN_IF_CATCH_EXCEPTION(
                 probe_side_output_column(mcol, _join_node->_left_output_slot_flags, current_offset,
                                          last_probe_index, probe_size, all_match_one, false));
     }
@@ -665,7 +665,7 @@ Status ProcessHashTableProbe<JoinOpType>::do_process_with_other_join_conjuncts(
         }
         {
             SCOPED_TIMER(_probe_side_output_timer);
-            RETURN_IF_CATCH_BAD_ALLOC(probe_side_output_column(
+            RETURN_IF_CATCH_EXCEPTION(probe_side_output_column(
                     mcol, _join_node->_left_output_slot_flags, current_offset, last_probe_index,
                     probe_size, all_match_one, true));
         }
diff --git a/be/src/vec/exec/join/vhash_join_node.cpp b/be/src/vec/exec/join/vhash_join_node.cpp
index 9c8a8a10f0..2dc9e09073 100644
--- a/be/src/vec/exec/join/vhash_join_node.cpp
+++ b/be/src/vec/exec/join/vhash_join_node.cpp
@@ -106,7 +106,7 @@ struct ProcessHashTableBuild {
         // the hash table build bucket, which may waste a lot of memory.
         // TODO, use the NDV expansion of the key column in the optimizer statistics
         if (!_join_node->_build_unique) {
-            RETURN_IF_CATCH_BAD_ALLOC(hash_table_ctx.hash_table.expanse_for_add_elem(
+            RETURN_IF_CATCH_EXCEPTION(hash_table_ctx.hash_table.expanse_for_add_elem(
                     std::min<int>(_rows, config::hash_table_pre_expanse_max_rows)));
         }
 
@@ -479,7 +479,7 @@ Status HashJoinNode::pull(doris::RuntimeState* state, vectorized::Block* output_
     Status st;
     if (_probe_index < _probe_block.rows()) {
         DCHECK(_has_set_need_null_map_for_probe);
-        try {
+        RETURN_IF_CATCH_EXCEPTION({
             std::visit(
                     [&](auto&& arg, auto&& process_hashtable_ctx, auto need_null_map_for_probe,
                         auto ignore_null) {
@@ -516,9 +516,7 @@ Status HashJoinNode::pull(doris::RuntimeState* state, vectorized::Block* output_
                     *_hash_table_variants, *_process_hashtable_ctx_variants,
                     make_bool_variant(_need_null_map_for_probe),
                     make_bool_variant(_probe_ignore_null));
-        } catch (const doris::Exception& e) {
-            return Status::Error(e.code(), e.to_string());
-        }
+        });
     } else if (_probe_eos) {
         if (_is_right_semi_anti || (_is_outer_join && _join_op != TJoinOp::LEFT_OUTER_JOIN)) {
             std::visit(
@@ -774,7 +772,7 @@ Status HashJoinNode::sink(doris::RuntimeState* state, vectorized::Block* in_bloc
 
         if (in_block->rows() != 0) {
             SCOPED_TIMER(_build_side_merge_block_timer);
-            RETURN_IF_CATCH_BAD_ALLOC(_build_side_mutable_block.merge(*in_block));
+            RETURN_IF_CATCH_EXCEPTION(_build_side_mutable_block.merge(*in_block));
         }
 
         if (UNLIKELY(_build_side_mem_used - _build_side_last_mem_used > BUILD_BLOCK_MAX_SIZE)) {
diff --git a/be/src/vec/exec/join/vnested_loop_join_node.cpp b/be/src/vec/exec/join/vnested_loop_join_node.cpp
index c995929ca1..ee7d0cdc51 100644
--- a/be/src/vec/exec/join/vnested_loop_join_node.cpp
+++ b/be/src/vec/exec/join/vnested_loop_join_node.cpp
@@ -488,6 +488,7 @@ void VNestedLoopJoinNode::_reset_with_next_probe_row() {
         block->get_by_position(i).column->assume_mutable()->clear(); \
     }
 
+// need exception safety
 template <typename Filter, bool SetBuildSideFlag, bool SetProbeSideFlag>
 void VNestedLoopJoinNode::_do_filtering_and_update_visited_flags_impl(
         Block* block, int column_to_keep, int build_block_idx, int processed_blocks_num,
@@ -517,6 +518,7 @@ void VNestedLoopJoinNode::_do_filtering_and_update_visited_flags_impl(
     }
 }
 
+// need exception safety
 template <bool SetBuildSideFlag, bool SetProbeSideFlag, bool IgnoreNull>
 Status VNestedLoopJoinNode::_do_filtering_and_update_visited_flags(Block* block, bool materialize) {
     auto column_to_keep = block->columns();
diff --git a/be/src/vec/exec/join/vnested_loop_join_node.h b/be/src/vec/exec/join/vnested_loop_join_node.h
index 84f5f18d69..537ed2d7f0 100644
--- a/be/src/vec/exec/join/vnested_loop_join_node.h
+++ b/be/src/vec/exec/join/vnested_loop_join_node.h
@@ -105,10 +105,11 @@ private:
             }
 
             if constexpr (set_probe_side_flag) {
-                auto status =
-                        _do_filtering_and_update_visited_flags<set_build_side_flag,
-                                                               set_probe_side_flag, ignore_null>(
-                                &_join_block, !_is_left_semi_anti);
+                Status status;
+                RETURN_IF_CATCH_EXCEPTION(
+                        (status = _do_filtering_and_update_visited_flags<
+                                 set_build_side_flag, set_probe_side_flag, ignore_null>(
+                                 &_join_block, !_is_left_semi_anti)));
                 _update_additional_flags(&_join_block);
                 if (!status.ok()) {
                     return status;
@@ -141,10 +142,11 @@ private:
         }
 
         if constexpr (!set_probe_side_flag) {
-            Status status =
-                    _do_filtering_and_update_visited_flags<set_build_side_flag, set_probe_side_flag,
-                                                           ignore_null>(&_join_block,
-                                                                        !_is_right_semi_anti);
+            Status status;
+            RETURN_IF_CATCH_EXCEPTION(
+                    (status = _do_filtering_and_update_visited_flags<
+                             set_build_side_flag, set_probe_side_flag, ignore_null>(
+                             &_join_block, !_is_right_semi_anti)));
             _update_additional_flags(&_join_block);
             mutable_join_block = MutableBlock(&_join_block);
             if (!status.ok()) {
diff --git a/be/src/vec/exec/vaggregation_node.cpp b/be/src/vec/exec/vaggregation_node.cpp
index 77a0d0e7cb..c19a4b7730 100644
--- a/be/src/vec/exec/vaggregation_node.cpp
+++ b/be/src/vec/exec/vaggregation_node.cpp
@@ -1115,7 +1115,7 @@ Status AggregationNode::_pre_agg_with_serialized_key(doris::vectorized::Block* i
             _agg_data->_aggregated_method_variant));
 
     if (!ret_flag) {
-        RETURN_IF_CATCH_BAD_ALLOC(_emplace_into_hash_table(_places.data(), key_columns, rows));
+        RETURN_IF_CATCH_EXCEPTION(_emplace_into_hash_table(_places.data(), key_columns, rows));
 
         for (int i = 0; i < _aggregate_evaluators.size(); ++i) {
             RETURN_IF_ERROR(_aggregate_evaluators[i]->execute_batch_add(
diff --git a/be/src/vec/exec/varrow_scanner.cpp b/be/src/vec/exec/varrow_scanner.cpp
index 9e3137f7c6..eae0bb0deb 100644
--- a/be/src/vec/exec/varrow_scanner.cpp
+++ b/be/src/vec/exec/varrow_scanner.cpp
@@ -251,7 +251,7 @@ Status VArrowScanner::get_next(vectorized::Block* block, bool* eof) {
     RETURN_IF_ERROR(_cast_src_block(&_src_block));
 
     // materialize, src block => dest columns
-    return _fill_dest_block(block, eof);
+    RETURN_IF_CATCH_EXCEPTION({ return _fill_dest_block(block, eof); });
 }
 
 // arrow type ==arrow_column_to_doris_column==> primitive type(PT0) ==cast_src_block==>
diff --git a/be/src/vec/exprs/vexpr_context.cpp b/be/src/vec/exprs/vexpr_context.cpp
index 7d881e5ef2..ab8ee1d696 100644
--- a/be/src/vec/exprs/vexpr_context.cpp
+++ b/be/src/vec/exprs/vexpr_context.cpp
@@ -38,12 +38,10 @@ VExprContext::~VExprContext() {
 
 doris::Status VExprContext::execute(doris::vectorized::Block* block, int* result_column_id) {
     Status st;
-    try {
+    RETURN_IF_CATCH_EXCEPTION({
         st = _root->execute(this, block, result_column_id);
         _last_result_column_id = *result_column_id;
-    } catch (const doris::Exception& e) {
-        st = Status::Error(e.code(), e.to_string());
-    }
+    });
     return st;
 }
 
diff --git a/be/src/vec/functions/array/function_array_apply.cpp b/be/src/vec/functions/array/function_array_apply.cpp
index bb81f04fda..a47a28f281 100644
--- a/be/src/vec/functions/array/function_array_apply.cpp
+++ b/be/src/vec/functions/array/function_array_apply.cpp
@@ -68,8 +68,9 @@ public:
         const ColumnConst& rhs_value_column =
                 static_cast<const ColumnConst&>(*block.get_by_position(arguments[2]).column.get());
         ColumnPtr result_ptr;
-        RETURN_IF_ERROR(_execute(*src_nested_column, nested_type, src_offsets, condition,
-                                 rhs_value_column, &result_ptr));
+        RETURN_IF_CATCH_EXCEPTION(
+                RETURN_IF_ERROR(_execute(*src_nested_column, nested_type, src_offsets, condition,
+                                         rhs_value_column, &result_ptr)));
         block.replace_by_position(result, std::move(result_ptr));
         return Status::OK();
     }
@@ -107,6 +108,7 @@ private:
         __builtin_unreachable();
     }
 
+    // need exception safety
     template <typename T, ApplyOp op>
     ColumnPtr _apply_internal(const IColumn& src_column, const ColumnArray::Offsets64& src_offsets,
                               const ColumnConst& cmp) {
@@ -144,6 +146,7 @@ private:
         return ColumnArray::create(filtered, std::move(column_offsets));
     }
 
+// need exception safety
 #define APPLY_ALL_TYPES(src_column, src_offsets, OP, cmp, dst)                     \
     do {                                                                           \
         WhichDataType which(remove_nullable(nested_type));                         \
@@ -186,6 +189,7 @@ private:
         }                                                                          \
     } while (0)
 
+    // need exception safety
     Status _execute(const IColumn& nested_src, DataTypePtr nested_type,
                     const ColumnArray::Offsets64& offsets, const std::string& condition,
                     const ColumnConst& rhs_value_column, ColumnPtr* dst) {
diff --git a/be/src/vec/functions/array/function_array_with_constant.cpp b/be/src/vec/functions/array/function_array_with_constant.cpp
index 5ee91b97a3..bb8e789525 100644
--- a/be/src/vec/functions/array/function_array_with_constant.cpp
+++ b/be/src/vec/functions/array/function_array_with_constant.cpp
@@ -78,7 +78,8 @@ public:
         }
         auto clone = value->clone_empty();
         clone->reserve(input_rows_count);
-        value->replicate(array_sizes.data(), offset, *clone->assume_mutable().get());
+        RETURN_IF_CATCH_EXCEPTION(
+                value->replicate(array_sizes.data(), offset, *clone->assume_mutable().get()));
         if (!clone->is_nullable()) {
             clone = ColumnNullable::create(std::move(clone), ColumnUInt8::create(clone->size(), 0));
         }
diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp
index ad3796477d..d1bcf7fcca 100644
--- a/be/src/vec/sink/vtablet_sink.cpp
+++ b/be/src/vec/sink/vtablet_sink.cpp
@@ -1201,7 +1201,8 @@ Status VOlapTableSink::send(RuntimeState* state, vectorized::Block* input_block,
             for (size_t i = 0; i < filter_col.size(); ++i) {
                 filter_data[i] = !_filter_bitmap.Get(i);
             }
-            vectorized::Block::filter_block_internal(&block, filter_col, block.columns());
+            RETURN_IF_CATCH_EXCEPTION(
+                    vectorized::Block::filter_block_internal(&block, filter_col, block.columns()));
         }
     }
     // Add block to node channel


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


[doris] 03/36: [regression-test](prepared statement) Fix connection error when test framework used lower jdbc version (#18665)

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

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

commit 08f4cf03e8fd948e6d1bf8097e4c6417ba6645b6
Author: lihangyu <15...@163.com>
AuthorDate: Sun Apr 16 18:13:45 2023 +0800

    [regression-test](prepared statement) Fix connection error when test framework used lower jdbc version (#18665)
---
 .../java/org/apache/doris/qe/ConnectProcessor.java |  6 ++++
 .../test_compaction_uniq_keys_row_store.groovy     | 25 ++++++++++----
 .../suites/point_query_p0/test_point_query.groovy  | 40 +++++++++++++++++-----
 3 files changed, 55 insertions(+), 16 deletions(-)

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 2d8d4da210..12b4b289da 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
@@ -212,6 +212,12 @@ public class ConnectProcessor {
         packetBuf.getInt();
         LOG.debug("execute prepared statement {}", stmtId);
         PrepareStmtContext prepareCtx = ctx.getPreparedStmt(String.valueOf(stmtId));
+        if (prepareCtx == null) {
+            LOG.debug("No such statement in context, stmtId:{}", stmtId);
+            ctx.getState().setError(ErrorCode.ERR_UNKNOWN_COM_ERROR,
+                    "msg: Not supported such prepared statement");
+            return;
+        }
         int paramCount = prepareCtx.stmt.getParmCount();
         // null bitmap
         byte[] nullbitmapData = new byte[(paramCount + 7) / 8];
diff --git a/regression-test/suites/compaction/test_compaction_uniq_keys_row_store.groovy b/regression-test/suites/compaction/test_compaction_uniq_keys_row_store.groovy
index ff79af35f4..06499b7986 100644
--- a/regression-test/suites/compaction/test_compaction_uniq_keys_row_store.groovy
+++ b/regression-test/suites/compaction/test_compaction_uniq_keys_row_store.groovy
@@ -19,7 +19,10 @@ import org.codehaus.groovy.runtime.IOGroovyMethods
 
 
 suite("test_compaction_uniq_keys_row_store") {
-    def tableName = "compaction_uniq_keys_row_store_regression_test"
+    def realDb = "regression_test_serving_p0"
+    def tableName = realDb + ".compaction_uniq_keys_row_store_regression_test"
+    sql "CREATE DATABASE IF NOT EXISTS ${realDb}"
+
     def setPrepareStmtArgs = {stmt, user_id, date, datev2, datetimev2_1, datetimev2_2, city, age, sex ->
         java.text.SimpleDateFormat formater = new java.text.SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SS");
         stmt.setInt(1, user_id)
@@ -73,7 +76,20 @@ suite("test_compaction_uniq_keys_row_store") {
         def checkValue = { ->
             def user = context.config.jdbcUser
             def password = context.config.jdbcPassword
-            def url = context.config.jdbcUrl + "&useServerPrepStmts=true"
+            // Parse url
+            String jdbcUrl = context.config.jdbcUrl
+            String urlWithoutSchema = jdbcUrl.substring(jdbcUrl.indexOf("://") + 3)
+            def sql_ip = urlWithoutSchema.substring(0, urlWithoutSchema.indexOf(":"))
+            def sql_port
+            if (urlWithoutSchema.indexOf("/") >= 0) {
+                // e.g: jdbc:mysql://locahost:8080/?a=b
+                sql_port = urlWithoutSchema.substring(urlWithoutSchema.indexOf(":") + 1, urlWithoutSchema.indexOf("/"))
+            } else {
+                // e.g: jdbc:mysql://locahost:8080
+                sql_port = urlWithoutSchema.substring(urlWithoutSchema.indexOf(":") + 1)
+            }
+            // set server side prepared statment url
+            def url="jdbc:mysql://" + sql_ip + ":" + sql_port + "/" + realDb + "?&useServerPrepStmts=true"
             def result1 = connect(user=user, password=password, url=url) {
                 def stmt = prepareStatement """ SELECT * FROM ${tableName} t where user_id = ? and date = ? and datev2 = ? and datetimev2_1 = ? and datetimev2_2 = ? and city = ? and age = ? and sex = ?; """
                 setPrepareStmtArgs stmt, 1, '2017-10-01', '2017-10-01', '2017-10-01 11:11:11.21', '2017-10-01 11:11:11.11', 'Beijing', 10, 1
@@ -97,10 +113,7 @@ suite("test_compaction_uniq_keys_row_store") {
 
         def user = context.config.jdbcUser
         def password = context.config.jdbcPassword
-        def url = context.config.jdbcUrl + "&useServerPrepStmts=true"
         def tablets = null
-        def result1 = connect(user=user, password=password, url=url) {
-
         sql """ DROP TABLE IF EXISTS ${tableName} """
         sql """
             CREATE TABLE IF NOT EXISTS ${tableName} (
@@ -157,11 +170,9 @@ suite("test_compaction_uniq_keys_row_store") {
             """
         //TabletId,ReplicaIdBackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,PathHash,MetaUrl,CompactionStatus
         tablets = sql """ show tablets from ${tableName}; """
-        }
 
         checkValue()
 
-
         // trigger compactions for all tablets in ${tableName}
         for (String[] tablet in tablets) {
             String tablet_id = tablet[0]
diff --git a/regression-test/suites/point_query_p0/test_point_query.groovy b/regression-test/suites/point_query_p0/test_point_query.groovy
index d69527dff1..96db424a34 100644
--- a/regression-test/suites/point_query_p0/test_point_query.groovy
+++ b/regression-test/suites/point_query_p0/test_point_query.groovy
@@ -18,10 +18,26 @@
 import java.math.BigDecimal;
 
 suite("test_point_query") {
-    def tableName = "tbl_point_query"
     def user = context.config.jdbcUser
     def password = context.config.jdbcPassword
-    def url = context.config.jdbcUrl + "&useServerPrepStmts=true"
+    def realDb = "regression_test_serving_p0"
+    def tableName = realDb + ".tbl_point_query"
+    sql "CREATE DATABASE IF NOT EXISTS ${realDb}"
+
+    // Parse url
+    String jdbcUrl = context.config.jdbcUrl
+    String urlWithoutSchema = jdbcUrl.substring(jdbcUrl.indexOf("://") + 3)
+    def sql_ip = urlWithoutSchema.substring(0, urlWithoutSchema.indexOf(":"))
+    def sql_port
+    if (urlWithoutSchema.indexOf("/") >= 0) {
+        // e.g: jdbc:mysql://locahost:8080/?a=b
+        sql_port = urlWithoutSchema.substring(urlWithoutSchema.indexOf(":") + 1, urlWithoutSchema.indexOf("/"))
+    } else {
+        // e.g: jdbc:mysql://locahost:8080
+        sql_port = urlWithoutSchema.substring(urlWithoutSchema.indexOf(":") + 1)
+    }
+    // set server side prepared statment url
+    def url="jdbc:mysql://" + sql_ip + ":" + sql_port + "/" + realDb + "?&useServerPrepStmts=true"
 
     def generateString = {len ->
         def str = ""
@@ -31,8 +47,6 @@ suite("test_point_query") {
         return str
     }
 
-    // def url = context.config.jdbcUrl
-    def result1 = connect(user=user, password=password, url=url) {
     sql """DROP TABLE IF EXISTS ${tableName}"""
     test {
         // abnormal case
@@ -82,6 +96,14 @@ suite("test_point_query") {
       sql """ INSERT INTO ${tableName} VALUES(252, 120939.11130, "${generateString(252)}", "laooq", "2030-01-02", "2020-01-01 12:36:38", 252, "7022-01-01 11:30:38") """
       sql """ INSERT INTO ${tableName} VALUES(298, 120939.11130, "${generateString(298)}", "laooq", "2030-01-02", "2020-01-01 12:36:38", 298, "7022-01-01 11:30:38") """
 
+    def nprep_sql = {sql_str->
+        def url_without_prep ="jdbc:mysql://" + sql_ip + ":" + sql_port + "/" + realDb
+        connect(user=user, password=password, url=url_without_prep) {
+            sql sql_str
+        }
+    }
+    // def url = context.config.jdbcUrl
+    def result1 = connect(user=user, password=password, url=url) {
       def stmt = prepareStatement "select * from ${tableName} where k1 = ? and k2 = ? and k3 = ?"
       assertEquals(stmt.class, com.mysql.cj.jdbc.ServerPreparedStatement);
       stmt.setInt(1, 1231)
@@ -125,26 +147,26 @@ suite("test_point_query") {
       qe_point_select stmt_fn
       qe_point_select stmt_fn
 
-      sql """
+      nprep_sql """
           ALTER table ${tableName} ADD COLUMN new_column0 INT default "0";
           """
       sleep(1);
-      sql """ INSERT INTO ${tableName} VALUES(1235, 120939.11130, "a    ddd", "laooq", "2030-01-02", "2020-01-01 12:36:38", 22.822, "7022-01-01 11:30:38", 123) """
+      nprep_sql """ INSERT INTO ${tableName} VALUES(1235, 120939.11130, "a    ddd", "laooq", "2030-01-02", "2020-01-01 12:36:38", 22.822, "7022-01-01 11:30:38", 123) """
       stmt.setBigDecimal(1, new BigDecimal("120939.11130"))
       stmt.setString(2, "a    ddd")
       qe_point_select stmt
       qe_point_select stmt
       // invalidate cache
-      sql """ INSERT INTO ${tableName} VALUES(1235, 120939.11130, "a    ddd", "xxxxxx", "2030-01-02", "2020-01-01 12:36:38", 22.822, "7022-01-01 11:30:38", 123) """
+      nprep_sql """ INSERT INTO ${tableName} VALUES(1235, 120939.11130, "a    ddd", "xxxxxx", "2030-01-02", "2020-01-01 12:36:38", 22.822, "7022-01-01 11:30:38", 123) """
       qe_point_select stmt
       qe_point_select stmt
       qe_point_select stmt
-      sql """
+      nprep_sql """
           ALTER table ${tableName} ADD COLUMN new_column1 INT default "0";
           """
       qe_point_select stmt
       qe_point_select stmt
-      sql """
+      nprep_sql """
           ALTER table ${tableName} DROP COLUMN new_column1;
           """
       qe_point_select stmt


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


[doris] 19/36: [fix](merge-on-write) the mow keyset check is quite slow when loading large data set, remove it from release version (#18750)

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

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

commit f28944d9fea3378c54c90c4af07e6a051c13fec1
Author: zhannngchen <48...@users.noreply.github.com>
AuthorDate: Tue Apr 18 09:51:25 2023 +0800

    [fix](merge-on-write) the mow keyset check is quite slow when loading large data set, remove it from release version (#18750)
---
 be/src/olap/rowset/beta_rowset_writer.cpp        | 2 ++
 be/src/olap/rowset/segment_v2/segment_writer.cpp | 4 ++++
 be/src/olap/tablet.cpp                           | 2 +-
 3 files changed, 7 insertions(+), 1 deletion(-)

diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp
index 3b7bd5490f..8b9b04a82c 100644
--- a/be/src/olap/rowset/beta_rowset_writer.cpp
+++ b/be/src/olap/rowset/beta_rowset_writer.cpp
@@ -610,10 +610,12 @@ void BetaRowsetWriter::_build_rowset_meta(std::shared_ptr<RowsetMeta> rowset_met
             total_data_size += itr.second.data_size;
             total_index_size += itr.second.index_size;
             segments_encoded_key_bounds.push_back(itr.second.key_bounds);
+#ifndef NDEBUG
             if (_context.enable_unique_key_merge_on_write) {
                 DCHECK(itr.second.key_set.get() != nullptr);
                 key_set.insert(itr.second.key_set->begin(), itr.second.key_set->end());
             }
+#endif
         }
     }
     _num_mow_keys = key_set.size();
diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp
index 32787a29f4..3363f71cde 100644
--- a/be/src/olap/rowset/segment_v2/segment_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp
@@ -219,7 +219,9 @@ Status SegmentWriter::init(const std::vector<uint32_t>& col_ids, bool has_key,
             _primary_key_index_builder.reset(
                     new PrimaryKeyIndexBuilder(_file_writer, seq_col_length));
             RETURN_IF_ERROR(_primary_key_index_builder->init());
+#ifndef NDEBUG
             _key_set.reset(new std::unordered_set<std::string>());
+#endif
         } else {
             _short_key_index_builder.reset(
                     new ShortKeyIndexBuilder(_segment_id, _opts.num_rows_per_block));
@@ -327,8 +329,10 @@ Status SegmentWriter::append_block(const vectorized::Block* block, size_t row_po
             // create primary indexes
             for (size_t pos = 0; pos < num_rows; pos++) {
                 std::string key = _full_encode_keys(key_columns, pos);
+#ifndef NDEBUG
                 DCHECK(_key_set.get() != nullptr);
                 _key_set->insert(key);
+#endif
                 if (_tablet_schema->has_sequence_col()) {
                     _encode_seq_column(seq_column, pos, &key);
                 }
diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp
index 8abe60c80c..76d203333c 100644
--- a/be/src/olap/tablet.cpp
+++ b/be/src/olap/tablet.cpp
@@ -2655,7 +2655,7 @@ Status Tablet::update_delete_bitmap(const RowsetSharedPtr& rowset, const TabletT
     RETURN_IF_ERROR(calc_delete_bitmap(rowset->rowset_id(), segments, &rowset_ids_to_add,
                                        delete_bitmap, cur_version - 1, true));
 
-    // Check the delete_bitmap correctness.
+    // Check the delete_bitmap correctness, now the check is only enabled in DEBUG env.
     if (load_info->num_keys != 0) {
         DeleteBitmap rs_bm(tablet_id());
         delete_bitmap->subset({rowset->rowset_id(), 0, 0},


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


[doris] 07/36: [Fix](catalog)Fix hudi-catalog get file split error (#18644) (#18673)

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

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

commit 9013b3712c269b42eefa68ebb1cefd559b32dc45
Author: Calvin Kirs <AC...@yeah.net>
AuthorDate: Sun Apr 16 21:56:14 2023 +0800

    [Fix](catalog)Fix hudi-catalog get file split error (#18644) (#18673)
    
    `hudi-common` depends on `parque-avro`, but the dependency scope is `provide`.
    When we use `hudi-catalog`, `HoodieAvroWriteSupport` will be called. This method depends on `parque-avro`, so it will generate ClassNotFound
    Describe your changes.
---
 fe/fe-core/pom.xml | 5 ++++-
 fe/pom.xml         | 5 +++++
 2 files changed, 9 insertions(+), 1 deletion(-)

diff --git a/fe/fe-core/pom.xml b/fe/fe-core/pom.xml
index 090f2b8a66..3856db03cc 100644
--- a/fe/fe-core/pom.xml
+++ b/fe/fe-core/pom.xml
@@ -701,7 +701,10 @@ under the License.
             <groupId>org.apache.hudi</groupId>
             <artifactId>hudi-hadoop-mr</artifactId>
         </dependency>
-
+        <dependency>
+            <groupId>org.apache.parquet</groupId>
+            <artifactId>parquet-avro</artifactId>
+        </dependency>        
         <dependency> 
             <groupId>org.mariadb.jdbc</groupId>
             <artifactId>mariadb-java-client</artifactId>
diff --git a/fe/pom.xml b/fe/pom.xml
index f776568de8..b53353fd9a 100644
--- a/fe/pom.xml
+++ b/fe/pom.xml
@@ -837,6 +837,11 @@ under the License.
                 <artifactId>hudi-hadoop-mr</artifactId>
                 <version>${hudi.version}</version>
             </dependency>
+            <dependency>
+                <groupId>org.apache.parquet</groupId>
+                <artifactId>parquet-avro</artifactId>
+                <version>${parquet.version}</version>
+            </dependency>
 
             <dependency>
                 <groupId>org.apache.parquet</groupId>


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


[doris] 24/36: [fix](olap) fix lost disable_auto_compaction info when fe restart (#18757)

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

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

commit 6dc7237346cb3d133d2d3366b6edd48e4bcf3b8f
Author: xueweizhang <zx...@163.com>
AuthorDate: Tue Apr 18 14:11:40 2023 +0800

    [fix](olap) fix lost disable_auto_compaction info when fe restart (#18757)
    
    Signed-off-by: nextdreamblue <zx...@163.com>
---
 fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
index f8b1c15082..add5162d75 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
@@ -347,7 +347,8 @@ public class TableProperty implements Writable {
                 .buildCompressionType()
                 .buildStoragePolicy()
                 .buildEnableLightSchemaChange()
-                .buildStoreRowColumn();
+                .buildStoreRowColumn()
+                .buildDisableAutoCompaction();
         if (Env.getCurrentEnvJournalVersion() < FeMetaVersion.VERSION_105) {
             // get replica num from property map and create replica allocation
             String repNum = tableProperty.properties.remove(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM);


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