You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by li...@apache.org on 2022/05/17 01:37:23 UTC

[incubator-doris] branch master updated: [feature-wip](parquet-vec) Support parquet scanner in vectorized engine (#9433)

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

lihaopeng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git


The following commit(s) were added to refs/heads/master by this push:
     new bee5c2f8aa [feature-wip](parquet-vec) Support parquet scanner in vectorized engine (#9433)
bee5c2f8aa is described below

commit bee5c2f8aa25fdaa4589aa808f2629e05d342c2b
Author: yinzhijian <37...@qq.com>
AuthorDate: Tue May 17 09:37:17 2022 +0800

    [feature-wip](parquet-vec) Support parquet scanner in vectorized engine (#9433)
---
 be/src/exec/base_scanner.h                         |   1 +
 be/src/exec/broker_scan_node.cpp                   |  13 +-
 be/src/exec/parquet_reader.cpp                     |  16 +
 be/src/exec/parquet_reader.h                       |   3 +
 be/src/exec/parquet_scanner.h                      |   4 +-
 be/src/http/action/stream_load.cpp                 |  40 +-
 be/src/olap/rowset/segment_v2/rle_page.h           |   1 +
 be/src/vec/CMakeLists.txt                          |   2 +
 be/src/vec/data_types/data_type_factory.cpp        |  63 +++
 be/src/vec/data_types/data_type_factory.hpp        |   3 +
 be/src/vec/exec/vparquet_scanner.cpp               | 311 +++++++++++
 be/src/vec/exec/vparquet_scanner.h                 |  70 +++
 be/src/vec/functions/function_cast.h               |   3 +-
 be/src/vec/utils/arrow_column_to_doris_column.cpp  | 289 ++++++++++
 be/src/vec/utils/arrow_column_to_doris_column.h    |  40 ++
 be/test/CMakeLists.txt                             |   1 +
 .../utils/arrow_column_to_doris_column_test.cpp    | 609 +++++++++++++++++++++
 .../main/java/org/apache/doris/analysis/Expr.java  |  17 +
 .../org/apache/doris/analysis/TupleDescriptor.java |  15 +
 .../main/java/org/apache/doris/catalog/Type.java   |   4 +
 .../src/main/java/org/apache/doris/load/Load.java  | 154 +++++-
 .../org/apache/doris/planner/BrokerScanNode.java   |   4 +-
 .../apache/doris/planner/StreamLoadScanNode.java   |   4 +-
 .../java/org/apache/doris/task/StreamLoadTask.java |   2 +
 24 files changed, 1621 insertions(+), 48 deletions(-)

diff --git a/be/src/exec/base_scanner.h b/be/src/exec/base_scanner.h
index 02c2f56880..2fccc62db8 100644
--- a/be/src/exec/base_scanner.h
+++ b/be/src/exec/base_scanner.h
@@ -35,6 +35,7 @@ class RuntimeState;
 class ExprContext;
 
 namespace vectorized {
+class VExprContext;
 class IColumn;
 using MutableColumnPtr = IColumn::MutablePtr;
 } // namespace vectorized
diff --git a/be/src/exec/broker_scan_node.cpp b/be/src/exec/broker_scan_node.cpp
index c1144495c2..f2f3742cf8 100644
--- a/be/src/exec/broker_scan_node.cpp
+++ b/be/src/exec/broker_scan_node.cpp
@@ -33,6 +33,7 @@
 #include "util/thread.h"
 #include "vec/exec/vbroker_scanner.h"
 #include "vec/exec/vjson_scanner.h"
+#include "vec/exec/vparquet_scanner.h"
 
 namespace doris {
 
@@ -225,9 +226,15 @@ std::unique_ptr<BaseScanner> BrokerScanNode::create_scanner(const TBrokerScanRan
     BaseScanner* scan = nullptr;
     switch (scan_range.ranges[0].format_type) {
     case TFileFormatType::FORMAT_PARQUET:
-        scan = new ParquetScanner(_runtime_state, runtime_profile(), scan_range.params,
-                                  scan_range.ranges, scan_range.broker_addresses,
-                                  _pre_filter_texprs, counter);
+        if (_vectorized) {
+            scan = new vectorized::VParquetScanner(
+                    _runtime_state, runtime_profile(), scan_range.params, scan_range.ranges,
+                    scan_range.broker_addresses, _pre_filter_texprs, counter);
+        } else {
+            scan = new ParquetScanner(_runtime_state, runtime_profile(), scan_range.params,
+                                      scan_range.ranges, scan_range.broker_addresses,
+                                      _pre_filter_texprs, counter);
+        }
         break;
     case TFileFormatType::FORMAT_ORC:
         scan = new ORCScanner(_runtime_state, runtime_profile(), scan_range.params,
diff --git a/be/src/exec/parquet_reader.cpp b/be/src/exec/parquet_reader.cpp
index ddb3531e17..3d2555161c 100644
--- a/be/src/exec/parquet_reader.cpp
+++ b/be/src/exec/parquet_reader.cpp
@@ -219,6 +219,22 @@ Status ParquetReaderWrap::read_record_batch(const std::vector<SlotDescriptor*>&
     return Status::OK();
 }
 
+Status ParquetReaderWrap::next_batch(std::shared_ptr<arrow::RecordBatch>* batch,
+                                     const std::vector<SlotDescriptor*>& tuple_slot_descs,
+                                     bool* eof) {
+    if (_batch->num_rows() == 0 || _current_line_of_batch != 0 || _current_line_of_group != 0) {
+        RETURN_IF_ERROR(read_record_batch(tuple_slot_descs, eof));
+    }
+    *batch = get_batch();
+    return Status::OK();
+}
+
+const std::shared_ptr<arrow::RecordBatch>& ParquetReaderWrap::get_batch() {
+    _current_line_of_batch += _batch->num_rows();
+    _current_line_of_group += _batch->num_rows();
+    return _batch;
+}
+
 Status ParquetReaderWrap::handle_timestamp(const std::shared_ptr<arrow::TimestampArray>& ts_array,
                                            uint8_t* buf, int32_t* wbytes) {
     const auto type = std::static_pointer_cast<arrow::TimestampType>(ts_array->type());
diff --git a/be/src/exec/parquet_reader.h b/be/src/exec/parquet_reader.h
index 2bd5b5a802..c62054800d 100644
--- a/be/src/exec/parquet_reader.h
+++ b/be/src/exec/parquet_reader.h
@@ -79,6 +79,8 @@ public:
     Status size(int64_t* size);
     Status init_parquet_reader(const std::vector<SlotDescriptor*>& tuple_slot_descs,
                                const std::string& timezone);
+    Status next_batch(std::shared_ptr<arrow::RecordBatch>* batch,
+                      const std::vector<SlotDescriptor*>& tuple_slot_descs, bool* eof);
 
 private:
     void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value,
@@ -86,6 +88,7 @@ private:
     Status column_indices(const std::vector<SlotDescriptor*>& tuple_slot_descs);
     Status set_field_null(Tuple* tuple, const SlotDescriptor* slot_desc);
     Status read_record_batch(const std::vector<SlotDescriptor*>& tuple_slot_descs, bool* eof);
+    const std::shared_ptr<arrow::RecordBatch>& get_batch();
     Status handle_timestamp(const std::shared_ptr<arrow::TimestampArray>& ts_array, uint8_t* buf,
                             int32_t* wbtyes);
 
diff --git a/be/src/exec/parquet_scanner.h b/be/src/exec/parquet_scanner.h
index 1e54c05d93..a34e4a19e6 100644
--- a/be/src/exec/parquet_scanner.h
+++ b/be/src/exec/parquet_scanner.h
@@ -65,11 +65,11 @@ public:
     // Close this scanner
     virtual void close();
 
-private:
+protected:
     // Read next buffer from reader
     Status open_next_reader();
 
-private:
+protected:
     //const TBrokerScanRangeParams& _params;
     const std::vector<TBrokerRangeDesc>& _ranges;
     const std::vector<TNetworkAddress>& _broker_addresses;
diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp
index 583026d69f..9efab0f975 100644
--- a/be/src/http/action/stream_load.cpp
+++ b/be/src/http/action/stream_load.cpp
@@ -78,27 +78,29 @@ static TFileFormatType::type parse_format(const std::string& format_str,
         return parse_format("CSV", compress_type);
     }
     TFileFormatType::type format_type = TFileFormatType::FORMAT_UNKNOWN;
-    if (boost::iequals(format_str, "CSV")) {
+    if (iequal(format_str, "CSV")) {
         if (compress_type.empty()) {
             format_type = TFileFormatType::FORMAT_CSV_PLAIN;
         }
-        if (boost::iequals(compress_type, "GZ")) {
+        if (iequal(compress_type, "GZ")) {
             format_type = TFileFormatType::FORMAT_CSV_GZ;
-        } else if (boost::iequals(compress_type, "LZO")) {
+        } else if (iequal(compress_type, "LZO")) {
             format_type = TFileFormatType::FORMAT_CSV_LZO;
-        } else if (boost::iequals(compress_type, "BZ2")) {
+        } else if (iequal(compress_type, "BZ2")) {
             format_type = TFileFormatType::FORMAT_CSV_BZ2;
-        } else if (boost::iequals(compress_type, "LZ4FRAME")) {
+        } else if (iequal(compress_type, "LZ4FRAME")) {
             format_type = TFileFormatType::FORMAT_CSV_LZ4FRAME;
-        } else if (boost::iequals(compress_type, "LZOP")) {
+        } else if (iequal(compress_type, "LZOP")) {
             format_type = TFileFormatType::FORMAT_CSV_LZOP;
-        } else if (boost::iequals(compress_type, "DEFLATE")) {
+        } else if (iequal(compress_type, "DEFLATE")) {
             format_type = TFileFormatType::FORMAT_CSV_DEFLATE;
         }
-    } else if (boost::iequals(format_str, "JSON")) {
+    } else if (iequal(format_str, "JSON")) {
         if (compress_type.empty()) {
             format_type = TFileFormatType::FORMAT_JSON;
         }
+    } else if (iequal(format_str, "PARQUET")) {
+        format_type = TFileFormatType::FORMAT_PARQUET;
     }
     return format_type;
 }
@@ -264,12 +266,12 @@ Status StreamLoadAction::_on_header(HttpRequest* http_req, StreamLoadContext* ct
 
     // get format of this put
     if (!http_req->header(HTTP_COMPRESS_TYPE).empty() &&
-        boost::iequals(http_req->header(HTTP_FORMAT_KEY), "JSON")) {
+        iequal(http_req->header(HTTP_FORMAT_KEY), "JSON")) {
         return Status::InternalError("compress data of JSON format is not supported.");
     }
     std::string format_str = http_req->header(HTTP_FORMAT_KEY);
-    if (boost::iequals(format_str, BeConsts::CSV_WITH_NAMES) ||
-        boost::iequals(format_str, BeConsts::CSV_WITH_NAMES_AND_TYPES)) {
+    if (iequal(format_str, BeConsts::CSV_WITH_NAMES) ||
+        iequal(format_str, BeConsts::CSV_WITH_NAMES_AND_TYPES)) {
         ctx->header_type = format_str;
         //treat as CSV
         format_str = BeConsts::CSV;
@@ -291,7 +293,7 @@ Status StreamLoadAction::_on_header(HttpRequest* http_req, StreamLoadContext* ct
     size_t json_max_body_bytes = config::streaming_load_json_max_mb * 1024 * 1024;
     bool read_json_by_line = false;
     if (!http_req->header(HTTP_READ_JSON_BY_LINE).empty()) {
-        if (boost::iequals(http_req->header(HTTP_READ_JSON_BY_LINE), "true")) {
+        if (iequal(http_req->header(HTTP_READ_JSON_BY_LINE), "true")) {
             read_json_by_line = true;
         }
     }
@@ -440,9 +442,9 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext*
         request.__set_negative(false);
     }
     if (!http_req->header(HTTP_STRICT_MODE).empty()) {
-        if (boost::iequals(http_req->header(HTTP_STRICT_MODE), "false")) {
+        if (iequal(http_req->header(HTTP_STRICT_MODE), "false")) {
             request.__set_strictMode(false);
-        } else if (boost::iequals(http_req->header(HTTP_STRICT_MODE), "true")) {
+        } else if (iequal(http_req->header(HTTP_STRICT_MODE), "true")) {
             request.__set_strictMode(true);
         } else {
             return Status::InvalidArgument("Invalid strict mode format. Must be bool type");
@@ -465,7 +467,7 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext*
         request.__set_json_root(http_req->header(HTTP_JSONROOT));
     }
     if (!http_req->header(HTTP_STRIP_OUTER_ARRAY).empty()) {
-        if (boost::iequals(http_req->header(HTTP_STRIP_OUTER_ARRAY), "true")) {
+        if (iequal(http_req->header(HTTP_STRIP_OUTER_ARRAY), "true")) {
             request.__set_strip_outer_array(true);
         } else {
             request.__set_strip_outer_array(false);
@@ -474,7 +476,7 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext*
         request.__set_strip_outer_array(false);
     }
     if (!http_req->header(HTTP_NUM_AS_STRING).empty()) {
-        if (boost::iequals(http_req->header(HTTP_NUM_AS_STRING), "true")) {
+        if (iequal(http_req->header(HTTP_NUM_AS_STRING), "true")) {
             request.__set_num_as_string(true);
         } else {
             request.__set_num_as_string(false);
@@ -483,7 +485,7 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext*
         request.__set_num_as_string(false);
     }
     if (!http_req->header(HTTP_FUZZY_PARSE).empty()) {
-        if (boost::iequals(http_req->header(HTTP_FUZZY_PARSE), "true")) {
+        if (iequal(http_req->header(HTTP_FUZZY_PARSE), "true")) {
             request.__set_fuzzy_parse(true);
         } else {
             request.__set_fuzzy_parse(false);
@@ -493,7 +495,7 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext*
     }
 
     if (!http_req->header(HTTP_READ_JSON_BY_LINE).empty()) {
-        if (boost::iequals(http_req->header(HTTP_READ_JSON_BY_LINE), "true")) {
+        if (iequal(http_req->header(HTTP_READ_JSON_BY_LINE), "true")) {
             request.__set_read_json_by_line(true);
         } else {
             request.__set_read_json_by_line(false);
@@ -517,7 +519,7 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext*
     }
 
     if (!http_req->header(HTTP_LOAD_TO_SINGLE_TABLET).empty()) {
-        if (boost::iequals(http_req->header(HTTP_LOAD_TO_SINGLE_TABLET), "true")) {
+        if (iequal(http_req->header(HTTP_LOAD_TO_SINGLE_TABLET), "true")) {
             request.__set_load_to_single_tablet(true);
         } else {
             request.__set_load_to_single_tablet(false);
diff --git a/be/src/olap/rowset/segment_v2/rle_page.h b/be/src/olap/rowset/segment_v2/rle_page.h
index 5944a47c4f..c6eb413aa7 100644
--- a/be/src/olap/rowset/segment_v2/rle_page.h
+++ b/be/src/olap/rowset/segment_v2/rle_page.h
@@ -104,6 +104,7 @@ public:
 
     void reset() override {
         _count = 0;
+        _finished = false;
         _rle_encoder->Clear();
         _rle_encoder->Reserve(RLE_PAGE_HEADER_SIZE, 0);
     }
diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt
index 22fa489f46..265d6fd884 100644
--- a/be/src/vec/CMakeLists.txt
+++ b/be/src/vec/CMakeLists.txt
@@ -102,6 +102,7 @@ set(VEC_FILES
   exec/vbroker_scan_node.cpp
   exec/vbroker_scanner.cpp
   exec/vjson_scanner.cpp
+  exec/vparquet_scanner.cpp
   exec/join/vhash_join_node.cpp
   exprs/vectorized_agg_fn.cpp
   exprs/vectorized_fn_call.cpp
@@ -191,6 +192,7 @@ set(VEC_FILES
   runtime/vdata_stream_recvr.cpp
   runtime/vdata_stream_mgr.cpp
   runtime/vpartition_info.cpp
+  utils/arrow_column_to_doris_column.cpp
   runtime/vsorted_run_merger.cpp)
 
 add_library(Vec STATIC
diff --git a/be/src/vec/data_types/data_type_factory.cpp b/be/src/vec/data_types/data_type_factory.cpp
index 555c6417ce..557b978c8e 100644
--- a/be/src/vec/data_types/data_type_factory.cpp
+++ b/be/src/vec/data_types/data_type_factory.cpp
@@ -260,4 +260,67 @@ DataTypePtr DataTypeFactory::create_data_type(const PColumnMeta& pcolumn) {
     return nested;
 }
 
+DataTypePtr DataTypeFactory::create_data_type(const arrow::Type::type& type, bool is_nullable) {
+    DataTypePtr nested = nullptr;
+    switch (type) {
+    case ::arrow::Type::BOOL:
+        nested = std::make_shared<vectorized::DataTypeUInt8>();
+        break;
+    case ::arrow::Type::INT8:
+        nested = std::make_shared<vectorized::DataTypeInt8>();
+        break;
+    case ::arrow::Type::UINT8:
+        nested = std::make_shared<vectorized::DataTypeUInt8>();
+        break;
+    case ::arrow::Type::INT16:
+        nested = std::make_shared<vectorized::DataTypeInt16>();
+        break;
+    case ::arrow::Type::UINT16:
+        nested = std::make_shared<vectorized::DataTypeUInt16>();
+        break;
+    case ::arrow::Type::INT32:
+        nested = std::make_shared<vectorized::DataTypeInt32>();
+        break;
+    case ::arrow::Type::UINT32:
+        nested = std::make_shared<vectorized::DataTypeUInt32>();
+        break;
+    case ::arrow::Type::INT64:
+        nested = std::make_shared<vectorized::DataTypeInt64>();
+        break;
+    case ::arrow::Type::UINT64:
+        nested = std::make_shared<vectorized::DataTypeUInt64>();
+        break;
+    case ::arrow::Type::HALF_FLOAT:
+    case ::arrow::Type::FLOAT:
+        nested = std::make_shared<vectorized::DataTypeFloat32>();
+        break;
+    case ::arrow::Type::DOUBLE:
+        nested = std::make_shared<vectorized::DataTypeFloat64>();
+        break;
+    case ::arrow::Type::DATE32:
+        nested = std::make_shared<vectorized::DataTypeDate>();
+        break;
+    case ::arrow::Type::DATE64:
+    case ::arrow::Type::TIMESTAMP:
+        nested = std::make_shared<vectorized::DataTypeDateTime>();
+        break;
+    case ::arrow::Type::BINARY:
+    case ::arrow::Type::FIXED_SIZE_BINARY:
+    case ::arrow::Type::STRING:
+        nested = std::make_shared<vectorized::DataTypeString>();
+        break;
+    case ::arrow::Type::DECIMAL:
+        nested = std::make_shared<vectorized::DataTypeDecimal<vectorized::Decimal128>>(27, 9);
+        break;
+    default:
+        DCHECK(false) << "invalid arrow type:" << (int)type;
+        break;
+    }
+
+    if (nested && is_nullable) {
+        return std::make_shared<vectorized::DataTypeNullable>(nested);
+    }
+    return nested;
+}
+
 } // namespace doris::vectorized
diff --git a/be/src/vec/data_types/data_type_factory.hpp b/be/src/vec/data_types/data_type_factory.hpp
index 4ecb290487..3b667c6f72 100644
--- a/be/src/vec/data_types/data_type_factory.hpp
+++ b/be/src/vec/data_types/data_type_factory.hpp
@@ -22,6 +22,7 @@
 #include <mutex>
 #include <string>
 
+#include "arrow/type.h"
 #include "gen_cpp/data.pb.h"
 #include "olap/field.h"
 #include "olap/tablet_schema.h"
@@ -87,6 +88,8 @@ public:
 
     DataTypePtr create_data_type(const PColumnMeta& pcolumn);
 
+    DataTypePtr create_data_type(const arrow::Type::type& type, bool is_nullable);
+
 private:
     DataTypePtr _create_primitive_data_type(const FieldType& type) const;
 
diff --git a/be/src/vec/exec/vparquet_scanner.cpp b/be/src/vec/exec/vparquet_scanner.cpp
new file mode 100644
index 0000000000..6a891850a7
--- /dev/null
+++ b/be/src/vec/exec/vparquet_scanner.cpp
@@ -0,0 +1,311 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "vec/exec/vparquet_scanner.h"
+
+#include "exec/parquet_reader.h"
+#include "exprs/expr.h"
+#include "runtime/descriptors.h"
+#include "runtime/exec_env.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/functions/simple_function_factory.h"
+#include "vec/utils/arrow_column_to_doris_column.h"
+
+namespace doris::vectorized {
+
+VParquetScanner::VParquetScanner(RuntimeState* state, RuntimeProfile* profile,
+                                 const TBrokerScanRangeParams& params,
+                                 const std::vector<TBrokerRangeDesc>& ranges,
+                                 const std::vector<TNetworkAddress>& broker_addresses,
+                                 const std::vector<TExpr>& pre_filter_texprs,
+                                 ScannerCounter* counter)
+        : ParquetScanner(state, profile, params, ranges, broker_addresses, pre_filter_texprs,
+                         counter),
+          _batch(nullptr),
+          _arrow_batch_cur_idx(0),
+          _num_of_columns_from_file(0) {}
+VParquetScanner::~VParquetScanner() {}
+
+Status VParquetScanner::open() {
+    RETURN_IF_ERROR(ParquetScanner::open());
+    if (_ranges.empty()) {
+        return Status::OK();
+    }
+    auto range = _ranges[0];
+    _num_of_columns_from_file = range.__isset.num_of_columns_from_file
+                                        ? implicit_cast<int>(range.num_of_columns_from_file)
+                                        : implicit_cast<int>(_src_slot_descs.size());
+
+    // check consistency
+    if (range.__isset.num_of_columns_from_file) {
+        int size = range.columns_from_path.size();
+        for (const auto& r : _ranges) {
+            if (r.columns_from_path.size() != size) {
+                return Status::InternalError("ranges have different number of columns.");
+            }
+        }
+    }
+    return Status::OK();
+}
+
+// get next available arrow batch
+Status VParquetScanner::_next_arrow_batch() {
+    _arrow_batch_cur_idx = 0;
+    // first, init file reader
+    if (_cur_file_reader == nullptr || _cur_file_eof) {
+        RETURN_IF_ERROR(open_next_reader());
+        _cur_file_eof = false;
+    }
+    // second, loop until find available arrow batch or EOF
+    while (!_scanner_eof) {
+        RETURN_IF_ERROR(_cur_file_reader->next_batch(&_batch, _src_slot_descs, &_cur_file_eof));
+        if (_cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            _cur_file_eof = false;
+            continue;
+        }
+        if (_batch->num_rows() == 0) {
+            continue;
+        }
+        return Status::OK();
+    }
+    return Status::EndOfFile("EOF");
+}
+
+Status VParquetScanner::_init_arrow_batch_if_necessary() {
+    // 1. init batch if first time
+    // 2. reset reader if end of file
+    Status status;
+    if (_scanner_eof) {
+        return Status::EndOfFile("EOF");
+    }
+    if (_batch == nullptr || _arrow_batch_cur_idx >= _batch->num_rows()) {
+        return _next_arrow_batch();
+    }
+    return status;
+}
+
+Status VParquetScanner::_init_src_block(Block* block) {
+    size_t batch_pos = 0;
+    block->clear();
+    for (auto i = 0; i < _num_of_columns_from_file; ++i) {
+        SlotDescriptor* slot_desc = _src_slot_descs[i];
+        if (slot_desc == nullptr) {
+            continue;
+        }
+        auto* array = _batch->column(batch_pos++).get();
+        // let src column be nullable for simplify converting
+        // TODO, support not nullable for exec efficiently
+        auto is_nullable = true;
+        DataTypePtr data_type =
+                DataTypeFactory::instance().create_data_type(array->type()->id(), is_nullable);
+        if (data_type == nullptr) {
+            return Status::NotSupported(
+                    fmt::format("Not support arrow type:{}", array->type()->name()));
+        }
+        MutableColumnPtr data_column = data_type->create_column();
+        block->insert(
+                ColumnWithTypeAndName(std::move(data_column), data_type, slot_desc->col_name()));
+    }
+    return Status::OK();
+}
+
+Status VParquetScanner::get_next(vectorized::Block* block, bool* eof) {
+    // overall of type converting:
+    // arrow type ==arrow_column_to_doris_column==> primitive type(PT0) ==cast_src_block==>
+    // primitive type(PT1) ==materialize_block==> dest primitive type
+
+    // first, we need to convert the arrow type to the corresponding internal type,
+    // such as arrow::INT16 to TYPE_SMALLINT(PT0).
+    // why need first step? we cannot convert the arrow type to type in src desc directly,
+    // it's too hard to achieve.
+
+    // second, convert PT0 to the type in src desc, such as TYPE_SMALLINT to TYPE_VARCHAR.(PT1)
+    // why need second step? the materialize step only accepts types specified in src desc.
+
+    // finally, through the materialized, convert to the type in dest desc, such as TYPE_DATETIME.
+    SCOPED_TIMER(_read_timer);
+    // init arrow batch
+    {
+        Status st = _init_arrow_batch_if_necessary();
+        if (!st.ok()) {
+            if (!st.is_end_of_file()) {
+                return st;
+            }
+            *eof = true;
+            return Status::OK();
+        }
+    }
+    Block src_block;
+    RETURN_IF_ERROR(_init_src_block(&src_block));
+    // convert arrow batch to block until reach the batch_size
+    while (!_scanner_eof) {
+        // cast arrow type to PT0 and append it to src block
+        // for example: arrow::Type::INT16 => TYPE_SMALLINT
+        RETURN_IF_ERROR(_append_batch_to_src_block(&src_block));
+        // finalize the src block if full
+        if (src_block.rows() >= _state->batch_size()) {
+            break;
+        }
+        auto status = _next_arrow_batch();
+        // if ok, append the batch to the src columns
+        if (status.ok()) {
+            continue;
+        }
+        // return error if not EOF
+        if (!status.is_end_of_file()) {
+            return status;
+        }
+        _cur_file_eof = true;
+        break;
+    }
+    COUNTER_UPDATE(_rows_read_counter, src_block.rows());
+    SCOPED_TIMER(_materialize_timer);
+    // cast PT0 => PT1
+    // for example: TYPE_SMALLINT => TYPE_VARCHAR
+    RETURN_IF_ERROR(_cast_src_block(&src_block));
+    // range of current file
+    _fill_columns_from_path(&src_block);
+    RETURN_IF_ERROR(_eval_conjunts(&src_block));
+    // materialize, src block => dest columns
+    RETURN_IF_ERROR(_materialize_block(&src_block, block));
+    *eof = _scanner_eof;
+    return Status::OK();
+}
+
+// eval conjuncts, for example: t1 > 1
+Status VParquetScanner::_eval_conjunts(Block* block) {
+    for (auto& vctx : _vpre_filter_ctxs) {
+        size_t orig_rows = block->rows();
+        RETURN_IF_ERROR(VExprContext::filter_block(vctx, block, block->columns()));
+        _counter->num_rows_unselected += orig_rows - block->rows();
+    }
+    return Status::OK();
+}
+
+void VParquetScanner::_fill_columns_from_path(Block* block) {
+    const TBrokerRangeDesc& range = _ranges.at(_next_range - 1);
+    if (range.__isset.num_of_columns_from_file) {
+        size_t start = range.num_of_columns_from_file;
+        size_t rows = block->rows();
+        for (size_t i = 0; i < range.columns_from_path.size(); ++i) {
+            auto slot_desc = _src_slot_descs.at(i + start);
+            if (slot_desc == nullptr) continue;
+            auto is_nullable = slot_desc->is_nullable();
+            DataTypePtr data_type =
+                    DataTypeFactory::instance().create_data_type(TYPE_VARCHAR, is_nullable);
+            MutableColumnPtr data_column = data_type->create_column();
+            const std::string& column_from_path = range.columns_from_path[i];
+            for (size_t i = 0; i < rows; ++i) {
+                data_column->insert_data(const_cast<char*>(column_from_path.c_str()),
+                                         column_from_path.size());
+            }
+            block->insert(ColumnWithTypeAndName(std::move(data_column), data_type,
+                                                slot_desc->col_name()));
+        }
+    }
+}
+
+Status VParquetScanner::_materialize_block(Block* block, Block* dest_block) {
+    int ctx_idx = 0;
+    size_t orig_rows = block->rows();
+    auto filter_column = ColumnUInt8::create(orig_rows, 1);
+    for (auto slot_desc : _dest_tuple_desc->slots()) {
+        if (!slot_desc->is_materialized()) {
+            continue;
+        }
+        int dest_index = ctx_idx++;
+
+        VExprContext* ctx = _dest_vexpr_ctx[dest_index];
+        int result_column_id = 0;
+        // PT1 => dest primitive type
+        RETURN_IF_ERROR(ctx->execute(block, &result_column_id));
+        ColumnPtr& ptr = block->safe_get_by_position(result_column_id).column;
+        if (!slot_desc->is_nullable()) {
+            if (auto* nullable_column = check_and_get_column<ColumnNullable>(*ptr)) {
+                if (nullable_column->has_null()) {
+                    // fill filter if src has null value and dest column is not nullable
+                    IColumn::Filter& filter = assert_cast<ColumnUInt8&>(*filter_column).get_data();
+                    const ColumnPtr& null_column_ptr = nullable_column->get_null_map_column_ptr();
+                    const auto& column_data =
+                            assert_cast<const ColumnUInt8&>(*null_column_ptr).get_data();
+                    for (size_t i = 0; i < null_column_ptr->size(); ++i) {
+                        filter[i] &= !column_data[i];
+                    }
+                }
+                ptr = nullable_column->get_nested_column_ptr();
+            }
+        }
+        dest_block->insert(vectorized::ColumnWithTypeAndName(
+                std::move(ptr), slot_desc->get_data_type_ptr(), slot_desc->col_name()));
+    }
+    size_t dest_size = dest_block->columns();
+    // do filter
+    dest_block->insert(vectorized::ColumnWithTypeAndName(
+            std::move(filter_column), std::make_shared<vectorized::DataTypeUInt8>(),
+            "filter column"));
+    RETURN_IF_ERROR(Block::filter_block(dest_block, dest_size, dest_size));
+    _counter->num_rows_filtered += orig_rows - dest_block->rows();
+    return Status::OK();
+}
+
+// arrow type ==arrow_column_to_doris_column==> primitive type(PT0) ==cast_src_block==>
+// primitive type(PT1) ==materialize_block==> dest primitive type
+Status VParquetScanner::_cast_src_block(Block* block) {
+    // cast primitive type(PT0) to primitive type(PT1)
+    for (size_t i = 0; i < _num_of_columns_from_file; ++i) {
+        SlotDescriptor* slot_desc = _src_slot_descs[i];
+        if (slot_desc == nullptr) {
+            continue;
+        }
+        auto& arg = block->get_by_name(slot_desc->col_name());
+        // remove nullable here, let the get_function decide whether nullable
+        auto return_type = slot_desc->get_data_type_ptr();
+        ColumnsWithTypeAndName arguments {
+                arg,
+                {DataTypeString().create_column_const(
+                         arg.column->size(), remove_nullable(return_type)->get_family_name()),
+                 std::make_shared<DataTypeString>(), ""}};
+        auto func_cast =
+                SimpleFunctionFactory::instance().get_function("CAST", arguments, return_type);
+        RETURN_IF_ERROR(func_cast->execute(nullptr, *block, {i}, i, arg.column->size()));
+        block->get_by_position(i).type = std::move(return_type);
+    }
+    return Status::OK();
+}
+
+Status VParquetScanner::_append_batch_to_src_block(Block* block) {
+    size_t num_elements = std::min<size_t>((_state->batch_size() - block->rows()),
+                                           (_batch->num_rows() - _arrow_batch_cur_idx));
+    size_t column_pos = 0;
+    for (auto i = 0; i < _num_of_columns_from_file; ++i) {
+        SlotDescriptor* slot_desc = _src_slot_descs[i];
+        if (slot_desc == nullptr) {
+            continue;
+        }
+        auto* array = _batch->column(column_pos++).get();
+        auto& column_with_type_and_name = block->get_by_name(slot_desc->col_name());
+        RETURN_IF_ERROR(arrow_column_to_doris_column(array, _arrow_batch_cur_idx,
+                                                     column_with_type_and_name, num_elements,
+                                                     _state->timezone()));
+    }
+
+    _arrow_batch_cur_idx += num_elements;
+    return Status::OK();
+}
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/exec/vparquet_scanner.h b/be/src/vec/exec/vparquet_scanner.h
new file mode 100644
index 0000000000..754aabc4c3
--- /dev/null
+++ b/be/src/vec/exec/vparquet_scanner.h
@@ -0,0 +1,70 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <arrow/array.h>
+#include <exec/parquet_scanner.h>
+
+#include <map>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "runtime/mem_pool.h"
+#include "util/runtime_profile.h"
+
+namespace doris::vectorized {
+
+// VParquet scanner convert the data read from Parquet to doris's columns.
+class VParquetScanner : public ParquetScanner {
+public:
+    VParquetScanner(RuntimeState* state, RuntimeProfile* profile,
+                    const TBrokerScanRangeParams& params,
+                    const std::vector<TBrokerRangeDesc>& ranges,
+                    const std::vector<TNetworkAddress>& broker_addresses,
+                    const std::vector<TExpr>& pre_filter_texprs, ScannerCounter* counter);
+
+    virtual ~VParquetScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open();
+
+    Status get_next(Block* block, bool* eof);
+
+private:
+    Status _next_arrow_batch();
+    Status _init_arrow_batch_if_necessary();
+    Status _init_src_block(Block* block);
+    Status _append_batch_to_src_block(Block* block);
+    Status _cast_src_block(Block* block);
+    Status _eval_conjunts(Block* block);
+    Status _materialize_block(Block* block, Block* dest_block);
+    void _fill_columns_from_path(Block* block);
+
+private:
+    std::shared_ptr<arrow::RecordBatch> _batch;
+    size_t _arrow_batch_cur_idx;
+    int _num_of_columns_from_file;
+};
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h
index 092842f5fa..071874f30d 100644
--- a/be/src/vec/functions/function_cast.h
+++ b/be/src/vec/functions/function_cast.h
@@ -965,8 +965,9 @@ private:
                    !(check_and_get_data_type<DataTypeDateTime>(from_type.get()) ||
                      check_and_get_data_type<DataTypeDate>(from_type.get()))) {
             function = FunctionConvertToTimeType<DataType, NameCast>::create();
-        } else
+        } else {
             function = FunctionTo<DataType>::Type::create();
+        }
 
         /// Check conversion using underlying function
         { function->get_return_type(ColumnsWithTypeAndName(1, {nullptr, from_type, ""})); }
diff --git a/be/src/vec/utils/arrow_column_to_doris_column.cpp b/be/src/vec/utils/arrow_column_to_doris_column.cpp
new file mode 100644
index 0000000000..a02fc92b48
--- /dev/null
+++ b/be/src/vec/utils/arrow_column_to_doris_column.cpp
@@ -0,0 +1,289 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "vec/utils/arrow_column_to_doris_column.h"
+
+#include <arrow/array.h>
+#include <arrow/record_batch.h>
+#include <arrow/status.h>
+
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/scalar.h"
+#include "arrow/type.h"
+#include "arrow/type_fwd.h"
+#include "arrow/type_traits.h"
+#include "gutil/casts.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/data_types/data_type_decimal.h"
+#include "vec/runtime/vdatetime_value.h"
+
+#define FOR_ARROW_TYPES(M)                            \
+    M(::arrow::Type::BOOL, TYPE_BOOLEAN)              \
+    M(::arrow::Type::INT8, TYPE_TINYINT)              \
+    M(::arrow::Type::UINT8, TYPE_TINYINT)             \
+    M(::arrow::Type::INT16, TYPE_SMALLINT)            \
+    M(::arrow::Type::UINT16, TYPE_SMALLINT)           \
+    M(::arrow::Type::INT32, TYPE_INT)                 \
+    M(::arrow::Type::UINT32, TYPE_INT)                \
+    M(::arrow::Type::INT64, TYPE_BIGINT)              \
+    M(::arrow::Type::UINT64, TYPE_BIGINT)             \
+    M(::arrow::Type::HALF_FLOAT, TYPE_FLOAT)          \
+    M(::arrow::Type::FLOAT, TYPE_FLOAT)               \
+    M(::arrow::Type::DOUBLE, TYPE_DOUBLE)             \
+    M(::arrow::Type::BINARY, TYPE_VARCHAR)            \
+    M(::arrow::Type::FIXED_SIZE_BINARY, TYPE_VARCHAR) \
+    M(::arrow::Type::STRING, TYPE_VARCHAR)            \
+    M(::arrow::Type::TIMESTAMP, TYPE_DATETIME)        \
+    M(::arrow::Type::DATE32, TYPE_DATE)               \
+    M(::arrow::Type::DATE64, TYPE_DATETIME)           \
+    M(::arrow::Type::DECIMAL, TYPE_DECIMALV2)
+
+#define FOR_ARROW_NUMERIC_TYPES(M)      \
+    M(arrow::Type::UINT8, UInt8)        \
+    M(arrow::Type::INT8, Int8)          \
+    M(arrow::Type::INT16, Int16)        \
+    M(arrow::Type::UINT16, UInt16)      \
+    M(arrow::Type::INT32, Int32)        \
+    M(arrow::Type::UINT32, UInt32)      \
+    M(arrow::Type::UINT64, UInt64)      \
+    M(arrow::Type::INT64, Int64)        \
+    M(arrow::Type::HALF_FLOAT, Float32) \
+    M(arrow::Type::FLOAT, Float32)      \
+    M(arrow::Type::DOUBLE, Float64)
+
+namespace doris::vectorized {
+
+const PrimitiveType arrow_type_to_primitive_type(::arrow::Type::type type) {
+    switch (type) {
+#define DISPATCH(ARROW_TYPE, CPP_TYPE) \
+    case ARROW_TYPE:                   \
+        return CPP_TYPE;
+        FOR_ARROW_TYPES(DISPATCH)
+#undef DISPATCH
+    default:
+        break;
+    }
+    return INVALID_TYPE;
+}
+
+static size_t fill_nullable_column(const arrow::Array* array, size_t array_idx,
+                                   vectorized::ColumnNullable* nullable_column,
+                                   size_t num_elements) {
+    size_t null_elements_count = 0;
+    NullMap& map_data = nullable_column->get_null_map_data();
+    for (size_t i = 0; i < num_elements; ++i) {
+        auto is_null = array->IsNull(array_idx + i);
+        map_data.emplace_back(is_null);
+        null_elements_count += is_null;
+    }
+    return null_elements_count;
+}
+
+/// Inserts chars and offsets right into internal column data to reduce an overhead.
+/// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars.
+/// Also internal strings are null terminated.
+static Status convert_column_with_string_data(const arrow::Array* array, size_t array_idx,
+                                              MutableColumnPtr& data_column, size_t num_elements) {
+    PaddedPODArray<UInt8>& column_chars_t = assert_cast<ColumnString&>(*data_column).get_chars();
+    PaddedPODArray<UInt32>& column_offsets = assert_cast<ColumnString&>(*data_column).get_offsets();
+
+    auto concrete_array = down_cast<const arrow::BinaryArray*>(array);
+    std::shared_ptr<arrow::Buffer> buffer = concrete_array->value_data();
+
+    for (size_t offset_i = array_idx; offset_i < array_idx + num_elements; ++offset_i) {
+        if (!concrete_array->IsNull(offset_i) && buffer) {
+            const auto* raw_data = buffer->data() + concrete_array->value_offset(offset_i);
+            column_chars_t.insert(raw_data, raw_data + concrete_array->value_length(offset_i));
+        }
+        column_chars_t.emplace_back('\0');
+
+        column_offsets.emplace_back(column_chars_t.size());
+    }
+    return Status::OK();
+}
+
+static Status convert_column_with_fixed_size_data(const arrow::Array* array, size_t array_idx,
+                                                  MutableColumnPtr& data_column,
+                                                  size_t num_elements) {
+    PaddedPODArray<UInt8>& column_chars_t = assert_cast<ColumnString&>(*data_column).get_chars();
+    PaddedPODArray<UInt32>& column_offsets = assert_cast<ColumnString&>(*data_column).get_offsets();
+
+    auto concrete_array = down_cast<const arrow::FixedSizeBinaryArray*>(array);
+    uint32_t width = concrete_array->byte_width();
+    const auto* array_data = concrete_array->GetValue(array_idx);
+
+    for (size_t offset_i = 0; offset_i < num_elements; ++offset_i) {
+        if (!concrete_array->IsNull(offset_i)) {
+            const auto* raw_data = array_data + (offset_i * width);
+            column_chars_t.insert(raw_data, raw_data + width);
+        }
+        column_chars_t.emplace_back('\0');
+        column_offsets.emplace_back(column_chars_t.size());
+    }
+    return Status::OK();
+}
+
+/// Inserts numeric data right into internal column data to reduce an overhead
+template <typename NumericType, typename VectorType = ColumnVector<NumericType>>
+static Status convert_column_with_numeric_data(const arrow::Array* array, size_t array_idx,
+                                               MutableColumnPtr& data_column, size_t num_elements) {
+    auto& column_data = static_cast<VectorType&>(*data_column).get_data();
+    /// buffers[0] is a null bitmap and buffers[1] are actual values
+    std::shared_ptr<arrow::Buffer> buffer = array->data()->buffers[1];
+    const auto* raw_data = reinterpret_cast<const NumericType*>(buffer->data()) + array_idx;
+    column_data.insert(raw_data, raw_data + num_elements);
+    return Status::OK();
+}
+
+static Status convert_column_with_boolean_data(const arrow::Array* array, size_t array_idx,
+                                               MutableColumnPtr& data_column, size_t num_elements) {
+    auto& column_data = static_cast<ColumnVector<UInt8>&>(*data_column).get_data();
+    auto concrete_array = down_cast<const arrow::BooleanArray*>(array);
+    for (size_t bool_i = array_idx; bool_i < array_idx + num_elements; ++bool_i) {
+        column_data.emplace_back(concrete_array->Value(bool_i));
+    }
+    return Status::OK();
+}
+
+static int64_t time_unit_divisor(arrow::TimeUnit::type unit) {
+    // Doris only supports seconds
+    switch (unit) {
+    case arrow::TimeUnit::type::SECOND: {
+        return 1L;
+    }
+    case arrow::TimeUnit::type::MILLI: {
+        return 1000L;
+    }
+    case arrow::TimeUnit::type::MICRO: {
+        return 1000000L;
+    }
+    case arrow::TimeUnit::type::NANO: {
+        return 1000000000L;
+    }
+    default:
+        return 0L;
+    }
+}
+
+template <typename ArrowType>
+static Status convert_column_with_timestamp_data(const arrow::Array* array, size_t array_idx,
+                                                 MutableColumnPtr& data_column, size_t num_elements,
+                                                 const std::string& timezone) {
+    auto& column_data = static_cast<ColumnVector<Int64>&>(*data_column).get_data();
+    auto concrete_array = down_cast<const ArrowType*>(array);
+    int64_t divisor = 1;
+    int64_t multiplier = 1;
+    if constexpr (std::is_same_v<ArrowType, arrow::TimestampArray>) {
+        const auto type = std::static_pointer_cast<arrow::TimestampType>(array->type());
+        divisor = time_unit_divisor(type->unit());
+        if (divisor == 0L) {
+            return Status::InternalError(fmt::format("Invalid Time Type:{}", type->name()));
+        }
+    } else if constexpr (std::is_same_v<ArrowType, arrow::Date32Array>) {
+        multiplier = 24 * 60 * 60; // day => secs
+    } else if constexpr (std::is_same_v<ArrowType, arrow::Date64Array>) {
+        divisor = 1000; //ms => secs
+    }
+
+    for (size_t value_i = array_idx; value_i < array_idx + num_elements; ++value_i) {
+        VecDateTimeValue v;
+        v.from_unixtime(static_cast<Int64>(concrete_array->Value(value_i)) / divisor * multiplier,
+                        timezone);
+        if constexpr (std::is_same_v<ArrowType, arrow::Date32Array>) {
+            v.cast_to_date();
+        }
+        column_data.emplace_back(binary_cast<VecDateTimeValue, Int64>(v));
+    }
+    return Status::OK();
+}
+
+static Status convert_column_with_decimal_data(const arrow::Array* array, size_t array_idx,
+                                               MutableColumnPtr& data_column, size_t num_elements) {
+    auto& column_data =
+            static_cast<ColumnDecimal<vectorized::Decimal128>&>(*data_column).get_data();
+    auto concrete_array = down_cast<const arrow::DecimalArray*>(array);
+    const auto* arrow_decimal_type = static_cast<arrow::DecimalType*>(array->type().get());
+    // TODO check precision
+    //size_t precision = arrow_decimal_type->precision();
+    const auto scale = arrow_decimal_type->scale();
+
+    for (size_t value_i = array_idx; value_i < array_idx + num_elements; ++value_i) {
+        auto value =
+                *reinterpret_cast<const vectorized::Decimal128*>(concrete_array->Value(value_i));
+        // convert scale to 9
+        if (scale != 9) {
+            value = convert_decimals<vectorized::DataTypeDecimal<vectorized::Decimal128>,
+                                     vectorized::DataTypeDecimal<vectorized::Decimal128>>(value,
+                                                                                          scale, 9);
+        }
+        column_data.emplace_back(value);
+    }
+    return Status::OK();
+}
+
+Status arrow_column_to_doris_column(const arrow::Array* arrow_column, size_t arrow_batch_cur_idx,
+                                    ColumnWithTypeAndName& doirs_column, size_t num_elements,
+                                    const std::string& timezone) {
+    // src column always be nullable for simpify converting
+    assert(doirs_column.column->is_nullable());
+    MutableColumnPtr data_column = nullptr;
+    if (doirs_column.column->is_nullable()) {
+        auto* nullable_column = reinterpret_cast<vectorized::ColumnNullable*>(
+                (*std::move(doirs_column.column)).mutate().get());
+        fill_nullable_column(arrow_column, arrow_batch_cur_idx, nullable_column, num_elements);
+        data_column = nullable_column->get_nested_column_ptr();
+    } else {
+        data_column = (*std::move(doirs_column.column)).mutate();
+    }
+    // process data
+    switch (arrow_column->type()->id()) {
+    case arrow::Type::STRING:
+    case arrow::Type::BINARY:
+        return convert_column_with_string_data(arrow_column, arrow_batch_cur_idx, data_column,
+                                               num_elements);
+    case arrow::Type::FIXED_SIZE_BINARY:
+        return convert_column_with_fixed_size_data(arrow_column, arrow_batch_cur_idx, data_column,
+                                                   num_elements);
+#define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE)             \
+    case ARROW_NUMERIC_TYPE:                                       \
+        return convert_column_with_numeric_data<CPP_NUMERIC_TYPE>( \
+                arrow_column, arrow_batch_cur_idx, data_column, num_elements);
+        FOR_ARROW_NUMERIC_TYPES(DISPATCH)
+#undef DISPATCH
+    case arrow::Type::BOOL:
+        return convert_column_with_boolean_data(arrow_column, arrow_batch_cur_idx, data_column,
+                                                num_elements);
+    case arrow::Type::DATE32:
+        return convert_column_with_timestamp_data<arrow::Date32Array>(
+                arrow_column, arrow_batch_cur_idx, data_column, num_elements, timezone);
+    case arrow::Type::DATE64:
+        return convert_column_with_timestamp_data<arrow::Date64Array>(
+                arrow_column, arrow_batch_cur_idx, data_column, num_elements, timezone);
+    case arrow::Type::TIMESTAMP:
+        return convert_column_with_timestamp_data<arrow::TimestampArray>(
+                arrow_column, arrow_batch_cur_idx, data_column, num_elements, timezone);
+    case arrow::Type::DECIMAL:
+        return convert_column_with_decimal_data(arrow_column, arrow_batch_cur_idx, data_column,
+                                                num_elements);
+    default:
+        break;
+    }
+    return Status::NotSupported(
+            fmt::format("Not support arrow type:{}", arrow_column->type()->name()));
+}
+} // namespace doris::vectorized
diff --git a/be/src/vec/utils/arrow_column_to_doris_column.h b/be/src/vec/utils/arrow_column_to_doris_column.h
new file mode 100644
index 0000000000..2e70fee11a
--- /dev/null
+++ b/be/src/vec/utils/arrow_column_to_doris_column.h
@@ -0,0 +1,40 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <arrow/type.h>
+#include <arrow/type_fwd.h>
+
+#include <iostream>
+#include <memory>
+
+#include "common/status.h"
+#include "runtime/primitive_type.h"
+#include "vec/core/column_with_type_and_name.h"
+
+// This files contains some utilities to convert Doris internal
+// data format from Apache Arrow format.
+namespace doris::vectorized {
+
+const PrimitiveType arrow_type_to_primitive_type(::arrow::Type::type type);
+
+Status arrow_column_to_doris_column(const arrow::Array* arrow_column, size_t arrow_batch_cur_idx,
+                                    ColumnWithTypeAndName& doirs_column, size_t num_elements,
+                                    const std::string& timezone);
+
+} // namespace doris::vectorized
diff --git a/be/test/CMakeLists.txt b/be/test/CMakeLists.txt
index f39cb68926..38700a4bc2 100644
--- a/be/test/CMakeLists.txt
+++ b/be/test/CMakeLists.txt
@@ -357,6 +357,7 @@ set(VEC_TEST_FILES
     vec/function/function_test_util.cpp
     vec/function/table_function_test.cpp
     vec/runtime/vdata_stream_test.cpp
+    vec/utils/arrow_column_to_doris_column_test.cpp
 )
 
 add_executable(doris_be_test
diff --git a/be/test/vec/utils/arrow_column_to_doris_column_test.cpp b/be/test/vec/utils/arrow_column_to_doris_column_test.cpp
new file mode 100644
index 0000000000..0252cf4883
--- /dev/null
+++ b/be/test/vec/utils/arrow_column_to_doris_column_test.cpp
@@ -0,0 +1,609 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "vec/utils/arrow_column_to_doris_column.h"
+
+#include <arrow/array.h>
+#include <arrow/builder.h>
+#include <arrow/memory_pool.h>
+#include <arrow/record_batch.h>
+#include <arrow/status.h>
+#include <arrow/testing/gtest_util.h>
+#include <arrow/testing/util.h>
+#include <arrow/util/bit_util.h>
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_base.h"
+#include "arrow/scalar.h"
+#include "arrow/type.h"
+#include "arrow/type_fwd.h"
+#include "arrow/type_traits.h"
+#include "gutil/casts.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/data_types/data_type_decimal.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/functions/simple_function_factory.h"
+#include "vec/runtime/vdatetime_value.h"
+
+namespace doris::vectorized {
+
+template <typename ArrowType, typename ArrowCppType = typename arrow::TypeTraits<ArrowType>::CType>
+ArrowCppType string_to_arrow_datetime(std::shared_ptr<ArrowType> type, const std::string& value) {
+    VecDateTimeValue tv;
+    tv.from_date_str(value.c_str(), value.size());
+    int64_t unix_seconds = 0;
+    tv.unix_timestamp(&unix_seconds, "UTC");
+    if constexpr (std::is_same_v<ArrowType, arrow::TimestampType>) {
+        arrow::TimeUnit::type unit = type->unit();
+        VecDateTimeValue vdtv;
+        vdtv.from_unixtime(unix_seconds, "UTC");
+        vdtv.unix_timestamp(&unix_seconds, type->timezone());
+        switch (unit) {
+        case arrow::TimeUnit::SECOND:
+            return unix_seconds;
+        case arrow::TimeUnit::MILLI:
+            return unix_seconds * 1000L;
+        case arrow::TimeUnit::MICRO:
+            return unix_seconds * 1000'000L;
+        case arrow::TimeUnit::NANO:
+            return unix_seconds * 1000'000'000L;
+        default:
+            assert(false);
+        }
+    } else if constexpr (std::is_same_v<ArrowType, arrow::Date32Type>) {
+        return unix_seconds / (24 * 3600);
+    } else if constexpr (std::is_same_v<ArrowType, arrow::Date64Type>) {
+        return unix_seconds * 1000L;
+    } else {
+        assert(false);
+    }
+    return 0;
+}
+
+template <typename ArrowType, bool is_nullable,
+          typename ArrowCppType = typename arrow::TypeTraits<ArrowType>::CType>
+std::shared_ptr<arrow::Array> create_constant_numeric_array(size_t num_elements, ArrowCppType value,
+                                                            std::shared_ptr<arrow::DataType> type,
+                                                            size_t& counter) {
+    std::vector<std::shared_ptr<arrow::Buffer>> buffers;
+    buffers.resize(2);
+    size_t null_bitmap_byte_size = (num_elements + 7) / 8;
+    size_t data_byte_size = num_elements * sizeof(value);
+    auto buffer0_res = arrow::AllocateBuffer(null_bitmap_byte_size);
+    buffers[0] = std::move(buffer0_res.ValueOrDie());
+    auto buffer1_res = arrow::AllocateBuffer(data_byte_size);
+    buffers[1] = std::move(buffer1_res.ValueOrDie());
+    auto* nulls = buffers[0]->mutable_data();
+    auto* data = (ArrowCppType*)buffers[1]->mutable_data();
+
+    for (auto i = 0; i < num_elements; ++i) {
+        if (is_nullable && (i % 2 == 0)) {
+            arrow::bit_util::ClearBit(nulls, i);
+        } else {
+            arrow::bit_util::SetBit(nulls, i);
+        }
+        data[i] = value;
+    }
+    counter += num_elements;
+    using ArrayType = typename arrow::TypeTraits<ArrowType>::ArrayType;
+    auto array_data = std::make_shared<arrow::ArrayData>(type, num_elements, buffers);
+    auto array = std::make_shared<ArrayType>(array_data);
+    return std::static_pointer_cast<arrow::Array>(array);
+}
+
+template <typename ArrowType, typename ColumnType, bool is_nullable,
+          typename ArrowCppType = typename arrow::TypeTraits<ArrowType>::CType>
+void test_arrow_to_datetime_column(std::shared_ptr<ArrowType> type, ColumnWithTypeAndName& column,
+                                   size_t num_elements, ArrowCppType arrow_datetime,
+                                   VecDateTimeValue datetime, size_t& counter) {
+    ASSERT_EQ(column.column->size(), counter);
+    auto array = create_constant_numeric_array<ArrowType, is_nullable>(num_elements, arrow_datetime,
+                                                                       type, counter);
+    std::string time_zone = "UTC";
+    if constexpr (std::is_same_v<ArrowType, arrow::TimestampType>) {
+        time_zone = type->timezone();
+    }
+    auto ret = arrow_column_to_doris_column(array.get(), 0, column, num_elements, time_zone);
+    ASSERT_EQ(ret.ok(), true);
+    ASSERT_EQ(column.column->size(), counter);
+    MutableColumnPtr data_column = nullptr;
+    vectorized::ColumnNullable* nullable_column = nullptr;
+    if (column.column->is_nullable()) {
+        nullable_column = reinterpret_cast<vectorized::ColumnNullable*>(
+                (*std::move(column.column)).mutate().get());
+        data_column = nullable_column->get_nested_column_ptr();
+    } else {
+        data_column = (*std::move(column.column)).mutate();
+    }
+    auto& datetime_data = static_cast<ColumnType&>(*data_column).get_data();
+    for (auto i = 0; i < num_elements; ++i) {
+        auto idx = counter - num_elements + i;
+        if (is_nullable) {
+            ASSERT_NE(nullable_column, nullptr);
+            NullMap& map_data = nullable_column->get_null_map_data();
+            if (i % 2 == 0) {
+                ASSERT_EQ(map_data[idx], true);
+            } else {
+                ASSERT_EQ(map_data[idx], false);
+                auto val = binary_cast<VecDateTimeValue, Int64>(datetime);
+                ASSERT_EQ(datetime_data[idx], val);
+            }
+        } else {
+            auto val = binary_cast<VecDateTimeValue, Int64>(datetime);
+            ASSERT_EQ(datetime_data[idx], val);
+        }
+    }
+}
+
+template <typename ArrowType, typename ColumnType, bool is_nullable>
+void test_datetime(std::shared_ptr<ArrowType> type, const std::vector<std::string>& test_cases,
+                   size_t num_elements) {
+    using ArrowCppType = typename arrow::TypeTraits<ArrowType>::CType;
+    size_t counter = 0;
+    auto pt = arrow_type_to_primitive_type(type->id());
+    ASSERT_NE(pt, INVALID_TYPE);
+    DataTypePtr data_type = DataTypeFactory::instance().create_data_type(pt, true);
+    MutableColumnPtr data_column = data_type->create_column();
+    ColumnWithTypeAndName column(std::move(data_column), data_type, "test_datatime_column");
+    for (auto& value : test_cases) {
+        ArrowCppType arrow_datetime = string_to_arrow_datetime<ArrowType>(type, value);
+        VecDateTimeValue tv;
+        tv.from_date_str(value.c_str(), value.size());
+        test_arrow_to_datetime_column<ArrowType, ColumnType, is_nullable>(
+                type, column, num_elements, arrow_datetime, tv, counter);
+    }
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_date32_to_date) {
+    auto type = std::make_shared<arrow::Date32Type>();
+    std::vector<std::string> test_cases = {{"1970-01-01"}, {"2021-05-30"}, {"2022-05-08"}};
+    test_datetime<arrow::Date32Type, ColumnVector<Int64>, false>(type, test_cases, 32);
+    test_datetime<arrow::Date32Type, ColumnVector<Int64>, true>(type, test_cases, 32);
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_date64_to_datetime) {
+    auto type = std::make_shared<arrow::Date64Type>();
+    std::vector<std::string> test_cases = {
+            {"1970-01-01 12:12:12"}, {"2021-05-30 22:22:22"}, {"2022-05-08 00:00:01"}};
+    test_datetime<arrow::Date64Type, ColumnVector<Int64>, false>(type, test_cases, 64);
+    test_datetime<arrow::Date64Type, ColumnVector<Int64>, true>(type, test_cases, 64);
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_timestamp_to_datetime) {
+    auto type = std::make_shared<arrow::Date64Type>();
+    std::vector<std::string> test_cases = {
+            {"1970-01-01 12:12:12"}, {"2021-05-30 22:22:22"}, {"2022-05-08 00:00:01"}};
+    std::vector<std::string> zones = {"UTC",    "GMT",           "CST",          "+01:00",
+                                      "-09:00", "Asia/Shanghai", "Europe/Zurich"};
+    std::vector<arrow::TimeUnit::type> time_units = {arrow::TimeUnit::SECOND,
+                                                     arrow::TimeUnit::MICRO, arrow::TimeUnit::MILLI,
+                                                     arrow::TimeUnit::NANO};
+    for (auto& unit : time_units) {
+        for (auto& zone : zones) {
+            auto type = std::make_shared<arrow::TimestampType>(unit, zone);
+            test_datetime<arrow::TimestampType, ColumnVector<Int64>, false>(type, test_cases, 64);
+            test_datetime<arrow::TimestampType, ColumnVector<Int64>, true>(type, test_cases, 64);
+        }
+    }
+}
+
+template <typename ArrowType, typename CppType, bool is_nullable,
+          typename ColumnType = ColumnVector<CppType>,
+          typename ArrowCppType = typename arrow::TypeTraits<ArrowType>::CType>
+void test_arrow_to_numeric_column(std::shared_ptr<ArrowType> type, ColumnWithTypeAndName& column,
+                                  size_t num_elements, ArrowCppType arrow_numeric, CppType numeric,
+                                  size_t& counter) {
+    ASSERT_EQ(column.column->size(), counter);
+    auto array = create_constant_numeric_array<ArrowType, is_nullable>(num_elements, arrow_numeric,
+                                                                       type, counter);
+    auto ret = arrow_column_to_doris_column(array.get(), 0, column, num_elements, "UTC");
+    ASSERT_EQ(ret.ok(), true);
+    ASSERT_EQ(column.column->size(), counter);
+    MutableColumnPtr data_column = nullptr;
+    vectorized::ColumnNullable* nullable_column = nullptr;
+    if (column.column->is_nullable()) {
+        nullable_column = reinterpret_cast<vectorized::ColumnNullable*>(
+                (*std::move(column.column)).mutate().get());
+        data_column = nullable_column->get_nested_column_ptr();
+    } else {
+        data_column = (*std::move(column.column)).mutate();
+    }
+    auto& numeric_data = static_cast<ColumnType&>(*data_column).get_data();
+    for (auto i = 0; i < num_elements; ++i) {
+        auto idx = counter - num_elements + i;
+        if (is_nullable) {
+            ASSERT_NE(nullable_column, nullptr);
+            NullMap& map_data = nullable_column->get_null_map_data();
+            if (i % 2 == 0) {
+                ASSERT_EQ(map_data[idx], true);
+            } else {
+                ASSERT_EQ(map_data[idx], false);
+                ASSERT_EQ(numeric_data[idx], numeric);
+            }
+        } else {
+            ASSERT_EQ(numeric_data[idx], numeric);
+        }
+    }
+}
+
+template <typename ArrowType, typename CppType, bool is_nullable,
+          typename ColumnType = ColumnVector<CppType>>
+void test_numeric(std::shared_ptr<ArrowType> type, const std::vector<CppType>& test_cases,
+                  size_t num_elements) {
+    using ArrowCppType = typename arrow::TypeTraits<ArrowType>::CType;
+    size_t counter = 0;
+    auto pt = arrow_type_to_primitive_type(type->id());
+    ASSERT_NE(pt, INVALID_TYPE);
+    DataTypePtr data_type = DataTypeFactory::instance().create_data_type(pt, true);
+    MutableColumnPtr data_column = data_type->create_column();
+    ColumnWithTypeAndName column(std::move(data_column), data_type, "test_numeric_column");
+    for (auto& value : test_cases) {
+        test_arrow_to_numeric_column<ArrowType, CppType, is_nullable>(
+                type, column, num_elements, ArrowCppType(value), value, counter);
+    }
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_int8) {
+    auto type = std::make_shared<arrow::Int8Type>();
+    std::vector<Int8> test_cases = {1, -1, -128, 127, int8_t(255)};
+    test_numeric<arrow::Int8Type, Int8, false>(type, test_cases, 64);
+    test_numeric<arrow::Int8Type, Int8, true>(type, test_cases, 64);
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_uint8) {
+    auto type = std::make_shared<arrow::UInt8Type>();
+    std::vector<UInt8> test_cases = {uint8_t(-1), uint8_t(1), uint8_t(-128), uint8_t(127),
+                                     uint8_t(255)};
+    test_numeric<arrow::UInt8Type, UInt8, false>(type, test_cases, 64);
+    test_numeric<arrow::UInt8Type, UInt8, true>(type, test_cases, 64);
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_uint16) {
+    auto type = std::make_shared<arrow::UInt16Type>();
+    std::vector<UInt16> test_cases = {uint16_t(-1), uint16_t(1), uint16_t(-128), uint16_t(127),
+                                      uint16_t(65535)};
+    test_numeric<arrow::UInt16Type, UInt16, false>(type, test_cases, 64);
+    test_numeric<arrow::UInt16Type, UInt16, true>(type, test_cases, 64);
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_uint32) {
+    auto type = std::make_shared<arrow::UInt32Type>();
+    std::vector<UInt32> test_cases = {uint32_t(-1), uint32_t(1), uint32_t(-65535), uint32_t(65535),
+                                      uint32_t(4294967295)};
+    test_numeric<arrow::UInt32Type, UInt32, false>(type, test_cases, 64);
+    test_numeric<arrow::UInt32Type, UInt32, true>(type, test_cases, 64);
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_uint64) {
+    auto type = std::make_shared<arrow::UInt64Type>();
+    std::vector<UInt64> test_cases = {uint64_t(-1),
+                                      uint64_t(1),
+                                      uint64_t(-4294967295),
+                                      uint64_t(4294967295),
+                                      uint64_t(std::numeric_limits<uint64_t>::min()),
+                                      uint64_t(std::numeric_limits<uint64_t>::max())};
+    test_numeric<arrow::UInt64Type, UInt64, false>(type, test_cases, 64);
+    test_numeric<arrow::UInt64Type, UInt64, true>(type, test_cases, 64);
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_float64) {
+    auto type = std::make_shared<arrow::DoubleType>();
+    std::vector<double> test_cases = {double(-1.11f),
+                                      double(1.11f),
+                                      double(-4294967295),
+                                      double(4294967295),
+                                      double(std::numeric_limits<double>::min()),
+                                      double(std::numeric_limits<double>::max())};
+    test_numeric<arrow::DoubleType, Float64, false>(type, test_cases, 64);
+    test_numeric<arrow::DoubleType, Float64, true>(type, test_cases, 64);
+}
+
+template <bool is_nullable>
+std::shared_ptr<arrow::Array> create_decimal_array(size_t num_elements, int128_t decimal,
+                                                   std::shared_ptr<arrow::Decimal128Type> type,
+                                                   size_t& counter) {
+    std::vector<std::shared_ptr<arrow::Buffer>> buffers;
+    buffers.resize(2);
+    auto byte_width = type->byte_width();
+    auto buffer0_res = arrow::AllocateBuffer((num_elements + 7) / 8);
+    buffers[0] = std::move(buffer0_res.ValueOrDie());
+    auto buffer1_res = arrow::AllocateBuffer(byte_width * num_elements);
+    buffers[1] = std::move(buffer1_res.ValueOrDie());
+    auto* nulls = buffers[0]->mutable_data();
+    auto* data = buffers[1]->mutable_data();
+    for (auto i = 0; i < num_elements; ++i) {
+        if (is_nullable && (i % 2 == 0)) {
+            arrow::bit_util::ClearBit(nulls, i);
+        } else {
+            arrow::bit_util::SetBit(nulls, i);
+            memcpy(data + i * byte_width, &decimal, sizeof(decimal));
+        }
+    }
+    auto array_data = std::make_shared<arrow::ArrayData>(type, num_elements, buffers);
+    auto array = std::make_shared<arrow::Decimal128Array>(array_data);
+    counter += num_elements;
+    return array;
+}
+
+template <bool is_nullable>
+void test_arrow_to_decimal_column(std::shared_ptr<arrow::Decimal128Type> type,
+                                  ColumnWithTypeAndName& column, size_t num_elements,
+                                  int128_t arrow_value, int128_t expect_value, size_t& counter) {
+    ASSERT_EQ(column.column->size(), counter);
+    auto array = create_decimal_array<is_nullable>(num_elements, arrow_value, type, counter);
+    auto ret = arrow_column_to_doris_column(array.get(), 0, column, num_elements, "UTC");
+    ASSERT_EQ(ret.ok(), true);
+    ASSERT_EQ(column.column->size(), counter);
+    MutableColumnPtr data_column = nullptr;
+    vectorized::ColumnNullable* nullable_column = nullptr;
+    if (column.column->is_nullable()) {
+        nullable_column = reinterpret_cast<vectorized::ColumnNullable*>(
+                (*std::move(column.column)).mutate().get());
+        data_column = nullable_column->get_nested_column_ptr();
+    } else {
+        data_column = (*std::move(column.column)).mutate();
+    }
+    auto& decimal_data =
+            static_cast<ColumnDecimal<vectorized::Decimal128>&>(*data_column).get_data();
+    for (auto i = 0; i < num_elements; ++i) {
+        auto idx = counter - num_elements + i;
+        if (is_nullable) {
+            ASSERT_NE(nullable_column, nullptr);
+            NullMap& map_data = nullable_column->get_null_map_data();
+            if (i % 2 == 0) {
+                ASSERT_EQ(map_data[idx], true);
+            } else {
+                ASSERT_EQ(map_data[idx], false);
+                ASSERT_EQ(Int128(decimal_data[idx]), expect_value);
+            }
+        } else {
+            ASSERT_EQ(Int128(decimal_data[idx]), expect_value);
+        }
+    }
+}
+
+template <bool is_nullable>
+void test_decimalv2(std::shared_ptr<arrow::Decimal128Type> type,
+                    const std::vector<std::string>& test_cases, size_t num_elements) {
+    using ArrowCppType = typename arrow::TypeTraits<arrow::Decimal128Type>::CType;
+    size_t counter = 0;
+    auto pt = arrow_type_to_primitive_type(type->id());
+    ASSERT_NE(pt, INVALID_TYPE);
+    DataTypePtr data_type = DataTypeFactory::instance().create_data_type(pt, true);
+    MutableColumnPtr data_column = data_type->create_column();
+    ColumnWithTypeAndName column(std::move(data_column), data_type, "test_numeric_column");
+    for (auto& str : test_cases) {
+        DecimalV2Value decimal_value(str);
+        int128_t value = binary_cast<DecimalV2Value, int128_t>(decimal_value);
+        int128_t expect_value =
+                convert_decimals<vectorized::DataTypeDecimal<vectorized::Decimal128>,
+                                 vectorized::DataTypeDecimal<vectorized::Decimal128>>(
+                        value, type->scale(), 9);
+        test_arrow_to_decimal_column<is_nullable>(type, column, num_elements, value, expect_value,
+                                                  counter);
+    }
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_decimalv2) {
+    std::vector<std::string> test_cases = {"1.2345678", "-12.34567890", "99999999999.99999999",
+                                           "-99999999999.99999999"};
+    auto type_p27s9 = std::make_shared<arrow::Decimal128Type>(27, 9);
+    test_decimalv2<false>(type_p27s9, test_cases, 64);
+    test_decimalv2<true>(type_p27s9, test_cases, 64);
+
+    auto type_p27s25 = std::make_shared<arrow::Decimal128Type>(27, 25);
+    test_decimalv2<false>(type_p27s25, test_cases, 128);
+    test_decimalv2<true>(type_p27s25, test_cases, 128);
+}
+
+template <int bytes_width, bool is_nullable = false>
+static inline std::shared_ptr<arrow::Array> create_fixed_size_binary_array(int64_t num_elements,
+                                                                           const std::string& value,
+                                                                           size_t& counter) {
+    auto data_buf_size = bytes_width * num_elements;
+    auto data_buf_tmp = arrow::AllocateBuffer(data_buf_size);
+    std::shared_ptr<arrow::Buffer> data_buf = std::move(data_buf_tmp.ValueOrDie());
+    auto* p = data_buf->mutable_data();
+
+    auto null_bitmap_bytes = (num_elements + 7) / 8;
+    auto null_bitmap_tmp = arrow::AllocateBuffer(null_bitmap_bytes);
+    std::shared_ptr<arrow::Buffer> null_bitmap_buf = std::move(null_bitmap_tmp.ValueOrDie());
+    auto* nulls = null_bitmap_buf->mutable_data();
+
+    for (auto i = 0; i < num_elements; ++i) {
+        if (is_nullable && i % 2 == 0) {
+            arrow::bit_util::ClearBit(nulls, i);
+        } else {
+            arrow::bit_util::SetBit(nulls, i);
+        }
+        memcpy(p, value.c_str(), std::min(value.size() + 1, (std::string::size_type)bytes_width));
+        p += bytes_width;
+    }
+    auto type = std::make_shared<arrow::FixedSizeBinaryType>(bytes_width);
+    auto array = std::make_shared<arrow::FixedSizeBinaryArray>(type, num_elements, data_buf,
+                                                               null_bitmap_buf);
+    counter += num_elements;
+    return std::static_pointer_cast<arrow::Array>(array);
+}
+
+template <int bytes_width, bool is_nullable>
+void test_arrow_to_fixed_binary_column(ColumnWithTypeAndName& column, size_t num_elements,
+                                       const std::string value, size_t& counter) {
+    ASSERT_EQ(column.column->size(), counter);
+    auto array =
+            create_fixed_size_binary_array<bytes_width, is_nullable>(num_elements, value, counter);
+    auto ret = arrow_column_to_doris_column(array.get(), 0, column, num_elements, "UTC");
+    ASSERT_EQ(ret.ok(), true);
+    ASSERT_EQ(column.column->size(), counter);
+    MutableColumnPtr data_column = nullptr;
+    vectorized::ColumnNullable* nullable_column = nullptr;
+    if (column.column->is_nullable()) {
+        nullable_column = reinterpret_cast<vectorized::ColumnNullable*>(
+                (*std::move(column.column)).mutate().get());
+        data_column = nullable_column->get_nested_column_ptr();
+    } else {
+        data_column = (*std::move(column.column)).mutate();
+    }
+    auto& string_column = static_cast<ColumnString&>(*data_column);
+    auto string_size = std::min((std::string::size_type)bytes_width, value.size());
+    for (auto i = 0; i < num_elements; ++i) {
+        auto idx = counter - num_elements + i;
+        auto s = string_column.get_data_at(idx);
+        if (is_nullable) {
+            ASSERT_NE(nullable_column, nullptr);
+            NullMap& map_data = nullable_column->get_null_map_data();
+            if (i % 2 == 0) {
+                ASSERT_EQ(map_data[idx], true);
+                ASSERT_EQ(s.size, 0);
+            } else {
+                ASSERT_EQ(map_data[idx], false);
+                ASSERT_EQ(value.compare(0, string_size, s.to_string(), 0, string_size), 0);
+            }
+        } else {
+            ASSERT_EQ(value.compare(0, string_size, s.to_string(), 0, string_size), 0);
+        }
+    }
+}
+
+template <int bytes_width, bool is_nullable>
+void test_fixed_binary(const std::vector<std::string>& test_cases, size_t num_elements) {
+    size_t counter = 0;
+    auto pt = arrow_type_to_primitive_type(::arrow::Type::FIXED_SIZE_BINARY);
+    ASSERT_NE(pt, INVALID_TYPE);
+    DataTypePtr data_type = DataTypeFactory::instance().create_data_type(pt, true);
+    MutableColumnPtr data_column = data_type->create_column();
+    ColumnWithTypeAndName column(std::move(data_column), data_type, "test_fixed_binary_column");
+    for (auto& value : test_cases) {
+        test_arrow_to_fixed_binary_column<bytes_width, is_nullable>(column, num_elements, value,
+                                                                    counter);
+    }
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_fixed_binary) {
+    std::vector<std::string> test_cases = {"1.2345678", "-12.34567890", "99999999999.99999999",
+                                           "-99999999999.99999999"};
+    test_fixed_binary<10, false>(test_cases, 64);
+    test_fixed_binary<10, true>(test_cases, 64);
+
+    test_fixed_binary<255, false>(test_cases, 64);
+    test_fixed_binary<255, true>(test_cases, 64);
+}
+
+template <typename ArrowType, bool is_nullable = false>
+static inline std::shared_ptr<arrow::Array> create_binary_array(int64_t num_elements,
+                                                                const std::string& value,
+                                                                size_t& counter) {
+    using offset_type = typename ArrowType::offset_type;
+    size_t offsets_bytes = (num_elements + 1) * sizeof(offset_type);
+    auto offsets_buf_tmp = arrow::AllocateBuffer(offsets_bytes);
+    std::shared_ptr<arrow::Buffer> offsets_buf = std::move(offsets_buf_tmp.ValueOrDie());
+    auto* offsets = (offset_type*)offsets_buf->mutable_data();
+    offsets[0] = 0;
+
+    auto value_size = value.size();
+    size_t data_bytes = value_size * num_elements;
+    auto data_buf_tmp = arrow::AllocateBuffer(data_bytes);
+    std::shared_ptr<arrow::Buffer> data_buf = std::move(data_buf_tmp.ValueOrDie());
+    auto* data = data_buf->mutable_data();
+
+    auto null_bitmap_bytes = (num_elements + 7) / 8;
+    auto null_bitmap_tmp = arrow::AllocateBuffer(null_bitmap_bytes);
+    std::shared_ptr<arrow::Buffer> null_bitmap = std::move(null_bitmap_tmp.ValueOrDie());
+    auto nulls = null_bitmap->mutable_data();
+    auto data_off = 0;
+    for (auto i = 0; i < num_elements; ++i) {
+        if (is_nullable && i % 2 == 0) {
+            arrow::bit_util::ClearBit(nulls, i);
+        } else {
+            arrow::bit_util::SetBit(nulls, i);
+            memcpy(data + data_off, value.data(), value_size);
+            data_off += value_size;
+        }
+        offsets[i + 1] = data_off;
+    }
+
+    using ArrayType = typename arrow::TypeTraits<ArrowType>::ArrayType;
+    auto array = std::make_shared<ArrayType>(num_elements, offsets_buf, data_buf, null_bitmap);
+    counter += num_elements;
+    return std::static_pointer_cast<arrow::Array>(array);
+}
+
+template <typename ArrowType, bool is_nullable,
+          typename ArrowCppType = typename arrow::TypeTraits<ArrowType>::CType>
+void test_arrow_to_binary_column(ColumnWithTypeAndName& column, size_t num_elements,
+                                 ArrowCppType value, size_t& counter) {
+    ASSERT_EQ(column.column->size(), counter);
+    auto array = create_binary_array<ArrowType, is_nullable>(num_elements, value, counter);
+    auto ret = arrow_column_to_doris_column(array.get(), 0, column, num_elements, "UTC");
+    ASSERT_EQ(ret.ok(), true);
+    ASSERT_EQ(column.column->size(), counter);
+    MutableColumnPtr data_column = nullptr;
+    vectorized::ColumnNullable* nullable_column = nullptr;
+    if (column.column->is_nullable()) {
+        nullable_column = reinterpret_cast<vectorized::ColumnNullable*>(
+                (*std::move(column.column)).mutate().get());
+        data_column = nullable_column->get_nested_column_ptr();
+    } else {
+        data_column = (*std::move(column.column)).mutate();
+    }
+    auto& string_column = static_cast<ColumnString&>(*data_column);
+    for (auto i = 0; i < num_elements; ++i) {
+        auto idx = counter - num_elements + i;
+        auto s = string_column.get_data_at(idx);
+        if (is_nullable) {
+            ASSERT_NE(nullable_column, nullptr);
+            NullMap& map_data = nullable_column->get_null_map_data();
+            if (i % 2 == 0) {
+                ASSERT_EQ(map_data[idx], true);
+                ASSERT_EQ(s.size, 0);
+            } else {
+                ASSERT_EQ(map_data[idx], false);
+                ASSERT_EQ(value, s.to_string());
+            }
+        } else {
+            ASSERT_EQ(value, s.to_string());
+        }
+    }
+}
+
+template <typename ArrowType, bool is_nullable>
+void test_binary(const std::vector<std::string>& test_cases, size_t num_elements) {
+    size_t counter = 0;
+    DataTypePtr data_type = DataTypeFactory::instance().create_data_type(TYPE_VARCHAR, true);
+    MutableColumnPtr data_column = data_type->create_column();
+    ColumnWithTypeAndName column(std::move(data_column), data_type, "test_binary_column");
+    for (auto& value : test_cases) {
+        test_arrow_to_binary_column<ArrowType, is_nullable>(column, num_elements, value, counter);
+    }
+}
+
+TEST(ArrowColumnToDorisColumnTest, test_binary) {
+    std::vector<std::string> test_cases = {"1.2345678", "-12.34567890", "99999999999.99999999",
+                                           "-99999999999.99999999"};
+    test_binary<arrow::StringType, false>(test_cases, 64);
+    test_binary<arrow::StringType, true>(test_cases, 64);
+
+    test_binary<arrow::BinaryType, false>(test_cases, 64);
+    test_binary<arrow::BinaryType, true>(test_cases, 64);
+}
+} // namespace doris::vectorized
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
index 2a550a3715..c125e742fc 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
@@ -186,6 +186,23 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
                 public boolean apply(Expr arg) { return arg instanceof NullLiteral; }
             };
 
+    public static final com.google.common.base.Predicate<Expr> IS_VARCHAR_SLOT_REF_IMPLICIT_CAST =
+            new com.google.common.base.Predicate<Expr>() {
+                @Override
+                public boolean apply(Expr arg) {
+                    // exclude explicit cast. for example: cast(k1 as date)
+                    if (!arg.isImplicitCast()) {
+                        return false;
+                    }
+                    List<Expr> children = arg.getChildren();
+                    if (children.isEmpty()) {
+                        return false;
+                    }
+                    Expr child = children.get(0);
+                    return child instanceof SlotRef && child.getType().isVarchar();
+                }
+            };
+
     public void setSelectivity() {
         selectivity = -1;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TupleDescriptor.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TupleDescriptor.java
index edd1e299a9..45fab98fa8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TupleDescriptor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TupleDescriptor.java
@@ -105,6 +105,21 @@ public class TupleDescriptor {
         return slots;
     }
 
+    /**
+     * get slot desc by slot id.
+     *
+     * @param slotId slot id
+     * @return this slot's desc
+     */
+    public SlotDescriptor getSlot(int slotId) {
+        for (SlotDescriptor slotDesc : slots) {
+            if (slotDesc.getId().asInt() == slotId) {
+                return slotDesc;
+            }
+        }
+        return null;
+    }
+
     public void setCardinality(long cardinality) {
         this.cardinality = cardinality;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java
index e95b70d561..e23302d96c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java
@@ -192,6 +192,10 @@ public abstract class Type {
                 || isScalarType(PrimitiveType.STRING);
     }
 
+    public boolean isVarchar() {
+        return isScalarType(PrimitiveType.VARCHAR);
+    }
+
     // only metric types have the following constraint:
     // 1. don't support as key column
     // 2. don't support filter
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java
index 2b151c0452..81e500968e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java
@@ -95,6 +95,7 @@ import org.apache.doris.task.LoadTaskInfo;
 import org.apache.doris.task.PushTask;
 import org.apache.doris.thrift.TBrokerScanRangeParams;
 import org.apache.doris.thrift.TEtlState;
+import org.apache.doris.thrift.TFileFormatType;
 import org.apache.doris.thrift.TMiniLoadRequest;
 import org.apache.doris.thrift.TNetworkAddress;
 import org.apache.doris.thrift.TPriority;
@@ -930,7 +931,7 @@ public class Load {
      */
     public static void initColumns(Table tbl, List<ImportColumnDesc> columnExprs,
                                    Map<String, Pair<String, List<String>>> columnToHadoopFunction) throws UserException {
-        initColumns(tbl, columnExprs, columnToHadoopFunction, null, null, null, null, null, false);
+        initColumns(tbl, columnExprs, columnToHadoopFunction, null, null, null, null, null, null, false, false);
     }
 
     /*
@@ -940,10 +941,11 @@ public class Load {
     public static void initColumns(Table tbl, LoadTaskInfo.ImportColumnDescs columnDescs,
                                    Map<String, Pair<String, List<String>>> columnToHadoopFunction,
                                    Map<String, Expr> exprsByName, Analyzer analyzer, TupleDescriptor srcTupleDesc,
-                                   Map<String, SlotDescriptor> slotDescByName, TBrokerScanRangeParams params) throws UserException {
+                                   Map<String, SlotDescriptor> slotDescByName, TBrokerScanRangeParams params,
+                                   TFileFormatType formatType, boolean useVectorizedLoad) throws UserException {
         rewriteColumns(columnDescs);
         initColumns(tbl, columnDescs.descs, columnToHadoopFunction, exprsByName, analyzer,
-                srcTupleDesc, slotDescByName, params, true);
+                srcTupleDesc, slotDescByName, params, formatType, useVectorizedLoad, true);
     }
 
     /*
@@ -958,6 +960,7 @@ public class Load {
                                     Map<String, Pair<String, List<String>>> columnToHadoopFunction,
                                     Map<String, Expr> exprsByName, Analyzer analyzer, TupleDescriptor srcTupleDesc,
                                     Map<String, SlotDescriptor> slotDescByName, TBrokerScanRangeParams params,
+                                    TFileFormatType formatType, boolean useVectorizedLoad,
                                     boolean needInitSlotAndAnalyzeExprs) throws UserException {
         // We make a copy of the columnExprs so that our subsequent changes
         // to the columnExprs will not affect the original columnExprs.
@@ -1043,30 +1046,70 @@ public class Load {
         if (!needInitSlotAndAnalyzeExprs) {
             return;
         }
-
+        Set<String> exprSrcSlotName = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
+        for (ImportColumnDesc importColumnDesc : copiedColumnExprs) {
+            if (importColumnDesc.isColumn()) {
+                continue;
+            }
+            List<SlotRef> slots = Lists.newArrayList();
+            importColumnDesc.getExpr().collect(SlotRef.class, slots);
+            for (SlotRef slot : slots) {
+                String slotColumnName = slot.getColumnName();
+                exprSrcSlotName.add(slotColumnName);
+            }
+        }
+        // excludedColumns is columns that should be varchar type
+        Set<String> excludedColumns = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
         // init slot desc add expr map, also transform hadoop functions
         for (ImportColumnDesc importColumnDesc : copiedColumnExprs) {
             // make column name case match with real column name
             String columnName = importColumnDesc.getColumnName();
+            Column tblColumn = tbl.getColumn(columnName);
             String realColName;
-            if (tbl.getColumn(columnName) == null || importColumnDesc.getExpr() == null) {
+            if (tblColumn == null || tblColumn.getName() == null || importColumnDesc.getExpr() == null) {
                 realColName = columnName;
             } else {
-                realColName = tbl.getColumn(columnName).getName();
+                realColName = tblColumn.getName();
             }
             if (importColumnDesc.getExpr() != null) {
                 Expr expr = transformHadoopFunctionExpr(tbl, realColName, importColumnDesc.getExpr());
                 exprsByName.put(realColName, expr);
             } else {
                 SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc);
-                slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR));
+                // only support parquet format now
+                if (useVectorizedLoad  && formatType == TFileFormatType.FORMAT_PARQUET
+                        && tblColumn != null) {
+                    // in vectorized load
+                    // example: k1 is DATETIME in source file, and INT in schema, mapping exper is k1=year(k1)
+                    // we can not determine whether to use the type in the schema or the type inferred from expr
+                    // so use varchar type as before
+                    if (exprSrcSlotName.contains(columnName)) {
+                        // columns in expr args should be varchar type
+                        slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR));
+                        slotDesc.setColumn(new Column(realColName, PrimitiveType.VARCHAR));
+                        excludedColumns.add(realColName);
+                        // example k1, k2 = k1 + 1, k1 is not nullable, k2 is nullable
+                        // so we can not determine columns in expr args whether not nullable or nullable
+                        // slot in expr args use nullable as before
+                        slotDesc.setIsNullable(true);
+                    } else {
+                        // columns from files like parquet files can be parsed as the type in table schema
+                        slotDesc.setType(tblColumn.getType());
+                        slotDesc.setColumn(new Column(realColName, tblColumn.getType()));
+                        // non-nullable column is allowed in vectorized load with parquet format
+                        slotDesc.setIsNullable(tblColumn.isAllowNull());
+                    }
+                } else {
+                    // columns default be varchar type
+                    slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR));
+                    slotDesc.setColumn(new Column(realColName, PrimitiveType.VARCHAR));
+                    // ISSUE A: src slot should be nullable even if the column is not nullable.
+                    // because src slot is what we read from file, not represent to real column value.
+                    // If column is not nullable, error will be thrown when filling the dest slot,
+                    // which is not nullable.
+                    slotDesc.setIsNullable(true);
+                }
                 slotDesc.setIsMaterialized(true);
-                // ISSUE A: src slot should be nullable even if the column is not nullable.
-                // because src slot is what we read from file, not represent to real column value.
-                // If column is not nullable, error will be thrown when filling the dest slot,
-                // which is not nullable.
-                slotDesc.setIsNullable(true);
-                slotDesc.setColumn(new Column(realColName, PrimitiveType.VARCHAR));
                 params.addToSrcSlotIds(slotDesc.getId().asInt());
                 slotDescByName.put(realColName, slotDesc);
             }
@@ -1085,7 +1128,30 @@ public class Load {
         }
 
         LOG.debug("slotDescByName: {}, exprsByName: {}, mvDefineExpr: {}", slotDescByName, exprsByName, mvDefineExpr);
+        // we only support parquet format now
+        // use implicit deduction to convert columns
+        // that are not in the doris table from varchar to a more appropriate type
+        if (useVectorizedLoad && formatType == TFileFormatType.FORMAT_PARQUET) {
+            // analyze all exprs
+            Map<String, Expr> cloneExprsByName = Maps.newHashMap(exprsByName);
+            Map<String, Expr> cloneMvDefineExpr = Maps.newHashMap(mvDefineExpr);
+            analyzeAllExprs(tbl, analyzer, cloneExprsByName, cloneMvDefineExpr, slotDescByName, useVectorizedLoad);
+            // columns that only exist in mapping expr args, replace type with inferred from exprs,
+            // if there are more than one, choose the last except varchar type
+            // for example:
+            // k1 involves two mapping expr args: year(k1), t1=k1, k1's varchar type will be replaced by DATETIME
+            replaceVarcharWithCastType(cloneExprsByName, srcTupleDesc, excludedColumns);
+        }
+
+        // in vectorized load, reanalyze exprs with castExpr type
+        // otherwise analyze exprs with varchar type
+        analyzeAllExprs(tbl, analyzer, exprsByName, mvDefineExpr, slotDescByName, useVectorizedLoad);
+        LOG.debug("after init column, exprMap: {}", exprsByName);
+    }
 
+    private static void analyzeAllExprs(Table tbl, Analyzer analyzer, Map<String, Expr> exprsByName,
+                                            Map<String, Expr> mvDefineExpr, Map<String, SlotDescriptor> slotDescByName,
+                                            boolean useVectorizedLoad) throws UserException {
         // analyze all exprs
         for (Map.Entry<String, Expr> entry : exprsByName.entrySet()) {
             ExprSubstitutionMap smap = new ExprSubstitutionMap();
@@ -1094,14 +1160,17 @@ public class Load {
             for (SlotRef slot : slots) {
                 SlotDescriptor slotDesc = slotDescByName.get(slot.getColumnName());
                 if (slotDesc == null) {
-                    if (entry.getKey().equalsIgnoreCase(Column.DELETE_SIGN)) {
-                        throw new UserException("unknown reference column in DELETE ON clause:" + slot.getColumnName());
-                    } else if (entry.getKey().equalsIgnoreCase(Column.SEQUENCE_COL)) {
-                        throw new UserException("unknown reference column in ORDER BY clause:" + slot.getColumnName());
-                    } else {
-                        throw new UserException("unknown reference column, column=" + entry.getKey()
-                                + ", reference=" + slot.getColumnName());
+                    if (entry.getKey() != null) {
+                        if (entry.getKey().equalsIgnoreCase(Column.DELETE_SIGN)) {
+                            throw new UserException("unknown reference column in DELETE ON clause:"
+                                + slot.getColumnName());
+                        } else if (entry.getKey().equalsIgnoreCase(Column.SEQUENCE_COL)) {
+                            throw new UserException("unknown reference column in ORDER BY clause:"
+                                + slot.getColumnName());
+                        }
                     }
+                    throw new UserException("unknown reference column, column=" + entry.getKey()
+                            + ", reference=" + slot.getColumnName());
                 }
                 smap.getLhs().add(slot);
                 smap.getRhs().add(new SlotRef(slotDesc));
@@ -1149,7 +1218,50 @@ public class Load {
 
             exprsByName.put(entry.getKey(), expr);
         }
-        LOG.debug("after init column, exprMap: {}", exprsByName);
+    }
+
+    /**
+     * columns that only exist in mapping expr args, replace type with inferred from exprs.
+     *
+     * @param excludedColumns columns that the type should not be inferred from expr.
+     *                         1. column exists in both schema and expr args.
+     */
+    private static void replaceVarcharWithCastType(Map<String, Expr> exprsByName, TupleDescriptor srcTupleDesc,
+                                               Set<String> excludedColumns) throws UserException {
+        // if there are more than one, choose the last except varchar type.
+        // for example:
+        // k1 involves two mapping expr args: year(k1), t1=k1, k1's varchar type will be replaced by DATETIME.
+        for (Map.Entry<String, Expr> entry : exprsByName.entrySet()) {
+            List<CastExpr> casts = Lists.newArrayList();
+            // exclude explicit cast. for example: cast(k1 as date)
+            entry.getValue().collect(Expr.IS_VARCHAR_SLOT_REF_IMPLICIT_CAST, casts);
+            if (casts.isEmpty()) {
+                continue;
+            }
+
+            for (CastExpr cast : casts) {
+                Expr child = cast.getChild(0);
+                Type type = cast.getType();
+                if (type.isVarchar()) {
+                    continue;
+                }
+
+                SlotRef slotRef = (SlotRef) child;
+                String columnName = slotRef.getColumn().getName();
+                if (excludedColumns.contains(columnName)) {
+                    continue;
+                }
+
+                // replace src slot desc with cast return type
+                int slotId = slotRef.getSlotId().asInt();
+                SlotDescriptor srcSlotDesc = srcTupleDesc.getSlot(slotId);
+                if (srcSlotDesc == null) {
+                    throw new UserException("Unknown source slot descriptor. id: " + slotId);
+                }
+                srcSlotDesc.setType(type);
+                srcSlotDesc.setColumn(new Column(columnName, type));
+            }
+        }
     }
 
     public static void rewriteColumns(LoadTaskInfo.ImportColumnDescs columnDescs) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java
index 2db3b0ea6f..59fada152e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java
@@ -37,6 +37,7 @@ import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.BrokerUtil;
+import org.apache.doris.common.util.VectorizedUtil;
 import org.apache.doris.load.BrokerFileGroup;
 import org.apache.doris.load.Load;
 import org.apache.doris.load.loadv2.LoadTask;
@@ -268,7 +269,8 @@ public class BrokerScanNode extends LoadScanNode {
 
         Load.initColumns(targetTable, columnDescs,
                 context.fileGroup.getColumnToHadoopFunction(), context.exprMap, analyzer,
-                context.srcTupleDescriptor, context.slotDescByName, context.params);
+                context.srcTupleDescriptor, context.slotDescByName, context.params,
+                formatType(context.fileGroup.getFileFormat(), ""), VectorizedUtil.isVectorized());
     }
 
     private TScanRangeLocations newLocations(TBrokerScanRangeParams params, BrokerDesc brokerDesc)
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java
index 74131db12e..2e508f4f02 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java
@@ -27,6 +27,7 @@ import org.apache.doris.analysis.TupleDescriptor;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.common.UserException;
+import org.apache.doris.common.util.VectorizedUtil;
 import org.apache.doris.load.Load;
 import org.apache.doris.load.loadv2.LoadTask;
 import org.apache.doris.task.LoadTaskInfo;
@@ -140,7 +141,8 @@ public class StreamLoadScanNode extends LoadScanNode {
         }
 
         Load.initColumns(dstTable, columnExprDescs, null /* no hadoop function */,
-                exprsByName, analyzer, srcTupleDesc, slotDescByName, params);
+                exprsByName, analyzer, srcTupleDesc, slotDescByName, params,
+                taskInfo.getFormatType(), VectorizedUtil.isVectorized());
 
         // analyze where statement
         initAndSetPrecedingFilter(taskInfo.getPrecedingFilter(), this.srcTupleDesc, analyzer);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/StreamLoadTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/StreamLoadTask.java
index 2a1ab7f08b..7a01ff9e35 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/StreamLoadTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/StreamLoadTask.java
@@ -260,6 +260,8 @@ public class StreamLoadTask implements LoadTaskInfo {
         }
         switch (request.getFileType()) {
             case FILE_STREAM:
+            // fall through to case FILE_LOCAL
+            case FILE_LOCAL:
                 path = request.getPath();
                 break;
             default:


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