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 2024/04/12 03:30:44 UTC

(doris) branch branch-2.1 updated (babec88aa90 -> d31bca199f0)

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

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


    from babec88aa90 fix cloud mode from PR #32748
     new 36a1bf1d73f [feature][insert]Adapt the create tableĀ  statement to the nereids sql (#32458)
     new c68b3530175 [feature][insert]add FE UT and support CTAS for external table (#32525)
     new 70489fe749c [fix](insert)fix hive table sink write path (#32587)
     new 3343322965d [fix](insert)fix conversion of doris type to hive type (#32735)
     new b98d225183f [fix](insert)fix hive table sink type coercion and unify coercion (#32762)
     new bd364897d42 [feature](hive/iceberg)add doris's version in table properties (#32774)
     new 9ada38327b8 [feature](txn insert) txn insert support insert into select (#31666)
     new f3a6132214a [chore] Format regression-conf.groovy (#32713)
     new 716c1467505 [fix](insert)fix hive external return msgs and exception and pass all columns to BE (#32824)
     new 07f296734a2 [regression](insert)add hive DDL and CTAS regression case (#32924)
     new 3c9c6c18a80 [Enhancement](hive-writer) Write only regular fields to file in the hive-writer. (#33000)
     new 01b21da82d8 [feature](insert)add hive insert plan ut and remove redundant fields (#33051)
     new 7a05396cd11 [feature](multi-catalog)support catalog name when create/drop db (#33116)
     new f0ac21e231e [feature](external) process tbl/db exist when create/drop db/tbl (#33119)
     new e11db3f050c [feature](hive)support ExternalTransaction for writing exteral table (#32726)
     new 31a7060dbdb [testcase](hive)add exception test for hive txn (#33278)
     new 18fb8407ae1 [feature](insert)use optional location and add hive regression test (#33153)
     new f0463a90343 [Feature][Enhancement](hive-writer) Add hive-writer runtime profiles, change output file names (#33245)
     new d31bca199f0 [feature](iceberg)The new DDL syntax is added to create iceberg partitioned tables (#33338)

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


Summary of changes:
 be/src/common/config.h                             |    2 +-
 be/src/vec/runtime/vorc_transformer.cpp            |  104 +-
 be/src/vec/runtime/vorc_transformer.h              |   10 +-
 be/src/vec/runtime/vparquet_transformer.cpp        |   32 +-
 be/src/vec/runtime/vparquet_transformer.h          |   10 +-
 be/src/vec/sink/writer/vhive_partition_writer.cpp  |  169 +--
 be/src/vec/sink/writer/vhive_partition_writer.h    |   17 +-
 be/src/vec/sink/writer/vhive_table_writer.cpp      |  265 ++--
 be/src/vec/sink/writer/vhive_table_writer.h        |   30 +-
 .../main/java/org/apache/doris/common/Config.java  |    6 +-
 fe/fe-core/src/main/cup/sql_parser.cup             |   20 +-
 .../org/apache/doris/analysis/CreateDbStmt.java    |   16 +-
 .../java/org/apache/doris/analysis/DbName.java     |   83 ++
 .../java/org/apache/doris/analysis/DropDbStmt.java |   10 +-
 .../apache/doris/analysis/ShowCreateDbStmt.java    |   13 +-
 .../main/java/org/apache/doris/catalog/Env.java    |   16 +-
 .../doris/catalog/InternalSchemaInitializer.java   |    5 +-
 .../org/apache/doris/common/util/LocationPath.java |    7 +
 .../apache/doris/datasource/ExternalCatalog.java   |   18 +-
 .../apache/doris/datasource/InternalCatalog.java   |    4 +-
 .../apache/doris/datasource/hive/HMSCommitter.java |  754 -----------
 .../doris/datasource/hive/HMSExternalCatalog.java  |    6 +-
 .../doris/datasource/hive/HMSTransaction.java      | 1315 ++++++++++++++++++++
 .../datasource/hive/HiveDatabaseMetadata.java      |    9 +
 .../datasource/hive/HiveMetaStoreClientHelper.java |  103 +-
 .../doris/datasource/hive/HiveMetadataOps.java     |  193 ++-
 .../datasource/hive/HivePartitionStatistics.java   |    2 +-
 .../hive/HivePartitionWithStatistics.java          |    6 +-
 .../doris/datasource/hive/HiveTableMetadata.java   |   86 +-
 .../org/apache/doris/datasource/hive/HiveUtil.java |  213 +++-
 .../datasource/hive/ThriftHMSCachedClient.java     |  166 +--
 .../datasource/iceberg/DorisTypeToIcebergType.java |   14 +-
 .../datasource/iceberg/IcebergMetadataOps.java     |   42 +-
 .../doris/datasource/iceberg/IcebergUtils.java     |   94 +-
 .../datasource/operations/ExternalMetadataOps.java |    2 +
 .../main/java/org/apache/doris/fs/FileSystem.java  |    9 +
 .../org/apache/doris/fs/LocalDfsFileSystem.java    |  245 ++++
 .../java/org/apache/doris/fs/LocalFileSystem.java  |   76 --
 .../apache/doris/fs/remote/dfs/DFSFileSystem.java  |   45 +
 .../doris/insertoverwrite/InsertOverwriteUtil.java |   45 +-
 .../doris/nereids/parser/LogicalPlanBuilder.java   |    9 +-
 .../doris/nereids/parser/PartitionTableInfo.java   |    7 +-
 .../DistributionSpecTableSinkHashPartitioned.java  |    3 -
 .../doris/nereids/rules/analysis/BindSink.java     |  131 +-
 ...ogicalHiveTableSinkToPhysicalHiveTableSink.java |    3 +-
 .../trees/plans/commands/CreateTableCommand.java   |    9 +-
 .../trees/plans/commands/info/CreateTableInfo.java |   97 +-
 .../commands/insert/AbstractInsertExecutor.java    |    2 +-
 .../commands/insert/HiveInsertCommandContext.java  |    2 +-
 .../plans/commands/insert/HiveInsertExecutor.java  |   54 +-
 .../commands/insert/InsertIntoTableCommand.java    |   11 +-
 .../insert/InsertOverwriteTableCommand.java        |   92 +-
 .../trees/plans/commands/insert/InsertUtils.java   |    5 +
 .../plans/commands/insert/OlapInsertExecutor.java  |   30 +-
 .../trees/plans/logical/LogicalHiveTableSink.java  |   27 +-
 .../trees/plans/logical/LogicalOlapTableSink.java  |    8 +-
 .../trees/plans/logical/LogicalTableSink.java      |   17 +-
 .../plans/physical/PhysicalHiveTableSink.java      |   25 +-
 .../trees/plans/physical/PhysicalTableSink.java    |    3 +
 .../org/apache/doris/planner/HiveTableSink.java    |   13 +-
 .../java/org/apache/doris/qe/ConnectContext.java   |   13 +-
 .../main/java/org/apache/doris/qe/Coordinator.java |   30 +-
 .../apache/doris/qe/InsertStreamTxnExecutor.java   |    3 +-
 .../java/org/apache/doris/qe/ShowExecutor.java     |    6 +-
 .../java/org/apache/doris/qe/StmtExecutor.java     |   96 +-
 .../doris/transaction/HiveTransactionManager.java  |   79 ++
 .../Transaction.java}                              |   15 +-
 .../apache/doris/transaction/TransactionEntry.java |  163 ++-
 .../TransactionManager.java}                       |   22 +-
 .../TransactionManagerFactory.java}                |   17 +-
 .../apache/doris/analysis/CreateDbStmtTest.java    |    9 +-
 .../org/apache/doris/analysis/DropDbStmtTest.java  |    7 +-
 .../doris/analysis/ShowCreateDbStmtTest.java       |    5 +-
 .../doris/datasource/HMSCachedClientTest.java      |  328 +++++
 .../datasource/hive/HiveDDLAndDMLPlanTest.java     |  704 +++++++++++
 .../doris/datasource/hive/HiveMetadataOpsTest.java |  173 +++
 .../doris/datasource/hive/HmsCommitTest.java       |  375 ++++--
 .../datasource/iceberg/CreateIcebergTableTest.java |  196 +++
 .../java/org/apache/doris/qe/ShowExecutorTest.java |    5 +-
 .../apache/doris/utframe/TestWithFeService.java    |   18 +
 gensrc/thrift/DataSinks.thrift                     |    3 +-
 regression-test/conf/regression-conf.groovy        |    5 +-
 .../external_table_p0/hive/ddl/test_hive_ctas.out  |   92 ++
 .../external_table_p0/hive/ddl/test_hive_ddl.out   |   60 +
 .../hive/ddl/test_hive_write_type.out              |   18 +
 .../hive/write/test_hive_write_insert.out          |   66 +
 .../hive/write/test_hive_write_partitions.out      |   50 +
 regression-test/data/insert_p0/txn_insert.out      |  259 ++++
 .../pipeline/external/conf/regression-conf.groovy  |   10 +-
 .../pipeline/p0/conf/regression-conf.groovy        |   17 +-
 .../pipeline/p1/conf/regression-conf.groovy        |   17 +-
 .../hive/ddl/test_hive_ctas.groovy                 |  515 ++++++++
 .../hive/ddl/test_hive_ddl.groovy                  |  676 ++++++++++
 .../hive/ddl/test_hive_write_type.groovy           |  285 +++++
 .../hive/write/test_hive_write_insert.groovy       |  786 ++++++------
 .../hive/write/test_hive_write_partitions.groovy   |    3 +-
 regression-test/suites/insert_p0/txn_insert.groovy |  263 ++--
 97 files changed, 7845 insertions(+), 2354 deletions(-)
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/analysis/DbName.java
 delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSCommitter.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/fs/LocalDfsFileSystem.java
 delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/fs/LocalFileSystem.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/transaction/HiveTransactionManager.java
 copy fe/fe-core/src/main/java/org/apache/doris/{datasource/hive/HiveDatabaseMetadata.java => transaction/Transaction.java} (70%)
 copy fe/fe-core/src/main/java/org/apache/doris/{datasource/hive/HiveDatabaseMetadata.java => transaction/TransactionManager.java} (70%)
 copy fe/fe-core/src/main/java/org/apache/doris/{datasource/hive/HiveDatabaseMetadata.java => transaction/TransactionManagerFactory.java} (70%)
 create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/HMSCachedClientTest.java
 create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
 create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
 create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/CreateIcebergTableTest.java
 create mode 100644 regression-test/data/external_table_p0/hive/ddl/test_hive_ctas.out
 create mode 100644 regression-test/data/external_table_p0/hive/ddl/test_hive_ddl.out
 create mode 100644 regression-test/data/external_table_p0/hive/ddl/test_hive_write_type.out
 create mode 100644 regression-test/suites/external_table_p0/hive/ddl/test_hive_ctas.groovy
 create mode 100644 regression-test/suites/external_table_p0/hive/ddl/test_hive_ddl.groovy
 create mode 100644 regression-test/suites/external_table_p0/hive/ddl/test_hive_write_type.groovy


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


(doris) 18/19: [Feature][Enhancement](hive-writer) Add hive-writer runtime profiles, change output file names (#33245)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit f0463a9034389eb2f20de121ca70bc271f65b34a
Author: Qi Chen <ka...@gmail.com>
AuthorDate: Sun Apr 7 15:12:41 2024 +0800

    [Feature][Enhancement](hive-writer) Add hive-writer runtime profiles, change output file names (#33245)
    
    Issue Number: #31442
    
    - Add hive-writer runtime profiles.
    - Change output file names to `${query_id}${uuid}-${index}.${compression}.${format}`. e.g. `"d8735c6fa444a6d-acd392981e510c2b_34fbdcbb-b2e1-4f2c-b68c-a384238954a9-0.snappy.parquet"`. For the same partition writer, when the file size exceeds `hive_sink_max_file_size`, the currently written file will be closed and a new file will be generated, in which ${index} in the new file name will be incremented, while the rest will be the same .
---
 be/src/common/config.h                            |   2 +-
 be/src/vec/sink/writer/vhive_partition_writer.cpp |  55 ++++-
 be/src/vec/sink/writer/vhive_partition_writer.h   |  11 +-
 be/src/vec/sink/writer/vhive_table_writer.cpp     | 245 ++++++++++++----------
 be/src/vec/sink/writer/vhive_table_writer.h       |  27 ++-
 5 files changed, 212 insertions(+), 128 deletions(-)

diff --git a/be/src/common/config.h b/be/src/common/config.h
index 6076ce79433..ae39a6e5eb4 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -1268,7 +1268,7 @@ DECLARE_mInt64(table_sink_partition_write_skewed_data_processed_rebalance_thresh
 DECLARE_mInt32(table_sink_partition_write_max_partition_nums_per_writer);
 
 /** Hive sink configurations **/
-DECLARE_mInt64(hive_sink_max_file_size); // 1GB
+DECLARE_mInt64(hive_sink_max_file_size);
 
 // Number of open tries, default 1 means only try to open once.
 // Retry the Open num_retries time waiting 100 milliseconds between retries.
diff --git a/be/src/vec/sink/writer/vhive_partition_writer.cpp b/be/src/vec/sink/writer/vhive_partition_writer.cpp
index 7f9a0dd1b1e..f8fd13c5a27 100644
--- a/be/src/vec/sink/writer/vhive_partition_writer.cpp
+++ b/be/src/vec/sink/writer/vhive_partition_writer.cpp
@@ -32,8 +32,8 @@ VHivePartitionWriter::VHivePartitionWriter(
         const TDataSink& t_sink, std::string partition_name, TUpdateMode::type update_mode,
         const VExprContextSPtrs& output_expr_ctxs, const VExprContextSPtrs& write_output_expr_ctxs,
         const std::set<size_t>& non_write_columns_indices, const std::vector<THiveColumn>& columns,
-        WriteInfo write_info, std::string file_name, TFileFormatType::type file_format_type,
-        TFileCompressType::type hive_compress_type,
+        WriteInfo write_info, std::string file_name, int file_name_index,
+        TFileFormatType::type file_format_type, TFileCompressType::type hive_compress_type,
         const std::map<std::string, std::string>& hadoop_conf)
         : _partition_name(std::move(partition_name)),
           _update_mode(update_mode),
@@ -43,6 +43,7 @@ VHivePartitionWriter::VHivePartitionWriter(
           _columns(columns),
           _write_info(std::move(write_info)),
           _file_name(std::move(file_name)),
+          _file_name_index(file_name_index),
           _file_format_type(file_format_type),
           _hive_compress_type(hive_compress_type),
           _hadoop_conf(hadoop_conf)
@@ -55,12 +56,14 @@ Status VHivePartitionWriter::open(RuntimeState* state, RuntimeProfile* profile)
     std::vector<TNetworkAddress> broker_addresses;
     RETURN_IF_ERROR(FileFactory::create_file_writer(
             _write_info.file_type, state->exec_env(), broker_addresses, _hadoop_conf,
-            fmt::format("{}/{}", _write_info.write_path, _file_name), 0, _file_writer));
+            fmt::format("{}/{}", _write_info.write_path, _file_name, _file_name_index,
+                        _get_file_extension(_file_format_type, _hive_compress_type)),
+            0, _file_writer));
 
     std::vector<std::string> column_names;
     column_names.reserve(_columns.size());
     for (int i = 0; i < _columns.size(); i++) {
-        column_names.push_back(_columns[i].name);
+        column_names.emplace_back(_columns[i].name);
     }
 
     switch (_file_format_type) {
@@ -192,11 +195,53 @@ THivePartitionUpdate VHivePartitionWriter::_build_partition_update() {
     location.__set_write_path(_write_info.write_path);
     location.__set_target_path(_write_info.target_path);
     hive_partition_update.__set_location(location);
-    hive_partition_update.__set_file_names({_file_name});
+    hive_partition_update.__set_file_names(
+            {fmt::format("{}-{}{}", _file_name, _file_name_index,
+                         _get_file_extension(_file_format_type, _hive_compress_type))});
     hive_partition_update.__set_row_count(_row_count);
     hive_partition_update.__set_file_size(_input_size_in_bytes);
     return hive_partition_update;
 }
 
+std::string VHivePartitionWriter::_get_file_extension(TFileFormatType::type file_format_type,
+                                                      TFileCompressType::type write_compress_type) {
+    std::string compress_name;
+    switch (write_compress_type) {
+    case TFileCompressType::SNAPPYBLOCK: {
+        compress_name = ".snappy";
+        break;
+    }
+    case TFileCompressType::ZLIB: {
+        compress_name = ".zlib";
+        break;
+    }
+    case TFileCompressType::ZSTD: {
+        compress_name = ".zstd";
+        break;
+    }
+    default: {
+        compress_name = "";
+        break;
+    }
+    }
+
+    std::string file_format_name;
+    switch (file_format_type) {
+    case TFileFormatType::FORMAT_PARQUET: {
+        file_format_name = ".parquet";
+        break;
+    }
+    case TFileFormatType::FORMAT_ORC: {
+        file_format_name = ".orc";
+        break;
+    }
+    default: {
+        file_format_name = "";
+        break;
+    }
+    }
+    return fmt::format("{}{}", compress_name, file_format_name);
+}
+
 } // namespace vectorized
 } // namespace doris
diff --git a/be/src/vec/sink/writer/vhive_partition_writer.h b/be/src/vec/sink/writer/vhive_partition_writer.h
index 88488a94673..b8735ddcf7d 100644
--- a/be/src/vec/sink/writer/vhive_partition_writer.h
+++ b/be/src/vec/sink/writer/vhive_partition_writer.h
@@ -48,7 +48,8 @@ public:
                          const VExprContextSPtrs& write_output_expr_ctxs,
                          const std::set<size_t>& non_write_columns_indices,
                          const std::vector<THiveColumn>& columns, WriteInfo write_info,
-                         const std::string file_name, TFileFormatType::type file_format_type,
+                         std::string file_name, int file_name_index,
+                         TFileFormatType::type file_format_type,
                          TFileCompressType::type hive_compress_type,
                          const std::map<std::string, std::string>& hadoop_conf);
 
@@ -60,6 +61,10 @@ public:
 
     Status close(const Status& status);
 
+    inline const std::string& file_name() const { return _file_name; }
+
+    inline int file_name_index() const { return _file_name_index; }
+
     inline size_t written_len() { return _file_format_transformer->written_len(); }
 
 private:
@@ -69,6 +74,9 @@ private:
 
     THivePartitionUpdate _build_partition_update();
 
+    std::string _get_file_extension(TFileFormatType::type file_format_type,
+                                    TFileCompressType::type write_compress_type);
+
     std::string _path;
 
     std::string _partition_name;
@@ -85,6 +93,7 @@ private:
     const std::vector<THiveColumn>& _columns;
     WriteInfo _write_info;
     std::string _file_name;
+    int _file_name_index;
     TFileFormatType::type _file_format_type;
     TFileCompressType::type _hive_compress_type;
     const std::map<std::string, std::string>& _hadoop_conf;
diff --git a/be/src/vec/sink/writer/vhive_table_writer.cpp b/be/src/vec/sink/writer/vhive_table_writer.cpp
index e56090773b5..d43fc34b4e5 100644
--- a/be/src/vec/sink/writer/vhive_table_writer.cpp
+++ b/be/src/vec/sink/writer/vhive_table_writer.cpp
@@ -42,6 +42,19 @@ Status VHiveTableWriter::open(RuntimeState* state, RuntimeProfile* profile) {
     _state = state;
     _profile = profile;
 
+    // add all counter
+    _written_rows_counter = ADD_COUNTER(_profile, "WrittenRows", TUnit::UNIT);
+    _send_data_timer = ADD_TIMER(_profile, "SendDataTime");
+    _partition_writers_dispatch_timer =
+            ADD_CHILD_TIMER(_profile, "PartitionsDispatchTime", "SendDataTime");
+    _partition_writers_write_timer =
+            ADD_CHILD_TIMER(_profile, "PartitionsWriteTime", "SendDataTime");
+    _partition_writers_count = ADD_COUNTER(_profile, "PartitionsWriteCount", TUnit::UNIT);
+    _open_timer = ADD_TIMER(_profile, "OpenTime");
+    _close_timer = ADD_TIMER(_profile, "CloseTime");
+    _write_file_counter = ADD_COUNTER(_profile, "WriteFileCount", TUnit::UNIT);
+
+    SCOPED_TIMER(_open_timer);
     for (int i = 0; i < _t_sink.hive_table_sink.columns.size(); ++i) {
         switch (_t_sink.hive_table_sink.columns[i].column_type) {
         case THiveColumnType::PARTITION_KEY: {
@@ -68,94 +81,120 @@ Status VHiveTableWriter::open(RuntimeState* state, RuntimeProfile* profile) {
 }
 
 Status VHiveTableWriter::write(vectorized::Block& block) {
+    SCOPED_RAW_TIMER(&_send_data_ns);
     std::unordered_map<std::shared_ptr<VHivePartitionWriter>, IColumn::Filter> writer_positions;
-
+    _row_count += block.rows();
     auto& hive_table_sink = _t_sink.hive_table_sink;
 
     if (_partition_columns_input_index.empty()) {
-        auto writer_iter = _partitions_to_writers.find("");
-        if (writer_iter == _partitions_to_writers.end()) {
-            try {
-                std::shared_ptr<VHivePartitionWriter> writer = _create_partition_writer(block, -1);
-                _partitions_to_writers.insert({"", writer});
-                RETURN_IF_ERROR(writer->open(_state, _profile));
-                RETURN_IF_ERROR(writer->write(block));
-            } catch (doris::Exception& e) {
-                return e.to_status();
-            }
-            return Status::OK();
-        } else {
-            std::shared_ptr<VHivePartitionWriter> writer;
-            if (writer_iter->second->written_len() > config::hive_sink_max_file_size) {
-                static_cast<void>(writer_iter->second->close(Status::OK()));
-                _partitions_to_writers.erase(writer_iter);
+        std::shared_ptr<VHivePartitionWriter> writer;
+        {
+            SCOPED_RAW_TIMER(&_partition_writers_dispatch_ns);
+            auto writer_iter = _partitions_to_writers.find("");
+            if (writer_iter == _partitions_to_writers.end()) {
                 try {
                     writer = _create_partition_writer(block, -1);
-                    _partitions_to_writers.insert({"", writer});
-                    RETURN_IF_ERROR(writer->open(_state, _profile));
                 } catch (doris::Exception& e) {
                     return e.to_status();
                 }
+                _partitions_to_writers.insert({"", writer});
+                RETURN_IF_ERROR(writer->open(_state, _profile));
             } else {
-                writer = writer_iter->second;
+                if (writer_iter->second->written_len() > config::hive_sink_max_file_size) {
+                    std::string file_name(writer_iter->second->file_name());
+                    int file_name_index = writer_iter->second->file_name_index();
+                    {
+                        SCOPED_RAW_TIMER(&_close_ns);
+                        static_cast<void>(writer_iter->second->close(Status::OK()));
+                    }
+                    _partitions_to_writers.erase(writer_iter);
+                    try {
+                        writer = _create_partition_writer(block, -1, &file_name,
+                                                          file_name_index + 1);
+                    } catch (doris::Exception& e) {
+                        return e.to_status();
+                    }
+                    _partitions_to_writers.insert({"", writer});
+                    RETURN_IF_ERROR(writer->open(_state, _profile));
+                } else {
+                    writer = writer_iter->second;
+                }
             }
-            RETURN_IF_ERROR(writer->write(block));
-            return Status::OK();
         }
+        SCOPED_RAW_TIMER(&_partition_writers_write_ns);
+        RETURN_IF_ERROR(writer->write(block));
+        return Status::OK();
     }
 
-    for (int i = 0; i < block.rows(); ++i) {
-        std::vector<std::string> partition_values;
-        try {
-            partition_values = _create_partition_values(block, i);
-        } catch (doris::Exception& e) {
-            return e.to_status();
-        }
-        std::string partition_name = VHiveUtils::make_partition_name(
-                hive_table_sink.columns, _partition_columns_input_index, partition_values);
-
-        auto create_and_open_writer =
-                [&](const std::string& partition_name, int position,
-                    std::shared_ptr<VHivePartitionWriter>& writer_ptr) -> Status {
+    {
+        SCOPED_RAW_TIMER(&_partition_writers_dispatch_ns);
+        for (int i = 0; i < block.rows(); ++i) {
+            std::vector<std::string> partition_values;
             try {
-                auto writer = _create_partition_writer(block, position);
-                RETURN_IF_ERROR(writer->open(_state, _profile));
-                IColumn::Filter filter(block.rows(), 0);
-                filter[position] = 1;
-                writer_positions.insert({writer, std::move(filter)});
-                _partitions_to_writers.insert({partition_name, writer});
-                writer_ptr = writer;
+                partition_values = _create_partition_values(block, i);
             } catch (doris::Exception& e) {
                 return e.to_status();
             }
-            return Status::OK();
-        };
+            std::string partition_name = VHiveUtils::make_partition_name(
+                    hive_table_sink.columns, _partition_columns_input_index, partition_values);
 
-        auto writer_iter = _partitions_to_writers.find(partition_name);
-        if (writer_iter == _partitions_to_writers.end()) {
-            std::shared_ptr<VHivePartitionWriter> writer;
-            RETURN_IF_ERROR(create_and_open_writer(partition_name, i, writer));
-        } else {
-            std::shared_ptr<VHivePartitionWriter> writer;
-            if (writer_iter->second->written_len() > config::hive_sink_max_file_size) {
-                static_cast<void>(writer_iter->second->close(Status::OK()));
-                writer_positions.erase(writer_iter->second);
-                _partitions_to_writers.erase(writer_iter);
-                RETURN_IF_ERROR(create_and_open_writer(partition_name, i, writer));
-            } else {
-                writer = writer_iter->second;
-            }
-            auto writer_pos_iter = writer_positions.find(writer);
-            if (writer_pos_iter == writer_positions.end()) {
-                IColumn::Filter filter(block.rows(), 0);
-                filter[i] = 1;
-                writer_positions.insert({writer, std::move(filter)});
+            auto create_and_open_writer =
+                    [&](const std::string& partition_name, int position,
+                        const std::string* file_name, int file_name_index,
+                        std::shared_ptr<VHivePartitionWriter>& writer_ptr) -> Status {
+                try {
+                    auto writer =
+                            _create_partition_writer(block, position, file_name, file_name_index);
+                    RETURN_IF_ERROR(writer->open(_state, _profile));
+                    IColumn::Filter filter(block.rows(), 0);
+                    filter[position] = 1;
+                    writer_positions.insert({writer, std::move(filter)});
+                    _partitions_to_writers.insert({partition_name, writer});
+                    writer_ptr = writer;
+                } catch (doris::Exception& e) {
+                    return e.to_status();
+                }
+                return Status::OK();
+            };
+
+            auto writer_iter = _partitions_to_writers.find(partition_name);
+            if (writer_iter == _partitions_to_writers.end()) {
+                std::shared_ptr<VHivePartitionWriter> writer;
+                if (_partitions_to_writers.size() + 1 >
+                    config::table_sink_partition_write_max_partition_nums_per_writer) {
+                    return Status::InternalError(
+                            "Too many open partitions {}",
+                            config::table_sink_partition_write_max_partition_nums_per_writer);
+                }
+                RETURN_IF_ERROR(create_and_open_writer(partition_name, i, nullptr, 0, writer));
             } else {
-                writer_pos_iter->second[i] = 1;
+                std::shared_ptr<VHivePartitionWriter> writer;
+                if (writer_iter->second->written_len() > config::hive_sink_max_file_size) {
+                    std::string file_name(writer_iter->second->file_name());
+                    int file_name_index = writer_iter->second->file_name_index();
+                    {
+                        SCOPED_RAW_TIMER(&_close_ns);
+                        static_cast<void>(writer_iter->second->close(Status::OK()));
+                    }
+                    writer_positions.erase(writer_iter->second);
+                    _partitions_to_writers.erase(writer_iter);
+                    RETURN_IF_ERROR(create_and_open_writer(partition_name, i, &file_name,
+                                                           file_name_index + 1, writer));
+                } else {
+                    writer = writer_iter->second;
+                }
+                auto writer_pos_iter = writer_positions.find(writer);
+                if (writer_pos_iter == writer_positions.end()) {
+                    IColumn::Filter filter(block.rows(), 0);
+                    filter[i] = 1;
+                    writer_positions.insert({writer, std::move(filter)});
+                } else {
+                    writer_pos_iter->second[i] = 1;
+                }
             }
         }
     }
-
+    SCOPED_RAW_TIMER(&_partition_writers_write_ns);
     for (auto it = writer_positions.begin(); it != writer_positions.end(); ++it) {
         RETURN_IF_ERROR(it->first->write(block, &it->second));
     }
@@ -163,19 +202,34 @@ Status VHiveTableWriter::write(vectorized::Block& block) {
 }
 
 Status VHiveTableWriter::close(Status status) {
-    for (const auto& pair : _partitions_to_writers) {
-        Status st = pair.second->close(status);
-        if (st != Status::OK()) {
-            LOG(WARNING) << fmt::format("Unsupported type for partition {}", st.to_string());
-            continue;
+    int64_t partitions_to_writers_size = _partitions_to_writers.size();
+    {
+        SCOPED_RAW_TIMER(&_close_ns);
+        for (const auto& pair : _partitions_to_writers) {
+            Status st = pair.second->close(status);
+            if (st != Status::OK()) {
+                LOG(WARNING) << fmt::format("Unsupported type for partition {}", st.to_string());
+                continue;
+            }
         }
+        _partitions_to_writers.clear();
+    }
+    if (status.ok()) {
+        SCOPED_TIMER(_profile->total_time_counter());
+
+        COUNTER_SET(_written_rows_counter, static_cast<int64_t>(_row_count));
+        COUNTER_SET(_send_data_timer, _send_data_ns);
+        COUNTER_SET(_partition_writers_dispatch_timer, _partition_writers_dispatch_ns);
+        COUNTER_SET(_partition_writers_write_timer, _partition_writers_write_ns);
+        COUNTER_SET(_partition_writers_count, partitions_to_writers_size);
+        COUNTER_SET(_close_timer, _close_ns);
+        COUNTER_SET(_write_file_counter, _write_file_count);
     }
-    _partitions_to_writers.clear();
     return Status::OK();
 }
 
 std::shared_ptr<VHivePartitionWriter> VHiveTableWriter::_create_partition_writer(
-        vectorized::Block& block, int position) {
+        vectorized::Block& block, int position, const std::string* file_name, int file_name_index) {
     auto& hive_table_sink = _t_sink.hive_table_sink;
     std::vector<std::string> partition_values;
     std::string partition_name;
@@ -247,13 +301,12 @@ std::shared_ptr<VHivePartitionWriter> VHiveTableWriter::_create_partition_writer
         }
     }
 
+    _write_file_count++;
     return std::make_shared<VHivePartitionWriter>(
             _t_sink, std::move(partition_name), update_mode, _vec_output_expr_ctxs,
             _write_output_vexpr_ctxs, _non_write_columns_indices, hive_table_sink.columns,
-            std::move(write_info),
-            fmt::format("{}{}", _compute_file_name(),
-                        _get_file_extension(file_format_type, write_compress_type)),
-            file_format_type, write_compress_type, hive_table_sink.hadoop_config);
+            std::move(write_info), (file_name == nullptr) ? _compute_file_name() : *file_name,
+            file_name_index, file_format_type, write_compress_type, hive_table_sink.hadoop_config);
 }
 
 std::vector<std::string> VHiveTableWriter::_create_partition_values(vectorized::Block& block,
@@ -397,46 +450,6 @@ std::string VHiveTableWriter::_to_partition_value(const TypeDescriptor& type_des
     }
 }
 
-std::string VHiveTableWriter::_get_file_extension(TFileFormatType::type file_format_type,
-                                                  TFileCompressType::type write_compress_type) {
-    std::string compress_name;
-    switch (write_compress_type) {
-    case TFileCompressType::SNAPPYBLOCK: {
-        compress_name = ".snappy";
-        break;
-    }
-    case TFileCompressType::ZLIB: {
-        compress_name = ".zlib";
-        break;
-    }
-    case TFileCompressType::ZSTD: {
-        compress_name = ".zstd";
-        break;
-    }
-    default: {
-        compress_name = "";
-        break;
-    }
-    }
-
-    std::string file_format_name;
-    switch (file_format_type) {
-    case TFileFormatType::FORMAT_PARQUET: {
-        file_format_name = ".parquet";
-        break;
-    }
-    case TFileFormatType::FORMAT_ORC: {
-        file_format_name = ".orc";
-        break;
-    }
-    default: {
-        file_format_name = "";
-        break;
-    }
-    }
-    return fmt::format("{}{}", compress_name, file_format_name);
-}
-
 std::string VHiveTableWriter::_compute_file_name() {
     boost::uuids::uuid uuid = boost::uuids::random_generator()();
 
diff --git a/be/src/vec/sink/writer/vhive_table_writer.h b/be/src/vec/sink/writer/vhive_table_writer.h
index 9f48f6afde1..3a3f45a6db1 100644
--- a/be/src/vec/sink/writer/vhive_table_writer.h
+++ b/be/src/vec/sink/writer/vhive_table_writer.h
@@ -19,6 +19,7 @@
 
 #include <gen_cpp/DataSinks_types.h>
 
+#include "util/runtime_profile.h"
 #include "vec/exprs/vexpr_fwd.h"
 #include "vec/sink/writer/async_result_writer.h"
 
@@ -50,17 +51,15 @@ public:
     Status close(Status) override;
 
 private:
-    std::shared_ptr<VHivePartitionWriter> _create_partition_writer(vectorized::Block& block,
-                                                                   int position);
+    std::shared_ptr<VHivePartitionWriter> _create_partition_writer(
+            vectorized::Block& block, int position, const std::string* file_name = nullptr,
+            int file_name_index = 0);
 
     std::vector<std::string> _create_partition_values(vectorized::Block& block, int position);
 
     std::string _to_partition_value(const TypeDescriptor& type_desc,
                                     const ColumnWithTypeAndName& partition_column, int position);
 
-    std::string _get_file_extension(TFileFormatType::type file_format_type,
-                                    TFileCompressType::type write_compress_type);
-
     std::string _compute_file_name();
 
     // Currently it is a copy, maybe it is better to use move semantics to eliminate it.
@@ -72,6 +71,24 @@ private:
     std::unordered_map<std::string, std::shared_ptr<VHivePartitionWriter>> _partitions_to_writers;
 
     VExprContextSPtrs _write_output_vexpr_ctxs;
+
+    size_t _row_count = 0;
+
+    // profile counters
+    int64_t _send_data_ns = 0;
+    int64_t _partition_writers_dispatch_ns = 0;
+    int64_t _partition_writers_write_ns = 0;
+    int64_t _close_ns = 0;
+    int64_t _write_file_count = 0;
+
+    RuntimeProfile::Counter* _written_rows_counter = nullptr;
+    RuntimeProfile::Counter* _send_data_timer = nullptr;
+    RuntimeProfile::Counter* _partition_writers_dispatch_timer = nullptr;
+    RuntimeProfile::Counter* _partition_writers_write_timer = nullptr;
+    RuntimeProfile::Counter* _partition_writers_count = nullptr;
+    RuntimeProfile::Counter* _open_timer = nullptr;
+    RuntimeProfile::Counter* _close_timer = nullptr;
+    RuntimeProfile::Counter* _write_file_counter = nullptr;
 };
 } // namespace vectorized
 } // namespace doris


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


(doris) 04/19: [fix](insert)fix conversion of doris type to hive type (#32735)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 3343322965d4f379d74f6864a104b708cada3b21
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Tue Mar 26 11:14:30 2024 +0800

    [fix](insert)fix conversion of doris type to hive type (#32735)
    
    #31442
    
    create table
    fix doris to hive type, use primitiveType to check doris type.
---
 .../datasource/hive/HiveMetaStoreClientHelper.java | 117 ++++++++++++------
 .../org/apache/doris/datasource/hive/HiveUtil.java |  95 +++++++++++++++
 .../datasource/hive/ThriftHMSCachedClient.java     |  94 +--------------
 .../plans/physical/PhysicalHiveTableSink.java      |   1 +
 .../datasource/hive/HiveDDLAndDMLPlanTest.java     | 131 +++++++++++++++++++++
 5 files changed, 310 insertions(+), 128 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java
index 4c002275bde..3ebce966777 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java
@@ -586,42 +586,89 @@ public class HiveMetaStoreClientHelper {
      * Convert doris type to hive type.
      */
     public static String dorisTypeToHiveType(Type dorisType) {
-        if (dorisType.equals(Type.BOOLEAN)) {
-            return "boolean";
-        } else if (dorisType.equals(Type.TINYINT)) {
-            return "tinyint";
-        } else if (dorisType.equals(Type.SMALLINT)) {
-            return "smallint";
-        } else if (dorisType.equals(Type.INT)) {
-            return "int";
-        } else if (dorisType.equals(Type.BIGINT)) {
-            return "bigint";
-        } else if (dorisType.equals(Type.DATE) || dorisType.equals(Type.DATEV2)) {
-            return "date";
-        } else if (dorisType.equals(Type.DATETIME) || dorisType.equals(Type.DATETIMEV2)) {
-            return "timestamp";
-        } else if (dorisType.equals(Type.FLOAT)) {
-            return "float";
-        } else if (dorisType.equals(Type.DOUBLE)) {
-            return "double";
-        } else if (dorisType.equals(Type.STRING)) {
-            return "string";
-        } else if (dorisType.equals(Type.DEFAULT_DECIMALV3)) {
-            StringBuilder decimalType = new StringBuilder();
-            decimalType.append("decimal");
-            ScalarType scalarType = (ScalarType) dorisType;
-            int precision = scalarType.getScalarPrecision();
-            if (precision == 0) {
-                precision = ScalarType.DEFAULT_PRECISION;
+        if (dorisType.isScalarType()) {
+            PrimitiveType primitiveType = dorisType.getPrimitiveType();
+            switch (primitiveType) {
+                case BOOLEAN:
+                    return "boolean";
+                case TINYINT:
+                    return "tinyint";
+                case SMALLINT:
+                    return "smallint";
+                case INT:
+                    return "int";
+                case BIGINT:
+                    return "bigint";
+                case DATEV2:
+                case DATE:
+                    return "date";
+                case DATETIMEV2:
+                case DATETIME:
+                    return "timestamp";
+                case FLOAT:
+                    return "float";
+                case DOUBLE:
+                    return "double";
+                case CHAR: {
+                    ScalarType scalarType = (ScalarType) dorisType;
+                    return "char(" + scalarType.getLength() + ")";
+                }
+                case VARCHAR:
+                case STRING:
+                    return "string";
+                case DECIMAL32:
+                case DECIMAL64:
+                case DECIMAL128:
+                case DECIMAL256:
+                case DECIMALV2: {
+                    StringBuilder decimalType = new StringBuilder();
+                    decimalType.append("decimal");
+                    ScalarType scalarType = (ScalarType) dorisType;
+                    int precision = scalarType.getScalarPrecision();
+                    if (precision == 0) {
+                        precision = ScalarType.DEFAULT_PRECISION;
+                    }
+                    // decimal(precision, scale)
+                    int scale = scalarType.getScalarScale();
+                    decimalType.append("(");
+                    decimalType.append(precision);
+                    decimalType.append(",");
+                    decimalType.append(scale);
+                    decimalType.append(")");
+                    return decimalType.toString();
+                }
+                default:
+                    throw new HMSClientException("Unsupported primitive type conversion of " + dorisType.toSql());
+            }
+        } else if (dorisType.isArrayType()) {
+            ArrayType dorisArray = (ArrayType) dorisType;
+            Type itemType = dorisArray.getItemType();
+            return "array<" + dorisTypeToHiveType(itemType) + ">";
+        } else if (dorisType.isMapType()) {
+            MapType dorisMap = (MapType) dorisType;
+            Type keyType = dorisMap.getKeyType();
+            Type valueType = dorisMap.getValueType();
+            return "map<"
+                    + dorisTypeToHiveType(keyType)
+                    + ","
+                    + dorisTypeToHiveType(valueType)
+                    + ">";
+        } else if (dorisType.isStructType()) {
+            StructType dorisStruct = (StructType) dorisType;
+            StringBuilder structType = new StringBuilder();
+            structType.append("struct<");
+            ArrayList<StructField> fields = dorisStruct.getFields();
+            for (int i = 0; i < fields.size(); i++) {
+                StructField field = fields.get(i);
+                structType.append(field.getName());
+                structType.append(":");
+                structType.append(dorisTypeToHiveType(field.getType()));
+                if (i != fields.size() - 1) {
+                    structType.append(",");
+                }
             }
-            // decimal(precision, scale)
-            int scale = scalarType.getScalarScale();
-            decimalType.append("(");
-            decimalType.append(precision);
-            decimalType.append(",");
-            decimalType.append(scale);
-            decimalType.append(")");
-            return decimalType.toString();
+            structType.append(">");
+            return structType.toString();
         }
         throw new HMSClientException("Unsupported type conversion of " + dorisType.toSql());
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
index eb107464bfc..cc834894076 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
@@ -17,6 +17,8 @@
 
 package org.apache.doris.datasource.hive;
 
+import org.apache.doris.catalog.Column;
+import org.apache.doris.common.Pair;
 import org.apache.doris.common.UserException;
 import org.apache.doris.fs.remote.BrokerFileSystem;
 import org.apache.doris.fs.remote.RemoteFileSystem;
@@ -24,8 +26,13 @@ import org.apache.doris.fs.remote.RemoteFileSystem;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat;
 import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
 import org.apache.hadoop.mapred.InputFormat;
@@ -36,8 +43,13 @@ import org.apache.hadoop.util.ReflectionUtils;
 import java.io.UnsupportedEncodingException;
 import java.net.URLDecoder;
 import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
 import java.util.stream.Collectors;
 
 /**
@@ -157,4 +169,87 @@ public final class HiveUtil {
         }
         return resultBuilder.build();
     }
+
+    public static Table toHiveTable(HiveTableMetadata hiveTable) {
+        Objects.requireNonNull(hiveTable.getDbName(), "Hive database name should be not null");
+        Objects.requireNonNull(hiveTable.getTableName(), "Hive table name should be not null");
+        Table table = new Table();
+        table.setDbName(hiveTable.getDbName());
+        table.setTableName(hiveTable.getTableName());
+        // table.setOwner("");
+        int createTime = (int) System.currentTimeMillis() * 1000;
+        table.setCreateTime(createTime);
+        table.setLastAccessTime(createTime);
+        // table.setRetention(0);
+        String location = hiveTable.getProperties().get(HiveMetadataOps.LOCATION_URI_KEY);
+        Set<String> partitionSet = new HashSet<>(hiveTable.getPartitionKeys());
+        Pair<List<FieldSchema>, List<FieldSchema>> hiveSchema = toHiveSchema(hiveTable.getColumns(), partitionSet);
+
+        table.setSd(toHiveStorageDesc(hiveSchema.first, hiveTable.getBucketCols(), hiveTable.getNumBuckets(),
+                hiveTable.getFileFormat(), location));
+        table.setPartitionKeys(hiveSchema.second);
+
+        // table.setViewOriginalText(hiveTable.getViewSql());
+        // table.setViewExpandedText(hiveTable.getViewSql());
+        table.setTableType("MANAGED_TABLE");
+        table.setParameters(hiveTable.getProperties());
+        return table;
+    }
+
+    private static StorageDescriptor toHiveStorageDesc(List<FieldSchema> columns,
+            List<String> bucketCols, int numBuckets, String fileFormat, String location) {
+        StorageDescriptor sd = new StorageDescriptor();
+        sd.setCols(columns);
+        setFileFormat(fileFormat, sd);
+        if (StringUtils.isNotEmpty(location)) {
+            sd.setLocation(location);
+        }
+        sd.setBucketCols(bucketCols);
+        sd.setNumBuckets(numBuckets);
+        Map<String, String> parameters = new HashMap<>();
+        parameters.put("tag", "doris external hive talbe");
+        sd.setParameters(parameters);
+        return sd;
+    }
+
+    private static void setFileFormat(String fileFormat, StorageDescriptor sd) {
+        String inputFormat;
+        String outputFormat;
+        String serDe;
+        if (fileFormat.equalsIgnoreCase("orc")) {
+            inputFormat = "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat";
+            outputFormat = "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat";
+            serDe = "org.apache.hadoop.hive.ql.io.orc.OrcSerde";
+        } else if (fileFormat.equalsIgnoreCase("parquet")) {
+            inputFormat = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat";
+            outputFormat = "'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat";
+            serDe = "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe";
+        } else {
+            throw new IllegalArgumentException("Creating table with an unsupported file format: " + fileFormat);
+        }
+        SerDeInfo serDeInfo = new SerDeInfo();
+        serDeInfo.setSerializationLib(serDe);
+        sd.setSerdeInfo(serDeInfo);
+        sd.setInputFormat(inputFormat);
+        sd.setOutputFormat(outputFormat);
+    }
+
+    private static Pair<List<FieldSchema>, List<FieldSchema>> toHiveSchema(List<Column> columns,
+            Set<String> partitionSet) {
+        List<FieldSchema> hiveCols = new ArrayList<>();
+        List<FieldSchema> hiveParts = new ArrayList<>();
+        for (Column column : columns) {
+            FieldSchema hiveFieldSchema = new FieldSchema();
+            // TODO: add doc, just support doris type
+            hiveFieldSchema.setType(HiveMetaStoreClientHelper.dorisTypeToHiveType(column.getType()));
+            hiveFieldSchema.setName(column.getName());
+            hiveFieldSchema.setComment(column.getComment());
+            if (partitionSet.contains(column.getName())) {
+                hiveParts.add(hiveFieldSchema);
+            } else {
+                hiveCols.add(hiveFieldSchema);
+            }
+        }
+        return Pair.of(hiveCols, hiveParts);
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
index b5b1147447e..1f3d188ac6b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
@@ -18,9 +18,7 @@
 package org.apache.doris.datasource.hive;
 
 import org.apache.doris.analysis.TableName;
-import org.apache.doris.catalog.Column;
 import org.apache.doris.common.Config;
-import org.apache.doris.common.Pair;
 import org.apache.doris.datasource.DatabaseMetadata;
 import org.apache.doris.datasource.TableMetadata;
 import org.apache.doris.datasource.hive.event.MetastoreNotificationFetchException;
@@ -68,18 +66,14 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Optional;
 import java.util.Queue;
-import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
@@ -178,7 +172,7 @@ public class ThriftHMSCachedClient implements HMSCachedClient {
                 // String location,
                 if (tbl instanceof HiveTableMetadata) {
                     ugiDoAs(() -> {
-                        client.client.createTable(toHiveTable((HiveTableMetadata) tbl));
+                        client.client.createTable(HiveUtil.toHiveTable((HiveTableMetadata) tbl));
                         return null;
                     });
                 }
@@ -191,92 +185,6 @@ public class ThriftHMSCachedClient implements HMSCachedClient {
         }
     }
 
-    private static Table toHiveTable(HiveTableMetadata hiveTable) {
-        Objects.requireNonNull(hiveTable.getDbName(), "Hive database name should be not null");
-        Objects.requireNonNull(hiveTable.getTableName(), "Hive table name should be not null");
-        Table table = new Table();
-        table.setDbName(hiveTable.getDbName());
-        table.setTableName(hiveTable.getTableName());
-        // table.setOwner("");
-        int createTime = (int) System.currentTimeMillis() * 1000;
-        table.setCreateTime(createTime);
-        table.setLastAccessTime(createTime);
-        // table.setRetention(0);
-        String location = hiveTable.getProperties().get(HiveMetadataOps.LOCATION_URI_KEY);
-        Set<String> partitionSet = new HashSet<>(hiveTable.getPartitionKeys());
-        Pair<List<FieldSchema>, List<FieldSchema>> hiveSchema = toHiveSchema(hiveTable.getColumns(), partitionSet);
-
-        table.setSd(toHiveStorageDesc(hiveSchema.first, hiveTable.getBucketCols(), hiveTable.getNumBuckets(),
-                hiveTable.getFileFormat(), location));
-        table.setPartitionKeys(hiveSchema.second);
-
-        // table.setViewOriginalText(hiveTable.getViewSql());
-        // table.setViewExpandedText(hiveTable.getViewSql());
-        table.setTableType("MANAGED_TABLE");
-        table.setParameters(hiveTable.getProperties());
-        return table;
-    }
-
-    private static StorageDescriptor toHiveStorageDesc(List<FieldSchema> columns,
-                                                       List<String> bucketCols,
-                                                       int numBuckets,
-                                                       String fileFormat,
-                                                       String location) {
-        StorageDescriptor sd = new StorageDescriptor();
-        sd.setCols(columns);
-        setFileFormat(fileFormat, sd);
-        if (StringUtils.isNotEmpty(location)) {
-            sd.setLocation(location);
-        }
-        sd.setBucketCols(bucketCols);
-        sd.setNumBuckets(numBuckets);
-        Map<String, String> parameters = new HashMap<>();
-        parameters.put("tag", "doris external hive talbe");
-        sd.setParameters(parameters);
-        return sd;
-    }
-
-    private static void setFileFormat(String fileFormat, StorageDescriptor sd) {
-        String inputFormat;
-        String outputFormat;
-        String serDe;
-        if (fileFormat.equalsIgnoreCase("orc")) {
-            inputFormat = "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat";
-            outputFormat = "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat";
-            serDe = "org.apache.hadoop.hive.ql.io.orc.OrcSerde";
-        } else if (fileFormat.equalsIgnoreCase("parquet")) {
-            inputFormat = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat";
-            outputFormat = "'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat";
-            serDe = "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe";
-        } else {
-            throw new IllegalArgumentException("Creating table with an unsupported file format: " + fileFormat);
-        }
-        SerDeInfo serDeInfo = new SerDeInfo();
-        serDeInfo.setSerializationLib(serDe);
-        sd.setSerdeInfo(serDeInfo);
-        sd.setInputFormat(inputFormat);
-        sd.setOutputFormat(outputFormat);
-    }
-
-    private static Pair<List<FieldSchema>, List<FieldSchema>> toHiveSchema(List<Column> columns,
-                Set<String> partitionSet) {
-        List<FieldSchema> hiveCols = new ArrayList<>();
-        List<FieldSchema> hiveParts = new ArrayList<>();
-        for (Column column : columns) {
-            FieldSchema hiveFieldSchema = new FieldSchema();
-            // TODO: add doc, just support doris type
-            hiveFieldSchema.setType(HiveMetaStoreClientHelper.dorisTypeToHiveType(column.getType()));
-            hiveFieldSchema.setName(column.getName());
-            hiveFieldSchema.setComment(column.getComment());
-            if (partitionSet.contains(column.getName())) {
-                hiveParts.add(hiveFieldSchema);
-            } else {
-                hiveCols.add(hiveFieldSchema);
-            }
-        }
-        return Pair.of(hiveCols, hiveParts);
-    }
-
     @Override
     public void dropDatabase(String dbName) {
         try (ThriftHMSClient client = getClient()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java
index 8a37bb71cc6..a670290a163 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java
@@ -153,6 +153,7 @@ public class PhysicalHiveTableSink<CHILD_TYPE extends Plan> extends PhysicalTabl
                     columnIdx.add(i);
                 }
             }
+            // mapping partition id
             List<ExprId> exprIds = columnIdx.stream()
                     .map(idx -> child().getOutput().get(idx).getExprId())
                     .collect(Collectors.toList());
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
index b7e90218e10..8e2d436e219 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
@@ -22,10 +22,12 @@ import org.apache.doris.analysis.CreateDbStmt;
 import org.apache.doris.analysis.CreateTableStmt;
 import org.apache.doris.analysis.HashDistributionDesc;
 import org.apache.doris.analysis.SwitchStmt;
+import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.ExceptionChecker;
 import org.apache.doris.common.FeConstants;
+import org.apache.doris.datasource.TableMetadata;
 import org.apache.doris.nereids.parser.NereidsParser;
 import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand;
 import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand;
@@ -36,6 +38,8 @@ import org.apache.doris.utframe.TestWithFeService;
 import mockit.Mock;
 import mockit.MockUp;
 import mockit.Mocked;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Table;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
@@ -52,6 +56,8 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
     @Mocked
     private ThriftHMSCachedClient mockedHiveClient;
 
+    private List<FieldSchema> checkedHiveCols;
+
     @Override
     protected void runBeforeAll() throws Exception {
         connectContext.getSessionVariable().enableFallbackToOriginalPlanner = false;
@@ -101,6 +107,25 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                     }
                 };
             }
+
+            @Mock
+            public void createTable(TableMetadata tbl, boolean ignoreIfExists) {
+                if (tbl instanceof HiveTableMetadata) {
+                    Table table = HiveUtil.toHiveTable((HiveTableMetadata) tbl);
+                    if (checkedHiveCols == null) {
+                        // if checkedHiveCols is null, skip column check
+                        return;
+                    }
+                    List<FieldSchema> fieldSchemas = table.getSd().getCols();
+                    Assertions.assertEquals(checkedHiveCols.size(), fieldSchemas.size());
+                    for (int i = 0; i < checkedHiveCols.size(); i++) {
+                        FieldSchema checkedCol = checkedHiveCols.get(i);
+                        FieldSchema actualCol = fieldSchemas.get(i);
+                        Assertions.assertEquals(checkedCol.getName(), actualCol.getName().toLowerCase());
+                        Assertions.assertEquals(checkedCol.getType(), actualCol.getType().toLowerCase());
+                    }
+                }
+            }
         };
         CreateDbStmt createDbStmt = new CreateDbStmt(true, mockedDbName, dbProps);
         Env.getCurrentEnv().createDb(createDbStmt);
@@ -369,4 +394,110 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
         LogicalPlan plan2 = nereidsParser.parseSingle(insertSql2);
         Assertions.assertTrue(plan2 instanceof InsertOverwriteTableCommand);
     }
+
+    @Test
+    public void testComplexTypeCreateTable() throws Exception {
+        checkedHiveCols = new ArrayList<>(); // init it to enable check
+        switchHive();
+        useDatabase(mockedDbName);
+        String createArrayTypeTable = "CREATE TABLE complex_type_array(\n"
+                + "  `col1` ARRAY<BOOLEAN> COMMENT 'col1',\n"
+                + "  `col2` ARRAY<INT(11)> COMMENT 'col2',\n"
+                + "  `col3` ARRAY<DECIMAL(6,4)> COMMENT 'col3',\n"
+                + "  `col4` ARRAY<CHAR(11)> COMMENT 'col4',\n"
+                + "  `col5` ARRAY<CHAR> COMMENT 'col5'\n"
+                + ")  ENGINE=hive\n"
+                + "PROPERTIES ('file_format'='orc')";
+        List<FieldSchema> checkArrayCols = new ArrayList<>();
+        checkArrayCols.add(new FieldSchema("col1", "array<boolean>", ""));
+        checkArrayCols.add(new FieldSchema("col2", "array<int>", ""));
+        checkArrayCols.add(new FieldSchema("col3", "array<decimal(6,4)>", ""));
+        checkArrayCols.add(new FieldSchema("col4", "array<char(11)>", ""));
+        checkArrayCols.add(new FieldSchema("col5", "array<char(1)>", ""));
+        resetCheckedColumns(checkArrayCols);
+
+        LogicalPlan plan = createTablesAndReturnPlans(true, createArrayTypeTable).get(0);
+        List<Column> columns = ((CreateTableCommand) plan).getCreateTableInfo().translateToLegacyStmt().getColumns();
+        Assertions.assertEquals(5, columns.size());
+        dropTable("complex_type_array", true);
+
+        String createMapTypeTable = "CREATE TABLE complex_type_map(\n"
+                + "  `col1` MAP<int,string> COMMENT 'col1',\n"
+                + "  `col2` MAP<string,double> COMMENT 'col2',\n"
+                + "  `col3` MAP<string,BOOLEAN> COMMENT 'col3',\n"
+                + "  `col4` MAP<BOOLEAN,BOOLEAN> COMMENT 'col4'\n"
+                + ")  ENGINE=hive\n"
+                + "PROPERTIES ('file_format'='orc')";
+        checkArrayCols = new ArrayList<>();
+        checkArrayCols.add(new FieldSchema("col1", "map<int,string>", ""));
+        checkArrayCols.add(new FieldSchema("col2", "map<string,double>", ""));
+        checkArrayCols.add(new FieldSchema("col3", "map<string,boolean>", ""));
+        checkArrayCols.add(new FieldSchema("col4", "map<boolean,boolean>", ""));
+        resetCheckedColumns(checkArrayCols);
+
+        plan = createTablesAndReturnPlans(true, createMapTypeTable).get(0);
+        columns = ((CreateTableCommand) plan).getCreateTableInfo().translateToLegacyStmt().getColumns();
+        Assertions.assertEquals(4, columns.size());
+        dropTable("complex_type_map", true);
+
+        String createStructTypeTable = "CREATE TABLE complex_type_struct(\n"
+                + "  `col1` STRUCT<rates:ARRAY<double>,name:string> COMMENT 'col1',\n"
+                + "  `col2` STRUCT<id:INT,age:TINYINT> COMMENT 'col2',\n"
+                + "  `col3` STRUCT<pre:DECIMAL(6,4)> COMMENT 'col3',\n"
+                + "  `col4` STRUCT<bul:BOOLEAN,buls:ARRAY<BOOLEAN>> COMMENT 'col4'\n"
+                + ")  ENGINE=hive\n"
+                + "PROPERTIES ('file_format'='orc')";
+        checkArrayCols = new ArrayList<>();
+        checkArrayCols.add(new FieldSchema("col1", "struct<rates:array<double>,name:string>", ""));
+        checkArrayCols.add(new FieldSchema("col2", "struct<id:int,age:tinyint>", ""));
+        checkArrayCols.add(new FieldSchema("col3", "struct<pre:decimal(6,4)>", ""));
+        checkArrayCols.add(new FieldSchema("col4", "struct<bul:boolean,buls:array<boolean>>", ""));
+        resetCheckedColumns(checkArrayCols);
+
+        plan = createTablesAndReturnPlans(true, createStructTypeTable).get(0);
+        columns = ((CreateTableCommand) plan).getCreateTableInfo().translateToLegacyStmt().getColumns();
+        Assertions.assertEquals(4, columns.size());
+        dropTable("complex_type_struct", true);
+
+        String compoundTypeTable1 = "CREATE TABLE complex_type_compound1(\n"
+                + "  `col1` ARRAY<MAP<string,double>> COMMENT 'col1',\n"
+                + "  `col2` ARRAY<STRUCT<name:string,gender:boolean,rate:decimal(3,1)>> COMMENT 'col2'\n"
+                + ")  ENGINE=hive\n"
+                + "PROPERTIES ('file_format'='orc')";
+        checkArrayCols = new ArrayList<>();
+        checkArrayCols.add(new FieldSchema("col1", "array<map<string,double>>", ""));
+        checkArrayCols.add(new FieldSchema("col2",
+                "array<struct<name:string,gender:boolean,rate:decimal(3,1)>>", ""));
+        resetCheckedColumns(checkArrayCols);
+
+        plan = createTablesAndReturnPlans(true, compoundTypeTable1).get(0);
+        columns = ((CreateTableCommand) plan).getCreateTableInfo().translateToLegacyStmt().getColumns();
+        Assertions.assertEquals(2, columns.size());
+        dropTable("complex_type_compound1", true);
+
+        String compoundTypeTable2 = "CREATE TABLE complex_type_compound2(\n"
+                + "  `col1` MAP<string,ARRAY<double>> COMMENT 'col1',\n"
+                + "  `col2` MAP<string,ARRAY<MAP<int, string>>> COMMENT 'col2',\n"
+                + "  `col3` MAP<string,MAP<int,double>> COMMENT 'col3',\n"
+                + "  `col4` MAP<bigint,STRUCT<name:string,gender:boolean,rate:decimal(3,1)>> COMMENT 'col4'\n"
+                + ")  ENGINE=hive\n"
+                + "PROPERTIES ('file_format'='orc')";
+        checkArrayCols = new ArrayList<>();
+        checkArrayCols.add(new FieldSchema("col1", "map<string,array<double>>", ""));
+        checkArrayCols.add(new FieldSchema("col2", "map<string,array<map<int,string>>>", ""));
+        checkArrayCols.add(new FieldSchema("col3", "map<string,map<int,double>>", ""));
+        checkArrayCols.add(new FieldSchema("col4",
+                "map<bigint,struct<name:string,gender:boolean,rate:decimal(3,1)>>", ""));
+        resetCheckedColumns(checkArrayCols);
+
+        plan = createTablesAndReturnPlans(true, compoundTypeTable2).get(0);
+        columns = ((CreateTableCommand) plan).getCreateTableInfo().translateToLegacyStmt().getColumns();
+        Assertions.assertEquals(4, columns.size());
+        dropTable("complex_type_compound2", true);
+    }
+
+    private void resetCheckedColumns(List<FieldSchema> checkArrayCols) {
+        checkedHiveCols.clear();
+        checkedHiveCols.addAll(checkArrayCols);
+    }
 }


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


(doris) 14/19: [feature](external) process tbl/db exist when create/drop db/tbl (#33119)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit f0ac21e231e23cc87e64b7e2ce9a732e612da531
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Wed Apr 3 10:01:57 2024 +0800

    [feature](external) process tbl/db exist when create/drop db/tbl (#33119)
    
    Issue Number: #31442
---
 .../doris/datasource/hive/HiveMetadataOps.java     |  51 +++++++---
 .../org/apache/doris/datasource/hive/HiveUtil.java |  12 +++
 .../datasource/hive/ThriftHMSCachedClient.java     |  14 +--
 .../datasource/iceberg/IcebergMetadataOps.java     |  38 ++++++++
 .../datasource/operations/ExternalMetadataOps.java |   2 +
 .../datasource/hive/HiveDDLAndDMLPlanTest.java     | 103 ++++++++++++++++++---
 .../doris/datasource/hive/HiveMetadataOpsTest.java |   7 ++
 .../apache/doris/utframe/TestWithFeService.java    |  10 ++
 8 files changed, 198 insertions(+), 39 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
index a182aa9cc00..beeff694ae4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
@@ -27,6 +27,8 @@ import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.JdbcResource;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.UserException;
 import org.apache.doris.datasource.ExternalDatabase;
 import org.apache.doris.datasource.jdbc.client.JdbcClient;
@@ -40,7 +42,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -99,6 +100,14 @@ public class HiveMetadataOps implements ExternalMetadataOps {
         String fullDbName = stmt.getFullDbName();
         Map<String, String> properties = stmt.getProperties();
         long dbId = Env.getCurrentEnv().getNextId();
+        if (databaseExist(fullDbName)) {
+            if (stmt.isSetIfNotExists()) {
+                LOG.info("create database[{}] which already exists", fullDbName);
+                return;
+            } else {
+                ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, fullDbName);
+            }
+        }
         try {
             HiveDatabaseMetadata catalogDatabase = new HiveDatabaseMetadata();
             catalogDatabase.setDbName(fullDbName);
@@ -119,6 +128,14 @@ public class HiveMetadataOps implements ExternalMetadataOps {
     @Override
     public void dropDb(DropDbStmt stmt) throws DdlException {
         String dbName = stmt.getDbName();
+        if (!databaseExist(dbName)) {
+            if (stmt.isSetIfExists()) {
+                LOG.info("drop database[{}] which does not exist", dbName);
+                return;
+            } else {
+                ErrorReport.reportDdlException(ErrorCode.ERR_DB_DROP_EXISTS, dbName);
+            }
+        }
         try {
             client.dropDatabase(dbName);
             catalog.onRefresh(true);
@@ -135,6 +152,14 @@ public class HiveMetadataOps implements ExternalMetadataOps {
         if (db == null) {
             throw new UserException("Failed to get database: '" + dbName + "' in catalog: " + catalog.getName());
         }
+        if (tableExist(dbName, tblName)) {
+            if (stmt.isSetIfNotExists()) {
+                LOG.info("create table[{}] which already exists", tblName);
+                return;
+            } else {
+                ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tblName);
+            }
+        }
         try {
             Map<String, String> props = stmt.getProperties();
             String fileFormat = props.getOrDefault(FILE_FORMAT_KEY, Config.hive_default_file_format);
@@ -186,17 +211,6 @@ public class HiveMetadataOps implements ExternalMetadataOps {
         }
     }
 
-    private static List<FieldSchema> parsePartitionKeys(Map<String, String> props) {
-        List<FieldSchema> parsedKeys = new ArrayList<>();
-        String pkStr = props.getOrDefault("partition_keys", "");
-        if (pkStr.isEmpty()) {
-            return parsedKeys;
-        } else {
-            // TODO: parse string to partition keys list
-            return parsedKeys;
-        }
-    }
-
     @Override
     public void dropTable(DropTableStmt stmt) throws DdlException {
         String dbName = stmt.getDbName();
@@ -204,6 +218,14 @@ public class HiveMetadataOps implements ExternalMetadataOps {
         if (db == null) {
             throw new DdlException("Failed to get database: '" + dbName + "' in catalog: " + catalog.getName());
         }
+        if (!tableExist(dbName, stmt.getTableName())) {
+            if (stmt.isSetIfExists()) {
+                LOG.info("drop table[{}] which does not exist", dbName);
+                return;
+            } else {
+                ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, stmt.getTableName(), dbName);
+            }
+        }
         try {
             client.dropTable(dbName, stmt.getTableName());
             db.setUnInitialized(true);
@@ -222,6 +244,11 @@ public class HiveMetadataOps implements ExternalMetadataOps {
         return client.tableExists(dbName, tblName);
     }
 
+    @Override
+    public boolean databaseExist(String dbName) {
+        return listDatabaseNames().contains(dbName);
+    }
+
     public List<String> listDatabaseNames() {
         return client.getAllDatabases();
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
index e6b5c1095f3..0dc7eb5a386 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
@@ -29,6 +29,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
@@ -255,4 +256,15 @@ public final class HiveUtil {
         }
         return Pair.of(hiveCols, hiveParts);
     }
+
+    public static Database toHiveDatabase(HiveDatabaseMetadata hiveDb) {
+        Database database = new Database();
+        database.setName(hiveDb.getDbName());
+        if (StringUtils.isNotEmpty(hiveDb.getLocationUri())) {
+            database.setLocationUri(hiveDb.getLocationUri());
+        }
+        database.setParameters(hiveDb.getProperties());
+        database.setDescription(hiveDb.getComment());
+        return database;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
index 1f3d188ac6b..00fc0d03fa8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
@@ -31,7 +31,6 @@ import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
 import org.apache.hadoop.hive.common.ValidTxnList;
@@ -137,7 +136,7 @@ public class ThriftHMSCachedClient implements HMSCachedClient {
                 if (db instanceof HiveDatabaseMetadata) {
                     HiveDatabaseMetadata hiveDb = (HiveDatabaseMetadata) db;
                     ugiDoAs(() -> {
-                        client.client.createDatabase(toHiveDatabase(hiveDb));
+                        client.client.createDatabase(HiveUtil.toHiveDatabase(hiveDb));
                         return null;
                     });
                 }
@@ -150,17 +149,6 @@ public class ThriftHMSCachedClient implements HMSCachedClient {
         }
     }
 
-    private static Database toHiveDatabase(HiveDatabaseMetadata hiveDb) {
-        Database database = new Database();
-        database.setName(hiveDb.getDbName());
-        if (StringUtils.isNotEmpty(hiveDb.getLocationUri())) {
-            database.setLocationUri(hiveDb.getLocationUri());
-        }
-        database.setParameters(hiveDb.getProperties());
-        database.setDescription(hiveDb.getComment());
-        return database;
-    }
-
     @Override
     public void createTable(TableMetadata tbl, boolean ignoreIfExists) {
         if (tableExists(tbl.getDbName(), tbl.getTableName())) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java
index 0f43ef21c43..0c188fae301 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java
@@ -25,6 +25,8 @@ import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.StructField;
 import org.apache.doris.catalog.StructType;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.UserException;
 import org.apache.doris.datasource.DorisTypeVisitor;
 import org.apache.doris.datasource.ExternalCatalog;
@@ -67,6 +69,10 @@ public class IcebergMetadataOps implements ExternalMetadataOps {
         return catalog.tableExists(TableIdentifier.of(dbName, tblName));
     }
 
+    public boolean databaseExist(String dbName) {
+        return nsCatalog.namespaceExists(Namespace.of(dbName));
+    }
+
     public List<String> listDatabaseNames() {
         return nsCatalog.listNamespaces().stream()
                 .map(e -> e.toString())
@@ -84,6 +90,14 @@ public class IcebergMetadataOps implements ExternalMetadataOps {
         SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog;
         String dbName = stmt.getFullDbName();
         Map<String, String> properties = stmt.getProperties();
+        if (databaseExist(dbName)) {
+            if (stmt.isSetIfNotExists()) {
+                LOG.info("create database[{}] which already exists", dbName);
+                return;
+            } else {
+                ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, dbName);
+            }
+        }
         nsCatalog.createNamespace(Namespace.of(dbName), properties);
         dorisCatalog.onRefresh(true);
     }
@@ -97,6 +111,14 @@ public class IcebergMetadataOps implements ExternalMetadataOps {
             dorisCatalog.getIdToDb().remove(aLong);
             dorisCatalog.getDbNameToId().remove(dbName);
         }
+        if (!databaseExist(dbName)) {
+            if (stmt.isSetIfExists()) {
+                LOG.info("drop database[{}] which does not exist", dbName);
+                return;
+            } else {
+                ErrorReport.reportDdlException(ErrorCode.ERR_DB_DROP_EXISTS, dbName);
+            }
+        }
         nsCatalog.dropNamespace(Namespace.of(dbName));
         dorisCatalog.onRefresh(true);
     }
@@ -109,6 +131,14 @@ public class IcebergMetadataOps implements ExternalMetadataOps {
             throw new UserException("Failed to get database: '" + dbName + "' in catalog: " + dorisCatalog.getName());
         }
         String tableName = stmt.getTableName();
+        if (tableExist(dbName, tableName)) {
+            if (stmt.isSetIfNotExists()) {
+                LOG.info("create table[{}] which already exists", tableName);
+                return;
+            } else {
+                ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName);
+            }
+        }
         List<Column> columns = stmt.getColumns();
         List<StructField> collect = columns.stream()
                 .map(col -> new StructField(col.getName(), col.getType(), col.getComment(), col.isAllowNull()))
@@ -132,6 +162,14 @@ public class IcebergMetadataOps implements ExternalMetadataOps {
             throw new DdlException("Failed to get database: '" + dbName + "' in catalog: " + dorisCatalog.getName());
         }
         String tableName = stmt.getTableName();
+        if (!tableExist(dbName, tableName)) {
+            if (stmt.isSetIfExists()) {
+                LOG.info("drop table[{}] which does not exist", dbName);
+                return;
+            } else {
+                ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, tableName, dbName);
+            }
+        }
         catalog.dropTable(TableIdentifier.of(dbName, tableName));
         db.setUnInitialized(true);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java
index 2d9498d0b91..cb6ba35c149 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java
@@ -79,4 +79,6 @@ public interface ExternalMetadataOps {
      * @return
      */
     boolean tableExist(String dbName, String tblName);
+
+    boolean databaseExist(String dbName);
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
index a0f1ce498b2..a34e6cc7034 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
@@ -21,6 +21,7 @@ import org.apache.doris.analysis.CreateCatalogStmt;
 import org.apache.doris.analysis.CreateDbStmt;
 import org.apache.doris.analysis.CreateTableStmt;
 import org.apache.doris.analysis.DbName;
+import org.apache.doris.analysis.DropDbStmt;
 import org.apache.doris.analysis.HashDistributionDesc;
 import org.apache.doris.analysis.SwitchStmt;
 import org.apache.doris.catalog.Column;
@@ -29,6 +30,9 @@ import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.ExceptionChecker;
 import org.apache.doris.common.FeConstants;
+import org.apache.doris.datasource.DatabaseMetadata;
+import org.apache.doris.datasource.ExternalDatabase;
+import org.apache.doris.datasource.ExternalTable;
 import org.apache.doris.datasource.TableMetadata;
 import org.apache.doris.nereids.NereidsPlanner;
 import org.apache.doris.nereids.StatementContext;
@@ -52,6 +56,8 @@ import org.apache.doris.utframe.TestWithFeService;
 import mockit.Mock;
 import mockit.MockUp;
 import mockit.Mocked;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.junit.jupiter.api.Assertions;
@@ -66,6 +72,7 @@ import java.util.Optional;
 import java.util.Set;
 
 public class HiveDDLAndDMLPlanTest extends TestWithFeService {
+    private static final String mockedCtlName = "hive";
     private static final String mockedDbName = "mockedDb";
     private final NereidsParser nereidsParser = new NereidsParser();
 
@@ -74,6 +81,9 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
 
     private List<FieldSchema> checkedHiveCols;
 
+    private final Set<String> createdDbs = new HashSet<>();
+    private final Set<String> createdTables = new HashSet<>();
+
     @Override
     protected void runBeforeAll() throws Exception {
         connectContext.getSessionVariable().enableFallbackToOriginalPlanner = false;
@@ -104,7 +114,8 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
         createTable(createSourceInterPTable, true);
 
         // create external catalog and switch it
-        CreateCatalogStmt hiveCatalog = createStmt("create catalog hive properties('type' = 'hms',"
+        CreateCatalogStmt hiveCatalog = createStmt("create catalog " + mockedCtlName
+                + " properties('type' = 'hms',"
                 + " 'hive.metastore.uris' = 'thrift://192.168.0.1:9083');");
         Env.getCurrentEnv().getCatalogMgr().createCatalog(hiveCatalog);
         switchHive();
@@ -113,19 +124,37 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
         Map<String, String> dbProps = new HashMap<>();
         dbProps.put(HiveMetadataOps.LOCATION_URI_KEY, "file://loc/db");
         new MockUp<ThriftHMSCachedClient>(ThriftHMSCachedClient.class) {
+            @Mock
+            public void createDatabase(DatabaseMetadata db) {
+                if (db instanceof HiveDatabaseMetadata) {
+                    Database hiveDb = HiveUtil.toHiveDatabase((HiveDatabaseMetadata) db);
+                    createdDbs.add(hiveDb.getName());
+                }
+            }
+
+            @Mock
+            public Database getDatabase(String dbName) {
+                if (createdDbs.contains(dbName)) {
+                    return new Database(dbName, "", "", null);
+                }
+                return null;
+            }
+
+            @Mock
+            public boolean tableExists(String dbName, String tblName) {
+                return createdTables.contains(tblName);
+            }
+
             @Mock
             public List<String> getAllDatabases() {
-                return new ArrayList<String>() {
-                    {
-                        add(mockedDbName);
-                    }
-                };
+                return new ArrayList<>(createdDbs);
             }
 
             @Mock
             public void createTable(TableMetadata tbl, boolean ignoreIfExists) {
                 if (tbl instanceof HiveTableMetadata) {
                     Table table = HiveUtil.toHiveTable((HiveTableMetadata) tbl);
+                    createdTables.add(table.getTableName());
                     if (checkedHiveCols == null) {
                         // if checkedHiveCols is null, skip column check
                         return;
@@ -143,6 +172,8 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
         };
         CreateDbStmt createDbStmt = new CreateDbStmt(true, new DbName("hive", mockedDbName), dbProps);
         Env.getCurrentEnv().createDb(createDbStmt);
+        // checkout ifNotExists
+        Env.getCurrentEnv().createDb(createDbStmt);
         useDatabase(mockedDbName);
 
         // un-partitioned table
@@ -167,18 +198,29 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
         createTable(createSourceExtTable, true);
 
         HMSExternalCatalog hmsExternalCatalog = (HMSExternalCatalog) Env.getCurrentEnv().getCatalogMgr()
-                .getCatalog("hive");
+                .getCatalog(mockedCtlName);
+        new MockUp<HMSExternalCatalog>(HMSExternalCatalog.class) {
+            // mock after ThriftHMSCachedClient is mocked
+            @Mock
+            public ExternalDatabase<? extends ExternalTable> getDbNullable(String dbName) {
+                if (createdDbs.contains(dbName)) {
+                    return new HMSExternalDatabase(hmsExternalCatalog, RandomUtils.nextLong(), dbName);
+                }
+                return null;
+            }
+        };
         new MockUp<HMSExternalDatabase>(HMSExternalDatabase.class) {
+            // mock after ThriftHMSCachedClient is mocked
             @Mock
             HMSExternalTable getTableNullable(String tableName) {
-                return new HMSExternalTable(0, tableName, mockedDbName, hmsExternalCatalog);
+                if (createdTables.contains(tableName)) {
+                    return new HMSExternalTable(0, tableName, mockedDbName, hmsExternalCatalog);
+                }
+                return null;
             }
         };
         new MockUp<HMSExternalTable>(HMSExternalTable.class) {
-            @Mock
-            protected synchronized void makeSureInitialized() {
-                // mocked
-            }
+            // mock after ThriftHMSCachedClient is mocked
         };
     }
 
@@ -195,7 +237,38 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
     @Override
     protected void runAfterAll() throws Exception {
         switchHive();
-        dropDatabase(mockedDbName);
+        String createDbStmtStr = "DROP DATABASE IF EXISTS " + mockedDbName;
+        DropDbStmt createDbStmt = (DropDbStmt) parseAndAnalyzeStmt(createDbStmtStr);
+        Env.getCurrentEnv().dropDb(createDbStmt);
+        // check IF EXISTS
+        Env.getCurrentEnv().dropDb(createDbStmt);
+    }
+
+    @Test
+    public void testExistsDbOrTbl() throws Exception {
+        switchHive();
+        String db = "exists_db";
+        String createDbStmtStr = "CREATE DATABASE IF NOT EXISTS " + db;
+        createDatabaseWithSql(createDbStmtStr);
+        createDatabaseWithSql(createDbStmtStr);
+        useDatabase(db);
+
+        String createTableIfNotExists = "CREATE TABLE IF NOT EXISTS test_tbl(\n"
+                + "  `col1` BOOLEAN COMMENT 'col1',"
+                + "  `col2` INT COMMENT 'col2'"
+                + ")  ENGINE=hive\n"
+                + "PROPERTIES (\n"
+                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'file_format'='orc')";
+        createTable(createTableIfNotExists, true);
+        createTable(createTableIfNotExists, true);
+
+        dropTableWithSql("DROP TABLE IF EXISTS test_tbl");
+        dropTableWithSql("DROP TABLE IF EXISTS test_tbl");
+
+        String dropDbStmtStr = "DROP DATABASE IF EXISTS " + db;
+        dropDatabaseWithSql(dropDbStmtStr);
+        dropDatabaseWithSql(dropDbStmtStr);
     }
 
     @Test
@@ -220,7 +293,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
         createTable(createUnPartTable, true);
         dropTable("unpart_tbl", true);
 
-        String createPartTable = "CREATE TABLE `part_tbl`(\n"
+        String createPartTable = "CREATE TABLE IF NOT EXISTS `part_tbl`(\n"
                 + "  `col1` BOOLEAN COMMENT 'col1',\n"
                 + "  `col2` INT COMMENT 'col2',\n"
                 + "  `col3` BIGINT COMMENT 'col3',\n"
@@ -235,6 +308,8 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 + "  'location_uri'='hdfs://loc/db/tbl',\n"
                 + "  'file_format'='parquet')";
         createTable(createPartTable, true);
+        // check IF NOT EXISTS
+        createTable(createPartTable, true);
         dropTable("part_tbl", true);
 
         String createBucketedTableErr = "CREATE TABLE `err_buck_tbl`(\n"
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
index 54ea6c9a5f1..46d3e1b897d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
@@ -142,6 +142,13 @@ public class HiveMetadataOpsTest {
 
     @Test
     public void testCreateAndDropAll() throws UserException {
+        new MockUp<HMSExternalDatabase>(HMSExternalDatabase.class) {
+            // create table if getTableNullable return null
+            @Mock
+            HMSExternalTable getTableNullable(String tableName) {
+                return null;
+            }
+        };
         Map<String, String> dbProps = new HashMap<>();
         dbProps.put(HiveMetadataOps.LOCATION_URI_KEY, "file://loc/db");
         createDb("mockedDb", dbProps);
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 cb634666662..8b06b7a2633 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
@@ -605,6 +605,11 @@ public abstract class TestWithFeService {
         Env.getCurrentEnv().dropDb(createDbStmt);
     }
 
+    public void dropDatabaseWithSql(String dropDbSql) throws Exception {
+        DropDbStmt dropDbStmt = (DropDbStmt) parseAndAnalyzeStmt(dropDbSql);
+        Env.getCurrentEnv().dropDb(dropDbStmt);
+    }
+
     public void useDatabase(String dbName) {
         connectContext.setDatabase(dbName);
     }
@@ -653,6 +658,11 @@ public abstract class TestWithFeService {
         Env.getCurrentEnv().dropTable(dropTableStmt);
     }
 
+    public void dropTableWithSql(String dropTableSql) throws Exception {
+        DropTableStmt dropTableStmt = (DropTableStmt) parseAndAnalyzeStmt(dropTableSql);
+        Env.getCurrentEnv().dropTable(dropTableStmt);
+    }
+
     public void recoverTable(String table) throws Exception {
         RecoverTableStmt recoverTableStmt = (RecoverTableStmt) parseAndAnalyzeStmt(
                 "recover table " + table + ";", connectContext);


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


(doris) 10/19: [regression](insert)add hive DDL and CTAS regression case (#32924)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 07f296734a2c559dd27a7e912d91587e60fc1de3
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Fri Mar 29 20:26:16 2024 +0800

    [regression](insert)add hive DDL and CTAS regression case (#32924)
    
    Issue Number: #31442
    
    dependent on #32824
    
    add ddl(create and drop) test
    add ctas test
    add complex type test
    TODO:
    bucketed table test
    truncate test
    add/drop partition test
---
 .../main/java/org/apache/doris/common/Config.java  |   5 -
 .../apache/doris/datasource/ExternalCatalog.java   |  13 -
 .../datasource/hive/HiveDDLAndDMLPlanTest.java     |   1 -
 .../hive/ddl/test_hive_ddl_and_ctas.out            | 148 +++++++
 .../hive/ddl/test_hive_ddl_and_ctas.groovy         | 423 +++++++++++++++++++++
 5 files changed, 571 insertions(+), 19 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 11f747a35b0..7fc14ecff4c 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
@@ -2211,11 +2211,6 @@ public class Config extends ConfigBase {
         "Sample size for hive row count estimation."})
     public static int hive_stats_partition_sample_size = 3000;
 
-    @ConfField(mutable = true, masterOnly = true, description = {
-            "åÆē”Ø外č”ØDDL",
-            "Enable external table DDL"})
-    public static boolean enable_external_ddl = false;
-
     @ConfField(mutable = true, masterOnly = true, description = {
             "åÆē”ØHiveåˆ†ę”¶č”Ø",
             "Enable external hive bucket table"})
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
index 2238ee5de95..737705bd8b5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
@@ -29,7 +29,6 @@ import org.apache.doris.catalog.InfoSchemaDb;
 import org.apache.doris.catalog.Resource;
 import org.apache.doris.catalog.TableIf;
 import org.apache.doris.cluster.ClusterNamespace;
-import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.Version;
@@ -612,9 +611,6 @@ public abstract class ExternalCatalog
 
     @Override
     public void createDb(CreateDbStmt stmt) throws DdlException {
-        if (!Config.enable_external_ddl) {
-            throw new DdlException("Experimental. The config enable_external_ddl needs to be set to true.");
-        }
         makeSureInitialized();
         if (metadataOps == null) {
             LOG.warn("createDb not implemented");
@@ -630,9 +626,6 @@ public abstract class ExternalCatalog
 
     @Override
     public void dropDb(DropDbStmt stmt) throws DdlException {
-        if (!Config.enable_external_ddl) {
-            throw new DdlException("Experimental. The config enable_external_ddl needs to be set to true.");
-        }
         makeSureInitialized();
         if (metadataOps == null) {
             LOG.warn("dropDb not implemented");
@@ -648,9 +641,6 @@ public abstract class ExternalCatalog
 
     @Override
     public void createTable(CreateTableStmt stmt) throws UserException {
-        if (!Config.enable_external_ddl) {
-            throw new DdlException("Experimental. The config enable_external_ddl needs to be set to true.");
-        }
         makeSureInitialized();
         if (metadataOps == null) {
             LOG.warn("createTable not implemented");
@@ -666,9 +656,6 @@ public abstract class ExternalCatalog
 
     @Override
     public void dropTable(DropTableStmt stmt) throws DdlException {
-        if (!Config.enable_external_ddl) {
-            throw new DdlException("Experimental. The config enable_external_ddl needs to be set to true.");
-        }
         makeSureInitialized();
         if (metadataOps == null) {
             LOG.warn("dropTable not implemented");
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
index 1e1fb65a44c..9d51bf6005f 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
@@ -64,7 +64,6 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
         connectContext.getSessionVariable().enableNereidsTimeout = false;
         connectContext.getSessionVariable().enableNereidsDML = true;
         Config.enable_query_hive_views = false;
-        Config.enable_external_ddl = true;
         // create test internal table
         createDatabase(mockedDbName);
         useDatabase(mockedDbName);
diff --git a/regression-test/data/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.out b/regression-test/data/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.out
new file mode 100644
index 00000000000..f30081e70b6
--- /dev/null
+++ b/regression-test/data/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.out
@@ -0,0 +1,148 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !insert01 --
+true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
+
+-- !insert02 --
+\N	123	\N	\N	8.98
+true	123	9876543210	\N	stringval
+true	123	9876543210	123.4567	stringval
+
+-- !insert03 --
+\N	123	\N	\N	\N	\N	\N	varcharval	8.98
+false	1	9876543210	abcdefghij	2.3	6.28	0.0000	2223	stringval
+true	\N	9876543210	abcdefghij	2.3	6.28	\N	varcharval	stringval
+true	123	9876543210	\N	\N	\N	\N	varcharval	stringval
+true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
+
+-- !insert04 --
+true	1	1000	2.3	value_for_pt1	value_for_pt2
+
+-- !insert05 --
+true	1	1000	2.3
+true	1	1000	2.3
+true	1	1000	2.3
+
+-- !insert06 --
+\N	1	1000	1.3
+false	1	1000	\N
+true	1	1000	2.3
+true	1	1000	2.3
+true	1	1000	2.3
+
+-- !ctas_01 --
+1
+2
+2
+3
+3
+
+-- !ctas_02 --
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	value_for_pt22
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_03 --
+1	string value for col2
+1	string value for col2
+2	another string value for col2
+2	another string value for col2
+3	yet another string value for col2
+3	yet another string value for col2
+
+-- !ctas_04 --
+11	value_for_pt1	value_for_pt2
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	value_for_pt22
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_05 --
+1	string value for col2
+2	another string value for col2
+3	yet another string value for col2
+
+-- !ctas_06 --
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	value_for_pt22
+
+-- !complex_type01 --
+a	\N	\N	\N	\N	\N	\N	\N	\N	\N	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
+
+-- !complex_type02 --
+a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
+
+-- !insert01 --
+true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
+
+-- !insert02 --
+\N	123	\N	\N	8.98
+true	123	9876543210	\N	stringval
+true	123	9876543210	123.4567	stringval
+
+-- !insert03 --
+\N	123	\N	\N	\N	\N	\N	varcharval	8.98
+false	1	9876543210	abcdefghij	2.3	6.28	0.0000	2223	stringval
+true	\N	9876543210	abcdefghij	2.3	6.28	\N	varcharval	stringval
+true	123	9876543210	\N	\N	\N	\N	varcharval	stringval
+true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
+
+-- !insert04 --
+true	1	1000	2.3	value_for_pt1	value_for_pt2
+
+-- !insert05 --
+true	1	1000	2.3
+true	1	1000	2.3
+true	1	1000	2.3
+
+-- !insert06 --
+\N	1	1000	1.3
+false	1	1000	\N
+true	1	1000	2.3
+true	1	1000	2.3
+true	1	1000	2.3
+
+-- !ctas_01 --
+1
+2
+2
+3
+3
+
+-- !ctas_02 --
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	value_for_pt22
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_03 --
+1	string value for col2
+1	string value for col2
+2	another string value for col2
+2	another string value for col2
+3	yet another string value for col2
+3	yet another string value for col2
+
+-- !ctas_04 --
+11	value_for_pt1	value_for_pt2
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	value_for_pt22
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_05 --
+1	string value for col2
+2	another string value for col2
+3	yet another string value for col2
+
+-- !ctas_06 --
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	value_for_pt22
+
+-- !complex_type01 --
+a	\N	\N	\N	\N	\N	\N	\N	\N	\N	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
+
+-- !complex_type02 --
+a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
diff --git a/regression-test/suites/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.groovy b/regression-test/suites/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.groovy
new file mode 100644
index 00000000000..093ba674fc6
--- /dev/null
+++ b/regression-test/suites/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.groovy
@@ -0,0 +1,423 @@
+// 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_hive_ddl_and_ctas", "p0,external,hive,external_docker,external_docker_hive") {
+    String enabled = context.config.otherConfigs.get("enableHiveTest")
+    if (enabled != null && enabled.equalsIgnoreCase("true")) {
+        def file_formats = ["parquet", "orc"]
+
+        def test_db = { String catalog_name ->
+            sql """switch ${catalog_name}"""
+            sql """ create database if not exists `test_hive_db` """;
+            sql """use `test_hive_db`"""
+            sql """ drop database if exists `test_hive_db` """;
+        }
+
+        def test_loc_db = { String externalEnvIp, String hdfs_port, String catalog_name ->
+            sql """switch ${catalog_name}"""
+            sql """ create database if not exists `test_hive_loc_db`
+                    properties('location_uri'='hdfs://${externalEnvIp}:${hdfs_port}/tmp/hive/test_hive_loc_db')
+                """;
+            sql """use `test_hive_loc_db`"""
+            sql """ drop database if exists `test_hive_loc_db` """;
+        }
+
+        def test_db_tbl = { String file_format, String catalog_name ->
+            sql """switch ${catalog_name}"""
+            sql """ create database if not exists `test_hive_db` """;
+            sql """use `${catalog_name}`.`test_hive_db`"""
+
+            sql """
+                CREATE TABLE unpart_tbl_${file_format}(
+                  `col1` BOOLEAN COMMENT 'col1',
+                  `col2` INT COMMENT 'col2',
+                  `col3` BIGINT COMMENT 'col3',
+                  `col4` CHAR(10) COMMENT 'col4',
+                  `col5` FLOAT COMMENT 'col5',
+                  `col6` DOUBLE COMMENT 'col6',
+                  `col7` DECIMAL(9,4) COMMENT 'col7',
+                  `col8` VARCHAR(11) COMMENT 'col8',
+                  `col9` STRING COMMENT 'col9'
+                )  ENGINE=hive 
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+            """;
+
+            // test all columns
+            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9`) 
+                    VALUES 
+                    (true, 123, 9876543210, 'abcdefghij', 3.14, 6.28, 123.4567, 'varcharval', 'stringval');
+                """
+            order_qt_insert01 """ SELECT `col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9` FROM unpart_tbl_${file_format};  """
+
+            // test part of columns
+            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col8`, `col9`) 
+                    VALUES 
+                    (true, 123, 9876543210, 'varcharval', 'stringval');
+                """
+            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col8`, `col9`) 
+                    VALUES 
+                    (null, 123, 'varcharval', 8.98);
+                """
+            order_qt_insert02 """ SELECT `col1`, `col2`, `col3`, `col7`, `col9` FROM unpart_tbl_${file_format};  """
+
+            // test data diff
+            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9`) 
+                    VALUES 
+                    (true, null, 9876543210, 'abcdefghij', '2.3', 6.28, null, 'varcharval', 'stringval');
+                """
+            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9`) 
+                    VALUES 
+                    (false, '1', 9876543210, 'abcdefghij', '2.3', 6.28, 0, 2223, 'stringval');
+                """
+            order_qt_insert03 """ SELECT `col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9` FROM unpart_tbl_${file_format} """
+
+            sql """ drop table if exists unpart_tbl_${file_format}"""
+
+            // partitioned table test
+            sql """
+                CREATE TABLE part_tbl_${file_format}(
+                  `col1` BOOLEAN COMMENT 'col1',
+                  `col2` INT COMMENT 'col2',
+                  `col3` BIGINT COMMENT 'col3',
+                  `col4` DECIMAL(2,1) COMMENT 'col4',
+                  `pt1` VARCHAR COMMENT 'pt1',
+                  `pt2` VARCHAR COMMENT 'pt2'
+                )  ENGINE=hive 
+                PARTITION BY LIST (pt1, pt2) ()
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+            """;
+
+            // test all columns
+            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
+                VALUES (true, 1, 1000, 2.3, 'value_for_pt1', 'value_for_pt2')
+                """
+            order_qt_insert04 """ SELECT col1, col2, col3, col4, pt1, pt2 FROM part_tbl_${file_format};  """
+
+            // test part of columns
+            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
+                VALUES (true, 1, 1000, 2.3, 'value_for_pt1', 'value_for_pt2')
+                """
+            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
+                VALUES (true, 1, 1000, 2.3, 'value_for_pt1', 'value_for_pt2')
+                """
+            order_qt_insert05 """ SELECT col1, col2, col3, col4 FROM part_tbl_${file_format} """
+
+            // test data diff
+            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
+                VALUES (0, '1', 1000, null, 2.56, 'value_for_pt2')
+                """
+            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
+                VALUES (null, 1, '1000', '1.3', 'value_for_pt1', 2345)
+                """
+            order_qt_insert06 """ SELECT col1, col2, col3, col4 FROM part_tbl_${file_format}  """
+
+            sql """ drop table if exists part_tbl_${file_format}"""
+            sql """ drop database if exists `test_hive_db` """;
+        }
+
+        def test_ctas_tbl = { String file_format, String catalog_name ->
+            sql """ switch `${catalog_name}` """
+            sql """ create database if not exists `test_ctas` """;
+            sql """ switch internal """
+            sql """ create database if not exists test_ctas_olap """;
+            sql """ use internal.test_ctas_olap """
+
+            sql """
+                CREATE TABLE `unpart_ctas_olap_src` (
+                    `col1` INT COMMENT 'col1',
+                    `col2` STRING COMMENT 'col2'
+                )
+                ENGINE=olap
+                DISTRIBUTED BY HASH(col1) BUCKETS 16
+                PROPERTIES (
+                    'replication_num' = '1'
+                );
+            """
+
+            sql """ INSERT INTO `unpart_ctas_olap_src` (col1, col2) VALUES
+                (1, 'string value for col2'),
+                (2, 'another string value for col2'),
+                (3, 'yet another string value for col2'); 
+            """
+
+            sql """
+                CREATE TABLE `part_ctas_olap_src`(
+                    `col1` INT COMMENT 'col1',
+                    `pt1` VARCHAR(16) COMMENT 'pt1',
+                    `pt2` VARCHAR(16) COMMENT 'pt2'
+                )
+                ENGINE=olap
+                PARTITION BY LIST (pt1, pt2) (
+                    PARTITION pp1 VALUES IN(
+                        ('value_for_pt1', 'value_for_pt2'),
+                        ('value_for_pt11', 'value_for_pt22')
+                    )
+                )
+                DISTRIBUTED BY HASH(col1) BUCKETS 16
+                PROPERTIES (
+                    'replication_num' = '1'
+                );
+            """
+
+            sql """
+            INSERT INTO `part_ctas_olap_src` (col1, pt1, pt2) VALUES
+             (11, 'value_for_pt1', 'value_for_pt2'),
+             (22, 'value_for_pt11', 'value_for_pt22');
+            """
+
+            sql """ use `${catalog_name}`.`test_ctas` """
+            sql """
+                CREATE TABLE `unpart_ctas_src`(
+                  `col1` INT COMMENT 'col1',
+                  `col2` STRING COMMENT 'col2'
+                ) ENGINE=hive
+                PROPERTIES (
+                  'file_format'='parquet'
+                );
+            """
+
+            sql """ INSERT INTO `unpart_ctas_src` (col1, col2) VALUES
+                (1, 'string value for col2'),
+                (2, 'another string value for col2'),
+                (3, 'yet another string value for col2'); 
+            """
+
+            sql """
+                CREATE TABLE `part_ctas_src`(
+                  `col1` INT COMMENT 'col1',
+                  `pt1` VARCHAR COMMENT 'pt1',
+                  `pt2` VARCHAR COMMENT 'pt2'
+                ) ENGINE=hive
+                PARTITION BY LIST (pt1, pt2) ()
+                PROPERTIES (
+                  'file_format'='orc'
+                );
+            """
+
+            sql """
+            INSERT INTO `part_ctas_src` (col1, pt1, pt2) VALUES
+             (11, 'value_for_pt1', 'value_for_pt2'),
+             (22, 'value_for_pt11', 'value_for_pt22');
+            """
+
+            sql """ switch `${catalog_name}` """
+            // 1. external to external un-partitioned table
+            sql """ CREATE TABLE hive_ctas1 ENGINE=hive AS SELECT col1 FROM unpart_ctas_src; 
+            """
+
+            sql """ INSERT INTO hive_ctas1 SELECT col1 FROM unpart_ctas_src WHERE col1 > 1;
+            """
+
+            order_qt_ctas_01 """ SELECT * FROM hive_ctas1 """
+            sql """ DROP TABLE hive_ctas1 """
+
+            // 2. external to external partitioned table
+            sql """ CREATE TABLE hive_ctas2 ENGINE=hive AS SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>0;
+                """
+
+            sql """ INSERT INTO hive_ctas2 SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>=22;
+                """
+
+            order_qt_ctas_02 """ SELECT * FROM hive_ctas2  """
+            sql """ DROP TABLE hive_ctas2 """
+
+            // 3. internal to external un-partitioned table
+            sql """ CREATE TABLE ctas_o1 ENGINE=hive AS SELECT col1,col2 FROM internal.test_ctas_olap.unpart_ctas_olap_src;
+            """
+
+            sql """ INSERT INTO ctas_o1 SELECT col1,col2 FROM internal.test_ctas_olap.unpart_ctas_olap_src;
+            """
+
+            order_qt_ctas_03 """ SELECT * FROM ctas_o1  """
+            sql """ DROP TABLE ctas_o1 """
+
+            // 4. internal to external partitioned table
+            sql """ CREATE TABLE ctas_o2 ENGINE=hive AS SELECT col1,pt1,pt2 FROM internal.test_ctas_olap.part_ctas_olap_src WHERE col1>0;
+            """
+            sql """ INSERT INTO ctas_o2 SELECT col1,pt1,pt2 FROM internal.test_ctas_olap.part_ctas_olap_src WHERE col1>2;
+            """
+            order_qt_ctas_04 """ SELECT * FROM ctas_o2  """
+            sql """ DROP TABLE ctas_o2 """
+
+            // 5. check external to internal un-partitioned table
+            sql """ use internal.test_ctas_olap """
+            sql """  CREATE TABLE olap_ctas1
+                 PROPERTIES (
+                     "replication_allocation" = "tag.location.default: 1"
+                 ) AS SELECT col1,col2 
+                 FROM `${catalog_name}`.`test_ctas`.unpart_ctas_src;
+                """
+            order_qt_ctas_05 """ SELECT * FROM olap_ctas1  """
+            sql """ DROP TABLE olap_ctas1 """
+
+            // 6. check external to internal partitioned table
+            sql """ CREATE TABLE olap_ctas2 
+                PROPERTIES (
+                    "replication_allocation" = "tag.location.default: 1"
+                ) AS SELECT col1,pt1,pt2 
+                FROM `${catalog_name}`.`test_ctas`.part_ctas_src WHERE col1>0;
+                """
+            order_qt_ctas_06 """ SELECT * FROM olap_ctas2  """
+            sql """ DROP TABLE olap_ctas2 """
+
+            sql """ switch `${catalog_name}` """
+            sql """ DROP TABLE `test_ctas`.part_ctas_src """
+            sql """ DROP TABLE `test_ctas`.unpart_ctas_src """
+            sql """ drop database if exists `test_ctas` """;
+            sql """ DROP TABLE internal.test_ctas_olap.part_ctas_olap_src """
+            sql """ DROP TABLE internal.test_ctas_olap.unpart_ctas_olap_src """
+            sql """ switch internal """;
+            sql """ drop database if exists test_ctas_olap """;
+        }
+
+        def test_complex_type_tbl = { String file_format, String catalog_name ->
+            sql """ switch ${catalog_name} """
+            sql """ create database if not exists `test_complex_type` """;
+            sql """ use `${catalog_name}`.`test_complex_type` """
+
+            sql """
+                CREATE TABLE unpart_tbl_${file_format} (
+                  `col1` CHAR,
+                  `col2` CHAR(1),
+                  `col3` CHAR(16),
+                  `col4` VARCHAR,
+                  `col5` VARCHAR(255),
+                  `col6` DECIMAL(2,1),
+                  `col7` DECIMAL(5,0),
+                  `col8` DECIMAL(8,8),
+                  `col9` STRING,
+                  `col10` ARRAY<DECIMAL(4,3)>,
+                  `col11` ARRAY<CHAR(16)>,
+                  `col12` ARRAY<CHAR>,
+                  `col13` ARRAY<STRING>,
+                  `col14` ARRAY<MAP<INT, CHAR>>,
+                  `col15` MAP<BIGINT, CHAR>,
+                  `col16` MAP<BIGINT, DECIMAL(8,8)>,
+                  `col17` MAP<STRING, ARRAY<CHAR(16)>>,
+                  `col18` STRUCT<id:INT,gender:BOOLEAN,name:CHAR(16)>,
+                  `col19` STRUCT<scale:DECIMAL(7,4),metric:ARRAY<STRING>>,
+                  `col20` STRUCT<codes:ARRAY<INT>,props:MAP<STRING, ARRAY<CHAR(16)>>>
+                )  ENGINE=hive
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+            """;
+
+            sql """
+            INSERT INTO unpart_tbl_${file_format} (
+              col1, col2, col3, col4, col5, col6, col7, col8, col9,
+              col10, col11, col12, col13, col14, col15, col16, col17,
+              col18, col19, col20
+            ) VALUES (
+              'a', -- CHAR
+              'b', -- CHAR(1)
+              'c', -- CHAR(16)
+              'd', -- VARCHAR
+              'e', -- VARCHAR(255)
+              1.1, -- DECIMAL(2,1)
+              12345, -- DECIMAL(5,0)
+              0.12345678, -- DECIMAL(8,8)
+              'string', -- STRING
+              ARRAY(0.001, 0.002), -- ARRAY<DECIMAL(4,3)>
+              ARRAY('char1', 'char2'), -- ARRAY<CHAR(16)>
+              ARRAY('c', 'd'), -- ARRAY<CHAR>
+              ARRAY('string1', 'string2'), -- ARRAY<STRING>
+              ARRAY(MAP(1, 'a'), MAP(2, 'b')), -- ARRAY<MAP<INT, CHAR>>
+              MAP(1234567890123456789, 'a'), -- MAP<BIGINT, CHAR>
+              MAP(1234567890123456789, 0.12345678), -- MAP<BIGINT, DECIMAL(8,8)>
+              MAP('key', ARRAY('char1', 'char2')), -- MAP<STRING, ARRAY<CHAR(16)>>
+              STRUCT(1, TRUE, 'John Doe'), -- STRUCT<id:INT,gender:BOOLEAN,name:CHAR(16)>
+              STRUCT(123.4567, ARRAY('metric1', 'metric2')), -- STRUCT<scale:DECIMAL(7,4),metric:ARRAY<STRING>>
+              STRUCT(ARRAY(123, 456), MAP('key1', ARRAY('char1', 'char2'))) -- STRUCT<codes:ARRAY<INT>,props:MAP<STRING, ARRAY<CHAR(16)>>
+            );
+        """
+
+            sql """
+            INSERT INTO unpart_tbl_${file_format} (
+              col1, col11, col12, col13, col14, col15, col16, col17,
+              col18, col19
+            ) VALUES (
+              'a', -- CHAR
+              ARRAY('char1', 'char2'), -- ARRAY<CHAR(16)>
+              ARRAY('c', 'd'), -- ARRAY<CHAR>
+              ARRAY('string1', 'string2'), -- ARRAY<STRING>
+              ARRAY(MAP(1, 'a'), MAP(2, 'b')), -- ARRAY<MAP<INT, CHAR>>
+              MAP(1234567890123456789, 'a'), -- MAP<BIGINT, CHAR>
+              MAP(1234567890123456789, 0.12345678), -- MAP<BIGINT, DECIMAL(8,8)>
+              MAP('key', ARRAY('char1', 'char2')), -- MAP<STRING, ARRAY<CHAR(16)>>
+              STRUCT(1, TRUE, 'John Doe'), -- STRUCT<id:INT,gender:BOOLEAN,name:CHAR(16)>
+              STRUCT(123.4567, ARRAY('metric1', 'metric2')) -- STRUCT<scale:DECIMAL(7,4),metric:ARRAY<STRING>>
+            );
+        """
+
+            sql """
+            INSERT INTO unpart_tbl_${file_format} (
+              col1, col2, col3, col4, col5, col6, col7, col8, col9
+            ) VALUES (
+              'a', -- CHAR
+              'b', -- CHAR(1)
+              'c', -- CHAR(16)
+              'd', -- VARCHAR
+              'e', -- VARCHAR(255)
+              1.1, -- DECIMAL(2,1)
+              12345, -- DECIMAL(5,0)
+              0.12345678, -- DECIMAL(8,8)
+              'string' -- STRING
+            );
+        """
+
+            order_qt_complex_type01 """ SELECT * FROM unpart_tbl_${file_format} """
+            order_qt_complex_type02 """ SELECT * FROM unpart_tbl_${file_format} WHERE col2='b' """
+
+            sql """ DROP TABLE unpart_tbl_${file_format} """
+            sql """ drop database if exists `test_complex_type` """;
+        }
+
+        try {
+            String hms_port = context.config.otherConfigs.get("hms_port")
+            String hdfs_port = context.config.otherConfigs.get("hdfs_port")
+            String catalog_name = "test_hive_ddl_and_ctas"
+            String externalEnvIp = context.config.otherConfigs.get("externalEnvIp")
+
+            sql """drop catalog if exists ${catalog_name}"""
+            sql """create catalog if not exists ${catalog_name} properties (
+                'type'='hms',
+                'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}',
+                'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}'
+            );"""
+            sql """switch ${catalog_name}"""
+
+            sql """set enable_fallback_to_original_planner=false;"""
+
+            test_db(catalog_name)
+            test_loc_db(externalEnvIp, hdfs_port, catalog_name)
+            for (String file_format in file_formats) {
+                logger.info("Process file format" + file_format)
+                test_db_tbl(file_format, catalog_name)
+                test_ctas_tbl(file_format, catalog_name)
+                test_complex_type_tbl(file_format, catalog_name)
+                // todo: test bucket table: test_db_buck_tbl()
+            }
+            sql """drop catalog if exists ${catalog_name}"""
+        } finally {
+        }
+    }
+}


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


(doris) 05/19: [fix](insert)fix hive table sink type coercion and unify coercion (#32762)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit b98d225183f42cfe1dd19f9ed1e88d1e2f817c6b
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Tue Mar 26 17:05:46 2024 +0800

    [fix](insert)fix hive table sink type coercion and unify coercion (#32762)
    
    Issue Number: #31442
---
 .../doris/nereids/rules/analysis/BindSink.java     | 124 ++++++++++++---------
 .../trees/plans/logical/LogicalHiveTableSink.java  |   8 +-
 .../trees/plans/logical/LogicalOlapTableSink.java  |   8 +-
 .../trees/plans/logical/LogicalTableSink.java      |  17 ++-
 4 files changed, 85 insertions(+), 72 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java
index cac10b75ec3..20f05729822 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java
@@ -56,6 +56,8 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink;
 import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation;
 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.logical.LogicalTableSink;
+import org.apache.doris.nereids.trees.plans.logical.UnboundLogicalSink;
 import org.apache.doris.nereids.types.DataType;
 import org.apache.doris.nereids.types.StringType;
 import org.apache.doris.nereids.types.coercion.CharacterType;
@@ -186,6 +188,67 @@ public class BindSink implements AnalysisRuleFactory {
             throw new AnalysisException(e.getMessage(), e.getCause());
         }
 
+        Map<String, NamedExpression> columnToOutput = getColumnToOutput(ctx, table, isPartialUpdate,
+                boundSink, child);
+        LogicalProject<?> fullOutputProject = getOutputProjectByCoercion(table.getFullSchema(), child, columnToOutput);
+        return boundSink.withChildAndUpdateOutput(fullOutputProject);
+    }
+
+    private LogicalProject<?> getOutputProjectByCoercion(List<Column> tableSchema, LogicalPlan child,
+                                                         Map<String, NamedExpression> columnToOutput) {
+        List<NamedExpression> fullOutputExprs = ImmutableList.copyOf(columnToOutput.values());
+        if (child instanceof LogicalOneRowRelation) {
+            // remove default value slot in one row relation
+            child = ((LogicalOneRowRelation) child).withProjects(((LogicalOneRowRelation) child)
+                    .getProjects().stream()
+                    .filter(p -> !(p instanceof DefaultValueSlot))
+                    .collect(ImmutableList.toImmutableList()));
+        }
+        LogicalProject<?> fullOutputProject = new LogicalProject<>(fullOutputExprs, child);
+
+        // add cast project
+        List<NamedExpression> castExprs = Lists.newArrayList();
+        for (int i = 0; i < tableSchema.size(); ++i) {
+            Column col = tableSchema.get(i);
+            NamedExpression expr = columnToOutput.get(col.getName());
+            if (expr == null) {
+                // If `expr` is null, it means that the current load is a partial update
+                // and `col` should not be contained in the output of the sink node so
+                // we skip it.
+                continue;
+            }
+            DataType inputType = expr.getDataType();
+            DataType targetType = DataType.fromCatalogType(tableSchema.get(i).getType());
+            Expression castExpr = expr;
+            // TODO move string like type logic into TypeCoercionUtils#castIfNotSameType
+            if (isSourceAndTargetStringLikeType(inputType, targetType) && !inputType.equals(targetType)) {
+                int sourceLength = ((CharacterType) inputType).getLen();
+                int targetLength = ((CharacterType) targetType).getLen();
+                if (sourceLength == targetLength) {
+                    castExpr = TypeCoercionUtils.castIfNotSameType(castExpr, targetType);
+                } else if (sourceLength > targetLength && targetLength >= 0) {
+                    castExpr = new Substring(castExpr, Literal.of(1), Literal.of(targetLength));
+                } else if (targetType.isStringType()) {
+                    castExpr = new Cast(castExpr, StringType.INSTANCE);
+                }
+            } else {
+                castExpr = TypeCoercionUtils.castIfNotSameType(castExpr, targetType);
+            }
+            if (castExpr instanceof NamedExpression) {
+                castExprs.add(((NamedExpression) castExpr));
+            } else {
+                castExprs.add(new Alias(castExpr));
+            }
+        }
+        if (!castExprs.equals(fullOutputExprs)) {
+            fullOutputProject = new LogicalProject<Plan>(castExprs, fullOutputProject);
+        }
+        return fullOutputProject;
+    }
+
+    private static Map<String, NamedExpression> getColumnToOutput(
+            MatchingContext<? extends UnboundLogicalSink<Plan>> ctx,
+            TableIf table, boolean isPartialUpdate, LogicalTableSink<?> boundSink, LogicalPlan child) {
         // we need to insert all the columns of the target table
         // although some columns are not mentions.
         // so we add a projects to supply the default value.
@@ -225,11 +288,11 @@ public class BindSink implements AnalysisRuleFactory {
                     && !(columnToChildOutput.get(column) instanceof DefaultValueSlot)) {
                 columnToOutput.put(column.getName(), columnToChildOutput.get(column));
             } else {
-                if (table.hasSequenceCol()
+                if (table instanceof OlapTable && ((OlapTable) table).hasSequenceCol()
                         && column.getName().equals(Column.SEQUENCE_COL)
-                        && table.getSequenceMapCol() != null) {
+                        && ((OlapTable) table).getSequenceMapCol() != null) {
                     Optional<Column> seqCol = table.getFullSchema().stream()
-                            .filter(col -> col.getName().equals(table.getSequenceMapCol()))
+                            .filter(col -> col.getName().equals(((OlapTable) table).getSequenceMapCol()))
                             .findFirst();
                     if (!seqCol.isPresent()) {
                         throw new AnalysisException("sequence column is not contained in"
@@ -303,55 +366,7 @@ public class BindSink implements AnalysisRuleFactory {
                 }
             }
         }
-        List<NamedExpression> fullOutputExprs = ImmutableList.copyOf(columnToOutput.values());
-        if (child instanceof LogicalOneRowRelation) {
-            // remove default value slot in one row relation
-            child = ((LogicalOneRowRelation) child).withProjects(((LogicalOneRowRelation) child)
-                    .getProjects().stream()
-                    .filter(p -> !(p instanceof DefaultValueSlot))
-                    .collect(ImmutableList.toImmutableList()));
-        }
-        LogicalProject<?> fullOutputProject = new LogicalProject<>(fullOutputExprs, child);
-
-        // add cast project
-        List<NamedExpression> castExprs = Lists.newArrayList();
-        for (int i = 0; i < table.getFullSchema().size(); ++i) {
-            Column col = table.getFullSchema().get(i);
-            NamedExpression expr = columnToOutput.get(col.getName());
-            if (expr == null) {
-                // If `expr` is null, it means that the current load is a partial update
-                // and `col` should not be contained in the output of the sink node so
-                // we skip it.
-                continue;
-            }
-            DataType inputType = expr.getDataType();
-            DataType targetType = DataType.fromCatalogType(table.getFullSchema().get(i).getType());
-            Expression castExpr = expr;
-            // TODO move string like type logic into TypeCoercionUtils#castIfNotSameType
-            if (isSourceAndTargetStringLikeType(inputType, targetType) && !inputType.equals(targetType)) {
-                int sourceLength = ((CharacterType) inputType).getLen();
-                int targetLength = ((CharacterType) targetType).getLen();
-                if (sourceLength == targetLength) {
-                    castExpr = TypeCoercionUtils.castIfNotSameType(castExpr, targetType);
-                } else if (sourceLength > targetLength && targetLength >= 0) {
-                    castExpr = new Substring(castExpr, Literal.of(1), Literal.of(targetLength));
-                } else if (targetType.isStringType()) {
-                    castExpr = new Cast(castExpr, StringType.INSTANCE);
-                }
-            } else {
-                castExpr = TypeCoercionUtils.castIfNotSameType(castExpr, targetType);
-            }
-            if (castExpr instanceof NamedExpression) {
-                castExprs.add(((NamedExpression) castExpr));
-            } else {
-                castExprs.add(new Alias(castExpr));
-            }
-        }
-        if (!castExprs.equals(fullOutputExprs)) {
-            fullOutputProject = new LogicalProject<Plan>(castExprs, fullOutputProject);
-        }
-
-        return boundSink.withChildAndUpdateOutput(fullOutputProject);
+        return columnToOutput;
     }
 
     private Plan bindHiveTableSink(MatchingContext<UnboundHiveTableSink<Plan>> ctx) {
@@ -394,7 +409,10 @@ public class BindSink implements AnalysisRuleFactory {
         if (boundSink.getCols().size() != child.getOutput().size()) {
             throw new AnalysisException("insert into cols should be corresponding to the query output");
         }
-        return boundSink;
+        Map<String, NamedExpression> columnToOutput = getColumnToOutput(ctx, table, false,
+                boundSink, child);
+        LogicalProject<?> fullOutputProject = getOutputProjectByCoercion(table.getFullSchema(), child, columnToOutput);
+        return boundSink.withChildAndUpdateOutput(fullOutputProject);
     }
 
     private Pair<Database, OlapTable> bind(CascadesContext cascadesContext, UnboundTableSink<? extends Plan> sink) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHiveTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHiveTableSink.java
index 7ae217f3fb4..360d227b0f0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHiveTableSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHiveTableSink.java
@@ -47,7 +47,6 @@ public class LogicalHiveTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
     // bound data sink
     private final HMSExternalDatabase database;
     private final HMSExternalTable targetTable;
-    private final List<Column> cols;
     private final Set<String> hivePartitionKeys;
     private final DMLCommandType dmlCommandType;
 
@@ -63,10 +62,9 @@ public class LogicalHiveTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
                                 Optional<GroupExpression> groupExpression,
                                 Optional<LogicalProperties> logicalProperties,
                                 CHILD_TYPE child) {
-        super(PlanType.LOGICAL_HIVE_TABLE_SINK, outputExprs, groupExpression, logicalProperties, child);
+        super(PlanType.LOGICAL_HIVE_TABLE_SINK, outputExprs, groupExpression, logicalProperties, cols, child);
         this.database = Objects.requireNonNull(database, "database != null in LogicalHiveTableSink");
         this.targetTable = Objects.requireNonNull(targetTable, "targetTable != null in LogicalHiveTableSink");
-        this.cols = Utils.copyRequiredList(cols);
         this.dmlCommandType = dmlCommandType;
         this.hivePartitionKeys = hivePartitionKeys;
     }
@@ -99,10 +97,6 @@ public class LogicalHiveTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
         return targetTable;
     }
 
-    public List<Column> getCols() {
-        return cols;
-    }
-
     public Set<String> getHivePartitionKeys() {
         return hivePartitionKeys;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java
index c5e3336dabb..397c2927d84 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java
@@ -46,7 +46,6 @@ public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
     // bound data sink
     private final Database database;
     private final OlapTable targetTable;
-    private final List<Column> cols;
     private final List<Long> partitionIds;
     private final boolean isPartialUpdate;
     private final DMLCommandType dmlCommandType;
@@ -65,10 +64,9 @@ public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
             List<Long> partitionIds, List<NamedExpression> outputExprs, boolean isPartialUpdate,
             DMLCommandType dmlCommandType, Optional<GroupExpression> groupExpression,
             Optional<LogicalProperties> logicalProperties, CHILD_TYPE child) {
-        super(PlanType.LOGICAL_OLAP_TABLE_SINK, outputExprs, groupExpression, logicalProperties, child);
+        super(PlanType.LOGICAL_OLAP_TABLE_SINK, outputExprs, groupExpression, logicalProperties, cols, child);
         this.database = Objects.requireNonNull(database, "database != null in LogicalOlapTableSink");
         this.targetTable = Objects.requireNonNull(targetTable, "targetTable != null in LogicalOlapTableSink");
-        this.cols = Utils.copyRequiredList(cols);
         this.isPartialUpdate = isPartialUpdate;
         this.dmlCommandType = dmlCommandType;
         this.partitionIds = Utils.copyRequiredList(partitionIds);
@@ -97,10 +95,6 @@ public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
         return targetTable;
     }
 
-    public List<Column> getCols() {
-        return cols;
-    }
-
     public List<Long> getPartitionIds() {
         return partitionIds;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTableSink.java
index 90133274f6e..7aca0c599b4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTableSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTableSink.java
@@ -17,11 +17,14 @@
 
 package org.apache.doris.nereids.trees.plans.logical;
 
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.TableIf;
 import org.apache.doris.nereids.memo.GroupExpression;
 import org.apache.doris.nereids.properties.LogicalProperties;
 import org.apache.doris.nereids.trees.expressions.NamedExpression;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.util.Utils;
 
 import java.util.List;
 import java.util.Optional;
@@ -30,14 +33,18 @@ import java.util.Optional;
  * Logical table sink for all table type sink
  */
 public abstract class LogicalTableSink<CHILD_TYPE extends Plan> extends LogicalSink<CHILD_TYPE> {
-    public LogicalTableSink(PlanType type,
-            List<NamedExpression> outputExprs, CHILD_TYPE child) {
-        super(type, outputExprs, child);
-    }
+    protected final List<Column> cols;
 
     public LogicalTableSink(PlanType type, List<NamedExpression> outputExprs,
             Optional<GroupExpression> groupExpression,
-            Optional<LogicalProperties> logicalProperties, CHILD_TYPE child) {
+            Optional<LogicalProperties> logicalProperties, List<Column> cols, CHILD_TYPE child) {
         super(type, outputExprs, groupExpression, logicalProperties, child);
+        this.cols = Utils.copyRequiredList(cols);
+    }
+
+    public abstract TableIf getTargetTable();
+
+    public List<Column> getCols() {
+        return cols;
     }
 }


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


(doris) 01/19: [feature][insert]Adapt the create tableĀ  statement to the nereids sql (#32458)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 36a1bf1d73fc501cf66440b76921a6f0f7198e10
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Wed Mar 20 20:47:18 2024 +0800

    [feature][insert]Adapt the create tableĀ  statement to the nereids sql (#32458)
    
    issue: #31442
    
    1. adapt  create table statement from doris  to hive
    2. fix insert overwrite for table sink
    
    > The doris create hive table statement:
    
    ```
    mysql> CREATE TABLE buck2(
        ->     id int COMMENT 'col1',
        ->     name string COMMENT 'col2',
        ->     dt string COMMENT 'part1',
        ->     dtm string COMMENT 'part2'
        -> ) ENGINE=hive
        -> COMMENT "create tbl"
        -> PARTITION BY LIST (dt, dtm) ()
        -> DISTRIBUTED BY HASH (id) BUCKETS 16
        -> PROPERTIES(
        ->     "file_format" = "orc"
        -> );
    ```
    
    > generated  hive create table statement:
    
    ```
    CREATE TABLE `buck2`(
      `id` int COMMENT 'col1',
      `name` string COMMENT 'col2')
    PARTITIONED BY (
     `dt` string,
     `dtm` string)
    CLUSTERED BY (
      id)
    INTO 16 BUCKETS
    ROW FORMAT SERDE
      'org.apache.hadoop.hive.ql.io.orc.OrcSerde'
    STORED AS INPUTFORMAT
      'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat'
    OUTPUTFORMAT
      'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'
    LOCATION
      'hdfs://HDFS8000871/usr/hive/warehouse/jz3.db/buck2'
    TBLPROPERTIES (
      'transient_lastDdlTime'='1710840747',
      'doris.file_format'='orc')
    
    ```
---
 .../main/java/org/apache/doris/common/Config.java  |   5 +
 .../datasource/hive/HiveMetaStoreClientHelper.java |  16 ++
 .../doris/datasource/hive/HiveMetadataOps.java     |  88 ++++++++---
 .../doris/datasource/hive/HiveTableMetadata.java   |  59 ++++++--
 .../datasource/hive/ThriftHMSCachedClient.java     |  40 +++--
 .../doris/insertoverwrite/InsertOverwriteUtil.java |  45 +++---
 .../commands/insert/HiveInsertCommandContext.java  |   2 +-
 .../insert/InsertOverwriteTableCommand.java        |  92 +++++++-----
 .../trees/plans/physical/PhysicalTableSink.java    |   3 +
 .../doris/datasource/hive/HiveMetadataOpsTest.java | 162 +++++++++++++++++++++
 .../doris/datasource/hive/HmsCommitTest.java       |   5 +-
 11 files changed, 409 insertions(+), 108 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 0d4ddb947a7..11f747a35b0 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
@@ -2216,6 +2216,11 @@ public class Config extends ConfigBase {
             "Enable external table DDL"})
     public static boolean enable_external_ddl = false;
 
+    @ConfField(mutable = true, masterOnly = true, description = {
+            "åÆē”ØHiveåˆ†ę”¶č”Ø",
+            "Enable external hive bucket table"})
+    public static boolean enable_create_hive_bucket_table = false;
+
     @ConfField(mutable = true, masterOnly = true, description = {
             "Hive创å»ŗ外éƒØč”Øé»˜č®¤ęŒ‡å®šēš„ę–‡ä»¶ę ¼å¼",
             "Default hive file format for creating table."})
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java
index 4cbad277d43..4c002275bde 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java
@@ -606,6 +606,22 @@ public class HiveMetaStoreClientHelper {
             return "double";
         } else if (dorisType.equals(Type.STRING)) {
             return "string";
+        } else if (dorisType.equals(Type.DEFAULT_DECIMALV3)) {
+            StringBuilder decimalType = new StringBuilder();
+            decimalType.append("decimal");
+            ScalarType scalarType = (ScalarType) dorisType;
+            int precision = scalarType.getScalarPrecision();
+            if (precision == 0) {
+                precision = ScalarType.DEFAULT_PRECISION;
+            }
+            // decimal(precision, scale)
+            int scale = scalarType.getScalarScale();
+            decimalType.append("(");
+            decimalType.append(precision);
+            decimalType.append(",");
+            decimalType.append(scale);
+            decimalType.append(")");
+            return decimalType.toString();
         }
         throw new HMSClientException("Unsupported type conversion of " + dorisType.toSql());
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
index 9279c48fbaa..a182aa9cc00 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
@@ -19,8 +19,10 @@ package org.apache.doris.datasource.hive;
 
 import org.apache.doris.analysis.CreateDbStmt;
 import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.analysis.DistributionDesc;
 import org.apache.doris.analysis.DropDbStmt;
 import org.apache.doris.analysis.DropTableStmt;
+import org.apache.doris.analysis.HashDistributionDesc;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.JdbcResource;
 import org.apache.doris.common.Config;
@@ -34,7 +36,9 @@ import org.apache.doris.fs.remote.RemoteFileSystem;
 import org.apache.doris.fs.remote.dfs.DFSFileSystem;
 import org.apache.doris.thrift.THivePartitionUpdate;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Table;
@@ -42,25 +46,32 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.function.Function;
 
 public class HiveMetadataOps implements ExternalMetadataOps {
+    public static final String LOCATION_URI_KEY = "location_uri";
+    public static final String FILE_FORMAT_KEY = "file_format";
+    public static final Set<String> DORIS_HIVE_KEYS = ImmutableSet.of(FILE_FORMAT_KEY, LOCATION_URI_KEY);
     private static final Logger LOG = LogManager.getLogger(HiveMetadataOps.class);
     private static final int MIN_CLIENT_POOL_SIZE = 8;
-    private JdbcClientConfig jdbcClientConfig;
-    private HiveConf hiveConf;
-    private HMSExternalCatalog catalog;
-    private HMSCachedClient client;
+    private final HMSCachedClient client;
     private final RemoteFileSystem fs;
+    private final HMSExternalCatalog catalog;
 
     public HiveMetadataOps(HiveConf hiveConf, JdbcClientConfig jdbcClientConfig, HMSExternalCatalog catalog) {
+        this(catalog, createCachedClient(hiveConf,
+                Math.max(MIN_CLIENT_POOL_SIZE, Config.max_external_cache_loader_thread_pool_size),
+                jdbcClientConfig));
+    }
+
+    @VisibleForTesting
+    public HiveMetadataOps(HMSExternalCatalog catalog, HMSCachedClient client) {
         this.catalog = catalog;
-        this.hiveConf = hiveConf;
-        this.jdbcClientConfig = jdbcClientConfig;
-        this.client = createCachedClient(hiveConf,
-                Math.max(MIN_CLIENT_POOL_SIZE, Config.max_external_cache_loader_thread_pool_size), jdbcClientConfig);
+        this.client = client;
         this.fs = new DFSFileSystem(catalog.getProperties());
     }
 
@@ -91,10 +102,11 @@ public class HiveMetadataOps implements ExternalMetadataOps {
         try {
             HiveDatabaseMetadata catalogDatabase = new HiveDatabaseMetadata();
             catalogDatabase.setDbName(fullDbName);
-            catalogDatabase.setProperties(properties);
-            if (properties.containsKey("location_uri")) {
-                catalogDatabase.setLocationUri(properties.get("location_uri"));
+            if (properties.containsKey(LOCATION_URI_KEY)) {
+                catalogDatabase.setLocationUri(properties.get(LOCATION_URI_KEY));
             }
+            properties.remove(LOCATION_URI_KEY);
+            catalogDatabase.setProperties(properties);
             catalogDatabase.setComment(properties.getOrDefault("comment", ""));
             client.createDatabase(catalogDatabase);
             catalog.onRefresh(true);
@@ -124,16 +136,50 @@ public class HiveMetadataOps implements ExternalMetadataOps {
             throw new UserException("Failed to get database: '" + dbName + "' in catalog: " + catalog.getName());
         }
         try {
-            Map<String, String> props = stmt.getExtProperties();
-            String fileFormat = props.getOrDefault("file_format", Config.hive_default_file_format);
-            HiveTableMetadata catalogTable = HiveTableMetadata.of(dbName,
-                    tblName,
-                    stmt.getColumns(),
-                    parsePartitionKeys(props),
-                    props,
-                    fileFormat);
-
-            client.createTable(catalogTable, stmt.isSetIfNotExists());
+            Map<String, String> props = stmt.getProperties();
+            String fileFormat = props.getOrDefault(FILE_FORMAT_KEY, Config.hive_default_file_format);
+            Map<String, String> ddlProps = new HashMap<>();
+            for (Map.Entry<String, String> entry : props.entrySet()) {
+                String key = entry.getKey().toLowerCase();
+                if (DORIS_HIVE_KEYS.contains(entry.getKey().toLowerCase())) {
+                    ddlProps.put("doris." + key, entry.getValue());
+                } else {
+                    ddlProps.put(key, entry.getValue());
+                }
+            }
+            List<String> partitionColNames = new ArrayList<>();
+            if (stmt.getPartitionDesc() != null) {
+                partitionColNames.addAll(stmt.getPartitionDesc().getPartitionColNames());
+            }
+            HiveTableMetadata hiveTableMeta;
+            DistributionDesc bucketInfo = stmt.getDistributionDesc();
+            if (bucketInfo != null) {
+                if (Config.enable_create_hive_bucket_table) {
+                    if (bucketInfo instanceof HashDistributionDesc) {
+                        hiveTableMeta = HiveTableMetadata.of(dbName,
+                                tblName,
+                                stmt.getColumns(),
+                                partitionColNames,
+                                ((HashDistributionDesc) bucketInfo).getDistributionColumnNames(),
+                                bucketInfo.getBuckets(),
+                                ddlProps,
+                                fileFormat);
+                    } else {
+                        throw new UserException("External hive table only supports hash bucketing");
+                    }
+                } else {
+                    throw new UserException("Create hive bucket table need"
+                            + " set enable_create_hive_bucket_table to true");
+                }
+            } else {
+                hiveTableMeta = HiveTableMetadata.of(dbName,
+                        tblName,
+                        stmt.getColumns(),
+                        partitionColNames,
+                        ddlProps,
+                        fileFormat);
+            }
+            client.createTable(hiveTableMeta, stmt.isSetIfNotExists());
             db.setUnInitialized(true);
         } catch (Exception e) {
             throw new UserException(e.getMessage(), e);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTableMetadata.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTableMetadata.java
index fde0a2d4d04..d8de9d60734 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTableMetadata.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTableMetadata.java
@@ -20,32 +20,45 @@ package org.apache.doris.datasource.hive;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.datasource.TableMetadata;
 
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
 public class HiveTableMetadata implements TableMetadata {
-    private String dbName;
-    private String tableName;
-    private List<Column> columns;
-    private List<FieldSchema> partitionKeys;
-    private String fileFormat;
-    private Map<String, String> properties;
+    private final String dbName;
+    private final String tableName;
+    private final List<Column> columns;
+    private final List<String> partitionKeys;
+    private final String fileFormat;
+    private final Map<String, String> properties;
+    private List<String> bucketCols;
+    private int numBuckets;
     // private String viewSql;
 
     public HiveTableMetadata(String dbName,
                              String tblName,
                              List<Column> columns,
-                             List<FieldSchema> partitionKeys,
+                             List<String> partitionKeys,
+                             Map<String, String> props,
+                             String fileFormat) {
+        this(dbName, tblName, columns, partitionKeys, new ArrayList<>(), 0, props, fileFormat);
+    }
+
+    public HiveTableMetadata(String dbName, String tableName,
+                             List<Column> columns,
+                             List<String> partitionKeys,
+                             List<String> bucketCols,
+                             int numBuckets,
                              Map<String, String> props,
                              String fileFormat) {
         this.dbName = dbName;
-        this.tableName = tblName;
+        this.tableName = tableName;
         this.columns = columns;
         this.partitionKeys = partitionKeys;
-        this.fileFormat = fileFormat;
+        this.bucketCols = bucketCols;
+        this.numBuckets = numBuckets;
         this.properties = props;
+        this.fileFormat = fileFormat;
     }
 
     @Override
@@ -67,7 +80,7 @@ public class HiveTableMetadata implements TableMetadata {
         return columns;
     }
 
-    public List<FieldSchema> getPartitionKeys() {
+    public List<String> getPartitionKeys() {
         return partitionKeys;
     }
 
@@ -75,12 +88,32 @@ public class HiveTableMetadata implements TableMetadata {
         return fileFormat;
     }
 
+    public List<String> getBucketCols() {
+        return bucketCols;
+    }
+
+    public int getNumBuckets() {
+        return numBuckets;
+    }
+
     public static HiveTableMetadata of(String dbName,
                                        String tblName,
                                        List<Column> columns,
-                                       List<FieldSchema> partitionKeys,
+                                       List<String> partitionKeys,
                                        Map<String, String> props,
                                        String fileFormat) {
         return new HiveTableMetadata(dbName, tblName, columns, partitionKeys, props, fileFormat);
     }
+
+    public static HiveTableMetadata of(String dbName,
+                                       String tblName,
+                                       List<Column> columns,
+                                       List<String> partitionKeys,
+                                       List<String> bucketCols,
+                                       int numBuckets,
+                                       Map<String, String> props,
+                                       String fileFormat) {
+        return new HiveTableMetadata(dbName, tblName, columns, partitionKeys,
+                bucketCols, numBuckets, props, fileFormat);
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
index d4f63c5a8fb..b5b1147447e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
@@ -20,6 +20,7 @@ package org.apache.doris.datasource.hive;
 import org.apache.doris.analysis.TableName;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.common.Config;
+import org.apache.doris.common.Pair;
 import org.apache.doris.datasource.DatabaseMetadata;
 import org.apache.doris.datasource.TableMetadata;
 import org.apache.doris.datasource.hive.event.MetastoreNotificationFetchException;
@@ -71,12 +72,14 @@ import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.Queue;
+import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
@@ -199,11 +202,14 @@ public class ThriftHMSCachedClient implements HMSCachedClient {
         table.setCreateTime(createTime);
         table.setLastAccessTime(createTime);
         // table.setRetention(0);
-        String location = hiveTable.getProperties().get("external_location");
-        table.setSd(toHiveStorageDesc(hiveTable.getColumns(),
-                hiveTable.getFileFormat(),
-                location));
-        table.setPartitionKeys(hiveTable.getPartitionKeys());
+        String location = hiveTable.getProperties().get(HiveMetadataOps.LOCATION_URI_KEY);
+        Set<String> partitionSet = new HashSet<>(hiveTable.getPartitionKeys());
+        Pair<List<FieldSchema>, List<FieldSchema>> hiveSchema = toHiveSchema(hiveTable.getColumns(), partitionSet);
+
+        table.setSd(toHiveStorageDesc(hiveSchema.first, hiveTable.getBucketCols(), hiveTable.getNumBuckets(),
+                hiveTable.getFileFormat(), location));
+        table.setPartitionKeys(hiveSchema.second);
+
         // table.setViewOriginalText(hiveTable.getViewSql());
         // table.setViewExpandedText(hiveTable.getViewSql());
         table.setTableType("MANAGED_TABLE");
@@ -211,13 +217,19 @@ public class ThriftHMSCachedClient implements HMSCachedClient {
         return table;
     }
 
-    private static StorageDescriptor toHiveStorageDesc(List<Column> columns, String fileFormat, String location) {
+    private static StorageDescriptor toHiveStorageDesc(List<FieldSchema> columns,
+                                                       List<String> bucketCols,
+                                                       int numBuckets,
+                                                       String fileFormat,
+                                                       String location) {
         StorageDescriptor sd = new StorageDescriptor();
-        sd.setCols(toHiveColumns(columns));
+        sd.setCols(columns);
         setFileFormat(fileFormat, sd);
         if (StringUtils.isNotEmpty(location)) {
             sd.setLocation(location);
         }
+        sd.setBucketCols(bucketCols);
+        sd.setNumBuckets(numBuckets);
         Map<String, String> parameters = new HashMap<>();
         parameters.put("tag", "doris external hive talbe");
         sd.setParameters(parameters);
@@ -246,17 +258,23 @@ public class ThriftHMSCachedClient implements HMSCachedClient {
         sd.setOutputFormat(outputFormat);
     }
 
-    private static List<FieldSchema> toHiveColumns(List<Column> columns) {
-        List<FieldSchema> result = new ArrayList<>();
+    private static Pair<List<FieldSchema>, List<FieldSchema>> toHiveSchema(List<Column> columns,
+                Set<String> partitionSet) {
+        List<FieldSchema> hiveCols = new ArrayList<>();
+        List<FieldSchema> hiveParts = new ArrayList<>();
         for (Column column : columns) {
             FieldSchema hiveFieldSchema = new FieldSchema();
             // TODO: add doc, just support doris type
             hiveFieldSchema.setType(HiveMetaStoreClientHelper.dorisTypeToHiveType(column.getType()));
             hiveFieldSchema.setName(column.getName());
             hiveFieldSchema.setComment(column.getComment());
-            result.add(hiveFieldSchema);
+            if (partitionSet.contains(column.getName())) {
+                hiveParts.add(hiveFieldSchema);
+            } else {
+                hiveCols.add(hiveFieldSchema);
+            }
         }
-        return result;
+        return Pair.of(hiveCols, hiveParts);
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteUtil.java b/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteUtil.java
index 54f9895ab2c..c4d3068e09f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteUtil.java
@@ -24,6 +24,7 @@ import org.apache.doris.analysis.ReplacePartitionClause;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.TableIf;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.util.PropertyAnalyzer;
 
@@ -42,16 +43,18 @@ public class InsertOverwriteUtil {
     /**
      * add temp partitions
      *
-     * @param olapTable
+     * @param tableIf
      * @param partitionNames
      * @param tempPartitionNames
      * @throws DdlException
      */
-    public static void addTempPartitions(OlapTable olapTable, List<String> partitionNames,
-            List<String> tempPartitionNames) throws DdlException {
-        for (int i = 0; i < partitionNames.size(); i++) {
-            Env.getCurrentEnv().addPartitionLike((Database) olapTable.getDatabase(), olapTable.getName(),
-                    new AddPartitionLikeClause(tempPartitionNames.get(i), partitionNames.get(i), true));
+    public static void addTempPartitions(TableIf tableIf, List<String> partitionNames,
+                                         List<String> tempPartitionNames) throws DdlException {
+        if (tableIf instanceof OlapTable) {
+            for (int i = 0; i < partitionNames.size(); i++) {
+                Env.getCurrentEnv().addPartitionLike((Database) tableIf.getDatabase(), tableIf.getName(),
+                        new AddPartitionLikeClause(tempPartitionNames.get(i), partitionNames.get(i), true));
+            }
         }
     }
 
@@ -63,23 +66,25 @@ public class InsertOverwriteUtil {
      * @param tempPartitionNames
      * @throws DdlException
      */
-    public static void replacePartition(OlapTable olapTable, List<String> partitionNames,
+    public static void replacePartition(TableIf olapTable, List<String> partitionNames,
             List<String> tempPartitionNames) throws DdlException {
-        try {
-            if (!olapTable.writeLockIfExist()) {
-                return;
+        if (olapTable instanceof OlapTable) {
+            try {
+                if (!olapTable.writeLockIfExist()) {
+                    return;
+                }
+                Map<String, String> properties = Maps.newHashMap();
+                properties.put(PropertyAnalyzer.PROPERTIES_USE_TEMP_PARTITION_NAME, "false");
+                ReplacePartitionClause replacePartitionClause = new ReplacePartitionClause(
+                        new PartitionNames(false, partitionNames),
+                        new PartitionNames(true, tempPartitionNames), properties);
+                Env.getCurrentEnv()
+                        .replaceTempPartition((Database) olapTable.getDatabase(),
+                                (OlapTable) olapTable, replacePartitionClause);
+            } finally {
+                olapTable.writeUnlock();
             }
-            Map<String, String> properties = Maps.newHashMap();
-            properties.put(PropertyAnalyzer.PROPERTIES_USE_TEMP_PARTITION_NAME, "false");
-            ReplacePartitionClause replacePartitionClause = new ReplacePartitionClause(
-                    new PartitionNames(false, partitionNames),
-                    new PartitionNames(true, tempPartitionNames), properties);
-            Env.getCurrentEnv()
-                    .replaceTempPartition((Database) olapTable.getDatabase(), olapTable, replacePartitionClause);
-        } finally {
-            olapTable.writeUnlock();
         }
-
     }
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertCommandContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertCommandContext.java
index 9e4c2bc92a3..31e56fd6ccc 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertCommandContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertCommandContext.java
@@ -21,7 +21,7 @@ package org.apache.doris.nereids.trees.plans.commands.insert;
  * For Hive Table
  */
 public class HiveInsertCommandContext extends InsertCommandContext {
-    private boolean overwrite = true;
+    private boolean overwrite = false;
 
     public boolean isOverwrite() {
         return overwrite;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java
index 0741982c968..44c17545be5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java
@@ -42,6 +42,7 @@ import org.apache.doris.nereids.trees.plans.commands.ForwardWithSync;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 import org.apache.doris.nereids.trees.plans.logical.UnboundLogicalSink;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalTableSink;
 import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.qe.QueryState.MysqlStateType;
@@ -54,6 +55,7 @@ import org.apache.commons.collections.CollectionUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
 import java.util.Optional;
@@ -88,7 +90,8 @@ public class InsertOverwriteTableCommand extends Command implements ForwardWithS
     }
 
     public boolean isAutoDetectOverwrite() {
-        return ((UnboundTableSink<?>) this.logicalQuery).isAutoDetectPartition();
+        return (logicalQuery instanceof UnboundTableSink)
+                && ((UnboundTableSink<?>) this.logicalQuery).isAutoDetectPartition();
     }
 
     @Override
@@ -118,27 +121,31 @@ public class InsertOverwriteTableCommand extends Command implements ForwardWithS
         }
 
         Optional<TreeNode<?>> plan = (planner.getPhysicalPlan()
-                .<Set<TreeNode<?>>>collect(node -> node instanceof PhysicalOlapTableSink)).stream().findAny();
+                .<Set<TreeNode<?>>>collect(node -> node instanceof PhysicalTableSink)).stream().findAny();
         Preconditions.checkArgument(plan.isPresent(), "insert into command must contain OlapTableSinkNode");
-        PhysicalOlapTableSink<?> physicalOlapTableSink = ((PhysicalOlapTableSink<?>) plan.get());
-        OlapTable targetTable = physicalOlapTableSink.getTargetTable();
-        InternalDatabaseUtil
-                .checkDatabase(targetTable.getQualifiedDbName(), ConnectContext.get());
-        // check auth
-        if (!Env.getCurrentEnv().getAccessManager()
-                .checkTblPriv(ConnectContext.get(), targetTable.getQualifiedDbName(), targetTable.getName(),
-                        PrivPredicate.LOAD)) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD",
-                    ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(),
-                    targetTable.getQualifiedDbName() + ": " + targetTable.getName());
-        }
-
-        ConnectContext.get().setSkipAuth(true);
-        List<String> partitionNames = ((UnboundTableSink<?>) logicalQuery).getPartitions();
-        if (CollectionUtils.isEmpty(partitionNames)) {
-            partitionNames = Lists.newArrayList(targetTable.getPartitionNames());
+        PhysicalTableSink<?> physicalTableSink = ((PhysicalTableSink<?>) plan.get());
+        TableIf targetTable = physicalTableSink.getTargetTable();
+        List<String> partitionNames;
+        if (physicalTableSink instanceof PhysicalOlapTableSink) {
+            InternalDatabaseUtil
+                    .checkDatabase(((OlapTable) targetTable).getQualifiedDbName(), ConnectContext.get());
+            // check auth
+            if (!Env.getCurrentEnv().getAccessManager()
+                    .checkTblPriv(ConnectContext.get(), ((OlapTable) targetTable).getQualifiedDbName(),
+                            targetTable.getName(), PrivPredicate.LOAD)) {
+                ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD",
+                        ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(),
+                        ((OlapTable) targetTable).getQualifiedDbName() + ": " + targetTable.getName());
+            }
+            ConnectContext.get().setSkipAuth(true);
+            partitionNames = ((UnboundTableSink<?>) logicalQuery).getPartitions();
+            if (CollectionUtils.isEmpty(partitionNames)) {
+                partitionNames = Lists.newArrayList(targetTable.getPartitionNames());
+            }
+        } else {
+            // Do not create temp partition on FE
+            partitionNames = new ArrayList<>();
         }
-
         long taskId = 0;
         try {
             if (isAutoDetectOverwrite()) {
@@ -170,6 +177,18 @@ public class InsertOverwriteTableCommand extends Command implements ForwardWithS
         }
     }
 
+    private void runInsertCommand(LogicalPlan logicalQuery, InsertCommandContext insertCtx,
+                                  ConnectContext ctx, StmtExecutor executor) throws Exception {
+        InsertIntoTableCommand insertCommand = new InsertIntoTableCommand(logicalQuery, labelName,
+                Optional.of(insertCtx));
+        insertCommand.run(ctx, executor);
+        if (ctx.getState().getStateType() == MysqlStateType.ERR) {
+            String errMsg = Strings.emptyToNull(ctx.getState().getErrorMessage());
+            LOG.warn("InsertInto state error:{}", errMsg);
+            throw new UserException(errMsg);
+        }
+    }
+
     /**
      * insert into select. for sepecified temp partitions
      *
@@ -208,18 +227,11 @@ public class InsertOverwriteTableCommand extends Command implements ForwardWithS
                     sink.getDMLCommandType(),
                     (LogicalPlan) (sink.child(0)));
             insertCtx = new HiveInsertCommandContext();
-            ((HiveInsertCommandContext) insertCtx).setOverwrite(false);
+            ((HiveInsertCommandContext) insertCtx).setOverwrite(true);
         } else {
-            throw new RuntimeException("Current catalog does not support insert overwrite yet.");
-        }
-        InsertIntoTableCommand insertCommand =
-                new InsertIntoTableCommand(copySink, labelName, Optional.of(insertCtx));
-        insertCommand.run(ctx, executor);
-        if (ctx.getState().getStateType() == MysqlStateType.ERR) {
-            String errMsg = Strings.emptyToNull(ctx.getState().getErrorMessage());
-            LOG.warn("InsertInto state error:{}", errMsg);
-            throw new UserException(errMsg);
+            throw new UserException("Current catalog does not support insert overwrite yet.");
         }
+        runInsertCommand(copySink, insertCtx, ctx, executor);
     }
 
     /**
@@ -229,17 +241,19 @@ public class InsertOverwriteTableCommand extends Command implements ForwardWithS
      * @param executor executor
      */
     private void insertInto(ConnectContext ctx, StmtExecutor executor, long groupId) throws Exception {
-        UnboundTableSink<?> sink = (UnboundTableSink<?>) logicalQuery;
         // 1. for overwrite situation, we disable auto create partition.
-        // 2. we save and pass overwrite auto detect by insertCtx
-        OlapInsertCommandContext insertCtx = new OlapInsertCommandContext(false, sink.isAutoDetectPartition(), groupId);
-        InsertIntoTableCommand insertCommand = new InsertIntoTableCommand(sink, labelName, Optional.of(insertCtx));
-        insertCommand.run(ctx, executor);
-        if (ctx.getState().getStateType() == MysqlStateType.ERR) {
-            String errMsg = Strings.emptyToNull(ctx.getState().getErrorMessage());
-            LOG.warn("InsertInto state error:{}", errMsg);
-            throw new UserException(errMsg);
+        // 2. we save and pass overwrite auto-detected by insertCtx
+        InsertCommandContext insertCtx;
+        if (logicalQuery instanceof UnboundTableSink) {
+            insertCtx = new OlapInsertCommandContext(false,
+                    ((UnboundTableSink<?>) logicalQuery).isAutoDetectPartition(), groupId);
+        } else if (logicalQuery instanceof UnboundHiveTableSink) {
+            insertCtx = new HiveInsertCommandContext();
+            ((HiveInsertCommandContext) insertCtx).setOverwrite(true);
+        } else {
+            throw new UserException("Current catalog does not support insert overwrite yet.");
         }
+        runInsertCommand(logicalQuery, insertCtx, ctx, executor);
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTableSink.java
index 7feb53e24b0..1461c0626d5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTableSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTableSink.java
@@ -17,6 +17,7 @@
 
 package org.apache.doris.nereids.trees.plans.physical;
 
+import org.apache.doris.catalog.TableIf;
 import org.apache.doris.nereids.memo.GroupExpression;
 import org.apache.doris.nereids.properties.LogicalProperties;
 import org.apache.doris.nereids.properties.PhysicalProperties;
@@ -44,4 +45,6 @@ public abstract class PhysicalTableSink<CHILD_TYPE extends Plan> extends Physica
     }
 
     public abstract PhysicalProperties getRequirePhysicalProperties();
+
+    public abstract TableIf getTargetTable();
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
new file mode 100644
index 00000000000..af57aae703b
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
@@ -0,0 +1,162 @@
+// 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.hive;
+
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.analysis.DistributionDesc;
+import org.apache.doris.analysis.DropDbStmt;
+import org.apache.doris.analysis.DropTableStmt;
+import org.apache.doris.analysis.HashDistributionDesc;
+import org.apache.doris.analysis.KeysDesc;
+import org.apache.doris.analysis.PartitionDesc;
+import org.apache.doris.analysis.TableName;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.Type;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.UserException;
+import org.apache.doris.datasource.DatabaseMetadata;
+import org.apache.doris.datasource.ExternalDatabase;
+import org.apache.doris.datasource.ExternalTable;
+import org.apache.doris.datasource.TableMetadata;
+
+import mockit.Mock;
+import mockit.MockUp;
+import mockit.Mocked;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class HiveMetadataOpsTest {
+
+    private HiveMetadataOps metadataOps;
+
+    @Mocked
+    private HMSCachedClient mockedClient;
+    @Mocked
+    private HMSExternalCatalog mockedCatalog;
+
+    @BeforeEach
+    public void init() {
+        metadataOps = new HiveMetadataOps(mockedCatalog, mockedClient);
+        new MockUp<HMSExternalCatalog>(HMSExternalCatalog.class) {
+            @Mock
+            public ExternalDatabase<? extends ExternalTable> getDbNullable(String dbName) {
+                return new HMSExternalDatabase(mockedCatalog, 0L, "mockedDb");
+            }
+
+            @Mock
+            public void onRefresh(boolean invalidCache) {
+                // mocked
+            }
+        };
+        new MockUp<HMSCachedClient>(HMSCachedClient.class) {
+            @Mock
+            public void createDatabase(DatabaseMetadata catalogDatabase) {
+                // mocked
+            }
+
+            @Mock
+            public void dropDatabase(String dbName) {
+                // mocked
+            }
+
+            @Mock
+            public void dropTable(String dbName, String tableName) {
+                // mocked
+            }
+
+            @Mock
+            public void createTable(TableMetadata catalogTable, boolean ignoreIfExists) {
+                // mocked
+            }
+        };
+    }
+
+    private void createDb(String dbName, Map<String, String> props) throws DdlException {
+        CreateDbStmt createDbStmt = new CreateDbStmt(true, dbName, props);
+        metadataOps.createDb(createDbStmt);
+    }
+
+    private void dropDb(String dbName, boolean forceDrop) throws DdlException {
+        DropDbStmt dropDbStmt = new DropDbStmt(true, dbName, forceDrop);
+        metadataOps.dropDb(dropDbStmt);
+    }
+
+    private void createTable(TableName tableName,
+                             List<Column> cols,
+                             List<String> parts,
+                             List<String> buckets,
+                             Map<String, String> props)
+            throws UserException {
+        PartitionDesc partitionDesc = new PartitionDesc(parts, null);
+        DistributionDesc distributionDesc = null;
+        if (!buckets.isEmpty()) {
+            distributionDesc = new HashDistributionDesc(10, buckets);
+        }
+        List<String> colsName = cols.stream().map(Column::getName).collect(Collectors.toList());
+        CreateTableStmt stmt = new CreateTableStmt(true, false,
+                tableName,
+                cols, null,
+                "hive",
+                new KeysDesc(KeysType.AGG_KEYS, colsName),
+                partitionDesc,
+                distributionDesc,
+                props,
+                props,
+                "comment",
+                null, null);
+        metadataOps.createTable(stmt);
+    }
+
+    private void dropTable(TableName tableName, boolean forceDrop) throws DdlException {
+        DropTableStmt dropTblStmt = new DropTableStmt(true, tableName, forceDrop);
+        metadataOps.dropTable(dropTblStmt);
+    }
+
+    @Test
+    public void testCreateAndDropAll() throws UserException {
+        Map<String, String> dbProps = new HashMap<>();
+        dbProps.put(HiveMetadataOps.LOCATION_URI_KEY, "file://loc/db");
+        createDb("mockedDb", dbProps);
+        Map<String, String> tblProps = new HashMap<>();
+        tblProps.put(HiveMetadataOps.FILE_FORMAT_KEY, "orc");
+        tblProps.put(HiveMetadataOps.LOCATION_URI_KEY, "file://loc/tbl");
+        tblProps.put("fs.defaultFS", "hdfs://ha");
+        TableName tableName = new TableName("mockedCtl", "mockedDb", "mockedTbl");
+        List<Column> cols = new ArrayList<>();
+        cols.add(new Column("id", Type.BIGINT));
+        cols.add(new Column("pt", Type.STRING));
+        cols.add(new Column("rate", Type.DOUBLE));
+        cols.add(new Column("time", Type.DATETIME));
+        List<String> parts = new ArrayList<>();
+        parts.add("pt");
+        List<String> bucks = new ArrayList<>();
+        // bucks.add("id");
+        createTable(tableName, cols, parts, bucks, tblProps);
+        dropTable(tableName, true);
+        dropDb("mockedDb", true);
+        // TODO: use TestWithFeService to double check plan
+    }
+}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
index 1cfb2270f92..3098d65e952 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
@@ -24,7 +24,6 @@ import org.apache.doris.thrift.THivePartitionUpdate;
 import org.apache.doris.thrift.TUpdateMode;
 
 import com.google.common.collect.Lists;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.junit.After;
@@ -97,8 +96,8 @@ public class HmsCommitTest {
         List<Column> columns = new ArrayList<>();
         columns.add(new Column("c1", PrimitiveType.INT, true));
         columns.add(new Column("c2", PrimitiveType.STRING, true));
-        List<FieldSchema> partitionKeys = new ArrayList<>();
-        partitionKeys.add(new FieldSchema("c3", "string", "comment"));
+        List<String> partitionKeys = new ArrayList<>();
+        partitionKeys.add("c3");
         HiveTableMetadata tableMetadata = new HiveTableMetadata(
                 dbName, tbWithPartition, columns, partitionKeys,
                 new HashMap<>(), fileFormat);


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


(doris) 07/19: [feature](txn insert) txn insert support insert into select (#31666)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 9ada38327b8ae204987c5e309c2c3528da5776ff
Author: meiyi <my...@gmail.com>
AuthorDate: Sun Mar 10 17:39:05 2024 +0800

    [feature](txn insert) txn insert support insert into select (#31666)
---
 .../doris/nereids/parser/LogicalPlanBuilder.java   |   6 +-
 .../commands/insert/AbstractInsertExecutor.java    |   2 +-
 .../commands/insert/InsertIntoTableCommand.java    |   4 +-
 .../trees/plans/commands/insert/InsertUtils.java   |   5 +
 .../plans/commands/insert/OlapInsertExecutor.java  |  30 ++-
 .../java/org/apache/doris/qe/ConnectContext.java   |  13 +-
 .../apache/doris/qe/InsertStreamTxnExecutor.java   |   3 +-
 .../java/org/apache/doris/qe/StmtExecutor.java     |  90 +++----
 .../apache/doris/transaction/TransactionEntry.java | 163 ++++++++++++-
 regression-test/data/insert_p0/txn_insert.out      | 259 ++++++++++++++++++++
 regression-test/suites/insert_p0/txn_insert.groovy | 263 +++++++++++++++------
 11 files changed, 683 insertions(+), 155 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
index 66d4eebecff..874b6a0d432 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
@@ -546,7 +546,11 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
         if (isOverwrite) {
             command = new InsertOverwriteTableCommand(sink, labelName);
         } else {
-            if (ConnectContext.get() != null && ConnectContext.get().isTxnModel()) {
+            if (ConnectContext.get() != null && ConnectContext.get().isTxnModel()
+                    && sink.child() instanceof LogicalInlineTable) {
+                // FIXME: In legacy, the `insert into select 1` is handled as `insert into values`.
+                //  In nereids, the original way is throw an AnalysisException and fallback to legacy.
+                //  Now handle it as `insert into select`(a separate load job), should fix it as the legacy.
                 command = new BatchInsertIntoTableCommand(sink);
             } else {
                 command = new InsertIntoTableCommand(sink, labelName, Optional.empty());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/AbstractInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/AbstractInsertExecutor.java
index 2ee55caf18d..f0df240762b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/AbstractInsertExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/AbstractInsertExecutor.java
@@ -49,7 +49,7 @@ public abstract class AbstractInsertExecutor {
     protected long jobId;
     protected final ConnectContext ctx;
     protected final Coordinator coordinator;
-    protected final String labelName;
+    protected String labelName;
     protected final DatabaseIf database;
     protected final TableIf table;
     protected final long createTime = System.currentTimeMillis();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java
index 56b99d8288d..8bf454f0980 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java
@@ -152,7 +152,9 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync,
             Preconditions.checkArgument(plan.isPresent(), "insert into command must contain target table");
             PhysicalSink physicalSink = plan.get();
             DataSink sink = planner.getFragments().get(0).getSink();
-            String label = this.labelName.orElse(String.format("label_%x_%x", ctx.queryId().hi, ctx.queryId().lo));
+            // Transaction insert should reuse the label in the transaction.
+            String label = this.labelName.orElse(
+                    ctx.isTxnModel() ? null : String.format("label_%x_%x", ctx.queryId().hi, ctx.queryId().lo));
 
             if (physicalSink instanceof PhysicalOlapTableSink) {
                 if (GroupCommitInserter.groupCommit(ctx, sink, physicalSink)) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java
index c1894c50eb3..f0e7fab736c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java
@@ -165,6 +165,11 @@ public class InsertUtils {
     private static void beginBatchInsertTransaction(ConnectContext ctx,
             String dbName, String tblName, List<Column> columns) {
         TransactionEntry txnEntry = ctx.getTxnEntry();
+        if (txnEntry.isTransactionBegan()) {
+            // FIXME: support mix usage of `insert into values` and `insert into select`
+            throw new AnalysisException(
+                    "Transaction insert can not insert into values and insert into select at the same time");
+        }
         TTxnParams txnConf = txnEntry.getTxnConf();
         SessionVariable sessionVariable = ctx.getSessionVariable();
         long timeoutSecond = ctx.getExecTimeout();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java
index bb61c71b282..289f9a974d6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java
@@ -47,6 +47,7 @@ import org.apache.doris.service.FrontendOptions;
 import org.apache.doris.thrift.TOlapTableLocationParam;
 import org.apache.doris.thrift.TPartitionType;
 import org.apache.doris.transaction.TabletCommitInfo;
+import org.apache.doris.transaction.TransactionEntry;
 import org.apache.doris.transaction.TransactionState;
 import org.apache.doris.transaction.TransactionState.LoadJobSourceType;
 import org.apache.doris.transaction.TransactionState.TxnCoordinator;
@@ -86,10 +87,22 @@ public class OlapInsertExecutor extends AbstractInsertExecutor {
     @Override
     public void beginTransaction() {
         try {
-            this.txnId = Env.getCurrentGlobalTransactionMgr().beginTransaction(
-                    database.getId(), ImmutableList.of(table.getId()), labelName,
-                    new TxnCoordinator(TxnSourceType.FE, FrontendOptions.getLocalHostAddress()),
-                    LoadJobSourceType.INSERT_STREAMING, ctx.getExecTimeout());
+            if (ctx.isTxnModel()) {
+                TransactionEntry txnEntry = ctx.getTxnEntry();
+                // check the same label with begin
+                if (this.labelName != null && !this.labelName.equals(txnEntry.getLabel())) {
+                    throw new AnalysisException("Transaction insert expect label " + txnEntry.getLabel()
+                            + ", but got " + this.labelName);
+                }
+                txnEntry.beginTransaction(database, table);
+                this.txnId = txnEntry.getTransactionId();
+                this.labelName = txnEntry.getLabel();
+            } else {
+                this.txnId = Env.getCurrentGlobalTransactionMgr().beginTransaction(
+                        database.getId(), ImmutableList.of(table.getId()), labelName,
+                        new TxnCoordinator(TxnSourceType.FE, FrontendOptions.getLocalHostAddress()),
+                        LoadJobSourceType.INSERT_STREAMING, ctx.getExecTimeout());
+            }
         } catch (Exception e) {
             throw new AnalysisException("begin transaction failed. " + e.getMessage(), e);
         }
@@ -160,6 +173,12 @@ public class OlapInsertExecutor extends AbstractInsertExecutor {
 
     @Override
     protected void onComplete() throws UserException {
+        if (ctx.isTxnModel()) {
+            TransactionEntry txnEntry = ctx.getTxnEntry();
+            txnEntry.addCommitInfos((Table) table, coordinator.getCommitInfos());
+            return;
+        }
+
         if (ctx.getState().getStateType() == MysqlStateType.ERR) {
             try {
                 String errMsg = Strings.emptyToNull(ctx.getState().getErrorMessage());
@@ -241,7 +260,8 @@ public class OlapInsertExecutor extends AbstractInsertExecutor {
         // {'label':'my_label1', 'status':'visible', 'txnId':'123'}
         // {'label':'my_label1', 'status':'visible', 'txnId':'123' 'err':'error messages'}
         StringBuilder sb = new StringBuilder();
-        sb.append("{'label':'").append(labelName).append("', 'status':'").append(txnStatus.name());
+        sb.append("{'label':'").append(labelName).append("', 'status':'")
+                .append(ctx.isTxnModel() ? TransactionStatus.PREPARE.name() : txnStatus.name());
         sb.append("', 'txnId':'").append(txnId).append("'");
         if (table.getType() == TableType.MATERIALIZED_VIEW) {
             sb.append("', 'rows':'").append(loadedRows).append("'");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java
index a36d4aeacc9..d561494fe52 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java
@@ -402,14 +402,11 @@ public class ConnectContext {
 
     public void closeTxn() {
         if (isTxnModel()) {
-            if (isTxnBegin()) {
-                try {
-                    InsertStreamTxnExecutor executor = new InsertStreamTxnExecutor(getTxnEntry());
-                    executor.abortTransaction();
-                } catch (Exception e) {
-                    LOG.error("db: {}, txnId: {}, rollback error.", currentDb,
-                            txnEntry.getTxnConf().getTxnId(), e);
-                }
+            try {
+                txnEntry.abortTransaction();
+            } catch (Exception e) {
+                LOG.error("db: {}, txnId: {}, rollback error.", currentDb,
+                        txnEntry.getTransactionId(), e);
             }
             txnEntry = null;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/InsertStreamTxnExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/InsertStreamTxnExecutor.java
index 77d4c52ac9d..f37457cf58d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/InsertStreamTxnExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/InsertStreamTxnExecutor.java
@@ -17,6 +17,7 @@
 
 package org.apache.doris.qe;
 
+import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.common.Config;
@@ -67,7 +68,7 @@ public class InsertStreamTxnExecutor {
         // StreamLoadTask's id == request's load_id
         StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
         StreamLoadPlanner planner = new StreamLoadPlanner(
-                txnEntry.getDb(), (OlapTable) txnEntry.getTable(), streamLoadTask);
+                (Database) txnEntry.getDb(), (OlapTable) txnEntry.getTable(), streamLoadTask);
         // Will using load id as query id in fragment
         if (Config.enable_pipeline_load) {
             TPipelineFragmentParams tRequest = planner.planForPipeline(streamLoadTask.getId());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
index 8e33c9cde85..a52ed1c260e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
@@ -175,8 +175,6 @@ import org.apache.doris.thrift.TStatusCode;
 import org.apache.doris.thrift.TStreamLoadPutRequest;
 import org.apache.doris.thrift.TTxnParams;
 import org.apache.doris.thrift.TUniqueId;
-import org.apache.doris.thrift.TWaitingTxnStatusRequest;
-import org.apache.doris.thrift.TWaitingTxnStatusResult;
 import org.apache.doris.transaction.TabletCommitInfo;
 import org.apache.doris.transaction.TransactionEntry;
 import org.apache.doris.transaction.TransactionState;
@@ -512,10 +510,19 @@ public class StmtExecutor {
                         LOG.warn("Analyze failed. {}", context.getQueryIdentifier(), e);
                         throw ((NereidsException) e).getException();
                     }
-                    boolean isInsertIntoCommand = parsedStmt != null && parsedStmt instanceof LogicalPlanAdapter
+                    // FIXME: Force fallback for:
+                    //  1. group commit because nereids does not support it (see the following `isGroupCommit` variable)
+                    //  2. insert into command because some nereids cases fail (including case1)
+                    //  Skip force fallback for:
+                    //  1. Transaction insert because nereids support `insert into select` while legacy does not
+                    boolean isInsertCommand = parsedStmt != null
+                            && parsedStmt instanceof LogicalPlanAdapter
                             && ((LogicalPlanAdapter) parsedStmt).getLogicalPlan() instanceof InsertIntoTableCommand;
-                    if (e instanceof NereidsException
-                            && !context.getSessionVariable().enableFallbackToOriginalPlanner && !isInsertIntoCommand) {
+                    /*boolean isGroupCommit = (Config.wait_internal_group_commit_finish
+                            || context.sessionVariable.isEnableInsertGroupCommit()) && isInsertCommand;*/
+                    boolean forceFallback = isInsertCommand && !context.isTxnModel();
+                    if (e instanceof NereidsException && !context.getSessionVariable().enableFallbackToOriginalPlanner
+                            && !forceFallback) {
                         LOG.warn("Analyze failed. {}", context.getQueryIdentifier(), e);
                         throw ((NereidsException) e).getException();
                     }
@@ -595,7 +602,8 @@ public class StmtExecutor {
         LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStmt).getLogicalPlan();
         // when we in transaction mode, we only support insert into command and transaction command
         if (context.isTxnModel()) {
-            if (!(logicalPlan instanceof BatchInsertIntoTableCommand)) {
+            if (!(logicalPlan instanceof BatchInsertIntoTableCommand
+                    || logicalPlan instanceof InsertIntoTableCommand)) {
                 String errMsg = "This is in a transaction, only insert, commit, rollback is acceptable.";
                 throw new NereidsException(errMsg, new AnalysisException(errMsg));
             }
@@ -1701,18 +1709,6 @@ public class StmtExecutor {
         }
     }
 
-    private TWaitingTxnStatusResult getWaitingTxnStatus(TWaitingTxnStatusRequest request) throws Exception {
-        TWaitingTxnStatusResult statusResult = null;
-        if (Env.getCurrentEnv().isMaster()) {
-            statusResult = Env.getCurrentGlobalTransactionMgr()
-                    .getWaitingTxnStatus(request);
-        } else {
-            MasterTxnExecutor masterTxnExecutor = new MasterTxnExecutor(context);
-            statusResult = masterTxnExecutor.getWaitingTxnStatus(request);
-        }
-        return statusResult;
-    }
-
     private void handleTransactionStmt() throws Exception {
         if (context.getConnectType() == ConnectType.MYSQL) {
             // Every time set no send flag and clean all data in buffer
@@ -1721,6 +1717,7 @@ public class StmtExecutor {
         context.getState().setOk(0, 0, "");
         // create plan
         if (context.getTxnEntry() != null && context.getTxnEntry().getRowsInTransaction() == 0
+                && !context.getTxnEntry().isTransactionBegan()
                 && (parsedStmt instanceof TransactionCommitStmt || parsedStmt instanceof TransactionRollbackStmt)) {
             context.setTxnEntry(null);
         } else if (parsedStmt instanceof TransactionBeginStmt) {
@@ -1728,18 +1725,13 @@ public class StmtExecutor {
                 LOG.info("A transaction has already begin");
                 return;
             }
-            TTxnParams txnParams = new TTxnParams();
-            txnParams.setNeedTxn(true).setEnablePipelineTxnLoad(Config.enable_pipeline_load)
-                    .setThriftRpcTimeoutMs(5000).setTxnId(-1).setDb("").setTbl("");
-            if (context.getSessionVariable().getEnableInsertStrict()) {
-                txnParams.setMaxFilterRatio(0);
-            } else {
-                txnParams.setMaxFilterRatio(1.0);
-            }
             if (context.getTxnEntry() == null) {
                 context.setTxnEntry(new TransactionEntry());
             }
-            context.getTxnEntry().setTxnConf(txnParams);
+            context.getTxnEntry()
+                    .setTxnConf(new TTxnParams().setNeedTxn(true).setEnablePipelineTxnLoad(Config.enable_pipeline_load)
+                            .setThriftRpcTimeoutMs(5000).setTxnId(-1).setDb("").setTbl("")
+                            .setMaxFilterRatio(context.getSessionVariable().getEnableInsertStrict() ? 0 : 1.0));
             StringBuilder sb = new StringBuilder();
             sb.append("{'label':'").append(context.getTxnEntry().getLabel()).append("', 'status':'")
                     .append(TransactionStatus.PREPARE.name());
@@ -1750,39 +1742,13 @@ public class StmtExecutor {
                 LOG.info("No transaction to commit");
                 return;
             }
-
-            TTxnParams txnConf = context.getTxnEntry().getTxnConf();
             try {
-                InsertStreamTxnExecutor executor = new InsertStreamTxnExecutor(context.getTxnEntry());
-                if (context.getTxnEntry().getDataToSend().size() > 0) {
-                    // send rest data
-                    executor.sendData();
-                }
-                // commit txn
-                executor.commitTransaction();
-
-                // wait txn visible
-                TWaitingTxnStatusRequest request = new TWaitingTxnStatusRequest();
-                request.setDbId(txnConf.getDbId()).setTxnId(txnConf.getTxnId());
-                request.setLabelIsSet(false);
-                request.setTxnIdIsSet(true);
-
-                TWaitingTxnStatusResult statusResult = getWaitingTxnStatus(request);
-                TransactionStatus txnStatus = TransactionStatus.valueOf(statusResult.getTxnStatusId());
-                if (txnStatus == TransactionStatus.COMMITTED) {
-                    throw new AnalysisException("transaction commit successfully, BUT data will be visible later.");
-                } else if (txnStatus != TransactionStatus.VISIBLE) {
-                    String errMsg = "commit failed, rollback.";
-                    if (statusResult.getStatus().isSetErrorMsgs()
-                            && statusResult.getStatus().getErrorMsgs().size() > 0) {
-                        errMsg = String.join(". ", statusResult.getStatus().getErrorMsgs());
-                    }
-                    throw new AnalysisException(errMsg);
-                }
+                TransactionEntry txnEntry = context.getTxnEntry();
+                TransactionStatus txnStatus = txnEntry.commitTransaction();
                 StringBuilder sb = new StringBuilder();
-                sb.append("{'label':'").append(context.getTxnEntry().getLabel()).append("', 'status':'")
+                sb.append("{'label':'").append(txnEntry.getLabel()).append("', 'status':'")
                         .append(txnStatus.name()).append("', 'txnId':'")
-                        .append(context.getTxnEntry().getTxnConf().getTxnId()).append("'").append("}");
+                        .append(txnEntry.getTransactionId()).append("'").append("}");
                 context.getState().setOk(0, 0, sb.toString());
             } catch (Exception e) {
                 LOG.warn("Txn commit failed", e);
@@ -1796,14 +1762,12 @@ public class StmtExecutor {
                 return;
             }
             try {
-                // abort txn
-                InsertStreamTxnExecutor executor = new InsertStreamTxnExecutor(context.getTxnEntry());
-                executor.abortTransaction();
-
+                TransactionEntry txnEntry = context.getTxnEntry();
+                long txnId = txnEntry.abortTransaction();
                 StringBuilder sb = new StringBuilder();
-                sb.append("{'label':'").append(context.getTxnEntry().getLabel()).append("', 'status':'")
+                sb.append("{'label':'").append(txnEntry.getLabel()).append("', 'status':'")
                         .append(TransactionStatus.ABORTED.name()).append("', 'txnId':'")
-                        .append(context.getTxnEntry().getTxnConf().getTxnId()).append("'").append("}");
+                        .append(txnId).append("'").append("}");
                 context.getState().setOk(0, 0, sb.toString());
             } catch (Exception e) {
                 throw new AnalysisException(e.getMessage());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java
index 71368715794..43e6585e584 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java
@@ -18,19 +18,48 @@
 package org.apache.doris.transaction;
 
 import org.apache.doris.catalog.Database;
+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.DdlException;
+import org.apache.doris.common.MetaNotFoundException;
+import org.apache.doris.common.QuotaExceedException;
+import org.apache.doris.common.UserException;
 import org.apache.doris.proto.InternalService;
 import org.apache.doris.proto.Types;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.InsertStreamTxnExecutor;
+import org.apache.doris.qe.MasterTxnExecutor;
+import org.apache.doris.service.FrontendOptions;
 import org.apache.doris.system.Backend;
+import org.apache.doris.thrift.TTabletCommitInfo;
 import org.apache.doris.thrift.TTxnParams;
+import org.apache.doris.thrift.TWaitingTxnStatusRequest;
+import org.apache.doris.thrift.TWaitingTxnStatusResult;
+import org.apache.doris.transaction.TransactionState.LoadJobSourceType;
+import org.apache.doris.transaction.TransactionState.TxnCoordinator;
+import org.apache.doris.transaction.TransactionState.TxnSourceType;
+
+import com.google.common.collect.Lists;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.apache.thrift.TException;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
 
 public class TransactionEntry {
 
+    private static final Logger LOG = LogManager.getLogger(TransactionEntry.class);
+
     private String label = "";
-    private Database db;
+    private DatabaseIf database;
+
+    // for insert into values for one table
     private Table table;
     private Backend backend;
     private TTxnParams txnConf;
@@ -38,12 +67,19 @@ public class TransactionEntry {
     private long rowsInTransaction = 0;
     private Types.PUniqueId pLoadId;
 
+    // for insert into select for multi tables
+    private boolean isTransactionBegan = false;
+    private long transactionId = -1;
+    private TransactionState transactionState;
+    private List<Table> tableList = new ArrayList<>();
+    private List<TTabletCommitInfo> tabletCommitInfos = new ArrayList<>();
+
     public TransactionEntry() {
     }
 
     public TransactionEntry(TTxnParams txnConf, Database db, Table table) {
         this.txnConf = txnConf;
-        this.db = db;
+        this.database = db;
         this.table = table;
     }
 
@@ -55,12 +91,12 @@ public class TransactionEntry {
         this.label = label;
     }
 
-    public Database getDb() {
-        return db;
+    public DatabaseIf getDb() {
+        return database;
     }
 
     public void setDb(Database db) {
-        this.db = db;
+        this.database = db;
     }
 
     public Table getTable() {
@@ -126,4 +162,121 @@ public class TransactionEntry {
     public void setpLoadId(Types.PUniqueId pLoadId) {
         this.pLoadId = pLoadId;
     }
+
+    // Used for insert into select
+    public void beginTransaction(DatabaseIf database, TableIf table)
+            throws DdlException, BeginTransactionException, MetaNotFoundException, AnalysisException,
+            QuotaExceedException {
+        if (isTxnBegin()) {
+            // FIXME: support mix usage of `insert into values` and `insert into select`
+            throw new AnalysisException(
+                    "Transaction insert can not insert into values and insert into select at the same time");
+        }
+        if (!isTransactionBegan) {
+            this.transactionId = Env.getCurrentGlobalTransactionMgr().beginTransaction(
+                    database.getId(), Lists.newArrayList(table.getId()), label,
+                    new TxnCoordinator(TxnSourceType.FE, FrontendOptions.getLocalHostAddress()),
+                    LoadJobSourceType.INSERT_STREAMING, ConnectContext.get().getExecTimeout());
+            this.isTransactionBegan = true;
+            this.database = database;
+            this.transactionState = Env.getCurrentGlobalTransactionMgr()
+                    .getTransactionState(database.getId(), transactionId);
+        } else {
+            if (this.database.getId() != database.getId()) {
+                throw new AnalysisException(
+                        "Transaction insert must be in the same database, expect db_id=" + this.database.getId());
+            }
+            this.transactionState.getTableIdList().add(table.getId());
+        }
+    }
+
+    public TransactionStatus commitTransaction()
+            throws Exception {
+        if (isTransactionBegan) {
+            if (Env.getCurrentGlobalTransactionMgr()
+                    .commitAndPublishTransaction(database, tableList, transactionId,
+                            TabletCommitInfo.fromThrift(tabletCommitInfos), ConnectContext.get().getExecTimeout())) {
+                return TransactionStatus.VISIBLE;
+            } else {
+                return TransactionStatus.COMMITTED;
+            }
+        } else if (isTxnBegin()) {
+            InsertStreamTxnExecutor executor = new InsertStreamTxnExecutor(this);
+            if (dataToSend.size() > 0) {
+                // send rest data
+                executor.sendData();
+            }
+            // commit txn
+            executor.commitTransaction();
+
+            // wait txn visible
+            TWaitingTxnStatusRequest request = new TWaitingTxnStatusRequest();
+            request.setDbId(txnConf.getDbId()).setTxnId(txnConf.getTxnId());
+            request.setLabelIsSet(false);
+            request.setTxnIdIsSet(true);
+
+            TWaitingTxnStatusResult statusResult = getWaitingTxnStatus(request);
+            TransactionStatus txnStatus = TransactionStatus.valueOf(statusResult.getTxnStatusId());
+            if (txnStatus == TransactionStatus.COMMITTED) {
+                throw new AnalysisException("transaction commit successfully, BUT data will be visible later.");
+            } else if (txnStatus != TransactionStatus.VISIBLE) {
+                String errMsg = "commit failed, rollback.";
+                if (statusResult.getStatus().isSetErrorMsgs()
+                        && statusResult.getStatus().getErrorMsgs().size() > 0) {
+                    errMsg = String.join(". ", statusResult.getStatus().getErrorMsgs());
+                }
+                throw new AnalysisException(errMsg);
+            }
+            return txnStatus;
+        } else {
+            LOG.info("No transaction to commit");
+            return null;
+        }
+    }
+
+    private TWaitingTxnStatusResult getWaitingTxnStatus(TWaitingTxnStatusRequest request) throws Exception {
+        TWaitingTxnStatusResult statusResult = null;
+        if (Env.getCurrentEnv().isMaster()) {
+            statusResult = Env.getCurrentGlobalTransactionMgr()
+                    .getWaitingTxnStatus(request);
+        } else {
+            MasterTxnExecutor masterTxnExecutor = new MasterTxnExecutor(ConnectContext.get());
+            statusResult = masterTxnExecutor.getWaitingTxnStatus(request);
+        }
+        return statusResult;
+    }
+
+    public long abortTransaction()
+            throws UserException, TException, ExecutionException, InterruptedException, TimeoutException {
+        if (isTransactionBegan) {
+            Env.getCurrentGlobalTransactionMgr().abortTransaction(database.getId(), transactionId, "user rollback");
+            return transactionId;
+        } else if (isTxnBegin()) {
+            InsertStreamTxnExecutor executor = new InsertStreamTxnExecutor(this);
+            executor.abortTransaction();
+            return txnConf.getTxnId();
+        } else {
+            LOG.info("No transaction to abort");
+            return -1;
+        }
+    }
+
+    public long getTransactionId() {
+        if (isTransactionBegan) {
+            return transactionId;
+        } else if (isTxnBegin()) {
+            return txnConf.getTxnId();
+        } else {
+            return -1;
+        }
+    }
+
+    public void addCommitInfos(Table table, List<TTabletCommitInfo> commitInfos) {
+        this.tableList.add(table);
+        this.tabletCommitInfos.addAll(commitInfos);
+    }
+
+    public boolean isTransactionBegan() {
+        return this.isTransactionBegan;
+    }
 }
diff --git a/regression-test/data/insert_p0/txn_insert.out b/regression-test/data/insert_p0/txn_insert.out
index f4947a2ac65..f6f9a5a648e 100644
--- a/regression-test/data/insert_p0/txn_insert.out
+++ b/regression-test/data/insert_p0/txn_insert.out
@@ -39,3 +39,262 @@
 6
 8
 
+-- !select7 --
+
+-- !select8 --
+
+-- !select9 --
+
+-- !select1 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select2 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select3 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select4 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select5 --
+1	2
+3	4
+5	6
+7	8
+
+-- !select6 --
+2
+4
+6
+8
+
+-- !select7 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select8 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select9 --
+
+-- !select10 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select11 --
+\N	\N	\N	[null]	[null, 0]
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select12 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select13 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select14 --
+\N	\N	\N	[null]	[null, 0]
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select15 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select16 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select17 --
+\N	\N	\N	[null]	[null, 0]
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+100	2.2	abc	[]	[]
+101	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select18 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select19 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select20 --
+\N	\N	\N	[null]	[null, 0]
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+100	2.2	abc	[]	[]
+101	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select21 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select22 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select23 --
+\N	\N	\N	[null]	[null, 0]
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+100	2.2	abc	[]	[]
+101	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
+-- !select24 --
+\N	\N	\N	[null]	[null, 0]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+1	2.2	abc	[]	[]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+2	3.3	xyz	[1]	[1, 0]
+
diff --git a/regression-test/suites/insert_p0/txn_insert.groovy b/regression-test/suites/insert_p0/txn_insert.groovy
index 6e745073b1b..ef3d46141a7 100644
--- a/regression-test/suites/insert_p0/txn_insert.groovy
+++ b/regression-test/suites/insert_p0/txn_insert.groovy
@@ -21,74 +21,197 @@
 
 suite("txn_insert") {
     def table = "txn_insert_tbl"
-    sql """ DROP TABLE IF EXISTS $table """
-    sql """
-        create table $table (
-            k1 int, 
-            k2 double,
-            k3 varchar(100),
-            k4 array<int>,
-            k5 array<boolean>
-        ) distributed by hash(k1) buckets 1
-        properties("replication_num" = "1"); 
-    """
-
-    // begin and commit
-    sql """begin"""
-    sql """insert into $table values(1, 2.2, "abc", [], [])"""
-    sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
-    sql """insert into $table values(null, null, null, [null], [null, 0])"""
-    sql "commit"
-    sql "sync"
-    order_qt_select1 """select * from $table"""
-
-    // begin and rollback
-    sql "begin"
-    sql """insert into $table values(1, 2.2, "abc", [], [])"""
-    sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
-    sql "rollback"
-    sql "sync"
-    order_qt_select2 """select * from $table"""
-
-    // begin 2 times and commit
-    sql "begin"
-    sql """insert into $table values(1, 2.2, "abc", [], [])"""
-    sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
-    sql "begin"
-    sql """insert into $table values(1, 2.2, "abc", [], [])"""
-    sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
-    sql "commit"
-    sql "sync"
-    order_qt_select3 """select * from $table"""
-
-    // begin 2 times and rollback
-    sql "begin"
-    sql """insert into $table values(1, 2.2, "abc", [], [])"""
-    sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
-    sql "begin"
-    sql """insert into $table values(1, 2.2, "abc", [], [])"""
-    sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
-    sql "rollback"
-    sql "sync"
-    order_qt_select4 """select * from $table"""
-
-    // write to table with mv
-    table = table + "_mv"
-    sql """ DROP TABLE IF EXISTS $table """
-    sql """
-        create table $table (
-            id int default '10', 
-            c1 int default '10'
-        ) distributed by hash(id, c1) 
-        properties('replication_num'="1");
-    """
-    createMV """ create materialized view mv_${table} as select c1 from $table; """
-    sql "begin"
-    sql """insert into $table values(1, 2), (3, 4)"""
-    sql """insert into $table values(5, 6)"""
-    sql """insert into $table values(7, 8)"""
-    sql "commit"
-    sql "sync"
-    order_qt_select5 """select * from $table"""
-    order_qt_select6 """select c1 from $table"""
+    for (def use_nereids_planner : [false, true]) {
+        sql " SET enable_nereids_planner = $use_nereids_planner; "
+        sql " SET enable_fallback_to_original_planner = false; "
+
+        sql """ DROP TABLE IF EXISTS $table """
+        sql """
+            create table $table (
+                k1 int, 
+                k2 double,
+                k3 varchar(100),
+                k4 array<int>,
+                k5 array<boolean>
+            ) distributed by hash(k1) buckets 1
+            properties("replication_num" = "1"); 
+        """
+
+        // begin and commit
+        sql """begin"""
+        sql """insert into $table values(1, 2.2, "abc", [], [])"""
+        sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
+        sql """insert into $table values(null, null, null, [null], [null, 0])"""
+        sql "commit"
+        sql "sync"
+        order_qt_select1 """select * from $table"""
+
+        // begin and rollback
+        sql "begin"
+        sql """insert into $table values(1, 2.2, "abc", [], [])"""
+        sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
+        sql "rollback"
+        sql "sync"
+        order_qt_select2 """select * from $table"""
+
+        // begin 2 times and commit
+        sql "begin"
+        sql """insert into $table values(1, 2.2, "abc", [], [])"""
+        sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
+        sql "begin"
+        sql """insert into $table values(1, 2.2, "abc", [], [])"""
+        sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
+        sql "commit"
+        sql "sync"
+        order_qt_select3 """select * from $table"""
+
+        // begin 2 times and rollback
+        sql "begin"
+        sql """insert into $table values(1, 2.2, "abc", [], [])"""
+        sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
+        sql "begin"
+        sql """insert into $table values(1, 2.2, "abc", [], [])"""
+        sql """insert into $table values(2, 3.3, "xyz", [1], [1, 0])"""
+        sql "rollback"
+        sql "sync"
+        order_qt_select4 """select * from $table"""
+
+        // write to table with mv
+        def tableMV = table + "_mv"
+        do {
+            sql """ DROP TABLE IF EXISTS $tableMV """
+            sql """
+                create table $tableMV (
+                    id int default '10', 
+                    c1 int default '10'
+                ) distributed by hash(id, c1) 
+                properties('replication_num'="1");
+            """
+            createMV """ create materialized view mv_${tableMV} as select c1 from $tableMV; """
+            sql "begin"
+            sql """insert into $tableMV values(1, 2), (3, 4)"""
+            sql """insert into $tableMV values(5, 6)"""
+            sql """insert into $tableMV values(7, 8)"""
+            sql "commit"
+            sql "sync"
+            order_qt_select5 """select * from $tableMV"""
+            order_qt_select6 """select c1 from $tableMV"""
+        } while (0);
+
+        // ------------------- insert into select -------------------
+        for (int j = 0; j < 3; j++) {
+            def tableName = table + "_" + j
+            sql """ DROP TABLE IF EXISTS $tableName """
+            sql """
+                create table $tableName (
+                    k1 int, 
+                    k2 double,
+                    k3 varchar(100),
+                    k4 array<int>,
+                    k5 array<boolean>
+                ) distributed by hash(k1) buckets 1
+                properties("replication_num" = "1"); 
+            """
+        }
+
+        def result = sql """ show variables like 'enable_fallback_to_original_planner'; """
+        logger.info("enable_fallback_to_original_planner: $result")
+
+        // 1. insert into select to 3 tables: batch insert into select only supports nereids planner, and can't fallback
+        sql """ begin; """
+        if (use_nereids_planner) {
+            sql """ insert into ${table}_0 select * from $table; """
+            sql """ insert into ${table}_1 select * from $table; """
+            sql """ insert into ${table}_2 select * from ${table}_0; """
+        } else {
+            test {
+                sql """ insert into ${table}_0 select * from $table; """
+                exception "Insert into ** select is not supported in a transaction"
+            }
+        }
+        sql """ commit; """
+        sql "sync"
+        order_qt_select7 """select * from ${table}_0"""
+        order_qt_select8 """select * from ${table}_1"""
+        order_qt_select9 """select * from ${table}_2"""
+
+        // 2. with different label
+        if (use_nereids_planner) {
+            def label = UUID.randomUUID().toString().replaceAll("-", "")
+            def label2 = UUID.randomUUID().toString().replaceAll("-", "")
+            sql """ begin with label $label; """
+            test {
+                sql """ insert into ${table}_0 with label $label2 select * from $table; """
+                exception "Transaction insert expect label"
+            }
+            sql """ insert into ${table}_1 select * from $table; """
+            sql """ insert into ${table}_2 select * from ${table}_0; """
+            sql """ commit; """
+            sql "sync"
+            order_qt_select10 """select * from ${table}_0"""
+            order_qt_select11 """select * from ${table}_1"""
+            order_qt_select12 """select * from ${table}_2"""
+        }
+
+        // 3. insert into select and values
+        if (use_nereids_planner) {
+            sql """ begin; """
+            sql """ insert into ${table}_0 select * from $table where k1 = 1; """
+            test {
+                sql """insert into ${table}_1 values(1, 2.2, "abc", [], [])"""
+                exception "Transaction insert can not insert into values and insert into select at the same time"
+            }
+            sql """ insert into ${table}_1 select * from $table where k2 = 2.2 limit 1; """
+            sql """ commit; """
+            sql "sync"
+            order_qt_select13 """select * from ${table}_0"""
+            order_qt_select14 """select * from ${table}_1"""
+            order_qt_select15 """select * from ${table}_2"""
+        }
+
+        // 4. insert into values and select
+        if (use_nereids_planner) {
+            sql """ begin; """
+            sql """insert into ${table}_1 values(100, 2.2, "abc", [], [])"""
+            test {
+                sql """ insert into ${table}_0 select * from $table; """
+                exception "Transaction insert can not insert into values and insert into select at the same time"
+            }
+            sql """insert into ${table}_1 values(101, 2.2, "abc", [], [])"""
+            sql """ commit; """
+            sql "sync"
+            order_qt_select16 """select * from ${table}_0"""
+            order_qt_select17 """select * from ${table}_1"""
+            order_qt_select18 """select * from ${table}_2"""
+        }
+
+        // 5. rollback
+        if (use_nereids_planner) {
+            def label = UUID.randomUUID().toString().replaceAll("-", "")
+            sql """ begin with label $label; """
+            sql """ insert into ${table}_0 select * from $table where k1 = 1; """
+            sql """ insert into ${table}_1 select * from $table where k2 = 2.2 limit 1; """
+            sql """ rollback; """
+            logger.info("rollback $label")
+            sql "sync"
+            order_qt_select19 """select * from ${table}_0"""
+            order_qt_select20 """select * from ${table}_1"""
+            order_qt_select21 """select * from ${table}_2"""
+        }
+
+        // 6. insert select with error
+        if (use_nereids_planner) {
+            sql """ begin; """
+            test {
+                sql """ insert into ${table}_0 select * from $tableMV; """
+                exception "insert into cols should be corresponding to the query output"
+            }
+            sql """ insert into ${table}_1 select * from $table where k2 = 2.2 limit 1; """
+            sql """ commit; """
+            sql "sync"
+            order_qt_select22 """select * from ${table}_0"""
+            order_qt_select23 """select * from ${table}_1"""
+            order_qt_select24 """select * from ${table}_2"""
+        }
+    }
 }


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


(doris) 11/19: [Enhancement](hive-writer) Write only regular fields to file in the hive-writer. (#33000)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 3c9c6c18a80f42dc5d2506132660eaee96448aeb
Author: Qi Chen <ka...@gmail.com>
AuthorDate: Sun Mar 31 21:54:07 2024 +0800

    [Enhancement](hive-writer) Write only regular fields to file in the hive-writer. (#33000)
---
 be/src/vec/runtime/vorc_transformer.cpp           | 104 +++++++++++++++++--
 be/src/vec/runtime/vorc_transformer.h             |  10 +-
 be/src/vec/runtime/vparquet_transformer.cpp       |  32 +++++-
 be/src/vec/runtime/vparquet_transformer.h         |  10 +-
 be/src/vec/sink/writer/vhive_partition_writer.cpp | 118 ++++------------------
 be/src/vec/sink/writer/vhive_partition_writer.h   |   6 +-
 be/src/vec/sink/writer/vhive_table_writer.cpp     |  22 +++-
 be/src/vec/sink/writer/vhive_table_writer.h       |   3 +
 8 files changed, 182 insertions(+), 123 deletions(-)

diff --git a/be/src/vec/runtime/vorc_transformer.cpp b/be/src/vec/runtime/vorc_transformer.cpp
index 764a97ae5bc..a07f734acac 100644
--- a/be/src/vec/runtime/vorc_transformer.cpp
+++ b/be/src/vec/runtime/vorc_transformer.cpp
@@ -99,35 +99,45 @@ VOrcTransformer::VOrcTransformer(RuntimeState* state, doris::io::FileWriter* fil
         : VFileFormatTransformer(state, output_vexpr_ctxs, output_object_data),
           _file_writer(file_writer),
           _write_options(new orc::WriterOptions()),
-          _schema_str(&schema),
-          _schema(nullptr) {
+          _schema_str(&schema) {
     _write_options->setTimezoneName(_state->timezone());
     _write_options->setUseTightNumericVector(true);
 }
 
 VOrcTransformer::VOrcTransformer(RuntimeState* state, doris::io::FileWriter* file_writer,
                                  const VExprContextSPtrs& output_vexpr_ctxs,
-                                 std::unique_ptr<orc::Type> schema, bool output_object_data,
+                                 std::vector<std::string> column_names, bool output_object_data,
                                  orc::CompressionKind compression)
         : VFileFormatTransformer(state, output_vexpr_ctxs, output_object_data),
           _file_writer(file_writer),
+          _column_names(std::move(column_names)),
           _write_options(new orc::WriterOptions()),
-          _schema_str(nullptr),
-          _schema(std::move(schema)) {
+          _schema_str(nullptr) {
     _write_options->setTimezoneName(_state->timezone());
     _write_options->setUseTightNumericVector(true);
     _write_options->setCompression(compression);
 }
 
 Status VOrcTransformer::open() {
-    try {
-        if (_schema == nullptr && _schema_str != nullptr) {
+    if (_schema_str != nullptr) {
+        try {
             _schema = orc::Type::buildTypeFromString(*_schema_str);
+        } catch (const std::exception& e) {
+            return Status::InternalError("Orc build schema from \"{}\" failed: {}", *_schema_str,
+                                         e.what());
+        }
+    } else {
+        _schema = orc::createStructType();
+        for (int i = 0; i < _output_vexpr_ctxs.size(); i++) {
+            VExprSPtr column_expr = _output_vexpr_ctxs[i]->root();
+            try {
+                _schema->addStructField(_column_names[i], _build_orc_type(column_expr->type()));
+            } catch (doris::Exception& e) {
+                return e.to_status();
+            }
         }
-    } catch (const std::exception& e) {
-        return Status::InternalError("Orc build schema from \"{}\" failed: {}", *_schema_str,
-                                     e.what());
     }
+
     _output_stream = std::make_unique<VOrcOutputStream>(_file_writer);
     _writer = orc::createWriter(*_schema, _output_stream.get(), *_write_options);
     if (_writer == nullptr) {
@@ -136,6 +146,80 @@ Status VOrcTransformer::open() {
     return Status::OK();
 }
 
+std::unique_ptr<orc::Type> VOrcTransformer::_build_orc_type(const TypeDescriptor& type_descriptor) {
+    std::pair<Status, std::unique_ptr<orc::Type>> result;
+    switch (type_descriptor.type) {
+    case TYPE_BOOLEAN: {
+        return orc::createPrimitiveType(orc::BOOLEAN);
+    }
+    case TYPE_TINYINT: {
+        return orc::createPrimitiveType(orc::BYTE);
+    }
+    case TYPE_SMALLINT: {
+        return orc::createPrimitiveType(orc::SHORT);
+    }
+    case TYPE_INT: {
+        return orc::createPrimitiveType(orc::INT);
+    }
+    case TYPE_BIGINT: {
+        return orc::createPrimitiveType(orc::LONG);
+    }
+    case TYPE_FLOAT: {
+        return orc::createPrimitiveType(orc::FLOAT);
+    }
+    case TYPE_DOUBLE: {
+        return orc::createPrimitiveType(orc::DOUBLE);
+    }
+    case TYPE_CHAR: {
+        return orc::createCharType(orc::CHAR, type_descriptor.len);
+    }
+    case TYPE_VARCHAR: {
+        return orc::createCharType(orc::VARCHAR, type_descriptor.len);
+    }
+    case TYPE_STRING: {
+        return orc::createPrimitiveType(orc::STRING);
+    }
+    case TYPE_BINARY: {
+        return orc::createPrimitiveType(orc::STRING);
+    }
+    case TYPE_DATEV2: {
+        return orc::createPrimitiveType(orc::DATE);
+    }
+    case TYPE_DATETIMEV2: {
+        return orc::createPrimitiveType(orc::TIMESTAMP);
+    }
+    case TYPE_DECIMAL32: {
+        return orc::createDecimalType(type_descriptor.precision, type_descriptor.scale);
+    }
+    case TYPE_DECIMAL64: {
+        return orc::createDecimalType(type_descriptor.precision, type_descriptor.scale);
+    }
+    case TYPE_DECIMAL128I: {
+        return orc::createDecimalType(type_descriptor.precision, type_descriptor.scale);
+    }
+    case TYPE_STRUCT: {
+        std::unique_ptr<orc::Type> struct_type = orc::createStructType();
+        for (int j = 0; j < type_descriptor.children.size(); ++j) {
+            struct_type->addStructField(type_descriptor.field_names[j],
+                                        _build_orc_type(type_descriptor.children[j]));
+        }
+        return struct_type;
+    }
+    case TYPE_ARRAY: {
+        return orc::createListType(_build_orc_type(type_descriptor.children[0]));
+    }
+    case TYPE_MAP: {
+        return orc::createMapType(_build_orc_type(type_descriptor.children[0]),
+                                  _build_orc_type(type_descriptor.children[1]));
+    }
+    default: {
+        throw doris::Exception(doris::ErrorCode::INTERNAL_ERROR,
+                               "Unsupported type {} to build orc type",
+                               type_descriptor.debug_string());
+    }
+    }
+}
+
 std::unique_ptr<orc::ColumnVectorBatch> VOrcTransformer::_create_row_batch(size_t sz) {
     return _writer->createRowBatch(sz);
 }
diff --git a/be/src/vec/runtime/vorc_transformer.h b/be/src/vec/runtime/vorc_transformer.h
index 8cfc956c0cd..1f78e549a1b 100644
--- a/be/src/vec/runtime/vorc_transformer.h
+++ b/be/src/vec/runtime/vorc_transformer.h
@@ -79,8 +79,9 @@ public:
                     bool output_object_data);
 
     VOrcTransformer(RuntimeState* state, doris::io::FileWriter* file_writer,
-                    const VExprContextSPtrs& output_vexpr_ctxs, std::unique_ptr<orc::Type> schema,
-                    bool output_object_data, orc::CompressionKind compression);
+                    const VExprContextSPtrs& output_vexpr_ctxs,
+                    std::vector<std::string> column_names, bool output_object_data,
+                    orc::CompressionKind compression);
 
     ~VOrcTransformer() = default;
 
@@ -93,6 +94,8 @@ public:
     int64_t written_len() override;
 
 private:
+    std::unique_ptr<orc::Type> _build_orc_type(const TypeDescriptor& type_descriptor);
+
     std::unique_ptr<orc::ColumnVectorBatch> _create_row_batch(size_t sz);
     // The size of subtypes of a complex type may be different from
     // the size of the complex type itself,
@@ -101,6 +104,7 @@ private:
                              orc::ColumnVectorBatch* orc_col_batch);
 
     doris::io::FileWriter* _file_writer = nullptr;
+    std::vector<std::string> _column_names;
     std::unique_ptr<orc::OutputStream> _output_stream;
     std::unique_ptr<orc::WriterOptions> _write_options;
     const std::string* _schema_str;
@@ -117,4 +121,4 @@ private:
     static constexpr size_t BUFFER_RESERVED_SIZE = 40;
 };
 
-} // namespace doris::vectorized
\ No newline at end of file
+} // namespace doris::vectorized
diff --git a/be/src/vec/runtime/vparquet_transformer.cpp b/be/src/vec/runtime/vparquet_transformer.cpp
index b831b1e2dc5..0e5800750b0 100644
--- a/be/src/vec/runtime/vparquet_transformer.cpp
+++ b/be/src/vec/runtime/vparquet_transformer.cpp
@@ -197,6 +197,22 @@ void ParquetBuildHelper::build_version(parquet::WriterProperties::Builder& build
     }
 }
 
+VParquetTransformer::VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer,
+                                         const VExprContextSPtrs& output_vexpr_ctxs,
+                                         std::vector<std::string> column_names,
+                                         TParquetCompressionType::type compression_type,
+                                         bool parquet_disable_dictionary,
+                                         TParquetVersion::type parquet_version,
+                                         bool output_object_data)
+        : VFileFormatTransformer(state, output_vexpr_ctxs, output_object_data),
+          _column_names(std::move(column_names)),
+          _parquet_schemas(nullptr),
+          _compression_type(compression_type),
+          _parquet_disable_dictionary(parquet_disable_dictionary),
+          _parquet_version(parquet_version) {
+    _outstream = std::shared_ptr<ParquetOutputStream>(new ParquetOutputStream(file_writer));
+}
+
 VParquetTransformer::VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer,
                                          const VExprContextSPtrs& output_vexpr_ctxs,
                                          const std::vector<TParquetSchema>& parquet_schemas,
@@ -205,7 +221,7 @@ VParquetTransformer::VParquetTransformer(RuntimeState* state, doris::io::FileWri
                                          TParquetVersion::type parquet_version,
                                          bool output_object_data)
         : VFileFormatTransformer(state, output_vexpr_ctxs, output_object_data),
-          _parquet_schemas(parquet_schemas),
+          _parquet_schemas(&parquet_schemas),
           _compression_type(compression_type),
           _parquet_disable_dictionary(parquet_disable_dictionary),
           _parquet_version(parquet_version) {
@@ -238,10 +254,16 @@ Status VParquetTransformer::_parse_schema() {
     for (size_t i = 0; i < _output_vexpr_ctxs.size(); i++) {
         std::shared_ptr<arrow::DataType> type;
         RETURN_IF_ERROR(convert_to_arrow_type(_output_vexpr_ctxs[i]->root()->type(), &type));
-        std::shared_ptr<arrow::Field> field =
-                arrow::field(_parquet_schemas[i].schema_column_name, type,
-                             _output_vexpr_ctxs[i]->root()->is_nullable());
-        fields.emplace_back(field);
+        if (_parquet_schemas != nullptr) {
+            std::shared_ptr<arrow::Field> field =
+                    arrow::field(_parquet_schemas->operator[](i).schema_column_name, type,
+                                 _output_vexpr_ctxs[i]->root()->is_nullable());
+            fields.emplace_back(field);
+        } else {
+            std::shared_ptr<arrow::Field> field = arrow::field(
+                    _column_names[i], type, _output_vexpr_ctxs[i]->root()->is_nullable());
+            fields.emplace_back(field);
+        }
     }
     _arrow_schema = arrow::schema(std::move(fields));
     return Status::OK();
diff --git a/be/src/vec/runtime/vparquet_transformer.h b/be/src/vec/runtime/vparquet_transformer.h
index c9d2de59a51..78be0959363 100644
--- a/be/src/vec/runtime/vparquet_transformer.h
+++ b/be/src/vec/runtime/vparquet_transformer.h
@@ -89,6 +89,13 @@ public:
 // a wrapper of parquet output stream
 class VParquetTransformer final : public VFileFormatTransformer {
 public:
+    VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer,
+                        const VExprContextSPtrs& output_vexpr_ctxs,
+                        std::vector<std::string> column_names,
+                        TParquetCompressionType::type compression_type,
+                        bool parquet_disable_dictionary, TParquetVersion::type parquet_version,
+                        bool output_object_data);
+
     VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer,
                         const VExprContextSPtrs& output_vexpr_ctxs,
                         const std::vector<TParquetSchema>& parquet_schemas,
@@ -117,7 +124,8 @@ private:
     std::unique_ptr<parquet::arrow::FileWriter> _writer;
     std::shared_ptr<arrow::Schema> _arrow_schema;
 
-    const std::vector<TParquetSchema>& _parquet_schemas;
+    std::vector<std::string> _column_names;
+    const std::vector<TParquetSchema>* _parquet_schemas = nullptr;
     const TParquetCompressionType::type _compression_type;
     const bool _parquet_disable_dictionary;
     const TParquetVersion::type _parquet_version;
diff --git a/be/src/vec/sink/writer/vhive_partition_writer.cpp b/be/src/vec/sink/writer/vhive_partition_writer.cpp
index ca6e76862ba..7f9a0dd1b1e 100644
--- a/be/src/vec/sink/writer/vhive_partition_writer.cpp
+++ b/be/src/vec/sink/writer/vhive_partition_writer.cpp
@@ -20,6 +20,7 @@
 #include "io/file_factory.h"
 #include "io/fs/file_system.h"
 #include "runtime/runtime_state.h"
+#include "vec/columns/column_map.h"
 #include "vec/core/materialize_block.h"
 #include "vec/runtime/vorc_transformer.h"
 #include "vec/runtime/vparquet_transformer.h"
@@ -28,14 +29,17 @@ namespace doris {
 namespace vectorized {
 
 VHivePartitionWriter::VHivePartitionWriter(
-        const TDataSink& t_sink, const std::string partition_name, TUpdateMode::type update_mode,
-        const VExprContextSPtrs& output_expr_ctxs, const std::vector<THiveColumn>& columns,
-        WriteInfo write_info, const std::string file_name, TFileFormatType::type file_format_type,
+        const TDataSink& t_sink, std::string partition_name, TUpdateMode::type update_mode,
+        const VExprContextSPtrs& output_expr_ctxs, const VExprContextSPtrs& write_output_expr_ctxs,
+        const std::set<size_t>& non_write_columns_indices, const std::vector<THiveColumn>& columns,
+        WriteInfo write_info, std::string file_name, TFileFormatType::type file_format_type,
         TFileCompressType::type hive_compress_type,
         const std::map<std::string, std::string>& hadoop_conf)
         : _partition_name(std::move(partition_name)),
           _update_mode(update_mode),
           _vec_output_expr_ctxs(output_expr_ctxs),
+          _write_output_expr_ctxs(write_output_expr_ctxs),
+          _non_write_columns_indices(non_write_columns_indices),
           _columns(columns),
           _write_info(std::move(write_info)),
           _file_name(std::move(file_name)),
@@ -53,6 +57,12 @@ Status VHivePartitionWriter::open(RuntimeState* state, RuntimeProfile* profile)
             _write_info.file_type, state->exec_env(), broker_addresses, _hadoop_conf,
             fmt::format("{}/{}", _write_info.write_path, _file_name), 0, _file_writer));
 
+    std::vector<std::string> column_names;
+    column_names.reserve(_columns.size());
+    for (int i = 0; i < _columns.size(); i++) {
+        column_names.push_back(_columns[i].name);
+    }
+
     switch (_file_format_type) {
     case TFileFormatType::FORMAT_PARQUET: {
         bool parquet_disable_dictionary = false;
@@ -75,16 +85,8 @@ Status VHivePartitionWriter::open(RuntimeState* state, RuntimeProfile* profile)
                                          to_string(_hive_compress_type));
         }
         }
-        std::vector<TParquetSchema> parquet_schemas;
-        parquet_schemas.reserve(_columns.size());
-        for (int i = 0; i < _columns.size(); i++) {
-            VExprSPtr column_expr = _vec_output_expr_ctxs[i]->root();
-            TParquetSchema parquet_schema;
-            parquet_schema.schema_column_name = _columns[i].name;
-            parquet_schemas.emplace_back(std::move(parquet_schema));
-        }
         _file_format_transformer.reset(new VParquetTransformer(
-                state, _file_writer.get(), _vec_output_expr_ctxs, parquet_schemas,
+                state, _file_writer.get(), _write_output_expr_ctxs, std::move(column_names),
                 parquet_compression_type, parquet_disable_dictionary, TParquetVersion::PARQUET_1_0,
                 false));
         return _file_format_transformer->open();
@@ -112,21 +114,10 @@ Status VHivePartitionWriter::open(RuntimeState* state, RuntimeProfile* profile)
             return Status::InternalError("Unsupported type {} with orc", _hive_compress_type);
         }
         }
-        orc_compression_type = orc::CompressionKind::CompressionKind_ZLIB;
-
-        std::unique_ptr<orc::Type> root_schema = orc::createStructType();
-        for (int i = 0; i < _columns.size(); i++) {
-            VExprSPtr column_expr = _vec_output_expr_ctxs[i]->root();
-            try {
-                root_schema->addStructField(_columns[i].name, _build_orc_type(column_expr->type()));
-            } catch (doris::Exception& e) {
-                return e.to_status();
-            }
-        }
 
         _file_format_transformer.reset(
-                new VOrcTransformer(state, _file_writer.get(), _vec_output_expr_ctxs,
-                                    std::move(root_schema), false, orc_compression_type));
+                new VOrcTransformer(state, _file_writer.get(), _write_output_expr_ctxs,
+                                    std::move(column_names), false, orc_compression_type));
         return _file_format_transformer->open();
     }
     default: {
@@ -164,81 +155,6 @@ Status VHivePartitionWriter::write(vectorized::Block& block, vectorized::IColumn
     return Status::OK();
 }
 
-std::unique_ptr<orc::Type> VHivePartitionWriter::_build_orc_type(
-        const TypeDescriptor& type_descriptor) {
-    std::pair<Status, std::unique_ptr<orc::Type>> result;
-    switch (type_descriptor.type) {
-    case TYPE_BOOLEAN: {
-        return orc::createPrimitiveType(orc::BOOLEAN);
-    }
-    case TYPE_TINYINT: {
-        return orc::createPrimitiveType(orc::BYTE);
-    }
-    case TYPE_SMALLINT: {
-        return orc::createPrimitiveType(orc::SHORT);
-    }
-    case TYPE_INT: {
-        return orc::createPrimitiveType(orc::INT);
-    }
-    case TYPE_BIGINT: {
-        return orc::createPrimitiveType(orc::LONG);
-    }
-    case TYPE_FLOAT: {
-        return orc::createPrimitiveType(orc::FLOAT);
-    }
-    case TYPE_DOUBLE: {
-        return orc::createPrimitiveType(orc::DOUBLE);
-    }
-    case TYPE_CHAR: {
-        return orc::createCharType(orc::CHAR, type_descriptor.len);
-    }
-    case TYPE_VARCHAR: {
-        return orc::createCharType(orc::VARCHAR, type_descriptor.len);
-    }
-    case TYPE_STRING: {
-        return orc::createPrimitiveType(orc::STRING);
-    }
-    case TYPE_BINARY: {
-        return orc::createPrimitiveType(orc::STRING);
-    }
-    case TYPE_DATEV2: {
-        return orc::createPrimitiveType(orc::DATE);
-    }
-    case TYPE_DATETIMEV2: {
-        return orc::createPrimitiveType(orc::TIMESTAMP);
-    }
-    case TYPE_DECIMAL32: {
-        return orc::createDecimalType(type_descriptor.precision, type_descriptor.scale);
-    }
-    case TYPE_DECIMAL64: {
-        return orc::createDecimalType(type_descriptor.precision, type_descriptor.scale);
-    }
-    case TYPE_DECIMAL128I: {
-        return orc::createDecimalType(type_descriptor.precision, type_descriptor.scale);
-    }
-    case TYPE_STRUCT: {
-        std::unique_ptr<orc::Type> struct_type = orc::createStructType();
-        for (int j = 0; j < type_descriptor.children.size(); ++j) {
-            struct_type->addStructField(type_descriptor.field_names[j],
-                                        _build_orc_type(type_descriptor.children[j]));
-        }
-        return struct_type;
-    }
-    case TYPE_ARRAY: {
-        return orc::createListType(_build_orc_type(type_descriptor.children[0]));
-    }
-    case TYPE_MAP: {
-        return orc::createMapType(_build_orc_type(type_descriptor.children[0]),
-                                  _build_orc_type(type_descriptor.children[1]));
-    }
-    default: {
-        throw doris::Exception(doris::ErrorCode::INTERNAL_ERROR,
-                               "Unsupported type {} to build orc type",
-                               type_descriptor.debug_string());
-    }
-    }
-}
-
 Status VHivePartitionWriter::_projection_and_filter_block(doris::vectorized::Block& input_block,
                                                           const vectorized::IColumn::Filter* filter,
                                                           doris::vectorized::Block* output_block) {
@@ -263,6 +179,8 @@ Status VHivePartitionWriter::_projection_and_filter_block(doris::vectorized::Blo
 
     Block::filter_block_internal(output_block, columns_to_filter, *filter);
 
+    output_block->erase(_non_write_columns_indices);
+
     return status;
 }
 
diff --git a/be/src/vec/sink/writer/vhive_partition_writer.h b/be/src/vec/sink/writer/vhive_partition_writer.h
index d3c52800bea..88488a94673 100644
--- a/be/src/vec/sink/writer/vhive_partition_writer.h
+++ b/be/src/vec/sink/writer/vhive_partition_writer.h
@@ -45,6 +45,8 @@ public:
 
     VHivePartitionWriter(const TDataSink& t_sink, const std::string partition_name,
                          TUpdateMode::type update_mode, const VExprContextSPtrs& output_expr_ctxs,
+                         const VExprContextSPtrs& write_output_expr_ctxs,
+                         const std::set<size_t>& non_write_columns_indices,
                          const std::vector<THiveColumn>& columns, WriteInfo write_info,
                          const std::string file_name, TFileFormatType::type file_format_type,
                          TFileCompressType::type hive_compress_type,
@@ -61,8 +63,6 @@ public:
     inline size_t written_len() { return _file_format_transformer->written_len(); }
 
 private:
-    std::unique_ptr<orc::Type> _build_orc_type(const TypeDescriptor& type_descriptor);
-
     Status _projection_and_filter_block(doris::vectorized::Block& input_block,
                                         const vectorized::IColumn::Filter* filter,
                                         doris::vectorized::Block* output_block);
@@ -79,6 +79,8 @@ private:
     size_t _input_size_in_bytes = 0;
 
     const VExprContextSPtrs& _vec_output_expr_ctxs;
+    const VExprContextSPtrs& _write_output_expr_ctxs;
+    const std::set<size_t>& _non_write_columns_indices;
 
     const std::vector<THiveColumn>& _columns;
     WriteInfo _write_info;
diff --git a/be/src/vec/sink/writer/vhive_table_writer.cpp b/be/src/vec/sink/writer/vhive_table_writer.cpp
index 4ea5fcbf4ed..e56090773b5 100644
--- a/be/src/vec/sink/writer/vhive_table_writer.cpp
+++ b/be/src/vec/sink/writer/vhive_table_writer.cpp
@@ -43,8 +43,25 @@ Status VHiveTableWriter::open(RuntimeState* state, RuntimeProfile* profile) {
     _profile = profile;
 
     for (int i = 0; i < _t_sink.hive_table_sink.columns.size(); ++i) {
-        if (_t_sink.hive_table_sink.columns[i].column_type == THiveColumnType::PARTITION_KEY) {
+        switch (_t_sink.hive_table_sink.columns[i].column_type) {
+        case THiveColumnType::PARTITION_KEY: {
             _partition_columns_input_index.emplace_back(i);
+            _non_write_columns_indices.insert(i);
+            break;
+        }
+        case THiveColumnType::REGULAR: {
+            _write_output_vexpr_ctxs.push_back(_vec_output_expr_ctxs[i]);
+            break;
+        }
+        case THiveColumnType::SYNTHESIZED: {
+            _non_write_columns_indices.insert(i);
+            break;
+        }
+        default: {
+            throw doris::Exception(doris::ErrorCode::INTERNAL_ERROR,
+                                   "Illegal hive column type {}, it should not be here.",
+                                   to_string(_t_sink.hive_table_sink.columns[i].column_type));
+        }
         }
     }
     return Status::OK();
@@ -232,7 +249,8 @@ std::shared_ptr<VHivePartitionWriter> VHiveTableWriter::_create_partition_writer
 
     return std::make_shared<VHivePartitionWriter>(
             _t_sink, std::move(partition_name), update_mode, _vec_output_expr_ctxs,
-            hive_table_sink.columns, std::move(write_info),
+            _write_output_vexpr_ctxs, _non_write_columns_indices, hive_table_sink.columns,
+            std::move(write_info),
             fmt::format("{}{}", _compute_file_name(),
                         _get_file_extension(file_format_type, write_compress_type)),
             file_format_type, write_compress_type, hive_table_sink.hadoop_config);
diff --git a/be/src/vec/sink/writer/vhive_table_writer.h b/be/src/vec/sink/writer/vhive_table_writer.h
index a4681b32e3f..9f48f6afde1 100644
--- a/be/src/vec/sink/writer/vhive_table_writer.h
+++ b/be/src/vec/sink/writer/vhive_table_writer.h
@@ -68,7 +68,10 @@ private:
     RuntimeState* _state = nullptr;
     RuntimeProfile* _profile = nullptr;
     std::vector<int> _partition_columns_input_index;
+    std::set<size_t> _non_write_columns_indices;
     std::unordered_map<std::string, std::shared_ptr<VHivePartitionWriter>> _partitions_to_writers;
+
+    VExprContextSPtrs _write_output_vexpr_ctxs;
 };
 } // namespace vectorized
 } // namespace doris


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


(doris) 17/19: [feature](insert)use optional location and add hive regression test (#33153)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 18fb8407ae1fadead8ab925734375678b4300c66
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Wed Apr 10 21:24:09 2024 +0800

    [feature](insert)use optional location and add hive regression test (#33153)
---
 fe/fe-core/src/main/cup/sql_parser.cup             |   4 +-
 .../java/org/apache/doris/analysis/DbName.java     |   3 -
 .../apache/doris/analysis/ShowCreateDbStmt.java    |  13 +-
 .../apache/doris/datasource/InternalCatalog.java   |   4 +-
 .../datasource/hive/HiveDatabaseMetadata.java      |   9 +
 .../doris/datasource/hive/HiveMetadataOps.java     |  26 +-
 .../doris/datasource/hive/HiveTableMetadata.java   |  37 +-
 .../org/apache/doris/datasource/hive/HiveUtil.java |  52 +-
 .../doris/nereids/parser/LogicalPlanBuilder.java   |   3 +-
 .../trees/plans/commands/info/CreateTableInfo.java |  45 +-
 .../java/org/apache/doris/qe/ShowExecutor.java     |   6 +-
 .../doris/analysis/ShowCreateDbStmtTest.java       |   5 +-
 .../datasource/hive/HiveDDLAndDMLPlanTest.java     |  22 +-
 .../doris/datasource/hive/HmsCommitTest.java       |  18 +-
 .../java/org/apache/doris/qe/ShowExecutorTest.java |   5 +-
 .../external_table_p0/hive/ddl/test_hive_ctas.out  |  92 +++
 .../external_table_p0/hive/ddl/test_hive_ddl.out   |  60 ++
 .../hive/ddl/test_hive_ddl_and_ctas.out            | 148 -----
 .../hive/ddl/test_hive_write_type.out              |  18 +
 .../hive/ddl/test_hive_ctas.groovy                 | 515 ++++++++++++++++
 .../hive/ddl/test_hive_ddl.groovy                  | 676 +++++++++++++++++++++
 .../hive/ddl/test_hive_ddl_and_ctas.groovy         | 423 -------------
 .../hive/ddl/test_hive_write_type.groovy           | 285 +++++++++
 23 files changed, 1832 insertions(+), 637 deletions(-)

diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index 3a4f77fcc7a..26129f2d2c0 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -4039,11 +4039,11 @@ show_param ::=
         RESULT = new ShowCreateTableStmt(table, true, false);
     :}
     /* Create database */
-    | KW_CREATE KW_DATABASE ident:db
+    | KW_CREATE KW_DATABASE db_name:db
     {:
         RESULT = new ShowCreateDbStmt(db);
     :}
-    | KW_CREATE KW_SCHEMA ident:db
+    | KW_CREATE KW_SCHEMA db_name:db
     {:
         RESULT = new ShowCreateDbStmt(db);
     :}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DbName.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DbName.java
index 2b8e92ab982..934f2d73435 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DbName.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DbName.java
@@ -14,9 +14,6 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-// This file is copied from
-// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/TableName.java
-// and modified by Doris
 
 package org.apache.doris.analysis;
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java
index 0cadde094b8..1a51546e186 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java
@@ -40,19 +40,28 @@ public class ShowCreateDbStmt extends ShowStmt {
                     .addColumn(new Column("Create Database", ScalarType.createVarchar(30)))
                     .build();
 
+    private String ctl;
     private String db;
 
-    public ShowCreateDbStmt(String db) {
-        this.db = db;
+    public ShowCreateDbStmt(DbName db) {
+        this.ctl = db.getCtl();
+        this.db = db.getDb();
     }
 
     public String getDb() {
         return db;
     }
 
+    public String getCtl() {
+        return ctl;
+    }
+
     @Override
     public void analyze(Analyzer analyzer) throws AnalysisException, UserException {
         super.analyze(analyzer);
+        if (Strings.isNullOrEmpty(ctl)) {
+            ctl = Env.getCurrentEnv().getCurrentCatalog().getName();
+        }
         if (Strings.isNullOrEmpty(db)) {
             ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_DB_NAME, db);
         }
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 2bd2f21478b..9295ab56a18 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
@@ -1129,8 +1129,8 @@ public class InternalCatalog implements CatalogIf<Database> {
             createEsTable(db, stmt);
             return;
         } else if (engineName.equalsIgnoreCase("hive")) {
-            createHiveTable(db, stmt);
-            return;
+            // should use hive catalog to create external hive table
+            throw new UserException("Cannot create hive table in internal catalog, should switch to hive catalog.");
         } else if (engineName.equalsIgnoreCase("jdbc")) {
             createJdbcTable(db, stmt);
             return;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDatabaseMetadata.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDatabaseMetadata.java
index 50a80db3962..a1a383e0d07 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDatabaseMetadata.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDatabaseMetadata.java
@@ -21,6 +21,7 @@ import org.apache.doris.datasource.DatabaseMetadata;
 
 import lombok.Data;
 
+import java.util.HashMap;
 import java.util.Map;
 
 @Data
@@ -29,4 +30,12 @@ public class HiveDatabaseMetadata implements DatabaseMetadata {
     private String locationUri;
     private Map<String, String> properties;
     private String comment;
+
+    public Map<String, String> getProperties() {
+        return properties == null ? new HashMap<>() : properties;
+    }
+
+    public String getComment() {
+        return comment == null ? "" : comment;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
index 5cf362508e3..c5e74defc1a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
@@ -23,8 +23,10 @@ import org.apache.doris.analysis.DistributionDesc;
 import org.apache.doris.analysis.DropDbStmt;
 import org.apache.doris.analysis.DropTableStmt;
 import org.apache.doris.analysis.HashDistributionDesc;
+import org.apache.doris.analysis.PartitionDesc;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.JdbcResource;
+import org.apache.doris.catalog.PartitionType;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.ErrorCode;
@@ -48,11 +50,12 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.function.Function;
 
 public class HiveMetadataOps implements ExternalMetadataOps {
-    public static final String LOCATION_URI_KEY = "location_uri";
+    public static final String LOCATION_URI_KEY = "location";
     public static final String FILE_FORMAT_KEY = "file_format";
     public static final Set<String> DORIS_HIVE_KEYS = ImmutableSet.of(FILE_FORMAT_KEY, LOCATION_URI_KEY);
     private static final Logger LOG = LogManager.getLogger(HiveMetadataOps.class);
@@ -125,6 +128,7 @@ public class HiveMetadataOps implements ExternalMetadataOps {
             if (properties.containsKey(LOCATION_URI_KEY)) {
                 catalogDatabase.setLocationUri(properties.get(LOCATION_URI_KEY));
             }
+            // remove it when set
             properties.remove(LOCATION_URI_KEY);
             catalogDatabase.setProperties(properties);
             catalogDatabase.setComment(properties.getOrDefault("comment", ""));
@@ -185,8 +189,18 @@ public class HiveMetadataOps implements ExternalMetadataOps {
             }
             List<String> partitionColNames = new ArrayList<>();
             if (stmt.getPartitionDesc() != null) {
-                partitionColNames.addAll(stmt.getPartitionDesc().getPartitionColNames());
+                PartitionDesc partitionDesc = stmt.getPartitionDesc();
+                if (partitionDesc.getType() == PartitionType.RANGE) {
+                    throw new UserException("Only support 'LIST' partition type in hive catalog.");
+                }
+                partitionColNames.addAll(partitionDesc.getPartitionColNames());
+                if (!partitionDesc.getSinglePartitionDescs().isEmpty()) {
+                    throw new UserException("Partition values expressions is not supported in hive catalog.");
+                }
+
             }
+            String comment = stmt.getComment();
+            Optional<String> location = Optional.ofNullable(props.getOrDefault(LOCATION_URI_KEY, null));
             HiveTableMetadata hiveTableMeta;
             DistributionDesc bucketInfo = stmt.getDistributionDesc();
             if (bucketInfo != null) {
@@ -194,12 +208,14 @@ public class HiveMetadataOps implements ExternalMetadataOps {
                     if (bucketInfo instanceof HashDistributionDesc) {
                         hiveTableMeta = HiveTableMetadata.of(dbName,
                                 tblName,
+                                location,
                                 stmt.getColumns(),
                                 partitionColNames,
                                 ((HashDistributionDesc) bucketInfo).getDistributionColumnNames(),
                                 bucketInfo.getBuckets(),
                                 ddlProps,
-                                fileFormat);
+                                fileFormat,
+                                comment);
                     } else {
                         throw new UserException("External hive table only supports hash bucketing");
                     }
@@ -210,10 +226,12 @@ public class HiveMetadataOps implements ExternalMetadataOps {
             } else {
                 hiveTableMeta = HiveTableMetadata.of(dbName,
                         tblName,
+                        location,
                         stmt.getColumns(),
                         partitionColNames,
                         ddlProps,
-                        fileFormat);
+                        fileFormat,
+                        comment);
             }
             client.createTable(hiveTableMeta, stmt.isSetIfNotExists());
             db.setUnInitialized(true);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTableMetadata.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTableMetadata.java
index d8de9d60734..7f7a1ef7273 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTableMetadata.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTableMetadata.java
@@ -23,13 +23,16 @@ import org.apache.doris.datasource.TableMetadata;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 
 public class HiveTableMetadata implements TableMetadata {
     private final String dbName;
     private final String tableName;
+    private final Optional<String> location;
     private final List<Column> columns;
     private final List<String> partitionKeys;
     private final String fileFormat;
+    private final String comment;
     private final Map<String, String> properties;
     private List<String> bucketCols;
     private int numBuckets;
@@ -37,20 +40,24 @@ public class HiveTableMetadata implements TableMetadata {
 
     public HiveTableMetadata(String dbName,
                              String tblName,
+                             Optional<String> location,
                              List<Column> columns,
                              List<String> partitionKeys,
                              Map<String, String> props,
-                             String fileFormat) {
-        this(dbName, tblName, columns, partitionKeys, new ArrayList<>(), 0, props, fileFormat);
+                             String fileFormat,
+                             String comment) {
+        this(dbName, tblName, location, columns, partitionKeys, new ArrayList<>(), 0, props, fileFormat, comment);
     }
 
     public HiveTableMetadata(String dbName, String tableName,
+                             Optional<String> location,
                              List<Column> columns,
                              List<String> partitionKeys,
                              List<String> bucketCols,
                              int numBuckets,
                              Map<String, String> props,
-                             String fileFormat) {
+                             String fileFormat,
+                             String comment) {
         this.dbName = dbName;
         this.tableName = tableName;
         this.columns = columns;
@@ -59,6 +66,8 @@ public class HiveTableMetadata implements TableMetadata {
         this.numBuckets = numBuckets;
         this.properties = props;
         this.fileFormat = fileFormat;
+        this.location = location;
+        this.comment = comment;
     }
 
     @Override
@@ -71,6 +80,14 @@ public class HiveTableMetadata implements TableMetadata {
         return tableName;
     }
 
+    public Optional<String> getLocation() {
+        return location;
+    }
+
+    public String getComment() {
+        return comment == null ? "" : comment;
+    }
+
     @Override
     public Map<String, String> getProperties() {
         return properties;
@@ -98,22 +115,26 @@ public class HiveTableMetadata implements TableMetadata {
 
     public static HiveTableMetadata of(String dbName,
                                        String tblName,
+                                       Optional<String> location,
                                        List<Column> columns,
                                        List<String> partitionKeys,
                                        Map<String, String> props,
-                                       String fileFormat) {
-        return new HiveTableMetadata(dbName, tblName, columns, partitionKeys, props, fileFormat);
+                                       String fileFormat,
+                                       String comment) {
+        return new HiveTableMetadata(dbName, tblName, location, columns, partitionKeys, props, fileFormat, comment);
     }
 
     public static HiveTableMetadata of(String dbName,
                                        String tblName,
+                                       Optional<String> location,
                                        List<Column> columns,
                                        List<String> partitionKeys,
                                        List<String> bucketCols,
                                        int numBuckets,
                                        Map<String, String> props,
-                                       String fileFormat) {
-        return new HiveTableMetadata(dbName, tblName, columns, partitionKeys,
-                bucketCols, numBuckets, props, fileFormat);
+                                       String fileFormat,
+                                       String comment) {
+        return new HiveTableMetadata(dbName, tblName, location, columns, partitionKeys,
+                bucketCols, numBuckets, props, fileFormat, comment);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
index dfbfe786985..bca04215fc4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
@@ -23,17 +23,20 @@ import org.apache.doris.common.UserException;
 import org.apache.doris.datasource.ExternalCatalog;
 import org.apache.doris.fs.remote.BrokerFileSystem;
 import org.apache.doris.fs.remote.RemoteFileSystem;
+import org.apache.doris.nereids.exceptions.AnalysisException;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
@@ -53,6 +56,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
 
@@ -61,6 +65,10 @@ import java.util.stream.Collectors;
  */
 public final class HiveUtil {
 
+    public static final String COMPRESSION_KEY = "compression";
+    public static final Set<String> SUPPORTED_ORC_COMPRESSIONS = ImmutableSet.of("plain", "zlib", "snappy", "zstd");
+    public static final Set<String> SUPPORTED_PARQUET_COMPRESSIONS = ImmutableSet.of("plain", "snappy", "zstd");
+
     private HiveUtil() {
     }
 
@@ -187,12 +195,11 @@ public final class HiveUtil {
         table.setCreateTime(createTime);
         table.setLastAccessTime(createTime);
         // table.setRetention(0);
-        String location = hiveTable.getProperties().get(HiveMetadataOps.LOCATION_URI_KEY);
         Set<String> partitionSet = new HashSet<>(hiveTable.getPartitionKeys());
         Pair<List<FieldSchema>, List<FieldSchema>> hiveSchema = toHiveSchema(hiveTable.getColumns(), partitionSet);
 
         table.setSd(toHiveStorageDesc(hiveSchema.first, hiveTable.getBucketCols(), hiveTable.getNumBuckets(),
-                hiveTable.getFileFormat(), location));
+                hiveTable.getFileFormat(), hiveTable.getLocation()));
         table.setPartitionKeys(hiveSchema.second);
 
         // table.setViewOriginalText(hiveTable.getViewSql());
@@ -200,18 +207,44 @@ public final class HiveUtil {
         table.setTableType("MANAGED_TABLE");
         Map<String, String> props = new HashMap<>(hiveTable.getProperties());
         props.put(ExternalCatalog.DORIS_VERSION, ExternalCatalog.DORIS_VERSION_VALUE);
+        setCompressType(hiveTable, props);
+        // set hive table comment by table properties
+        props.put("comment", hiveTable.getComment());
         table.setParameters(props);
+        if (props.containsKey("owner")) {
+            table.setOwner(props.get("owner"));
+        }
         return table;
     }
 
+    private static void setCompressType(HiveTableMetadata hiveTable, Map<String, String> props) {
+        String fileFormat = hiveTable.getFileFormat();
+        String compression = props.get(COMPRESSION_KEY);
+        // on HMS, default orc compression type is zlib and default parquet compression type is snappy.
+        if (fileFormat.equalsIgnoreCase("parquet")) {
+            if (StringUtils.isNotEmpty(compression) && !SUPPORTED_PARQUET_COMPRESSIONS.contains(compression)) {
+                throw new AnalysisException("Unsupported parquet compression type " + compression);
+            }
+            props.putIfAbsent("parquet.compression", StringUtils.isEmpty(compression) ? "snappy" : compression);
+        } else if (fileFormat.equalsIgnoreCase("orc")) {
+            if (StringUtils.isNotEmpty(compression) && !SUPPORTED_ORC_COMPRESSIONS.contains(compression)) {
+                throw new AnalysisException("Unsupported orc compression type " + compression);
+            }
+            props.putIfAbsent("orc.compress", StringUtils.isEmpty(compression) ? "zlib" : compression);
+        } else {
+            throw new IllegalArgumentException("Compression is not supported on " + fileFormat);
+        }
+        // remove if exists
+        props.remove(COMPRESSION_KEY);
+    }
+
     private static StorageDescriptor toHiveStorageDesc(List<FieldSchema> columns,
-            List<String> bucketCols, int numBuckets, String fileFormat, String location) {
+            List<String> bucketCols, int numBuckets, String fileFormat, Optional<String> location) {
         StorageDescriptor sd = new StorageDescriptor();
         sd.setCols(columns);
         setFileFormat(fileFormat, sd);
-        if (StringUtils.isNotEmpty(location)) {
-            sd.setLocation(location);
-        }
+        location.ifPresent(sd::setLocation);
+
         sd.setBucketCols(bucketCols);
         sd.setNumBuckets(numBuckets);
         Map<String, String> parameters = new HashMap<>();
@@ -267,8 +300,13 @@ public final class HiveUtil {
         if (StringUtils.isNotEmpty(hiveDb.getLocationUri())) {
             database.setLocationUri(hiveDb.getLocationUri());
         }
-        database.setParameters(hiveDb.getProperties());
+        Map<String, String> props = hiveDb.getProperties();
+        database.setParameters(props);
         database.setDescription(hiveDb.getComment());
+        if (props.containsKey("owner")) {
+            database.setOwnerName(props.get("owner"));
+            database.setOwnerType(PrincipalType.USER);
+        }
         return database;
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
index 874b6a0d432..64c3572606f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
@@ -2440,7 +2440,8 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
         } else if (ctx.UNIQUE() != null) {
             keysType = KeysType.UNIQUE_KEYS;
         }
-        String engineName = ctx.engine != null ? ctx.engine.getText().toLowerCase() : "olap";
+        // when engineName is null, get engineName from current catalog later
+        String engineName = ctx.engine != null ? ctx.engine.getText().toLowerCase() : null;
         int bucketNum = FeConstants.default_bucket_num;
         if (ctx.INTEGER_VALUE() != null) {
             bucketNum = Integer.parseInt(ctx.INTEGER_VALUE().getText());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
index d2b6a1fd31d..0fc746da4e7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
@@ -42,6 +42,8 @@ import org.apache.doris.common.util.ParseUtil;
 import org.apache.doris.common.util.PropertyAnalyzer;
 import org.apache.doris.datasource.InternalCatalog;
 import org.apache.doris.datasource.es.EsUtil;
+import org.apache.doris.datasource.hive.HMSExternalCatalog;
+import org.apache.doris.datasource.iceberg.IcebergExternalCatalog;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.parser.PartitionTableInfo;
@@ -76,7 +78,7 @@ public class CreateTableInfo {
     private List<ColumnDefinition> columns;
     private final List<IndexDefinition> indexes;
     private final List<String> ctasColumns;
-    private final String engineName;
+    private String engineName;
     private KeysType keysType;
     private List<String> keys;
     private final String comment;
@@ -190,13 +192,22 @@ public class CreateTableInfo {
             throw new AnalysisException("table should contain at least one column");
         }
 
+        // analyze catalog name
+        if (Strings.isNullOrEmpty(ctlName)) {
+            if (ctx.getCurrentCatalog() != null) {
+                ctlName = ctx.getCurrentCatalog().getName();
+            } else {
+                ctlName = InternalCatalog.INTERNAL_CATALOG_NAME;
+            }
+        }
+        paddingEngineName(ctlName, ctx);
         checkEngineName();
 
         if (properties == null) {
             properties = Maps.newHashMap();
         }
 
-        if (Strings.isNullOrEmpty(engineName) || engineName.equalsIgnoreCase("olap")) {
+        if (engineName.equalsIgnoreCase("olap")) {
             if (distribution == null) {
                 throw new AnalysisException("Create olap table should contain distribution desc");
             }
@@ -209,12 +220,10 @@ public class CreateTableInfo {
             throw new AnalysisException(e.getMessage(), e);
         }
 
-        // analyze catalog name
-        if (Strings.isNullOrEmpty(ctlName)) {
-            if (ctx.getCurrentCatalog() != null) {
-                ctlName = ctx.getCurrentCatalog().getName();
-            } else {
-                ctlName = InternalCatalog.INTERNAL_CATALOG_NAME;
+        if (engineName.equals("olap")) {
+            if (!ctlName.equals(InternalCatalog.INTERNAL_CATALOG_NAME)) {
+                throw new AnalysisException("Cannot create olap table out of internal catalog."
+                    + " Make sure 'engine' type is specified when use the catalog: " + ctlName);
             }
         }
 
@@ -534,18 +543,34 @@ public class CreateTableInfo {
         }
     }
 
+    private void paddingEngineName(String ctlName, ConnectContext ctx) {
+        if (Strings.isNullOrEmpty(engineName)) {
+            if (InternalCatalog.INTERNAL_CATALOG_NAME.equals(ctlName)) {
+                engineName = "olap";
+            } else if (ctx.getCurrentCatalog() instanceof HMSExternalCatalog) {
+                engineName = "hive";
+            } else if (ctx.getCurrentCatalog() instanceof IcebergExternalCatalog) {
+                engineName = "iceberg";
+            } else {
+                // set to olap by default
+                engineName = "olap";
+            }
+        }
+    }
+
     /**
      * validate ctas definition
      */
     public void validateCreateTableAsSelect(List<String> qualifierTableName, List<ColumnDefinition> columns,
                                             ConnectContext ctx) {
+        String catalogName = qualifierTableName.get(0);
+        paddingEngineName(catalogName, ctx);
         this.columns = Utils.copyRequiredMutableList(columns);
         // bucket num is hard coded 10 to be consistent with legacy planner
         if (engineName.equals("olap") && this.distribution == null) {
-            String catalogName = qualifierTableName.get(0);
             if (!catalogName.equals(InternalCatalog.INTERNAL_CATALOG_NAME)) {
                 throw new AnalysisException("Cannot create olap table out of internal catalog."
-                        + "Make sure 'engine' type is specified when use the catalog: " + catalogName);
+                        + " Make sure 'engine' type is specified when use the catalog: " + catalogName);
             }
             this.distribution = new DistributionDescriptor(true, false, 10,
                     Lists.newArrayList(columns.get(0).getName()));
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
index fea976e11f8..1a407d2115b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
@@ -986,7 +986,11 @@ public class ShowExecutor {
         List<List<String>> rows = Lists.newArrayList();
 
         StringBuilder sb = new StringBuilder();
-        CatalogIf catalog = ctx.getCurrentCatalog();
+        String catalogName = showStmt.getCtl();
+        if (Strings.isNullOrEmpty(catalogName)) {
+            catalogName = Env.getCurrentEnv().getCurrentCatalog().getName();
+        }
+        CatalogIf<?> catalog = Env.getCurrentEnv().getCatalogMgr().getCatalogOrAnalysisException(catalogName);
         if (catalog instanceof HMSExternalCatalog) {
             String simpleDBName = ClusterNamespace.getNameFromFullName(showStmt.getDb());
             org.apache.hadoop.hive.metastore.api.Database db = ((HMSExternalCatalog) catalog).getClient()
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateDbStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateDbStmtTest.java
index 96638801781..dedea00d697 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateDbStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateDbStmtTest.java
@@ -19,6 +19,7 @@ package org.apache.doris.analysis;
 
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.UserException;
+import org.apache.doris.datasource.InternalCatalog;
 import org.apache.doris.mysql.privilege.AccessControllerManager;
 import org.apache.doris.mysql.privilege.MockedAuth;
 import org.apache.doris.qe.ConnectContext;
@@ -43,7 +44,7 @@ public class ShowCreateDbStmtTest {
 
     @Test
     public void testNormal() throws AnalysisException, UserException {
-        ShowCreateDbStmt stmt = new ShowCreateDbStmt("testDb");
+        ShowCreateDbStmt stmt = new ShowCreateDbStmt(new DbName(InternalCatalog.INTERNAL_CATALOG_NAME, "testDb"));
         stmt.analyze(AccessTestUtil.fetchAdminAnalyzer(true));
         Assert.assertEquals("testDb", stmt.getDb());
         Assert.assertEquals(2, stmt.getMetaData().getColumnCount());
@@ -52,7 +53,7 @@ public class ShowCreateDbStmtTest {
 
     @Test(expected = AnalysisException.class)
     public void testEmptyDb() throws AnalysisException, UserException {
-        ShowCreateDbStmt stmt = new ShowCreateDbStmt("");
+        ShowCreateDbStmt stmt = new ShowCreateDbStmt(new DbName(InternalCatalog.INTERNAL_CATALOG_NAME, ""));
         stmt.analyze(AccessTestUtil.fetchAdminAnalyzer(false));
         Assert.fail("No exception throws.");
     }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
index a34e6cc7034..dc35d38af68 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
@@ -182,7 +182,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 + "  `col2` STRING COMMENT 'col2'\n"
                 + ")  ENGINE=hive\n"
                 + "PROPERTIES (\n"
-                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'location'='hdfs://loc/db/tbl',\n"
                 + "  'file_format'='parquet')";
         createTable(createSourceExtUTable, true);
         // partitioned table
@@ -193,7 +193,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 + ")  ENGINE=hive\n"
                 + "PARTITION BY LIST (pt1, pt2) ()\n"
                 + "PROPERTIES (\n"
-                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'location'='hdfs://loc/db/tbl',\n"
                 + "  'file_format'='orc')";
         createTable(createSourceExtTable, true);
 
@@ -258,7 +258,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 + "  `col2` INT COMMENT 'col2'"
                 + ")  ENGINE=hive\n"
                 + "PROPERTIES (\n"
-                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'location'='hdfs://loc/db/tbl',\n"
                 + "  'file_format'='orc')";
         createTable(createTableIfNotExists, true);
         createTable(createTableIfNotExists, true);
@@ -288,7 +288,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 + "  `pt2` STRING COMMENT 'pt2'\n"
                 + ")  ENGINE=hive\n"
                 + "PROPERTIES (\n"
-                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'location'='hdfs://loc/db/tbl',\n"
                 + "  'file_format'='orc')";
         createTable(createUnPartTable, true);
         dropTable("unpart_tbl", true);
@@ -305,7 +305,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 + ")  ENGINE=hive\n"
                 + "PARTITION BY LIST (pt1, pt2) ()\n"
                 + "PROPERTIES (\n"
-                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'location'='hdfs://loc/db/tbl',\n"
                 + "  'file_format'='parquet')";
         createTable(createPartTable, true);
         // check IF NOT EXISTS
@@ -320,7 +320,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 + ")  ENGINE=hive\n"
                 + "DISTRIBUTED BY HASH (col2) BUCKETS 16\n"
                 + "PROPERTIES (\n"
-                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'location'='hdfs://loc/db/tbl',\n"
                 + "  'file_format'='orc')";
         ExceptionChecker.expectThrowsWithMsg(org.apache.doris.nereids.exceptions.AnalysisException.class,
                 "errCode = 2, detailMessage = errCode = 2,"
@@ -336,7 +336,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 + ")  ENGINE=hive\n"
                 + "DISTRIBUTED BY HASH (col2) BUCKETS 16\n"
                 + "PROPERTIES (\n"
-                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'location'='hdfs://loc/db/tbl',\n"
                 + "  'file_format'='orc')";
         createTable(createBucketedTableOk1, true);
         dropTable("buck_tbl", true);
@@ -352,7 +352,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 + "PARTITION BY LIST (pt2) ()\n"
                 + "DISTRIBUTED BY HASH (col2) BUCKETS 16\n"
                 + "PROPERTIES (\n"
-                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'location'='hdfs://loc/db/tbl',\n"
                 + "  'file_format'='orc')";
         createTable(createBucketedTableOk2, true);
         dropTable("part_buck_tbl", true);
@@ -430,15 +430,15 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
         createTable(createOlapSrc, true);
         switchHive();
         useDatabase(mockedDbName);
-        String olapCtasErr = "CREATE TABLE no_buck_olap AS SELECT * FROM internal.mockedDb.olap_src";
+        String olapCtasErr = "CREATE TABLE no_buck_olap ENGINE=olap AS SELECT * FROM internal.mockedDb.olap_src";
         LogicalPlan olapCtasErrPlan = nereidsParser.parseSingle(olapCtasErr);
         Assertions.assertTrue(olapCtasErrPlan instanceof CreateTableCommand);
         ExceptionChecker.expectThrowsWithMsg(org.apache.doris.nereids.exceptions.AnalysisException.class,
                 "Cannot create olap table out of internal catalog."
-                        + "Make sure 'engine' type is specified when use the catalog: hive",
+                        + " Make sure 'engine' type is specified when use the catalog: hive",
                 () -> ((CreateTableCommand) olapCtasErrPlan).run(connectContext, null));
 
-        String olapCtasOk = "CREATE TABLE internal.mockedDb.no_buck_olap"
+        String olapCtasOk = "CREATE TABLE internal.mockedDb.no_buck_olap ENGINE=olap"
                 + " PROPERTIES('replication_num' = '1')"
                 + " AS SELECT * FROM internal.mockedDb.olap_src";
         LogicalPlan olapCtasOkPlan = createTablesAndReturnPlans(true, olapCtasOk).get(0);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
index 4ec6ca84c52..fba91cb0b55 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
@@ -45,6 +45,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Optional;
 import java.util.UUID;
 
 public class HmsCommitTest {
@@ -107,23 +108,18 @@ public class HmsCommitTest {
         List<String> partitionKeys = new ArrayList<>();
         partitionKeys.add("c3");
         String fileFormat = "orc";
-        HashMap<String, String> params = new HashMap<String, String>() {{
-                put("location_uri", dbLocation + tbWithPartition);
-            }};
         HiveTableMetadata tableMetadata = new HiveTableMetadata(
-                dbName, tbWithPartition, columns, partitionKeys,
-                params, fileFormat);
+                dbName, tbWithPartition, Optional.of(dbLocation + tbWithPartition),
+                columns, partitionKeys,
+                new HashMap<>(), fileFormat, "");
         hmsClient.createTable(tableMetadata, true);
 
         // create table for tbWithoutPartition
-        HashMap<String, String> params2 = new HashMap<String, String>() {{
-                put("location_uri", dbLocation + tbWithPartition);
-            }};
         HiveTableMetadata tableMetadata2 = new HiveTableMetadata(
-                    dbName, tbWithoutPartition, columns, new ArrayList<>(),
-                    params2, fileFormat);
+                    dbName, tbWithoutPartition, Optional.of(dbLocation + tbWithPartition),
+                    columns, new ArrayList<>(),
+                    new HashMap<>(), fileFormat, "");
         hmsClient.createTable(tableMetadata2, true);
-
     }
 
     @After
diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java
index 5b8b374ad5a..c68715c17b7 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java
@@ -19,6 +19,7 @@ package org.apache.doris.qe;
 
 import org.apache.doris.analysis.AccessTestUtil;
 import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.DbName;
 import org.apache.doris.analysis.DescribeStmt;
 import org.apache.doris.analysis.HelpStmt;
 import org.apache.doris.analysis.SetType;
@@ -458,7 +459,7 @@ public class ShowExecutorTest {
         ctx.setEnv(env);
         ctx.setQualifiedUser("testUser");
 
-        ShowCreateDbStmt stmt = new ShowCreateDbStmt("testDb");
+        ShowCreateDbStmt stmt = new ShowCreateDbStmt(new DbName(InternalCatalog.INTERNAL_CATALOG_NAME, "testDb"));
         ShowExecutor executor = new ShowExecutor(ctx, stmt);
         ShowResultSet resultSet = executor.execute();
 
@@ -473,7 +474,7 @@ public class ShowExecutorTest {
         ctx.setEnv(env);
         ctx.setQualifiedUser("testUser");
 
-        ShowCreateDbStmt stmt = new ShowCreateDbStmt("emptyDb");
+        ShowCreateDbStmt stmt = new ShowCreateDbStmt(new DbName(InternalCatalog.INTERNAL_CATALOG_NAME, "emptyDb"));
         ShowExecutor executor = new ShowExecutor(ctx, stmt);
         executor.execute();
 
diff --git a/regression-test/data/external_table_p0/hive/ddl/test_hive_ctas.out b/regression-test/data/external_table_p0/hive/ddl/test_hive_ctas.out
new file mode 100644
index 00000000000..ccf3441c5c3
--- /dev/null
+++ b/regression-test/data/external_table_p0/hive/ddl/test_hive_ctas.out
@@ -0,0 +1,92 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !ctas_01 --
+2
+3
+
+-- !ctas_02 --
+2
+3
+
+-- !ctas_03 --
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_04 --
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_05 --
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_06 --
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_ex01 --
+2
+3
+
+-- !ctas_ex02 --
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	\N
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_03 --
+\N	another string value for col2
+\N	string value for col2
+\N	yet another string value for col2
+
+-- !ctas_04 --
+\N	11	value_for_pt1
+\N	22	value_for_pt11
+
+-- !ctas_types_01 --
+true	127	32767	2147483647	9223372036854775807	default	22.12345	3.141592653	99999.9999	default	default	2023-05-29	2023-05-29T23:19:34
+
+-- !ctas_types_02 --
+true	127	32767	2147483647	default	22.12345	3.141592653	99999.9999	default
+
+-- !ctas_01 --
+2
+3
+
+-- !ctas_02 --
+2
+3
+
+-- !ctas_03 --
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_04 --
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_05 --
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_06 --
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_ex01 --
+2
+3
+
+-- !ctas_ex02 --
+11	value_for_pt1	value_for_pt2
+22	value_for_pt11	\N
+22	value_for_pt11	value_for_pt22
+
+-- !ctas_03 --
+\N	another string value for col2
+\N	string value for col2
+\N	yet another string value for col2
+
+-- !ctas_04 --
+\N	11	value_for_pt1
+\N	22	value_for_pt11
+
+-- !ctas_types_01 --
+true	127	32767	2147483647	9223372036854775807	default	22.12345	3.141592653	99999.9999	default	default	2023-05-29	2023-05-29T23:19:34
+
+-- !ctas_types_02 --
+true	127	32767	2147483647	default	22.12345	3.141592653	99999.9999	default
diff --git a/regression-test/data/external_table_p0/hive/ddl/test_hive_ddl.out b/regression-test/data/external_table_p0/hive/ddl/test_hive_ddl.out
new file mode 100644
index 00000000000..eab813cf5f3
--- /dev/null
+++ b/regression-test/data/external_table_p0/hive/ddl/test_hive_ddl.out
@@ -0,0 +1,60 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !insert01 --
+true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
+
+-- !insert02 --
+\N	123	\N	\N	8.98
+true	123	9876543210	\N	stringval
+true	123	9876543210	123.4567	stringval
+
+-- !insert03 --
+\N	123	\N	\N	\N	\N	\N	varcharval	8.98
+false	1	9876543210	abcdefghij	2.3	6.28	0.0000	2223	stringval
+true	\N	9876543210	abcdefghij	2.3	6.28	\N	varcharval	stringval
+true	123	9876543210	\N	\N	\N	\N	varcharval	stringval
+true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
+
+-- !insert04 --
+true	1	1000	2.3	value_for_pt1	value_for_pt2
+
+-- !insert05 --
+true	1	1000	2.3
+true	1	1000	2.3
+true	1	1000	2.3
+
+-- !insert06 --
+\N	1	1000	1.3
+false	1	1000	\N
+true	1	1000	2.3
+true	1	1000	2.3
+true	1	1000	2.3
+
+-- !insert01 --
+true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
+
+-- !insert02 --
+\N	123	\N	\N	8.98
+true	123	9876543210	\N	stringval
+true	123	9876543210	123.4567	stringval
+
+-- !insert03 --
+\N	123	\N	\N	\N	\N	\N	varcharval	8.98
+false	1	9876543210	abcdefghij	2.3	6.28	0.0000	2223	stringval
+true	\N	9876543210	abcdefghij	2.3	6.28	\N	varcharval	stringval
+true	123	9876543210	\N	\N	\N	\N	varcharval	stringval
+true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
+
+-- !insert04 --
+true	1	1000	2.3	value_for_pt1	value_for_pt2
+
+-- !insert05 --
+true	1	1000	2.3
+true	1	1000	2.3
+true	1	1000	2.3
+
+-- !insert06 --
+\N	1	1000	1.3
+false	1	1000	\N
+true	1	1000	2.3
+true	1	1000	2.3
+true	1	1000	2.3
diff --git a/regression-test/data/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.out b/regression-test/data/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.out
deleted file mode 100644
index f30081e70b6..00000000000
--- a/regression-test/data/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.out
+++ /dev/null
@@ -1,148 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !insert01 --
-true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
-
--- !insert02 --
-\N	123	\N	\N	8.98
-true	123	9876543210	\N	stringval
-true	123	9876543210	123.4567	stringval
-
--- !insert03 --
-\N	123	\N	\N	\N	\N	\N	varcharval	8.98
-false	1	9876543210	abcdefghij	2.3	6.28	0.0000	2223	stringval
-true	\N	9876543210	abcdefghij	2.3	6.28	\N	varcharval	stringval
-true	123	9876543210	\N	\N	\N	\N	varcharval	stringval
-true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
-
--- !insert04 --
-true	1	1000	2.3	value_for_pt1	value_for_pt2
-
--- !insert05 --
-true	1	1000	2.3
-true	1	1000	2.3
-true	1	1000	2.3
-
--- !insert06 --
-\N	1	1000	1.3
-false	1	1000	\N
-true	1	1000	2.3
-true	1	1000	2.3
-true	1	1000	2.3
-
--- !ctas_01 --
-1
-2
-2
-3
-3
-
--- !ctas_02 --
-11	value_for_pt1	value_for_pt2
-22	value_for_pt11	value_for_pt22
-22	value_for_pt11	value_for_pt22
-
--- !ctas_03 --
-1	string value for col2
-1	string value for col2
-2	another string value for col2
-2	another string value for col2
-3	yet another string value for col2
-3	yet another string value for col2
-
--- !ctas_04 --
-11	value_for_pt1	value_for_pt2
-11	value_for_pt1	value_for_pt2
-22	value_for_pt11	value_for_pt22
-22	value_for_pt11	value_for_pt22
-
--- !ctas_05 --
-1	string value for col2
-2	another string value for col2
-3	yet another string value for col2
-
--- !ctas_06 --
-11	value_for_pt1	value_for_pt2
-22	value_for_pt11	value_for_pt22
-
--- !complex_type01 --
-a	\N	\N	\N	\N	\N	\N	\N	\N	\N	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	\N
-a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
-a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
-
--- !complex_type02 --
-a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
-a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
-
--- !insert01 --
-true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
-
--- !insert02 --
-\N	123	\N	\N	8.98
-true	123	9876543210	\N	stringval
-true	123	9876543210	123.4567	stringval
-
--- !insert03 --
-\N	123	\N	\N	\N	\N	\N	varcharval	8.98
-false	1	9876543210	abcdefghij	2.3	6.28	0.0000	2223	stringval
-true	\N	9876543210	abcdefghij	2.3	6.28	\N	varcharval	stringval
-true	123	9876543210	\N	\N	\N	\N	varcharval	stringval
-true	123	9876543210	abcdefghij	3.14	6.28	123.4567	varcharval	stringval
-
--- !insert04 --
-true	1	1000	2.3	value_for_pt1	value_for_pt2
-
--- !insert05 --
-true	1	1000	2.3
-true	1	1000	2.3
-true	1	1000	2.3
-
--- !insert06 --
-\N	1	1000	1.3
-false	1	1000	\N
-true	1	1000	2.3
-true	1	1000	2.3
-true	1	1000	2.3
-
--- !ctas_01 --
-1
-2
-2
-3
-3
-
--- !ctas_02 --
-11	value_for_pt1	value_for_pt2
-22	value_for_pt11	value_for_pt22
-22	value_for_pt11	value_for_pt22
-
--- !ctas_03 --
-1	string value for col2
-1	string value for col2
-2	another string value for col2
-2	another string value for col2
-3	yet another string value for col2
-3	yet another string value for col2
-
--- !ctas_04 --
-11	value_for_pt1	value_for_pt2
-11	value_for_pt1	value_for_pt2
-22	value_for_pt11	value_for_pt22
-22	value_for_pt11	value_for_pt22
-
--- !ctas_05 --
-1	string value for col2
-2	another string value for col2
-3	yet another string value for col2
-
--- !ctas_06 --
-11	value_for_pt1	value_for_pt2
-22	value_for_pt11	value_for_pt22
-
--- !complex_type01 --
-a	\N	\N	\N	\N	\N	\N	\N	\N	\N	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	\N
-a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
-a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
-
--- !complex_type02 --
-a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
-a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
diff --git a/regression-test/data/external_table_p0/hive/ddl/test_hive_write_type.out b/regression-test/data/external_table_p0/hive/ddl/test_hive_write_type.out
new file mode 100644
index 00000000000..a95bbd0b8d4
--- /dev/null
+++ b/regression-test/data/external_table_p0/hive/ddl/test_hive_write_type.out
@@ -0,0 +1,18 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !complex_type01 --
+a	\N	\N	\N	\N	\N	\N	\N	\N	\N	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
+
+-- !complex_type02 --
+a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
+
+-- !complex_type01 --
+a	\N	\N	\N	\N	\N	\N	\N	\N	\N	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
+
+-- !complex_type02 --
+a	b	c	d	e	1.1	12345	0.12345678	string	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
+a	b	c	d	e	1.1	12345	0.12345678	string	[0.001, 0.002]	["char1", "char2"]	["c", "d"]	["string1", "string2"]	[{1:"a"}, {2:"b"}]	{1234567890123456789:"a"}	{1234567890123456789:0.12345678}	{"key":["char1", "char2"]}	{"id": 1, "gender": 1, "name": "John Doe"}	{"scale": 123.4567, "metric": ["metric1", "metric2"]}	{"codes": [123, 456], "props": {"key1":["char1", "char2"]}}
diff --git a/regression-test/suites/external_table_p0/hive/ddl/test_hive_ctas.groovy b/regression-test/suites/external_table_p0/hive/ddl/test_hive_ctas.groovy
new file mode 100644
index 00000000000..1c69004867f
--- /dev/null
+++ b/regression-test/suites/external_table_p0/hive/ddl/test_hive_ctas.groovy
@@ -0,0 +1,515 @@
+// 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_hive_ctas", "p0,external,hive,external_docker,external_docker_hive") {
+    String enabled = context.config.otherConfigs.get("enableHiveTest")
+    if (enabled != null && enabled.equalsIgnoreCase("true")) {
+        def file_formats = ["parquet", "orc"]
+        def generateSrcDDLForCTAS = { String file_format, String catalog_name ->
+            sql """ switch `${catalog_name}` """
+            sql """ create database if not exists `test_ctas` """;
+            sql """ switch internal """
+            sql """ create database if not exists test_ctas_olap """;
+            sql """ use internal.test_ctas_olap """
+            sql """
+                CREATE TABLE `unpart_ctas_olap_src` (
+                    `col1` INT COMMENT 'col1',
+                    `col2` STRING COMMENT 'col2'
+                )
+                ENGINE=olap
+                DISTRIBUTED BY HASH(col1) BUCKETS 16
+                PROPERTIES (
+                    'replication_num' = '1'
+                );
+            """
+
+            sql """ INSERT INTO `unpart_ctas_olap_src` (col1, col2) VALUES
+                (1, 'string value for col2'),
+                (2, 'another string value for col2'),
+                (3, 'yet another string value for col2'); 
+            """
+
+            sql """
+                CREATE TABLE `part_ctas_olap_src`(
+                    `col1` INT COMMENT 'col1',
+                    `pt1` VARCHAR(16) COMMENT 'pt1',
+                    `pt2` VARCHAR(16) COMMENT 'pt2'
+                )
+                ENGINE=olap
+                PARTITION BY LIST (pt1, pt2) (
+                    PARTITION pp1 VALUES IN(
+                        ('value_for_pt1', 'value_for_pt2'),
+                        ('value_for_pt11', 'value_for_pt22')
+                    )
+                )
+                DISTRIBUTED BY HASH(col1) BUCKETS 16
+                PROPERTIES (
+                    'replication_num' = '1'
+                );
+            """
+
+            sql """
+            INSERT INTO `part_ctas_olap_src` (col1, pt1, pt2) VALUES
+             (11, 'value_for_pt1', 'value_for_pt2'),
+             (22, 'value_for_pt11', 'value_for_pt22');
+            """
+
+            sql """ use `${catalog_name}`.`test_ctas` """
+            sql """
+                CREATE TABLE `unpart_ctas_src`(
+                  `col1` INT COMMENT 'col1',
+                  `col2` STRING COMMENT 'col2'
+                ) ENGINE=hive
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                );
+            """
+
+            sql """ INSERT INTO `unpart_ctas_src` (col1, col2) VALUES
+                (1, 'string value for col2'),
+                (2, 'another string value for col2'),
+                (3, 'yet another string value for col2'); 
+            """
+
+            sql """
+                CREATE TABLE `part_ctas_src`(
+                  `col1` INT COMMENT 'col1',
+                  `pt1` VARCHAR COMMENT 'pt1',
+                  `pt2` VARCHAR COMMENT 'pt2'
+                ) ENGINE=hive
+                PARTITION BY LIST (pt1, pt2) (
+                    
+                )
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                );
+            """
+
+            sql """
+            INSERT INTO `part_ctas_src` (col1, pt1, pt2) VALUES
+             (11, 'value_for_pt1', 'value_for_pt2'),
+             (22, 'value_for_pt11', 'value_for_pt22');
+            """
+        }
+
+        def destroySrcDDLForCTAS = { String catalog_name ->
+            sql """ switch `${catalog_name}` """
+            sql """ DROP TABLE IF EXISTS `test_ctas`.part_ctas_src """
+            sql """ DROP TABLE IF EXISTS `test_ctas`.unpart_ctas_src """
+            sql """ drop database if exists `test_ctas` """;
+            sql """ DROP TABLE IF EXISTS internal.test_ctas_olap.part_ctas_olap_src """
+            sql """ DROP TABLE IF EXISTS internal.test_ctas_olap.unpart_ctas_olap_src """
+            sql """ switch internal """;
+            sql """ drop database if exists test_ctas_olap """;
+        }
+
+        def test_ctas_tbl = { String file_format, String catalog_name ->
+            generateSrcDDLForCTAS(file_format, catalog_name)
+            try {
+                sql """ switch `${catalog_name}` """
+                sql """ use test_ctas """
+                // 1. external to external un-partitioned table
+                sql """ CREATE TABLE hive_ctas1 ENGINE=hive AS SELECT col1 FROM unpart_ctas_src; 
+                """
+
+                sql """ INSERT INTO hive_ctas1 SELECT col1 FROM unpart_ctas_src WHERE col1 > 1;
+                """
+                sql """ INSERT OVERWRITE TABLE hive_ctas1 SELECT col1 FROM unpart_ctas_src WHERE col1 > 1;
+                """
+
+                order_qt_ctas_01 """ SELECT * FROM hive_ctas1 """
+                sql """ DROP TABLE hive_ctas1 """
+
+                // 2. external to external un-partitioned table with columns
+                sql """ CREATE TABLE hive_ctas2 (col1) ENGINE=hive AS SELECT col1 FROM unpart_ctas_src; 
+                """
+
+                sql """ INSERT INTO hive_ctas2 SELECT col1 FROM unpart_ctas_src WHERE col1 > 1;
+                """
+                sql """ INSERT OVERWRITE TABLE hive_ctas2 SELECT col1 FROM unpart_ctas_src WHERE col1 > 1;
+                """
+                order_qt_ctas_02 """ SELECT * FROM hive_ctas2  """
+                sql """ DROP TABLE hive_ctas2 """
+
+                // 3. external to external partitioned table
+                sql """ CREATE TABLE hive_ctas3 ENGINE=hive AS SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>0;
+                """
+
+                sql """ INSERT INTO hive_ctas3 SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>=22;
+                """
+                sql """ INSERT OVERWRITE TABLE hive_ctas3 SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>=22;
+                """
+                order_qt_ctas_03 """ SELECT * FROM hive_ctas3  """
+                sql """ DROP TABLE hive_ctas3 """
+
+                sql """ CREATE TABLE hive_ctas4 AS SELECT * FROM part_ctas_src WHERE col1>0;
+                """
+
+                sql """ INSERT INTO hive_ctas4 SELECT * FROM part_ctas_src WHERE col1>=22;
+                """
+                sql """ INSERT OVERWRITE TABLE hive_ctas4 SELECT * FROM part_ctas_src WHERE col1>=22;
+                """
+                order_qt_ctas_04 """ SELECT * FROM ${catalog_name}.test_ctas.hive_ctas4  """
+                sql """ DROP TABLE hive_ctas4 """
+
+                // 4. external to external partitioned table with partitions and cols
+                sql """ CREATE TABLE hive_ctas5 ENGINE=hive PARTITION BY LIST (pt1, pt2) ()
+                AS SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>0;
+                """
+
+                sql """ INSERT INTO hive_ctas5 SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>=22;
+                """
+                sql """ INSERT OVERWRITE TABLE hive_ctas5 SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>=22;
+                """
+                order_qt_ctas_05 """ SELECT * FROM hive_ctas5  """
+                sql """ DROP TABLE hive_ctas5 """
+
+                sql """ CREATE TABLE hive_ctas6 PARTITION BY LIST (pt1, pt2) ()
+                AS SELECT * FROM part_ctas_src WHERE col1>0;
+                """
+
+                sql """ INSERT INTO hive_ctas6 SELECT * FROM part_ctas_src WHERE col1>=22;
+                """
+                sql """ INSERT OVERWRITE TABLE hive_ctas6 SELECT * FROM part_ctas_src WHERE col1>=22;
+                """
+                order_qt_ctas_06 """ SELECT * FROM ${catalog_name}.test_ctas.hive_ctas6  """
+                sql """ DROP TABLE hive_ctas6 """
+
+            } finally {
+                destroySrcDDLForCTAS(catalog_name)
+            }
+        }
+
+        def test_ctas_extend = { String file_format, String catalog_name ->
+            generateSrcDDLForCTAS(file_format, catalog_name)
+            sql """ switch ${catalog_name} """
+
+            try {
+                sql """ DROP DATABASE IF EXISTS ${catalog_name}.test_ctas_ex """;
+                sql """ DROP DATABASE IF EXISTS `test_ctas_ex` """;
+                sql """ CREATE DATABASE IF NOT EXISTS ${catalog_name}.test_ctas_ex
+                    PROPERTIES (
+                        "location" = "/user/hive/warehouse/test_ctas_ex"
+                    )
+                    """;
+                sql """ CREATE DATABASE IF NOT EXISTS `test_ctas_ex`
+                    PROPERTIES (
+                        "location" = "/user/hive/warehouse/test_ctas_ex"
+                    )
+                    """;
+                sql """ use `${catalog_name}`.`test_ctas_ex` """
+
+                // 1. external to external un-partitioned table
+                sql """ DROP TABLE IF EXISTS ${catalog_name}.test_ctas_ex.hive_ctas1 """
+                sql """ CREATE TABLE ${catalog_name}.test_ctas_ex.hive_ctas1 (col1) ENGINE=hive 
+                        PROPERTIES (
+                            "location" = "/user/hive/warehouse/test_ctas_ex/loc_hive_ctas1",
+                            "file_format"="orc",
+                            "orc.compress"="zlib"
+                        ) AS SELECT col1 FROM test_ctas.unpart_ctas_src; 
+                    """
+                sql """ INSERT INTO ${catalog_name}.test_ctas_ex.hive_ctas1
+                        SELECT col1 FROM test_ctas.unpart_ctas_src WHERE col1 > 1;
+                    """
+                sql """ INSERT OVERWRITE TABLE ${catalog_name}.test_ctas_ex.hive_ctas1
+                        SELECT col1 FROM test_ctas.unpart_ctas_src WHERE col1 > 1;
+                    """
+                order_qt_ctas_ex01 """ SELECT * FROM hive_ctas1 """
+                sql """ DROP TABLE hive_ctas1 """
+
+                // 2. external to external partitioned table
+                sql """ DROP TABLE IF EXISTS ${catalog_name}.test_ctas_ex.hive_ctas2 """
+                sql """ CREATE TABLE ${catalog_name}.test_ctas_ex.hive_ctas2 (col1,pt1,pt2) ENGINE=hive 
+                        PARTITION BY LIST (pt1, pt2) ()
+                        PROPERTIES (
+                            "location" = "/user/hive/warehouse/test_ctas_ex/loc_hive_ctas2",
+                            "file_format"="parquet",
+                            "parquet.compression"="snappy"
+                        )
+                        AS SELECT col1,pt1,pt2 FROM test_ctas.part_ctas_src WHERE col1>0; 
+                    """
+                sql """ INSERT INTO ${catalog_name}.test_ctas_ex.hive_ctas2 (col1,pt1,pt2)
+                        SELECT col1,pt1,pt2 FROM test_ctas.part_ctas_src WHERE col1>=22;
+                    """
+                sql """ INSERT OVERWRITE TABLE hive_ctas2 (col1,pt1,pt2)
+                        SELECT col1,pt1,pt2 FROM test_ctas.part_ctas_src WHERE col1>=22;
+                    """
+                sql """ INSERT INTO ${catalog_name}.test_ctas_ex.hive_ctas2 (pt1,col1)
+                        SELECT pt1,col1 FROM test_ctas.part_ctas_src WHERE col1>=22;
+                    """
+                order_qt_ctas_ex02 """ SELECT * FROM hive_ctas2  """
+                sql """ DROP TABLE hive_ctas2 """
+
+                // 3. internal to external un-partitioned table
+                sql """ DROP TABLE IF EXISTS ${catalog_name}.test_ctas_ex.ctas_o1 """
+                sql """ CREATE TABLE ${catalog_name}.test_ctas_ex.ctas_o1 (col1,col2) ENGINE=hive
+                        PROPERTIES (
+                            "location" = "/user/hive/warehouse/test_ctas_ex/loc_ctas_o1",
+                            "file_format"="parquet",
+                            "parquet.compression"="snappy"
+                        )
+                        AS SELECT col1,col2 FROM internal.test_ctas_olap.unpart_ctas_olap_src;
+                """
+                sql """ INSERT INTO ${catalog_name}.test_ctas_ex.ctas_o1 (col2,col1)
+                        SELECT col2,col1 FROM internal.test_ctas_olap.unpart_ctas_olap_src;
+                """
+                sql """ INSERT OVERWRITE TABLE ${catalog_name}.test_ctas_ex.ctas_o1 (col2)
+                        SELECT col2 FROM internal.test_ctas_olap.unpart_ctas_olap_src;
+                """
+                order_qt_ctas_03 """ SELECT * FROM ctas_o1  """
+                sql """ DROP TABLE ctas_o1 """
+
+                // 4. internal to external partitioned table
+                sql """ DROP TABLE IF EXISTS ${catalog_name}.test_ctas_ex.ctas_o2 """
+                sql """ CREATE TABLE ${catalog_name}.test_ctas_ex.ctas_o2 (col1,col2,pt1) ENGINE=hive
+                        PARTITION BY LIST (pt1) ()
+                        PROPERTIES (
+                            "location" = "/user/hive/warehouse/test_ctas_ex/loc_ctas_o2",
+                            "file_format"="orc",
+                            "orc.compress"="zlib"
+                        ) 
+                        AS SELECT null as col1, pt2 as col2, pt1 FROM internal.test_ctas_olap.part_ctas_olap_src WHERE col1>0;
+                    """
+                sql """ INSERT INTO ${catalog_name}.test_ctas_ex.ctas_o2 (col1,pt1,col2)
+                        SELECT col1,pt1,pt2 FROM internal.test_ctas_olap.part_ctas_olap_src;
+                """
+                sql """ INSERT INTO ${catalog_name}.test_ctas_ex.ctas_o2 (col2,pt1,col1)
+                        SELECT pt2,pt1,col1 FROM internal.test_ctas_olap.part_ctas_olap_src;
+                """
+                sql """ INSERT OVERWRITE TABLE ${catalog_name}.test_ctas_ex.ctas_o2 (pt1,col2)
+                        SELECT pt1,col1 FROM internal.test_ctas_olap.part_ctas_olap_src;
+                """
+                order_qt_ctas_04 """ SELECT * FROM ctas_o2  """
+                sql """ DROP TABLE ctas_o2 """
+                sql """ DROP DATABASE IF EXISTS test_ctas_ex """
+            } finally {
+                destroySrcDDLForCTAS(catalog_name)
+            }
+        }
+
+        def test_ctas_exception = { String file_format, String catalog_name ->
+            sql """ switch ${catalog_name} """
+
+            sql """ create database if not exists `test_hive_ex_ctas` """;
+            test {
+                sql """ create database `test_hive_ex_ctas` """
+                exception "errCode = 2, detailMessage = Can't create database 'test_hive_ex_ctas'; database exists"
+            }
+            sql """use `${catalog_name}`.`test_hive_ex_ctas`"""
+            sql """ DROP DATABASE IF EXISTS ${catalog_name}.test_hive_ex_ctas """
+            // check ctas error
+            generateSrcDDLForCTAS(file_format, catalog_name)
+            try {
+                test {
+                    sql """ DROP DATABASE ${catalog_name}.test_no_exist """
+                    exception "errCode = 2, detailMessage = Can't drop database 'test_no_exist'; database doesn't exist"
+                }
+                sql """ DROP DATABASE IF EXISTS ${catalog_name}.test_err """
+                sql """ CREATE DATABASE ${catalog_name}.test_err """
+                test {
+                    sql """ CREATE DATABASE ${catalog_name}.test_err
+                    PROPERTIES (
+                        "location" = "/user/hive/warehouse/test_err",
+                        "owner" = "err"
+                    )
+                    """;
+                    exception "errCode = 2, detailMessage = Can't create database 'test_err'; database exists"
+                }
+                sql """ DROP DATABASE IF EXISTS ${catalog_name}.test_err """
+
+                sql """ CREATE DATABASE IF NOT EXISTS `test_no_err`""";
+                sql """ use `${catalog_name}`.`test_no_err` """
+
+                // 1. external to external un-partitioned table
+                test {
+                    sql """ DROP TABLE ${catalog_name}.test_no_err.hive_ctas1 """
+                    exception "errCode = 2, detailMessage = Unknown table 'hive_ctas1' in test_no_err"
+                }
+                test {
+                    sql """ CREATE TABLE ${catalog_name}.test_no_err.hive_ctas1 (col1) ENGINE=hive 
+                            PROPERTIES (
+                                "file_format"="orc",
+                                "orc.compress"="zstd"
+                            ) AS SELECT col1,col2 FROM test_ctas.unpart_ctas_src; 
+                        """
+                    exception "errCode = 2, detailMessage = ctas column size is not equal to the query's"
+                }
+
+                test {
+                    sql """ CREATE TABLE ${catalog_name}.test_no_err.ctas_o2 (col1,pt1,pt2) ENGINE=hive 
+                        PARTITION BY LIST (pt1,pt2,pt3) ()
+                        PROPERTIES (
+                            "file_format"="parquet",
+                            "orc.compress"="zstd"
+                        )
+                        AS SELECT * FROM test_ctas.part_ctas_src WHERE col1>0; 
+                    """
+                    exception "errCode = 2, detailMessage = partition key pt3 is not exists"
+                }
+
+                sql """ DROP TABLE IF EXISTS ${catalog_name}.test_no_err.ctas_o2 """
+                sql """ CREATE TABLE ${catalog_name}.test_no_err.ctas_o2 (col1,col2,pt1) ENGINE=hive 
+                        PARTITION BY LIST (pt1) ()
+                        PROPERTIES (
+                            "file_format"="parquet",
+                            "parquet.compression"="zstd"
+                        )
+                        AS SELECT col1,pt1 as col2,pt2 as pt1 FROM test_ctas.part_ctas_src WHERE col1>0; 
+                    """
+
+                test {
+                    sql """ INSERT INTO ${catalog_name}.test_no_err.ctas_o2 (col1,col2)
+                            SELECT col1 FROM test_ctas.part_ctas_src;
+                        """
+                    exception "errCode = 2, detailMessage = insert into cols should be corresponding to the query output"
+                }
+
+                test {
+                    sql """ INSERT INTO ${catalog_name}.test_no_err.ctas_o2 (col1)
+                            SELECT col1,pt1 as col2 FROM test_ctas.part_ctas_src WHERE col1>0;
+                        """
+                    exception "errCode = 2, detailMessage = insert into cols should be corresponding to the query output"
+                }
+                sql """ DROP TABLE IF EXISTS ${catalog_name}.test_no_err.ctas_o2  """
+                sql """ DROP DATABASE IF EXISTS test_no_err """
+
+            } finally {
+                destroySrcDDLForCTAS(catalog_name)
+            }
+        }
+
+        def test_ctas_all_types = { String file_format, String catalog_name ->
+            sql """ switch `${catalog_name}` """
+            sql """ CREATE DATABASE IF NOT EXISTS `test_ctas_all_type` """;
+            sql """ use test_ctas_all_type """;
+
+            sql """
+                CREATE TABLE IF NOT EXISTS all_types_ctas_${file_format}(
+                  `col1` BOOLEAN DEFAULT 'false' COMMENT 'col1',
+                  `col2` TINYINT DEFAULT '127' COMMENT 'col2',
+                  `col3` SMALLINT DEFAULT '32767' COMMENT 'col3',
+                  `col4` INT DEFAULT '2147483647' COMMENT 'col4',
+                  `col5` BIGINT DEFAULT '9223372036854775807' COMMENT 'col5',
+                  `col6` CHAR(10) DEFAULT 'default' COMMENT 'col6',
+                  `col7` FLOAT DEFAULT '1' COMMENT 'col7',
+                  `col8` DOUBLE DEFAULT '3.141592653' COMMENT 'col8',
+                  `col9` DECIMAL(9,4) DEFAULT '99999.9999' COMMENT 'col9',
+                  `col10` VARCHAR(11) DEFAULT 'default' COMMENT 'col10',
+                  `col11` STRING DEFAULT 'default' COMMENT 'col11',
+                  `col12` DATE DEFAULT '2023-05-29' COMMENT 'col12',
+                  `col13` DATETIME DEFAULT current_timestamp COMMENT 'col13'
+                )  ENGINE=hive
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+                """
+
+            sql """
+                    INSERT INTO all_types_ctas_${file_format} (
+                        col1,
+                        col2,
+                        col3,
+                        col4,
+                        col5,
+                        col6,
+                        col7,
+                        col8,
+                        col9,
+                        col10,
+                        col11,
+                        col12,
+                        col13
+                    ) VALUES (
+                        true,        -- col1 (BOOLEAN)
+                        127,         -- col2 (TINYINT)
+                        32767,       -- col3 (SMALLINT)
+                        2147483647,  -- col4 (INT)
+                        9223372036854775807, -- col5 (BIGINT)
+                        'default',   -- col6 (CHAR)
+                        22.12345,         -- col7 (FLOAT)
+                        3.141592653, -- col8 (DOUBLE)
+                        99999.9999,  -- col9 (DECIMAL)
+                        'default',   -- col10 (VARCHAR)
+                        'default',   -- col11 (STRING)
+                        '2023-05-29',-- col12 (DATE)
+                        '2023-05-29 23:19:34' -- col13 (DATETIME)
+                    );
+                """
+
+            sql """
+                    CREATE TABLE IF NOT EXISTS all_types_ctas1 AS SELECT * FROM all_types_ctas_${file_format}
+                """
+            sql """
+                    INSERT INTO all_types_ctas1 SELECT * FROM all_types_ctas_${file_format}
+                """
+            sql """
+                    INSERT OVERWRITE TABLE all_types_ctas1 SELECT * FROM all_types_ctas_${file_format}
+                """
+            order_qt_ctas_types_01 """ SELECT * FROM all_types_ctas1 """
+            sql """
+                    DROP TABLE all_types_ctas1
+                """
+
+            sql """
+                    CREATE TABLE IF NOT EXISTS all_types_ctas2 (col1, col2, col3, col4, col6, col7, col8, col9, col11) 
+                    AS SELECT col1, col2, col3, col4, col6, col7, col8, col9, col11 FROM all_types_ctas_${file_format}
+                """
+            sql """
+                    INSERT INTO all_types_ctas2 (col1, col3, col7, col9) 
+                    SELECT col1, col3, col7, col9 FROM all_types_ctas_${file_format}
+                """
+            sql """
+                    INSERT OVERWRITE TABLE all_types_ctas2 (col1, col2, col3, col4, col6, col7, col8, col9, col11)
+                    SELECT col1, col2, col3, col4, col6, col7, col8, col9, col11 FROM all_types_ctas_${file_format}
+                """
+            order_qt_ctas_types_02 """ SELECT * FROM all_types_ctas2 """
+            sql """
+                DROP TABLE all_types_ctas2
+                """
+            sql """
+                DROP TABLE all_types_ctas_${file_format}
+                """
+            sql """ drop database if exists `test_ctas_all_type` """;
+        }
+
+        try {
+            String hms_port = context.config.otherConfigs.get("hms_port")
+            String hdfs_port = context.config.otherConfigs.get("hdfs_port")
+            String catalog_name = "test_hive_ctas"
+            String externalEnvIp = context.config.otherConfigs.get("externalEnvIp")
+
+            sql """drop catalog if exists ${catalog_name}"""
+            sql """create catalog if not exists ${catalog_name} properties (
+                'type'='hms',
+                'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}',
+                'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}'
+            );"""
+            sql """switch ${catalog_name}"""
+
+            sql """set enable_fallback_to_original_planner=false;"""
+            for (String file_format in file_formats) {
+                logger.info("Process file format" + file_format)
+                test_ctas_tbl(file_format, catalog_name)
+                test_ctas_extend(file_format, catalog_name)
+                test_ctas_exception(file_format, catalog_name)
+                test_ctas_all_types(file_format, catalog_name)
+            }
+            sql """drop catalog if exists ${catalog_name}"""
+        } finally {
+        }
+    }
+}
diff --git a/regression-test/suites/external_table_p0/hive/ddl/test_hive_ddl.groovy b/regression-test/suites/external_table_p0/hive/ddl/test_hive_ddl.groovy
new file mode 100644
index 00000000000..b0543931371
--- /dev/null
+++ b/regression-test/suites/external_table_p0/hive/ddl/test_hive_ddl.groovy
@@ -0,0 +1,676 @@
+// 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_hive_ddl", "p0,external,hive,external_docker,external_docker_hive") {
+    String enabled = context.config.otherConfigs.get("enableHiveTest")
+    if (enabled != null && enabled.equalsIgnoreCase("true")) {
+        def file_formats = ["parquet", "orc"]
+
+        def test_db = { String catalog_name ->
+            logger.info("Test create/drop database...")
+            sql """switch ${catalog_name}"""
+            sql """ drop database if exists `test_hive_db` """;
+            sql """ create database if not exists ${catalog_name}.`test_hive_db` """;
+            def create_db_res = sql """ show create database test_hive_db """
+            logger.info("${create_db_res}")
+            assertTrue(create_db_res.toString().containsIgnoreCase("/user/hive/warehouse/test_hive_db.db"))
+            sql """ use `test_hive_db` """
+            sql """
+                    CREATE TABLE test_hive_db_has_tbl (
+                      `col` STRING COMMENT 'col'
+                    )  ENGINE=hive
+                 """
+            test {
+                sql """ drop database `test_hive_db` """;
+                exception "java.sql.SQLException: Unexpected exception: failed to drop database from hms client. reason: org.apache.hadoop.hive.metastore.api.InvalidOperationException: Database test_hive_db is not empty. One or more tables exist."
+            }
+
+            sql """ DROP TABLE `test_hive_db_has_tbl` """
+            sql """ drop database `test_hive_db` """;
+            sql """ drop database if exists `test_hive_db` """;
+        }
+
+        def test_loc_db = { String externalEnvIp, String hdfs_port, String catalog_name ->
+            logger.info("Test create/drop database with location...")
+            sql """switch ${catalog_name}"""
+            def loc = "${externalEnvIp}:${hdfs_port}/tmp/hive/test_hive_loc_db"
+            sql """ create database if not exists `test_hive_loc_db`
+                    properties('location'='hdfs://${loc}')
+                """
+
+            def create_db_res = sql """show create database test_hive_loc_db"""
+            logger.info("${create_db_res}")
+            assertTrue(create_db_res.toString().containsIgnoreCase("${loc}"))
+
+            sql """use `test_hive_loc_db`"""
+            sql """ drop database if exists `test_hive_loc_db` """;
+
+            sql """ create database if not exists `test_hive_loc_no_exist`
+                properties('location'='hdfs://${externalEnvIp}:${hdfs_port}/exist_check')
+            """
+            sql """ create database if not exists `test_hive_loc_exist`
+                properties('location'='hdfs://${externalEnvIp}:${hdfs_port}/exist_check')
+            """
+            sql """ drop database if exists `test_hive_loc_no_exist` """;
+            sql """ drop database if exists `test_hive_loc_exist` """;
+
+            try {
+                sql """ create database if not exists `test_hive_loc_no_exist`
+                        properties('location'='tt://${externalEnvIp}:${hdfs_port}/exist_check')
+                    """
+            } catch (Exception e) {
+                assertTrue(e.getMessage().contains("No FileSystem for scheme: tt"))
+            }
+            try {
+                sql """ create database if not exists `test_hive_loc_no_exist`
+                        properties('location'='hdfs://err_${externalEnvIp}:${hdfs_port}/exist_check')
+                    """
+            } catch (Exception e) {
+                assertTrue(e.getMessage().contains("Incomplete HDFS URI, no host"))
+            }
+            try {
+                sql """ create database if not exists `test_hive_loc_no_exist`
+                        properties('location'='hdfs:err//${externalEnvIp}:${hdfs_port}/exist_check')
+                    """
+            } catch (Exception e) {
+                assertTrue(e.getMessage().contains("Relative path in absolute URI"))
+            }
+        }
+
+        def test_loc_tbl = { String file_format, String externalEnvIp, String hdfs_port, String catalog_name ->
+            logger.info("Test create/drop table with location...")
+            sql """switch ${catalog_name}"""
+            def loc = "${externalEnvIp}:${hdfs_port}/tmp/hive/test_hive_loc_db"
+            sql  """ create database if not exists `test_hive_loc`
+                    properties('location'='hdfs://${loc}')
+                 """
+            sql """use `test_hive_loc`"""
+
+            // case1. the table default location is inherited from db
+            sql """DROP TABLE IF EXISTS `loc_tbl_${file_format}_default`"""
+            sql """
+                    CREATE TABLE loc_tbl_${file_format}_default (
+                      `col` STRING COMMENT 'col'
+                    )  ENGINE=hive 
+                    PROPERTIES (
+                      'file_format'='${file_format}'
+                    )
+                 """
+            def create_tbl_res = sql """ show create table loc_tbl_${file_format}_default """
+            logger.info("${create_tbl_res}")
+            assertTrue(create_tbl_res.toString().containsIgnoreCase("${loc}/loc_tbl_${file_format}_default"))
+
+            sql """ INSERT INTO loc_tbl_${file_format}_default values(1)  """
+
+            def tvfRes = sql """ SELECT * FROM hdfs(
+                                  'uri'='hdfs://${loc}/loc_tbl_${file_format}_default/*',
+                                  'format' = '${file_format}',
+                                  'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}'
+                                 )
+                             """
+            logger.info("${tvfRes}")
+            assertTrue(!tvfRes.isEmpty())
+            sql """DROP TABLE `loc_tbl_${file_format}_default`"""
+            def tvfDropRes = sql """ SELECT * FROM hdfs(
+                                  'uri'='hdfs://${loc}/loc_tbl_${file_format}_default/*',
+                                  'format' = '${file_format}',
+                                  'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}'
+                                 )
+                             """
+            logger.info("${tvfDropRes}")
+            assertTrue(tvfDropRes.isEmpty())
+
+            // case2. use a custom location to create table
+            def tbl_loc = "hdfs://${loc}/custom_loc"
+            sql """ DROP TABLE IF EXISTS loc_tbl_${file_format}_customm"""
+            sql """
+                    CREATE TABLE loc_tbl_${file_format}_custom (
+                      `col` STRING COMMENT 'col'
+                    )  ENGINE=hive 
+                    PROPERTIES (
+                      'file_format'='${file_format}',
+                      'location'='${tbl_loc}'
+                    )
+                 """
+            def create_tbl_res2 = sql """ show create table loc_tbl_${file_format}_custom """
+            logger.info("${create_tbl_res2}")
+            assertTrue(create_tbl_res2.toString().containsIgnoreCase("${tbl_loc}"))
+            sql """ INSERT INTO loc_tbl_${file_format}_custom values(1)  """
+            def tvfRes2 = sql """ SELECT * FROM hdfs(
+                                    'uri'='${tbl_loc}/*',
+                                    'format' = '${file_format}',
+                                    'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}'
+                                  )
+                              """
+            logger.info("${tvfRes2}")
+            assertTrue(!tvfRes2.isEmpty())
+            sql """DROP TABLE `loc_tbl_${file_format}_custom`"""
+            def tvfDropRes2 = sql """ SELECT * FROM hdfs(
+                                        'uri'='${tbl_loc}/*',
+                                        'format' = '${file_format}',
+                                        'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}'
+                                      )
+                                  """
+            logger.info("${tvfDropRes2}")
+            assertTrue(tvfDropRes2.isEmpty())
+
+            // case3. check default
+            sql """
+                CREATE TABLE all_default_values_${file_format}(
+                  `col1` BOOLEAN DEFAULT 'false' COMMENT 'col1',
+                  `col2` TINYINT DEFAULT '127' COMMENT 'col2',
+                  `col3` SMALLINT DEFAULT '32767' COMMENT 'col3',
+                  `col4` INT DEFAULT '2147483647' COMMENT 'col4',
+                  `col5` BIGINT DEFAULT '9223372036854775807' COMMENT 'col5',
+                  `col6` CHAR(10) DEFAULT 'default' COMMENT 'col6',
+                  `col7` FLOAT DEFAULT '1' COMMENT 'col7',
+                  `col8` DOUBLE DEFAULT '3.141592653' COMMENT 'col8',
+                  `col9` DECIMAL(9,4) DEFAULT '99999.9999' COMMENT 'col9',
+                  `col10` VARCHAR(11) DEFAULT 'default' COMMENT 'col10',
+                  `col11` STRING DEFAULT 'default' COMMENT 'col11',
+                  `col12` DATE DEFAULT '2023-05-29' COMMENT 'col12',
+                  `col13` DATETIME DEFAULT current_timestamp COMMENT 'col13'
+                )  ENGINE=hive
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+                """
+            // need support default insert:
+            //            sql """ INSERT INTO all_default_values_${file_format}
+            //                    VALUES(null, null, null, null, null, null, null, null, null, null, null, null, null)
+            //                """
+            //            sql """ INSERT INTO all_default_values_${file_format} (col1, col3, col5, col7, col12)
+            //                    VALUES(null, null, null, null)
+            //                """
+            //            order_qt_default_val01 """ SELECT * FROM all_default_values_${file_format} """
+            sql """DROP TABLE `all_default_values_${file_format}`"""
+
+            test {
+                sql """
+                CREATE TABLE all_default_values_${file_format}_err_bool(
+                  `col1` BOOLEAN DEFAULT '-1' COMMENT 'col1'
+                )  ENGINE=hive 
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+                """
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = Invalid BOOLEAN literal: -1"
+            }
+
+            test {
+                sql """
+                CREATE TABLE all_default_values_${file_format}_err_float(
+                  `col1` FLOAT DEFAULT '1.1234' COMMENT 'col1'
+                )  ENGINE=hive 
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+                """
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = Default value will loose precision: 1.1234"
+            }
+
+            test {
+                sql """
+                CREATE TABLE all_default_values_${file_format}_err_double(
+                  `col1` DOUBLE DEFAULT 'abc' COMMENT 'col1'
+                )  ENGINE=hive 
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+                """
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = Invalid floating-point literal: abc"
+            }
+
+            test {
+                sql """
+                CREATE TABLE all_default_values_${file_format}_err_int(
+                  `col1` INT DEFAULT 'abcd' COMMENT 'col1'
+                )  ENGINE=hive 
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+                """
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = Invalid number format: abcd"
+            }
+
+            test {
+                sql """
+                CREATE TABLE all_default_values_${file_format}_err_date(
+                  `col1` DATE DEFAULT '123' COMMENT 'col1'
+                )  ENGINE=hive 
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+                """
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = date literal [123] is invalid: null"
+            }
+
+            test {
+                sql """
+                CREATE TABLE all_default_values_${file_format}_err_datetime(
+                   `col1` DATETIME DEFAULT '1512561000000' COMMENT 'col1'
+                )  ENGINE=hive 
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+                """
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = date literal [1512561000000] is invalid: errCode = 2, detailMessage = Invalid date value: 1512561000000"
+            }
+
+            test {
+                sql """
+                CREATE TABLE all_default_values_${file_format}_err_datetime(
+                   `col1` DATETIME DEFAULT '2020-09-20 02:60' COMMENT 'col1'
+                )  ENGINE=hive 
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+                """
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = date literal [2020-09-20 02:60] is invalid: Text '2020-09-20 02:60' could not be parsed: Invalid value for MinuteOfHour (valid values 0 - 59): 60"
+            }
+
+            // case4. check some exceptions
+            def comment_check = sql """ CREATE TABLE ex_tbl_${file_format}(
+                                          `col1` INT COMMENT 'col1',
+                                          `col2` STRING COMMENT 'col2',
+                                          `pt1` VARCHAR COMMENT 'pt1'
+                                        )  ENGINE=hive 
+                                        COMMENT 'test'
+                                        PARTITION BY LIST (pt1) ()
+                                        PROPERTIES (
+                                          'file_format'='${file_format}'
+                                        )
+                                    """
+            def comment_check_res = sql """ show create table ex_tbl_${file_format} """
+            logger.info("${comment_check_res}")
+            assertTrue(comment_check_res.toString().containsIgnoreCase("COMMENT 'col1'"))
+            assertTrue(comment_check_res.toString().containsIgnoreCase("COMMENT 'col2'"))
+            sql """DROP TABLE `ex_tbl_${file_format}`"""
+
+            test {
+                sql """
+                        CREATE TABLE nullable_check (
+                            `col` STRING NOT NULL COMMENT 'col'
+                        )  ENGINE=hive 
+                        PROPERTIES (
+                            'file_format'='${file_format}'
+                        )
+                    """
+                exception "errCode = 2, detailMessage = hive catalog doesn't support column with 'NOT NULL'."
+            }
+
+            test {
+                sql """
+                        CREATE TABLE schema_check ENGINE=hive ;
+                    """
+                exception "AnalysisException, msg: Should contain at least one column in a table"
+            }
+            sql """ DROP DATABASE IF EXISTS `test_hive_loc` """
+        }
+
+        def test_tbl_compress = { String compression, String file_format, String catalog_name ->
+            logger.info("Test create table with compression...")
+            sql """ switch ${catalog_name} """
+            sql  """ create database if not exists `test_hive_compress`
+                 """
+            sql """use `test_hive_compress`"""
+
+            // check table compression here, write/test_hive_write_insert.groovy contains the insert into compression
+            sql """ DROP TABLE IF EXISTS tbl_${file_format}_${compression} """
+            sql """
+                    CREATE TABLE tbl_${file_format}_${compression} (
+                      `col` STRING COMMENT 'col'
+                    )  ENGINE=hive 
+                    PROPERTIES (
+                      'file_format'='${file_format}',
+                      'compression'='${compression}'
+                    )
+                 """
+            def create_tbl_res = sql """ show create table tbl_${file_format}_${compression} """
+            logger.info("${create_tbl_res}")
+            if (file_format.equals("parquet")) {
+                assertTrue(create_tbl_res.toString().containsIgnoreCase("'parquet.compression'='${compression}'"))
+            } else if (file_format.equals("orc")) {
+                assertTrue(create_tbl_res.toString().containsIgnoreCase("'orc.compress'='${compression}'"))
+            } else {
+                throw new Exception("Invalid compression type: ${compression} for tbl_${file_format}_${compression}")
+            }
+
+            sql """DROP TABLE `tbl_${file_format}_${compression}`"""
+            sql """ drop database if exists `test_hive_compress` """;
+        }
+
+        def test_create_tbl_cross_catalog = { String file_format, String catalog_name ->
+            sql """switch ${catalog_name}"""
+            sql """ CREATE DATABASE IF NOT EXISTS `test_olap_cross_catalog` """;
+            sql """ USE test_olap_cross_catalog """;
+            test {
+                sql """
+                        CREATE TABLE `test_olap_cross_catalog_tbl` (
+                            `col1` INT COMMENT 'col1',
+                            `col2` STRING COMMENT 'col2'
+                        )
+                        ENGINE=olap
+                        DISTRIBUTED BY HASH(col1) BUCKETS 16
+                        PROPERTIES (
+                            'replication_num' = '1'
+                        );
+                    """
+                exception "Cannot create olap table out of internal catalog. Make sure 'engine' type is specified when use the catalog: test_hive_ddl"
+            }
+
+            // test default engine is hive in hive catalog
+            sql """
+                    CREATE TABLE `test_olap_cross_catalog_tbl` (
+                        `col1` INT COMMENT 'col1',
+                        `col2` STRING COMMENT 'col2'
+                    )
+                """
+            sql """ DROP TABLE `test_olap_cross_catalog_tbl`
+                """
+
+            test {
+                sql """
+                        CREATE TABLE `test_olap_cross_catalog_tbl` (
+                            `col1` INT COMMENT 'col1',
+                            `col2` STRING COMMENT 'col2'
+                        ) DISTRIBUTED BY HASH(col1) BUCKETS 16
+                        PROPERTIES (
+                            'replication_num' = '1'
+                        );
+                    """
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = errCode = 2, detailMessage = Create hive bucket table need set enable_create_hive_bucket_table to true"
+            }
+
+            sql """ SWITCH internal """
+            sql """ CREATE DATABASE IF NOT EXISTS test_hive_cross_catalog """;
+            sql """ USE internal.test_hive_cross_catalog """
+
+            test {
+                sql """
+                        CREATE TABLE test_hive_cross_catalog_tbl (
+                          `col` STRING COMMENT 'col'
+                        )  ENGINE=hive 
+                    """
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = Cannot create hive table in internal catalog, should switch to hive catalog."
+            }
+
+            sql """ DROP DATABASE IF EXISTS test_olap_cross_catalog """
+            sql """ DROP DATABASE IF EXISTS test_hive_cross_catalog """
+        }
+
+        def test_db_tbl = { String file_format,  String externalEnvIp, String hdfs_port, String catalog_name ->
+            logger.info("Test create/drop table...")
+            sql """switch ${catalog_name}"""
+            sql """ create database if not exists `test_hive_db_tbl` """;
+            sql """use `${catalog_name}`.`test_hive_db_tbl`"""
+
+            sql """
+                CREATE TABLE unpart_tbl_${file_format}(
+                  `col1` BOOLEAN COMMENT 'col1',
+                  `col2` INT COMMENT 'col2',
+                  `col3` BIGINT COMMENT 'col3',
+                  `col4` CHAR(10) COMMENT 'col4',
+                  `col5` FLOAT COMMENT 'col5',
+                  `col6` DOUBLE COMMENT 'col6',
+                  `col7` DECIMAL(9,4) COMMENT 'col7',
+                  `col8` VARCHAR(11) COMMENT 'col8',
+                  `col9` STRING COMMENT 'col9',
+                  `col10` DATE COMMENT 'col10',
+                  `col11` DATETIME COMMENT 'col11'
+                )  ENGINE=hive 
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+            """;
+
+            // test all columns
+            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9`) 
+                    VALUES 
+                    (true, 123, 9876543210, 'abcdefghij', 3.14, 6.28, 123.4567, 'varcharval', 'stringval');
+                """
+            order_qt_insert01 """ SELECT `col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9` FROM unpart_tbl_${file_format};  """
+
+            // test part of columns
+            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col8`, `col9`) 
+                    VALUES 
+                    (true, 123, 9876543210, 'varcharval', 'stringval');
+                """
+            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col8`, `col9`) 
+                    VALUES 
+                    (null, 123, 'varcharval', 8.98);
+                """
+            order_qt_insert02 """ SELECT `col1`, `col2`, `col3`, `col7`, `col9` FROM unpart_tbl_${file_format};  """
+
+            // test data diff
+            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9`) 
+                    VALUES 
+                    (true, null, 9876543210, 'abcdefghij', '2.3', 6.28, null, 'varcharval', 'stringval');
+                """
+            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9`) 
+                    VALUES 
+                    (false, '1', 9876543210, 'abcdefghij', '2.3', 6.28, 0, 2223, 'stringval');
+                """
+            order_qt_insert03 """ SELECT `col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9` FROM unpart_tbl_${file_format} """
+
+            sql """ drop table if exists unpart_tbl_${file_format}"""
+
+            // partitioned table test
+            sql """
+                CREATE TABLE part_tbl_${file_format}(
+                  `col1` BOOLEAN COMMENT 'col1',
+                  `col2` INT COMMENT 'col2',
+                  `col3` BIGINT COMMENT 'col3',
+                  `col4` DECIMAL(2,1) COMMENT 'col4',
+                  `pt1` VARCHAR COMMENT 'pt1',
+                  `pt2` VARCHAR COMMENT 'pt2'
+                )  ENGINE=hive 
+                PARTITION BY LIST (pt1, pt2) ()
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+                """
+
+            // test all columns
+            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
+                VALUES (true, 1, 1000, 2.3, 'value_for_pt1', 'value_for_pt2')
+                """
+            order_qt_insert04 """ SELECT col1, col2, col3, col4, pt1, pt2 FROM part_tbl_${file_format};  """
+
+            // test part of columns
+            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
+                VALUES (true, 1, 1000, 2.3, 'value_for_pt1', 'value_for_pt2')
+                """
+            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
+                VALUES (true, 1, 1000, 2.3, 'value_for_pt1', 'value_for_pt2')
+                """
+            order_qt_insert05 """ SELECT col1, col2, col3, col4 FROM part_tbl_${file_format} """
+
+            // test data diff
+            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
+                VALUES (0, '1', 1000, null, 2.56, 'value_for_pt2')
+                """
+            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
+                VALUES (null, 1, '1000', '1.3', 'value_for_pt1', 2345)
+                """
+            order_qt_insert06 """ SELECT col1, col2, col3, col4 FROM part_tbl_${file_format}  """
+
+            sql """ drop table if exists part_tbl_${file_format} """
+
+            // test partitions
+            sql """
+                CREATE TABLE all_part_types_tbl_${file_format}(
+                  `col` INT COMMENT 'col',
+                  `pt1` BOOLEAN COMMENT 'pt1',
+                  `pt2` TINYINT COMMENT 'pt2',
+                  `pt3` SMALLINT COMMENT 'pt3',
+                  `pt4` INT COMMENT 'pt4',
+                  `pt5` BIGINT COMMENT 'pt5',
+                  `pt6` DATE COMMENT 'pt6',
+                  `pt7` DATETIME COMMENT 'pt7',
+                  `pt8` CHAR COMMENT 'pt8',
+                  `pt9` VARCHAR COMMENT 'pt9',
+                  `pt10` STRING COMMENT 'pt10'
+                )  ENGINE=hive 
+                PARTITION BY LIST (pt1, pt2, pt3, pt4, pt5, pt6, pt7, pt8, pt9, pt10) ()
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+                """
+            sql """ 
+                    INSERT INTO all_part_types_tbl_${file_format} (`col`, `pt1`, `pt2`, `pt3`, `pt4`, `pt5`, `pt6`, `pt7`, `pt8`, `pt9`, `pt10`)
+                    VALUES
+                    (1, true, 1, 123, 456789, 922232355, '2024-04-09', '2024-04-09 12:34:56', 'A', 'example', 'string_value');
+                """
+            def loc = "hdfs://${externalEnvIp}:${hdfs_port}/user/hive/warehouse/test_hive_db_tbl.db/all_part_types_tbl_${file_format}/pt1=1/pt2=1/pt3=123/pt4=456789/pt5=922232355/pt6=2024-04-09/*/pt8=A/pt9=example/pt10=string_value"
+            def pt_check = sql """ SELECT * FROM hdfs(
+                                    'uri'='${loc}/*',
+                                    'format' = '${file_format}',
+                                    'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}'
+                                  )
+                               """
+            logger.info("${pt_check}")
+            assertEquals(1, pt_check[0].size())
+            sql """ drop table if exists all_part_types_tbl_${file_format} """
+
+            test {
+                sql """
+                    CREATE TABLE all_part_types_tbl_${file_format}_err3(
+                      `col` INT COMMENT 'col',
+                      `pt1` STRING COMMENT 'pt1'
+                    )  ENGINE=hive 
+                    PARTITION BY LIST (pt000) ()
+                    PROPERTIES (
+                      'file_format'='${file_format}'
+                    )
+                    """
+                exception "errCode = 2, detailMessage = partition key pt000 is not exists"
+            }
+
+            test {
+                sql """ 
+                    CREATE TABLE all_part_types_tbl_${file_format}_err3(
+                      `col` INT COMMENT 'col',
+                      `pt1` STRING COMMENT 'pt1'
+                    )  ENGINE=hive 
+                    PARTITION BY LIST (pt1)
+                    (PARTITION pp VALUES IN ('2014-01-01'))
+                    PROPERTIES (
+                      'file_format'='${file_format}'
+                    ) 
+                    """
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = errCode = 2, detailMessage = Partition values expressions is not supported in hive catalog."
+            }
+
+            test {
+                sql """ 
+                    CREATE TABLE all_part_types_tbl_${file_format}_err3(
+                      `col` INT COMMENT 'col',
+                      `pt1` STRING COMMENT 'pt1'
+                    )  ENGINE=hive 
+                    PARTITION BY LIST (pt000)
+                    (PARTITION pp VALUES IN ('2014-01-01'))
+                    PROPERTIES (
+                      'file_format'='${file_format}'
+                    ) 
+                    """
+                exception "errCode = 2, detailMessage = partition key pt000 is not exists"
+            }
+
+            test {
+                sql """
+                    CREATE TABLE all_part_types_tbl_${file_format}_err1(
+                      `col` INT COMMENT 'col',
+                      `pt1` LARGEINT COMMENT 'pt1'
+                    )  ENGINE=hive 
+                    PARTITION BY LIST (pt1) ()
+                    PROPERTIES (
+                      'file_format'='${file_format}'
+                    )
+                    """
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = failed to create database from hms client. reason: org.apache.doris.datasource.hive.HMSClientException: Unsupported primitive type conversion of LARGEINT"
+            }
+
+            test {
+                sql """
+                    CREATE TABLE all_part_types_tbl_${file_format}_err2(
+                      `col` INT COMMENT 'col',
+                      `pt1` FLOAT COMMENT 'pt1'
+                    )  ENGINE=hive 
+                    PARTITION BY LIST (pt1) ()
+                    PROPERTIES (
+                      'file_format'='${file_format}'
+                    )
+                    """
+                exception "errCode = 2, detailMessage = Floating point type column can not be partition column"
+            }
+
+            test {
+                sql """
+                    CREATE TABLE all_part_types_tbl_${file_format}_err3(
+                      `col` INT COMMENT 'col',
+                      `pt1` DOUBLE COMMENT 'pt1'
+                    )  ENGINE=hive 
+                    PARTITION BY LIST (pt1) ()
+                    PROPERTIES (
+                      'file_format'='${file_format}'
+                    )
+                    """
+                exception "errCode = 2, detailMessage = Floating point type column can not be partition column"
+            }
+
+            sql """ drop database if exists `test_hive_db_tbl` """;
+        }
+
+
+        try {
+            String hms_port = context.config.otherConfigs.get("hms_port")
+            String hdfs_port = context.config.otherConfigs.get("hdfs_port")
+            String catalog_name = "test_hive_ddl"
+            String externalEnvIp = context.config.otherConfigs.get("externalEnvIp")
+
+            sql """drop catalog if exists ${catalog_name}"""
+            sql """create catalog if not exists ${catalog_name} properties (
+                'type'='hms',
+                'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}',
+                'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}'
+            );"""
+            sql """switch ${catalog_name}"""
+
+            sql """set enable_fallback_to_original_planner=false;"""
+            test_db(catalog_name)
+            test_loc_db(externalEnvIp, hdfs_port, catalog_name)
+            def compressions = ["snappy", "zlib", "zstd"]
+            for (String file_format in file_formats) {
+                logger.info("Process file format " + file_format)
+                test_loc_tbl(file_format, externalEnvIp, hdfs_port, catalog_name)
+                test_db_tbl(file_format, externalEnvIp, hdfs_port, catalog_name)
+
+                for (String compression in compressions) {
+                    if (file_format.equals("parquet") && compression.equals("zlib")) {
+                        continue
+                    }
+                    logger.info("Process file compression " + compression)
+                    test_tbl_compress(compression, file_format, catalog_name)
+                }
+                test_create_tbl_cross_catalog(file_format, catalog_name)
+            }
+            sql """drop catalog if exists ${catalog_name}"""
+        } finally {
+        }
+    }
+}
diff --git a/regression-test/suites/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.groovy b/regression-test/suites/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.groovy
deleted file mode 100644
index 093ba674fc6..00000000000
--- a/regression-test/suites/external_table_p0/hive/ddl/test_hive_ddl_and_ctas.groovy
+++ /dev/null
@@ -1,423 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-suite("test_hive_ddl_and_ctas", "p0,external,hive,external_docker,external_docker_hive") {
-    String enabled = context.config.otherConfigs.get("enableHiveTest")
-    if (enabled != null && enabled.equalsIgnoreCase("true")) {
-        def file_formats = ["parquet", "orc"]
-
-        def test_db = { String catalog_name ->
-            sql """switch ${catalog_name}"""
-            sql """ create database if not exists `test_hive_db` """;
-            sql """use `test_hive_db`"""
-            sql """ drop database if exists `test_hive_db` """;
-        }
-
-        def test_loc_db = { String externalEnvIp, String hdfs_port, String catalog_name ->
-            sql """switch ${catalog_name}"""
-            sql """ create database if not exists `test_hive_loc_db`
-                    properties('location_uri'='hdfs://${externalEnvIp}:${hdfs_port}/tmp/hive/test_hive_loc_db')
-                """;
-            sql """use `test_hive_loc_db`"""
-            sql """ drop database if exists `test_hive_loc_db` """;
-        }
-
-        def test_db_tbl = { String file_format, String catalog_name ->
-            sql """switch ${catalog_name}"""
-            sql """ create database if not exists `test_hive_db` """;
-            sql """use `${catalog_name}`.`test_hive_db`"""
-
-            sql """
-                CREATE TABLE unpart_tbl_${file_format}(
-                  `col1` BOOLEAN COMMENT 'col1',
-                  `col2` INT COMMENT 'col2',
-                  `col3` BIGINT COMMENT 'col3',
-                  `col4` CHAR(10) COMMENT 'col4',
-                  `col5` FLOAT COMMENT 'col5',
-                  `col6` DOUBLE COMMENT 'col6',
-                  `col7` DECIMAL(9,4) COMMENT 'col7',
-                  `col8` VARCHAR(11) COMMENT 'col8',
-                  `col9` STRING COMMENT 'col9'
-                )  ENGINE=hive 
-                PROPERTIES (
-                  'file_format'='${file_format}'
-                )
-            """;
-
-            // test all columns
-            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9`) 
-                    VALUES 
-                    (true, 123, 9876543210, 'abcdefghij', 3.14, 6.28, 123.4567, 'varcharval', 'stringval');
-                """
-            order_qt_insert01 """ SELECT `col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9` FROM unpart_tbl_${file_format};  """
-
-            // test part of columns
-            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col8`, `col9`) 
-                    VALUES 
-                    (true, 123, 9876543210, 'varcharval', 'stringval');
-                """
-            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col8`, `col9`) 
-                    VALUES 
-                    (null, 123, 'varcharval', 8.98);
-                """
-            order_qt_insert02 """ SELECT `col1`, `col2`, `col3`, `col7`, `col9` FROM unpart_tbl_${file_format};  """
-
-            // test data diff
-            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9`) 
-                    VALUES 
-                    (true, null, 9876543210, 'abcdefghij', '2.3', 6.28, null, 'varcharval', 'stringval');
-                """
-            sql """ INSERT INTO unpart_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9`) 
-                    VALUES 
-                    (false, '1', 9876543210, 'abcdefghij', '2.3', 6.28, 0, 2223, 'stringval');
-                """
-            order_qt_insert03 """ SELECT `col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9` FROM unpart_tbl_${file_format} """
-
-            sql """ drop table if exists unpart_tbl_${file_format}"""
-
-            // partitioned table test
-            sql """
-                CREATE TABLE part_tbl_${file_format}(
-                  `col1` BOOLEAN COMMENT 'col1',
-                  `col2` INT COMMENT 'col2',
-                  `col3` BIGINT COMMENT 'col3',
-                  `col4` DECIMAL(2,1) COMMENT 'col4',
-                  `pt1` VARCHAR COMMENT 'pt1',
-                  `pt2` VARCHAR COMMENT 'pt2'
-                )  ENGINE=hive 
-                PARTITION BY LIST (pt1, pt2) ()
-                PROPERTIES (
-                  'file_format'='${file_format}'
-                )
-            """;
-
-            // test all columns
-            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
-                VALUES (true, 1, 1000, 2.3, 'value_for_pt1', 'value_for_pt2')
-                """
-            order_qt_insert04 """ SELECT col1, col2, col3, col4, pt1, pt2 FROM part_tbl_${file_format};  """
-
-            // test part of columns
-            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
-                VALUES (true, 1, 1000, 2.3, 'value_for_pt1', 'value_for_pt2')
-                """
-            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
-                VALUES (true, 1, 1000, 2.3, 'value_for_pt1', 'value_for_pt2')
-                """
-            order_qt_insert05 """ SELECT col1, col2, col3, col4 FROM part_tbl_${file_format} """
-
-            // test data diff
-            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
-                VALUES (0, '1', 1000, null, 2.56, 'value_for_pt2')
-                """
-            sql """ INSERT INTO part_tbl_${file_format} (col1, col2, col3, col4, pt1, pt2)
-                VALUES (null, 1, '1000', '1.3', 'value_for_pt1', 2345)
-                """
-            order_qt_insert06 """ SELECT col1, col2, col3, col4 FROM part_tbl_${file_format}  """
-
-            sql """ drop table if exists part_tbl_${file_format}"""
-            sql """ drop database if exists `test_hive_db` """;
-        }
-
-        def test_ctas_tbl = { String file_format, String catalog_name ->
-            sql """ switch `${catalog_name}` """
-            sql """ create database if not exists `test_ctas` """;
-            sql """ switch internal """
-            sql """ create database if not exists test_ctas_olap """;
-            sql """ use internal.test_ctas_olap """
-
-            sql """
-                CREATE TABLE `unpart_ctas_olap_src` (
-                    `col1` INT COMMENT 'col1',
-                    `col2` STRING COMMENT 'col2'
-                )
-                ENGINE=olap
-                DISTRIBUTED BY HASH(col1) BUCKETS 16
-                PROPERTIES (
-                    'replication_num' = '1'
-                );
-            """
-
-            sql """ INSERT INTO `unpart_ctas_olap_src` (col1, col2) VALUES
-                (1, 'string value for col2'),
-                (2, 'another string value for col2'),
-                (3, 'yet another string value for col2'); 
-            """
-
-            sql """
-                CREATE TABLE `part_ctas_olap_src`(
-                    `col1` INT COMMENT 'col1',
-                    `pt1` VARCHAR(16) COMMENT 'pt1',
-                    `pt2` VARCHAR(16) COMMENT 'pt2'
-                )
-                ENGINE=olap
-                PARTITION BY LIST (pt1, pt2) (
-                    PARTITION pp1 VALUES IN(
-                        ('value_for_pt1', 'value_for_pt2'),
-                        ('value_for_pt11', 'value_for_pt22')
-                    )
-                )
-                DISTRIBUTED BY HASH(col1) BUCKETS 16
-                PROPERTIES (
-                    'replication_num' = '1'
-                );
-            """
-
-            sql """
-            INSERT INTO `part_ctas_olap_src` (col1, pt1, pt2) VALUES
-             (11, 'value_for_pt1', 'value_for_pt2'),
-             (22, 'value_for_pt11', 'value_for_pt22');
-            """
-
-            sql """ use `${catalog_name}`.`test_ctas` """
-            sql """
-                CREATE TABLE `unpart_ctas_src`(
-                  `col1` INT COMMENT 'col1',
-                  `col2` STRING COMMENT 'col2'
-                ) ENGINE=hive
-                PROPERTIES (
-                  'file_format'='parquet'
-                );
-            """
-
-            sql """ INSERT INTO `unpart_ctas_src` (col1, col2) VALUES
-                (1, 'string value for col2'),
-                (2, 'another string value for col2'),
-                (3, 'yet another string value for col2'); 
-            """
-
-            sql """
-                CREATE TABLE `part_ctas_src`(
-                  `col1` INT COMMENT 'col1',
-                  `pt1` VARCHAR COMMENT 'pt1',
-                  `pt2` VARCHAR COMMENT 'pt2'
-                ) ENGINE=hive
-                PARTITION BY LIST (pt1, pt2) ()
-                PROPERTIES (
-                  'file_format'='orc'
-                );
-            """
-
-            sql """
-            INSERT INTO `part_ctas_src` (col1, pt1, pt2) VALUES
-             (11, 'value_for_pt1', 'value_for_pt2'),
-             (22, 'value_for_pt11', 'value_for_pt22');
-            """
-
-            sql """ switch `${catalog_name}` """
-            // 1. external to external un-partitioned table
-            sql """ CREATE TABLE hive_ctas1 ENGINE=hive AS SELECT col1 FROM unpart_ctas_src; 
-            """
-
-            sql """ INSERT INTO hive_ctas1 SELECT col1 FROM unpart_ctas_src WHERE col1 > 1;
-            """
-
-            order_qt_ctas_01 """ SELECT * FROM hive_ctas1 """
-            sql """ DROP TABLE hive_ctas1 """
-
-            // 2. external to external partitioned table
-            sql """ CREATE TABLE hive_ctas2 ENGINE=hive AS SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>0;
-                """
-
-            sql """ INSERT INTO hive_ctas2 SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>=22;
-                """
-
-            order_qt_ctas_02 """ SELECT * FROM hive_ctas2  """
-            sql """ DROP TABLE hive_ctas2 """
-
-            // 3. internal to external un-partitioned table
-            sql """ CREATE TABLE ctas_o1 ENGINE=hive AS SELECT col1,col2 FROM internal.test_ctas_olap.unpart_ctas_olap_src;
-            """
-
-            sql """ INSERT INTO ctas_o1 SELECT col1,col2 FROM internal.test_ctas_olap.unpart_ctas_olap_src;
-            """
-
-            order_qt_ctas_03 """ SELECT * FROM ctas_o1  """
-            sql """ DROP TABLE ctas_o1 """
-
-            // 4. internal to external partitioned table
-            sql """ CREATE TABLE ctas_o2 ENGINE=hive AS SELECT col1,pt1,pt2 FROM internal.test_ctas_olap.part_ctas_olap_src WHERE col1>0;
-            """
-            sql """ INSERT INTO ctas_o2 SELECT col1,pt1,pt2 FROM internal.test_ctas_olap.part_ctas_olap_src WHERE col1>2;
-            """
-            order_qt_ctas_04 """ SELECT * FROM ctas_o2  """
-            sql """ DROP TABLE ctas_o2 """
-
-            // 5. check external to internal un-partitioned table
-            sql """ use internal.test_ctas_olap """
-            sql """  CREATE TABLE olap_ctas1
-                 PROPERTIES (
-                     "replication_allocation" = "tag.location.default: 1"
-                 ) AS SELECT col1,col2 
-                 FROM `${catalog_name}`.`test_ctas`.unpart_ctas_src;
-                """
-            order_qt_ctas_05 """ SELECT * FROM olap_ctas1  """
-            sql """ DROP TABLE olap_ctas1 """
-
-            // 6. check external to internal partitioned table
-            sql """ CREATE TABLE olap_ctas2 
-                PROPERTIES (
-                    "replication_allocation" = "tag.location.default: 1"
-                ) AS SELECT col1,pt1,pt2 
-                FROM `${catalog_name}`.`test_ctas`.part_ctas_src WHERE col1>0;
-                """
-            order_qt_ctas_06 """ SELECT * FROM olap_ctas2  """
-            sql """ DROP TABLE olap_ctas2 """
-
-            sql """ switch `${catalog_name}` """
-            sql """ DROP TABLE `test_ctas`.part_ctas_src """
-            sql """ DROP TABLE `test_ctas`.unpart_ctas_src """
-            sql """ drop database if exists `test_ctas` """;
-            sql """ DROP TABLE internal.test_ctas_olap.part_ctas_olap_src """
-            sql """ DROP TABLE internal.test_ctas_olap.unpart_ctas_olap_src """
-            sql """ switch internal """;
-            sql """ drop database if exists test_ctas_olap """;
-        }
-
-        def test_complex_type_tbl = { String file_format, String catalog_name ->
-            sql """ switch ${catalog_name} """
-            sql """ create database if not exists `test_complex_type` """;
-            sql """ use `${catalog_name}`.`test_complex_type` """
-
-            sql """
-                CREATE TABLE unpart_tbl_${file_format} (
-                  `col1` CHAR,
-                  `col2` CHAR(1),
-                  `col3` CHAR(16),
-                  `col4` VARCHAR,
-                  `col5` VARCHAR(255),
-                  `col6` DECIMAL(2,1),
-                  `col7` DECIMAL(5,0),
-                  `col8` DECIMAL(8,8),
-                  `col9` STRING,
-                  `col10` ARRAY<DECIMAL(4,3)>,
-                  `col11` ARRAY<CHAR(16)>,
-                  `col12` ARRAY<CHAR>,
-                  `col13` ARRAY<STRING>,
-                  `col14` ARRAY<MAP<INT, CHAR>>,
-                  `col15` MAP<BIGINT, CHAR>,
-                  `col16` MAP<BIGINT, DECIMAL(8,8)>,
-                  `col17` MAP<STRING, ARRAY<CHAR(16)>>,
-                  `col18` STRUCT<id:INT,gender:BOOLEAN,name:CHAR(16)>,
-                  `col19` STRUCT<scale:DECIMAL(7,4),metric:ARRAY<STRING>>,
-                  `col20` STRUCT<codes:ARRAY<INT>,props:MAP<STRING, ARRAY<CHAR(16)>>>
-                )  ENGINE=hive
-                PROPERTIES (
-                  'file_format'='${file_format}'
-                )
-            """;
-
-            sql """
-            INSERT INTO unpart_tbl_${file_format} (
-              col1, col2, col3, col4, col5, col6, col7, col8, col9,
-              col10, col11, col12, col13, col14, col15, col16, col17,
-              col18, col19, col20
-            ) VALUES (
-              'a', -- CHAR
-              'b', -- CHAR(1)
-              'c', -- CHAR(16)
-              'd', -- VARCHAR
-              'e', -- VARCHAR(255)
-              1.1, -- DECIMAL(2,1)
-              12345, -- DECIMAL(5,0)
-              0.12345678, -- DECIMAL(8,8)
-              'string', -- STRING
-              ARRAY(0.001, 0.002), -- ARRAY<DECIMAL(4,3)>
-              ARRAY('char1', 'char2'), -- ARRAY<CHAR(16)>
-              ARRAY('c', 'd'), -- ARRAY<CHAR>
-              ARRAY('string1', 'string2'), -- ARRAY<STRING>
-              ARRAY(MAP(1, 'a'), MAP(2, 'b')), -- ARRAY<MAP<INT, CHAR>>
-              MAP(1234567890123456789, 'a'), -- MAP<BIGINT, CHAR>
-              MAP(1234567890123456789, 0.12345678), -- MAP<BIGINT, DECIMAL(8,8)>
-              MAP('key', ARRAY('char1', 'char2')), -- MAP<STRING, ARRAY<CHAR(16)>>
-              STRUCT(1, TRUE, 'John Doe'), -- STRUCT<id:INT,gender:BOOLEAN,name:CHAR(16)>
-              STRUCT(123.4567, ARRAY('metric1', 'metric2')), -- STRUCT<scale:DECIMAL(7,4),metric:ARRAY<STRING>>
-              STRUCT(ARRAY(123, 456), MAP('key1', ARRAY('char1', 'char2'))) -- STRUCT<codes:ARRAY<INT>,props:MAP<STRING, ARRAY<CHAR(16)>>
-            );
-        """
-
-            sql """
-            INSERT INTO unpart_tbl_${file_format} (
-              col1, col11, col12, col13, col14, col15, col16, col17,
-              col18, col19
-            ) VALUES (
-              'a', -- CHAR
-              ARRAY('char1', 'char2'), -- ARRAY<CHAR(16)>
-              ARRAY('c', 'd'), -- ARRAY<CHAR>
-              ARRAY('string1', 'string2'), -- ARRAY<STRING>
-              ARRAY(MAP(1, 'a'), MAP(2, 'b')), -- ARRAY<MAP<INT, CHAR>>
-              MAP(1234567890123456789, 'a'), -- MAP<BIGINT, CHAR>
-              MAP(1234567890123456789, 0.12345678), -- MAP<BIGINT, DECIMAL(8,8)>
-              MAP('key', ARRAY('char1', 'char2')), -- MAP<STRING, ARRAY<CHAR(16)>>
-              STRUCT(1, TRUE, 'John Doe'), -- STRUCT<id:INT,gender:BOOLEAN,name:CHAR(16)>
-              STRUCT(123.4567, ARRAY('metric1', 'metric2')) -- STRUCT<scale:DECIMAL(7,4),metric:ARRAY<STRING>>
-            );
-        """
-
-            sql """
-            INSERT INTO unpart_tbl_${file_format} (
-              col1, col2, col3, col4, col5, col6, col7, col8, col9
-            ) VALUES (
-              'a', -- CHAR
-              'b', -- CHAR(1)
-              'c', -- CHAR(16)
-              'd', -- VARCHAR
-              'e', -- VARCHAR(255)
-              1.1, -- DECIMAL(2,1)
-              12345, -- DECIMAL(5,0)
-              0.12345678, -- DECIMAL(8,8)
-              'string' -- STRING
-            );
-        """
-
-            order_qt_complex_type01 """ SELECT * FROM unpart_tbl_${file_format} """
-            order_qt_complex_type02 """ SELECT * FROM unpart_tbl_${file_format} WHERE col2='b' """
-
-            sql """ DROP TABLE unpart_tbl_${file_format} """
-            sql """ drop database if exists `test_complex_type` """;
-        }
-
-        try {
-            String hms_port = context.config.otherConfigs.get("hms_port")
-            String hdfs_port = context.config.otherConfigs.get("hdfs_port")
-            String catalog_name = "test_hive_ddl_and_ctas"
-            String externalEnvIp = context.config.otherConfigs.get("externalEnvIp")
-
-            sql """drop catalog if exists ${catalog_name}"""
-            sql """create catalog if not exists ${catalog_name} properties (
-                'type'='hms',
-                'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}',
-                'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}'
-            );"""
-            sql """switch ${catalog_name}"""
-
-            sql """set enable_fallback_to_original_planner=false;"""
-
-            test_db(catalog_name)
-            test_loc_db(externalEnvIp, hdfs_port, catalog_name)
-            for (String file_format in file_formats) {
-                logger.info("Process file format" + file_format)
-                test_db_tbl(file_format, catalog_name)
-                test_ctas_tbl(file_format, catalog_name)
-                test_complex_type_tbl(file_format, catalog_name)
-                // todo: test bucket table: test_db_buck_tbl()
-            }
-            sql """drop catalog if exists ${catalog_name}"""
-        } finally {
-        }
-    }
-}
diff --git a/regression-test/suites/external_table_p0/hive/ddl/test_hive_write_type.groovy b/regression-test/suites/external_table_p0/hive/ddl/test_hive_write_type.groovy
new file mode 100644
index 00000000000..07953ed1575
--- /dev/null
+++ b/regression-test/suites/external_table_p0/hive/ddl/test_hive_write_type.groovy
@@ -0,0 +1,285 @@
+// 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_hive_write_type", "p0,external,hive,external_docker,external_docker_hive") {
+    String enabled = context.config.otherConfigs.get("enableHiveTest")
+    if (enabled != null && enabled.equalsIgnoreCase("true")) {
+        def file_formats = ["parquet", "orc"]
+        def test_complex_type_tbl = { String file_format, String catalog_name ->
+            sql """ switch ${catalog_name} """
+            sql """ create database if not exists `test_complex_type` """;
+            sql """ use `${catalog_name}`.`test_complex_type` """
+
+            sql """
+                CREATE TABLE unpart_tbl_${file_format} (
+                  `col1` CHAR,
+                  `col2` CHAR(1),
+                  `col3` CHAR(16),
+                  `col4` VARCHAR,
+                  `col5` VARCHAR(255),
+                  `col6` DECIMAL(2,1),
+                  `col7` DECIMAL(5,0),
+                  `col8` DECIMAL(8,8),
+                  `col9` STRING,
+                  `col10` ARRAY<DECIMAL(4,3)>,
+                  `col11` ARRAY<CHAR(16)>,
+                  `col12` ARRAY<CHAR>,
+                  `col13` ARRAY<STRING>,
+                  `col14` ARRAY<MAP<INT, CHAR>>,
+                  `col15` MAP<BIGINT, CHAR>,
+                  `col16` MAP<BIGINT, DECIMAL(8,8)>,
+                  `col17` MAP<STRING, ARRAY<CHAR(16)>>,
+                  `col18` STRUCT<id:INT,gender:BOOLEAN,name:CHAR(16)>,
+                  `col19` STRUCT<scale:DECIMAL(7,4),metric:ARRAY<STRING>>,
+                  `col20` STRUCT<codes:ARRAY<INT>,props:MAP<STRING, ARRAY<CHAR(16)>>>
+                )  ENGINE=hive
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+            """;
+
+            sql """
+            INSERT INTO unpart_tbl_${file_format} (
+              col1, col2, col3, col4, col5, col6, col7, col8, col9,
+              col10, col11, col12, col13, col14, col15, col16, col17,
+              col18, col19, col20
+            ) VALUES (
+              'a', -- CHAR
+              'b', -- CHAR(1)
+              'c', -- CHAR(16)
+              'd', -- VARCHAR
+              'e', -- VARCHAR(255)
+              1.1, -- DECIMAL(2,1)
+              12345, -- DECIMAL(5,0)
+              0.12345678, -- DECIMAL(8,8)
+              'string', -- STRING
+              ARRAY(0.001, 0.002), -- ARRAY<DECIMAL(4,3)>
+              ARRAY('char1', 'char2'), -- ARRAY<CHAR(16)>
+              ARRAY('c', 'd'), -- ARRAY<CHAR>
+              ARRAY('string1', 'string2'), -- ARRAY<STRING>
+              ARRAY(MAP(1, 'a'), MAP(2, 'b')), -- ARRAY<MAP<INT, CHAR>>
+              MAP(1234567890123456789, 'a'), -- MAP<BIGINT, CHAR>
+              MAP(1234567890123456789, 0.12345678), -- MAP<BIGINT, DECIMAL(8,8)>
+              MAP('key', ARRAY('char1', 'char2')), -- MAP<STRING, ARRAY<CHAR(16)>>
+              STRUCT(1, TRUE, 'John Doe'), -- STRUCT<id:INT,gender:BOOLEAN,name:CHAR(16)>
+              STRUCT(123.4567, ARRAY('metric1', 'metric2')), -- STRUCT<scale:DECIMAL(7,4),metric:ARRAY<STRING>>
+              STRUCT(ARRAY(123, 456), MAP('key1', ARRAY('char1', 'char2'))) -- STRUCT<codes:ARRAY<INT>,props:MAP<STRING, ARRAY<CHAR(16)>>
+            );
+            """
+
+            sql """
+            INSERT OVERWRITE TABLE unpart_tbl_${file_format} (
+              col1, col2, col3, col4, col5, col6, col7, col8, col9,
+              col10, col11, col12, col13, col14, col15, col16, col17,
+              col18, col19, col20
+            ) VALUES (
+              'a', -- CHAR
+              'b', -- CHAR(1)
+              'c', -- CHAR(16)
+              'd', -- VARCHAR
+              'e', -- VARCHAR(255)
+              1.1, -- DECIMAL(2,1)
+              12345, -- DECIMAL(5,0)
+              0.12345678, -- DECIMAL(8,8)
+              'string', -- STRING
+              ARRAY(0.001, 0.002), -- ARRAY<DECIMAL(4,3)>
+              ARRAY('char1', 'char2'), -- ARRAY<CHAR(16)>
+              ARRAY('c', 'd'), -- ARRAY<CHAR>
+              ARRAY('string1', 'string2'), -- ARRAY<STRING>
+              ARRAY(MAP(1, 'a'), MAP(2, 'b')), -- ARRAY<MAP<INT, CHAR>>
+              MAP(1234567890123456789, 'a'), -- MAP<BIGINT, CHAR>
+              MAP(1234567890123456789, 0.12345678), -- MAP<BIGINT, DECIMAL(8,8)>
+              MAP('key', ARRAY('char1', 'char2')), -- MAP<STRING, ARRAY<CHAR(16)>>
+              STRUCT(1, TRUE, 'John Doe'), -- STRUCT<id:INT,gender:BOOLEAN,name:CHAR(16)>
+              STRUCT(123.4567, ARRAY('metric1', 'metric2')), -- STRUCT<scale:DECIMAL(7,4),metric:ARRAY<STRING>>
+              STRUCT(ARRAY(123, 456), MAP('key1', ARRAY('char1', 'char2'))) -- STRUCT<codes:ARRAY<INT>,props:MAP<STRING, ARRAY<CHAR(16)>>
+            );
+            """
+
+            sql """
+            INSERT INTO unpart_tbl_${file_format} (
+              col1, col11, col12, col13, col14, col15, col16, col17,
+              col18, col19
+            ) VALUES (
+              'a', -- CHAR
+              ARRAY('char1', 'char2'), -- ARRAY<CHAR(16)>
+              ARRAY('c', 'd'), -- ARRAY<CHAR>
+              ARRAY('string1', 'string2'), -- ARRAY<STRING>
+              ARRAY(MAP(1, 'a'), MAP(2, 'b')), -- ARRAY<MAP<INT, CHAR>>
+              MAP(1234567890123456789, 'a'), -- MAP<BIGINT, CHAR>
+              MAP(1234567890123456789, 0.12345678), -- MAP<BIGINT, DECIMAL(8,8)>
+              MAP('key', ARRAY('char1', 'char2')), -- MAP<STRING, ARRAY<CHAR(16)>>
+              STRUCT(1, TRUE, 'John Doe'), -- STRUCT<id:INT,gender:BOOLEAN,name:CHAR(16)>
+              STRUCT(123.4567, ARRAY('metric1', 'metric2')) -- STRUCT<scale:DECIMAL(7,4),metric:ARRAY<STRING>>
+            );
+            """
+
+            sql """
+            INSERT INTO unpart_tbl_${file_format} (
+              col1, col2, col3, col4, col5, col6, col7, col8, col9
+            ) VALUES (
+              'a', -- CHAR
+              'b', -- CHAR(1)
+              'c', -- CHAR(16)
+              'd', -- VARCHAR
+              'e', -- VARCHAR(255)
+              1.1, -- DECIMAL(2,1)
+              12345, -- DECIMAL(5,0)
+              0.12345678, -- DECIMAL(8,8)
+              'string' -- STRING
+            );
+            """
+
+            order_qt_complex_type01 """ SELECT * FROM unpart_tbl_${file_format} """
+            order_qt_complex_type02 """ SELECT * FROM unpart_tbl_${file_format} WHERE col2='b' """
+
+            sql """ DROP TABLE unpart_tbl_${file_format} """
+            sql """ drop database if exists `test_complex_type` """;
+        }
+
+        def test_insert_exception = { String file_format, String catalog_name ->
+            sql """ switch ${catalog_name} """
+
+            sql """ create database if not exists `test_hive_ex` """;
+            test {
+                sql """ create database `test_hive_ex` """
+                exception "errCode = 2, detailMessage = Can't create database 'test_hive_ex'; database exists"
+            }
+            sql """use `${catalog_name}`.`test_hive_ex`"""
+
+            sql """
+                CREATE TABLE IF NOT EXISTS test_hive_ex.ex_tbl_${file_format}(
+                  `col1` BOOLEAN COMMENT 'col1',
+                  `col2` INT COMMENT 'col2',
+                  `col3` BIGINT COMMENT 'col3',
+                  `col4` CHAR(10) COMMENT 'col4',
+                  `col5` FLOAT COMMENT 'col5',
+                  `col6` DOUBLE COMMENT 'col6',
+                  `col7` DECIMAL(6,4) COMMENT 'col7',
+                  `col8` VARCHAR(11) COMMENT 'col8',
+                  `col9` STRING COMMENT 'col9',
+                  `pt1` VARCHAR COMMENT 'pt1',
+                  `pt2` STRING COMMENT 'pt2',
+                  `pt3` DATE COMMENT 'pt3'
+                )  ENGINE=hive 
+                PARTITION BY LIST (pt1, pt2) ()
+                PROPERTIES (
+                  'file_format'='${file_format}'
+                )
+            """;
+
+            try {
+                // test  columns
+                sql """ INSERT INTO ex_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9`) 
+                    VALUES 
+                    (true, 123, 987654321099, 'abcdefghij', 3.1214, 63.28, 123.4567, 'varcharval', 'stringval');
+                """
+            } catch (Exception e) {
+                // BE err msg need use string contains to check
+                assertTrue(e.getMessage().contains("[E-124]Arithmetic overflow, convert failed from 1234567, expected data is [-999999, 999999]"))
+            }
+
+            try {
+                // test type diff columns
+                sql """ INSERT INTO ex_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `col6`, `col7`, `col8`, `col9`) 
+                    VALUES 
+                    ('1', 123, 987654319, 'abcdefghij', '3.15', '6.28', 123.4567, 432, 'stringval');
+                """
+            } catch (Exception e) {
+                assertTrue(e.getMessage().contains("[E-124]Arithmetic overflow, convert failed from 1234567, expected data is [-999999, 999999]"))
+            }
+
+            test {
+                sql """
+                        CREATE TABLE test_hive_ex.ex_tbl_${file_format}(
+                          `col1` BOOLEAN COMMENT 'col1'
+                        )  ENGINE=hive 
+                        PROPERTIES (
+                          'file_format'='${file_format}'
+                        )
+                    """;
+                exception "errCode = 2, detailMessage = errCode = 2, detailMessage = Table 'ex_tbl_${file_format}' already exists"
+            }
+
+            test {
+                // test columns
+                sql """ INSERT INTO ex_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`) 
+                        VALUES 
+                        (true, 123, 9876543210, 'abcdefghij', 3.14, 6.28, 123.4567, 'varcharval', 'stringval');
+                """
+                exception "errCode = 2, detailMessage = Column count doesn't match value count"
+            }
+
+            test {
+                // test columns
+                sql """ INSERT INTO ex_tbl_${file_format} (`col1`, `col2`, `col3`, `col4`, `col5`, `pt00`) 
+                    VALUES 
+                    (true, 123, 9876543210, 'abcdefghij', 3.14, 'error');
+                """
+                exception "errCode = 2, detailMessage = Unknown column 'pt00' in target table."
+            }
+
+            test {
+                sql """ INSERT INTO ex_tbl_${file_format} partition(`pt1`,`pt2`) (`col1`, `col9`) 
+                    VALUES 
+                    ('abcdefghij', 'error', true, 123);
+                """
+                exception "errCode = 2, detailMessage = Column count doesn't match value count"
+            }
+
+            sql """ INSERT INTO ex_tbl_${file_format} partition(`pt1`,`pt2`) (`col3`, `col6`, `col9`) 
+                VALUES 
+                (9876543210, 6.28, 'no_error');
+            """
+
+            test {
+                sql """ INSERT INTO ex_tbl_${file_format} partition(`pt0`, `pt1`,`pt3`) (`col3`, `col6`, `col9`) 
+                    VALUES 
+                    ('err', 'err', 'err', 9876543210, 6.28, 'error');
+                """
+                exception "errCode = 2, detailMessage = Column count doesn't match value count"
+            }
+
+            sql """ DROP TABLE ${catalog_name}.test_hive_ex.ex_tbl_${file_format} """
+            sql """ DROP DATABASE ${catalog_name}.test_hive_ex """
+        }
+
+        try {
+            String hms_port = context.config.otherConfigs.get("hms_port")
+            String hdfs_port = context.config.otherConfigs.get("hdfs_port")
+            String catalog_name = "test_hive_write_type"
+            String externalEnvIp = context.config.otherConfigs.get("externalEnvIp")
+
+            sql """drop catalog if exists ${catalog_name}"""
+            sql """create catalog if not exists ${catalog_name} properties (
+                'type'='hms',
+                'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}',
+                'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}'
+            );"""
+            sql """switch ${catalog_name}"""
+
+            sql """set enable_fallback_to_original_planner=false;"""
+            for (String file_format in file_formats) {
+                logger.info("Process file format" + file_format)
+                test_complex_type_tbl(file_format, catalog_name)
+                test_insert_exception(file_format, catalog_name)
+            }
+            sql """drop catalog if exists ${catalog_name}"""
+        } finally {
+        }
+    }
+}


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


(doris) 03/19: [fix](insert)fix hive table sink write path (#32587)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 70489fe749cf3194872edc2199ebd286495124ea
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Fri Mar 22 10:37:26 2024 +0800

    [fix](insert)fix hive table sink write path (#32587)
    
    issue: #31442
    
    fix hive table sink write path to hdfs://${hdfs_root}/tmp/.doris_staging/${user}
---
 .../src/main/java/org/apache/doris/common/util/LocationPath.java   | 7 +++++++
 .../src/main/java/org/apache/doris/planner/HiveTableSink.java      | 3 +--
 2 files changed, 8 insertions(+), 2 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/LocationPath.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/LocationPath.java
index e9fb932b6dd..005a8f2cb84 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/LocationPath.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/LocationPath.java
@@ -43,6 +43,7 @@ import java.nio.file.InvalidPathException;
 import java.nio.file.Paths;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.UUID;
 
 public class LocationPath {
     private static final Logger LOG = LogManager.getLogger(LocationPath.class);
@@ -398,6 +399,12 @@ public class LocationPath {
         return new Path(location);
     }
 
+    public static String getTempWritePath(String loc, String prefix) {
+        Path tempRoot = new Path(loc, prefix);
+        Path tempPath = new Path(tempRoot, UUID.randomUUID().toString().replace("-", ""));
+        return tempPath.toString();
+    }
+
     @Override
     public String toString() {
         return get();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java
index 4652c132a9e..c2debe3a4af 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java
@@ -47,7 +47,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Optional;
 import java.util.Set;
-import java.util.UUID;
 import java.util.stream.Collectors;
 
 public class HiveTableSink extends DataSink {
@@ -153,7 +152,7 @@ public class HiveTableSink extends DataSink {
 
     private String createTempPath(String location) {
         String user = ConnectContext.get().getUserIdentity().getQualifiedUser();
-        return location + "/.doris_staging/" + user + "/" + UUID.randomUUID().toString().replace("-", "");
+        return LocationPath.getTempWritePath(location, "/tmp/.doris_staging/" + user);
     }
 
     private void setCompressType(THiveTableSink tSink, TFileFormatType formatType) {


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


(doris) 15/19: [feature](hive)support ExternalTransaction for writing exteral table (#32726)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit e11db3f050ca8f633dd359c9a810365dcbb238d4
Author: wuwenchi <wu...@hotmail.com>
AuthorDate: Thu Apr 4 08:10:49 2024 +0800

    [feature](hive)support ExternalTransaction for writing exteral table (#32726)
    
    
    Issue #31442
    
    Add `TransactionManager` and `Transaction`.
    
    ```
    public interface Transaction {
        void commit() throws UserException;
        void rollback();
    }
    public interface TransactionManager {
        long begin();
        void commit(long id) throws UserException;
        void rollback(long id);
        Transaction getTransaction(long id);
    }
    ```
    `TransactionManager` is used to manage all external transactions:
    The application layer should manage the entire transaction through this `TransactionManager`, like:
    ```
    transactionManager.commit();
    transactionManager.rollback();
    ```
    
    `Transaction` is an interface. You can implement this interface according to the specific content, such as `HMSTransaction` currently implemented, iceberg that may be implemented in the future, etc.
---
 .../apache/doris/datasource/ExternalCatalog.java   |    2 +
 .../apache/doris/datasource/hive/HMSCommitter.java |  754 -----------
 .../doris/datasource/hive/HMSExternalCatalog.java  |    6 +-
 .../doris/datasource/hive/HMSTransaction.java      | 1322 ++++++++++++++++++++
 .../doris/datasource/hive/HiveMetadataOps.java     |   24 +-
 .../datasource/hive/HivePartitionStatistics.java   |    2 +-
 .../hive/HivePartitionWithStatistics.java          |    6 +-
 .../plans/commands/insert/HiveInsertExecutor.java  |   34 +-
 .../main/java/org/apache/doris/qe/Coordinator.java |   30 +-
 .../doris/transaction/HiveTransactionManager.java  |   79 ++
 .../Transaction.java}                              |   25 +-
 .../TransactionManager.java}                       |   40 +-
 .../TransactionManagerFactory.java}                |   25 +-
 .../doris/datasource/hive/HmsCommitTest.java       |   72 +-
 14 files changed, 1535 insertions(+), 886 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
index 737705bd8b5..a3525321edf 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
@@ -50,6 +50,7 @@ import org.apache.doris.persist.gson.GsonPostProcessable;
 import org.apache.doris.persist.gson.GsonUtils;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.qe.MasterCatalogExecutor;
+import org.apache.doris.transaction.TransactionManager;
 
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
@@ -111,6 +112,7 @@ public abstract class ExternalCatalog
     private boolean objectCreated = false;
     protected boolean invalidCacheInInit = true;
     protected ExternalMetadataOps metadataOps;
+    protected TransactionManager transactionManager;
 
     private ExternalSchemaCache schemaCache;
     private String comment;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSCommitter.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSCommitter.java
deleted file mode 100644
index af26f36d6b9..00000000000
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSCommitter.java
+++ /dev/null
@@ -1,754 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-// This file is copied from
-// https://github.com/trinodb/trino/blob/438/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java
-// and modified by Doris
-
-package org.apache.doris.datasource.hive;
-
-import org.apache.doris.backup.Status;
-import org.apache.doris.common.Pair;
-import org.apache.doris.fs.remote.RemoteFile;
-import org.apache.doris.fs.remote.RemoteFileSystem;
-import org.apache.doris.thrift.THivePartitionUpdate;
-import org.apache.doris.thrift.TUpdateMode;
-
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Verify;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import io.airlift.concurrent.MoreFutures;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Queue;
-import java.util.StringJoiner;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.stream.Collectors;
-
-public class HMSCommitter {
-    private static final Logger LOG = LogManager.getLogger(HMSCommitter.class);
-    private final HiveMetadataOps hiveOps;
-    private final RemoteFileSystem fs;
-    private final Table table;
-
-    // update statistics for unPartitioned table or existed partition
-    private final List<UpdateStatisticsTask> updateStatisticsTasks = new ArrayList<>();
-    Executor updateStatisticsExecutor = Executors.newFixedThreadPool(16);
-
-    // add new partition
-    private final AddPartitionsTask addPartitionsTask = new AddPartitionsTask();
-    private static final int PARTITION_COMMIT_BATCH_SIZE = 20;
-
-    // for file system rename operation
-    // whether to cancel the file system tasks
-    private final AtomicBoolean fileSystemTaskCancelled = new AtomicBoolean(false);
-    // file system tasks that are executed asynchronously, including rename_file, rename_dir
-    private final List<CompletableFuture<?>> asyncFileSystemTaskFutures = new ArrayList<>();
-    // when aborted, we need to delete all files under this path, even the current directory
-    private final Queue<DirectoryCleanUpTask> directoryCleanUpTasksForAbort = new ConcurrentLinkedQueue<>();
-    // when aborted, we need restore directory
-    private final List<RenameDirectoryTask> renameDirectoryTasksForAbort = new ArrayList<>();
-    // when finished, we need clear some directories
-    private final List<String> clearDirsForFinish = new ArrayList<>();
-    Executor fileSystemExecutor = Executors.newFixedThreadPool(16);
-
-    public HMSCommitter(HiveMetadataOps hiveOps, RemoteFileSystem fs, Table table) {
-        this.hiveOps = hiveOps;
-        this.fs = fs;
-        this.table = table;
-    }
-
-    public void commit(List<THivePartitionUpdate> hivePUs) {
-        try {
-            prepare(mergePartitions(hivePUs));
-            doCommit();
-        } catch (Throwable t) {
-            LOG.warn("Failed to commit for {}.{}, abort it.", table.getDbName(), table.getTableName());
-            try {
-                cancelUnStartedAsyncFileSystemTask();
-                undoUpdateStatisticsTasks();
-                undoAddPartitionsTask();
-                waitForAsyncFileSystemTaskSuppressThrowable();
-                runDirectoryClearUpTasksForAbort();
-                runRenameDirTasksForAbort();
-            } catch (Throwable e) {
-                t.addSuppressed(new Exception("Failed to roll back after commit failure", e));
-            }
-            throw t;
-        } finally {
-            runClearPathsForFinish();
-        }
-    }
-
-    public void prepare(List<THivePartitionUpdate> hivePUs) {
-
-        List<Pair<THivePartitionUpdate, HivePartitionStatistics>> insertExistsPartitions = new ArrayList<>();
-
-        for (THivePartitionUpdate pu : hivePUs) {
-            TUpdateMode updateMode = pu.getUpdateMode();
-            HivePartitionStatistics hivePartitionStatistics = HivePartitionStatistics.fromCommonStatistics(
-                    pu.getRowCount(),
-                    pu.getFileNamesSize(),
-                    pu.getFileSize());
-            if (table.getPartitionKeysSize() == 0) {
-                Preconditions.checkArgument(hivePUs.size() == 1,
-                        "When updating a non-partitioned table, multiple partitions should not be written");
-                switch (updateMode) {
-                    case APPEND:
-                        prepareAppendTable(pu, hivePartitionStatistics);
-                        break;
-                    case OVERWRITE:
-                        prepareOverwriteTable(pu, hivePartitionStatistics);
-                        break;
-                    default:
-                        throw new RuntimeException("Not support mode:[" + updateMode + "] in unPartitioned table");
-                }
-            } else {
-                switch (updateMode) {
-                    case NEW:
-                        prepareCreateNewPartition(pu, hivePartitionStatistics);
-                        break;
-                    case APPEND:
-                        insertExistsPartitions.add(Pair.of(pu, hivePartitionStatistics));
-                        break;
-                    case OVERWRITE:
-                        prepareOverwritePartition(pu, hivePartitionStatistics);
-                        break;
-                    default:
-                        throw new RuntimeException("Not support mode:[" + updateMode + "] in unPartitioned table");
-                }
-            }
-        }
-
-        if (!insertExistsPartitions.isEmpty()) {
-            prepareInsertExistPartition(insertExistsPartitions);
-        }
-    }
-
-    public List<THivePartitionUpdate> mergePartitions(List<THivePartitionUpdate> hivePUs) {
-        Map<String, THivePartitionUpdate> mm = new HashMap<>();
-        for (THivePartitionUpdate pu : hivePUs) {
-            if (mm.containsKey(pu.getName())) {
-                THivePartitionUpdate old = mm.get(pu.getName());
-                old.setFileSize(old.getFileSize() + pu.getFileSize());
-                old.setRowCount(old.getRowCount() + pu.getRowCount());
-                old.getFileNames().addAll(pu.getFileNames());
-            } else {
-                mm.put(pu.getName(), pu);
-            }
-        }
-        return new ArrayList<>(mm.values());
-    }
-
-    public void doCommit() {
-        waitForAsyncFileSystemTasks();
-        doAddPartitionsTask();
-        doUpdateStatisticsTasks();
-    }
-
-    public void rollback() {
-
-    }
-
-    public void cancelUnStartedAsyncFileSystemTask() {
-        fileSystemTaskCancelled.set(true);
-    }
-
-    private void undoUpdateStatisticsTasks() {
-        ImmutableList.Builder<CompletableFuture<?>> undoUpdateFutures = ImmutableList.builder();
-        for (UpdateStatisticsTask task : updateStatisticsTasks) {
-            undoUpdateFutures.add(CompletableFuture.runAsync(() -> {
-                try {
-                    task.undo(hiveOps);
-                } catch (Throwable throwable) {
-                    LOG.warn("Failed to rollback: {}", task.getDescription(), throwable);
-                }
-            }, updateStatisticsExecutor));
-        }
-
-        for (CompletableFuture<?> undoUpdateFuture : undoUpdateFutures.build()) {
-            MoreFutures.getFutureValue(undoUpdateFuture);
-        }
-    }
-
-    private void undoAddPartitionsTask() {
-        if (addPartitionsTask.isEmpty()) {
-            return;
-        }
-
-        HivePartition firstPartition = addPartitionsTask.getPartitions().get(0).getPartition();
-        String dbName = firstPartition.getDbName();
-        String tableName = firstPartition.getTblName();
-        List<List<String>> rollbackFailedPartitions = addPartitionsTask.rollback(hiveOps);
-        if (!rollbackFailedPartitions.isEmpty()) {
-            LOG.warn("Failed to rollback: add_partition for partition values {}.{}.{}",
-                    dbName, tableName, rollbackFailedPartitions);
-        }
-    }
-
-    private void waitForAsyncFileSystemTaskSuppressThrowable() {
-        for (CompletableFuture<?> future : asyncFileSystemTaskFutures) {
-            try {
-                future.get();
-            } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-            } catch (Throwable t) {
-                // ignore
-            }
-        }
-    }
-
-    public void prepareAppendTable(THivePartitionUpdate pu, HivePartitionStatistics ps) {
-        String targetPath = pu.getLocation().getTargetPath();
-        String writePath = pu.getLocation().getWritePath();
-        if (!targetPath.equals(writePath)) {
-            fs.asyncRename(
-                    fileSystemExecutor,
-                    asyncFileSystemTaskFutures,
-                    fileSystemTaskCancelled,
-                    writePath,
-                    targetPath,
-                    pu.getFileNames());
-        }
-        directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, false));
-        updateStatisticsTasks.add(
-            new UpdateStatisticsTask(
-                table.getDbName(),
-                table.getTableName(),
-                Optional.empty(),
-                ps,
-                true
-            ));
-    }
-
-    public void prepareOverwriteTable(THivePartitionUpdate pu, HivePartitionStatistics ps) {
-        String targetPath = pu.getLocation().getTargetPath();
-        String writePath = pu.getLocation().getWritePath();
-        if (!targetPath.equals(writePath)) {
-            Path path = new Path(targetPath);
-            String oldTablePath = new Path(path.getParent(), "_temp_" + path.getName()).toString();
-            Status status = fs.renameDir(
-                    targetPath,
-                    oldTablePath,
-                    () -> renameDirectoryTasksForAbort.add(new RenameDirectoryTask(oldTablePath, targetPath)));
-            if (!status.ok()) {
-                throw new RuntimeException(
-                    "Error to rename dir from " + targetPath + " to " + oldTablePath + status.getErrMsg());
-            }
-            clearDirsForFinish.add(oldTablePath);
-
-            status =  fs.renameDir(
-                writePath,
-                targetPath,
-                () -> directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true)));
-            if (!status.ok()) {
-                throw new RuntimeException(
-                    "Error to rename dir from " + writePath + " to " + targetPath + ":" + status.getErrMsg());
-            }
-        }
-        updateStatisticsTasks.add(
-            new UpdateStatisticsTask(
-                table.getDbName(),
-                table.getTableName(),
-                Optional.empty(),
-                ps,
-                false
-            ));
-    }
-
-    public void prepareCreateNewPartition(THivePartitionUpdate pu, HivePartitionStatistics ps) {
-
-        String targetPath = pu.getLocation().getTargetPath();
-        String writePath = pu.getLocation().getWritePath();
-
-        if (!targetPath.equals(writePath)) {
-            fs.asyncRenameDir(
-                    fileSystemExecutor,
-                    asyncFileSystemTaskFutures,
-                    fileSystemTaskCancelled,
-                    writePath,
-                    targetPath,
-                    () -> directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true)));
-        }
-
-        StorageDescriptor sd = table.getSd();
-
-        HivePartition hivePartition = new HivePartition(
-                table.getDbName(),
-                table.getTableName(),
-                false,
-                sd.getInputFormat(),
-                pu.getLocation().getTargetPath(),
-                HiveUtil.toPartitionValues(pu.getName()),
-                Maps.newHashMap(),
-                sd.getOutputFormat(),
-                sd.getSerdeInfo().getSerializationLib(),
-                hiveOps.getClient().getSchema(table.getDbName(), table.getTableName())
-        );
-        HivePartitionWithStatistics partitionWithStats =
-                new HivePartitionWithStatistics(pu.getName(), hivePartition, ps);
-        addPartitionsTask.addPartition(partitionWithStats);
-    }
-
-    public void prepareInsertExistPartition(List<Pair<THivePartitionUpdate, HivePartitionStatistics>> partitions) {
-        for (List<Pair<THivePartitionUpdate, HivePartitionStatistics>> partitionBatch :
-                    Iterables.partition(partitions, 100)) {
-            List<String> partitionNames = partitionBatch.stream()
-                    .map(pair -> pair.first.getName())
-                    .collect(Collectors.toList());
-
-            Map<String, Partition> partitionsByNamesMap = HiveUtil.convertToNamePartitionMap(
-                    partitionNames,
-                    hiveOps.getClient().getPartitions(table.getDbName(), table.getTableName(), partitionNames));
-
-            for (int i = 0; i < partitionsByNamesMap.size(); i++) {
-                String partitionName = partitionNames.get(i);
-                if (partitionsByNamesMap.get(partitionName) == null) {
-                    // Prevent this partition from being deleted by other engines
-                    throw new RuntimeException("Not found partition: " + partitionName);
-                }
-
-                THivePartitionUpdate pu = partitionBatch.get(i).first;
-                HivePartitionStatistics updateStats = partitionBatch.get(i).second;
-
-                String writePath = pu.getLocation().getWritePath();
-                String targetPath = pu.getLocation().getTargetPath();
-                directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, false));
-
-                if (!targetPath.equals(writePath)) {
-                    fs.asyncRename(
-                            fileSystemExecutor,
-                            asyncFileSystemTaskFutures,
-                            fileSystemTaskCancelled,
-                            writePath,
-                            targetPath,
-                            pu.getFileNames());
-                }
-
-                updateStatisticsTasks.add(
-                    new UpdateStatisticsTask(
-                            table.getDbName(),
-                            table.getTableName(),
-                            Optional.of(pu.getName()),
-                            updateStats,
-                            true));
-            }
-        }
-    }
-
-
-    public void prepareOverwritePartition(THivePartitionUpdate pu, HivePartitionStatistics ps) {
-        String targetPath = pu.getLocation().getTargetPath();
-        String writePath = pu.getLocation().getWritePath();
-        if (!targetPath.equals(writePath)) {
-            Path path = new Path(targetPath);
-            String oldPartitionPath = new Path(path.getParent(), "_temp_" + path.getName()).toString();
-            Status status = fs.renameDir(
-                    targetPath,
-                    oldPartitionPath,
-                    () -> renameDirectoryTasksForAbort.add(new RenameDirectoryTask(oldPartitionPath, targetPath)));
-            if (!status.ok()) {
-                throw new RuntimeException(
-                    "Error to rename dir from " + targetPath + " to " + oldPartitionPath + ":" + status.getErrMsg());
-            }
-            clearDirsForFinish.add(oldPartitionPath);
-
-            status = fs.renameDir(
-                    writePath,
-                    targetPath,
-                    () -> directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true)));
-            if (!status.ok()) {
-                throw new RuntimeException(
-                    "Error to rename dir from " + writePath + " to " + targetPath + ":" + status.getErrMsg());
-            }
-        }
-        updateStatisticsTasks.add(
-            new UpdateStatisticsTask(
-                table.getDbName(),
-                table.getTableName(),
-                Optional.of(pu.getName()),
-                ps,
-                false
-            ));
-    }
-
-
-    private void waitForAsyncFileSystemTasks() {
-        for (CompletableFuture<?> future : asyncFileSystemTaskFutures) {
-            MoreFutures.getFutureValue(future, RuntimeException.class);
-        }
-    }
-
-    private void doAddPartitionsTask() {
-        if (!addPartitionsTask.isEmpty()) {
-            addPartitionsTask.run(hiveOps);
-        }
-    }
-
-    private void doUpdateStatisticsTasks() {
-        ImmutableList.Builder<CompletableFuture<?>> updateStatsFutures = ImmutableList.builder();
-        List<String> failedTaskDescriptions = new ArrayList<>();
-        List<Throwable> suppressedExceptions = new ArrayList<>();
-        for (UpdateStatisticsTask task : updateStatisticsTasks) {
-            updateStatsFutures.add(CompletableFuture.runAsync(() -> {
-                try {
-                    task.run(hiveOps);
-                } catch (Throwable t) {
-                    synchronized (suppressedExceptions) {
-                        addSuppressedExceptions(suppressedExceptions, t, failedTaskDescriptions, task.getDescription());
-                    }
-                }
-            }, updateStatisticsExecutor));
-        }
-
-        for (CompletableFuture<?> executeUpdateFuture : updateStatsFutures.build()) {
-            MoreFutures.getFutureValue(executeUpdateFuture);
-        }
-        if (!suppressedExceptions.isEmpty()) {
-            StringBuilder message = new StringBuilder();
-            message.append("Failed to execute some updating statistics tasks: ");
-            Joiner.on("; ").appendTo(message, failedTaskDescriptions);
-            RuntimeException exception = new RuntimeException(message.toString());
-            suppressedExceptions.forEach(exception::addSuppressed);
-            throw exception;
-        }
-    }
-
-    private static void addSuppressedExceptions(
-            List<Throwable> suppressedExceptions,
-            Throwable t,
-            List<String> descriptions,
-            String description) {
-        descriptions.add(description);
-        // A limit is needed to avoid having a huge exception object. 5 was chosen arbitrarily.
-        if (suppressedExceptions.size() < 5) {
-            suppressedExceptions.add(t);
-        }
-    }
-
-    private static class AddPartition {
-
-    }
-
-    private static class UpdateStatisticsTask {
-        private final String dbName;
-        private final String tableName;
-        private final Optional<String> partitionName;
-        private final HivePartitionStatistics updatePartitionStat;
-        private final boolean merge;
-
-        private boolean done;
-
-        public UpdateStatisticsTask(String dbName, String tableName, Optional<String> partitionName,
-                                    HivePartitionStatistics statistics, boolean merge) {
-            this.dbName = Objects.requireNonNull(dbName, "dbName is null");
-            this.tableName = Objects.requireNonNull(tableName, "tableName is null");
-            this.partitionName = Objects.requireNonNull(partitionName, "partitionName is null");
-            this.updatePartitionStat = Objects.requireNonNull(statistics, "statistics is null");
-            this.merge = merge;
-        }
-
-        public void run(HiveMetadataOps hiveOps) {
-            if (partitionName.isPresent()) {
-                hiveOps.updatePartitionStatistics(dbName, tableName, partitionName.get(), this::updateStatistics);
-            } else {
-                hiveOps.updateTableStatistics(dbName, tableName, this::updateStatistics);
-            }
-            done = true;
-        }
-
-        public void undo(HiveMetadataOps hmsOps) {
-            if (!done) {
-                return;
-            }
-            if (partitionName.isPresent()) {
-                hmsOps.updatePartitionStatistics(dbName, tableName, partitionName.get(), this::resetStatistics);
-            } else {
-                hmsOps.updateTableStatistics(dbName, tableName, this::resetStatistics);
-            }
-        }
-
-        public String getDescription() {
-            if (partitionName.isPresent()) {
-                return "alter partition parameters " + tableName + " " + partitionName.get();
-            } else {
-                return "alter table parameters " +  tableName;
-            }
-        }
-
-        private HivePartitionStatistics updateStatistics(HivePartitionStatistics currentStats) {
-            return merge ? HivePartitionStatistics.merge(currentStats, updatePartitionStat) : updatePartitionStat;
-        }
-
-        private HivePartitionStatistics resetStatistics(HivePartitionStatistics currentStatistics) {
-            return HivePartitionStatistics
-                    .reduce(currentStatistics, updatePartitionStat, HivePartitionStatistics.ReduceOperator.SUBTRACT);
-        }
-    }
-
-    public static class AddPartitionsTask {
-        private final List<HivePartitionWithStatistics> partitions = new ArrayList<>();
-        private final List<List<String>> createdPartitionValues = new ArrayList<>();
-
-        public boolean isEmpty() {
-            return partitions.isEmpty();
-        }
-
-        public List<HivePartitionWithStatistics> getPartitions() {
-            return partitions;
-        }
-
-        public void addPartition(HivePartitionWithStatistics partition) {
-            partitions.add(partition);
-        }
-
-        public void run(HiveMetadataOps hiveOps) {
-            HivePartition firstPartition = partitions.get(0).getPartition();
-            String dbName = firstPartition.getDbName();
-            String tableName = firstPartition.getTblName();
-            List<List<HivePartitionWithStatistics>> batchedPartitions =
-                    Lists.partition(partitions, PARTITION_COMMIT_BATCH_SIZE);
-            for (List<HivePartitionWithStatistics> batch : batchedPartitions) {
-                try {
-                    hiveOps.addPartitions(dbName, tableName, batch);
-                    for (HivePartitionWithStatistics partition : batch) {
-                        createdPartitionValues.add(partition.getPartition().getPartitionValues());
-                    }
-                } catch (Throwable t) {
-                    LOG.warn("Failed to add partition", t);
-                    throw t;
-                }
-            }
-            partitions.clear();
-        }
-
-        public List<List<String>> rollback(HiveMetadataOps hiveOps) {
-            HivePartition firstPartition = partitions.get(0).getPartition();
-            String dbName = firstPartition.getDbName();
-            String tableName = firstPartition.getTblName();
-            List<List<String>> rollbackFailedPartitions = new ArrayList<>();
-            for (List<String> createdPartitionValue : createdPartitionValues) {
-                try {
-                    hiveOps.dropPartition(dbName, tableName, createdPartitionValue, false);
-                } catch (Throwable t) {
-                    LOG.warn("Failed to drop partition on {}.{}.{} when rollback",
-                            dbName, tableName, rollbackFailedPartitions);
-                    rollbackFailedPartitions.add(createdPartitionValue);
-                }
-            }
-            return rollbackFailedPartitions;
-        }
-    }
-
-    private static class DirectoryCleanUpTask {
-        private final Path path;
-        private final boolean deleteEmptyDir;
-
-        public DirectoryCleanUpTask(String path, boolean deleteEmptyDir) {
-            this.path = new Path(path);
-            this.deleteEmptyDir = deleteEmptyDir;
-        }
-
-        public Path getPath() {
-            return path;
-        }
-
-        public boolean isDeleteEmptyDir() {
-            return deleteEmptyDir;
-        }
-
-        @Override
-        public String toString() {
-            return new StringJoiner(", ", DirectoryCleanUpTask.class.getSimpleName() + "[", "]")
-                .add("path=" + path)
-                .add("deleteEmptyDir=" + deleteEmptyDir)
-                .toString();
-        }
-    }
-
-    public static class DeleteRecursivelyResult {
-        private final boolean dirNoLongerExists;
-        private final List<String> notDeletedEligibleItems;
-
-        public DeleteRecursivelyResult(boolean dirNoLongerExists, List<String> notDeletedEligibleItems) {
-            this.dirNoLongerExists = dirNoLongerExists;
-            this.notDeletedEligibleItems = notDeletedEligibleItems;
-        }
-
-        public boolean dirNotExists() {
-            return dirNoLongerExists;
-        }
-
-        public List<String> getNotDeletedEligibleItems() {
-            return notDeletedEligibleItems;
-        }
-    }
-
-    private void runDirectoryClearUpTasksForAbort() {
-        for (DirectoryCleanUpTask cleanUpTask : directoryCleanUpTasksForAbort) {
-            recursiveDeleteItems(cleanUpTask.getPath(), cleanUpTask.isDeleteEmptyDir());
-        }
-    }
-
-    private static class RenameDirectoryTask {
-        private final String renameFrom;
-        private final String renameTo;
-
-        public RenameDirectoryTask(String renameFrom, String renameTo) {
-            this.renameFrom = renameFrom;
-            this.renameTo = renameTo;
-        }
-
-        public String getRenameFrom() {
-            return renameFrom;
-        }
-
-        public String getRenameTo() {
-            return renameTo;
-        }
-
-        @Override
-        public String toString() {
-            return new StringJoiner(", ", RenameDirectoryTask.class.getSimpleName() + "[", "]")
-                .add("renameFrom:" + renameFrom)
-                .add("renameTo:" + renameTo)
-                .toString();
-        }
-    }
-
-    private void runRenameDirTasksForAbort() {
-        Status status;
-        for (RenameDirectoryTask task : renameDirectoryTasksForAbort) {
-            status = fs.exists(task.getRenameFrom());
-            if (status.ok()) {
-                status = fs.renameDir(task.getRenameFrom(), task.getRenameTo(), () -> {});
-                if (!status.ok()) {
-                    LOG.warn("Failed to abort rename dir from {} to {}:{}",
-                            task.getRenameFrom(), task.getRenameTo(), status.getErrMsg());
-                }
-            }
-        }
-    }
-
-    private void runClearPathsForFinish() {
-        Status status;
-        for (String path : clearDirsForFinish) {
-            status = fs.delete(path);
-            if (!status.ok()) {
-                LOG.warn("Failed to recursively delete path {}:{}", path, status.getErrCode());
-            }
-        }
-    }
-
-
-    private void recursiveDeleteItems(Path directory, boolean deleteEmptyDir) {
-        DeleteRecursivelyResult deleteResult = recursiveDeleteFiles(directory, deleteEmptyDir);
-
-        if (!deleteResult.getNotDeletedEligibleItems().isEmpty()) {
-            LOG.warn("Failed to delete directory {}. Some eligible items can't be deleted: {}.",
-                    directory.toString(), deleteResult.getNotDeletedEligibleItems());
-        } else if (deleteEmptyDir && !deleteResult.dirNotExists()) {
-            LOG.warn("Failed to delete directory {} due to dir isn't empty", directory.toString());
-        }
-    }
-
-    public DeleteRecursivelyResult recursiveDeleteFiles(Path directory, boolean deleteEmptyDir) {
-        try {
-            if (!fs.exists(directory.getName()).ok()) {
-                return new DeleteRecursivelyResult(true, ImmutableList.of());
-            }
-        } catch (Exception e) {
-            ImmutableList.Builder<String> notDeletedEligibleItems = ImmutableList.builder();
-            notDeletedEligibleItems.add(directory.toString() + "/*");
-            return new DeleteRecursivelyResult(false, notDeletedEligibleItems.build());
-        }
-
-        return doRecursiveDeleteFiles(directory, deleteEmptyDir);
-    }
-
-    private DeleteRecursivelyResult doRecursiveDeleteFiles(Path directory, boolean deleteEmptyDir) {
-        List<RemoteFile> remoteFiles = new ArrayList<>();
-
-        Status status = fs.list(directory.getName(), remoteFiles);
-        if (!status.ok()) {
-            ImmutableList.Builder<String> notDeletedEligibleItems = ImmutableList.builder();
-            notDeletedEligibleItems.add(directory + "/*");
-            return new DeleteRecursivelyResult(false, notDeletedEligibleItems.build());
-        }
-
-        boolean isEmptyDir = true;
-        List<String> notDeletedEligibleItems = new ArrayList<>();
-        for (RemoteFile file : remoteFiles) {
-            if (file.isFile()) {
-                Path filePath = file.getPath();
-                isEmptyDir = false;
-                // TODO Check if this file was created by this query
-                if (!deleteIfExists(filePath)) {
-                    notDeletedEligibleItems.add(filePath.toString());
-                }
-            } else if (file.isDirectory()) {
-                DeleteRecursivelyResult subResult = doRecursiveDeleteFiles(file.getPath(), deleteEmptyDir);
-                if (!subResult.dirNotExists()) {
-                    isEmptyDir = false;
-                }
-                if (!subResult.getNotDeletedEligibleItems().isEmpty()) {
-                    notDeletedEligibleItems.addAll(subResult.getNotDeletedEligibleItems());
-                }
-            } else {
-                isEmptyDir = false;
-                notDeletedEligibleItems.add(file.getPath().toString());
-            }
-        }
-
-        if (isEmptyDir && deleteEmptyDir) {
-            Verify.verify(notDeletedEligibleItems.isEmpty());
-            if (!deleteIfExists(directory)) {
-                return new DeleteRecursivelyResult(false, ImmutableList.of(directory + "/"));
-            }
-            // all items of the location have been deleted.
-            return new DeleteRecursivelyResult(true, ImmutableList.of());
-        }
-
-        return new DeleteRecursivelyResult(false, notDeletedEligibleItems);
-    }
-
-    public boolean deleteIfExists(Path path) {
-        Status status = fs.delete(path.getName());
-        if (status.ok()) {
-            return true;
-        }
-        return !fs.exists(path.getName()).ok();
-    }
-}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java
index 0f2a7bb2acb..4474e546500 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java
@@ -34,6 +34,7 @@ import org.apache.doris.datasource.jdbc.client.JdbcClientConfig;
 import org.apache.doris.datasource.operations.ExternalMetadataOperations;
 import org.apache.doris.datasource.property.PropertyConverter;
 import org.apache.doris.datasource.property.constants.HMSProperties;
+import org.apache.doris.transaction.TransactionManagerFactory;
 
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
@@ -145,7 +146,10 @@ public class HMSExternalCatalog extends ExternalCatalog {
                     AuthenticationConfig.HADOOP_KERBEROS_PRINCIPAL,
                     AuthenticationConfig.HADOOP_KERBEROS_KEYTAB));
         }
-        metadataOps = ExternalMetadataOperations.newHiveMetadataOps(hiveConf, jdbcClientConfig, this);
+        HiveMetadataOps hiveOps = ExternalMetadataOperations.newHiveMetadataOps(hiveConf, jdbcClientConfig, this);
+        transactionManager = TransactionManagerFactory.createHiveTransactionManager(hiveOps);
+        transactionManager.setEditLog(Env.getCurrentEnv().getEditLog());
+        metadataOps = hiveOps;
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java
new file mode 100644
index 00000000000..c3e8d00c5d1
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java
@@ -0,0 +1,1322 @@
+// 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.
+// This file is copied from
+// https://github.com/trinodb/trino/blob/438/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java
+// https://github.com/trinodb/trino/blob/438/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java
+// and modified by Doris
+
+package org.apache.doris.datasource.hive;
+
+import org.apache.doris.backup.Status;
+import org.apache.doris.common.Pair;
+import org.apache.doris.fs.remote.RemoteFile;
+import org.apache.doris.fs.remote.RemoteFileSystem;
+import org.apache.doris.thrift.THivePartitionUpdate;
+import org.apache.doris.thrift.TUpdateMode;
+import org.apache.doris.transaction.Transaction;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Verify;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import io.airlift.concurrent.MoreFutures;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.StringJoiner;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+public class HMSTransaction implements Transaction {
+    private static final Logger LOG = LogManager.getLogger(HMSTransaction.class);
+    private final HiveMetadataOps hiveOps;
+    private final RemoteFileSystem fs;
+    private String dbName;
+    private String tbName;
+
+    private final Map<DatabaseTableName, Action<TableAndMore>> tableActions = new HashMap<>();
+    private final Map<DatabaseTableName, Map<List<String>, Action<PartitionAndMore>>>
+            partitionActions = new HashMap<>();
+
+    private HmsCommitter hmsCommitter;
+    private List<THivePartitionUpdate> hivePartitionUpdates = Lists.newArrayList();
+
+    public HMSTransaction(HiveMetadataOps hiveOps) {
+        this.hiveOps = hiveOps;
+        this.fs = hiveOps.getFs();
+    }
+
+    @Override
+    public void commit() {
+        doCommit();
+    }
+
+    public String getDbName() {
+        return dbName;
+    }
+
+    public String getTbName() {
+        return tbName;
+    }
+
+    public List<THivePartitionUpdate> mergePartitions(List<THivePartitionUpdate> hivePUs) {
+        Map<String, THivePartitionUpdate> mm = new HashMap<>();
+        for (THivePartitionUpdate pu : hivePUs) {
+            if (mm.containsKey(pu.getName())) {
+                THivePartitionUpdate old = mm.get(pu.getName());
+                old.setFileSize(old.getFileSize() + pu.getFileSize());
+                old.setRowCount(old.getRowCount() + pu.getRowCount());
+                old.getFileNames().addAll(pu.getFileNames());
+            } else {
+                mm.put(pu.getName(), pu);
+            }
+        }
+        return new ArrayList<>(mm.values());
+    }
+
+    @Override
+    public void rollback() {
+        if (hmsCommitter != null) {
+            hmsCommitter.rollback();
+        }
+    }
+
+    public void finishInsertTable(String dbName, String tbName) {
+        this.tbName = tbName;
+        this.dbName = dbName;
+        List<THivePartitionUpdate> mergedPUs = mergePartitions(hivePartitionUpdates);
+        Table table = getTable(dbName, tbName);
+        List<Pair<THivePartitionUpdate, HivePartitionStatistics>> insertExistsPartitions = new ArrayList<>();
+        for (THivePartitionUpdate pu : mergedPUs) {
+            TUpdateMode updateMode = pu.getUpdateMode();
+            HivePartitionStatistics hivePartitionStatistics = HivePartitionStatistics.fromCommonStatistics(
+                    pu.getRowCount(),
+                    pu.getFileNamesSize(),
+                    pu.getFileSize());
+            String writePath = pu.getLocation().getWritePath();
+            if (table.getPartitionKeysSize() == 0) {
+                Preconditions.checkArgument(mergedPUs.size() == 1,
+                        "When updating a non-partitioned table, multiple partitions should not be written");
+                switch (updateMode) {
+                    case APPEND:
+                        finishChangingExistingTable(
+                                ActionType.INSERT_EXISTING,
+                                dbName,
+                                tbName,
+                                writePath,
+                                pu.getFileNames(),
+                                hivePartitionStatistics);
+                        break;
+                    case OVERWRITE:
+                        dropTable(dbName, tbName);
+                        createTable(table, writePath, pu.getFileNames(), hivePartitionStatistics);
+                        break;
+                    default:
+                        throw new RuntimeException("Not support mode:[" + updateMode + "] in unPartitioned table");
+                }
+            } else {
+                switch (updateMode) {
+                    case APPEND:
+                        // insert into existing partition
+                        insertExistsPartitions.add(Pair.of(pu, hivePartitionStatistics));
+                        break;
+                    case NEW:
+                    case OVERWRITE:
+                        StorageDescriptor sd = table.getSd();
+                        HivePartition hivePartition = new HivePartition(
+                                dbName,
+                                tbName,
+                                false,
+                                sd.getInputFormat(),
+                                pu.getLocation().getTargetPath(),
+                                HiveUtil.toPartitionValues(pu.getName()),
+                                Maps.newHashMap(),
+                                sd.getOutputFormat(),
+                                sd.getSerdeInfo().getSerializationLib(),
+                                hiveOps.getClient().getSchema(dbName, tbName)
+                        );
+                        if (updateMode == TUpdateMode.OVERWRITE) {
+                            dropPartition(dbName, tbName, hivePartition.getPartitionValues(), true);
+                        }
+                        addPartition(
+                                dbName, tbName, hivePartition, writePath,
+                                pu.getName(), pu.getFileNames(), hivePartitionStatistics);
+                        break;
+                    default:
+                        throw new RuntimeException("Not support mode:[" + updateMode + "] in partitioned table");
+                }
+            }
+        }
+
+        if (!insertExistsPartitions.isEmpty()) {
+            convertToInsertExistingPartitionAction(insertExistsPartitions);
+        }
+    }
+
+    public void doCommit() {
+        hmsCommitter = new HmsCommitter();
+
+        try {
+            for (Map.Entry<DatabaseTableName, Action<TableAndMore>> entry : tableActions.entrySet()) {
+                Action<TableAndMore> action = entry.getValue();
+                switch (action.getType()) {
+                    case INSERT_EXISTING:
+                        hmsCommitter.prepareInsertExistingTable(action.getData());
+                        break;
+                    case ALTER:
+                        hmsCommitter.prepareAlterTable(action.getData());
+                        break;
+                    default:
+                        throw new UnsupportedOperationException("Unsupported table action type: " + action.getType());
+                }
+            }
+
+            for (Map.Entry<DatabaseTableName, Map<List<String>, Action<PartitionAndMore>>> tableEntry
+                    : partitionActions.entrySet()) {
+                for (Map.Entry<List<String>, Action<PartitionAndMore>> partitionEntry :
+                        tableEntry.getValue().entrySet()) {
+                    Action<PartitionAndMore> action = partitionEntry.getValue();
+                    switch (action.getType()) {
+                        case INSERT_EXISTING:
+                            hmsCommitter.prepareInsertExistPartition(action.getData());
+                            break;
+                        case ADD:
+                            hmsCommitter.prepareAddPartition(action.getData());
+                            break;
+                        case ALTER:
+                            hmsCommitter.prepareAlterPartition(action.getData());
+                            break;
+                        default:
+                            throw new UnsupportedOperationException(
+                                "Unsupported partition action type: " + action.getType());
+                    }
+                }
+            }
+
+            hmsCommitter.waitForAsyncFileSystemTasks();
+            hmsCommitter.doAddPartitionsTask();
+            hmsCommitter.doUpdateStatisticsTasks();
+        } catch (Throwable t) {
+            LOG.warn("Failed to commit for {}.{}, abort it.", dbName, tbName);
+            hmsCommitter.cancelUnStartedAsyncFileSystemTask();
+            hmsCommitter.undoUpdateStatisticsTasks();
+            hmsCommitter.undoAddPartitionsTask();
+            hmsCommitter.waitForAsyncFileSystemTaskSuppressThrowable();
+            hmsCommitter.runDirectoryClearUpTasksForAbort();
+            hmsCommitter.runRenameDirTasksForAbort();
+            throw t;
+        } finally {
+            hmsCommitter.runClearPathsForFinish();
+        }
+    }
+
+    public void updateHivePartitionUpdates(List<THivePartitionUpdate> pus) {
+        synchronized (this) {
+            hivePartitionUpdates.addAll(pus);
+        }
+    }
+
+    // for test
+    public void setHivePartitionUpdates(List<THivePartitionUpdate> hivePartitionUpdates) {
+        this.hivePartitionUpdates = hivePartitionUpdates;
+    }
+
+    public long getUpdateCnt() {
+        return hivePartitionUpdates.stream().mapToLong(THivePartitionUpdate::getRowCount).sum();
+    }
+
+    private void convertToInsertExistingPartitionAction(
+            List<Pair<THivePartitionUpdate, HivePartitionStatistics>> partitions) {
+        DatabaseTableName databaseTableName = new DatabaseTableName(dbName, tbName);
+        Map<List<String>, Action<PartitionAndMore>> partitionActionsForTable =
+                partitionActions.computeIfAbsent(databaseTableName, k -> new HashMap<>());
+
+        for (List<Pair<THivePartitionUpdate, HivePartitionStatistics>> partitionBatch :
+                Iterables.partition(partitions, 100)) {
+
+            List<String> partitionNames = partitionBatch
+                    .stream()
+                    .map(pair -> pair.first.getName())
+                    .collect(Collectors.toList());
+
+            // check in partitionAction
+            Action<PartitionAndMore> oldPartitionAction = partitionActionsForTable.get(partitionNames);
+            if (oldPartitionAction != null) {
+                switch (oldPartitionAction.getType()) {
+                    case DROP:
+                    case DROP_PRESERVE_DATA:
+                        throw new RuntimeException(
+                                "Not found partition from partition actions"
+                                        + "for " + databaseTableName + ", partitions: " + partitionNames);
+                    case ADD:
+                    case ALTER:
+                    case INSERT_EXISTING:
+                    case MERGE:
+                        throw new UnsupportedOperationException(
+                                "Inserting into a partition that were added, altered,"
+                                        + "or inserted into in the same transaction is not supported");
+                    default:
+                        throw new IllegalStateException("Unknown action type: " + oldPartitionAction.getType());
+                }
+            }
+
+            Map<String, Partition> partitionsByNamesMap = HiveUtil.convertToNamePartitionMap(
+                    partitionNames,
+                    hiveOps.getClient().getPartitions(dbName, tbName, partitionNames));
+
+            for (int i = 0; i < partitionsByNamesMap.size(); i++) {
+                String partitionName = partitionNames.get(i);
+                // check from hms
+                Partition partition = partitionsByNamesMap.get(partitionName);
+                if (partition == null) {
+                    // Prevent this partition from being deleted by other engines
+                    throw new RuntimeException(
+                            "Not found partition from hms for " + databaseTableName
+                                    + ", partitions: " + partitionNames);
+                }
+                THivePartitionUpdate pu = partitionBatch.get(i).first;
+                HivePartitionStatistics updateStats = partitionBatch.get(i).second;
+
+                StorageDescriptor sd = partition.getSd();
+                List<String> partitionValues = HiveUtil.toPartitionValues(pu.getName());
+
+                HivePartition hivePartition = new HivePartition(
+                        dbName,
+                        tbName,
+                        false,
+                        sd.getInputFormat(),
+                        partition.getSd().getLocation(),
+                        partitionValues,
+                        partition.getParameters(),
+                        sd.getOutputFormat(),
+                        sd.getSerdeInfo().getSerializationLib(),
+                        hiveOps.getClient().getSchema(dbName, tbName)
+                );
+
+                partitionActionsForTable.put(
+                        partitionValues,
+                        new Action<>(
+                                ActionType.INSERT_EXISTING,
+                                new PartitionAndMore(
+                                    hivePartition,
+                                    pu.getLocation().getWritePath(),
+                                    pu.getName(),
+                                    pu.getFileNames(),
+                                    updateStats
+                                ))
+                );
+            }
+        }
+    }
+
+    private static void addSuppressedExceptions(
+            List<Throwable> suppressedExceptions,
+            Throwable t,
+            List<String> descriptions,
+            String description) {
+        descriptions.add(description);
+        // A limit is needed to avoid having a huge exception object. 5 was chosen arbitrarily.
+        if (suppressedExceptions.size() < 5) {
+            suppressedExceptions.add(t);
+        }
+    }
+
+    private static class UpdateStatisticsTask {
+        private final String dbName;
+        private final String tableName;
+        private final Optional<String> partitionName;
+        private final HivePartitionStatistics updatePartitionStat;
+        private final boolean merge;
+
+        private boolean done;
+
+        public UpdateStatisticsTask(String dbName, String tableName, Optional<String> partitionName,
+                                    HivePartitionStatistics statistics, boolean merge) {
+            this.dbName = Objects.requireNonNull(dbName, "dbName is null");
+            this.tableName = Objects.requireNonNull(tableName, "tableName is null");
+            this.partitionName = Objects.requireNonNull(partitionName, "partitionName is null");
+            this.updatePartitionStat = Objects.requireNonNull(statistics, "statistics is null");
+            this.merge = merge;
+        }
+
+        public void run(HiveMetadataOps hiveOps) {
+            if (partitionName.isPresent()) {
+                hiveOps.updatePartitionStatistics(dbName, tableName, partitionName.get(), this::updateStatistics);
+            } else {
+                hiveOps.updateTableStatistics(dbName, tableName, this::updateStatistics);
+            }
+            done = true;
+        }
+
+        public void undo(HiveMetadataOps hmsOps) {
+            if (!done) {
+                return;
+            }
+            if (partitionName.isPresent()) {
+                hmsOps.updatePartitionStatistics(dbName, tableName, partitionName.get(), this::resetStatistics);
+            } else {
+                hmsOps.updateTableStatistics(dbName, tableName, this::resetStatistics);
+            }
+        }
+
+        public String getDescription() {
+            if (partitionName.isPresent()) {
+                return "alter partition parameters " + tableName + " " + partitionName.get();
+            } else {
+                return "alter table parameters " +  tableName;
+            }
+        }
+
+        private HivePartitionStatistics updateStatistics(HivePartitionStatistics currentStats) {
+            return merge ? HivePartitionStatistics.merge(currentStats, updatePartitionStat) : updatePartitionStat;
+        }
+
+        private HivePartitionStatistics resetStatistics(HivePartitionStatistics currentStatistics) {
+            return HivePartitionStatistics
+                    .reduce(currentStatistics, updatePartitionStat, HivePartitionStatistics.ReduceOperator.SUBTRACT);
+        }
+    }
+
+    public static class AddPartitionsTask {
+        private final List<HivePartitionWithStatistics> partitions = new ArrayList<>();
+        private final List<List<String>> createdPartitionValues = new ArrayList<>();
+
+        public boolean isEmpty() {
+            return partitions.isEmpty();
+        }
+
+        public List<HivePartitionWithStatistics> getPartitions() {
+            return partitions;
+        }
+
+        public void addPartition(HivePartitionWithStatistics partition) {
+            partitions.add(partition);
+        }
+
+        public void run(HiveMetadataOps hiveOps) {
+            HivePartition firstPartition = partitions.get(0).getPartition();
+            String dbName = firstPartition.getDbName();
+            String tableName = firstPartition.getTblName();
+            List<List<HivePartitionWithStatistics>> batchedPartitions = Lists.partition(partitions, 20);
+            for (List<HivePartitionWithStatistics> batch : batchedPartitions) {
+                try {
+                    hiveOps.addPartitions(dbName, tableName, batch);
+                    for (HivePartitionWithStatistics partition : batch) {
+                        createdPartitionValues.add(partition.getPartition().getPartitionValues());
+                    }
+                } catch (Throwable t) {
+                    LOG.warn("Failed to add partition", t);
+                    throw t;
+                }
+            }
+            partitions.clear();
+        }
+
+        public List<List<String>> rollback(HiveMetadataOps hiveOps) {
+            HivePartition firstPartition = partitions.get(0).getPartition();
+            String dbName = firstPartition.getDbName();
+            String tableName = firstPartition.getTblName();
+            List<List<String>> rollbackFailedPartitions = new ArrayList<>();
+            for (List<String> createdPartitionValue : createdPartitionValues) {
+                try {
+                    hiveOps.dropPartition(dbName, tableName, createdPartitionValue, false);
+                } catch (Throwable t) {
+                    LOG.warn("Failed to drop partition on {}.{}.{} when rollback",
+                            dbName, tableName, rollbackFailedPartitions);
+                    rollbackFailedPartitions.add(createdPartitionValue);
+                }
+            }
+            return rollbackFailedPartitions;
+        }
+    }
+
+    private static class DirectoryCleanUpTask {
+        private final Path path;
+        private final boolean deleteEmptyDir;
+
+        public DirectoryCleanUpTask(String path, boolean deleteEmptyDir) {
+            this.path = new Path(path);
+            this.deleteEmptyDir = deleteEmptyDir;
+        }
+
+        public Path getPath() {
+            return path;
+        }
+
+        public boolean isDeleteEmptyDir() {
+            return deleteEmptyDir;
+        }
+
+        @Override
+        public String toString() {
+            return new StringJoiner(", ", DirectoryCleanUpTask.class.getSimpleName() + "[", "]")
+                .add("path=" + path)
+                .add("deleteEmptyDir=" + deleteEmptyDir)
+                .toString();
+        }
+    }
+
+    private static class DeleteRecursivelyResult {
+        private final boolean dirNoLongerExists;
+        private final List<String> notDeletedEligibleItems;
+
+        public DeleteRecursivelyResult(boolean dirNoLongerExists, List<String> notDeletedEligibleItems) {
+            this.dirNoLongerExists = dirNoLongerExists;
+            this.notDeletedEligibleItems = notDeletedEligibleItems;
+        }
+
+        public boolean dirNotExists() {
+            return dirNoLongerExists;
+        }
+
+        public List<String> getNotDeletedEligibleItems() {
+            return notDeletedEligibleItems;
+        }
+    }
+
+    private static class RenameDirectoryTask {
+        private final String renameFrom;
+        private final String renameTo;
+
+        public RenameDirectoryTask(String renameFrom, String renameTo) {
+            this.renameFrom = renameFrom;
+            this.renameTo = renameTo;
+        }
+
+        public String getRenameFrom() {
+            return renameFrom;
+        }
+
+        public String getRenameTo() {
+            return renameTo;
+        }
+
+        @Override
+        public String toString() {
+            return new StringJoiner(", ", RenameDirectoryTask.class.getSimpleName() + "[", "]")
+                .add("renameFrom:" + renameFrom)
+                .add("renameTo:" + renameTo)
+                .toString();
+        }
+    }
+
+
+
+    private void recursiveDeleteItems(Path directory, boolean deleteEmptyDir) {
+        DeleteRecursivelyResult deleteResult = recursiveDeleteFiles(directory, deleteEmptyDir);
+
+        if (!deleteResult.getNotDeletedEligibleItems().isEmpty()) {
+            LOG.warn("Failed to delete directory {}. Some eligible items can't be deleted: {}.",
+                    directory.toString(), deleteResult.getNotDeletedEligibleItems());
+        } else if (deleteEmptyDir && !deleteResult.dirNotExists()) {
+            LOG.warn("Failed to delete directory {} due to dir isn't empty", directory.toString());
+        }
+    }
+
+    private DeleteRecursivelyResult recursiveDeleteFiles(Path directory, boolean deleteEmptyDir) {
+        try {
+            if (!fs.exists(directory.getName()).ok()) {
+                return new DeleteRecursivelyResult(true, ImmutableList.of());
+            }
+        } catch (Exception e) {
+            ImmutableList.Builder<String> notDeletedEligibleItems = ImmutableList.builder();
+            notDeletedEligibleItems.add(directory.toString() + "/*");
+            return new DeleteRecursivelyResult(false, notDeletedEligibleItems.build());
+        }
+
+        return doRecursiveDeleteFiles(directory, deleteEmptyDir);
+    }
+
+    private DeleteRecursivelyResult doRecursiveDeleteFiles(Path directory, boolean deleteEmptyDir) {
+        List<RemoteFile> remoteFiles = new ArrayList<>();
+
+        Status status = fs.list(directory.getName(), remoteFiles);
+        if (!status.ok()) {
+            ImmutableList.Builder<String> notDeletedEligibleItems = ImmutableList.builder();
+            notDeletedEligibleItems.add(directory + "/*");
+            return new DeleteRecursivelyResult(false, notDeletedEligibleItems.build());
+        }
+
+        boolean isEmptyDir = true;
+        List<String> notDeletedEligibleItems = new ArrayList<>();
+        for (RemoteFile file : remoteFiles) {
+            if (file.isFile()) {
+                Path filePath = file.getPath();
+                isEmptyDir = false;
+                // TODO Check if this file was created by this query
+                if (!deleteIfExists(filePath)) {
+                    notDeletedEligibleItems.add(filePath.toString());
+                }
+            } else if (file.isDirectory()) {
+                DeleteRecursivelyResult subResult = doRecursiveDeleteFiles(file.getPath(), deleteEmptyDir);
+                if (!subResult.dirNotExists()) {
+                    isEmptyDir = false;
+                }
+                if (!subResult.getNotDeletedEligibleItems().isEmpty()) {
+                    notDeletedEligibleItems.addAll(subResult.getNotDeletedEligibleItems());
+                }
+            } else {
+                isEmptyDir = false;
+                notDeletedEligibleItems.add(file.getPath().toString());
+            }
+        }
+
+        if (isEmptyDir && deleteEmptyDir) {
+            Verify.verify(notDeletedEligibleItems.isEmpty());
+            if (!deleteIfExists(directory)) {
+                return new DeleteRecursivelyResult(false, ImmutableList.of(directory + "/"));
+            }
+            // all items of the location have been deleted.
+            return new DeleteRecursivelyResult(true, ImmutableList.of());
+        }
+
+        return new DeleteRecursivelyResult(false, notDeletedEligibleItems);
+    }
+
+    public boolean deleteIfExists(Path path) {
+        Status status = fs.delete(path.getName());
+        if (status.ok()) {
+            return true;
+        }
+        return !fs.exists(path.getName()).ok();
+    }
+
+    public static class DatabaseTableName {
+        private final String dbName;
+        private final String tbName;
+
+        public DatabaseTableName(String dbName, String tbName) {
+            this.dbName = dbName;
+            this.tbName = tbName;
+        }
+
+        @Override
+        public boolean equals(Object other) {
+            if (this == other) {
+                return true;
+            }
+
+            if (other == null || getClass() != other.getClass()) {
+                return false;
+            }
+
+            DatabaseTableName that = (DatabaseTableName) other;
+            return Objects.equals(dbName, that.dbName) && Objects.equals(tbName, that.tbName);
+        }
+
+        @Override
+        public String toString() {
+            return dbName + "." + tbName;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(dbName, tbName);
+        }
+
+        public String getTbName() {
+            return tbName;
+        }
+
+        public String getDbName() {
+            return dbName;
+        }
+    }
+
+    private static class TableAndMore {
+        private final Table table;
+        private final String currentLocation;
+        private final List<String> fileNames;
+        private final HivePartitionStatistics statisticsUpdate;
+
+        public TableAndMore(
+                Table table,
+                String currentLocation,
+                List<String> fileNames,
+                HivePartitionStatistics statisticsUpdate) {
+            this.table = Objects.requireNonNull(table, "table is null");
+            this.currentLocation = Objects.requireNonNull(currentLocation);
+            this.fileNames = Objects.requireNonNull(fileNames);
+            this.statisticsUpdate = Objects.requireNonNull(statisticsUpdate, "statisticsUpdate is null");
+        }
+
+        public Table getTable() {
+            return table;
+        }
+
+        public String getCurrentLocation() {
+            return currentLocation;
+        }
+
+        public List<String> getFileNames() {
+            return fileNames;
+        }
+
+        public HivePartitionStatistics getStatisticsUpdate() {
+            return statisticsUpdate;
+        }
+
+        @Override
+        public String toString() {
+            return MoreObjects.toStringHelper(this)
+                .add("table", table)
+                .add("statisticsUpdate", statisticsUpdate)
+                .toString();
+        }
+    }
+
+    private static class PartitionAndMore {
+        private final HivePartition partition;
+        private final String currentLocation;
+        private final String partitionName;
+        private final List<String> fileNames;
+        private final HivePartitionStatistics statisticsUpdate;
+
+        public PartitionAndMore(
+                HivePartition partition,
+                String currentLocation,
+                String partitionName,
+                List<String> fileNames,
+                HivePartitionStatistics statisticsUpdate) {
+            this.partition = Objects.requireNonNull(partition, "partition is null");
+            this.currentLocation = Objects.requireNonNull(currentLocation, "currentLocation is null");
+            this.partitionName = Objects.requireNonNull(partitionName, "partition is null");
+            this.fileNames = Objects.requireNonNull(fileNames, "fileNames is null");
+            this.statisticsUpdate = Objects.requireNonNull(statisticsUpdate, "statisticsUpdate is null");
+        }
+
+        public HivePartition getPartition() {
+            return partition;
+        }
+
+        public String getCurrentLocation() {
+            return currentLocation;
+        }
+
+        public String getPartitionName() {
+            return partitionName;
+        }
+
+        public List<String> getFileNames() {
+            return fileNames;
+        }
+
+        public HivePartitionStatistics getStatisticsUpdate() {
+            return statisticsUpdate;
+        }
+
+        @Override
+        public String toString() {
+            return MoreObjects.toStringHelper(this)
+                .add("partition", partition)
+                .add("currentLocation", currentLocation)
+                .add("fileNames", fileNames)
+                .toString();
+        }
+    }
+
+    private enum ActionType {
+        // drop a table/partition
+        DROP,
+        // drop a table/partition but will preserve data
+        DROP_PRESERVE_DATA,
+        // add a table/partition
+        ADD,
+        // drop then add a table/partition, like overwrite
+        ALTER,
+        // insert into an existing table/partition
+        INSERT_EXISTING,
+        // merger into an existing table/partition
+        MERGE,
+    }
+
+    public static class Action<T> {
+        private final ActionType type;
+        private final T data;
+
+        public Action(ActionType type, T data) {
+            this.type = Objects.requireNonNull(type, "type is null");
+            if (type == ActionType.DROP || type == ActionType.DROP_PRESERVE_DATA) {
+                Preconditions.checkArgument(data == null, "data is not null");
+            } else {
+                Objects.requireNonNull(data, "data is null");
+            }
+            this.data = data;
+        }
+
+        public ActionType getType() {
+            return type;
+        }
+
+        public T getData() {
+            Preconditions.checkState(type != ActionType.DROP);
+            return data;
+        }
+
+        @Override
+        public String toString() {
+            return MoreObjects.toStringHelper(this)
+                .add("type", type)
+                .add("data", data)
+                .toString();
+        }
+    }
+
+    public synchronized Table getTable(String databaseName, String tableName) {
+        Action<TableAndMore> tableAction = tableActions.get(new DatabaseTableName(databaseName, tableName));
+        if (tableAction == null) {
+            return hiveOps.getClient().getTable(databaseName, tableName);
+        }
+        switch (tableAction.getType()) {
+            case ADD:
+            case ALTER:
+            case INSERT_EXISTING:
+            case MERGE:
+                return tableAction.getData().getTable();
+            case DROP:
+            case DROP_PRESERVE_DATA:
+                break;
+            default:
+                throw new IllegalStateException("Unknown action type: " + tableAction.getType());
+        }
+        throw new RuntimeException("Not Found table: " + databaseName + "." + tableName);
+    }
+
+    public synchronized void finishChangingExistingTable(
+            ActionType actionType,
+            String databaseName,
+            String tableName,
+            String location,
+            List<String> fileNames,
+            HivePartitionStatistics statisticsUpdate) {
+        DatabaseTableName databaseTableName = new DatabaseTableName(databaseName, tableName);
+        Action<TableAndMore> oldTableAction = tableActions.get(databaseTableName);
+        if (oldTableAction == null) {
+            Table table = hiveOps.getClient().getTable(databaseTableName.getDbName(), databaseTableName.getTbName());
+            tableActions.put(
+                    databaseTableName,
+                    new Action<>(
+                        actionType,
+                            new TableAndMore(
+                                table,
+                                location,
+                                fileNames,
+                                statisticsUpdate)));
+            return;
+        }
+
+        switch (oldTableAction.getType()) {
+            case DROP:
+                throw new RuntimeException("Not found table: " + databaseTableName);
+            case ADD:
+            case ALTER:
+            case INSERT_EXISTING:
+            case MERGE:
+                throw new UnsupportedOperationException(
+                        "Inserting into an unpartitioned table that were added, altered,"
+                                + "or inserted into in the same transaction is not supported");
+            case DROP_PRESERVE_DATA:
+                break;
+            default:
+                throw new IllegalStateException("Unknown action type: " + oldTableAction.getType());
+        }
+    }
+
+    public synchronized void createTable(
+            Table table, String location, List<String> fileNames,  HivePartitionStatistics statistics) {
+        // When creating a table, it should never have partition actions. This is just a sanity check.
+        checkNoPartitionAction(dbName, tbName);
+        DatabaseTableName databaseTableName = new DatabaseTableName(dbName, tbName);
+        Action<TableAndMore> oldTableAction = tableActions.get(databaseTableName);
+        TableAndMore tableAndMore = new TableAndMore(table, location, fileNames, statistics);
+        if (oldTableAction == null) {
+            tableActions.put(databaseTableName, new Action<>(ActionType.ADD, tableAndMore));
+            return;
+        }
+        switch (oldTableAction.getType()) {
+            case DROP:
+                tableActions.put(databaseTableName, new Action<>(ActionType.ALTER, tableAndMore));
+                return;
+
+            case ADD:
+            case ALTER:
+            case INSERT_EXISTING:
+            case MERGE:
+                throw new RuntimeException("Table already exists: " + databaseTableName);
+            case DROP_PRESERVE_DATA:
+                break;
+            default:
+                throw new IllegalStateException("Unknown action type: " + oldTableAction.getType());
+        }
+    }
+
+
+    public synchronized void dropTable(String databaseName, String tableName) {
+        // Dropping table with partition actions requires cleaning up staging data, which is not implemented yet.
+        checkNoPartitionAction(databaseName, tableName);
+        DatabaseTableName databaseTableName = new DatabaseTableName(databaseName, tableName);
+        Action<TableAndMore> oldTableAction = tableActions.get(databaseTableName);
+        if (oldTableAction == null || oldTableAction.getType() == ActionType.ALTER) {
+            tableActions.put(databaseTableName, new Action<>(ActionType.DROP, null));
+            return;
+        }
+        switch (oldTableAction.getType()) {
+            case DROP:
+                throw new RuntimeException("Not found table: " + databaseTableName);
+            case ADD:
+            case ALTER:
+            case INSERT_EXISTING:
+            case MERGE:
+                throw new RuntimeException("Dropping a table added/modified in the same transaction is not supported");
+            case DROP_PRESERVE_DATA:
+                break;
+            default:
+                throw new IllegalStateException("Unknown action type: " + oldTableAction.getType());
+        }
+    }
+
+
+    private void checkNoPartitionAction(String databaseName, String tableName) {
+        Map<List<String>, Action<PartitionAndMore>> partitionActionsForTable =
+                partitionActions.get(new DatabaseTableName(databaseName, tableName));
+        if (partitionActionsForTable != null && !partitionActionsForTable.isEmpty()) {
+            throw new RuntimeException(
+                    "Cannot make schema changes to a table with modified partitions in the same transaction");
+        }
+    }
+
+    public synchronized void addPartition(
+            String databaseName,
+            String tableName,
+            HivePartition partition,
+            String currentLocation,
+            String partitionName,
+            List<String> files,
+            HivePartitionStatistics statistics) {
+        Map<List<String>, Action<PartitionAndMore>> partitionActionsForTable =
+                partitionActions.computeIfAbsent(new DatabaseTableName(databaseName, tableName), k -> new HashMap<>());
+        Action<PartitionAndMore> oldPartitionAction = partitionActionsForTable.get(partition.getPartitionValues());
+        if (oldPartitionAction == null) {
+            partitionActionsForTable.put(
+                    partition.getPartitionValues(),
+                    new Action<>(
+                            ActionType.ADD,
+                            new PartitionAndMore(partition, currentLocation, partitionName, files, statistics))
+            );
+            return;
+        }
+        switch (oldPartitionAction.getType()) {
+            case DROP:
+            case DROP_PRESERVE_DATA:
+                partitionActionsForTable.put(
+                        partition.getPartitionValues(),
+                        new Action<>(
+                                ActionType.ALTER,
+                                new PartitionAndMore(partition, currentLocation, partitionName, files, statistics))
+                );
+                return;
+            case ADD:
+            case ALTER:
+            case INSERT_EXISTING:
+            case MERGE:
+                throw new RuntimeException(
+                    "Partition already exists for table: "
+                        + databaseName + "." + tableName + ", partition values: " + partition.getPartitionValues());
+            default:
+                throw new IllegalStateException("Unknown action type: " + oldPartitionAction.getType());
+        }
+    }
+
+    public synchronized void dropPartition(
+            String databaseName,
+            String tableName,
+            List<String> partitionValues,
+            boolean deleteData) {
+        DatabaseTableName databaseTableName = new DatabaseTableName(databaseName, tableName);
+        Map<List<String>, Action<PartitionAndMore>> partitionActionsForTable =
+                partitionActions.computeIfAbsent(databaseTableName, k -> new HashMap<>());
+        Action<PartitionAndMore> oldPartitionAction = partitionActionsForTable.get(partitionValues);
+        if (oldPartitionAction == null) {
+            if (deleteData) {
+                partitionActionsForTable.put(partitionValues, new Action<>(ActionType.DROP, null));
+            } else {
+                partitionActionsForTable.put(partitionValues, new Action<>(ActionType.DROP_PRESERVE_DATA, null));
+            }
+            return;
+        }
+        switch (oldPartitionAction.getType()) {
+            case DROP:
+            case DROP_PRESERVE_DATA:
+                throw new RuntimeException(
+                        "Not found partition from partition actions for " + databaseTableName
+                                + ", partitions: " + partitionValues);
+            case ADD:
+            case ALTER:
+            case INSERT_EXISTING:
+            case MERGE:
+                throw new RuntimeException(
+                        "Dropping a partition added in the same transaction is not supported: "
+                                + databaseTableName + ", partition values: " + partitionValues);
+            default:
+                throw new IllegalStateException("Unknown action type: " + oldPartitionAction.getType());
+        }
+    }
+
+    class HmsCommitter {
+
+        // update statistics for unPartitioned table or existed partition
+        private final List<UpdateStatisticsTask> updateStatisticsTasks = new ArrayList<>();
+        Executor updateStatisticsExecutor = Executors.newFixedThreadPool(16);
+
+        // add new partition
+        private final AddPartitionsTask addPartitionsTask = new AddPartitionsTask();
+
+        // for file system rename operation
+        // whether to cancel the file system tasks
+        private final AtomicBoolean fileSystemTaskCancelled = new AtomicBoolean(false);
+        // file system tasks that are executed asynchronously, including rename_file, rename_dir
+        private final List<CompletableFuture<?>> asyncFileSystemTaskFutures = new ArrayList<>();
+        // when aborted, we need to delete all files under this path, even the current directory
+        private final Queue<DirectoryCleanUpTask> directoryCleanUpTasksForAbort = new ConcurrentLinkedQueue<>();
+        // when aborted, we need restore directory
+        private final List<RenameDirectoryTask> renameDirectoryTasksForAbort = new ArrayList<>();
+        // when finished, we need clear some directories
+        private final List<String> clearDirsForFinish = new ArrayList<>();
+        Executor fileSystemExecutor = Executors.newFixedThreadPool(16);
+
+        public void cancelUnStartedAsyncFileSystemTask() {
+            fileSystemTaskCancelled.set(true);
+        }
+
+        private void undoUpdateStatisticsTasks() {
+            ImmutableList.Builder<CompletableFuture<?>> undoUpdateFutures = ImmutableList.builder();
+            for (UpdateStatisticsTask task : updateStatisticsTasks) {
+                undoUpdateFutures.add(CompletableFuture.runAsync(() -> {
+                    try {
+                        task.undo(hiveOps);
+                    } catch (Throwable throwable) {
+                        LOG.warn("Failed to rollback: {}", task.getDescription(), throwable);
+                    }
+                }, updateStatisticsExecutor));
+            }
+
+            for (CompletableFuture<?> undoUpdateFuture : undoUpdateFutures.build()) {
+                MoreFutures.getFutureValue(undoUpdateFuture);
+            }
+        }
+
+        private void undoAddPartitionsTask() {
+            if (addPartitionsTask.isEmpty()) {
+                return;
+            }
+
+            HivePartition firstPartition = addPartitionsTask.getPartitions().get(0).getPartition();
+            String dbName = firstPartition.getDbName();
+            String tableName = firstPartition.getTblName();
+            List<List<String>> rollbackFailedPartitions = addPartitionsTask.rollback(hiveOps);
+            if (!rollbackFailedPartitions.isEmpty()) {
+                LOG.warn("Failed to rollback: add_partition for partition values {}.{}.{}",
+                        dbName, tableName, rollbackFailedPartitions);
+            }
+        }
+
+        private void waitForAsyncFileSystemTaskSuppressThrowable() {
+            for (CompletableFuture<?> future : asyncFileSystemTaskFutures) {
+                try {
+                    future.get();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                } catch (Throwable t) {
+                    // ignore
+                }
+            }
+        }
+
+        public void prepareInsertExistingTable(TableAndMore tableAndMore) {
+            Table table = tableAndMore.getTable();
+            String targetPath = table.getSd().getLocation();
+            String writePath = tableAndMore.getCurrentLocation();
+            if (!targetPath.equals(writePath)) {
+                fs.asyncRename(
+                        fileSystemExecutor,
+                        asyncFileSystemTaskFutures,
+                        fileSystemTaskCancelled,
+                        writePath,
+                        targetPath,
+                        tableAndMore.getFileNames());
+            }
+            directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, false));
+            updateStatisticsTasks.add(
+                new UpdateStatisticsTask(
+                        dbName,
+                        tbName,
+                        Optional.empty(),
+                        tableAndMore.getStatisticsUpdate(),
+                        true
+                    ));
+        }
+
+        public void prepareAlterTable(TableAndMore tableAndMore) {
+            Table table = tableAndMore.getTable();
+            String targetPath = table.getSd().getLocation();
+            String writePath = tableAndMore.getCurrentLocation();
+            if (!targetPath.equals(writePath)) {
+                Path path = new Path(targetPath);
+                String oldTablePath = new Path(path.getParent(), "_temp_" + path.getName()).toString();
+                Status status = fs.renameDir(
+                        targetPath,
+                        oldTablePath,
+                        () -> renameDirectoryTasksForAbort.add(new RenameDirectoryTask(oldTablePath, targetPath)));
+                if (!status.ok()) {
+                    throw new RuntimeException(
+                        "Error to rename dir from " + targetPath + " to " + oldTablePath + status.getErrMsg());
+                }
+                clearDirsForFinish.add(oldTablePath);
+
+                status =  fs.renameDir(
+                        writePath,
+                        targetPath,
+                        () -> directoryCleanUpTasksForAbort.add(
+                                new DirectoryCleanUpTask(targetPath, true)));
+                if (!status.ok()) {
+                    throw new RuntimeException(
+                        "Error to rename dir from " + writePath + " to " + targetPath + ":" + status.getErrMsg());
+                }
+            }
+            updateStatisticsTasks.add(
+                new UpdateStatisticsTask(
+                    dbName,
+                    tbName,
+                    Optional.empty(),
+                    tableAndMore.getStatisticsUpdate(),
+                    false
+                ));
+        }
+
+        public void prepareAddPartition(PartitionAndMore partitionAndMore) {
+
+            HivePartition partition = partitionAndMore.getPartition();
+            String targetPath = partition.getPath();
+            String writePath = partitionAndMore.getCurrentLocation();
+
+            if (!targetPath.equals(writePath)) {
+                fs.asyncRenameDir(
+                        fileSystemExecutor,
+                        asyncFileSystemTaskFutures,
+                        fileSystemTaskCancelled,
+                        writePath,
+                        targetPath,
+                        () -> directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true)));
+            }
+
+            StorageDescriptor sd = getTable(dbName, tbName).getSd();
+
+            HivePartition hivePartition = new HivePartition(
+                    dbName,
+                    tbName,
+                    false,
+                    sd.getInputFormat(),
+                    targetPath,
+                    partition.getPartitionValues(),
+                    Maps.newHashMap(),
+                    sd.getOutputFormat(),
+                    sd.getSerdeInfo().getSerializationLib(),
+                    hiveOps.getClient().getSchema(dbName, tbName)
+            );
+
+            HivePartitionWithStatistics partitionWithStats =
+                    new HivePartitionWithStatistics(
+                            partitionAndMore.getPartitionName(),
+                            hivePartition,
+                            partitionAndMore.getStatisticsUpdate());
+            addPartitionsTask.addPartition(partitionWithStats);
+        }
+
+        public void prepareInsertExistPartition(PartitionAndMore partitionAndMore) {
+
+            HivePartition partition = partitionAndMore.getPartition();
+            String targetPath = partition.getPath();
+            String writePath = partitionAndMore.getCurrentLocation();
+            directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, false));
+
+            if (!targetPath.equals(writePath)) {
+                fs.asyncRename(
+                        fileSystemExecutor,
+                        asyncFileSystemTaskFutures,
+                        fileSystemTaskCancelled,
+                        writePath,
+                        targetPath,
+                        partitionAndMore.getFileNames());
+            }
+
+            updateStatisticsTasks.add(
+                new UpdateStatisticsTask(
+                    dbName,
+                    tbName,
+                    Optional.of(partitionAndMore.getPartitionName()),
+                    partitionAndMore.getStatisticsUpdate(),
+                    true));
+        }
+
+        private void runDirectoryClearUpTasksForAbort() {
+            for (DirectoryCleanUpTask cleanUpTask : directoryCleanUpTasksForAbort) {
+                recursiveDeleteItems(cleanUpTask.getPath(), cleanUpTask.isDeleteEmptyDir());
+            }
+        }
+
+        private void runRenameDirTasksForAbort() {
+            Status status;
+            for (RenameDirectoryTask task : renameDirectoryTasksForAbort) {
+                status = fs.exists(task.getRenameFrom());
+                if (status.ok()) {
+                    status = fs.renameDir(task.getRenameFrom(), task.getRenameTo(), () -> {});
+                    if (!status.ok()) {
+                        LOG.warn("Failed to abort rename dir from {} to {}:{}",
+                                task.getRenameFrom(), task.getRenameTo(), status.getErrMsg());
+                    }
+                }
+            }
+        }
+
+        private void runClearPathsForFinish() {
+            Status status;
+            for (String path : clearDirsForFinish) {
+                status = fs.delete(path);
+                if (!status.ok()) {
+                    LOG.warn("Failed to recursively delete path {}:{}", path, status.getErrCode());
+                }
+            }
+        }
+
+        public void prepareAlterPartition(PartitionAndMore partitionAndMore) {
+            HivePartition partition = partitionAndMore.getPartition();
+            String targetPath = partition.getPath();
+            String writePath = partitionAndMore.getCurrentLocation();
+
+            if (!targetPath.equals(writePath)) {
+                Path path = new Path(targetPath);
+                String oldPartitionPath = new Path(path.getParent(), "_temp_" + path.getName()).toString();
+                Status status = fs.renameDir(
+                        targetPath,
+                        oldPartitionPath,
+                        () -> renameDirectoryTasksForAbort.add(new RenameDirectoryTask(oldPartitionPath, targetPath)));
+                if (!status.ok()) {
+                    throw new RuntimeException(
+                        "Error to rename dir "
+                                + "from " + targetPath
+                                + " to " + oldPartitionPath + ":" + status.getErrMsg());
+                }
+                clearDirsForFinish.add(oldPartitionPath);
+
+                status = fs.renameDir(
+                    writePath,
+                    targetPath,
+                    () -> directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true)));
+                if (!status.ok()) {
+                    throw new RuntimeException(
+                        "Error to rename dir from " + writePath + " to " + targetPath + ":" + status.getErrMsg());
+                }
+            }
+
+            updateStatisticsTasks.add(
+                new UpdateStatisticsTask(
+                    dbName,
+                    tbName,
+                    Optional.of(partitionAndMore.getPartitionName()),
+                    partitionAndMore.getStatisticsUpdate(),
+                    false
+                ));
+        }
+
+
+        private void waitForAsyncFileSystemTasks() {
+            for (CompletableFuture<?> future : asyncFileSystemTaskFutures) {
+                MoreFutures.getFutureValue(future, RuntimeException.class);
+            }
+        }
+
+        private void doAddPartitionsTask() {
+            if (!addPartitionsTask.isEmpty()) {
+                addPartitionsTask.run(hiveOps);
+            }
+        }
+
+        private void doUpdateStatisticsTasks() {
+            ImmutableList.Builder<CompletableFuture<?>> updateStatsFutures = ImmutableList.builder();
+            List<String> failedTaskDescriptions = new ArrayList<>();
+            List<Throwable> suppressedExceptions = new ArrayList<>();
+            for (UpdateStatisticsTask task : updateStatisticsTasks) {
+                updateStatsFutures.add(CompletableFuture.runAsync(() -> {
+                    try {
+                        task.run(hiveOps);
+                    } catch (Throwable t) {
+                        synchronized (suppressedExceptions) {
+                            addSuppressedExceptions(
+                                    suppressedExceptions, t, failedTaskDescriptions, task.getDescription());
+                        }
+                    }
+                }, updateStatisticsExecutor));
+            }
+
+            for (CompletableFuture<?> executeUpdateFuture : updateStatsFutures.build()) {
+                MoreFutures.getFutureValue(executeUpdateFuture);
+            }
+            if (!suppressedExceptions.isEmpty()) {
+                StringBuilder message = new StringBuilder();
+                message.append("Failed to execute some updating statistics tasks: ");
+                Joiner.on("; ").appendTo(message, failedTaskDescriptions);
+                RuntimeException exception = new RuntimeException(message.toString());
+                suppressedExceptions.forEach(exception::addSuppressed);
+                throw exception;
+            }
+        }
+
+        public void doCommit() {
+            waitForAsyncFileSystemTasks();
+            doAddPartitionsTask();
+            doUpdateStatisticsTasks();
+        }
+
+        public void rollback() {
+            cancelUnStartedAsyncFileSystemTask();
+            undoUpdateStatisticsTasks();
+            undoAddPartitionsTask();
+            waitForAsyncFileSystemTaskSuppressThrowable();
+            runDirectoryClearUpTasksForAbort();
+            runRenameDirTasksForAbort();
+        }
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
index beeff694ae4..f3556d13a57 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
@@ -36,13 +36,11 @@ import org.apache.doris.datasource.jdbc.client.JdbcClientConfig;
 import org.apache.doris.datasource.operations.ExternalMetadataOps;
 import org.apache.doris.fs.remote.RemoteFileSystem;
 import org.apache.doris.fs.remote.dfs.DFSFileSystem;
-import org.apache.doris.thrift.THivePartitionUpdate;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -73,6 +71,7 @@ public class HiveMetadataOps implements ExternalMetadataOps {
     public HiveMetadataOps(HMSExternalCatalog catalog, HMSCachedClient client) {
         this.catalog = catalog;
         this.client = client;
+        // TODO Currently only supports DFSFileSystem, more types will be supported in the future
         this.fs = new DFSFileSystem(catalog.getProperties());
     }
 
@@ -80,6 +79,10 @@ public class HiveMetadataOps implements ExternalMetadataOps {
         return client;
     }
 
+    public RemoteFileSystem getFs() {
+        return fs;
+    }
+
     public static HMSCachedClient createCachedClient(HiveConf hiveConf, int thriftClientPoolSize,
                                                      JdbcClientConfig jdbcClientConfig) {
         if (hiveConf != null) {
@@ -253,23 +256,6 @@ public class HiveMetadataOps implements ExternalMetadataOps {
         return client.getAllDatabases();
     }
 
-    public void commit(String dbName,
-                       String tableName,
-                       List<THivePartitionUpdate> hivePUs) {
-        Table table = client.getTable(dbName, tableName);
-        HMSCommitter hmsCommitter = new HMSCommitter(this, fs, table);
-        hmsCommitter.commit(hivePUs);
-        try {
-            Env.getCurrentEnv().getCatalogMgr().refreshExternalTable(
-                    dbName,
-                    tableName,
-                    catalog.getName(),
-                    true);
-        } catch (DdlException e) {
-            LOG.warn("Failed to refresh table {}.{} : {}", dbName, tableName, e.getMessage());
-        }
-    }
-
     public void updateTableStatistics(
             String dbName,
             String tableName,
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionStatistics.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionStatistics.java
index 49b14504750..df13e6737b5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionStatistics.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionStatistics.java
@@ -25,7 +25,7 @@ import com.google.common.collect.ImmutableMap;
 import java.util.Map;
 
 public class HivePartitionStatistics {
-    private static final HivePartitionStatistics EMPTY =
+    public static final HivePartitionStatistics EMPTY =
             new HivePartitionStatistics(HiveCommonStatistics.EMPTY, ImmutableMap.of());
 
     private final HiveCommonStatistics commonStatistics;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java
index b7c28b68ff0..e72374aa5f2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java
@@ -18,9 +18,9 @@
 package org.apache.doris.datasource.hive;
 
 public class HivePartitionWithStatistics {
-    private String name;
-    private HivePartition partition;
-    private HivePartitionStatistics statistics;
+    private final String name;
+    private final HivePartition partition;
+    private final HivePartitionStatistics statistics;
 
     public HivePartitionWithStatistics(String name, HivePartition partition, HivePartitionStatistics statistics) {
         this.name = name;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java
index 76976165526..66dfe763e46 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java
@@ -17,13 +17,12 @@
 
 package org.apache.doris.nereids.trees.plans.commands.insert;
 
+import org.apache.doris.catalog.Env;
 import org.apache.doris.common.ErrorCode;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.DebugUtil;
-import org.apache.doris.datasource.ExternalCatalog;
 import org.apache.doris.datasource.hive.HMSExternalTable;
-import org.apache.doris.datasource.hive.HiveMetadataOps;
-import org.apache.doris.datasource.operations.ExternalMetadataOps;
+import org.apache.doris.datasource.hive.HMSTransaction;
 import org.apache.doris.nereids.NereidsPlanner;
 import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.trees.plans.Plan;
@@ -35,14 +34,13 @@ import org.apache.doris.planner.PlanFragment;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.qe.QueryState;
 import org.apache.doris.qe.StmtExecutor;
-import org.apache.doris.thrift.THivePartitionUpdate;
+import org.apache.doris.transaction.TransactionManager;
 import org.apache.doris.transaction.TransactionStatus;
 
 import com.google.common.base.Strings;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import java.util.List;
 import java.util.Optional;
 
 /**
@@ -53,6 +51,8 @@ public class HiveInsertExecutor extends AbstractInsertExecutor {
     private static final long INVALID_TXN_ID = -1L;
     private long txnId = INVALID_TXN_ID;
     private TransactionStatus txnStatus = TransactionStatus.ABORTED;
+    private final TransactionManager transactionManager;
+    private final String catalogName;
 
     /**
      * constructor
@@ -61,6 +61,8 @@ public class HiveInsertExecutor extends AbstractInsertExecutor {
                               String labelName, NereidsPlanner planner,
                               Optional<InsertCommandContext> insertCtx) {
         super(ctx, table, labelName, planner, insertCtx);
+        catalogName = table.getCatalog().getName();
+        transactionManager = table.getCatalog().getTransactionManager();
     }
 
     public long getTxnId() {
@@ -69,7 +71,9 @@ public class HiveInsertExecutor extends AbstractInsertExecutor {
 
     @Override
     public void beginTransaction() {
-        // TODO: use hive txn rather than internal txn
+        txnId = transactionManager.begin();
+        HMSTransaction transaction = (HMSTransaction) transactionManager.getTransaction(txnId);
+        coordinator.setHivePartitionUpdateFunc(transaction::updateHivePartitionUpdates);
     }
 
     @Override
@@ -93,13 +97,18 @@ public class HiveInsertExecutor extends AbstractInsertExecutor {
         if (ctx.getState().getStateType() == QueryState.MysqlStateType.ERR) {
             LOG.warn("errors when abort txn. {}", ctx.getQueryIdentifier());
         } else {
-            // TODO use transaction
-            List<THivePartitionUpdate> ups = coordinator.getHivePartitionUpdates();
-            loadedRows = ups.stream().mapToLong(THivePartitionUpdate::getRowCount).sum();
-            ExternalCatalog catalog = ((HMSExternalTable) table).getCatalog();
-            ExternalMetadataOps metadataOps = catalog.getMetadataOps();
-            ((HiveMetadataOps) metadataOps).commit(((HMSExternalTable) table).getDbName(), table.getName(), ups);
+            HMSTransaction transaction = (HMSTransaction) transactionManager.getTransaction(txnId);
+            loadedRows = transaction.getUpdateCnt();
+            String dbName = ((HMSExternalTable) table).getDbName();
+            String tbName = table.getName();
+            transaction.finishInsertTable(dbName, tbName);
+            transactionManager.commit(txnId);
             txnStatus = TransactionStatus.COMMITTED;
+            Env.getCurrentEnv().getCatalogMgr().refreshExternalTable(
+                    dbName,
+                    tbName,
+                    catalogName,
+                    true);
         }
     }
 
@@ -117,6 +126,7 @@ public class HiveInsertExecutor extends AbstractInsertExecutor {
             }
         }
         ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, t.getMessage());
+        transactionManager.rollback(txnId);
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
index 9ac96c27c87..b00854a84ea 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
@@ -152,6 +152,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Consumer;
 import java.util.stream.Collectors;
 
 public class Coordinator implements CoordInterface {
@@ -235,8 +236,8 @@ public class Coordinator implements CoordInterface {
     private final List<TTabletCommitInfo> commitInfos = Lists.newArrayList();
     private final List<TErrorTabletInfo> errorTabletInfos = Lists.newArrayList();
 
-    // TODO moved to ExternalTransactionManager
-    private final List<THivePartitionUpdate> hivePartitionUpdates = Lists.newArrayList();
+    // Collect all hivePartitionUpdates obtained from be
+    Consumer<List<THivePartitionUpdate>> hivePartitionUpdateFunc;
 
     // Input parameter
     private long jobId = -1; // job which this task belongs to
@@ -503,10 +504,6 @@ public class Coordinator implements CoordInterface {
         return errorTabletInfos;
     }
 
-    public List<THivePartitionUpdate> getHivePartitionUpdates() {
-        return hivePartitionUpdates;
-    }
-
     public Map<String, Integer> getBeToInstancesNum() {
         Map<String, Integer> result = Maps.newTreeMap();
         if (enablePipelineEngine) {
@@ -2456,13 +2453,8 @@ public class Coordinator implements CoordInterface {
         // TODO: more ranges?
     }
 
-    private void updateHivePartitionUpdates(List<THivePartitionUpdate> hivePartitionUpdates) {
-        lock.lock();
-        try {
-            this.hivePartitionUpdates.addAll(hivePartitionUpdates);
-        } finally {
-            lock.unlock();
-        }
+    public void setHivePartitionUpdateFunc(Consumer<List<THivePartitionUpdate>> hivePartitionUpdateFunc) {
+        this.hivePartitionUpdateFunc = hivePartitionUpdateFunc;
     }
 
     // update job progress from BE
@@ -2512,8 +2504,8 @@ public class Coordinator implements CoordInterface {
             if (params.isSetErrorTabletInfos()) {
                 updateErrorTabletInfos(params.getErrorTabletInfos());
             }
-            if (params.isSetHivePartitionUpdates()) {
-                updateHivePartitionUpdates(params.getHivePartitionUpdates());
+            if (params.isSetHivePartitionUpdates() && hivePartitionUpdateFunc != null) {
+                hivePartitionUpdateFunc.accept(params.getHivePartitionUpdates());
             }
 
             Preconditions.checkArgument(params.isSetDetailedReport());
@@ -2577,8 +2569,8 @@ public class Coordinator implements CoordInterface {
                 if (params.isSetErrorTabletInfos()) {
                     updateErrorTabletInfos(params.getErrorTabletInfos());
                 }
-                if (params.isSetHivePartitionUpdates()) {
-                    updateHivePartitionUpdates(params.getHivePartitionUpdates());
+                if (params.isSetHivePartitionUpdates() && hivePartitionUpdateFunc != null) {
+                    hivePartitionUpdateFunc.accept(params.getHivePartitionUpdates());
                 }
                 if (LOG.isDebugEnabled()) {
                     LOG.debug("Query {} instance {} is marked done",
@@ -2649,8 +2641,8 @@ public class Coordinator implements CoordInterface {
                 if (params.isSetErrorTabletInfos()) {
                     updateErrorTabletInfos(params.getErrorTabletInfos());
                 }
-                if (params.isSetHivePartitionUpdates()) {
-                    updateHivePartitionUpdates(params.getHivePartitionUpdates());
+                if (params.isSetHivePartitionUpdates() && hivePartitionUpdateFunc != null) {
+                    hivePartitionUpdateFunc.accept(params.getHivePartitionUpdates());
                 }
                 instancesDoneLatch.markedCountDown(params.getFragmentInstanceId(), -1L);
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/HiveTransactionManager.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/HiveTransactionManager.java
new file mode 100644
index 00000000000..07304fb23ab
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/HiveTransactionManager.java
@@ -0,0 +1,79 @@
+// 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.transaction;
+
+import org.apache.doris.common.UserException;
+import org.apache.doris.datasource.hive.HMSTransaction;
+import org.apache.doris.datasource.hive.HiveMetadataOps;
+import org.apache.doris.persist.EditLog;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class HiveTransactionManager implements TransactionManager {
+
+    private final Map<Long, HMSTransaction> transactions = new ConcurrentHashMap<>();
+    private final TransactionIdGenerator idGenerator = new TransactionIdGenerator();
+    private final HiveMetadataOps ops;
+
+    public HiveTransactionManager(HiveMetadataOps ops) {
+        this.ops = ops;
+    }
+
+    public Long getNextTransactionId() {
+        return idGenerator.getNextTransactionId();
+    }
+
+    @Override
+    public void setEditLog(EditLog editLog) {
+        this.idGenerator.setEditLog(editLog);
+    }
+
+    @Override
+    public long begin() {
+        long id = idGenerator.getNextTransactionId();
+        HMSTransaction hiveTransaction = new HMSTransaction(ops);
+        transactions.put(id, hiveTransaction);
+        return id;
+    }
+
+    @Override
+    public void commit(long id) throws UserException {
+        getTransactionWithException(id).commit();
+        transactions.remove(id);
+    }
+
+    @Override
+    public void rollback(long id) {
+        getTransactionWithException(id).rollback();
+        transactions.remove(id);
+    }
+
+    @Override
+    public HMSTransaction getTransaction(long id) {
+        return getTransactionWithException(id);
+    }
+
+    public HMSTransaction getTransactionWithException(long id) {
+        HMSTransaction hiveTransaction = transactions.get(id);
+        if (hiveTransaction == null) {
+            throw new RuntimeException("Can't find transaction for " + id);
+        }
+        return hiveTransaction;
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/Transaction.java
similarity index 56%
copy from fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java
copy to fe/fe-core/src/main/java/org/apache/doris/transaction/Transaction.java
index b7c28b68ff0..b319fb78983 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/Transaction.java
@@ -15,28 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
-package org.apache.doris.datasource.hive;
+package org.apache.doris.transaction;
 
-public class HivePartitionWithStatistics {
-    private String name;
-    private HivePartition partition;
-    private HivePartitionStatistics statistics;
+import org.apache.doris.common.UserException;
 
-    public HivePartitionWithStatistics(String name, HivePartition partition, HivePartitionStatistics statistics) {
-        this.name = name;
-        this.partition = partition;
-        this.statistics = statistics;
-    }
+public interface Transaction {
 
-    public String getName() {
-        return name;
-    }
+    void commit() throws UserException;
 
-    public HivePartition getPartition() {
-        return partition;
-    }
-
-    public HivePartitionStatistics getStatistics() {
-        return statistics;
-    }
+    void rollback();
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionManager.java
similarity index 56%
copy from fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java
copy to fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionManager.java
index b7c28b68ff0..daacdecf152 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionManager.java
@@ -15,28 +15,20 @@
 // specific language governing permissions and limitations
 // under the License.
 
-package org.apache.doris.datasource.hive;
-
-public class HivePartitionWithStatistics {
-    private String name;
-    private HivePartition partition;
-    private HivePartitionStatistics statistics;
-
-    public HivePartitionWithStatistics(String name, HivePartition partition, HivePartitionStatistics statistics) {
-        this.name = name;
-        this.partition = partition;
-        this.statistics = statistics;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public HivePartition getPartition() {
-        return partition;
-    }
-
-    public HivePartitionStatistics getStatistics() {
-        return statistics;
-    }
+package org.apache.doris.transaction;
+
+import org.apache.doris.common.UserException;
+import org.apache.doris.persist.EditLog;
+
+public interface TransactionManager {
+
+    void setEditLog(EditLog editLog);
+
+    long begin();
+
+    void commit(long id) throws UserException;
+
+    void rollback(long id);
+
+    Transaction getTransaction(long id);
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionManagerFactory.java
similarity index 56%
copy from fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java
copy to fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionManagerFactory.java
index b7c28b68ff0..334258a3f12 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionWithStatistics.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionManagerFactory.java
@@ -15,28 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
-package org.apache.doris.datasource.hive;
+package org.apache.doris.transaction;
 
-public class HivePartitionWithStatistics {
-    private String name;
-    private HivePartition partition;
-    private HivePartitionStatistics statistics;
+import org.apache.doris.datasource.hive.HiveMetadataOps;
 
-    public HivePartitionWithStatistics(String name, HivePartition partition, HivePartitionStatistics statistics) {
-        this.name = name;
-        this.partition = partition;
-        this.statistics = statistics;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public HivePartition getPartition() {
-        return partition;
-    }
+public class TransactionManagerFactory {
 
-    public HivePartitionStatistics getStatistics() {
-        return statistics;
+    public static TransactionManager createHiveTransactionManager(HiveMetadataOps ops) {
+        return new HiveTransactionManager(ops);
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
index 3098d65e952..fc939625ea9 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
@@ -17,13 +17,17 @@
 
 package org.apache.doris.datasource.hive;
 
+import org.apache.doris.backup.Status;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.fs.remote.dfs.DFSFileSystem;
 import org.apache.doris.thrift.THiveLocationParams;
 import org.apache.doris.thrift.THivePartitionUpdate;
 import org.apache.doris.thrift.TUpdateMode;
 
 import com.google.common.collect.Lists;
+import mockit.Mock;
+import mockit.MockUp;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.junit.After;
@@ -41,6 +45,9 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 @Ignore
 public class HmsCommitTest {
@@ -61,6 +68,7 @@ public class HmsCommitTest {
         dbLocation = "file://" + warehousePath.toAbsolutePath() + "/";
         createTestHiveCatalog();
         createTestHiveDatabase();
+        mockFs();
     }
 
     @AfterClass
@@ -90,22 +98,55 @@ public class HmsCommitTest {
         hmsClient.createDatabase(dbMetadata);
     }
 
+    public static void mockFs() {
+
+        new MockUp<DFSFileSystem>(DFSFileSystem.class) {
+            @Mock
+            public void asyncRenameDir(Executor executor,
+                                       List<CompletableFuture<?>> renameFileFutures,
+                                       AtomicBoolean cancelled,
+                                       String origFilePath,
+                                       String destFilePath,
+                                       Runnable runWhenPathNotExist) {
+            }
+
+            @Mock
+            public void asyncRename(Executor executor,
+                                    List<CompletableFuture<?>> renameFileFutures,
+                                    AtomicBoolean cancelled,
+                                    String origFilePath,
+                                    String destFilePath,
+                                    List<String> fileNames) {
+            }
+
+            @Mock
+            public Status renameDir(String origFilePath,
+                                    String destFilePath,
+                                    Runnable runWhenPathNotExist) {
+                return Status.OK;
+            }
+        };
+    }
+
     @Before
     public void before() {
         // create table
         List<Column> columns = new ArrayList<>();
         columns.add(new Column("c1", PrimitiveType.INT, true));
         columns.add(new Column("c2", PrimitiveType.STRING, true));
+        columns.add(new Column("c3", PrimitiveType.STRING, false));
         List<String> partitionKeys = new ArrayList<>();
         partitionKeys.add("c3");
         HiveTableMetadata tableMetadata = new HiveTableMetadata(
                 dbName, tbWithPartition, columns, partitionKeys,
                 new HashMap<>(), fileFormat);
         hmsClient.createTable(tableMetadata, true);
+
         HiveTableMetadata tableMetadata2 = new HiveTableMetadata(
                 dbName, tbWithoutPartition, columns, new ArrayList<>(),
                 new HashMap<>(), fileFormat);
         hmsClient.createTable(tableMetadata2, true);
+
     }
 
     @After
@@ -118,11 +159,7 @@ public class HmsCommitTest {
     public void testNewPartitionForUnPartitionedTable() {
         List<THivePartitionUpdate> pus = new ArrayList<>();
         pus.add(createRandomNew("a"));
-        try {
-            hmsOps.commit(dbName, tbWithoutPartition, pus);
-        } catch (Exception e) {
-            Assert.assertEquals("Not support mode:[NEW] in unPartitioned table", e.getMessage());
-        }
+        Assert.assertThrows(Exception.class, () -> commit(dbName, tbWithoutPartition, pus));
     }
 
     @Test
@@ -131,7 +168,7 @@ public class HmsCommitTest {
         pus.add(createRandomAppend(""));
         pus.add(createRandomAppend(""));
         pus.add(createRandomAppend(""));
-        hmsOps.commit(dbName, tbWithoutPartition, pus);
+        commit(dbName, tbWithoutPartition, pus);
         Table table = hmsClient.getTable(dbName, tbWithoutPartition);
         assertNumRows(3, table);
 
@@ -139,7 +176,7 @@ public class HmsCommitTest {
         pus2.add(createRandomAppend(""));
         pus2.add(createRandomAppend(""));
         pus2.add(createRandomAppend(""));
-        hmsOps.commit(dbName, tbWithoutPartition, pus2);
+        commit(dbName, tbWithoutPartition, pus2);
         table = hmsClient.getTable(dbName, tbWithoutPartition);
         assertNumRows(6, table);
     }
@@ -151,7 +188,7 @@ public class HmsCommitTest {
         pus.add(createRandomOverwrite(""));
         pus.add(createRandomOverwrite(""));
         pus.add(createRandomOverwrite(""));
-        hmsOps.commit(dbName, tbWithoutPartition, pus);
+        commit(dbName, tbWithoutPartition, pus);
         Table table = hmsClient.getTable(dbName, tbWithoutPartition);
         assertNumRows(3, table);
     }
@@ -165,7 +202,7 @@ public class HmsCommitTest {
         pus.add(createRandomNew("b"));
         pus.add(createRandomNew("b"));
         pus.add(createRandomNew("c"));
-        hmsOps.commit(dbName, tbWithPartition, pus);
+        commit(dbName, tbWithPartition, pus);
 
         Partition pa = hmsClient.getPartition(dbName, tbWithPartition, Lists.newArrayList("a"));
         assertNumRows(3, pa);
@@ -186,7 +223,7 @@ public class HmsCommitTest {
         pus.add(createRandomAppend("b"));
         pus.add(createRandomAppend("b"));
         pus.add(createRandomAppend("c"));
-        hmsOps.commit(dbName, tbWithPartition, pus);
+        commit(dbName, tbWithPartition, pus);
 
         Partition pa = hmsClient.getPartition(dbName, tbWithPartition, Lists.newArrayList("a"));
         assertNumRows(6, pa);
@@ -203,7 +240,7 @@ public class HmsCommitTest {
         pus.add(createRandomOverwrite("a"));
         pus.add(createRandomOverwrite("b"));
         pus.add(createRandomOverwrite("c"));
-        hmsOps.commit(dbName, tbWithPartition, pus);
+        commit(dbName, tbWithPartition, pus);
 
         Partition pa = hmsClient.getPartition(dbName, tbWithPartition, Lists.newArrayList("a"));
         assertNumRows(1, pa);
@@ -221,14 +258,14 @@ public class HmsCommitTest {
             pus.add(createRandomNew("" + i));
         }
 
-        hmsOps.commit(dbName, tbWithPartition, pus);
+        commit(dbName, tbWithPartition, pus);
         for (int i = 0; i < nums; i++) {
             Partition p = hmsClient.getPartition(dbName, tbWithPartition, Lists.newArrayList("" + i));
             assertNumRows(1, p);
         }
 
         try {
-            hmsOps.commit(dbName, tbWithPartition, pus);
+            commit(dbName, tbWithPartition, pus);
         } catch (Exception e) {
             Assert.assertTrue(e.getMessage().contains("failed to add partitions"));
         }
@@ -277,4 +314,13 @@ public class HmsCommitTest {
     public THivePartitionUpdate createRandomOverwrite(String partition) {
         return genOnePartitionUpdate("c3=" + partition, TUpdateMode.OVERWRITE);
     }
+
+    public void commit(String dbName,
+                       String tableName,
+                       List<THivePartitionUpdate> hivePUs) {
+        HMSTransaction hmsTransaction = new HMSTransaction(hmsOps);
+        hmsTransaction.setHivePartitionUpdates(hivePUs);
+        hmsTransaction.finishInsertTable(dbName, tableName);
+        hmsTransaction.commit();
+    }
 }


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


(doris) 09/19: [fix](insert)fix hive external return msgs and exception and pass all columns to BE (#32824)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 716c14675056e2b180138aa6ad913d5ccee66ef6
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Thu Mar 28 19:08:36 2024 +0800

    [fix](insert)fix hive external return msgs and exception and pass all columns to BE (#32824)
    
    [fix](insert)fix hive external return msgs and exception and pass all columns to BE
---
 .../doris/nereids/parser/PartitionTableInfo.java   |   7 +-
 .../trees/plans/commands/info/CreateTableInfo.java |  35 +-
 .../plans/commands/insert/HiveInsertExecutor.java  |  22 +-
 .../commands/insert/InsertIntoTableCommand.java    |   5 +
 .../org/apache/doris/planner/HiveTableSink.java    |  10 +-
 .../java/org/apache/doris/qe/StmtExecutor.java     |   8 +-
 .../datasource/hive/HiveDDLAndDMLPlanTest.java     |  12 +-
 gensrc/thrift/DataSinks.thrift                     |   3 +-
 .../hive/write/test_hive_write_insert.out          |  66 ++
 .../hive/write/test_hive_write_partitions.out      |  50 ++
 .../pipeline/external/conf/regression-conf.groovy  |   1 +
 .../pipeline/p0/conf/regression-conf.groovy        |   1 +
 .../hive/write/test_hive_write_insert.groovy       | 786 ++++++++++-----------
 .../hive/write/test_hive_write_partitions.groovy   |   3 +-
 14 files changed, 573 insertions(+), 436 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java
index 3256effdc3f..fcf35b739c4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java
@@ -119,7 +119,8 @@ public class PartitionTableInfo {
             && partitionDefs.stream().allMatch(p -> p instanceof InPartition);
     }
 
-    private void validatePartitionColumn(ColumnDefinition column, ConnectContext ctx, boolean isEnableMergeOnWrite) {
+    private void validatePartitionColumn(ColumnDefinition column, ConnectContext ctx,
+                                         boolean isEnableMergeOnWrite, boolean isExternal) {
         if (!column.isKey()
                 && (!column.getAggType().equals(AggregateType.NONE) || isEnableMergeOnWrite)) {
             throw new AnalysisException("The partition column could not be aggregated column");
@@ -127,7 +128,7 @@ public class PartitionTableInfo {
         if (column.getType().isFloatLikeType()) {
             throw new AnalysisException("Floating point type column can not be partition column");
         }
-        if (column.getType().isStringType()) {
+        if (column.getType().isStringType() && !isExternal) {
             throw new AnalysisException("String Type should not be used in partition column["
                 + column.getName() + "].");
         }
@@ -173,7 +174,7 @@ public class PartitionTableInfo {
                     throw new AnalysisException(
                             String.format("partition key %s is not exists", p));
                 }
-                validatePartitionColumn(columnMap.get(p), ctx, isEnableMergeOnWrite);
+                validatePartitionColumn(columnMap.get(p), ctx, isEnableMergeOnWrite, isExternal);
             });
 
             Set<String> partitionColumnSets = Sets.newHashSet();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
index 07c55ca9af1..d2b6a1fd31d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
@@ -251,6 +251,18 @@ public class CreateTableInfo {
                         + "please use `DECIMALV3`.");
             }
         }
+        // check duplicated columns
+        Map<String, ColumnDefinition> columnMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+        columns.forEach(c -> {
+            if (columnMap.put(c.getName(), c) != null) {
+                try {
+                    ErrorReport.reportAnalysisException(ErrorCode.ERR_DUP_FIELDNAME,
+                            c.getName());
+                } catch (Exception e) {
+                    throw new AnalysisException(e.getMessage(), e.getCause());
+                }
+            }
+        });
 
         if (engineName.equalsIgnoreCase("olap")) {
             properties = PropertyAnalyzer.rewriteReplicaAllocationProperties(ctlName, dbName,
@@ -413,19 +425,6 @@ public class CreateTableInfo {
                 }
             }
 
-            // validate partitions
-            Map<String, ColumnDefinition> columnMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
-            columns.forEach(c -> {
-                if (columnMap.put(c.getName(), c) != null) {
-                    try {
-                        ErrorReport.reportAnalysisException(ErrorCode.ERR_DUP_FIELDNAME,
-                                c.getName());
-                    } catch (Exception e) {
-                        throw new AnalysisException(e.getMessage(), e.getCause());
-                    }
-                }
-            });
-
             // validate partition
             partitionTableInfo.extractPartitionColumns();
             partitionTableInfo.validatePartitionInfo(columnMap, properties, ctx, isEnableMergeOnWrite, isExternal);
@@ -467,9 +466,17 @@ public class CreateTableInfo {
                     "Iceberg doesn't support 'DISTRIBUTE BY', "
                         + "and you can use 'bucket(num, column)' in 'PARTITIONED BY'.");
             }
-
             for (ColumnDefinition columnDef : columns) {
+                if (!columnDef.isNullable()
+                        && engineName.equalsIgnoreCase("hive")) {
+                    throw new AnalysisException(engineName + " catalog doesn't support column with 'NOT NULL'.");
+                }
                 columnDef.setIsKey(true);
+                columnDef.setAggType(AggregateType.NONE);
+            }
+            // TODO: support iceberg partition check
+            if (engineName.equalsIgnoreCase("hive")) {
+                partitionTableInfo.validatePartitionInfo(columnMap, properties, ctx, false, true);
             }
         }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java
index b054a8b5878..76976165526 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java
@@ -95,6 +95,7 @@ public class HiveInsertExecutor extends AbstractInsertExecutor {
         } else {
             // TODO use transaction
             List<THivePartitionUpdate> ups = coordinator.getHivePartitionUpdates();
+            loadedRows = ups.stream().mapToLong(THivePartitionUpdate::getRowCount).sum();
             ExternalCatalog catalog = ((HMSExternalTable) table).getCatalog();
             ExternalMetadataOps metadataOps = catalog.getMetadataOps();
             ((HiveMetadataOps) metadataOps).commit(((HMSExternalTable) table).getDbName(), table.getName(), ups);
@@ -108,18 +109,33 @@ public class HiveInsertExecutor extends AbstractInsertExecutor {
         String queryId = DebugUtil.printId(ctx.queryId());
         // if any throwable being thrown during insert operation, first we should abort this txn
         LOG.warn("insert [{}] with query id {} failed", labelName, queryId, t);
+        StringBuilder sb = new StringBuilder(t.getMessage());
         if (txnId != INVALID_TXN_ID) {
             LOG.warn("insert [{}] with query id {} abort txn {} failed", labelName, queryId, txnId);
-            StringBuilder sb = new StringBuilder(t.getMessage());
             if (!Strings.isNullOrEmpty(coordinator.getTrackingUrl())) {
                 sb.append(". url: ").append(coordinator.getTrackingUrl());
             }
-            ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, sb.toString());
         }
+        ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, t.getMessage());
     }
 
     @Override
     protected void afterExec(StmtExecutor executor) {
-        // TODO: set THivePartitionUpdate
+        StringBuilder sb = new StringBuilder();
+        sb.append("{");
+        sb.append("'status':'")
+                .append(ctx.isTxnModel() ? TransactionStatus.PREPARE.name() : txnStatus.name());
+        // sb.append("', 'txnId':'").append(txnId).append("'");
+        if (!Strings.isNullOrEmpty(errMsg)) {
+            sb.append(", 'err':'").append(errMsg).append("'");
+        }
+        sb.append("}");
+        ctx.getState().setOk(loadedRows, 0, sb.toString());
+        // set insert result in connection context,
+        // so that user can use `show insert result` to get info of the last insert operation.
+        ctx.setOrUpdateInsertResult(txnId, labelName, database.getFullName(), table.getName(),
+                txnStatus, loadedRows, 0);
+        // update it, so that user can get loaded rows in fe.audit.log
+        ctx.updateReturnRows((int) loadedRows);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java
index 8bf454f0980..3cfd40cabb6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java
@@ -27,6 +27,7 @@ import org.apache.doris.datasource.hive.HMSExternalTable;
 import org.apache.doris.load.loadv2.LoadStatistic;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.nereids.NereidsPlanner;
+import org.apache.doris.nereids.analyzer.UnboundTableSink;
 import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.glue.LogicalPlanAdapter;
 import org.apache.doris.nereids.trees.plans.Explainable;
@@ -197,6 +198,10 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync,
         insertExecutor.executeSingleInsert(executor, jobId);
     }
 
+    public boolean isExternalTableSink() {
+        return !(logicalQuery instanceof UnboundTableSink);
+    }
+
     @Override
     public Plan getExplainPlan(ConnectContext ctx) {
         return InsertUtils.getPlanForExplain(ctx, this.logicalQuery);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java
index c2debe3a4af..a4fa1d11cb0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java
@@ -87,7 +87,6 @@ public class HiveTableSink extends DataSink {
 
     /**
      * check sink params and generate thrift data sink to BE
-     * @param insertCols target table columns
      * @param insertCtx insert info context
      * @throws AnalysisException if source file format cannot be read
      */
@@ -97,22 +96,19 @@ public class HiveTableSink extends DataSink {
         tSink.setDbName(targetTable.getDbName());
         tSink.setTableName(targetTable.getName());
         Set<String> partNames = new HashSet<>(targetTable.getPartitionColumnNames());
-        Set<String> colNames = targetTable.getColumns()
-                .stream().map(Column::getName)
-                .collect(Collectors.toSet());
+        List<Column> allColumns = targetTable.getColumns();
+        Set<String> colNames = allColumns.stream().map(Column::getName).collect(Collectors.toSet());
         colNames.removeAll(partNames);
         List<THiveColumn> targetColumns = new ArrayList<>();
-        for (Column col : insertCols) {
+        for (Column col : allColumns) {
             if (partNames.contains(col.getName())) {
                 THiveColumn tHiveColumn = new THiveColumn();
                 tHiveColumn.setName(col.getName());
-                tHiveColumn.setDataType(col.getType().toThrift());
                 tHiveColumn.setColumnType(THiveColumnType.PARTITION_KEY);
                 targetColumns.add(tHiveColumn);
             } else if (colNames.contains(col.getName())) {
                 THiveColumn tHiveColumn = new THiveColumn();
                 tHiveColumn.setName(col.getName());
-                tHiveColumn.setDataType(col.getType().toThrift());
                 tHiveColumn.setColumnType(THiveColumnType.REGULAR);
                 targetColumns.add(tHiveColumn);
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
index a52ed1c260e..b07447a0e91 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
@@ -515,12 +515,18 @@ public class StmtExecutor {
                     //  2. insert into command because some nereids cases fail (including case1)
                     //  Skip force fallback for:
                     //  1. Transaction insert because nereids support `insert into select` while legacy does not
+                    //  2. Nereids support insert into external table while legacy does not
                     boolean isInsertCommand = parsedStmt != null
                             && parsedStmt instanceof LogicalPlanAdapter
                             && ((LogicalPlanAdapter) parsedStmt).getLogicalPlan() instanceof InsertIntoTableCommand;
                     /*boolean isGroupCommit = (Config.wait_internal_group_commit_finish
                             || context.sessionVariable.isEnableInsertGroupCommit()) && isInsertCommand;*/
-                    boolean forceFallback = isInsertCommand && !context.isTxnModel();
+                    boolean isExternalTableInsert = false;
+                    if (isInsertCommand) {
+                        isExternalTableInsert = ((InsertIntoTableCommand) ((LogicalPlanAdapter) parsedStmt)
+                                .getLogicalPlan()).isExternalTableSink();
+                    }
+                    boolean forceFallback = isInsertCommand && !isExternalTableInsert && !context.isTxnModel();
                     if (e instanceof NereidsException && !context.getSessionVariable().enableFallbackToOriginalPlanner
                             && !forceFallback) {
                         LOG.warn("Analyze failed. {}", context.getQueryIdentifier(), e);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
index 8e2d436e219..1e1fb65a44c 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
@@ -80,8 +80,8 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
         // partitioned table
         String createSourceInterPTable = "CREATE TABLE `part_ctas_olap`(\n"
                 + "  `col1` INT COMMENT 'col1',\n"
-                + "  `pt1` VARCHAR COMMENT 'pt1',\n"
-                + "  `pt2` VARCHAR COMMENT 'pt2'\n"
+                + "  `pt1` VARCHAR(16) COMMENT 'pt1',\n"
+                + "  `pt2` VARCHAR(16) COMMENT 'pt2'\n"
                 + ")  ENGINE=olap\n"
                 + "PARTITION BY LIST (pt1, pt2) ()\n"
                 + "DISTRIBUTED BY HASH (col1) BUCKETS 16\n"
@@ -143,8 +143,8 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
         // partitioned table
         String createSourceExtTable = "CREATE TABLE `part_ctas_src`(\n"
                 + "  `col1` INT COMMENT 'col1',\n"
-                + "  `pt1` STRING COMMENT 'pt1',\n"
-                + "  `pt2` STRING COMMENT 'pt2'\n"
+                + "  `pt1` VARCHAR COMMENT 'pt1',\n"
+                + "  `pt2` VARCHAR COMMENT 'pt2'\n"
                 + ")  ENGINE=hive\n"
                 + "PARTITION BY LIST (pt1, pt2) ()\n"
                 + "PROPERTIES (\n"
@@ -211,7 +211,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 + "  `col2` INT COMMENT 'col2',\n"
                 + "  `col3` BIGINT COMMENT 'col3',\n"
                 + "  `col4` DECIMAL(5,2) COMMENT 'col4',\n"
-                + "  `pt1` STRING COMMENT 'pt1',\n"
+                + "  `pt1` VARCHAR(16) COMMENT 'pt1',\n"
                 + "  `pt2` STRING COMMENT 'pt2',\n"
                 + "  `col5` DATE COMMENT 'col5',\n"
                 + "  `col6` DATETIME COMMENT 'col6'\n"
@@ -257,7 +257,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 + "  `col2` INT COMMENT 'col2',\n"
                 + "  `col3` BIGINT COMMENT 'col3',\n"
                 + "  `col4` DECIMAL(5,2) COMMENT 'col4',\n"
-                + "  `pt1` STRING COMMENT 'pt1',\n"
+                + "  `pt1` VARCHAR(16) COMMENT 'pt1',\n"
                 + "  `pt2` STRING COMMENT 'pt2'\n"
                 + ")  ENGINE=hive\n"
                 + "PARTITION BY LIST (pt2) ()\n"
diff --git a/gensrc/thrift/DataSinks.thrift b/gensrc/thrift/DataSinks.thrift
index e7683c25fd4..93de397b27b 100644
--- a/gensrc/thrift/DataSinks.thrift
+++ b/gensrc/thrift/DataSinks.thrift
@@ -308,8 +308,7 @@ enum THiveColumnType {
 
 struct THiveColumn {
   1: optional string name
-  2: optional Types.TTypeDesc data_type
-  3: optional THiveColumnType column_type
+  2: optional THiveColumnType column_type
 }
 
 struct THivePartition {
diff --git a/regression-test/data/external_table_p0/hive/write/test_hive_write_insert.out b/regression-test/data/external_table_p0/hive/write/test_hive_write_insert.out
index 7b5c0e7d7bc..ad359b8cf6f 100644
--- a/regression-test/data/external_table_p0/hive/write/test_hive_write_insert.out
+++ b/regression-test/data/external_table_p0/hive/write/test_hive_write_insert.out
@@ -22,6 +22,7 @@ false	-7	-15	16	-9223372036854775808	-123.45	-123456.789	123456789	-1234.5678	-1
 \N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
 
 -- !q06 --
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
 
 -- !q01 --
 false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-21	2024-03-21T12:00	2024-03-21T12:00:00.123456	2024-03-21T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{2:20}	{2:200000000000}	{2.2:20.2}	{2.2:20.2}	{0:1}	{2.2:2.2}	{2.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
@@ -53,6 +54,9 @@ true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5
 \N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	\N
 
 -- !q05 --
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{2:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[3.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	\N
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{3:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[8.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	\N
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	\N
 
 -- !q01 --
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
@@ -63,12 +67,25 @@ false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-12345678
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 
+-- !q03 --
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-21	2024-03-21T12:00	2024-03-21T12:00:00.123457	2024-03-21T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{2:20}	{2:200000000000}	{2.2:20.2}	{2.2:20.2}	{0:1}	{2.2:2.2}	{2.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123457	2024-03-22T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+
 -- !q04 --
 false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123457	2024-03-22T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
 false	-7	-15	16	-9223372036854775808	-123.45	-123456.789	123456789	-1234.5678	-123456.789012	-123456789.012345678901	str	binary_value	2024-03-25	2024-03-25T12:00	2024-03-25T12:00:00.123457	2024-03-25T12:00:00.123457	char_value11111	char_value22222	char_value33333	varchar_value11111	varchar_value22222	varchar_value33333	{"key7":"value1"}	{"key7":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{5.3456:2.3456}	{5.34567890:2.34567890}	{2.34567890 [...]
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 
+-- !q05 --
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123457	2024-03-22T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+
 -- !q01 --
 false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-21	2024-03-21T12:00	2024-03-21T12:00:00.123456	2024-03-21T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{2:20}	{2:200000000000}	{2.2:20.2}	{2.2:20.2}	{0:1}	{2.2:2.2}	{2.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
 false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123456	2024-03-22T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
@@ -82,6 +99,22 @@ false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-12345678
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123456	2024-03-20T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123456	2024-03-20T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 
+-- !q03 --
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{2:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[3.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{3:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[8.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240322
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240320
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-21	2024-03-21T12:00	2024-03-21T12:00:00.123456	2024-03-21T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{2:20}	{2:200000000000}	{2.2:20.2}	{2.2:20.2}	{0:1}	{2.2:2.2}	{2.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-21	2024-03-21T12:00	2024-03-21T12:00:00.123456	2024-03-21T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{2:20}	{2:200000000000}	{2.2:20.2}	{2.2:20.2}	{0:1}	{2.2:2.2}	{2.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123456	2024-03-22T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123456	2024-03-22T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123456	2024-03-20T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123456	2024-03-20T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+
+-- !q04 --
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{2:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[3.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{3:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[8.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240322
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240320
+
 -- !q01 --
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 
@@ -105,6 +138,7 @@ false	-7	-15	16	-9223372036854775808	-123.45	-123456.789	123456789	-1234.5678	-1
 \N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
 
 -- !q06 --
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
 
 -- !q01 --
 false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-21	2024-03-21T12:00	2024-03-21T12:00:00.123456	2024-03-21T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{2:20}	{2:200000000000}	{2.2:20.2}	{2.2:20.2}	{0:1}	{2.2:2.2}	{2.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
@@ -136,6 +170,9 @@ true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5
 \N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	\N
 
 -- !q05 --
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{2:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[3.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	\N
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{3:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[8.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	\N
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	\N
 
 -- !q01 --
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
@@ -146,12 +183,25 @@ false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-12345678
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 
+-- !q03 --
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-21	2024-03-21T12:00	2024-03-21T12:00:00.123457	2024-03-21T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{2:20}	{2:200000000000}	{2.2:20.2}	{2.2:20.2}	{0:1}	{2.2:2.2}	{2.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123457	2024-03-22T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+
 -- !q04 --
 false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123457	2024-03-22T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
 false	-7	-15	16	-9223372036854775808	-123.45	-123456.789	123456789	-1234.5678	-123456.789012	-123456789.012345678901	str	binary_value	2024-03-25	2024-03-25T12:00	2024-03-25T12:00:00.123457	2024-03-25T12:00:00.123457	char_value11111	char_value22222	char_value33333	varchar_value11111	varchar_value22222	varchar_value33333	{"key7":"value1"}	{"key7":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{5.3456:2.3456}	{5.34567890:2.34567890}	{2.34567890 [...]
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 
+-- !q05 --
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123457	2024-03-22T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123457	2024-03-20T12:00:00.123457	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+
 -- !q01 --
 false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-21	2024-03-21T12:00	2024-03-21T12:00:00.123456	2024-03-21T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{2:20}	{2:200000000000}	{2.2:20.2}	{2.2:20.2}	{0:1}	{2.2:2.2}	{2.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
 false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123456	2024-03-22T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
@@ -165,3 +215,19 @@ false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-12345678
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123456	2024-03-20T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123456	2024-03-20T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
 
+-- !q03 --
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{2:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[3.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{3:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[8.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240322
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240320
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-21	2024-03-21T12:00	2024-03-21T12:00:00.123456	2024-03-21T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{2:20}	{2:200000000000}	{2.2:20.2}	{2.2:20.2}	{0:1}	{2.2:2.2}	{2.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-21	2024-03-21T12:00	2024-03-21T12:00:00.123456	2024-03-21T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{2:20}	{2:200000000000}	{2.2:20.2}	{2.2:20.2}	{0:1}	{2.2:2.2}	{2.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123456	2024-03-22T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123456	2024-03-22T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123456	2024-03-20T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5678	123456.789012	123456789.012345678901	string_value	binary_value	2024-03-20	2024-03-20T12:00	2024-03-20T12:00:00.123456	2024-03-20T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"a":"b"}	{1:10}	{1:100000000000}	{1.1:10.1}	{1.1:10.1}	{1:0}	{1.1:1.1}	{1.23:1.23}	{1.2345:1.2345}	{1.23456789:1.23456789}	{1.23456789:1.23456789 [...]
+
+-- !q04 --
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{2:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[3.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240321
+\N	\N	\N	\N	\N	-123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{3:20}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[8.4567, 4.5678]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240322
+\N	\N	\N	\N	\N	123.45	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	{1:10}	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[1.2345, 2.3456]	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N	[null, "value1", "value2"]	\N	\N	\N	20240320
+
diff --git a/regression-test/data/external_table_p0/hive/write/test_hive_write_partitions.out b/regression-test/data/external_table_p0/hive/write/test_hive_write_partitions.out
index 2c0202874ce..463c6ef68a0 100644
--- a/regression-test/data/external_table_p0/hive/write/test_hive_write_partitions.out
+++ b/regression-test/data/external_table_p0/hive/write/test_hive_write_partitions.out
@@ -63,6 +63,31 @@ true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5
 2	-123456.789012	string_value	62 69 6e 61 72 79 5f 76 61 6c 75 65	2024-03-21	2024-03-21T12:00	char_value1                                       	varchar_value1
 7	123456.789012	string_value	62 69 6e 61 72 79 5f 76 61 6c 75 65	2024-03-20	2024-03-20T12:00	char_value1                                       	varchar_value1
 
+-- !q01 --
+1	\N	-128	\N	-2147483648	\N	-123.45	\N
+
+-- !q02 --
+1	\N	-128	\N	-2147483648	\N	-123.45	\N
+2	\N	-128	\N	-2147483648	\N	-123.45	\N
+
+-- !q03 --
+1	\N	-128	\N	-2147483648	\N	-123.45	\N
+2	\N	-128	\N	-2147483648	\N	-123.45	\N
+3	\N	127	\N	2147483647	\N	123.45	\N
+
+-- !q04 --
+1	\N	-128	\N	-2147483648	\N	-123.45	\N
+2	\N	-128	\N	-2147483648	\N	-123.45	\N
+3	\N	127	\N	2147483647	\N	123.45	\N
+3	\N	127	\N	2147483647	\N	123.45	\N
+
+-- !q05 --
+1	\N	-128	\N	-2147483648	\N	-123.45	\N
+2	\N	-128	\N	-2147483648	\N	-123.45	\N
+3	\N	127	\N	2147483647	\N	123.45	\N
+3	\N	127	\N	2147483647	\N	123.45	\N
+7	true	127	32767	2147483647	9223372036854775807	123.45	123456.789
+
 -- !q01 --
 false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-21	2024-03-21T12:00	2024-03-21T12:00:00.123456	2024-03-21T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{2:20}	{2:200000000000}	{2.2:20.2}	{2.2:20.2}	{0:1}	{2.2:2.2}	{2.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
 false	-128	-32768	-2147483648	-9223372036854775808	-123.45	-123456.789	-123456789	-1234.5678	-123456.789012	-123456789.012345678901	string_value	binary_value	2024-03-22	2024-03-22T12:00	2024-03-22T12:00:00.123456	2024-03-22T12:00:00.123456	char_value1	char_value2	char_value3	varchar_value1	varchar_value2	varchar_value3	{"key1":"value1"}	{"key1":"value1"}	{"x":"y"}	{3:20}	{3:200000000000}	{3.2:20.2}	{3.2:20.2}	{0:1}	{3.2:2.2}	{3.34:2.34}	{2.3456:2.3456}	{2.34567890:2.34567890}	{2.34567890 [...]
@@ -127,3 +152,28 @@ true	127	32767	2147483647	9223372036854775807	123.45	123456.789	123456789	1234.5
 2	-123456.789012	string_value	62 69 6e 61 72 79 5f 76 61 6c 75 65	2024-03-21	2024-03-21T12:00	char_value1                                       	varchar_value1
 7	123456.789012	string_value	62 69 6e 61 72 79 5f 76 61 6c 75 65	2024-03-20	2024-03-20T12:00	char_value1                                       	varchar_value1
 
+-- !q01 --
+1	\N	-128	\N	-2147483648	\N	-123.45	\N
+
+-- !q02 --
+1	\N	-128	\N	-2147483648	\N	-123.45	\N
+2	\N	-128	\N	-2147483648	\N	-123.45	\N
+
+-- !q03 --
+1	\N	-128	\N	-2147483648	\N	-123.45	\N
+2	\N	-128	\N	-2147483648	\N	-123.45	\N
+3	\N	127	\N	2147483647	\N	123.45	\N
+
+-- !q04 --
+1	\N	-128	\N	-2147483648	\N	-123.45	\N
+2	\N	-128	\N	-2147483648	\N	-123.45	\N
+3	\N	127	\N	2147483647	\N	123.45	\N
+3	\N	127	\N	2147483647	\N	123.45	\N
+
+-- !q05 --
+1	\N	-128	\N	-2147483648	\N	-123.45	\N
+2	\N	-128	\N	-2147483648	\N	-123.45	\N
+3	\N	127	\N	2147483647	\N	123.45	\N
+3	\N	127	\N	2147483647	\N	123.45	\N
+7	true	127	32767	2147483647	9223372036854775807	123.45	123456.789
+
diff --git a/regression-test/pipeline/external/conf/regression-conf.groovy b/regression-test/pipeline/external/conf/regression-conf.groovy
index aa85ad46138..8b7012b292c 100644
--- a/regression-test/pipeline/external/conf/regression-conf.groovy
+++ b/regression-test/pipeline/external/conf/regression-conf.groovy
@@ -61,6 +61,7 @@ excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as th
     "test_cast_string_to_array," +
     "test_refresh_mtmv," +
     "test_spark_load," +
+    "test_broker_load_func," +
     "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line
 
 // this directories will not be executed
diff --git a/regression-test/pipeline/p0/conf/regression-conf.groovy b/regression-test/pipeline/p0/conf/regression-conf.groovy
index e9e12f44eb9..c3b9eb9fcc5 100644
--- a/regression-test/pipeline/p0/conf/regression-conf.groovy
+++ b/regression-test/pipeline/p0/conf/regression-conf.groovy
@@ -69,6 +69,7 @@ excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as th
     "test_profile," +
     "test_refresh_mtmv," +
     "test_spark_load," +
+    "test_broker_load_func," +
     "test_stream_stub_fault_injection," +
     "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line
 
diff --git a/regression-test/suites/external_table_p0/hive/write/test_hive_write_insert.groovy b/regression-test/suites/external_table_p0/hive/write/test_hive_write_insert.groovy
index e5f57d7f78d..401b1cea413 100644
--- a/regression-test/suites/external_table_p0/hive/write/test_hive_write_insert.groovy
+++ b/regression-test/suites/external_table_p0/hive/write/test_hive_write_insert.groovy
@@ -26,67 +26,67 @@ suite("test_hive_write_insert", "p0,external,hive,external_docker,external_docke
         sql """
         INSERT INTO all_types_${format_compression}
         VALUES (
-          CAST(1 AS BOOLEAN), -- boolean_col
-          CAST(127 AS TINYINT), -- tinyint_col
-          CAST(32767 AS SMALLINT), -- smallint_col
-          CAST(2147483647 AS INT), -- int_col
-          CAST(9223372036854775807 AS BIGINT), -- bigint_col
-          CAST(123.45 AS FLOAT), -- float_col
-          CAST(123456.789 AS DOUBLE), -- double_col
+          1, -- boolean_col
+          127, -- tinyint_col
+          32767, -- smallint_col
+          2147483647, -- int_col
+          9223372036854775807, -- bigint_col
+          123.45, -- float_col
+          123456.789, -- double_col
           CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1
           CAST(1234.5678 AS DECIMAL(8,4)), -- decimal_col2
           CAST(123456.789012 AS DECIMAL(18,6)), -- decimal_col3
           CAST(123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4
-          CAST('string_value' AS STRING), -- string_col
+          'string_value', -- string_col
           'binary_value', -- binary_col
           '2024-03-20', -- date_col
           '2024-03-20 12:00:00', -- timestamp_col1
           '2024-03-20 12:00:00.123456789', -- timestamp_col2
           '2024-03-20 12:00:00.123456789', -- timestamp_col3
-          CAST('char_value1' AS CHAR(50)), -- char_col1
-          CAST('char_value2' AS CHAR(100)), -- char_col2
-          CAST('char_value3' AS CHAR(255)), -- char_col3
-          CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1
-          CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2
-          CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3
-          MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string
-          MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar
-          MAP(CAST('a' AS CHAR(10)), CAST('b' AS CHAR(10))), -- t_map_char
-          MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int
-          MAP(CAST(1 AS BIGINT), CAST(100000000000 AS BIGINT)), -- t_map_bigint
+          'char_value1', -- char_col1
+          'char_value2', -- char_col2
+          'char_value3', -- char_col3
+          'varchar_value1', -- varchar_col1
+          'varchar_value2', -- varchar_col2
+          'varchar_value3', -- varchar_col3
+          MAP('key1', 'value1'), -- t_map_string
+          MAP('key1', 'value1'), -- t_map_varchar
+          MAP('a', 'b'), -- t_map_char
+          MAP(1, 10), -- t_map_int
+          MAP(1, 100000000000), -- t_map_bigint
           MAP(CAST(1.1 AS FLOAT), CAST(10.1 AS FLOAT)), -- t_map_float
           MAP(CAST(1.1 AS DOUBLE), CAST(10.1 AS DOUBLE)), -- t_map_double
-          MAP(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_map_boolean
+          MAP(TRUE, FALSE), -- t_map_boolean
           MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2
           MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4
           MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8
           MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17
           MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18
           MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38
-          ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string
-          ARRAY(CAST(1 AS INT), CAST(2 AS INT), CAST(3 AS INT)), -- t_array_int
-          ARRAY(CAST(100000000000 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_array_bigint
+          ARRAY('string1', 'string2'), -- t_array_string
+          ARRAY(1, 2, 3), -- t_array_int
+          ARRAY(100000000000, 200000000000), -- t_array_bigint
           ARRAY(CAST(1.1 AS FLOAT), CAST(2.2 AS FLOAT)), -- t_array_float
           ARRAY(CAST(1.123456789 AS DOUBLE), CAST(2.123456789 AS DOUBLE)), -- t_array_double
-          ARRAY(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_array_boolean
-          ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar
-          ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char
+          ARRAY(TRUE, FALSE), -- t_array_boolean
+          ARRAY('varchar1', 'varchar2'), -- t_array_varchar
+          ARRAY('char1', 'char2'), -- t_array_char
           ARRAY(CAST(1.1 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_array_decimal_precision_2
           ARRAY(CAST(1.23 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_array_decimal_precision_4
           ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
           ARRAY(CAST(1.23456789 AS DECIMAL(17,8)), CAST(2.34567891 AS DECIMAL(17,8))), -- t_array_decimal_precision_17
-          ARRAY(CAST(1.23456789 AS DECIMAL(18,8)), CAST(2.34567891 AS DECIMAL(18,8))), -- t_array_decimal_precision_18
-          ARRAY(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_array_decimal_precision_38
-          NAMED_STRUCT('s_bigint', CAST(1234567890 AS BIGINT)), -- t_struct_bigint
-          MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(123 AS INT)))), -- t_complex
-          NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested
-          NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null
-          NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls
-          NAMED_STRUCT('struct_field1', CAST(123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls
+          ARRAY(1.23456789, 2.34567891), -- t_array_decimal_precision_18
+          ARRAY(1.234567890123456789, 2.345678901234567890), -- t_array_decimal_precision_38
+          NAMED_STRUCT('s_bigint', 1234567890), -- t_struct_bigint
+          MAP('key', ARRAY(NAMED_STRUCT('s_int', 123))), -- t_complex
+          NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested
+          NAMED_STRUCT('struct_field_null', null, 'struct_field_null2', null), -- t_struct_null
+          NAMED_STRUCT('struct_non_nulls_after_nulls1', 123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls
+          NAMED_STRUCT('struct_field1', 123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', 123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls
           MAP('null_key', null), -- t_map_null_value
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-          ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between
-          ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between
+          ARRAY('value1', 'value2', null), -- t_array_string_ending_with_nulls
           ARRAY(null, null, null), -- t_array_string_all_nulls
           20240320 -- dt
         );
@@ -97,197 +97,197 @@ suite("test_hive_write_insert", "p0,external,hive,external_docker,external_docke
         sql """
         INSERT INTO all_types_${format_compression}
         VALUES (
-          CAST(1 AS BOOLEAN), -- boolean_col
-          CAST(127 AS TINYINT), -- tinyint_col
-          CAST(32767 AS SMALLINT), -- smallint_col
-          CAST(2147483647 AS INT), -- int_col
-          CAST(9223372036854775807 AS BIGINT), -- bigint_col
+          1, -- boolean_col
+          127, -- tinyint_col
+          32767, -- smallint_col
+          2147483647, -- int_col
+          9223372036854775807, -- bigint_col
           CAST(123.45 AS FLOAT), -- float_col
           CAST(123456.789 AS DOUBLE), -- double_col
           CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1
           CAST(1234.5678 AS DECIMAL(8,4)), -- decimal_col2
           CAST(123456.789012 AS DECIMAL(18,6)), -- decimal_col3
           CAST(123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4
-          CAST('string_value' AS STRING), -- string_col
+          'string_value', -- string_col
           'binary_value', -- binary_col
           '2024-03-20', -- date_col
           '2024-03-20 12:00:00', -- timestamp_col1
           '2024-03-20 12:00:00.123456789', -- timestamp_col2
           '2024-03-20 12:00:00.123456789', -- timestamp_col3
-          CAST('char_value1' AS CHAR(50)), -- char_col1
-          CAST('char_value2' AS CHAR(100)), -- char_col2
-          CAST('char_value3' AS CHAR(255)), -- char_col3
-          CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1
-          CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2
-          CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3
-          MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string
-          MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar
-          MAP(CAST('a' AS CHAR(10)), CAST('b' AS CHAR(10))), -- t_map_char
-          MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int
-          MAP(CAST(1 AS BIGINT), CAST(100000000000 AS BIGINT)), -- t_map_bigint
+          'char_value1', -- char_col1
+          'char_value2', -- char_col2
+          'char_value3', -- char_col3
+          'varchar_value1', -- varchar_col1
+          'varchar_value2', -- varchar_col2
+          'varchar_value3', -- varchar_col3
+          MAP('key1', 'value1'), -- t_map_string
+          MAP('key1', 'value1'), -- t_map_varchar
+          MAP('a', 'b'), -- t_map_char
+          MAP(1, 10), -- t_map_int
+          MAP(1, 100000000000), -- t_map_bigint
           MAP(CAST(1.1 AS FLOAT), CAST(10.1 AS FLOAT)), -- t_map_float
           MAP(CAST(1.1 AS DOUBLE), CAST(10.1 AS DOUBLE)), -- t_map_double
-          MAP(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_map_boolean
+          MAP(true, false), -- t_map_boolean
           MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2
           MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4
           MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8
           MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17
           MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18
           MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38
-          ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string
-          ARRAY(CAST(1 AS INT), CAST(2 AS INT), CAST(3 AS INT)), -- t_array_int
-          ARRAY(CAST(100000000000 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_array_bigint
+          ARRAY('string1', 'string2'), -- t_array_string
+          ARRAY(1, 2, 3), -- t_array_int
+          ARRAY(100000000000, 200000000000), -- t_array_bigint
           ARRAY(CAST(1.1 AS FLOAT), CAST(2.2 AS FLOAT)), -- t_array_float
           ARRAY(CAST(1.123456789 AS DOUBLE), CAST(2.123456789 AS DOUBLE)), -- t_array_double
-          ARRAY(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_array_boolean
-          ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar
-          ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char
+          ARRAY(true, false), -- t_array_boolean
+          ARRAY('varchar1', 'varchar2'), -- t_array_varchar
+          ARRAY('char1', 'char2'), -- t_array_char
           ARRAY(CAST(1.1 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_array_decimal_precision_2
           ARRAY(CAST(1.23 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_array_decimal_precision_4
           ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
           ARRAY(CAST(1.23456789 AS DECIMAL(17,8)), CAST(2.34567891 AS DECIMAL(17,8))), -- t_array_decimal_precision_17
           ARRAY(CAST(1.23456789 AS DECIMAL(18,8)), CAST(2.34567891 AS DECIMAL(18,8))), -- t_array_decimal_precision_18
           ARRAY(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_array_decimal_precision_38
-          NAMED_STRUCT('s_bigint', CAST(1234567890 AS BIGINT)), -- t_struct_bigint
-          MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(123 AS INT)))), -- t_complex
-          NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested
-          NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null
-          NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls
-          NAMED_STRUCT('struct_field1', CAST(123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls
+          NAMED_STRUCT('s_bigint', 1234567890), -- t_struct_bigint
+          MAP('key', ARRAY(NAMED_STRUCT('s_int', 123))), -- t_complex
+          NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested
+          NAMED_STRUCT('struct_field_null', null, 'struct_field_null2', null), -- t_struct_null
+          NAMED_STRUCT('struct_non_nulls_after_nulls1', 123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls
+          NAMED_STRUCT('struct_field1', 123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', 123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls
           MAP('null_key', null), -- t_map_null_value
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-          ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between
-          ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between
+          ARRAY('value1', 'value2', null), -- t_array_string_ending_with_nulls
           ARRAY(null, null, null), -- t_array_string_all_nulls
           20240320 -- dt
        ),
        (
-          CAST(0 AS BOOLEAN), -- boolean_col
-          CAST(-128 AS TINYINT), -- tinyint_col
-          CAST(-32768 AS SMALLINT), -- smallint_col
-          CAST(-2147483648 AS INT), -- int_col
-          CAST(-9223372036854775808 AS BIGINT), -- bigint_col
+          0, -- boolean_col
+          -128, -- tinyint_col
+          -32768, -- smallint_col
+          -2147483648, -- int_col
+          -9223372036854775808, -- bigint_col
           CAST(-123.45 AS FLOAT), -- float_col
           CAST(-123456.789 AS DOUBLE), -- double_col
           CAST(-123456789 AS DECIMAL(9,0)), -- decimal_col1
           CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2
           CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3
           CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4
-          CAST('string_value' AS STRING), -- string_col
+          'string_value', -- string_col
           'binary_value', -- binary_col
           '2024-03-21', -- date_col
           '2024-03-21 12:00:00', -- timestamp_col1
           '2024-03-21 12:00:00.123456789', -- timestamp_col2
           '2024-03-21 12:00:00.123456789', -- timestamp_col3
-          CAST('char_value1' AS CHAR(50)), -- char_col1
-          CAST('char_value2' AS CHAR(100)), -- char_col2
-          CAST('char_value3' AS CHAR(255)), -- char_col3
-          CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1
-          CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2
-          CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3
-          MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string
-          MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar
-          MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char
-          MAP(CAST(2 AS INT), CAST(20 AS INT)), -- t_map_int
-          MAP(CAST(2 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint
+          'char_value1', -- char_col1
+          'char_value2', -- char_col2
+          'char_value3', -- char_col3
+          'varchar_value1', -- varchar_col1
+          'varchar_value2', -- varchar_col2
+          'varchar_value3', -- varchar_col3
+          MAP('key1', 'value1'), -- t_map_string
+          MAP('key1', 'value1'), -- t_map_varchar
+          MAP('x', 'y'), -- t_map_char
+          MAP(2, 20), -- t_map_int
+          MAP(2, 200000000000), -- t_map_bigint
           MAP(CAST(2.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float
           MAP(CAST(2.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double
-          MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean
+          MAP(false, true), -- t_map_boolean
           MAP(CAST(2.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2
           MAP(CAST(2.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4
           MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8
           MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17
           MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18
           MAP(CAST(2.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38
-          ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string
-          ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int
-          ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint
+          ARRAY('string1', 'string2'), -- t_array_string
+          ARRAY(4, 5, 6), -- t_array_int
+          ARRAY(300000000000, 400000000000), -- t_array_bigint
           ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float
           ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double
-          ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean
-          ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar
-          ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char
+          ARRAY(false, true), -- t_array_boolean
+          ARRAY('varchar1', 'varchar2'), -- t_array_varchar
+          ARRAY('char1', 'char2'), -- t_array_char
           ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2
           ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4
           ARRAY(CAST(3.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
           ARRAY(CAST(3.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17
           ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18
           ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38
-          NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint
-          MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex
-          NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested
-          NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null
-          NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls
-          NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls
+          NAMED_STRUCT('s_bigint', -1234567890), -- t_struct_bigint
+          MAP('key', ARRAY(NAMED_STRUCT('s_int', -123))), -- t_complex
+          NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested
+          NAMED_STRUCT('struct_field_null', null, 'struct_field_null2', null), -- t_struct_null
+          NAMED_STRUCT('struct_non_nulls_after_nulls1', -123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls
+          NAMED_STRUCT('struct_field1', -123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', -123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls
           MAP('null_key', null), -- t_map_null_value
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-          ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between
-          ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between
+          ARRAY('value1', 'value2', null), -- t_array_string_ending_with_nulls
           ARRAY(null, null, null), -- t_array_string_all_nulls
           20240321 -- dt
         ),
         (
-          CAST(0 AS BOOLEAN), -- boolean_col
-          CAST(-128 AS TINYINT), -- tinyint_col
-          CAST(-32768 AS SMALLINT), -- smallint_col
-          CAST(-2147483648 AS INT), -- int_col
-          CAST(-9223372036854775808 AS BIGINT), -- bigint_col
+          0, -- boolean_col
+          -128, -- tinyint_col
+          -32768, -- smallint_col
+          -2147483648, -- int_col
+          -9223372036854775808, -- bigint_col
           CAST(-123.45 AS FLOAT), -- float_col
           CAST(-123456.789 AS DOUBLE), -- double_col
           CAST(-123456789 AS DECIMAL(9,0)), -- decimal_col1
           CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2
           CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3
           CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4
-          CAST('string_value' AS STRING), -- string_col
+          'string_value', -- string_col
           'binary_value', -- binary_col
           '2024-03-22', -- date_col
           '2024-03-22 12:00:00', -- timestamp_col1
           '2024-03-22 12:00:00.123456789', -- timestamp_col2
           '2024-03-22 12:00:00.123456789', -- timestamp_col3
-          CAST('char_value1' AS CHAR(50)), -- char_col1
-          CAST('char_value2' AS CHAR(100)), -- char_col2
-          CAST('char_value3' AS CHAR(255)), -- char_col3
-          CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1
-          CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2
-          CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3
-          MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string
-          MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar
-          MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char
-          MAP(CAST(3 AS INT), CAST(20 AS INT)), -- t_map_int
-          MAP(CAST(3 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint
+          'char_value1', -- char_col1
+          'char_value2', -- char_col2
+          'char_value3', -- char_col3
+          'varchar_value1', -- varchar_col1
+          'varchar_value2', -- varchar_col2
+          'varchar_value3', -- varchar_col3
+          MAP('key1', 'value1'), -- t_map_string
+          MAP('key1', 'value1'), -- t_map_varchar
+          MAP('x', 'y'), -- t_map_char
+          MAP(3, 20), -- t_map_int
+          MAP(3, 200000000000), -- t_map_bigint
           MAP(CAST(3.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float
           MAP(CAST(3.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double
-          MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean
+          MAP(false, true), -- t_map_boolean
           MAP(CAST(3.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2
           MAP(CAST(3.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4
           MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8
           MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17
           MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18
           MAP(CAST(3.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38
-          ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string
-          ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int
-          ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint
+          ARRAY('string1', 'string2'), -- t_array_string
+          ARRAY(4, 5, 6), -- t_array_int
+          ARRAY(300000000000, 400000000000), -- t_array_bigint
           ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float
           ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double
-          ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean
-          ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar
-          ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char
+          ARRAY(false, true), -- t_array_boolean
+          ARRAY('varchar1', 'varchar2'), -- t_array_varchar
+          ARRAY('char1', 'char2'), -- t_array_char
           ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2
           ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4
           ARRAY(CAST(8.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
           ARRAY(CAST(3.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17
           ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18
           ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38
-          NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint
-          MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex
-          NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested
-          NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null
-          NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls
-          NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls
+          NAMED_STRUCT('s_bigint', -1234567890), -- t_struct_bigint
+          MAP('key', ARRAY(NAMED_STRUCT('s_int', -123))), -- t_complex
+          NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested
+          NAMED_STRUCT('struct_field_null', null, 'struct_field_null2', null), -- t_struct_null
+          NAMED_STRUCT('struct_non_nulls_after_nulls1', -123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls
+          NAMED_STRUCT('struct_field1', -123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', -123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls
           MAP('null_key', null), -- t_map_null_value
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-          ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between
-          ARRAY(CAST('value11' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between
+          ARRAY('value11', 'value2', null), -- t_array_string_ending_with_nulls
           ARRAY(null, null, null), -- t_array_string_all_nulls
           20240322 -- dt
         );
@@ -299,9 +299,9 @@ suite("test_hive_write_insert", "p0,external,hive,external_docker,external_docke
         INSERT INTO all_types_${format_compression}(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls)
         VALUES (
           CAST(123.45 AS FLOAT), -- float_col
-          MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int
+          MAP(1, 10), -- t_map_int
           ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)) -- t_array_string_starting_with_nulls
+          ARRAY(null, 'value1', 'value2') -- t_array_string_starting_with_nulls
         );
         """
         order_qt_q03 """ select * from all_types_${format_compression};
@@ -310,67 +310,67 @@ suite("test_hive_write_insert", "p0,external,hive,external_docker,external_docke
         sql """
         insert overwrite table all_types_${format_compression}
         VALUES (
-          CAST(0 AS BOOLEAN), -- boolean_col
-          CAST(-7 AS TINYINT), -- tinyint_col
-          CAST(-15 AS SMALLINT), -- smallint_col
-          CAST(16 AS INT), -- int_col
-          CAST(-9223372036854775808 AS BIGINT), -- bigint_col
+          0, -- boolean_col
+          -7, -- tinyint_col
+          -15, -- smallint_col
+          16, -- int_col
+          -9223372036854775808, -- bigint_col
           CAST(-123.45 AS FLOAT), -- float_col
           CAST(-123456.789 AS DOUBLE), -- double_col
           CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1
           CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2
           CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3
           CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4
-          CAST('str' AS STRING), -- string_col
+          'str', -- string_col
           'binary_value', -- binary_col
           '2024-03-25', -- date_col
           '2024-03-25 12:00:00', -- timestamp_col1
           '2024-03-25 12:00:00.123456789', -- timestamp_col2
           '2024-03-25 12:00:00.123456789', -- timestamp_col3
-          CAST('char_value11111' AS CHAR(50)), -- char_col1
-          CAST('char_value22222' AS CHAR(100)), -- char_col2
-          CAST('char_value33333' AS CHAR(255)), -- char_col3
-          CAST('varchar_value11111' AS VARCHAR(50)), -- varchar_col1
-          CAST('varchar_value22222' AS VARCHAR(100)), -- varchar_col2
-          CAST('varchar_value33333' AS VARCHAR(255)), -- varchar_col3
-          MAP(CAST('key7' AS STRING), CAST('value1' AS STRING)), -- t_map_string
-          MAP(CAST('key7' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar
-          MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char
-          MAP(CAST(3 AS INT), CAST(20 AS INT)), -- t_map_int
-          MAP(CAST(3 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint
+          'char_value11111', -- char_col1
+          'char_value22222', -- char_col2
+          'char_value33333', -- char_col3
+          'varchar_value11111', -- varchar_col1
+          'varchar_value22222', -- varchar_col2
+          'varchar_value33333', -- varchar_col3
+          MAP('key7', 'value1'), -- t_map_string
+          MAP('key7', 'value1'), -- t_map_varchar
+          MAP('x', 'y'), -- t_map_char
+          MAP(3, 20), -- t_map_int
+          MAP(3, 200000000000), -- t_map_bigint
           MAP(CAST(3.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float
           MAP(CAST(3.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double
-          MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean
+          MAP(false, true), -- t_map_boolean
           MAP(CAST(3.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2
           MAP(CAST(3.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4
           MAP(CAST(5.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8
           MAP(CAST(5.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17
           MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18
           MAP(CAST(7.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38
-          ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string
-          ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int
-          ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint
+          ARRAY('string1', 'string2'), -- t_array_string
+          ARRAY(4, 5, 6), -- t_array_int
+          ARRAY(300000000000, 400000000000), -- t_array_bigint
           ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float
           ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double
-          ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean
-          ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar
-          ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char
+          ARRAY(false, true), -- t_array_boolean
+          ARRAY('varchar1', 'varchar2'), -- t_array_varchar
+          ARRAY('char1', 'char2'), -- t_array_char
           ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2
           ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4
           ARRAY(CAST(9.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
           ARRAY(CAST(6.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17
           ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18
           ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38
-          NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint
-          MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex
-          NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested
-          NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null
-          NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls
-          NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls
+          NAMED_STRUCT('s_bigint', -1234567890), -- t_struct_bigint
+          MAP('key', ARRAY(NAMED_STRUCT('s_int', -123))), -- t_complex
+          NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested
+          NAMED_STRUCT('struct_field_null', null, 'struct_field_null2', null), -- t_struct_null
+          NAMED_STRUCT('struct_non_nulls_after_nulls1', -123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls
+          NAMED_STRUCT('struct_field1', -123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', -123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls
           MAP('null_key', null), -- t_map_null_value
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-          ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between
-          ARRAY(CAST('value11' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between
+          ARRAY('value11', 'value2', null), -- t_array_string_ending_with_nulls
           ARRAY(null, null, null), -- t_array_string_all_nulls
           20240325 -- dt
         );
@@ -381,10 +381,10 @@ suite("test_hive_write_insert", "p0,external,hive,external_docker,external_docke
         sql """
         INSERT overwrite table all_types_${format_compression}(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt)
         VALUES (
-          CAST(123.45 AS FLOAT), -- float_col
-          MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int
+          123.45, -- float_col
+          MAP(1, 10), -- t_map_int
           ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
           20240321 -- dt
         );
         """
@@ -393,7 +393,6 @@ suite("test_hive_write_insert", "p0,external,hive,external_docker,external_docke
 
         logger.info("hive sql: " + """ truncate table all_types_${format_compression}; """)
         hive_docker """ truncate table all_types_${format_compression}; """
-        sql """refresh catalog ${catalog_name};"""
         order_qt_q06 """ select * from all_types_${format_compression};
         """
     }
@@ -445,7 +444,6 @@ suite("test_hive_write_insert", "p0,external,hive,external_docker,external_docke
 
         logger.info("hive sql: " + """ truncate table all_types_${format_compression}; """)
         hive_docker """ truncate table all_types_${format_compression}; """
-        sql """refresh catalog ${catalog_name};"""
         order_qt_q05 """
         select * from all_types_${format_compression};
         """
@@ -460,376 +458,372 @@ suite("test_hive_write_insert", "p0,external,hive,external_docker,external_docke
         sql """
         INSERT INTO all_types_par_${format_compression}_${catalog_name}_q03
         VALUES (
-          CAST(1 AS BOOLEAN), -- boolean_col
-          CAST(127 AS TINYINT), -- tinyint_col
-          CAST(32767 AS SMALLINT), -- smallint_col
-          CAST(2147483647 AS INT), -- int_col
-          CAST(9223372036854775807 AS BIGINT), -- bigint_col
-          CAST(123.45 AS FLOAT), -- float_col
-          CAST(123456.789 AS DOUBLE), -- double_col
-          CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1
-          CAST(1234.5678 AS DECIMAL(8,4)), -- decimal_col2
+          1, -- boolean_col
+          127, -- tinyint_col
+          32767, -- smallint_col
+          2147483647, -- int_col
+          9223372036854775807, -- bigint_col
+          123.45, -- float_col
+          123456.789, -- double_col
+          123456789, -- decimal_col1
+          1234.5678, -- decimal_col2
           CAST(123456.789012 AS DECIMAL(18,6)), -- decimal_col3
           CAST(123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4
-          CAST('string_value' AS STRING), -- string_col
+          'string_value', -- string_col
           'binary_value', -- binary_col
           '2024-03-20', -- date_col
           '2024-03-20 12:00:00', -- timestamp_col1
           '2024-03-20 12:00:00.123456789', -- timestamp_col2
           '2024-03-20 12:00:00.123456789', -- timestamp_col3
-          CAST('char_value1' AS CHAR(50)), -- char_col1
-          CAST('char_value2' AS CHAR(100)), -- char_col2
-          CAST('char_value3' AS CHAR(255)), -- char_col3
-          CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1
-          CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2
-          CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3
-          MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string
-          MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar
-          MAP(CAST('a' AS CHAR(10)), CAST('b' AS CHAR(10))), -- t_map_char
-          MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int
-          MAP(CAST(1 AS BIGINT), CAST(100000000000 AS BIGINT)), -- t_map_bigint
+          'char_value1', -- char_col1
+          'char_value2', -- char_col2
+          'char_value3', -- char_col3
+          'varchar_value1', -- varchar_col1
+          'varchar_value2', -- varchar_col2
+          'varchar_value3', -- varchar_col3
+          MAP('key1', 'value1'), -- t_map_string
+          MAP('key1', 'value1'), -- t_map_varchar
+          MAP('a', 'b'), -- t_map_char
+          MAP(1, 10), -- t_map_int
+          MAP(1, 100000000000), -- t_map_bigint
           MAP(CAST(1.1 AS FLOAT), CAST(10.1 AS FLOAT)), -- t_map_float
           MAP(CAST(1.1 AS DOUBLE), CAST(10.1 AS DOUBLE)), -- t_map_double
-          MAP(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_map_boolean
+          MAP(true, false), -- t_map_boolean
           MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2
           MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4
           MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8
           MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17
           MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18
           MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38
-          ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string
-          ARRAY(CAST(1 AS INT), CAST(2 AS INT), CAST(3 AS INT)), -- t_array_int
-          ARRAY(CAST(100000000000 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_array_bigint
+          ARRAY('string1', 'string2'), -- t_array_string
+          ARRAY(1, 2, 3), -- t_array_int
+          ARRAY(100000000000, 200000000000), -- t_array_bigint
           ARRAY(CAST(1.1 AS FLOAT), CAST(2.2 AS FLOAT)), -- t_array_float
           ARRAY(CAST(1.123456789 AS DOUBLE), CAST(2.123456789 AS DOUBLE)), -- t_array_double
-          ARRAY(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_array_boolean
-          ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar
-          ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char
+          ARRAY(true, false), -- t_array_boolean
+          ARRAY('varchar1', 'varchar2'), -- t_array_varchar
+          ARRAY('char1', 'char2'), -- t_array_char
           ARRAY(CAST(1.1 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_array_decimal_precision_2
           ARRAY(CAST(1.23 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_array_decimal_precision_4
           ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
           ARRAY(CAST(1.23456789 AS DECIMAL(17,8)), CAST(2.34567891 AS DECIMAL(17,8))), -- t_array_decimal_precision_17
           ARRAY(CAST(1.23456789 AS DECIMAL(18,8)), CAST(2.34567891 AS DECIMAL(18,8))), -- t_array_decimal_precision_18
           ARRAY(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_array_decimal_precision_38
-          NAMED_STRUCT('s_bigint', CAST(1234567890 AS BIGINT)), -- t_struct_bigint
-          MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(123 AS INT)))), -- t_complex
-          NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested
-          NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null
-          NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls
-          NAMED_STRUCT('struct_field1', CAST(123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls
+          NAMED_STRUCT('s_bigint', 1234567890), -- t_struct_bigint
+          MAP('key', ARRAY(NAMED_STRUCT('s_int', 123))), -- t_complex
+          NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested
+          NAMED_STRUCT('struct_field_null', null, 'struct_field_null2', null), -- t_struct_null
+          NAMED_STRUCT('struct_non_nulls_after_nulls1', 123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls
+          NAMED_STRUCT('struct_field1', 123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', 123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls
           MAP('null_key', null), -- t_map_null_value
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-          ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between
-          ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between
+          ARRAY('value1', 'value2', null), -- t_array_string_ending_with_nulls
           ARRAY(null, null, null), -- t_array_string_all_nulls
           20240320 -- dt
         );
         """
         order_qt_q01 """ select * from all_types_par_${format_compression}_${catalog_name}_q03;
         """
-        //sql """refresh catalog ${catalog_name};"""
 
         sql """
-        INSERT INTO all_types_par_${format_compression}_${catalog_name}_q03
+        
+INSERT INTO all_types_par_${format_compression}_${catalog_name}_q03
         VALUES  (
-          CAST(1 AS BOOLEAN), -- boolean_col
-          CAST(127 AS TINYINT), -- tinyint_col
-          CAST(32767 AS SMALLINT), -- smallint_col
-          CAST(2147483647 AS INT), -- int_col
-          CAST(9223372036854775807 AS BIGINT), -- bigint_col
+          1, -- boolean_col
+          127, -- tinyint_col
+          32767, -- smallint_col
+          2147483647, -- int_col
+          9223372036854775807, -- bigint_col
           CAST(123.45 AS FLOAT), -- float_col
           CAST(123456.789 AS DOUBLE), -- double_col
           CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1
           CAST(1234.5678 AS DECIMAL(8,4)), -- decimal_col2
           CAST(123456.789012 AS DECIMAL(18,6)), -- decimal_col3
           CAST(123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4
-          CAST('string_value' AS STRING), -- string_col
+          'string_value', -- string_col
           'binary_value', -- binary_col
           '2024-03-20', -- date_col
           '2024-03-20 12:00:00', -- timestamp_col1
           '2024-03-20 12:00:00.123456789', -- timestamp_col2
           '2024-03-20 12:00:00.123456789', -- timestamp_col3
-          CAST('char_value1' AS CHAR(50)), -- char_col1
-          CAST('char_value2' AS CHAR(100)), -- char_col2
-          CAST('char_value3' AS CHAR(255)), -- char_col3
-          CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1
-          CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2
-          CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3
-          MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string
-          MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar
-          MAP(CAST('a' AS CHAR(10)), CAST('b' AS CHAR(10))), -- t_map_char
-          MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int
-          MAP(CAST(1 AS BIGINT), CAST(100000000000 AS BIGINT)), -- t_map_bigint
+          'char_value1', -- char_col1
+          'char_value2', -- char_col2
+          'char_value3', -- char_col3
+          'varchar_value1', -- varchar_col1
+          'varchar_value2', -- varchar_col2
+          'varchar_value3', -- varchar_col3
+          MAP('key1', 'value1'), -- t_map_string
+          MAP('key1', 'value1'), -- t_map_varchar
+          MAP('a', 'b'), -- t_map_char
+          MAP(1, 10), -- t_map_int
+          MAP(1, 100000000000), -- t_map_bigint
           MAP(CAST(1.1 AS FLOAT), CAST(10.1 AS FLOAT)), -- t_map_float
           MAP(CAST(1.1 AS DOUBLE), CAST(10.1 AS DOUBLE)), -- t_map_double
-          MAP(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_map_boolean
+          MAP(true, false), -- t_map_boolean
           MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2
           MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4
           MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8
           MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17
           MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18
           MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38
-          ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string
-          ARRAY(CAST(1 AS INT), CAST(2 AS INT), CAST(3 AS INT)), -- t_array_int
-          ARRAY(CAST(100000000000 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_array_bigint
+          ARRAY('string1', 'string2'), -- t_array_string
+          ARRAY(1, 2, 3), -- t_array_int
+          ARRAY(100000000000, 200000000000), -- t_array_bigint
           ARRAY(CAST(1.1 AS FLOAT), CAST(2.2 AS FLOAT)), -- t_array_float
           ARRAY(CAST(1.123456789 AS DOUBLE), CAST(2.123456789 AS DOUBLE)), -- t_array_double
-          ARRAY(CAST(true AS BOOLEAN), CAST(false AS BOOLEAN)), -- t_array_boolean
-          ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar
-          ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char
+          ARRAY(true, false), -- t_array_boolean
+          ARRAY('varchar1', 'varchar2'), -- t_array_varchar
+          ARRAY('char1', 'char2'), -- t_array_char
           ARRAY(CAST(1.1 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_array_decimal_precision_2
           ARRAY(CAST(1.23 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_array_decimal_precision_4
           ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
           ARRAY(CAST(1.23456789 AS DECIMAL(17,8)), CAST(2.34567891 AS DECIMAL(17,8))), -- t_array_decimal_precision_17
           ARRAY(CAST(1.23456789 AS DECIMAL(18,8)), CAST(2.34567891 AS DECIMAL(18,8))), -- t_array_decimal_precision_18
           ARRAY(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_array_decimal_precision_38
-          NAMED_STRUCT('s_bigint', CAST(1234567890 AS BIGINT)), -- t_struct_bigint
-          MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(123 AS INT)))), -- t_complex
-          NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested
-          NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null
-          NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls
-          NAMED_STRUCT('struct_field1', CAST(123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls
+          NAMED_STRUCT('s_bigint', 1234567890), -- t_struct_bigint
+          MAP('key', ARRAY(NAMED_STRUCT('s_int', 123))), -- t_complex
+          NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested
+          NAMED_STRUCT('struct_field_null', null, 'struct_field_null2', null), -- t_struct_null
+          NAMED_STRUCT('struct_non_nulls_after_nulls1', 123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls
+          NAMED_STRUCT('struct_field1', 123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', 123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls
           MAP('null_key', null), -- t_map_null_value
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-          ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between
-          ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between
+          ARRAY('value1', 'value2', null), -- t_array_string_ending_with_nulls
           ARRAY(null, null, null), -- t_array_string_all_nulls
           20240320 -- dt
        ),
        (
-          CAST(0 AS BOOLEAN), -- boolean_col
-          CAST(-128 AS TINYINT), -- tinyint_col
-          CAST(-32768 AS SMALLINT), -- smallint_col
-          CAST(-2147483648 AS INT), -- int_col
-          CAST(-9223372036854775808 AS BIGINT), -- bigint_col
+          0, -- boolean_col
+          -128, -- tinyint_col
+          -32768, -- smallint_col
+          -2147483648, -- int_col
+          -9223372036854775808, -- bigint_col
           CAST(-123.45 AS FLOAT), -- float_col
           CAST(-123456.789 AS DOUBLE), -- double_col
           CAST(-123456789 AS DECIMAL(9,0)), -- decimal_col1
           CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2
           CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3
           CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4
-          CAST('string_value' AS STRING), -- string_col
+          'string_value', -- string_col
           'binary_value', -- binary_col
           '2024-03-21', -- date_col
           '2024-03-21 12:00:00', -- timestamp_col1
           '2024-03-21 12:00:00.123456789', -- timestamp_col2
           '2024-03-21 12:00:00.123456789', -- timestamp_col3
-          CAST('char_value1' AS CHAR(50)), -- char_col1
-          CAST('char_value2' AS CHAR(100)), -- char_col2
-          CAST('char_value3' AS CHAR(255)), -- char_col3
-          CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1
-          CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2
-          CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3
-          MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string
-          MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar
-          MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char
-          MAP(CAST(2 AS INT), CAST(20 AS INT)), -- t_map_int
-          MAP(CAST(2 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint
+          'char_value1', -- char_col1
+          'char_value2', -- char_col2
+          'char_value3', -- char_col3
+          'varchar_value1', -- varchar_col1
+          'varchar_value2', -- varchar_col2
+          'varchar_value3', -- varchar_col3
+          MAP('key1', 'value1'), -- t_map_string
+          MAP('key1', 'value1'), -- t_map_varchar
+          MAP('x', 'y'), -- t_map_char
+          MAP(2, 20), -- t_map_int
+          MAP(2, 200000000000), -- t_map_bigint
           MAP(CAST(2.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float
           MAP(CAST(2.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double
-          MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean
+          MAP(false, true), -- t_map_boolean
           MAP(CAST(2.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2
           MAP(CAST(2.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4
           MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8
           MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17
           MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18
           MAP(CAST(2.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38
-          ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string
-          ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int
-          ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint
+          ARRAY('string1', 'string2'), -- t_array_string
+          ARRAY(4, 5, 6), -- t_array_int
+          ARRAY(300000000000, 400000000000), -- t_array_bigint
           ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float
           ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double
-          ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean
-          ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar
-          ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char
+          ARRAY(false, true), -- t_array_boolean
+          ARRAY('varchar1', 'varchar2'), -- t_array_varchar
+          ARRAY('char1', 'char2'), -- t_array_char
           ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2
           ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4
           ARRAY(CAST(3.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
           ARRAY(CAST(3.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17
           ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18
           ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38
-          NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint
-          MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex
-          NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested
-          NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null
-          NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls
-          NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls
+          NAMED_STRUCT('s_bigint', -1234567890), -- t_struct_bigint
+          MAP('key', ARRAY(NAMED_STRUCT('s_int', -123))), -- t_complex
+          NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested
+          NAMED_STRUCT('struct_field_null', null, 'struct_field_null2', null), -- t_struct_null
+          NAMED_STRUCT('struct_non_nulls_after_nulls1', -123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls
+          NAMED_STRUCT('struct_field1', -123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', -123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls
           MAP('null_key', null), -- t_map_null_value
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-          ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between
-          ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between
+          ARRAY('value1', 'value2', null), -- t_array_string_ending_with_nulls
           ARRAY(null, null, null), -- t_array_string_all_nulls
           20240321 -- dt
         ),
         (
-          CAST(0 AS BOOLEAN), -- boolean_col
-          CAST(-128 AS TINYINT), -- tinyint_col
-          CAST(-32768 AS SMALLINT), -- smallint_col
-          CAST(-2147483648 AS INT), -- int_col
-          CAST(-9223372036854775808 AS BIGINT), -- bigint_col
-          CAST(-123.45 AS FLOAT), -- float_col
-          CAST(-123456.789 AS DOUBLE), -- double_col
+          0, -- boolean_col
+          -128, -- tinyint_col
+          -32768, -- smallint_col
+          -2147483648, -- int_col
+          -9223372036854775808, -- bigint_col
+          -123.45, -- float_col
+          -123456.789, -- double_col
           CAST(-123456789 AS DECIMAL(9,0)), -- decimal_col1
           CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2
           CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3
           CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4
-          CAST('string_value' AS STRING), -- string_col
+          'string_value', -- string_col
           'binary_value', -- binary_col
           '2024-03-22', -- date_col
           '2024-03-22 12:00:00', -- timestamp_col1
           '2024-03-22 12:00:00.123456789', -- timestamp_col2
           '2024-03-22 12:00:00.123456789', -- timestamp_col3
-          CAST('char_value1' AS CHAR(50)), -- char_col1
-          CAST('char_value2' AS CHAR(100)), -- char_col2
-          CAST('char_value3' AS CHAR(255)), -- char_col3
-          CAST('varchar_value1' AS VARCHAR(50)), -- varchar_col1
-          CAST('varchar_value2' AS VARCHAR(100)), -- varchar_col2
-          CAST('varchar_value3' AS VARCHAR(255)), -- varchar_col3
-          MAP(CAST('key1' AS STRING), CAST('value1' AS STRING)), -- t_map_string
-          MAP(CAST('key1' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar
-          MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char
-          MAP(CAST(3 AS INT), CAST(20 AS INT)), -- t_map_int
-          MAP(CAST(3 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint
+          'char_value1', -- char_col1
+          'char_value2', -- char_col2
+          'char_value3', -- char_col3
+          'varchar_value1', -- varchar_col1
+          'varchar_value2', -- varchar_col2
+          'varchar_value3', -- varchar_col3
+          MAP('key1', 'value1'), -- t_map_string
+          MAP('key1', 'value1'), -- t_map_varchar
+          MAP('x', 'y'), -- t_map_char
+          MAP(3, 20), -- t_map_int
+          MAP(3, 200000000000), -- t_map_bigint
           MAP(CAST(3.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float
           MAP(CAST(3.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double
-          MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean
+          MAP(false, true), -- t_map_boolean
           MAP(CAST(3.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2
           MAP(CAST(3.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4
           MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8
           MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17
           MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18
           MAP(CAST(3.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38
-          ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string
-          ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int
-          ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint
+          ARRAY('string1', 'string2'), -- t_array_string
+          ARRAY(4, 5, 6), -- t_array_int
+          ARRAY(300000000000, 400000000000), -- t_array_bigint
           ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float
           ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double
-          ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean
-          ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar
-          ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char
+          ARRAY(false, true), -- t_array_boolean
+          ARRAY('varchar1', 'varchar2'), -- t_array_varchar
+          ARRAY('char1', 'char2'), -- t_array_char
           ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2
           ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4
           ARRAY(CAST(8.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
           ARRAY(CAST(3.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17
           ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18
           ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38
-          NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint
-          MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex
-          NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested
-          NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null
-          NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls
-          NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls
+          NAMED_STRUCT('s_bigint', -1234567890), -- t_struct_bigint
+          MAP('key', ARRAY(NAMED_STRUCT('s_int', -123))), -- t_complex
+          NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested
+          NAMED_STRUCT('struct_field_null', null, 'struct_field_null2', null), -- t_struct_null
+          NAMED_STRUCT('struct_non_nulls_after_nulls1', -123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls
+          NAMED_STRUCT('struct_field1', -123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', -123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls
           MAP('null_key', null), -- t_map_null_value
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-          ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between
-          ARRAY(CAST('value11' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between
+          ARRAY('value11', 'value2', null), -- t_array_string_ending_with_nulls
           ARRAY(null, null, null), -- t_array_string_all_nulls
           20240322 -- dt
         );
         """
         order_qt_q02 """ select * from all_types_par_${format_compression}_${catalog_name}_q03;
         """
-//        disable it temporarily
-//        sql """refresh catalog ${catalog_name};"""
-
-//        sql """
-//        INSERT INTO all_types_par_${format_compression}_${catalog_name}_q03(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt)
-//        VALUES (
-//          CAST(123.45 AS FLOAT), -- float_col
-//          MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int
-//          ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
-//          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-//          20240321 -- dt
-//        );
-//        """
-//        order_qt_q03 """ select * from all_types_par_${format_compression}_${catalog_name}_q03;
-//        """
+
+        sql """
+        INSERT INTO all_types_par_${format_compression}_${catalog_name}_q03(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt)
+        VALUES (
+          123.45, -- float_col
+          MAP(1, 10), -- t_map_int
+          ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          20240321 -- dt
+        );
+        """
+        order_qt_q03 """ select * from all_types_par_${format_compression}_${catalog_name}_q03;
+        """
 
         sql """
         insert overwrite table all_types_par_${format_compression}_${catalog_name}_q03
         VALUES (
-          CAST(0 AS BOOLEAN), -- boolean_col
-          CAST(-7 AS TINYINT), -- tinyint_col
-          CAST(-15 AS SMALLINT), -- smallint_col
-          CAST(16 AS INT), -- int_col
-          CAST(-9223372036854775808 AS BIGINT), -- bigint_col
+          0, -- boolean_col
+          -7, -- tinyint_col
+          -15, -- smallint_col
+          16, -- int_col
+          -9223372036854775808, -- bigint_col
           CAST(-123.45 AS FLOAT), -- float_col
           CAST(-123456.789 AS DOUBLE), -- double_col
           CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1
           CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2
           CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3
           CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4
-          CAST('str' AS STRING), -- string_col
+          'str', -- string_col
           'binary_value', -- binary_col
           '2024-03-25', -- date_col
           '2024-03-25 12:00:00', -- timestamp_col1
           '2024-03-25 12:00:00.123456789', -- timestamp_col2
           '2024-03-25 12:00:00.123456789', -- timestamp_col3
-          CAST('char_value11111' AS CHAR(50)), -- char_col1
-          CAST('char_value22222' AS CHAR(100)), -- char_col2
-          CAST('char_value33333' AS CHAR(255)), -- char_col3
-          CAST('varchar_value11111' AS VARCHAR(50)), -- varchar_col1
-          CAST('varchar_value22222' AS VARCHAR(100)), -- varchar_col2
-          CAST('varchar_value33333' AS VARCHAR(255)), -- varchar_col3
-          MAP(CAST('key7' AS STRING), CAST('value1' AS STRING)), -- t_map_string
-          MAP(CAST('key7' AS VARCHAR(65535)), CAST('value1' AS VARCHAR(65535))), -- t_map_varchar
-          MAP(CAST('x' AS CHAR(10)), CAST('y' AS CHAR(10))), -- t_map_char
-          MAP(CAST(3 AS INT), CAST(20 AS INT)), -- t_map_int
-          MAP(CAST(3 AS BIGINT), CAST(200000000000 AS BIGINT)), -- t_map_bigint
+          'char_value11111', -- char_col1
+          'char_value22222', -- char_col2
+          'char_value33333', -- char_col3
+          'varchar_value11111', -- varchar_col1
+          'varchar_value22222', -- varchar_col2
+          'varchar_value33333', -- varchar_col3
+          MAP('key7', 'value1'), -- t_map_string
+          MAP('key7', 'value1'), -- t_map_varchar
+          MAP('x', 'y'), -- t_map_char
+          MAP(3, 20), -- t_map_int
+          MAP(3, 200000000000), -- t_map_bigint
           MAP(CAST(3.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float
           MAP(CAST(3.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double
-          MAP(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_map_boolean
+          MAP(false, true), -- t_map_boolean
           MAP(CAST(3.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2
           MAP(CAST(3.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4
           MAP(CAST(5.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8
           MAP(CAST(5.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17
           MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18
           MAP(CAST(7.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38
-          ARRAY(CAST('string1' AS STRING), CAST('string2' AS STRING)), -- t_array_string
-          ARRAY(CAST(4 AS INT), CAST(5 AS INT), CAST(6 AS INT)), -- t_array_int
-          ARRAY(CAST(300000000000 AS BIGINT), CAST(400000000000 AS BIGINT)), -- t_array_bigint
+          ARRAY('string1', 'string2'), -- t_array_string
+          ARRAY(4, 5, 6), -- t_array_int
+          ARRAY(300000000000, 400000000000), -- t_array_bigint
           ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float
           ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double
-          ARRAY(CAST(false AS BOOLEAN), CAST(true AS BOOLEAN)), -- t_array_boolean
-          ARRAY(CAST('varchar1' AS VARCHAR(65535)), CAST('varchar2' AS VARCHAR(65535))), -- t_array_varchar
-          ARRAY(CAST('char1' AS CHAR(10)), CAST('char2' AS CHAR(10))), -- t_array_char
+          ARRAY(false, true), -- t_array_boolean
+          ARRAY('varchar1', 'varchar2'), -- t_array_varchar
+          ARRAY('char1', 'char2'), -- t_array_char
           ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2
           ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4
           ARRAY(CAST(9.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
           ARRAY(CAST(6.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17
           ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18
           ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38
-          NAMED_STRUCT('s_bigint', CAST(-1234567890 AS BIGINT)), -- t_struct_bigint
-          MAP(CAST('key' AS STRING), ARRAY(NAMED_STRUCT('s_int', CAST(-123 AS INT)))), -- t_complex
-          NAMED_STRUCT('struct_field', ARRAY(CAST('value1' AS STRING), CAST('value2' AS STRING))), -- t_struct_nested
-          NAMED_STRUCT('struct_field_null', CAST(null AS STRING), 'struct_field_null2', CAST(null AS STRING)), -- t_struct_null
-          NAMED_STRUCT('struct_non_nulls_after_nulls1', CAST(-123 AS INT), 'struct_non_nulls_after_nulls2', CAST('value' AS STRING)), -- t_struct_non_nulls_after_nulls
-          NAMED_STRUCT('struct_field1', CAST(-123 AS INT), 'struct_field2', CAST('value' AS STRING), 'strict_field3', NAMED_STRUCT('nested_struct_field1', CAST(-123 AS INT), 'nested_struct_field2', CAST('nested_value' AS STRING))), -- t_nested_struct_non_nulls_after_nulls
+          NAMED_STRUCT('s_bigint', -1234567890), -- t_struct_bigint
+          MAP('key', ARRAY(NAMED_STRUCT('s_int', -123))), -- t_complex
+          NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested
+          NAMED_STRUCT('struct_field_null', null, 'struct_field_null2', null), -- t_struct_null
+          NAMED_STRUCT('struct_non_nulls_after_nulls1', -123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls
+          NAMED_STRUCT('struct_field1', -123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', -123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls
           MAP('null_key', null), -- t_map_null_value
-          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-          ARRAY(CAST('value1' AS STRING), null, CAST('value2' AS STRING)), -- t_array_string_with_nulls_in_between
-          ARRAY(CAST('value11' AS STRING), CAST('value2' AS STRING), null), -- t_array_string_ending_with_nulls
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between
+          ARRAY('value11', 'value2', null), -- t_array_string_ending_with_nulls
           ARRAY(null, null, null), -- t_array_string_all_nulls
           20240321 -- dt
         );
         """
         order_qt_q04 """ select * from all_types_par_${format_compression}_${catalog_name}_q03;
         """
-//        disable it temporarily
-//        sql """refresh catalog ${catalog_name};"""
-//
-//        sql """
-//        INSERT overwrite table all_types_par_${format_compression}_${catalog_name}_q03(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt)
-//        VALUES (
-//          CAST(123.45 AS FLOAT), -- float_col
-//          MAP(CAST(1 AS INT), CAST(10 AS INT)), -- t_map_int
-//          ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
-//          ARRAY(null, CAST('value1' AS STRING), CAST('value2' AS STRING)), -- t_array_string_starting_with_nulls
-//          20240321 -- dt
-//        );
-//        """
-//        order_qt_q05 """ select * from all_types_par_${format_compression}_${catalog_name}_q03;
-//        """
+
+        sql """
+        INSERT overwrite table all_types_par_${format_compression}_${catalog_name}_q03(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt)
+        VALUES (
+          CAST(123.45 AS FLOAT), -- float_col
+          MAP(1, 10), -- t_map_int
+          ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8
+          ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls
+          20240321 -- dt
+        );
+        """
+        order_qt_q05 """ select * from all_types_par_${format_compression}_${catalog_name}_q03;
+        """
 
         logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q03; """)
         hive_docker """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q03; """
@@ -848,7 +842,6 @@ suite("test_hive_write_insert", "p0,external,hive,external_docker,external_docke
         """
         order_qt_q01 """ select * from all_types_par_${format_compression}_${catalog_name}_q04;
         """
-        //sql """refresh catalog ${catalog_name};"""
 
         sql """
         INSERT INTO all_types_par_${format_compression}_${catalog_name}_q04
@@ -866,24 +859,21 @@ suite("test_hive_write_insert", "p0,external,hive,external_docker,external_docke
         """
         order_qt_q02 """ select * from all_types_par_${format_compression}_${catalog_name}_q04;
         """
-        //sql """refresh catalog ${catalog_name};"""
-
-        //sql """
-        //INSERT INTO all_types_par_${format_compression}_${catalog_name}_q04(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt)
-        //SELECT float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt FROM all_types_parquet_snappy_src;
-        //"""
-        //order_qt_q03 """ select * from all_types_par_${format_compression}_${catalog_name}_q04;
-        //"""
-        //sql """refresh catalog ${catalog_name};"""
-
-        //sql """
-        //INSERT OVERWRITE TABLE all_types_par_${format_compression}_${catalog_name}_q04(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt)
-        //SELECT float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt FROM all_types_parquet_snappy_src;
-        //"""
-        //order_qt_q04 """
-        //select * from all_types_par_${format_compression}_${catalog_name}_q04;
-        //"""
-        //sql """refresh catalog ${catalog_name};"""
+
+        sql """
+        INSERT INTO all_types_par_${format_compression}_${catalog_name}_q04(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt)
+        SELECT float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt FROM all_types_parquet_snappy_src;
+        """
+        order_qt_q03 """ select * from all_types_par_${format_compression}_${catalog_name}_q04;
+        """
+
+        sql """
+        INSERT OVERWRITE TABLE all_types_par_${format_compression}_${catalog_name}_q04(float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt)
+        SELECT float_col, t_map_int, t_array_decimal_precision_8, t_array_string_starting_with_nulls, dt FROM all_types_parquet_snappy_src;
+        """
+        order_qt_q04 """
+        select * from all_types_par_${format_compression}_${catalog_name}_q04;
+        """
 
         logger.info("hive sql: " + """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q04; """)
         hive_docker """ DROP TABLE IF EXISTS all_types_par_${format_compression}_${catalog_name}_q04; """
diff --git a/regression-test/suites/external_table_p0/hive/write/test_hive_write_partitions.groovy b/regression-test/suites/external_table_p0/hive/write/test_hive_write_partitions.groovy
index d2baf6ddc61..4e2d3bdd37e 100644
--- a/regression-test/suites/external_table_p0/hive/write/test_hive_write_partitions.groovy
+++ b/regression-test/suites/external_table_p0/hive/write/test_hive_write_partitions.groovy
@@ -214,8 +214,7 @@ suite("test_hive_write_partitions", "p0,external,hive,external_docker,external_d
                 q01(format_compression, catalog_name)
                 q02(format_compression, catalog_name)
                 q03(format_compression, catalog_name)
-//                disable it temporarily
-//                q04(format_compression, catalog_name)
+                q04(format_compression, catalog_name)
             }
             sql """drop catalog if exists ${catalog_name}"""
         } finally {


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


(doris) 19/19: [feature](iceberg)The new DDL syntax is added to create iceberg partitioned tables (#33338)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit d31bca199f0a4f66de1ac2400f86c8c58a2ac0af
Author: wuwenchi <wu...@hotmail.com>
AuthorDate: Wed Apr 10 20:39:53 2024 +0800

    [feature](iceberg)The new DDL syntax is added to create iceberg partitioned tables (#33338)
    
    support partition by :
    
    ```
    create table tb1 (c1 string, ts datetime) engine = iceberg partition by (c1, day(ts)) () properties ("a"="b")
    ```
---
 .../datasource/iceberg/DorisTypeToIcebergType.java |  14 +-
 .../datasource/iceberg/IcebergMetadataOps.java     |   2 +-
 .../doris/datasource/iceberg/IcebergUtils.java     |  94 +++++-----
 .../trees/plans/commands/info/CreateTableInfo.java |   4 +
 .../datasource/iceberg/CreateIcebergTableTest.java | 196 +++++++++++++++++++++
 5 files changed, 247 insertions(+), 63 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/DorisTypeToIcebergType.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/DorisTypeToIcebergType.java
index d6370c583da..52e4b6cf17a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/DorisTypeToIcebergType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/DorisTypeToIcebergType.java
@@ -102,27 +102,19 @@ public class DorisTypeToIcebergType extends DorisTypeVisitor<Type> {
         PrimitiveType primitiveType = atomic.getPrimitiveType();
         if (primitiveType.equals(PrimitiveType.BOOLEAN)) {
             return Types.BooleanType.get();
-        } else if (primitiveType.equals(PrimitiveType.TINYINT)
-                || primitiveType.equals(PrimitiveType.SMALLINT)
-                || primitiveType.equals(PrimitiveType.INT)) {
+        } else if (primitiveType.equals(PrimitiveType.INT)) {
             return Types.IntegerType.get();
-        } else if (primitiveType.equals(PrimitiveType.BIGINT)
-                || primitiveType.equals(PrimitiveType.LARGEINT)) {
+        } else if (primitiveType.equals(PrimitiveType.BIGINT)) {
             return Types.LongType.get();
         } else if (primitiveType.equals(PrimitiveType.FLOAT)) {
             return Types.FloatType.get();
         } else if (primitiveType.equals(PrimitiveType.DOUBLE)) {
             return Types.DoubleType.get();
-        } else if (primitiveType.equals(PrimitiveType.CHAR)
-                || primitiveType.equals(PrimitiveType.VARCHAR)
-                || primitiveType.equals(PrimitiveType.STRING)) {
+        } else if (primitiveType.equals(PrimitiveType.STRING)) {
             return Types.StringType.get();
         } else if (primitiveType.equals(PrimitiveType.DATE)
                 || primitiveType.equals(PrimitiveType.DATEV2)) {
             return Types.DateType.get();
-        } else if (primitiveType.equals(PrimitiveType.TIME)
-                || primitiveType.equals(PrimitiveType.TIMEV2)) {
-            return Types.TimeType.get();
         } else if (primitiveType.equals(PrimitiveType.DECIMALV2)
                 || primitiveType.isDecimalV3Type()) {
             return Types.DecimalType.of(
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java
index 0c188fae301..18efd7f1b7e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java
@@ -149,7 +149,7 @@ public class IcebergMetadataOps implements ExternalMetadataOps {
         Schema schema = new Schema(visit.asNestedType().asStructType().fields());
         Map<String, String> properties = stmt.getProperties();
         properties.put(ExternalCatalog.DORIS_VERSION, ExternalCatalog.DORIS_VERSION_VALUE);
-        PartitionSpec partitionSpec = IcebergUtils.solveIcebergPartitionSpec(properties, schema);
+        PartitionSpec partitionSpec = IcebergUtils.solveIcebergPartitionSpec(stmt.getPartitionDesc(), schema);
         catalog.createTable(TableIdentifier.of(dbName, tableName), schema, partitionSpec, properties);
         db.setUnInitialized(true);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java
index 08c4be4ceac..70c384a0c4b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java
@@ -25,10 +25,12 @@ import org.apache.doris.analysis.DateLiteral;
 import org.apache.doris.analysis.DecimalLiteral;
 import org.apache.doris.analysis.Expr;
 import org.apache.doris.analysis.FloatLiteral;
+import org.apache.doris.analysis.FunctionCallExpr;
 import org.apache.doris.analysis.InPredicate;
 import org.apache.doris.analysis.IntLiteral;
 import org.apache.doris.analysis.LiteralExpr;
 import org.apache.doris.analysis.NullLiteral;
+import org.apache.doris.analysis.PartitionDesc;
 import org.apache.doris.analysis.SlotRef;
 import org.apache.doris.analysis.StringLiteral;
 import org.apache.doris.analysis.Subquery;
@@ -63,8 +65,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 /**
  * Iceberg utils
@@ -78,7 +78,6 @@ public class IcebergUtils {
         }
     };
     static long MILLIS_TO_NANO_TIME = 1000;
-    private static final Pattern PARTITION_REG = Pattern.compile("(\\w+)\\((\\d+)?,?(\\w+)\\)");
     // https://iceberg.apache.org/spec/#schemas-and-data-types
     // All time and timestamp values are stored with microsecond precision
     private static final int ICEBERG_DATETIME_SCALE_MS = 6;
@@ -415,57 +414,51 @@ public class IcebergUtils {
         return slotRef;
     }
 
-    // "partition"="c1;day(c1);bucket(4,c3)"
-    public static PartitionSpec solveIcebergPartitionSpec(Map<String, String> properties, Schema schema)
+    public static PartitionSpec solveIcebergPartitionSpec(PartitionDesc partitionDesc, Schema schema)
             throws UserException {
-        if (properties.containsKey("partition")) {
-            PartitionSpec.Builder builder = PartitionSpec.builderFor(schema);
-            String par = properties.get("partition").replaceAll(" ", "");
-            String[] pars = par.split(";");
-            for (String func : pars) {
-                if (func.contains("(")) {
-                    Matcher matcher = PARTITION_REG.matcher(func);
-                    if (matcher.matches()) {
-                        switch (matcher.group(1).toLowerCase()) {
-                            case "bucket":
-                                builder.bucket(matcher.group(3), Integer.parseInt(matcher.group(2)));
-                                break;
-                            case "year":
-                            case "years":
-                                builder.year(matcher.group(3));
-                                break;
-                            case "month":
-                            case "months":
-                                builder.month(matcher.group(3));
-                                break;
-                            case "date":
-                            case "day":
-                            case "days":
-                                builder.day(matcher.group(3));
-                                break;
-                            case "date_hour":
-                            case "hour":
-                            case "hours":
-                                builder.hour(matcher.group(3));
-                                break;
-                            case "truncate":
-                                builder.truncate(matcher.group(3), Integer.parseInt(matcher.group(2)));
-                                break;
-                            default:
-                                throw new UserException("unsupported partition for " + matcher.group(1));
-                        }
-                    } else {
-                        throw new UserException("failed to get partition info from " + func);
-                    }
-                } else {
-                    builder.identity(func);
+        if (partitionDesc == null) {
+            return PartitionSpec.unpartitioned();
+        }
+
+        ArrayList<Expr> partitionExprs = partitionDesc.getPartitionExprs();
+        PartitionSpec.Builder builder = PartitionSpec.builderFor(schema);
+        for (Expr expr : partitionExprs) {
+            if (expr instanceof SlotRef) {
+                builder.identity(((SlotRef) expr).getColumnName());
+            } else if (expr instanceof FunctionCallExpr) {
+                String exprName = expr.getExprName();
+                List<Expr> params = ((FunctionCallExpr) expr).getParams().exprs();
+                switch (exprName.toLowerCase()) {
+                    case "bucket":
+                        builder.bucket(params.get(1).getExprName(), Integer.parseInt(params.get(0).getStringValue()));
+                        break;
+                    case "year":
+                    case "years":
+                        builder.year(params.get(0).getExprName());
+                        break;
+                    case "month":
+                    case "months":
+                        builder.month(params.get(0).getExprName());
+                        break;
+                    case "date":
+                    case "day":
+                    case "days":
+                        builder.day(params.get(0).getExprName());
+                        break;
+                    case "date_hour":
+                    case "hour":
+                    case "hours":
+                        builder.hour(params.get(0).getExprName());
+                        break;
+                    case "truncate":
+                        builder.truncate(params.get(1).getExprName(), Integer.parseInt(params.get(0).getStringValue()));
+                        break;
+                    default:
+                        throw new UserException("unsupported partition for " + exprName);
                 }
             }
-            properties.remove("partition");
-            return builder.build();
-        } else {
-            return PartitionSpec.unpartitioned();
         }
+        return builder.build();
     }
 
     private static Type icebergPrimitiveTypeToDorisType(org.apache.iceberg.types.Type.PrimitiveType primitive) {
@@ -567,5 +560,4 @@ public class IcebergUtils {
         }
         return -1;
     }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
index 0fc746da4e7..4a5a547f022 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
@@ -773,4 +773,8 @@ public class CreateTableInfo {
                 partitionDesc, distributionDesc, Maps.newHashMap(properties), extProperties,
                 comment, addRollups, null);
     }
+
+    public void setIsExternal(boolean isExternal) {
+        this.isExternal = isExternal;
+    }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/CreateIcebergTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/CreateIcebergTableTest.java
new file mode 100644
index 00000000000..863607af9d7
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/CreateIcebergTableTest.java
@@ -0,0 +1,196 @@
+// 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.iceberg;
+
+import org.apache.doris.analysis.CreateCatalogStmt;
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.analysis.DbName;
+import org.apache.doris.common.UserException;
+import org.apache.doris.datasource.CatalogFactory;
+import org.apache.doris.nereids.parser.NereidsParser;
+import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand;
+import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.qe.ConnectContext;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.List;
+import java.util.UUID;
+
+public class CreateIcebergTableTest {
+
+    public static String warehouse;
+    public static IcebergHadoopExternalCatalog icebergCatalog;
+    public static IcebergMetadataOps ops;
+    public static String dbName = "testdb";
+    public static ConnectContext connectContext;
+
+    @BeforeClass
+    public static void beforeClass() throws Throwable {
+        Path warehousePath = Files.createTempDirectory("test_warehouse_");
+        warehouse = "file://" + warehousePath.toAbsolutePath() + "/";
+
+        HashMap<String, String> param = new HashMap<>();
+        param.put("type", "iceberg");
+        param.put("iceberg.catalog.type", "hadoop");
+        param.put("warehouse", warehouse);
+
+        // create catalog
+        CreateCatalogStmt createCatalogStmt = new CreateCatalogStmt(true, "iceberg", "", param, "comment");
+        icebergCatalog = (IcebergHadoopExternalCatalog) CatalogFactory.createFromStmt(1, createCatalogStmt);
+        icebergCatalog.setInitialized(true);
+
+        // create db
+        ops = new IcebergMetadataOps(icebergCatalog, icebergCatalog.getCatalog());
+        CreateDbStmt createDbStmt = new CreateDbStmt(true, new DbName("iceberg", dbName), null);
+        ops.createDb(createDbStmt);
+        icebergCatalog.setInitialized(true);
+        IcebergExternalDatabase db = new IcebergExternalDatabase(icebergCatalog, 1L, dbName);
+        icebergCatalog.addDatabaseForTest(db);
+
+        // context
+        connectContext = new ConnectContext();
+        connectContext.setThreadLocalInfo();
+    }
+
+    @Test
+    public void testSimpleTable() throws UserException {
+        TableIdentifier tb = TableIdentifier.of(dbName, getTableName());
+        String sql = "create table " + tb + " (id int) engine = iceberg";
+        createTable(sql);
+        Table table = ops.getCatalog().loadTable(tb);
+        Schema schema = table.schema();
+        Assert.assertEquals(1, schema.columns().size());
+        Assert.assertEquals(PartitionSpec.unpartitioned(), table.spec());
+    }
+
+    @Test
+    public void testProperties() throws UserException {
+        TableIdentifier tb = TableIdentifier.of(dbName, getTableName());
+        String sql = "create table " + tb + " (id int) engine = iceberg properties(\"a\"=\"b\")";
+        createTable(sql);
+        Table table = ops.getCatalog().loadTable(tb);
+        Schema schema = table.schema();
+        Assert.assertEquals(1, schema.columns().size());
+        Assert.assertEquals(PartitionSpec.unpartitioned(), table.spec());
+        Assert.assertEquals("b", table.properties().get("a"));
+    }
+
+    @Test
+    public void testType() throws UserException {
+        TableIdentifier tb = TableIdentifier.of(dbName, getTableName());
+        String sql = "create table " + tb + " ("
+                + "c0 int, "
+                + "c1 bigint, "
+                + "c2 float, "
+                + "c3 double, "
+                + "c4 string, "
+                + "c5 date, "
+                + "c6 decimal(20, 10), "
+                + "c7 datetime"
+                + ") engine = iceberg "
+                + "properties(\"a\"=\"b\")";
+        createTable(sql);
+        Table table = ops.getCatalog().loadTable(tb);
+        Schema schema = table.schema();
+        List<Types.NestedField> columns = schema.columns();
+        Assert.assertEquals(8, columns.size());
+        Assert.assertEquals(Type.TypeID.INTEGER, columns.get(0).type().typeId());
+        Assert.assertEquals(Type.TypeID.LONG, columns.get(1).type().typeId());
+        Assert.assertEquals(Type.TypeID.FLOAT, columns.get(2).type().typeId());
+        Assert.assertEquals(Type.TypeID.DOUBLE, columns.get(3).type().typeId());
+        Assert.assertEquals(Type.TypeID.STRING, columns.get(4).type().typeId());
+        Assert.assertEquals(Type.TypeID.DATE, columns.get(5).type().typeId());
+        Assert.assertEquals(Type.TypeID.DECIMAL, columns.get(6).type().typeId());
+        Assert.assertEquals(Type.TypeID.TIMESTAMP, columns.get(7).type().typeId());
+    }
+
+    @Test
+    public void testPartition() throws UserException {
+        TableIdentifier tb = TableIdentifier.of(dbName, getTableName());
+        String sql = "create table " + tb + " ("
+                + "id int, "
+                + "ts1 datetime, "
+                + "ts2 datetime, "
+                + "ts3 datetime, "
+                + "ts4 datetime, "
+                + "dt1 date, "
+                + "dt2 date, "
+                + "dt3 date, "
+                + "s string"
+                + ") engine = iceberg "
+                + "partition by ("
+                + "id, "
+                + "bucket(2, id), "
+                + "year(ts1), "
+                + "year(dt1), "
+                + "month(ts2), "
+                + "month(dt2), "
+                + "day(ts3), "
+                + "day(dt3), "
+                + "hour(ts4), "
+                + "truncate(10, s)) ()"
+                + "properties(\"a\"=\"b\")";
+        createTable(sql);
+        Table table = ops.getCatalog().loadTable(tb);
+        Schema schema = table.schema();
+        Assert.assertEquals(9, schema.columns().size());
+        PartitionSpec spec = PartitionSpec.builderFor(schema)
+                .identity("id")
+                .bucket("id", 2)
+                .year("ts1")
+                .year("dt1")
+                .month("ts2")
+                .month("dt2")
+                .day("ts3")
+                .day("dt3")
+                .hour("ts4")
+                .truncate("s", 10)
+                .build();
+        Assert.assertEquals(spec, table.spec());
+        Assert.assertEquals("b", table.properties().get("a"));
+    }
+
+    public void createTable(String sql) throws UserException {
+        LogicalPlan plan = new NereidsParser().parseSingle(sql);
+        Assertions.assertTrue(plan instanceof CreateTableCommand);
+        CreateTableInfo createTableInfo = ((CreateTableCommand) plan).getCreateTableInfo();
+        createTableInfo.setIsExternal(true);
+        CreateTableStmt createTableStmt = createTableInfo.translateToLegacyStmt();
+        ops.createTable(createTableStmt);
+    }
+
+    public String getTableName() {
+        String s = "test_tb_" + UUID.randomUUID();
+        return s.replaceAll("-", "");
+    }
+}


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


(doris) 13/19: [feature](multi-catalog)support catalog name when create/drop db (#33116)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 7a05396cd11e68e4b2db131dfe9c46ac025d9d34
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Tue Apr 2 22:54:45 2024 +0800

    [feature](multi-catalog)support catalog name when create/drop db (#33116)
    
    Issue Number: #31442
---
 fe/fe-core/src/main/cup/sql_parser.cup             | 16 +++-
 .../org/apache/doris/analysis/CreateDbStmt.java    | 16 +++-
 .../java/org/apache/doris/analysis/DbName.java     | 86 ++++++++++++++++++++++
 .../java/org/apache/doris/analysis/DropDbStmt.java | 10 ++-
 .../main/java/org/apache/doris/catalog/Env.java    | 16 +++-
 .../doris/catalog/InternalSchemaInitializer.java   |  5 +-
 .../apache/doris/analysis/CreateDbStmtTest.java    |  9 ++-
 .../org/apache/doris/analysis/DropDbStmtTest.java  |  7 +-
 .../datasource/hive/HiveDDLAndDMLPlanTest.java     |  3 +-
 .../doris/datasource/hive/HiveMetadataOpsTest.java |  5 +-
 10 files changed, 151 insertions(+), 22 deletions(-)

diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index 07f62881b41..3a4f77fcc7a 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -772,6 +772,7 @@ nonterminal ArrayList<Long> opt_tablet_list, tablet_list;
 nonterminal TableSample opt_table_sample, table_sample;
 nonterminal TableSnapshot opt_table_snapshot, table_snapshot;
 nonterminal TableName table_name, opt_table_name;
+nonterminal DbName db_name;
 nonterminal FunctionName function_name;
 nonterminal EncryptKeyName encryptkey_name;
 nonterminal Expr pre_filter_clause;
@@ -1766,11 +1767,11 @@ opt_intermediate_type ::=
 // Create Statement
 create_stmt ::=
     /* Database */
-    KW_CREATE KW_DATABASE opt_if_not_exists:ifNotExists ident:db opt_properties:properties
+    KW_CREATE KW_DATABASE opt_if_not_exists:ifNotExists db_name:db opt_properties:properties
     {:
         RESULT = new CreateDbStmt(ifNotExists, db, properties);
     :}
-    | KW_CREATE KW_SCHEMA opt_if_not_exists:ifNotExists ident:db
+    | KW_CREATE KW_SCHEMA opt_if_not_exists:ifNotExists db_name:db
     {:
         RESULT = new CreateDbStmt(ifNotExists, db, null);
     :}
@@ -2989,11 +2990,11 @@ revoke_stmt ::=
 // Drop statement
 drop_stmt ::=
     /* Database */
-    KW_DROP KW_DATABASE opt_if_exists:ifExists ident:db opt_force:force
+    KW_DROP KW_DATABASE opt_if_exists:ifExists db_name:db opt_force:force
     {:
         RESULT = new DropDbStmt(ifExists, db, force);
     :}
-    | KW_DROP KW_SCHEMA opt_if_exists:ifExists ident:db opt_force:force
+    | KW_DROP KW_SCHEMA opt_if_exists:ifExists db_name:db opt_force:force
     {:
         RESULT = new DropDbStmt(ifExists, db, force);
     :}
@@ -5793,6 +5794,13 @@ table_name ::=
     {: RESULT = new TableName(ctl, db, tbl); :}
     ;
 
+db_name ::=
+    ident:db
+    {: RESULT = new DbName(null, db); :}
+    | ident:ctl DOT ident:db
+    {: RESULT = new DbName(ctl, db); :}
+    ;
+
 colocate_group_name ::=
     ident:group
     {:
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateDbStmt.java
index 6828e46fa65..39754244e96 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateDbStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateDbStmt.java
@@ -27,17 +27,21 @@ import org.apache.doris.common.util.PrintableMap;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.qe.ConnectContext;
 
+import org.apache.commons.lang3.StringUtils;
+
 import java.util.HashMap;
 import java.util.Map;
 
 public class CreateDbStmt extends DdlStmt {
     private boolean ifNotExists;
+    private String ctlName;
     private String dbName;
     private Map<String, String> properties;
 
-    public CreateDbStmt(boolean ifNotExists, String dbName, Map<String, String> properties) {
+    public CreateDbStmt(boolean ifNotExists, DbName dbName, Map<String, String> properties) {
         this.ifNotExists = ifNotExists;
-        this.dbName = dbName;
+        this.ctlName = dbName.getCtl();
+        this.dbName = dbName.getDb();
         this.properties = properties == null ? new HashMap<>() : properties;
     }
 
@@ -45,6 +49,10 @@ public class CreateDbStmt extends DdlStmt {
         return dbName;
     }
 
+    public String getCtlName() {
+        return ctlName;
+    }
+
     public boolean isSetIfNotExists() {
         return ifNotExists;
     }
@@ -56,6 +64,10 @@ public class CreateDbStmt extends DdlStmt {
     @Override
     public void analyze(Analyzer analyzer) throws UserException {
         super.analyze(analyzer);
+        if (StringUtils.isEmpty(ctlName)) {
+            ctlName = Env.getCurrentEnv().getCurrentCatalog().getName();
+        }
+        FeNameFormat.checkCatalogName(ctlName);
         FeNameFormat.checkDbName(dbName);
         InternalDatabaseUtil.checkDatabase(dbName, ConnectContext.get());
         if (!Env.getCurrentEnv().getAccessManager().checkDbPriv(ConnectContext.get(), dbName, PrivPredicate.CREATE)) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DbName.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DbName.java
new file mode 100644
index 00000000000..2b8e92ab982
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DbName.java
@@ -0,0 +1,86 @@
+// 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.
+// This file is copied from
+// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/TableName.java
+// and modified by Doris
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.datasource.InternalCatalog;
+
+import java.util.Objects;
+
+public class DbName {
+    private String ctl;
+    private String db;
+
+    public DbName(String ctl, String db) {
+        this.ctl = ctl;
+        this.db = db;
+    }
+
+    public String getCtl() {
+        return ctl;
+    }
+
+    public void setCtl(String ctl) {
+        this.ctl = ctl;
+    }
+
+    public String getDb() {
+        return db;
+    }
+
+    public void setDb(String db) {
+        this.db = db;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder stringBuilder = new StringBuilder();
+        if (ctl != null && !ctl.equals(InternalCatalog.INTERNAL_CATALOG_NAME)) {
+            stringBuilder.append(ctl).append(".");
+        }
+        stringBuilder.append(db);
+        return stringBuilder.toString();
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (other instanceof DbName) {
+            return toString().equals(other.toString());
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(ctl, db);
+    }
+
+    public String toSql() {
+        StringBuilder stringBuilder = new StringBuilder();
+        if (ctl != null && !ctl.equals(InternalCatalog.INTERNAL_CATALOG_NAME)) {
+            stringBuilder.append("`").append(ctl).append("`.");
+        }
+        stringBuilder.append("`").append(db).append("`");
+        return stringBuilder.toString();
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java
index 08a1e630eb5..bd00d06cc5c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java
@@ -32,12 +32,14 @@ import com.google.common.base.Strings;
 // DROP DBč”Øč¾¾å¼
 public class DropDbStmt extends DdlStmt {
     private boolean ifExists;
+    private String ctlName;
     private String dbName;
     private boolean forceDrop;
 
-    public DropDbStmt(boolean ifExists, String dbName, boolean forceDrop) {
+    public DropDbStmt(boolean ifExists, DbName dbName, boolean forceDrop) {
         this.ifExists = ifExists;
-        this.dbName = dbName;
+        this.ctlName = dbName.getCtl();
+        this.dbName = dbName.getDb();
         this.forceDrop = forceDrop;
     }
 
@@ -45,6 +47,10 @@ public class DropDbStmt extends DdlStmt {
         return ifExists;
     }
 
+    public String getCtlName() {
+        return ctlName;
+    }
+
     public String getDbName() {
         return this.dbName;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
index dd6ec52bac0..69906597b79 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
@@ -2967,7 +2967,13 @@ public class Env {
 
     // The interface which DdlExecutor needs.
     public void createDb(CreateDbStmt stmt) throws DdlException {
-        getCurrentCatalog().createDb(stmt);
+        CatalogIf<?> catalogIf;
+        if (StringUtils.isEmpty(stmt.getCtlName())) {
+            catalogIf = getCurrentCatalog();
+        } else {
+            catalogIf = catalogMgr.getCatalog(stmt.getCtlName());
+        }
+        catalogIf.createDb(stmt);
     }
 
     // For replay edit log, need't lock metadata
@@ -2980,7 +2986,13 @@ public class Env {
     }
 
     public void dropDb(DropDbStmt stmt) throws DdlException {
-        getCurrentCatalog().dropDb(stmt);
+        CatalogIf<?> catalogIf;
+        if (StringUtils.isEmpty(stmt.getCtlName())) {
+            catalogIf = getCurrentCatalog();
+        } else {
+            catalogIf = catalogMgr.getCatalog(stmt.getCtlName());
+        }
+        catalogIf.dropDb(stmt);
     }
 
     public void replayDropDb(String dbName, boolean isForceDrop, Long recycleTime) throws DdlException {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/InternalSchemaInitializer.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/InternalSchemaInitializer.java
index 93617ac0f5e..0e12367210d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/InternalSchemaInitializer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/InternalSchemaInitializer.java
@@ -21,6 +21,7 @@ import org.apache.doris.analysis.AlterClause;
 import org.apache.doris.analysis.AlterTableStmt;
 import org.apache.doris.analysis.CreateDbStmt;
 import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.analysis.DbName;
 import org.apache.doris.analysis.DistributionDesc;
 import org.apache.doris.analysis.DropTableStmt;
 import org.apache.doris.analysis.HashDistributionDesc;
@@ -164,8 +165,8 @@ public class InternalSchemaInitializer extends Thread {
 
     @VisibleForTesting
     public static void createDb() {
-        CreateDbStmt createDbStmt = new CreateDbStmt(true, FeConstants.INTERNAL_DB_NAME,
-                null);
+        CreateDbStmt createDbStmt = new CreateDbStmt(true,
+                new DbName("internal", FeConstants.INTERNAL_DB_NAME), null);
         try {
             Env.getCurrentEnv().createDb(createDbStmt);
         } catch (DdlException e) {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateDbStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateDbStmtTest.java
index 6d8319de7c4..3ef1685128f 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateDbStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateDbStmtTest.java
@@ -48,7 +48,7 @@ public class CreateDbStmtTest {
 
     @Test
     public void testAnalyzeNormal() throws UserException {
-        CreateDbStmt dbStmt = new CreateDbStmt(false, "test", null);
+        CreateDbStmt dbStmt = new CreateDbStmt(false, new DbName(null, "test"), null);
         dbStmt.analyze(analyzer);
         Assert.assertEquals("test", dbStmt.getFullDbName());
         Assert.assertEquals("CREATE DATABASE `test`", dbStmt.toString());
@@ -56,7 +56,7 @@ public class CreateDbStmtTest {
 
     @Test(expected = AnalysisException.class)
     public void testAnalyzeWithException() throws UserException {
-        CreateDbStmt stmt = new CreateDbStmt(false, "", null);
+        CreateDbStmt stmt = new CreateDbStmt(false, new DbName("", ""), null);
         stmt.analyze(analyzer);
         Assert.fail("no exception");
     }
@@ -66,8 +66,9 @@ public class CreateDbStmtTest {
         Map<String, String> properties = new HashMap<>();
         properties.put("iceberg.database", "doris");
         properties.put("iceberg.hive.metastore.uris", "thrift://127.0.0.1:9087");
-        CreateDbStmt stmt = new CreateDbStmt(false, "test", properties);
+        CreateDbStmt stmt = new CreateDbStmt(false, new DbName("ctl", "test"), properties);
         stmt.analyze(analyzer);
+        Assert.assertEquals("ctl", stmt.getCtlName());
         Assert.assertEquals("test", stmt.getFullDbName());
         Assert.assertEquals("CREATE DATABASE `test`\n"
                 + "PROPERTIES (\n"
@@ -81,7 +82,7 @@ public class CreateDbStmtTest {
         Map<String, String> properties = new HashMap<>();
         properties.put("iceberg.database", "doris");
         properties.put("iceberg.hive.metastore.uris", "thrift://127.0.0.1:9087");
-        CreateDbStmt stmt = new CreateDbStmt(false, "", properties);
+        CreateDbStmt stmt = new CreateDbStmt(false, new DbName("", ""), properties);
         stmt.analyze(analyzer);
         Assert.fail("No exception throws.");
     }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java
index e91e56c53a8..6b545efc200 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java
@@ -45,16 +45,17 @@ public class DropDbStmtTest {
 
     @Test
     public void testNormal() throws UserException, AnalysisException {
-        DropDbStmt stmt = new DropDbStmt(false, "test", true);
+        DropDbStmt stmt = new DropDbStmt(false, new DbName("test", "test"), true);
 
         stmt.analyze(analyzer);
+        Assert.assertEquals("test", stmt.getCtlName());
         Assert.assertEquals("test", stmt.getDbName());
         Assert.assertEquals("DROP DATABASE `test`", stmt.toString());
     }
 
     @Test(expected = AnalysisException.class)
     public void testFailed() throws UserException, AnalysisException {
-        DropDbStmt stmt = new DropDbStmt(false, "", true);
+        DropDbStmt stmt = new DropDbStmt(false, new DbName("", ""), true);
 
         stmt.analyze(analyzer);
         Assert.fail("no exception");
@@ -62,7 +63,7 @@ public class DropDbStmtTest {
 
     @Test
     public void testNoPriv() {
-        DropDbStmt stmt = new DropDbStmt(false, "", true);
+        DropDbStmt stmt = new DropDbStmt(false, new DbName("", ""), true);
         try {
             stmt.analyze(AccessTestUtil.fetchBlockAnalyzer());
         } catch (AnalysisException e) {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
index 8247bd84b4d..a0f1ce498b2 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
@@ -20,6 +20,7 @@ package org.apache.doris.datasource.hive;
 import org.apache.doris.analysis.CreateCatalogStmt;
 import org.apache.doris.analysis.CreateDbStmt;
 import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.analysis.DbName;
 import org.apache.doris.analysis.HashDistributionDesc;
 import org.apache.doris.analysis.SwitchStmt;
 import org.apache.doris.catalog.Column;
@@ -140,7 +141,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
                 }
             }
         };
-        CreateDbStmt createDbStmt = new CreateDbStmt(true, mockedDbName, dbProps);
+        CreateDbStmt createDbStmt = new CreateDbStmt(true, new DbName("hive", mockedDbName), dbProps);
         Env.getCurrentEnv().createDb(createDbStmt);
         useDatabase(mockedDbName);
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
index 6c156807fc6..54ea6c9a5f1 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
@@ -19,6 +19,7 @@ package org.apache.doris.datasource.hive;
 
 import org.apache.doris.analysis.CreateDbStmt;
 import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.analysis.DbName;
 import org.apache.doris.analysis.DistributionDesc;
 import org.apache.doris.analysis.DropDbStmt;
 import org.apache.doris.analysis.DropTableStmt;
@@ -99,12 +100,12 @@ public class HiveMetadataOpsTest {
     }
 
     private void createDb(String dbName, Map<String, String> props) throws DdlException {
-        CreateDbStmt createDbStmt = new CreateDbStmt(true, dbName, props);
+        CreateDbStmt createDbStmt = new CreateDbStmt(true, new DbName("hive", dbName), props);
         metadataOps.createDb(createDbStmt);
     }
 
     private void dropDb(String dbName, boolean forceDrop) throws DdlException {
-        DropDbStmt dropDbStmt = new DropDbStmt(true, dbName, forceDrop);
+        DropDbStmt dropDbStmt = new DropDbStmt(true, new DbName("hive", dbName), forceDrop);
         metadataOps.dropDb(dropDbStmt);
     }
 


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


(doris) 08/19: [chore] Format regression-conf.groovy (#32713)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit f3a6132214aadcc78676e4a81605593d711ea05a
Author: Gavin Chou <ga...@gmail.com>
AuthorDate: Mon Mar 25 23:12:02 2024 +0800

    [chore] Format regression-conf.groovy (#32713)
---
 regression-test/conf/regression-conf.groovy             |  5 ++++-
 .../pipeline/external/conf/regression-conf.groovy       |  9 ++++++++-
 regression-test/pipeline/p0/conf/regression-conf.groovy | 16 ++++++++++++++--
 regression-test/pipeline/p1/conf/regression-conf.groovy | 17 +++++++++++++++--
 4 files changed, 41 insertions(+), 6 deletions(-)

diff --git a/regression-test/conf/regression-conf.groovy b/regression-test/conf/regression-conf.groovy
index c6764a2ce86..041744c9b02 100644
--- a/regression-test/conf/regression-conf.groovy
+++ b/regression-test/conf/regression-conf.groovy
@@ -86,7 +86,10 @@ excludeGroups = ""
 // this suites will not be executed
 excludeSuites = "test_broker_load"
 // this directories will not be executed
-excludeDirectories = "segcompaction_p2,workload_manager_p1"
+excludeDirectories = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line
+    "segcompaction_p2," +
+    "workload_manager_p1," +
+    "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line
 
 customConf1 = "test_custom_conf_value"
 
diff --git a/regression-test/pipeline/external/conf/regression-conf.groovy b/regression-test/pipeline/external/conf/regression-conf.groovy
index 3c9071886db..aa85ad46138 100644
--- a/regression-test/pipeline/external/conf/regression-conf.groovy
+++ b/regression-test/pipeline/external/conf/regression-conf.groovy
@@ -55,7 +55,14 @@ testDirectories = ""
 // this groups will not be executed
 excludeGroups = ""
 // this suites will not be executed
-excludeSuites = "test_cast_string_to_array,test_broker_load,test_spark_load,test_analyze_stats_p1,test_refresh_mtmv"
+excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line
+    "test_analyze_stats_p1," +
+    "test_broker_load," +
+    "test_cast_string_to_array," +
+    "test_refresh_mtmv," +
+    "test_spark_load," +
+    "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line
+
 // this directories will not be executed
 excludeDirectories = ""
 
diff --git a/regression-test/pipeline/p0/conf/regression-conf.groovy b/regression-test/pipeline/p0/conf/regression-conf.groovy
index 32f4a6e9f59..e9e12f44eb9 100644
--- a/regression-test/pipeline/p0/conf/regression-conf.groovy
+++ b/regression-test/pipeline/p0/conf/regression-conf.groovy
@@ -61,10 +61,22 @@ excludeGroups = ""
 // this suites will not be executed
 // load_stream_fault_injection may cause bad disk
 
-excludeSuites = "test_stream_stub_fault_injection,test_index_failure_injection,test_dump_image,test_profile,test_spark_load,test_refresh_mtmv,test_bitmap_filter,test_information_schema_external"
+excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line
+    "test_bitmap_filter," +
+    "test_dump_image," +
+    "test_index_failure_injection," +
+    "test_information_schema_external," +
+    "test_profile," +
+    "test_refresh_mtmv," +
+    "test_spark_load," +
+    "test_stream_stub_fault_injection," +
+    "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line
 
 // this directories will not be executed
-excludeDirectories = "workload_manager_p1,nereids_rules_p0/subquery"
+excludeDirectories = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line
+    "nereids_rules_p0/subquery," +
+    "workload_manager_p1," +
+    "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line
 
 customConf1 = "test_custom_conf_value"
 
diff --git a/regression-test/pipeline/p1/conf/regression-conf.groovy b/regression-test/pipeline/p1/conf/regression-conf.groovy
index 517fd4b4dce..398e4a41ad4 100644
--- a/regression-test/pipeline/p1/conf/regression-conf.groovy
+++ b/regression-test/pipeline/p1/conf/regression-conf.groovy
@@ -52,9 +52,22 @@ testGroups = ""
 // empty suite will test all suite
 testSuites = ""
 // this suites will not be executed
-excludeSuites = "test_big_pad,test_profile,test_broker_load,test_spark_load,test_analyze_stats_p1,test_refresh_mtmv,test_bitmap_filter"
+excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line
+    "test_analyze_stats_p1," +
+    "test_big_pad," +
+    "test_bitmap_filter," +
+    "test_broker_load," +
+    "test_profile," +
+    "test_refresh_mtmv," +
+    "test_spark_load," +
+    "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line
+
 // this dir will not be executed
-excludeDirectories = "workload_manager_p1,fault_injection_p0"
+excludeDirectories = "000_the_start_sentinel_do_not_touch," + // keep this line as the first line
+    "fault_injection_p0," +
+    "workload_manager_p1," +
+    "zzz_the_end_sentinel_do_not_touch" // keep this line as the last line
+
 cacheDataPath="/data/regression/"
 
 s3Endpoint = "cos.ap-hongkong.myqcloud.com"


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


(doris) 16/19: [testcase](hive)add exception test for hive txn (#33278)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 31a7060dbdb88c8e81be00aada4a2df35eca625c
Author: wuwenchi <wu...@hotmail.com>
AuthorDate: Wed Apr 10 15:09:01 2024 +0800

    [testcase](hive)add exception test for hive txn (#33278)
    
    Issue #31442
    #32726
    
    1. add LocalDfsFileSystem to manipulate local files.
    2. add HMSCachedClientTest to analog HMS services.
    3. add test for rollback commit.
---
 .../doris/datasource/hive/HMSTransaction.java      |  91 +++--
 .../doris/datasource/hive/HiveMetadataOps.java     |  14 +-
 .../org/apache/doris/datasource/hive/HiveUtil.java |  65 +++-
 .../datasource/hive/ThriftHMSCachedClient.java     |  76 +----
 .../main/java/org/apache/doris/fs/FileSystem.java  |   9 +
 .../org/apache/doris/fs/LocalDfsFileSystem.java    | 245 ++++++++++++++
 .../java/org/apache/doris/fs/LocalFileSystem.java  |  76 -----
 .../apache/doris/fs/remote/dfs/DFSFileSystem.java  |  45 +++
 .../doris/datasource/HMSCachedClientTest.java      | 328 ++++++++++++++++++
 .../doris/datasource/hive/HmsCommitTest.java       | 374 ++++++++++++++++-----
 10 files changed, 1033 insertions(+), 290 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java
index c3e8d00c5d1..84221b74e7f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java
@@ -23,8 +23,8 @@ package org.apache.doris.datasource.hive;
 
 import org.apache.doris.backup.Status;
 import org.apache.doris.common.Pair;
+import org.apache.doris.fs.FileSystem;
 import org.apache.doris.fs.remote.RemoteFile;
-import org.apache.doris.fs.remote.RemoteFileSystem;
 import org.apache.doris.thrift.THivePartitionUpdate;
 import org.apache.doris.thrift.TUpdateMode;
 import org.apache.doris.transaction.Transaction;
@@ -47,11 +47,13 @@ import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.Queue;
+import java.util.Set;
 import java.util.StringJoiner;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -63,7 +65,7 @@ import java.util.stream.Collectors;
 public class HMSTransaction implements Transaction {
     private static final Logger LOG = LogManager.getLogger(HMSTransaction.class);
     private final HiveMetadataOps hiveOps;
-    private final RemoteFileSystem fs;
+    private final FileSystem fs;
     private String dbName;
     private String tbName;
 
@@ -115,8 +117,8 @@ public class HMSTransaction implements Transaction {
     }
 
     public void finishInsertTable(String dbName, String tbName) {
-        this.tbName = tbName;
         this.dbName = dbName;
+        this.tbName = tbName;
         List<THivePartitionUpdate> mergedPUs = mergePartitions(hivePartitionUpdates);
         Table table = getTable(dbName, tbName);
         List<Pair<THivePartitionUpdate, HivePartitionStatistics>> insertExistsPartitions = new ArrayList<>();
@@ -226,17 +228,10 @@ public class HMSTransaction implements Transaction {
                 }
             }
 
-            hmsCommitter.waitForAsyncFileSystemTasks();
-            hmsCommitter.doAddPartitionsTask();
-            hmsCommitter.doUpdateStatisticsTasks();
+            hmsCommitter.doCommit();
         } catch (Throwable t) {
             LOG.warn("Failed to commit for {}.{}, abort it.", dbName, tbName);
-            hmsCommitter.cancelUnStartedAsyncFileSystemTask();
-            hmsCommitter.undoUpdateStatisticsTasks();
-            hmsCommitter.undoAddPartitionsTask();
-            hmsCommitter.waitForAsyncFileSystemTaskSuppressThrowable();
-            hmsCommitter.runDirectoryClearUpTasksForAbort();
-            hmsCommitter.runRenameDirTasksForAbort();
+            hmsCommitter.rollback();
             throw t;
         } finally {
             hmsCommitter.runClearPathsForFinish();
@@ -354,7 +349,7 @@ public class HMSTransaction implements Transaction {
         }
     }
 
-    private static class UpdateStatisticsTask {
+    public static class UpdateStatisticsTask {
         private final String dbName;
         private final String tableName;
         private final Optional<String> partitionName;
@@ -442,7 +437,6 @@ public class HMSTransaction implements Transaction {
                     throw t;
                 }
             }
-            partitions.clear();
         }
 
         public List<List<String>> rollback(HiveMetadataOps hiveOps) {
@@ -548,7 +542,7 @@ public class HMSTransaction implements Transaction {
 
     private DeleteRecursivelyResult recursiveDeleteFiles(Path directory, boolean deleteEmptyDir) {
         try {
-            if (!fs.exists(directory.getName()).ok()) {
+            if (!fs.exists(directory.toString()).ok()) {
                 return new DeleteRecursivelyResult(true, ImmutableList.of());
             }
         } catch (Exception e) {
@@ -561,57 +555,53 @@ public class HMSTransaction implements Transaction {
     }
 
     private DeleteRecursivelyResult doRecursiveDeleteFiles(Path directory, boolean deleteEmptyDir) {
-        List<RemoteFile> remoteFiles = new ArrayList<>();
-
-        Status status = fs.list(directory.getName(), remoteFiles);
-        if (!status.ok()) {
+        List<RemoteFile> allFiles = new ArrayList<>();
+        Set<String> allDirs = new HashSet<>();
+        Status statusFile = fs.listFiles(directory.toString(), allFiles);
+        Status statusDir = fs.listDirectories(directory.toString(), allDirs);
+        if (!statusFile.ok() || !statusDir.ok()) {
             ImmutableList.Builder<String> notDeletedEligibleItems = ImmutableList.builder();
             notDeletedEligibleItems.add(directory + "/*");
             return new DeleteRecursivelyResult(false, notDeletedEligibleItems.build());
         }
 
-        boolean isEmptyDir = true;
-        List<String> notDeletedEligibleItems = new ArrayList<>();
-        for (RemoteFile file : remoteFiles) {
-            if (file.isFile()) {
-                Path filePath = file.getPath();
-                isEmptyDir = false;
-                // TODO Check if this file was created by this query
-                if (!deleteIfExists(filePath)) {
-                    notDeletedEligibleItems.add(filePath.toString());
-                }
-            } else if (file.isDirectory()) {
-                DeleteRecursivelyResult subResult = doRecursiveDeleteFiles(file.getPath(), deleteEmptyDir);
-                if (!subResult.dirNotExists()) {
-                    isEmptyDir = false;
-                }
-                if (!subResult.getNotDeletedEligibleItems().isEmpty()) {
-                    notDeletedEligibleItems.addAll(subResult.getNotDeletedEligibleItems());
-                }
-            } else {
-                isEmptyDir = false;
-                notDeletedEligibleItems.add(file.getPath().toString());
+        boolean allDescendentsDeleted = true;
+        ImmutableList.Builder<String> notDeletedEligibleItems = ImmutableList.builder();
+        for (RemoteFile file : allFiles) {
+            String fileName = file.getName();
+            if (!deleteIfExists(new Path(fileName))) {
+                allDescendentsDeleted = false;
+                notDeletedEligibleItems.add(fileName);
             }
         }
 
-        if (isEmptyDir && deleteEmptyDir) {
-            Verify.verify(notDeletedEligibleItems.isEmpty());
+        for (String dir : allDirs) {
+            DeleteRecursivelyResult subResult = doRecursiveDeleteFiles(new Path(dir), deleteEmptyDir);
+            if (!subResult.dirNotExists()) {
+                allDescendentsDeleted = false;
+            }
+            if (!subResult.getNotDeletedEligibleItems().isEmpty()) {
+                notDeletedEligibleItems.addAll(subResult.getNotDeletedEligibleItems());
+            }
+        }
+
+        if (allDescendentsDeleted && deleteEmptyDir) {
+            Verify.verify(notDeletedEligibleItems.build().isEmpty());
             if (!deleteIfExists(directory)) {
                 return new DeleteRecursivelyResult(false, ImmutableList.of(directory + "/"));
             }
             // all items of the location have been deleted.
             return new DeleteRecursivelyResult(true, ImmutableList.of());
         }
-
-        return new DeleteRecursivelyResult(false, notDeletedEligibleItems);
+        return new DeleteRecursivelyResult(false, notDeletedEligibleItems.build());
     }
 
     public boolean deleteIfExists(Path path) {
-        Status status = fs.delete(path.getName());
+        Status status = fs.delete(path.toString());
         if (status.ok()) {
             return true;
         }
-        return !fs.exists(path.getName()).ok();
+        return !fs.exists(path.toString()).ok();
     }
 
     public static class DatabaseTableName {
@@ -1039,9 +1029,6 @@ public class HMSTransaction implements Transaction {
         }
 
         private void undoAddPartitionsTask() {
-            if (addPartitionsTask.isEmpty()) {
-                return;
-            }
 
             HivePartition firstPartition = addPartitionsTask.getPartitions().get(0).getPartition();
             String dbName = firstPartition.getDbName();
@@ -1304,10 +1291,16 @@ public class HMSTransaction implements Transaction {
             }
         }
 
+        public void doNothing() {
+            // do nothing
+            // only for regression test and unit test to throw exception
+        }
+
         public void doCommit() {
             waitForAsyncFileSystemTasks();
             doAddPartitionsTask();
             doUpdateStatisticsTasks();
+            doNothing();
         }
 
         public void rollback() {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
index f3556d13a57..5cf362508e3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java
@@ -34,7 +34,7 @@ import org.apache.doris.datasource.ExternalDatabase;
 import org.apache.doris.datasource.jdbc.client.JdbcClient;
 import org.apache.doris.datasource.jdbc.client.JdbcClientConfig;
 import org.apache.doris.datasource.operations.ExternalMetadataOps;
-import org.apache.doris.fs.remote.RemoteFileSystem;
+import org.apache.doris.fs.FileSystem;
 import org.apache.doris.fs.remote.dfs.DFSFileSystem;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -58,7 +58,7 @@ public class HiveMetadataOps implements ExternalMetadataOps {
     private static final Logger LOG = LogManager.getLogger(HiveMetadataOps.class);
     private static final int MIN_CLIENT_POOL_SIZE = 8;
     private final HMSCachedClient client;
-    private final RemoteFileSystem fs;
+    private final FileSystem fs;
     private final HMSExternalCatalog catalog;
 
     public HiveMetadataOps(HiveConf hiveConf, JdbcClientConfig jdbcClientConfig, HMSExternalCatalog catalog) {
@@ -75,11 +75,19 @@ public class HiveMetadataOps implements ExternalMetadataOps {
         this.fs = new DFSFileSystem(catalog.getProperties());
     }
 
+    // for test
+    public HiveMetadataOps(HMSExternalCatalog catalog, HMSCachedClient client, FileSystem fs) {
+        this.catalog = catalog;
+        this.client = client;
+        this.fs = fs;
+    }
+
+
     public HMSCachedClient getClient() {
         return client;
     }
 
-    public RemoteFileSystem getFs() {
+    public FileSystem getFs() {
         return fs;
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
index 0dc7eb5a386..dfbfe786985 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
@@ -25,10 +25,12 @@ import org.apache.doris.fs.remote.BrokerFileSystem;
 import org.apache.doris.fs.remote.RemoteFileSystem;
 
 import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Partition;
@@ -167,7 +169,9 @@ public final class HiveUtil {
         ImmutableMap.Builder<String, Partition> resultBuilder = ImmutableMap.builder();
         for (Map.Entry<String, List<String>> entry : partitionNameToPartitionValues.entrySet()) {
             Partition partition = partitionValuesToPartition.get(entry.getValue());
-            resultBuilder.put(entry.getKey(), partition);
+            if (partition != null) {
+                resultBuilder.put(entry.getKey(), partition);
+            }
         }
         return resultBuilder.build();
     }
@@ -267,4 +271,63 @@ public final class HiveUtil {
         database.setDescription(hiveDb.getComment());
         return database;
     }
+
+    public static Map<String, String> updateStatisticsParameters(
+            Map<String, String> parameters,
+            HiveCommonStatistics statistics) {
+        HashMap<String, String> result = new HashMap<>(parameters);
+
+        result.put(StatsSetupConst.NUM_FILES, String.valueOf(statistics.getFileCount()));
+        result.put(StatsSetupConst.ROW_COUNT, String.valueOf(statistics.getRowCount()));
+        result.put(StatsSetupConst.TOTAL_SIZE, String.valueOf(statistics.getTotalFileBytes()));
+
+        // CDH 5.16 metastore ignores stats unless STATS_GENERATED_VIA_STATS_TASK is set
+        // https://github.com/cloudera/hive/blob/cdh5.16.2-release/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java#L227-L231
+        if (!parameters.containsKey("STATS_GENERATED_VIA_STATS_TASK")) {
+            result.put("STATS_GENERATED_VIA_STATS_TASK", "workaround for potential lack of HIVE-12730");
+        }
+
+        return result;
+    }
+
+    public static HivePartitionStatistics toHivePartitionStatistics(Map<String, String> params) {
+        long rowCount = Long.parseLong(params.getOrDefault(StatsSetupConst.ROW_COUNT, "-1"));
+        long totalSize = Long.parseLong(params.getOrDefault(StatsSetupConst.TOTAL_SIZE, "-1"));
+        long numFiles = Long.parseLong(params.getOrDefault(StatsSetupConst.NUM_FILES, "-1"));
+        return HivePartitionStatistics.fromCommonStatistics(rowCount, numFiles, totalSize);
+    }
+
+    public static Partition toMetastoreApiPartition(HivePartitionWithStatistics partitionWithStatistics) {
+        Partition partition =
+                toMetastoreApiPartition(partitionWithStatistics.getPartition());
+        partition.setParameters(updateStatisticsParameters(
+                partition.getParameters(), partitionWithStatistics.getStatistics().getCommonStatistics()));
+        return partition;
+    }
+
+    public static Partition toMetastoreApiPartition(HivePartition hivePartition) {
+        Partition result = new Partition();
+        result.setDbName(hivePartition.getDbName());
+        result.setTableName(hivePartition.getTblName());
+        result.setValues(hivePartition.getPartitionValues());
+        result.setSd(makeStorageDescriptorFromHivePartition(hivePartition));
+        result.setParameters(hivePartition.getParameters());
+        return result;
+    }
+
+    public static StorageDescriptor makeStorageDescriptorFromHivePartition(HivePartition partition) {
+        SerDeInfo serdeInfo = new SerDeInfo();
+        serdeInfo.setName(partition.getTblName());
+        serdeInfo.setSerializationLib(partition.getSerde());
+
+        StorageDescriptor sd = new StorageDescriptor();
+        sd.setLocation(Strings.emptyToNull(partition.getPath()));
+        sd.setCols(partition.getColumns());
+        sd.setSerdeInfo(serdeInfo);
+        sd.setInputFormat(partition.getInputFormat());
+        sd.setOutputFormat(partition.getOutputFormat());
+        sd.setParameters(ImmutableMap.of());
+
+        return sd;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
index 00fc0d03fa8..9fae854645b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/ThriftHMSCachedClient.java
@@ -27,11 +27,8 @@ import org.apache.doris.datasource.property.constants.HMSProperties;
 import com.aliyun.datalake.metastore.hive2.ProxyMetaStoreClient;
 import com.amazonaws.glue.catalog.metastore.AWSCatalogMetastoreClient;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
-import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
@@ -56,8 +53,6 @@ import org.apache.hadoop.hive.metastore.api.LockState;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
 import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.SerDeInfo;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
@@ -67,7 +62,6 @@ import org.apache.logging.log4j.Logger;
 import java.security.PrivilegedExceptionAction;
 import java.util.BitSet;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -605,11 +599,11 @@ public class ThriftHMSCachedClient implements HMSCachedClient {
 
             Table originTable = getTable(dbName, tableName);
             Map<String, String> originParams = originTable.getParameters();
-            HivePartitionStatistics updatedStats = update.apply(toHivePartitionStatistics(originParams));
+            HivePartitionStatistics updatedStats = update.apply(HiveUtil.toHivePartitionStatistics(originParams));
 
             Table newTable = originTable.deepCopy();
             Map<String, String> newParams =
-                    updateStatisticsParameters(originParams, updatedStats.getCommonStatistics());
+                    HiveUtil.updateStatisticsParameters(originParams, updatedStats.getCommonStatistics());
             newParams.put("transient_lastDdlTime", String.valueOf(System.currentTimeMillis() / 1000));
             newTable.setParameters(newParams);
             client.client.alter_table(dbName, tableName, newTable);
@@ -633,11 +627,11 @@ public class ThriftHMSCachedClient implements HMSCachedClient {
 
             Partition originPartition = partitions.get(0);
             Map<String, String> originParams = originPartition.getParameters();
-            HivePartitionStatistics updatedStats = update.apply(toHivePartitionStatistics(originParams));
+            HivePartitionStatistics updatedStats = update.apply(HiveUtil.toHivePartitionStatistics(originParams));
 
             Partition modifiedPartition = originPartition.deepCopy();
             Map<String, String> newParams =
-                    updateStatisticsParameters(originParams, updatedStats.getCommonStatistics());
+                    HiveUtil.updateStatisticsParameters(originParams, updatedStats.getCommonStatistics());
             newParams.put("transient_lastDdlTime", String.valueOf(System.currentTimeMillis() / 1000));
             modifiedPartition.setParameters(newParams);
             client.client.alter_partition(dbName, tableName, modifiedPartition);
@@ -650,7 +644,7 @@ public class ThriftHMSCachedClient implements HMSCachedClient {
     public void addPartitions(String dbName, String tableName, List<HivePartitionWithStatistics> partitions) {
         try (ThriftHMSClient client = getClient()) {
             List<Partition> hivePartitions = partitions.stream()
-                    .map(ThriftHMSCachedClient::toMetastoreApiPartition)
+                    .map(HiveUtil::toMetastoreApiPartition)
                     .collect(Collectors.toList());
             client.client.add_partitions(hivePartitions);
         } catch (Exception e) {
@@ -666,64 +660,4 @@ public class ThriftHMSCachedClient implements HMSCachedClient {
             throw new RuntimeException("failed to drop partition for " + dbName + "." + tableName);
         }
     }
-
-    private static HivePartitionStatistics toHivePartitionStatistics(Map<String, String> params) {
-        long rowCount = Long.parseLong(params.getOrDefault(StatsSetupConst.ROW_COUNT, "-1"));
-        long totalSize = Long.parseLong(params.getOrDefault(StatsSetupConst.TOTAL_SIZE, "-1"));
-        long numFiles = Long.parseLong(params.getOrDefault(StatsSetupConst.NUM_FILES, "-1"));
-        return HivePartitionStatistics.fromCommonStatistics(rowCount, numFiles, totalSize);
-    }
-
-    private static Map<String, String> updateStatisticsParameters(
-            Map<String, String> parameters,
-            HiveCommonStatistics statistics) {
-        HashMap<String, String> result = new HashMap<>(parameters);
-
-        result.put(StatsSetupConst.NUM_FILES, String.valueOf(statistics.getFileCount()));
-        result.put(StatsSetupConst.ROW_COUNT, String.valueOf(statistics.getRowCount()));
-        result.put(StatsSetupConst.TOTAL_SIZE, String.valueOf(statistics.getTotalFileBytes()));
-
-        // CDH 5.16 metastore ignores stats unless STATS_GENERATED_VIA_STATS_TASK is set
-        // https://github.com/cloudera/hive/blob/cdh5.16.2-release/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java#L227-L231
-        if (!parameters.containsKey("STATS_GENERATED_VIA_STATS_TASK")) {
-            result.put("STATS_GENERATED_VIA_STATS_TASK", "workaround for potential lack of HIVE-12730");
-        }
-
-        return result;
-    }
-
-    public static Partition toMetastoreApiPartition(HivePartitionWithStatistics partitionWithStatistics) {
-        Partition partition =
-                toMetastoreApiPartition(partitionWithStatistics.getPartition());
-        partition.setParameters(updateStatisticsParameters(
-                partition.getParameters(), partitionWithStatistics.getStatistics().getCommonStatistics()));
-        return partition;
-    }
-
-    public static Partition toMetastoreApiPartition(HivePartition hivePartition) {
-        Partition result = new Partition();
-        result.setDbName(hivePartition.getDbName());
-        result.setTableName(hivePartition.getTblName());
-        result.setValues(hivePartition.getPartitionValues());
-        result.setSd(makeStorageDescriptorFromHivePartition(hivePartition));
-        result.setParameters(hivePartition.getParameters());
-        return result;
-    }
-
-    private static StorageDescriptor makeStorageDescriptorFromHivePartition(HivePartition partition) {
-        SerDeInfo serdeInfo = new SerDeInfo();
-        serdeInfo.setName(partition.getTblName());
-        serdeInfo.setSerializationLib(partition.getSerde());
-
-        StorageDescriptor sd = new StorageDescriptor();
-        sd.setLocation(Strings.emptyToNull(partition.getPath()));
-        sd.setCols(partition.getColumns());
-        sd.setSerdeInfo(serdeInfo);
-        sd.setInputFormat(partition.getInputFormat());
-        sd.setOutputFormat(partition.getOutputFormat());
-        sd.setParameters(ImmutableMap.of());
-
-        return sd;
-    }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java
index 0470d8b3714..369fc917d77 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java
@@ -23,6 +23,7 @@ import org.apache.doris.fs.remote.RemoteFile;
 
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -86,4 +87,12 @@ public interface FileSystem {
     }
 
     Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly);
+
+    default Status listFiles(String remotePath, List<RemoteFile> result) {
+        throw new UnsupportedOperationException("Unsupported operation list files on current file system.");
+    }
+
+    default Status listDirectories(String remotePath, Set<String> result) {
+        throw new UnsupportedOperationException("Unsupported operation list directores on current file system.");
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/LocalDfsFileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fs/LocalDfsFileSystem.java
new file mode 100644
index 00000000000..0faf1916db0
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/fs/LocalDfsFileSystem.java
@@ -0,0 +1,245 @@
+// 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.fs;
+
+import org.apache.doris.backup.Status;
+import org.apache.doris.common.UserException;
+import org.apache.doris.fs.remote.RemoteFile;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class LocalDfsFileSystem implements FileSystem {
+
+    public LocalFileSystem fs = LocalFileSystem.getLocal(new Configuration());
+
+    public LocalDfsFileSystem() throws IOException {
+    }
+
+    @Override
+    public Map<String, String> getProperties() {
+        return null;
+    }
+
+    @Override
+    public Status exists(String remotePath) {
+        boolean exists = false;
+        try {
+            exists = fs.exists(new Path(remotePath));
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        if (exists) {
+            return Status.OK;
+        } else {
+            return new Status(Status.ErrCode.NOT_FOUND, "");
+        }
+    }
+
+    @Override
+    public Status downloadWithFileSize(String remoteFilePath, String localFilePath, long fileSize) {
+        return null;
+    }
+
+    @Override
+    public Status upload(String localPath, String remotePath) {
+        return null;
+    }
+
+    @Override
+    public Status directUpload(String content, String remoteFile) {
+        return null;
+    }
+
+    @Override
+    public Status rename(String origFilePath, String destFilePath) {
+        try {
+            fs.rename(new Path(origFilePath), new Path(destFilePath));
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return Status.OK;
+    }
+
+    @Override
+    public Status renameDir(String origFilePath, String destFilePath, Runnable runWhenPathNotExist) {
+        Status status = exists(destFilePath);
+        if (status.ok()) {
+            throw new RuntimeException("Destination directory already exists: " + destFilePath);
+        }
+        String targetParent = new Path(destFilePath).getParent().toString();
+        status = exists(targetParent);
+        if (Status.ErrCode.NOT_FOUND.equals(status.getErrCode())) {
+            status = makeDir(targetParent);
+        }
+        if (!status.ok()) {
+            throw new RuntimeException(status.getErrMsg());
+        }
+
+        runWhenPathNotExist.run();
+
+        return rename(origFilePath, destFilePath);
+    }
+
+    @Override
+    public void asyncRename(Executor executor,
+                            List<CompletableFuture<?>> renameFileFutures,
+                            AtomicBoolean cancelled,
+                            String origFilePath,
+                            String destFilePath,
+                            List<String> fileNames)  {
+        for (String fileName : fileNames) {
+            Path source = new Path(origFilePath, fileName);
+            Path target = new Path(destFilePath, fileName);
+            renameFileFutures.add(CompletableFuture.runAsync(() -> {
+                if (cancelled.get()) {
+                    return;
+                }
+                Status status = rename(source.toString(), target.toString());
+                if (!status.ok()) {
+                    throw new RuntimeException(status.getErrMsg());
+                }
+            }, executor));
+        }
+    }
+
+    @Override
+    public void asyncRenameDir(Executor executor,
+                               List<CompletableFuture<?>> renameFileFutures,
+                               AtomicBoolean cancelled,
+                               String origFilePath,
+                               String destFilePath,
+                               Runnable runWhenPathNotExist) {
+        renameFileFutures.add(CompletableFuture.runAsync(() -> {
+            if (cancelled.get()) {
+                return;
+            }
+            Status status = renameDir(origFilePath, destFilePath, runWhenPathNotExist);
+            if (!status.ok()) {
+                throw new RuntimeException(status.getErrMsg());
+            }
+        }, executor));
+    }
+
+    @Override
+    public Status delete(String remotePath) {
+        try {
+            fs.delete(new Path(remotePath), true);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return Status.OK;
+    }
+
+    @Override
+    public Status makeDir(String remotePath) {
+        try {
+            fs.mkdirs(new Path(remotePath));
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return Status.OK;
+    }
+
+    @Override
+    public RemoteFiles listLocatedFiles(String remotePath, boolean onlyFiles, boolean recursive) throws UserException {
+        return null;
+    }
+
+    @Override
+    public Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly) {
+        try {
+            FileStatus[] locatedFileStatusRemoteIterator = fs.listStatus(new Path(remotePath));
+            if (locatedFileStatusRemoteIterator == null) {
+                return Status.OK;
+            }
+            for (FileStatus fileStatus : locatedFileStatusRemoteIterator) {
+                RemoteFile remoteFile = new RemoteFile(
+                        fileNameOnly ? fileStatus.getPath().getName() : fileStatus.getPath().toString(),
+                        !fileStatus.isDirectory(), fileStatus.isDirectory() ? -1 : fileStatus.getLen(),
+                        fileStatus.getBlockSize(), fileStatus.getModificationTime());
+                result.add(remoteFile);
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return Status.OK;
+    }
+
+    @Override
+    public Status listFiles(String remotePath, List<RemoteFile> result) {
+        RemoteIterator<LocatedFileStatus> iterator;
+        try {
+            Path dirPath = new Path(remotePath);
+            iterator = fs.listFiles(dirPath, true);
+            while (iterator.hasNext()) {
+                LocatedFileStatus next = iterator.next();
+                String location = next.getPath().toString();
+                String child = location.substring(dirPath.toString().length());
+                while (child.startsWith("/")) {
+                    child = child.substring(1);
+                }
+                if (!child.contains("/")) {
+                    result.add(new RemoteFile(location, next.isFile(), next.getLen(), next.getBlockSize()));
+                }
+            }
+        } catch (IOException e) {
+            return new Status(Status.ErrCode.COMMON_ERROR, e.getMessage());
+        }
+        return Status.OK;
+    }
+
+    @Override
+    public Status listDirectories(String remotePath, Set<String> result) {
+        try {
+            FileStatus[] fileStatuses = fs.listStatus(new Path(remotePath));
+            result.addAll(
+                    Arrays.stream(fileStatuses)
+                        .filter(FileStatus::isDirectory)
+                        .map(file -> file.getPath().toString() + "/")
+                        .collect(ImmutableSet.toImmutableSet()));
+        } catch (IOException e) {
+            return new Status(Status.ErrCode.COMMON_ERROR, e.getMessage());
+        }
+        return Status.OK;
+    }
+
+    public void createFile(String path) throws IOException {
+        Path path1 = new Path(path);
+        if (!exists(path1.getParent().toString()).ok()) {
+            makeDir(path1.getParent().toString());
+        }
+        FSDataOutputStream build = fs.createFile(path1).build();
+        build.close();
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/LocalFileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fs/LocalFileSystem.java
deleted file mode 100644
index 1baaf9bd2f7..00000000000
--- a/fe/fe-core/src/main/java/org/apache/doris/fs/LocalFileSystem.java
+++ /dev/null
@@ -1,76 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-package org.apache.doris.fs;
-
-import org.apache.doris.backup.Status;
-import org.apache.doris.fs.remote.RemoteFile;
-
-import java.util.List;
-import java.util.Map;
-
-public class LocalFileSystem implements FileSystem {
-    @Override
-    public Status exists(String remotePath) {
-        throw new UnsupportedOperationException("Unsupported operation on local file system.");
-    }
-
-    @Override
-    public Status downloadWithFileSize(String remoteFilePath, String localFilePath, long fileSize) {
-        throw new UnsupportedOperationException("Unsupported operation on local file system.");
-    }
-
-    @Override
-    public Status upload(String localPath, String remotePath) {
-        throw new UnsupportedOperationException("Unsupported operation on local file system.");
-    }
-
-    @Override
-    public Status directUpload(String content, String remoteFile) {
-        throw new UnsupportedOperationException("Unsupported operation on local file system.");
-    }
-
-    @Override
-    public Status rename(String origFilePath, String destFilePath) {
-        throw new UnsupportedOperationException("Unsupported operation on local file system.");
-    }
-
-    @Override
-    public Status delete(String remotePath) {
-        throw new UnsupportedOperationException("Unsupported operation on local file system.");
-    }
-
-    @Override
-    public Status makeDir(String remotePath) {
-        throw new UnsupportedOperationException("Unsupported operation on local file system.");
-    }
-
-    @Override
-    public RemoteFiles listLocatedFiles(String remotePath, boolean onlyFiles, boolean recursive) {
-        throw new UnsupportedOperationException("Unsupported operation on local file system.");
-    }
-
-    @Override
-    public Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly) {
-        throw new UnsupportedOperationException("Unsupported operation on local file system.");
-    }
-
-    @Override
-    public Map<String, String> getProperties() {
-        throw new UnsupportedOperationException("Unsupported operation on local file system.");
-    }
-}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/dfs/DFSFileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/dfs/DFSFileSystem.java
index fabc341389e..7e3032ca807 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/dfs/DFSFileSystem.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/dfs/DFSFileSystem.java
@@ -31,12 +31,15 @@ import org.apache.doris.fs.remote.RemoteFileSystem;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -51,9 +54,11 @@ import java.nio.ByteBuffer;
 import java.nio.file.FileVisitOption;
 import java.nio.file.Files;
 import java.nio.file.Paths;
+import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -523,4 +528,44 @@ public class DFSFileSystem extends RemoteFileSystem {
         }
         return Status.OK;
     }
+
+    @Override
+    public Status listFiles(String remotePath, List<RemoteFile> result) {
+        RemoteIterator<LocatedFileStatus> iterator;
+        try {
+            FileSystem fileSystem = nativeFileSystem(remotePath);
+            Path dirPath = new Path(remotePath);
+            iterator = fileSystem.listFiles(dirPath, true);
+            while (iterator.hasNext()) {
+                LocatedFileStatus next = iterator.next();
+                String location = next.getPath().toString();
+                String child = location.substring(dirPath.toString().length());
+                while (child.startsWith("/")) {
+                    child = child.substring(1);
+                }
+                if (!child.contains("/")) {
+                    result.add(new RemoteFile(location, next.isFile(), next.getLen(), next.getBlockSize()));
+                }
+            }
+        } catch (Exception e) {
+            return new Status(Status.ErrCode.COMMON_ERROR, e.getMessage());
+        }
+        return Status.OK;
+    }
+
+    @Override
+    public Status listDirectories(String remotePath, Set<String> result) {
+        try {
+            FileSystem fileSystem = nativeFileSystem(remotePath);
+            FileStatus[] fileStatuses = fileSystem.listStatus(new Path(remotePath));
+            result.addAll(
+                    Arrays.stream(fileStatuses)
+                        .filter(FileStatus::isDirectory)
+                        .map(file -> file.getPath().toString() + "/")
+                        .collect(ImmutableSet.toImmutableSet()));
+        } catch (Exception e) {
+            return new Status(Status.ErrCode.COMMON_ERROR, e.getMessage());
+        }
+        return Status.OK;
+    }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/HMSCachedClientTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/HMSCachedClientTest.java
new file mode 100644
index 00000000000..126df780dbf
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/HMSCachedClientTest.java
@@ -0,0 +1,328 @@
+// 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;
+
+import org.apache.doris.analysis.TableName;
+import org.apache.doris.datasource.hive.HMSCachedClient;
+import org.apache.doris.datasource.hive.HMSTransaction;
+import org.apache.doris.datasource.hive.HiveDatabaseMetadata;
+import org.apache.doris.datasource.hive.HivePartitionStatistics;
+import org.apache.doris.datasource.hive.HivePartitionWithStatistics;
+import org.apache.doris.datasource.hive.HiveTableMetadata;
+import org.apache.doris.datasource.hive.HiveUtil;
+import org.apache.doris.datasource.hive.event.MetastoreNotificationFetchException;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hive.common.ValidWriteIdList;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class HMSCachedClientTest implements HMSCachedClient {
+
+    public Map<HMSTransaction.DatabaseTableName, List<Partition>> partitions = new ConcurrentHashMap<>();
+    public Map<String, List<Table>> tables = new HashMap<>();
+    public List<Database> dbs = new ArrayList<>();
+
+    @Override
+    public Database getDatabase(String dbName) {
+        for (Database db : this.dbs) {
+            if (db.getName().equals(dbName)) {
+                return db;
+            }
+        }
+        throw new RuntimeException("can't found database: " + dbName);
+    }
+
+    @Override
+    public List<String> getAllDatabases() {
+        return null;
+    }
+
+    @Override
+    public List<String> getAllTables(String dbName) {
+        return null;
+    }
+
+    @Override
+    public boolean tableExists(String dbName, String tblName) {
+        List<Table> tablesList = getTableList(dbName);
+        for (Table table : tablesList) {
+            if (table.getTableName().equals(tblName)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public List<String> listPartitionNames(String dbName, String tblName) {
+        List<Partition> partitionList = getPartitionList(dbName, tblName);
+        ArrayList<String> ret = new ArrayList<>();
+        for (Partition partition : partitionList) {
+            StringBuilder names = new StringBuilder();
+            List<String> values = partition.getValues();
+            for (int i = 0; i < values.size(); i++) {
+                names.append(values.get(i));
+                if (i < values.size() - 1) {
+                    names.append("/");
+                }
+            }
+            ret.add(names.toString());
+        }
+        return ret;
+    }
+
+    @Override
+    public List<Partition> listPartitions(String dbName, String tblName) {
+        return getPartitionList(dbName, tblName);
+    }
+
+    @Override
+    public List<String> listPartitionNames(String dbName, String tblName, long maxListPartitionNum) {
+        return listPartitionNames(dbName, tblName);
+    }
+
+    @Override
+    public Partition getPartition(String dbName, String tblName, List<String> partitionValues) {
+        synchronized (this) {
+            List<Partition> partitionList = getPartitionList(dbName, tblName);
+            for (Partition partition : partitionList) {
+                if (partition.getValues().equals(partitionValues)) {
+                    return partition;
+                }
+            }
+            throw new RuntimeException("can't found partition");
+        }
+    }
+
+    @Override
+    public List<Partition> getPartitions(String dbName, String tblName, List<String> partitionNames) {
+        synchronized (this) {
+            List<Partition> partitionList = getPartitionList(dbName, tblName);
+            ArrayList<Partition> ret = new ArrayList<>();
+            List<List<String>> partitionValuesList =
+                    partitionNames
+                            .stream()
+                            .map(HiveUtil::toPartitionValues)
+                            .collect(Collectors.toList());
+            partitionValuesList.forEach(values -> {
+                for (Partition partition : partitionList) {
+                    if (partition.getValues().equals(values)) {
+                        ret.add(partition);
+                        break;
+                    }
+                }
+            });
+            return ret;
+        }
+    }
+
+    @Override
+    public Table getTable(String dbName, String tblName) {
+        List<Table> tablesList = getTableList(dbName);
+        for (Table table : tablesList) {
+            if (table.getTableName().equals(tblName)) {
+                return table;
+            }
+        }
+        throw new RuntimeException("can't found table: " + tblName);
+    }
+
+    @Override
+    public List<FieldSchema> getSchema(String dbName, String tblName) {
+        return null;
+    }
+
+    @Override
+    public List<ColumnStatisticsObj> getTableColumnStatistics(String dbName, String tblName, List<String> columns) {
+        return null;
+    }
+
+    @Override
+    public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(String dbName, String tblName, List<String> partNames, List<String> columns) {
+        return null;
+    }
+
+    @Override
+    public CurrentNotificationEventId getCurrentNotificationEventId() {
+        return null;
+    }
+
+    @Override
+    public NotificationEventResponse getNextNotification(long lastEventId, int maxEvents, IMetaStoreClient.NotificationFilter filter) throws MetastoreNotificationFetchException {
+        return null;
+    }
+
+    @Override
+    public long openTxn(String user) {
+        return 0;
+    }
+
+    @Override
+    public void commitTxn(long txnId) {
+
+    }
+
+    @Override
+    public ValidWriteIdList getValidWriteIds(String fullTableName, long currentTransactionId) {
+        return null;
+    }
+
+    @Override
+    public void acquireSharedLock(String queryId, long txnId, String user, TableName tblName, List<String> partitionNames, long timeoutMs) {
+
+    }
+
+    @Override
+    public String getCatalogLocation(String catalogName) {
+        return null;
+    }
+
+    @Override
+    public void createDatabase(DatabaseMetadata db) {
+        dbs.add(HiveUtil.toHiveDatabase((HiveDatabaseMetadata) db));
+        tables.put(db.getDbName(), new ArrayList<>());
+    }
+
+    @Override
+    public void dropDatabase(String dbName) {
+        Database db = getDatabase(dbName);
+        this.dbs.remove(db);
+    }
+
+    @Override
+    public void dropTable(String dbName, String tableName) {
+        Table table = getTable(dbName, tableName);
+        this.tables.get(dbName).remove(table);
+        this.partitions.remove(new HMSTransaction.DatabaseTableName(dbName, tableName));
+    }
+
+    @Override
+    public void createTable(TableMetadata tbl, boolean ignoreIfExists) {
+        String dbName = tbl.getDbName();
+        String tbName = tbl.getTableName();
+        if (tableExists(dbName, tbName)) {
+            throw new RuntimeException("Table '" + tbName + "' has existed in '" + dbName + "'.");
+        }
+
+        List<Table> tableList = getTableList(tbl.getDbName());
+        tableList.add(HiveUtil.toHiveTable((HiveTableMetadata) tbl));
+        HMSTransaction.DatabaseTableName key = new HMSTransaction.DatabaseTableName(dbName, tbName);
+        partitions.put(key, new ArrayList<>());
+    }
+
+    @Override
+    public void updateTableStatistics(String dbName, String tableName, Function<HivePartitionStatistics, HivePartitionStatistics> update) {
+        synchronized (this) {
+            Table originTable = getTable(dbName, tableName);
+            Map<String, String> originParams = originTable.getParameters();
+            HivePartitionStatistics updatedStats = update.apply(HiveUtil.toHivePartitionStatistics(originParams));
+
+            Table newTable = originTable.deepCopy();
+            Map<String, String> newParams =
+                    HiveUtil.updateStatisticsParameters(originParams, updatedStats.getCommonStatistics());
+            newParams.put("transient_lastDdlTime", String.valueOf(System.currentTimeMillis() / 1000));
+            newTable.setParameters(newParams);
+            List<Table> tableList = getTableList(dbName);
+            tableList.remove(originTable);
+            tableList.add(newTable);
+        }
+    }
+
+    @Override
+    public void updatePartitionStatistics(String dbName, String tableName, String partitionName, Function<HivePartitionStatistics, HivePartitionStatistics> update) {
+
+        synchronized (this) {
+            List<Partition> partitions = getPartitions(dbName, tableName, ImmutableList.of(partitionName));
+            if (partitions.size() != 1) {
+                throw new RuntimeException("Metastore returned multiple partitions for name: " + partitionName);
+            }
+
+            Partition originPartition = partitions.get(0);
+            Map<String, String> originParams = originPartition.getParameters();
+            HivePartitionStatistics updatedStats = update.apply(HiveUtil.toHivePartitionStatistics(originParams));
+
+            Partition modifiedPartition = originPartition.deepCopy();
+            Map<String, String> newParams =
+                    HiveUtil.updateStatisticsParameters(originParams, updatedStats.getCommonStatistics());
+            newParams.put("transient_lastDdlTime", String.valueOf(System.currentTimeMillis() / 1000));
+            modifiedPartition.setParameters(newParams);
+
+            List<Partition> partitionList = getPartitionList(dbName, tableName);
+            partitionList.remove(originPartition);
+            partitionList.add(modifiedPartition);
+        }
+    }
+
+    @Override
+    public void addPartitions(String dbName, String tableName, List<HivePartitionWithStatistics> partitions) {
+        synchronized (this) {
+            List<Partition> partitionList = getPartitionList(dbName, tableName);
+            List<Partition> hivePartitions = partitions.stream()
+                    .map(HiveUtil::toMetastoreApiPartition)
+                    .collect(Collectors.toList());
+            partitionList.addAll(hivePartitions);
+        }
+    }
+
+    @Override
+    public void dropPartition(String dbName, String tableName, List<String> partitionValues, boolean deleteData) {
+        synchronized (this) {
+            List<Partition> partitionList = getPartitionList(dbName, tableName);
+            for (int j = 0; j < partitionList.size(); j++) {
+                Partition partition = partitionList.get(j);
+                if (partition.getValues().equals(partitionValues)) {
+                    partitionList.remove(partition);
+                    return;
+                }
+            }
+            throw new RuntimeException("can't found the partition");
+        }
+    }
+
+    public List<Partition> getPartitionList(String dbName, String tableName) {
+        HMSTransaction.DatabaseTableName key = new HMSTransaction.DatabaseTableName(dbName, tableName);
+        List<Partition> partitionList = this.partitions.get(key);
+        if (partitionList == null) {
+            throw new RuntimeException("can't found table: " + key);
+        }
+        return partitionList;
+    }
+
+    public List<Table> getTableList(String dbName) {
+        List<Table> tablesList = this.tables.get(dbName);
+        if (tablesList == null) {
+            throw new RuntimeException("can't found database: " + dbName);
+        }
+        return tablesList;
+    }
+}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
index fc939625ea9..4ec6ca84c52 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java
@@ -17,10 +17,10 @@
 
 package org.apache.doris.datasource.hive;
 
-import org.apache.doris.backup.Status;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.PrimitiveType;
-import org.apache.doris.fs.remote.dfs.DFSFileSystem;
+import org.apache.doris.datasource.HMSCachedClientTest;
+import org.apache.doris.fs.LocalDfsFileSystem;
 import org.apache.doris.thrift.THiveLocationParams;
 import org.apache.doris.thrift.THivePartitionUpdate;
 import org.apache.doris.thrift.TUpdateMode;
@@ -28,6 +28,7 @@ import org.apache.doris.thrift.TUpdateMode;
 import com.google.common.collect.Lists;
 import mockit.Mock;
 import mockit.MockUp;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.junit.After;
@@ -35,59 +36,57 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
+import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.concurrent.atomic.AtomicBoolean;
 
-@Ignore
 public class HmsCommitTest {
 
-    private static HMSExternalCatalog hmsCatalog;
     private static HiveMetadataOps hmsOps;
     private static HMSCachedClient hmsClient;
     private static final String dbName = "test_db";
     private static final String tbWithPartition = "test_tb_with_partition";
     private static final String tbWithoutPartition = "test_tb_without_partition";
-    private static Path warehousePath;
+    private static LocalDfsFileSystem fs;
     static String dbLocation;
-    private String fileFormat = "orc";
+    static String writeLocation;
+    static String uri = "thrift://127.0.0.1:9083";
+    static boolean hasRealHmsService = false;
 
     @BeforeClass
     public static void beforeClass() throws Throwable {
-        warehousePath = Files.createTempDirectory("test_warehouse_");
+        Path warehousePath = Files.createTempDirectory("test_warehouse_");
+        Path writePath = Files.createTempDirectory("test_write_");
         dbLocation = "file://" + warehousePath.toAbsolutePath() + "/";
+        writeLocation = "file://" + writePath.toAbsolutePath() + "/";
         createTestHiveCatalog();
         createTestHiveDatabase();
-        mockFs();
     }
 
     @AfterClass
     public static void afterClass() {
-        hmsClient.dropTable(dbName, tbWithPartition);
-        hmsClient.dropTable(dbName, tbWithoutPartition);
         hmsClient.dropDatabase(dbName);
     }
 
-    public static void createTestHiveCatalog() {
-        Map<String, String> props = new HashMap<>();
-        props.put("type", "hms");
-        props.put("hive.metastore.uris", "thrift://127.0.0.1:9083");
-        props.put("hadoop.username", "hadoop");
-        hmsCatalog = new HMSExternalCatalog(1, "hive_catalog", null, props, "comment");
-        hmsCatalog.setInitialized();
-        hmsCatalog.initLocalObjectsImpl();
-        hmsOps = (HiveMetadataOps) hmsCatalog.getMetadataOps();
-        hmsClient = hmsOps.getClient();
+    public static void createTestHiveCatalog() throws IOException {
+        fs = new LocalDfsFileSystem();
+
+        if (hasRealHmsService) {
+            // If you have a real HMS service, then you can use this client to create real connections for testing
+            HiveConf entries = new HiveConf();
+            entries.set("hive.metastore.uris", uri);
+            hmsClient = new ThriftHMSCachedClient(entries, 2);
+        } else {
+            hmsClient = new HMSCachedClientTest();
+        }
+        hmsOps = new HiveMetadataOps(null, hmsClient, fs);
     }
 
     public static void createTestHiveDatabase() {
@@ -98,53 +97,31 @@ public class HmsCommitTest {
         hmsClient.createDatabase(dbMetadata);
     }
 
-    public static void mockFs() {
-
-        new MockUp<DFSFileSystem>(DFSFileSystem.class) {
-            @Mock
-            public void asyncRenameDir(Executor executor,
-                                       List<CompletableFuture<?>> renameFileFutures,
-                                       AtomicBoolean cancelled,
-                                       String origFilePath,
-                                       String destFilePath,
-                                       Runnable runWhenPathNotExist) {
-            }
-
-            @Mock
-            public void asyncRename(Executor executor,
-                                    List<CompletableFuture<?>> renameFileFutures,
-                                    AtomicBoolean cancelled,
-                                    String origFilePath,
-                                    String destFilePath,
-                                    List<String> fileNames) {
-            }
-
-            @Mock
-            public Status renameDir(String origFilePath,
-                                    String destFilePath,
-                                    Runnable runWhenPathNotExist) {
-                return Status.OK;
-            }
-        };
-    }
-
     @Before
     public void before() {
-        // create table
+        // create table for tbWithPartition
         List<Column> columns = new ArrayList<>();
         columns.add(new Column("c1", PrimitiveType.INT, true));
         columns.add(new Column("c2", PrimitiveType.STRING, true));
         columns.add(new Column("c3", PrimitiveType.STRING, false));
         List<String> partitionKeys = new ArrayList<>();
         partitionKeys.add("c3");
+        String fileFormat = "orc";
+        HashMap<String, String> params = new HashMap<String, String>() {{
+                put("location_uri", dbLocation + tbWithPartition);
+            }};
         HiveTableMetadata tableMetadata = new HiveTableMetadata(
                 dbName, tbWithPartition, columns, partitionKeys,
-                new HashMap<>(), fileFormat);
+                params, fileFormat);
         hmsClient.createTable(tableMetadata, true);
 
+        // create table for tbWithoutPartition
+        HashMap<String, String> params2 = new HashMap<String, String>() {{
+                put("location_uri", dbLocation + tbWithPartition);
+            }};
         HiveTableMetadata tableMetadata2 = new HiveTableMetadata(
-                dbName, tbWithoutPartition, columns, new ArrayList<>(),
-                new HashMap<>(), fileFormat);
+                    dbName, tbWithoutPartition, columns, new ArrayList<>(),
+                    params2, fileFormat);
         hmsClient.createTable(tableMetadata2, true);
 
     }
@@ -156,45 +133,45 @@ public class HmsCommitTest {
     }
 
     @Test
-    public void testNewPartitionForUnPartitionedTable() {
+    public void testNewPartitionForUnPartitionedTable() throws IOException {
         List<THivePartitionUpdate> pus = new ArrayList<>();
-        pus.add(createRandomNew("a"));
+        pus.add(createRandomNew(null));
         Assert.assertThrows(Exception.class, () -> commit(dbName, tbWithoutPartition, pus));
     }
 
     @Test
-    public void testAppendPartitionForUnPartitionedTable() {
+    public void testAppendPartitionForUnPartitionedTable() throws IOException {
         List<THivePartitionUpdate> pus = new ArrayList<>();
-        pus.add(createRandomAppend(""));
-        pus.add(createRandomAppend(""));
-        pus.add(createRandomAppend(""));
+        pus.add(createRandomAppend(null));
+        pus.add(createRandomAppend(null));
+        pus.add(createRandomAppend(null));
         commit(dbName, tbWithoutPartition, pus);
         Table table = hmsClient.getTable(dbName, tbWithoutPartition);
         assertNumRows(3, table);
 
         List<THivePartitionUpdate> pus2 = new ArrayList<>();
-        pus2.add(createRandomAppend(""));
-        pus2.add(createRandomAppend(""));
-        pus2.add(createRandomAppend(""));
+        pus2.add(createRandomAppend(null));
+        pus2.add(createRandomAppend(null));
+        pus2.add(createRandomAppend(null));
         commit(dbName, tbWithoutPartition, pus2);
         table = hmsClient.getTable(dbName, tbWithoutPartition);
         assertNumRows(6, table);
     }
 
     @Test
-    public void testOverwritePartitionForUnPartitionedTable() {
+    public void testOverwritePartitionForUnPartitionedTable() throws IOException {
         testAppendPartitionForUnPartitionedTable();
         List<THivePartitionUpdate> pus = new ArrayList<>();
-        pus.add(createRandomOverwrite(""));
-        pus.add(createRandomOverwrite(""));
-        pus.add(createRandomOverwrite(""));
+        pus.add(createRandomOverwrite(null));
+        pus.add(createRandomOverwrite(null));
+        pus.add(createRandomOverwrite(null));
         commit(dbName, tbWithoutPartition, pus);
         Table table = hmsClient.getTable(dbName, tbWithoutPartition);
         assertNumRows(3, table);
     }
 
     @Test
-    public void testNewPartitionForPartitionedTable() {
+    public void testNewPartitionForPartitionedTable() throws IOException {
         List<THivePartitionUpdate> pus = new ArrayList<>();
         pus.add(createRandomNew("a"));
         pus.add(createRandomNew("a"));
@@ -213,7 +190,7 @@ public class HmsCommitTest {
     }
 
     @Test
-    public void testAppendPartitionForPartitionedTable() {
+    public void testAppendPartitionForPartitionedTable() throws IOException {
         testNewPartitionForPartitionedTable();
 
         List<THivePartitionUpdate> pus = new ArrayList<>();
@@ -234,7 +211,7 @@ public class HmsCommitTest {
     }
 
     @Test
-    public void testOverwritePartitionForPartitionedTable() {
+    public void testOverwritePartitionForPartitionedTable() throws IOException {
         testAppendPartitionForPartitionedTable();
         List<THivePartitionUpdate> pus = new ArrayList<>();
         pus.add(createRandomOverwrite("a"));
@@ -251,7 +228,7 @@ public class HmsCommitTest {
     }
 
     @Test
-    public void testNewManyPartitionForPartitionedTable() {
+    public void testNewManyPartitionForPartitionedTable() throws IOException {
         List<THivePartitionUpdate> pus = new ArrayList<>();
         int nums = 150;
         for (int i = 0; i < nums; i++) {
@@ -265,12 +242,30 @@ public class HmsCommitTest {
         }
 
         try {
-            commit(dbName, tbWithPartition, pus);
+            commit(dbName, tbWithPartition, Collections.singletonList(createRandomNew("1")));
         } catch (Exception e) {
             Assert.assertTrue(e.getMessage().contains("failed to add partitions"));
         }
     }
 
+    @Test
+    public void testErrorPartitionTypeFromHmsCheck() throws IOException {
+        // first add three partition: a,b,c
+        testNewPartitionForPartitionedTable();
+
+        // second append two partition: a,x
+        // but there is no 'x' partition in the previous table, so when verifying based on HMS,
+        // it will throw exception
+        List<THivePartitionUpdate> pus = new ArrayList<>();
+        pus.add(createRandomAppend("a"));
+        pus.add(createRandomAppend("x"));
+
+        Assert.assertThrows(
+                Exception.class,
+                () -> commit(dbName, tbWithPartition, pus)
+        );
+    }
+
     public void assertNumRows(long expected, Partition p) {
         Assert.assertEquals(expected, Long.parseLong(p.getParameters().get("numRows")));
     }
@@ -279,40 +274,62 @@ public class HmsCommitTest {
         Assert.assertEquals(expected, Long.parseLong(t.getParameters().get("numRows")));
     }
 
-    public THivePartitionUpdate genOnePartitionUpdate(String partitionValue, TUpdateMode mode) {
+    public THivePartitionUpdate genOnePartitionUpdate(TUpdateMode mode) throws IOException {
+        return genOnePartitionUpdate("", mode);
+    }
+
+    public THivePartitionUpdate genOnePartitionUpdate(String partitionValue, TUpdateMode mode) throws IOException {
 
         String uuid = UUID.randomUUID().toString();
         THiveLocationParams location = new THiveLocationParams();
-        String targetPath = dbLocation + uuid;
+        String targetPath = dbLocation + uuid + "/" + partitionValue;
+
         location.setTargetPath(targetPath);
-        location.setWritePath(targetPath);
+        location.setWritePath(writeLocation + partitionValue);
 
         THivePartitionUpdate pu = new THivePartitionUpdate();
-        pu.setName(partitionValue);
+        if (partitionValue != null) {
+            pu.setName(partitionValue);
+        }
         pu.setUpdateMode(mode);
         pu.setRowCount(1);
         pu.setFileSize(1);
         pu.setLocation(location);
+        String f1 = uuid + "f1";
+        String f2 = uuid + "f2";
+        String f3 = uuid + "f3";
+
         pu.setFileNames(new ArrayList<String>() {
             {
-                add(targetPath + "/f1");
-                add(targetPath + "/f2");
-                add(targetPath + "/f3");
+                add(f1);
+                add(f2);
+                add(f3);
             }
         });
+
+        if (mode != TUpdateMode.NEW) {
+            fs.makeDir(targetPath);
+        }
+
+        fs.createFile(writeLocation + partitionValue + "/" + f1);
+        fs.createFile(writeLocation + partitionValue + "/" + f2);
+        fs.createFile(writeLocation + partitionValue + "/" + f3);
         return pu;
     }
 
-    public THivePartitionUpdate createRandomNew(String partition) {
-        return genOnePartitionUpdate("c3=" + partition, TUpdateMode.NEW);
+    public THivePartitionUpdate createRandomNew(String partition) throws IOException {
+        return partition == null ? genOnePartitionUpdate(TUpdateMode.NEW) :
+            genOnePartitionUpdate("c3=" + partition, TUpdateMode.NEW);
     }
 
-    public THivePartitionUpdate createRandomAppend(String partition) {
-        return genOnePartitionUpdate("c3=" + partition, TUpdateMode.APPEND);
+    public THivePartitionUpdate createRandomAppend(String partition) throws IOException {
+        return partition == null ? genOnePartitionUpdate(TUpdateMode.APPEND) :
+            genOnePartitionUpdate("c3=" + partition, TUpdateMode.APPEND);
     }
 
-    public THivePartitionUpdate createRandomOverwrite(String partition) {
-        return genOnePartitionUpdate("c3=" + partition, TUpdateMode.OVERWRITE);
+    public THivePartitionUpdate createRandomOverwrite(String partition) throws IOException {
+        return partition == null ? genOnePartitionUpdate(TUpdateMode.OVERWRITE) :
+            genOnePartitionUpdate("c3=" + partition, TUpdateMode.OVERWRITE);
     }
 
     public void commit(String dbName,
@@ -323,4 +340,181 @@ public class HmsCommitTest {
         hmsTransaction.finishInsertTable(dbName, tableName);
         hmsTransaction.commit();
     }
+
+    public void mockAddPartitionTaskException(Runnable runnable) {
+        new MockUp<HMSTransaction.AddPartitionsTask>(HMSTransaction.AddPartitionsTask.class) {
+            @Mock
+            private void run(HiveMetadataOps hiveOps) {
+                runnable.run();
+                throw new RuntimeException("failed to add partition");
+            }
+        };
+    }
+
+    public void mockDoOther(Runnable runnable) {
+        new MockUp<HMSTransaction.HmsCommitter>(HMSTransaction.HmsCommitter.class) {
+            @Mock
+            private void doNothing() {
+                runnable.run();
+                throw new RuntimeException("failed to do nothing");
+            }
+        };
+    }
+
+    public void mockUpdateStatisticsTaskException(Runnable runnable) {
+        new MockUp<HMSTransaction.UpdateStatisticsTask>(HMSTransaction.UpdateStatisticsTask.class) {
+            @Mock
+            private void run(HiveMetadataOps hiveOps) {
+                runnable.run();
+                throw new RuntimeException("failed to update partition");
+            }
+        };
+    }
+
+    @Test
+    public void testRollbackNewPartitionForPartitionedTableForFilesystem() throws IOException {
+        List<THivePartitionUpdate> pus = new ArrayList<>();
+        pus.add(createRandomNew("a"));
+
+        THiveLocationParams location = pus.get(0).getLocation();
+
+        // For new partition, there should be no target path
+        Assert.assertFalse(fs.exists(location.getTargetPath()).ok());
+        Assert.assertTrue(fs.exists(location.getWritePath()).ok());
+
+        mockAddPartitionTaskException(() -> {
+            // When the commit is completed, these files should be renamed successfully
+            String targetPath = location.getTargetPath();
+            Assert.assertTrue(fs.exists(targetPath).ok());
+            for (String file : pus.get(0).getFileNames()) {
+                Assert.assertTrue(fs.exists(targetPath + "/" + file).ok());
+            }
+        });
+
+        try {
+            commit(dbName, tbWithPartition, pus);
+            Assert.assertTrue(false);
+        } catch (Exception e) {
+            // ignore
+        }
+
+        // After rollback, these files will be deleted
+        String targetPath = location.getTargetPath();
+        Assert.assertFalse(fs.exists(targetPath).ok());
+        for (String file : pus.get(0).getFileNames()) {
+            Assert.assertFalse(fs.exists(targetPath + "/" + file).ok());
+        }
+    }
+
+
+    @Test
+    public void testRollbackNewPartitionForPartitionedTableWithNewPartition() throws IOException {
+        // first create three partitions: a,b,c
+        testNewPartitionForPartitionedTable();
+
+        // second add 'new partition' for 'x'
+        //        add 'append partition' for 'a'
+        // when 'doCommit', 'new partition' will be executed before 'append partition'
+        // so, when 'rollback', the 'x' partition will be added and then deleted
+        List<THivePartitionUpdate> pus = new ArrayList<>();
+        pus.add(createRandomNew("x"));
+        pus.add(createRandomAppend("a"));
+
+        THiveLocationParams location = pus.get(0).getLocation();
+
+        // For new partition, there should be no target path
+        Assert.assertFalse(fs.exists(location.getTargetPath()).ok());
+        Assert.assertTrue(fs.exists(location.getWritePath()).ok());
+
+        mockUpdateStatisticsTaskException(() -> {
+            // When the commit is completed, these files should be renamed successfully
+            String targetPath = location.getTargetPath();
+            Assert.assertTrue(fs.exists(targetPath).ok());
+            for (String file : pus.get(0).getFileNames()) {
+                Assert.assertTrue(fs.exists(targetPath + "/" + file).ok());
+            }
+            // new partition will be executed before append partition,
+            // so, we can get the new partition
+            Partition px = hmsClient.getPartition(dbName, tbWithPartition, Lists.newArrayList("x"));
+            assertNumRows(1, px);
+        });
+
+        try {
+            commit(dbName, tbWithPartition, pus);
+            Assert.assertTrue(false);
+        } catch (Exception e) {
+            // ignore
+        }
+
+        // After rollback, these files will be deleted
+        String targetPath = location.getTargetPath();
+        Assert.assertFalse(fs.exists(targetPath).ok());
+        for (String file : pus.get(0).getFileNames()) {
+            Assert.assertFalse(fs.exists(targetPath + "/" + file).ok());
+        }
+        // x partition will be deleted
+        Assert.assertThrows(
+                "the 'x' partition should be deleted",
+                Exception.class,
+                () -> hmsClient.getPartition(dbName, tbWithPartition, Lists.newArrayList("x"))
+        );
+    }
+
+    @Test
+    public void testRollbackNewPartitionForPartitionedTableWithNewAppendPartition() throws IOException {
+        // first create three partitions: a,b,c
+        testNewPartitionForPartitionedTable();
+
+        // second add 'new partition' for 'x'
+        //        add 'append partition' for 'a'
+        List<THivePartitionUpdate> pus = new ArrayList<>();
+        pus.add(createRandomNew("x"));
+        pus.add(createRandomAppend("a"));
+
+        THiveLocationParams location = pus.get(0).getLocation();
+
+        // For new partition, there should be no target path
+        Assert.assertFalse(fs.exists(location.getTargetPath()).ok());
+        Assert.assertTrue(fs.exists(location.getWritePath()).ok());
+
+        mockDoOther(() -> {
+            // When the commit is completed, these files should be renamed successfully
+            String targetPath = location.getTargetPath();
+            Assert.assertTrue(fs.exists(targetPath).ok());
+            for (String file : pus.get(0).getFileNames()) {
+                Assert.assertTrue(fs.exists(targetPath + "/" + file).ok());
+            }
+            // new partition will be executed,
+            // so, we can get the new partition
+            Partition px = hmsClient.getPartition(dbName, tbWithPartition, Lists.newArrayList("x"));
+            assertNumRows(1, px);
+            // append partition will be executed,
+            // so, we can get the updated partition
+            Partition pa = hmsClient.getPartition(dbName, tbWithPartition, Lists.newArrayList("a"));
+            assertNumRows(4, pa);
+        });
+
+        try {
+            commit(dbName, tbWithPartition, pus);
+            Assert.assertTrue(false);
+        } catch (Exception e) {
+            // ignore
+        }
+
+        // After rollback, these files will be deleted
+        String targetPath = location.getTargetPath();
+        Assert.assertFalse(fs.exists(targetPath).ok());
+        for (String file : pus.get(0).getFileNames()) {
+            Assert.assertFalse(fs.exists(targetPath + "/" + file).ok());
+        }
+        // x partition will be deleted
+        Assert.assertThrows(
+                "the 'x' partition should be deleted",
+                Exception.class,
+                () -> hmsClient.getPartition(dbName, tbWithPartition, Lists.newArrayList("x"))
+        );
+        // the 'a' partition should be rollback
+        Partition pa = hmsClient.getPartition(dbName, tbWithPartition, Lists.newArrayList("a"));
+        assertNumRows(3, pa);
+    }
 }


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


(doris) 12/19: [feature](insert)add hive insert plan ut and remove redundant fields (#33051)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 01b21da82d85a3af451c6772942396577b147030
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Sat Mar 30 22:47:24 2024 +0800

    [feature](insert)add hive insert plan ut and remove redundant fields (#33051)
    
    add hive insert sink plan UT case
    remove some deprecated code
---
 .../DistributionSpecTableSinkHashPartitioned.java  |   3 -
 .../doris/nereids/rules/analysis/BindSink.java     |   7 -
 ...ogicalHiveTableSinkToPhysicalHiveTableSink.java |   3 +-
 .../commands/insert/InsertIntoTableCommand.java    |   2 +-
 .../trees/plans/logical/LogicalHiveTableSink.java  |  19 +--
 .../plans/physical/PhysicalHiveTableSink.java      |  24 +--
 .../datasource/hive/HiveDDLAndDMLPlanTest.java     | 166 ++++++++++++++++++---
 7 files changed, 161 insertions(+), 63 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecTableSinkHashPartitioned.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecTableSinkHashPartitioned.java
index 4333bd956ee..e9fc32c1fd6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecTableSinkHashPartitioned.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecTableSinkHashPartitioned.java
@@ -26,9 +26,6 @@ import java.util.List;
  */
 public class DistributionSpecTableSinkHashPartitioned extends DistributionSpec {
 
-    public static final DistributionSpecTableSinkHashPartitioned INSTANCE =
-            new DistributionSpecTableSinkHashPartitioned();
-
     private List<ExprId> outputColExprIds;
 
     public DistributionSpecTableSinkHashPartitioned() {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java
index 20f05729822..2400337d50e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java
@@ -68,12 +68,10 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
 
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.Set;
 import java.util.stream.Collectors;
 
 /**
@@ -389,15 +387,10 @@ public class BindSink implements AnalysisRuleFactory {
                 return column;
             }).collect(ImmutableList.toImmutableList());
         }
-        Set<String> hivePartitionKeys = table.getRemoteTable()
-                .getPartitionKeys().stream()
-                .map(FieldSchema::getName)
-                .collect(Collectors.toSet());
         LogicalHiveTableSink<?> boundSink = new LogicalHiveTableSink<>(
                 database,
                 table,
                 bindColumns,
-                hivePartitionKeys,
                 child.getOutput().stream()
                         .map(NamedExpression.class::cast)
                         .collect(ImmutableList.toImmutableList()),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalHiveTableSinkToPhysicalHiveTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalHiveTableSinkToPhysicalHiveTableSink.java
index f2128658616..153216d6ac7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalHiveTableSinkToPhysicalHiveTableSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalHiveTableSinkToPhysicalHiveTableSink.java
@@ -42,8 +42,7 @@ public class LogicalHiveTableSinkToPhysicalHiveTableSink extends OneImplementati
                     sink.getLogicalProperties(),
                     null,
                     null,
-                    sink.child(),
-                    sink.getHivePartitionKeys());
+                    sink.child());
         }).toRule(RuleType.LOGICAL_HIVE_TABLE_SINK_TO_PHYSICAL_HIVE_TABLE_SINK_RULE);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java
index 3cfd40cabb6..61024345c06 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java
@@ -76,7 +76,7 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync,
      * constructor
      */
     public InsertIntoTableCommand(LogicalPlan logicalQuery, Optional<String> labelName,
-            Optional<InsertCommandContext> insertCtx) {
+                                  Optional<InsertCommandContext> insertCtx) {
         super(PlanType.INSERT_INTO_TABLE_COMMAND);
         this.logicalQuery = Objects.requireNonNull(logicalQuery, "logicalQuery should not be null");
         this.labelName = Objects.requireNonNull(labelName, "labelName should not be null");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHiveTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHiveTableSink.java
index 360d227b0f0..147f14e5e69 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHiveTableSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHiveTableSink.java
@@ -37,7 +37,6 @@ import com.google.common.collect.ImmutableList;
 import java.util.List;
 import java.util.Objects;
 import java.util.Optional;
-import java.util.Set;
 
 /**
  * logical hive table sink for insert command
@@ -47,7 +46,6 @@ public class LogicalHiveTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
     // bound data sink
     private final HMSExternalDatabase database;
     private final HMSExternalTable targetTable;
-    private final Set<String> hivePartitionKeys;
     private final DMLCommandType dmlCommandType;
 
     /**
@@ -56,7 +54,6 @@ public class LogicalHiveTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
     public LogicalHiveTableSink(HMSExternalDatabase database,
                                 HMSExternalTable targetTable,
                                 List<Column> cols,
-                                Set<String> hivePartitionKeys,
                                 List<NamedExpression> outputExprs,
                                 DMLCommandType dmlCommandType,
                                 Optional<GroupExpression> groupExpression,
@@ -66,26 +63,25 @@ public class LogicalHiveTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
         this.database = Objects.requireNonNull(database, "database != null in LogicalHiveTableSink");
         this.targetTable = Objects.requireNonNull(targetTable, "targetTable != null in LogicalHiveTableSink");
         this.dmlCommandType = dmlCommandType;
-        this.hivePartitionKeys = hivePartitionKeys;
     }
 
     public Plan withChildAndUpdateOutput(Plan child) {
         List<NamedExpression> output = child.getOutput().stream()
                 .map(NamedExpression.class::cast)
                 .collect(ImmutableList.toImmutableList());
-        return new LogicalHiveTableSink<>(database, targetTable, cols, hivePartitionKeys, output,
+        return new LogicalHiveTableSink<>(database, targetTable, cols, output,
                 dmlCommandType, Optional.empty(), Optional.empty(), child);
     }
 
     @Override
     public Plan withChildren(List<Plan> children) {
         Preconditions.checkArgument(children.size() == 1, "LogicalHiveTableSink only accepts one child");
-        return new LogicalHiveTableSink<>(database, targetTable, cols, hivePartitionKeys, outputExprs,
+        return new LogicalHiveTableSink<>(database, targetTable, cols, outputExprs,
                 dmlCommandType, Optional.empty(), Optional.empty(), children.get(0));
     }
 
     public LogicalHiveTableSink<CHILD_TYPE> withOutputExprs(List<NamedExpression> outputExprs) {
-        return new LogicalHiveTableSink<>(database, targetTable, cols, hivePartitionKeys, outputExprs,
+        return new LogicalHiveTableSink<>(database, targetTable, cols, outputExprs,
                 dmlCommandType, Optional.empty(), Optional.empty(), child());
     }
 
@@ -97,10 +93,6 @@ public class LogicalHiveTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
         return targetTable;
     }
 
-    public Set<String> getHivePartitionKeys() {
-        return hivePartitionKeys;
-    }
-
     public DMLCommandType getDmlCommandType() {
         return dmlCommandType;
     }
@@ -134,7 +126,6 @@ public class LogicalHiveTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
                 "database", database.getFullName(),
                 "targetTable", targetTable.getName(),
                 "cols", cols,
-                "hivePartitionKeys", hivePartitionKeys,
                 "dmlCommandType", dmlCommandType
         );
     }
@@ -146,14 +137,14 @@ public class LogicalHiveTableSink<CHILD_TYPE extends Plan> extends LogicalTableS
 
     @Override
     public Plan withGroupExpression(Optional<GroupExpression> groupExpression) {
-        return new LogicalHiveTableSink<>(database, targetTable, cols, hivePartitionKeys, outputExprs,
+        return new LogicalHiveTableSink<>(database, targetTable, cols, outputExprs,
                 dmlCommandType, groupExpression, Optional.of(getLogicalProperties()), child());
     }
 
     @Override
     public Plan withGroupExprLogicalPropChildren(Optional<GroupExpression> groupExpression,
             Optional<LogicalProperties> logicalProperties, List<Plan> children) {
-        return new LogicalHiveTableSink<>(database, targetTable, cols, hivePartitionKeys, outputExprs,
+        return new LogicalHiveTableSink<>(database, targetTable, cols, outputExprs,
                 dmlCommandType, groupExpression, logicalProperties, children.get(0));
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java
index a670290a163..58141e61bf8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHiveTableSink.java
@@ -35,7 +35,6 @@ import org.apache.doris.nereids.util.Utils;
 import org.apache.doris.statistics.Statistics;
 
 import com.google.common.collect.ImmutableList;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -50,7 +49,6 @@ public class PhysicalHiveTableSink<CHILD_TYPE extends Plan> extends PhysicalTabl
     private final HMSExternalDatabase database;
     private final HMSExternalTable targetTable;
     private final List<Column> cols;
-    private final Set<String> hivePartitionKeys;
 
     /**
      * constructor
@@ -61,10 +59,9 @@ public class PhysicalHiveTableSink<CHILD_TYPE extends Plan> extends PhysicalTabl
                                  List<NamedExpression> outputExprs,
                                  Optional<GroupExpression> groupExpression,
                                  LogicalProperties logicalProperties,
-                                 CHILD_TYPE child,
-                                 Set<String> hivePartitionKeys) {
+                                 CHILD_TYPE child) {
         this(database, targetTable, cols, outputExprs, groupExpression, logicalProperties,
-                PhysicalProperties.GATHER, null, child, hivePartitionKeys);
+                PhysicalProperties.GATHER, null, child);
     }
 
     /**
@@ -78,14 +75,12 @@ public class PhysicalHiveTableSink<CHILD_TYPE extends Plan> extends PhysicalTabl
                                  LogicalProperties logicalProperties,
                                  PhysicalProperties physicalProperties,
                                  Statistics statistics,
-                                 CHILD_TYPE child,
-                                 Set<String> hivePartitionKeys) {
+                                 CHILD_TYPE child) {
         super(PlanType.PHYSICAL_HIVE_TABLE_SINK, outputExprs, groupExpression,
                 logicalProperties, physicalProperties, statistics, child);
         this.database = Objects.requireNonNull(database, "database != null in PhysicalHiveTableSink");
         this.targetTable = Objects.requireNonNull(targetTable, "targetTable != null in PhysicalHiveTableSink");
         this.cols = Utils.copyRequiredList(cols);
-        this.hivePartitionKeys = hivePartitionKeys;
     }
 
     public HMSExternalDatabase getDatabase() {
@@ -103,7 +98,7 @@ public class PhysicalHiveTableSink<CHILD_TYPE extends Plan> extends PhysicalTabl
     @Override
     public Plan withChildren(List<Plan> children) {
         return new PhysicalHiveTableSink<>(database, targetTable, cols, outputExprs, groupExpression,
-                getLogicalProperties(), physicalProperties, statistics, children.get(0), hivePartitionKeys);
+                getLogicalProperties(), physicalProperties, statistics, children.get(0));
     }
 
     @Override
@@ -119,20 +114,20 @@ public class PhysicalHiveTableSink<CHILD_TYPE extends Plan> extends PhysicalTabl
     @Override
     public Plan withGroupExpression(Optional<GroupExpression> groupExpression) {
         return new PhysicalHiveTableSink<>(database, targetTable, cols, outputExprs,
-                groupExpression, getLogicalProperties(), child(), hivePartitionKeys);
+                groupExpression, getLogicalProperties(), child());
     }
 
     @Override
     public Plan withGroupExprLogicalPropChildren(Optional<GroupExpression> groupExpression,
                                                  Optional<LogicalProperties> logicalProperties, List<Plan> children) {
         return new PhysicalHiveTableSink<>(database, targetTable, cols, outputExprs,
-                groupExpression, logicalProperties.get(), children.get(0), hivePartitionKeys);
+                groupExpression, logicalProperties.get(), children.get(0));
     }
 
     @Override
     public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) {
         return new PhysicalHiveTableSink<>(database, targetTable, cols, outputExprs,
-                groupExpression, getLogicalProperties(), physicalProperties, statistics, child(), hivePartitionKeys);
+                groupExpression, getLogicalProperties(), physicalProperties, statistics, child());
     }
 
     /**
@@ -140,10 +135,7 @@ public class PhysicalHiveTableSink<CHILD_TYPE extends Plan> extends PhysicalTabl
      */
     @Override
     public PhysicalProperties getRequirePhysicalProperties() {
-        Set<String> hivePartitionKeys = targetTable.getRemoteTable()
-                .getPartitionKeys().stream()
-                .map(FieldSchema::getName)
-                .collect(Collectors.toSet());
+        Set<String> hivePartitionKeys = targetTable.getPartitionColumnNames();
         if (!hivePartitionKeys.isEmpty()) {
             List<Integer> columnIdx = new ArrayList<>();
             List<Column> fullSchema = targetTable.getFullSchema();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
index 9d51bf6005f..8247bd84b4d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
@@ -24,15 +24,28 @@ import org.apache.doris.analysis.HashDistributionDesc;
 import org.apache.doris.analysis.SwitchStmt;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Env;
+import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.ExceptionChecker;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.datasource.TableMetadata;
+import org.apache.doris.nereids.NereidsPlanner;
+import org.apache.doris.nereids.StatementContext;
 import org.apache.doris.nereids.parser.NereidsParser;
+import org.apache.doris.nereids.properties.DistributionSpecTableSinkHashPartitioned;
+import org.apache.doris.nereids.properties.DistributionSpecTableSinkRandomPartitioned;
+import org.apache.doris.nereids.properties.PhysicalProperties;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.PlanType;
 import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand;
 import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand;
 import org.apache.doris.nereids.trees.plans.commands.insert.InsertOverwriteTableCommand;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.logical.UnboundLogicalSink;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalHiveTableSink;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan;
+import org.apache.doris.nereids.util.MemoTestUtils;
 import org.apache.doris.utframe.TestWithFeService;
 
 import mockit.Mock;
@@ -45,9 +58,11 @@ import org.junit.jupiter.api.Test;
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.Set;
 
 public class HiveDDLAndDMLPlanTest extends TestWithFeService {
     private static final String mockedDbName = "mockedDb";
@@ -61,7 +76,6 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
     @Override
     protected void runBeforeAll() throws Exception {
         connectContext.getSessionVariable().enableFallbackToOriginalPlanner = false;
-        connectContext.getSessionVariable().enableNereidsTimeout = false;
         connectContext.getSessionVariable().enableNereidsDML = true;
         Config.enable_query_hive_views = false;
         // create test internal table
@@ -90,7 +104,7 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
 
         // create external catalog and switch it
         CreateCatalogStmt hiveCatalog = createStmt("create catalog hive properties('type' = 'hms',"
-                        + " 'hive.metastore.uris' = 'thrift://192.168.0.1:9083');");
+                + " 'hive.metastore.uris' = 'thrift://192.168.0.1:9083');");
         Env.getCurrentEnv().getCatalogMgr().createCatalog(hiveCatalog);
         switchHive();
 
@@ -366,32 +380,144 @@ public class HiveDDLAndDMLPlanTest extends TestWithFeService {
         Assertions.assertEquals(16, stmt2.getDistributionDesc().getBuckets());
     }
 
+    private static void mockTargetTable(List<Column> schema, Set<String> partNames) {
+        new MockUp<HMSExternalTable>(HMSExternalTable.class) {
+            @Mock
+            public boolean isView() {
+                return false;
+            }
+
+            @Mock
+            public List<Column> getFullSchema() {
+                return schema;
+            }
+
+            @Mock
+            public Set<String> getPartitionColumnNames() {
+                return partNames;
+            }
+        };
+    }
+
     @Test
     public void testInsertIntoPlanSql() throws Exception {
         switchHive();
         useDatabase(mockedDbName);
-        String insertSql = "INSERT INTO unpart_ctas_src values(1, 'v1')";
-        LogicalPlan plan = nereidsParser.parseSingle(insertSql);
-        Assertions.assertTrue(plan instanceof InsertIntoTableCommand);
-        // TODO check plan node, exchange node
+        String insertTable = "insert_table";
+        createTargetTable(insertTable);
+
+        // test un-partitioned table
+        List<Column> schema = new ArrayList<Column>() {
+            {
+                add(new Column("col1", PrimitiveType.INT));
+                add(new Column("col2", PrimitiveType.STRING));
+                add(new Column("col3", PrimitiveType.DECIMAL32));
+                add(new Column("col4", PrimitiveType.CHAR));
+            }
+        };
+
+        mockTargetTable(schema, new HashSet<>());
+        String unPartTargetTable = "unpart_" + insertTable;
+        String insertSql = "INSERT INTO " + unPartTargetTable + " values(1, 'v1', 32.1, 'aabb')";
+        PhysicalPlan physicalSink = getPhysicalPlan(insertSql, PhysicalProperties.SINK_RANDOM_PARTITIONED,
+                false);
+        checkUnpartTableSinkPlan(schema, unPartTargetTable, physicalSink);
+
+        String insertOverwriteSql = "INSERT OVERWRITE TABLE " + unPartTargetTable + " values(1, 'v1', 32.1, 'aabb')";
+        PhysicalPlan physicalOverwriteSink = getPhysicalPlan(insertOverwriteSql, PhysicalProperties.SINK_RANDOM_PARTITIONED,
+                true);
+        checkUnpartTableSinkPlan(schema, unPartTargetTable, physicalOverwriteSink);
+
+        // test partitioned table
+        schema = new ArrayList<Column>() {
+            {
+                add(new Column("col1", PrimitiveType.INT));
+                add(new Column("pt1", PrimitiveType.VARCHAR));
+                add(new Column("pt2", PrimitiveType.STRING));
+                add(new Column("pt3", PrimitiveType.DATE));
+            }
+        };
+        Set<String> parts = new HashSet<String>() {
+            {
+                add("pt1");
+                add("pt2");
+                add("pt3");
+            }
+        };
+        mockTargetTable(schema, parts);
+        String partTargetTable = "part_" + insertTable;
+
+        String insertSql2 = "INSERT INTO " + partTargetTable + " values(1, 'v1', 'v2', '2020-03-13')";
+        PhysicalPlan physicalSink2 = getPhysicalPlan(insertSql2,
+                new PhysicalProperties(new DistributionSpecTableSinkHashPartitioned()), false);
+        checkPartTableSinkPlan(schema, partTargetTable, physicalSink2);
+
+        String insertOverwrite2 = "INSERT OVERWRITE TABLE " + partTargetTable + " values(1, 'v1', 'v2', '2020-03-13')";
+        PhysicalPlan physicalOverwriteSink2 = getPhysicalPlan(insertOverwrite2,
+                new PhysicalProperties(new DistributionSpecTableSinkHashPartitioned()), true);
+        checkPartTableSinkPlan(schema, partTargetTable, physicalOverwriteSink2);
+    }
 
-        String insertSql2 = "INSERT INTO part_ctas_src values(1, 'v1', 'v2')";
-        LogicalPlan plan2 = nereidsParser.parseSingle(insertSql2);
-        Assertions.assertTrue(plan2 instanceof InsertIntoTableCommand);
+    private static void checkUnpartTableSinkPlan(List<Column> schema, String unPartTargetTable, PhysicalPlan physicalSink) {
+        Assertions.assertSame(physicalSink.getType(), PlanType.PHYSICAL_DISTRIBUTE);
+        // check exchange
+        PhysicalDistribute<?> distribute = (PhysicalDistribute<?>) physicalSink;
+        Assertions.assertTrue(distribute.getDistributionSpec() instanceof DistributionSpecTableSinkRandomPartitioned);
+        Assertions.assertSame(distribute.child(0).getType(), PlanType.PHYSICAL_HIVE_TABLE_SINK);
+        // check sink
+        PhysicalHiveTableSink<?> physicalHiveSink = (PhysicalHiveTableSink<?>) physicalSink.child(0);
+        Assertions.assertEquals(unPartTargetTable, physicalHiveSink.getTargetTable().getName());
+        Assertions.assertEquals(schema.size(), physicalHiveSink.getOutput().size());
     }
 
-    @Test
-    public void testInsertOverwritePlanSql() throws Exception {
-        switchHive();
-        useDatabase(mockedDbName);
-        String insertSql = "INSERT OVERWRITE TABLE unpart_ctas_src values(2, 'v2')";
-        LogicalPlan plan = nereidsParser.parseSingle(insertSql);
-        Assertions.assertTrue(plan instanceof InsertOverwriteTableCommand);
-        // TODO check plan node, exchange node
+    private static void checkPartTableSinkPlan(List<Column> schema, String unPartTargetTable, PhysicalPlan physicalSink) {
+        Assertions.assertSame(physicalSink.getType(), PlanType.PHYSICAL_DISTRIBUTE);
+        // check exchange
+        PhysicalDistribute<?> distribute2 = (PhysicalDistribute<?>) physicalSink;
+        Assertions.assertTrue(distribute2.getDistributionSpec() instanceof DistributionSpecTableSinkHashPartitioned);
+        Assertions.assertSame(distribute2.child(0).getType(), PlanType.PHYSICAL_HIVE_TABLE_SINK);
+        // check sink
+        PhysicalHiveTableSink<?> physicalHiveSink2 = (PhysicalHiveTableSink<?>) physicalSink.child(0);
+        Assertions.assertEquals(unPartTargetTable, physicalHiveSink2.getTargetTable().getName());
+        Assertions.assertEquals(schema.size(), physicalHiveSink2.getOutput().size());
+    }
 
-        String insertSql2 = "INSERT OVERWRITE TABLE part_ctas_src values(2, 'v3', 'v4')";
-        LogicalPlan plan2 = nereidsParser.parseSingle(insertSql2);
-        Assertions.assertTrue(plan2 instanceof InsertOverwriteTableCommand);
+    private void createTargetTable(String tableName) throws Exception {
+        String createInsertTable = "CREATE TABLE `unpart_" + tableName + "`(\n"
+                + "  `col1` INT COMMENT 'col1',\n"
+                + "  `col2` STRING COMMENT 'col2',\n"
+                + "  `col3` DECIMAL(3,1) COMMENT 'col3',\n"
+                + "  `col4` CHAR(11) COMMENT 'col4'\n"
+                + ")  ENGINE=hive\n"
+                + "PROPERTIES ('file_format'='orc')";
+        createTable(createInsertTable, true);
+
+        String createInsertPTable = "CREATE TABLE `part_" + tableName + "`(\n"
+                + "  `col1` INT COMMENT 'col1',\n"
+                + "  `pt1` VARCHAR(16) COMMENT 'pt1',\n"
+                + "  `pt2` STRING COMMENT 'pt2',\n"
+                + "  `pt3` DATE COMMENT 'pt3'\n"
+                + ")  ENGINE=hive\n"
+                + "PARTITION BY LIST (pt1, pt2, pt3) ()\n"
+                + "PROPERTIES ('file_format'='orc')";
+        createTable(createInsertPTable, true);
+    }
+
+    private PhysicalPlan getPhysicalPlan(String insertSql, PhysicalProperties physicalProperties,
+                                         boolean isOverwrite) {
+        LogicalPlan plan = nereidsParser.parseSingle(insertSql);
+        StatementContext statementContext = MemoTestUtils.createStatementContext(connectContext, insertSql);
+        Plan exPlan;
+        if (isOverwrite) {
+            Assertions.assertTrue(plan instanceof InsertOverwriteTableCommand);
+            exPlan = ((InsertOverwriteTableCommand) plan).getExplainPlan(connectContext);
+        } else {
+            Assertions.assertTrue(plan instanceof InsertIntoTableCommand);
+            exPlan = ((InsertIntoTableCommand) plan).getExplainPlan(connectContext);
+        }
+        Assertions.assertTrue(exPlan instanceof UnboundLogicalSink);
+        NereidsPlanner planner = new NereidsPlanner(statementContext);
+        return planner.plan((UnboundLogicalSink<?>) exPlan, physicalProperties);
     }
 
     @Test


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


(doris) 02/19: [feature][insert]add FE UT and support CTAS for external table (#32525)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit c68b3530175dff3659856c9d553a08636f2369f7
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Sun Mar 24 21:46:58 2024 +0800

    [feature][insert]add FE UT and support CTAS for external table (#32525)
    
    1. add FE ut for create hive table
    2. support external CTAS:
    
    > source table:
    ```
    mysql> show create table hive.jz3.test;
    
    CREATE TABLE `test`(
      `id` int COMMENT 'col1',
      `name` string COMMENT 'col2')
    PARTITIONED BY (
     `dt` string,
     `dtm` string)
    ROW FORMAT SERDE
      'org.apache.hadoop.hive.ql.io.orc.OrcSerde'
    STORED AS INPUTFORMAT
      'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat'
    OUTPUTFORMAT
      'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'
    LOCATION
      'hdfs://HDFS8000871/usr/hive/warehouse/jz3.db/test'
    TBLPROPERTIES (
      'transient_lastDdlTime'='1710837792',
      'file_format'='orc')
    ```
    
    
    > create unpartitioned target table
    ```
    mysql> create table hive.jz3.ctas engine=hive as select * from hive.jz3.test;
    mysql> show create table ctas;
    
    CREATE TABLE `ctas`(
      `id` int COMMENT '',
      `name` string COMMENT '',
      `dt` string COMMENT '',
      `dtm` string COMMENT '')
    ROW FORMAT SERDE
      'org.apache.hadoop.hive.ql.io.orc.OrcSerde'
    STORED AS INPUTFORMAT
      'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat'
    OUTPUTFORMAT
      'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'
    LOCATION
      'hdfs://HDFS8000871/usr/hive/warehouse/jz3.db/ctas'
    TBLPROPERTIES (
      'transient_lastDdlTime'='1710860377')
    
    ```
    
    
    > create partitioned target table
    ```
    mysql> create table hive.jz3.ctas1 engine=hive partition by list (dt,dtm) () as select * from hive.jz3.test;
    mysql> show create table hive.jz3.ctas1;
    
    CREATE TABLE `ctas1`(
      `id` int COMMENT '',
      `name` string COMMENT '')
    PARTITIONED BY (
     `dt` string,
     `dtm` string)
    ROW FORMAT SERDE
      'org.apache.hadoop.hive.ql.io.orc.OrcSerde'
    STORED AS INPUTFORMAT
      'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat'
    OUTPUTFORMAT
      'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'
    LOCATION
      'hdfs://HDFS8000871/usr/hive/warehouse/jz3.db/ctas1'
    TBLPROPERTIES (
      'transient_lastDdlTime'='1710919070')
    ```
---
 .../trees/plans/commands/CreateTableCommand.java   |   9 +-
 .../trees/plans/commands/info/CreateTableInfo.java |  17 +-
 .../datasource/hive/HiveDDLAndDMLPlanTest.java     | 372 +++++++++++++++++++++
 .../doris/datasource/hive/HiveMetadataOpsTest.java |   5 +-
 .../apache/doris/utframe/TestWithFeService.java    |   8 +
 5 files changed, 405 insertions(+), 6 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateTableCommand.java
index 1a6004e939a..613c29136d4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateTableCommand.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateTableCommand.java
@@ -23,6 +23,7 @@ import org.apache.doris.analysis.TableName;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.FeConstants;
 import org.apache.doris.nereids.NereidsPlanner;
 import org.apache.doris.nereids.analyzer.UnboundResultSink;
 import org.apache.doris.nereids.analyzer.UnboundTableSinkCreator;
@@ -48,6 +49,7 @@ import org.apache.doris.nereids.types.StringType;
 import org.apache.doris.nereids.types.TinyIntType;
 import org.apache.doris.nereids.types.VarcharType;
 import org.apache.doris.nereids.types.coercion.CharacterType;
+import org.apache.doris.nereids.util.RelationUtil;
 import org.apache.doris.nereids.util.TypeCoercionUtils;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.qe.QueryState.MysqlStateType;
@@ -141,7 +143,8 @@ public class CreateTableCommand extends Command implements ForwardWithSync {
             // if the column is an expression, we set it to nullable, otherwise according to the nullable of the slot.
             columnsOfQuery.add(new ColumnDefinition(s.getName(), dataType, !s.isColumnFromTable() || s.nullable()));
         }
-        createTableInfo.validateCreateTableAsSelect(columnsOfQuery.build(), ctx);
+        List<String> qualifierTableName = RelationUtil.getQualifierName(ctx, createTableInfo.getTableNameParts());
+        createTableInfo.validateCreateTableAsSelect(qualifierTableName, columnsOfQuery.build(), ctx);
         CreateTableStmt createTableStmt = createTableInfo.translateToLegacyStmt();
         if (LOG.isDebugEnabled()) {
             LOG.debug("Nereids start to execute the ctas command, query id: {}, tableName: {}",
@@ -156,7 +159,9 @@ public class CreateTableCommand extends Command implements ForwardWithSync {
         query = UnboundTableSinkCreator.createUnboundTableSink(createTableInfo.getTableNameParts(),
                 ImmutableList.of(), ImmutableList.of(), ImmutableList.of(), query);
         try {
-            new InsertIntoTableCommand(query, Optional.empty(), Optional.empty()).run(ctx, executor);
+            if (!FeConstants.runningUnitTest) {
+                new InsertIntoTableCommand(query, Optional.empty(), Optional.empty()).run(ctx, executor);
+            }
             if (ctx.getState().getStateType() == MysqlStateType.ERR) {
                 handleFallbackFailedCtas(ctx);
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
index 33ec21ed66a..07c55ca9af1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
@@ -527,11 +527,22 @@ public class CreateTableInfo {
         }
     }
 
-    public void validateCreateTableAsSelect(List<ColumnDefinition> columns, ConnectContext ctx) {
+    /**
+     * validate ctas definition
+     */
+    public void validateCreateTableAsSelect(List<String> qualifierTableName, List<ColumnDefinition> columns,
+                                            ConnectContext ctx) {
         this.columns = Utils.copyRequiredMutableList(columns);
         // bucket num is hard coded 10 to be consistent with legacy planner
-        this.distribution = new DistributionDescriptor(true, false, 10,
-                Lists.newArrayList(columns.get(0).getName()));
+        if (engineName.equals("olap") && this.distribution == null) {
+            String catalogName = qualifierTableName.get(0);
+            if (!catalogName.equals(InternalCatalog.INTERNAL_CATALOG_NAME)) {
+                throw new AnalysisException("Cannot create olap table out of internal catalog."
+                        + "Make sure 'engine' type is specified when use the catalog: " + catalogName);
+            }
+            this.distribution = new DistributionDescriptor(true, false, 10,
+                    Lists.newArrayList(columns.get(0).getName()));
+        }
         validate(ctx);
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
new file mode 100644
index 00000000000..b7e90218e10
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveDDLAndDMLPlanTest.java
@@ -0,0 +1,372 @@
+// 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.hive;
+
+import org.apache.doris.analysis.CreateCatalogStmt;
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.analysis.HashDistributionDesc;
+import org.apache.doris.analysis.SwitchStmt;
+import org.apache.doris.catalog.Env;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.ExceptionChecker;
+import org.apache.doris.common.FeConstants;
+import org.apache.doris.nereids.parser.NereidsParser;
+import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand;
+import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand;
+import org.apache.doris.nereids.trees.plans.commands.insert.InsertOverwriteTableCommand;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.utframe.TestWithFeService;
+
+import mockit.Mock;
+import mockit.MockUp;
+import mockit.Mocked;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+public class HiveDDLAndDMLPlanTest extends TestWithFeService {
+    private static final String mockedDbName = "mockedDb";
+    private final NereidsParser nereidsParser = new NereidsParser();
+
+    @Mocked
+    private ThriftHMSCachedClient mockedHiveClient;
+
+    @Override
+    protected void runBeforeAll() throws Exception {
+        connectContext.getSessionVariable().enableFallbackToOriginalPlanner = false;
+        connectContext.getSessionVariable().enableNereidsTimeout = false;
+        connectContext.getSessionVariable().enableNereidsDML = true;
+        Config.enable_query_hive_views = false;
+        Config.enable_external_ddl = true;
+        // create test internal table
+        createDatabase(mockedDbName);
+        useDatabase(mockedDbName);
+        String createSourceInterTable = "CREATE TABLE `unpart_ctas_olap`(\n"
+                + "  `col1` INT COMMENT 'col1',\n"
+                + "  `col2` STRING COMMENT 'col2'\n"
+                + ")  ENGINE=olap\n"
+                + "DISTRIBUTED BY HASH (col1) BUCKETS 16\n"
+                + "PROPERTIES (\n"
+                + "  'replication_num' = '1')";
+        createTable(createSourceInterTable, true);
+
+        // partitioned table
+        String createSourceInterPTable = "CREATE TABLE `part_ctas_olap`(\n"
+                + "  `col1` INT COMMENT 'col1',\n"
+                + "  `pt1` VARCHAR COMMENT 'pt1',\n"
+                + "  `pt2` VARCHAR COMMENT 'pt2'\n"
+                + ")  ENGINE=olap\n"
+                + "PARTITION BY LIST (pt1, pt2) ()\n"
+                + "DISTRIBUTED BY HASH (col1) BUCKETS 16\n"
+                + "PROPERTIES (\n"
+                + "  'replication_num' = '1')";
+        createTable(createSourceInterPTable, true);
+
+        // create external catalog and switch it
+        CreateCatalogStmt hiveCatalog = createStmt("create catalog hive properties('type' = 'hms',"
+                        + " 'hive.metastore.uris' = 'thrift://192.168.0.1:9083');");
+        Env.getCurrentEnv().getCatalogMgr().createCatalog(hiveCatalog);
+        switchHive();
+
+        // create db and use it
+        Map<String, String> dbProps = new HashMap<>();
+        dbProps.put(HiveMetadataOps.LOCATION_URI_KEY, "file://loc/db");
+        new MockUp<ThriftHMSCachedClient>(ThriftHMSCachedClient.class) {
+            @Mock
+            public List<String> getAllDatabases() {
+                return new ArrayList<String>() {
+                    {
+                        add(mockedDbName);
+                    }
+                };
+            }
+        };
+        CreateDbStmt createDbStmt = new CreateDbStmt(true, mockedDbName, dbProps);
+        Env.getCurrentEnv().createDb(createDbStmt);
+        useDatabase(mockedDbName);
+
+        // un-partitioned table
+        String createSourceExtUTable = "CREATE TABLE `unpart_ctas_src`(\n"
+                + "  `col1` INT COMMENT 'col1',\n"
+                + "  `col2` STRING COMMENT 'col2'\n"
+                + ")  ENGINE=hive\n"
+                + "PROPERTIES (\n"
+                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'file_format'='parquet')";
+        createTable(createSourceExtUTable, true);
+        // partitioned table
+        String createSourceExtTable = "CREATE TABLE `part_ctas_src`(\n"
+                + "  `col1` INT COMMENT 'col1',\n"
+                + "  `pt1` STRING COMMENT 'pt1',\n"
+                + "  `pt2` STRING COMMENT 'pt2'\n"
+                + ")  ENGINE=hive\n"
+                + "PARTITION BY LIST (pt1, pt2) ()\n"
+                + "PROPERTIES (\n"
+                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'file_format'='orc')";
+        createTable(createSourceExtTable, true);
+
+        HMSExternalCatalog hmsExternalCatalog = (HMSExternalCatalog) Env.getCurrentEnv().getCatalogMgr()
+                .getCatalog("hive");
+        new MockUp<HMSExternalDatabase>(HMSExternalDatabase.class) {
+            @Mock
+            HMSExternalTable getTableNullable(String tableName) {
+                return new HMSExternalTable(0, tableName, mockedDbName, hmsExternalCatalog);
+            }
+        };
+        new MockUp<HMSExternalTable>(HMSExternalTable.class) {
+            @Mock
+            protected synchronized void makeSureInitialized() {
+                // mocked
+            }
+        };
+    }
+
+    private void switchHive() throws Exception {
+        SwitchStmt switchHive = (SwitchStmt) parseAndAnalyzeStmt("switch hive;");
+        Env.getCurrentEnv().changeCatalog(connectContext, switchHive.getCatalogName());
+    }
+
+    private void switchInternal() throws Exception {
+        SwitchStmt switchInternal = (SwitchStmt) parseAndAnalyzeStmt("switch internal;");
+        Env.getCurrentEnv().changeCatalog(connectContext, switchInternal.getCatalogName());
+    }
+
+    @Override
+    protected void runAfterAll() throws Exception {
+        switchHive();
+        dropDatabase(mockedDbName);
+    }
+
+    @Test
+    public void testCreateAndDropWithSql() throws Exception {
+        switchHive();
+        useDatabase(mockedDbName);
+        Optional<?> hiveDb = Env.getCurrentEnv().getCurrentCatalog().getDb(mockedDbName);
+        Assertions.assertTrue(hiveDb.isPresent());
+        Assertions.assertTrue(hiveDb.get() instanceof HMSExternalDatabase);
+
+        String createUnPartTable = "CREATE TABLE unpart_tbl(\n"
+                + "  `col1` BOOLEAN COMMENT 'col1',\n"
+                + "  `col2` INT COMMENT 'col2',\n"
+                + "  `col3` BIGINT COMMENT 'col3',\n"
+                + "  `col4` DECIMAL(5,2) COMMENT 'col4',\n"
+                + "  `pt1` STRING COMMENT 'pt1',\n"
+                + "  `pt2` STRING COMMENT 'pt2'\n"
+                + ")  ENGINE=hive\n"
+                + "PROPERTIES (\n"
+                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'file_format'='orc')";
+        createTable(createUnPartTable, true);
+        dropTable("unpart_tbl", true);
+
+        String createPartTable = "CREATE TABLE `part_tbl`(\n"
+                + "  `col1` BOOLEAN COMMENT 'col1',\n"
+                + "  `col2` INT COMMENT 'col2',\n"
+                + "  `col3` BIGINT COMMENT 'col3',\n"
+                + "  `col4` DECIMAL(5,2) COMMENT 'col4',\n"
+                + "  `pt1` STRING COMMENT 'pt1',\n"
+                + "  `pt2` STRING COMMENT 'pt2',\n"
+                + "  `col5` DATE COMMENT 'col5',\n"
+                + "  `col6` DATETIME COMMENT 'col6'\n"
+                + ")  ENGINE=hive\n"
+                + "PARTITION BY LIST (pt1, pt2) ()\n"
+                + "PROPERTIES (\n"
+                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'file_format'='parquet')";
+        createTable(createPartTable, true);
+        dropTable("part_tbl", true);
+
+        String createBucketedTableErr = "CREATE TABLE `err_buck_tbl`(\n"
+                + "  `col1` BOOLEAN COMMENT 'col1',\n"
+                + "  `col2` INT COMMENT 'col2',\n"
+                + "  `col3` BIGINT COMMENT 'col3',\n"
+                + "  `col4` DECIMAL(5,2) COMMENT 'col4'\n"
+                + ")  ENGINE=hive\n"
+                + "DISTRIBUTED BY HASH (col2) BUCKETS 16\n"
+                + "PROPERTIES (\n"
+                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'file_format'='orc')";
+        ExceptionChecker.expectThrowsWithMsg(org.apache.doris.nereids.exceptions.AnalysisException.class,
+                "errCode = 2, detailMessage = errCode = 2,"
+                        + " detailMessage = Create hive bucket table need set enable_create_hive_bucket_table to true",
+                () -> createTable(createBucketedTableErr, true));
+
+        Config.enable_create_hive_bucket_table = true;
+        String createBucketedTableOk1 = "CREATE TABLE `buck_tbl`(\n"
+                + "  `col1` BOOLEAN COMMENT 'col1',\n"
+                + "  `col2` INT COMMENT 'col2',\n"
+                + "  `col3` BIGINT COMMENT 'col3',\n"
+                + "  `col4` DECIMAL(5,2) COMMENT 'col4'\n"
+                + ")  ENGINE=hive\n"
+                + "DISTRIBUTED BY HASH (col2) BUCKETS 16\n"
+                + "PROPERTIES (\n"
+                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'file_format'='orc')";
+        createTable(createBucketedTableOk1, true);
+        dropTable("buck_tbl", true);
+
+        String createBucketedTableOk2 = "CREATE TABLE `part_buck_tbl`(\n"
+                + "  `col1` BOOLEAN COMMENT 'col1',\n"
+                + "  `col2` INT COMMENT 'col2',\n"
+                + "  `col3` BIGINT COMMENT 'col3',\n"
+                + "  `col4` DECIMAL(5,2) COMMENT 'col4',\n"
+                + "  `pt1` STRING COMMENT 'pt1',\n"
+                + "  `pt2` STRING COMMENT 'pt2'\n"
+                + ")  ENGINE=hive\n"
+                + "PARTITION BY LIST (pt2) ()\n"
+                + "DISTRIBUTED BY HASH (col2) BUCKETS 16\n"
+                + "PROPERTIES (\n"
+                + "  'location_uri'='hdfs://loc/db/tbl',\n"
+                + "  'file_format'='orc')";
+        createTable(createBucketedTableOk2, true);
+        dropTable("part_buck_tbl", true);
+    }
+
+    @Test
+    public void testCTASPlanSql() throws Exception {
+        switchHive();
+        useDatabase(mockedDbName);
+        // external to external table
+        String ctas1 = "CREATE TABLE hive_ctas1 AS SELECT col1 FROM unpart_ctas_src WHERE col2='a';";
+        LogicalPlan st1 = nereidsParser.parseSingle(ctas1);
+        Assertions.assertTrue(st1 instanceof CreateTableCommand);
+        // ((CreateTableCommand) st1).run(connectContext, null);
+        String its1 = "INSERT INTO hive_ctas1 SELECT col1 FROM unpart_ctas_src WHERE col2='a';";
+        LogicalPlan it1 = nereidsParser.parseSingle(its1);
+        Assertions.assertTrue(it1 instanceof InsertIntoTableCommand);
+        // ((InsertIntoTableCommand) it1).run(connectContext, null);
+        // partitioned table
+        String ctasU1 = "CREATE TABLE hive_ctas2 AS SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>0;";
+        LogicalPlan stU1 = nereidsParser.parseSingle(ctasU1);
+        Assertions.assertTrue(stU1 instanceof CreateTableCommand);
+        // ((CreateTableCommand) stU1).run(connectContext, null);
+        String itsp1 = "INSERT INTO hive_ctas2 SELECT col1,pt1,pt2 FROM part_ctas_src WHERE col1>0;";
+        LogicalPlan itp1 = nereidsParser.parseSingle(itsp1);
+        Assertions.assertTrue(itp1 instanceof InsertIntoTableCommand);
+        // ((InsertIntoTableCommand) itp1).run(connectContext, null);
+
+        // external to internal table
+        switchInternal();
+        useDatabase(mockedDbName);
+        String ctas2 = "CREATE TABLE olap_ctas1 AS SELECT col1,col2 FROM hive.mockedDb.unpart_ctas_src WHERE col2='a';";
+        LogicalPlan st2 = nereidsParser.parseSingle(ctas2);
+        Assertions.assertTrue(st2 instanceof CreateTableCommand);
+        // ((CreateTableCommand) st2).run(connectContext, null);
+
+        // partitioned table
+        String ctasU2 = "CREATE TABLE olap_ctas2 AS SELECT col1,pt1,pt2 FROM hive.mockedDb.part_ctas_src WHERE col1>0;";
+        LogicalPlan stU2 = nereidsParser.parseSingle(ctasU2);
+        Assertions.assertTrue(stU2 instanceof CreateTableCommand);
+        // ((CreateTableCommand) stU2).run(connectContext, null);
+
+        // internal to external table
+        String ctas3 = "CREATE TABLE hive.mockedDb.ctas_o1 AS SELECT col1,col2 FROM unpart_ctas_olap WHERE col2='a';";
+        LogicalPlan st3 = nereidsParser.parseSingle(ctas3);
+        Assertions.assertTrue(st3 instanceof CreateTableCommand);
+        // ((CreateTableCommand) st3).run(connectContext, null);
+
+        String its2 = "INSERT INTO hive.mockedDb.ctas_o1 SELECT col1,col2 FROM unpart_ctas_olap WHERE col2='a';";
+        LogicalPlan it2 = nereidsParser.parseSingle(its2);
+        Assertions.assertTrue(it2 instanceof InsertIntoTableCommand);
+        // ((InsertIntoTableCommand) it2).run(connectContext, null);
+
+        String ctasP3 = "CREATE TABLE hive.mockedDb.ctas_o2 AS SELECT col1,pt1,pt2 FROM part_ctas_olap WHERE col1>0;";
+        LogicalPlan stP3 = nereidsParser.parseSingle(ctasP3);
+        Assertions.assertTrue(stP3 instanceof CreateTableCommand);
+        // ((CreateTableCommand) stP3).run(connectContext, null);
+
+        String itsp2 = "INSERT INTO hive.mockedDb.ctas_o2 SELECT col1,pt1,pt2 FROM part_ctas_olap WHERE col1>0;";
+        LogicalPlan itp2 = nereidsParser.parseSingle(itsp2);
+        Assertions.assertTrue(itp2 instanceof InsertIntoTableCommand);
+        // ((InsertIntoTableCommand) itp2).run(connectContext, null);
+
+        // test olap CTAS in hive catalog
+        FeConstants.runningUnitTest = true;
+        String createOlapSrc = "CREATE TABLE `olap_src`(\n"
+                + "  `col1` BOOLEAN COMMENT 'col1',\n"
+                + "  `col2` INT COMMENT 'col2',\n"
+                + "  `col3` BIGINT COMMENT 'col3',\n"
+                + "  `col4` DECIMAL(5,2) COMMENT 'col4'\n"
+                + ")\n"
+                + "DISTRIBUTED BY HASH (col1) BUCKETS 100\n"
+                + "PROPERTIES (\n"
+                + "  'replication_num' = '1')";
+        createTable(createOlapSrc, true);
+        switchHive();
+        useDatabase(mockedDbName);
+        String olapCtasErr = "CREATE TABLE no_buck_olap AS SELECT * FROM internal.mockedDb.olap_src";
+        LogicalPlan olapCtasErrPlan = nereidsParser.parseSingle(olapCtasErr);
+        Assertions.assertTrue(olapCtasErrPlan instanceof CreateTableCommand);
+        ExceptionChecker.expectThrowsWithMsg(org.apache.doris.nereids.exceptions.AnalysisException.class,
+                "Cannot create olap table out of internal catalog."
+                        + "Make sure 'engine' type is specified when use the catalog: hive",
+                () -> ((CreateTableCommand) olapCtasErrPlan).run(connectContext, null));
+
+        String olapCtasOk = "CREATE TABLE internal.mockedDb.no_buck_olap"
+                + " PROPERTIES('replication_num' = '1')"
+                + " AS SELECT * FROM internal.mockedDb.olap_src";
+        LogicalPlan olapCtasOkPlan = createTablesAndReturnPlans(true, olapCtasOk).get(0);
+        CreateTableStmt stmt = ((CreateTableCommand) olapCtasOkPlan).getCreateTableInfo().translateToLegacyStmt();
+        Assertions.assertTrue(stmt.getDistributionDesc() instanceof HashDistributionDesc);
+        Assertions.assertEquals(10, stmt.getDistributionDesc().getBuckets());
+        // ((CreateTableCommand) olapCtasOkPlan).run(connectContext, null);
+
+        String olapCtasOk2 = "CREATE TABLE internal.mockedDb.no_buck_olap2 DISTRIBUTED BY HASH (col1) BUCKETS 16"
+                + " PROPERTIES('replication_num' = '1')"
+                + " AS SELECT * FROM internal.mockedDb.olap_src";
+        LogicalPlan olapCtasOk2Plan = createTablesAndReturnPlans(true, olapCtasOk2).get(0);
+        CreateTableStmt stmt2 = ((CreateTableCommand) olapCtasOk2Plan).getCreateTableInfo().translateToLegacyStmt();
+        Assertions.assertTrue(stmt2.getDistributionDesc() instanceof HashDistributionDesc);
+        Assertions.assertEquals(16, stmt2.getDistributionDesc().getBuckets());
+    }
+
+    @Test
+    public void testInsertIntoPlanSql() throws Exception {
+        switchHive();
+        useDatabase(mockedDbName);
+        String insertSql = "INSERT INTO unpart_ctas_src values(1, 'v1')";
+        LogicalPlan plan = nereidsParser.parseSingle(insertSql);
+        Assertions.assertTrue(plan instanceof InsertIntoTableCommand);
+        // TODO check plan node, exchange node
+
+        String insertSql2 = "INSERT INTO part_ctas_src values(1, 'v1', 'v2')";
+        LogicalPlan plan2 = nereidsParser.parseSingle(insertSql2);
+        Assertions.assertTrue(plan2 instanceof InsertIntoTableCommand);
+    }
+
+    @Test
+    public void testInsertOverwritePlanSql() throws Exception {
+        switchHive();
+        useDatabase(mockedDbName);
+        String insertSql = "INSERT OVERWRITE TABLE unpart_ctas_src values(2, 'v2')";
+        LogicalPlan plan = nereidsParser.parseSingle(insertSql);
+        Assertions.assertTrue(plan instanceof InsertOverwriteTableCommand);
+        // TODO check plan node, exchange node
+
+        String insertSql2 = "INSERT OVERWRITE TABLE part_ctas_src values(2, 'v3', 'v4')";
+        LogicalPlan plan2 = nereidsParser.parseSingle(insertSql2);
+        Assertions.assertTrue(plan2 instanceof InsertOverwriteTableCommand);
+    }
+}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
index af57aae703b..6c156807fc6 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java
@@ -48,6 +48,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 
+/**
+ * just overlay all metadata operations here.
+ * @see HiveDDLAndDMLPlanTest Use it if you need to verify correctness.
+ */
 public class HiveMetadataOpsTest {
 
     private HiveMetadataOps metadataOps;
@@ -157,6 +161,5 @@ public class HiveMetadataOpsTest {
         createTable(tableName, cols, parts, bucks, tblProps);
         dropTable(tableName, true);
         dropDb("mockedDb", true);
-        // TODO: use TestWithFeService to double check plan
     }
 }
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 c43ab447885..cb634666662 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
@@ -115,6 +115,7 @@ import java.net.SocketException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
@@ -677,10 +678,16 @@ public abstract class TestWithFeService {
     }
 
     public void createTables(boolean enableNereids, String... sqls) throws Exception {
+        createTablesAndReturnPlans(enableNereids, sqls);
+    }
+
+    public List<LogicalPlan> createTablesAndReturnPlans(boolean enableNereids, String... sqls) throws Exception {
+        List<LogicalPlan> logicalPlans = new ArrayList<>();
         if (enableNereids) {
             for (String sql : sqls) {
                 NereidsParser nereidsParser = new NereidsParser();
                 LogicalPlan parsed = nereidsParser.parseSingle(sql);
+                logicalPlans.add(parsed);
                 StmtExecutor stmtExecutor = new StmtExecutor(connectContext, sql);
                 if (parsed instanceof CreateTableCommand) {
                     ((CreateTableCommand) parsed).run(connectContext, stmtExecutor);
@@ -693,6 +700,7 @@ public abstract class TestWithFeService {
             }
         }
         updateReplicaPathHash();
+        return logicalPlans;
     }
 
     public void createView(String sql) throws Exception {


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


(doris) 06/19: [feature](hive/iceberg)add doris's version in table properties (#32774)

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.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit bd364897d42e484755acbe5096c00ec3a8e9afbd
Author: wuwenchi <wu...@hotmail.com>
AuthorDate: Tue Mar 26 17:09:53 2024 +0800

    [feature](hive/iceberg)add doris's version in table properties (#32774)
    
    issue #31442
    when create a external table, we can add doris's version in table's properties.
---
 .../src/main/java/org/apache/doris/datasource/ExternalCatalog.java   | 3 +++
 .../src/main/java/org/apache/doris/datasource/hive/HiveUtil.java     | 5 ++++-
 .../java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java | 2 ++
 3 files changed, 9 insertions(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
index d9af5557117..2238ee5de95 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
@@ -32,6 +32,7 @@ import org.apache.doris.cluster.ClusterNamespace;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.UserException;
+import org.apache.doris.common.Version;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.common.util.Util;
@@ -85,6 +86,8 @@ public abstract class ExternalCatalog
     private static final Logger LOG = LogManager.getLogger(ExternalCatalog.class);
 
     public static final String ENABLE_AUTO_ANALYZE = "enable.auto.analyze";
+    public static final String DORIS_VERSION = "doris.version";
+    public static final String DORIS_VERSION_VALUE = Version.DORIS_BUILD_VERSION + "-" + Version.DORIS_BUILD_SHORT_HASH;
 
     // Unique id of this catalog, will be assigned after catalog is loaded.
     @SerializedName(value = "id")
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
index cc834894076..e6b5c1095f3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java
@@ -20,6 +20,7 @@ package org.apache.doris.datasource.hive;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.UserException;
+import org.apache.doris.datasource.ExternalCatalog;
 import org.apache.doris.fs.remote.BrokerFileSystem;
 import org.apache.doris.fs.remote.RemoteFileSystem;
 
@@ -192,7 +193,9 @@ public final class HiveUtil {
         // table.setViewOriginalText(hiveTable.getViewSql());
         // table.setViewExpandedText(hiveTable.getViewSql());
         table.setTableType("MANAGED_TABLE");
-        table.setParameters(hiveTable.getProperties());
+        Map<String, String> props = new HashMap<>(hiveTable.getProperties());
+        props.put(ExternalCatalog.DORIS_VERSION, ExternalCatalog.DORIS_VERSION_VALUE);
+        table.setParameters(props);
         return table;
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java
index 2caf3c1d16d..0f43ef21c43 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java
@@ -27,6 +27,7 @@ import org.apache.doris.catalog.StructType;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.UserException;
 import org.apache.doris.datasource.DorisTypeVisitor;
+import org.apache.doris.datasource.ExternalCatalog;
 import org.apache.doris.datasource.ExternalDatabase;
 import org.apache.doris.datasource.operations.ExternalMetadataOps;
 
@@ -117,6 +118,7 @@ public class IcebergMetadataOps implements ExternalMetadataOps {
                 DorisTypeVisitor.visit(structType, new DorisTypeToIcebergType(structType));
         Schema schema = new Schema(visit.asNestedType().asStructType().fields());
         Map<String, String> properties = stmt.getProperties();
+        properties.put(ExternalCatalog.DORIS_VERSION, ExternalCatalog.DORIS_VERSION_VALUE);
         PartitionSpec partitionSpec = IcebergUtils.solveIcebergPartitionSpec(properties, schema);
         catalog.createTable(TableIdentifier.of(dbName, tableName), schema, partitionSpec, properties);
         db.setUnInitialized(true);


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