From 000bc5edb841ab76f0562dc9d8923ef8ef06e4dd Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Sat, 7 May 2022 14:31:39 +0800 Subject: [PATCH 01/21] Support parquet scanner in vectorized engine --- be/src/exec/base_scanner.cpp | 17 + be/src/exec/base_scanner.h | 6 + be/src/exec/broker_scan_node.cpp | 13 +- be/src/exec/parquet_reader.cpp | 18 + be/src/exec/parquet_reader.h | 4 + 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/exec/vparquet_reader.cpp | 70 ++++ be/src/vec/exec/vparquet_reader.h | 60 ++++ be/src/vec/exec/vparquet_scanner.cpp | 308 ++++++++++++++++++ be/src/vec/exec/vparquet_scanner.h | 69 ++++ be/src/vec/functions/function_cast.h | 3 +- .../utils/arrow_column_to_doris_column.cpp | 279 ++++++++++++++++ .../vec/utils/arrow_column_to_doris_column.h | 40 +++ .../java/org/apache/doris/analysis/Expr.java | 17 + .../doris/analysis/TupleDescriptor.java | 9 + .../java/org/apache/doris/catalog/Type.java | 4 + .../main/java/org/apache/doris/load/Load.java | 114 ++++++- .../apache/doris/planner/BrokerScanNode.java | 4 +- .../doris/planner/StreamLoadScanNode.java | 4 +- .../org/apache/doris/task/StreamLoadTask.java | 3 + 23 files changed, 1049 insertions(+), 40 deletions(-) create mode 100644 be/src/vec/exec/vparquet_reader.cpp create mode 100644 be/src/vec/exec/vparquet_reader.h create mode 100644 be/src/vec/exec/vparquet_scanner.cpp create mode 100644 be/src/vec/exec/vparquet_scanner.h create mode 100644 be/src/vec/utils/arrow_column_to_doris_column.cpp create mode 100644 be/src/vec/utils/arrow_column_to_doris_column.h diff --git a/be/src/exec/base_scanner.cpp b/be/src/exec/base_scanner.cpp index ca5b08831fdfb2..8dd4b0f6c5db4a 100644 --- a/be/src/exec/base_scanner.cpp +++ b/be/src/exec/base_scanner.cpp @@ -111,6 +111,11 @@ Status BaseScanner::init_expr_ctxes() { Expr::create_expr_trees(_state->obj_pool(), _pre_filter_texprs, &_pre_filter_ctxs)); RETURN_IF_ERROR(Expr::prepare(_pre_filter_ctxs, _state, *_row_desc, _mem_tracker)); RETURN_IF_ERROR(Expr::open(_pre_filter_ctxs, _state)); + // vec + RETURN_IF_ERROR( + vectorized::VExpr::create_expr_trees(_state->obj_pool(), _pre_filter_texprs, &_pre_filter_vctxs)); + RETURN_IF_ERROR(vectorized::VExpr::prepare(_pre_filter_vctxs, _state, *_row_desc, _mem_tracker)); + RETURN_IF_ERROR(vectorized::VExpr::open(_pre_filter_vctxs, _state)); } // Construct dest slots information @@ -138,6 +143,12 @@ Status BaseScanner::init_expr_ctxes() { RETURN_IF_ERROR(ctx->prepare(_state, *_row_desc.get(), _mem_tracker)); RETURN_IF_ERROR(ctx->open(_state)); _dest_expr_ctx.emplace_back(ctx); + // vec + vectorized::VExprContext* vctx = nullptr; + RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(_state->obj_pool(), it->second, &vctx)); + RETURN_IF_ERROR(vctx->prepare(_state, *_row_desc.get(), _mem_tracker)); + RETURN_IF_ERROR(vctx->open(_state)); + _dest_vexpr_ctxs.emplace_back(vctx); if (has_slot_id_map) { auto it = _params.dest_sid_to_src_sid_without_trans.find(slot_desc->id()); if (it == std::end(_params.dest_sid_to_src_sid_without_trans)) { @@ -284,6 +295,12 @@ void BaseScanner::close() { if (!_pre_filter_ctxs.empty()) { Expr::close(_pre_filter_ctxs, _state); } + if (!_pre_filter_vctxs.empty()) { + vectorized::VExpr::close(_pre_filter_vctxs, _state); + } + if (!_dest_vexpr_ctxs.empty()) { + vectorized::VExpr::close(_dest_vexpr_ctxs, _state); + } } } // namespace doris diff --git a/be/src/exec/base_scanner.h b/be/src/exec/base_scanner.h index 13285ab6aa687b..4e824ad5184b94 100644 --- a/be/src/exec/base_scanner.h +++ b/be/src/exec/base_scanner.h @@ -20,6 +20,7 @@ #include "common/status.h" #include "exprs/expr.h" +#include "vec/exprs/vexpr.h" #include "runtime/tuple.h" #include "util/runtime_profile.h" @@ -34,6 +35,7 @@ class RuntimeState; class ExprContext; namespace vectorized { +class VExprContext; class IColumn; using MutableColumnPtr = IColumn::MutablePtr; } // namespace vectorized @@ -95,6 +97,8 @@ class BaseScanner { // Dest tuple descriptor and dest expr context const TupleDescriptor* _dest_tuple_desc; std::vector _dest_expr_ctx; + // for vec + std::vector _dest_vexpr_ctxs; // the map values of dest slot id to src slot desc // if there is not key of dest slot id in dest_sid_to_src_sid_without_trans, it will be set to nullptr std::vector _src_slot_descs_order_by_dest; @@ -104,6 +108,8 @@ class BaseScanner { // and will be converted to `_pre_filter_ctxs` when scanner is open. const std::vector _pre_filter_texprs; std::vector _pre_filter_ctxs; + // for vec + std::vector _pre_filter_vctxs; bool _strict_mode; diff --git a/be/src/exec/broker_scan_node.cpp b/be/src/exec/broker_scan_node.cpp index e1dbd255ecef7e..03f4e6fe830aa8 100644 --- a/be/src/exec/broker_scan_node.cpp +++ b/be/src/exec/broker_scan_node.cpp @@ -22,6 +22,7 @@ #include "common/object_pool.h" #include "vec/exec/vbroker_scanner.h" +#include "vec/exec/vparquet_scanner.h" #include "exec/json_scanner.h" #include "exec/orc_scanner.h" #include "exec/parquet_scanner.h" @@ -224,9 +225,15 @@ std::unique_ptr 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 ddb3531e176f7e..dbd63ec92e46ba 100644 --- a/be/src/exec/parquet_reader.cpp +++ b/be/src/exec/parquet_reader.cpp @@ -219,6 +219,24 @@ Status ParquetReaderWrap::read_record_batch(const std::vector& return Status::OK(); } +Status ParquetReaderWrap::next_batch(std::shared_ptr* batch, + const std::vector& 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& 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& ts_array, uint8_t* buf, int32_t* wbytes) { const auto type = std::static_pointer_cast(ts_array->type()); diff --git a/be/src/exec/parquet_reader.h b/be/src/exec/parquet_reader.h index 2bd5b5a8021141..7bf3f235dfed77 100644 --- a/be/src/exec/parquet_reader.h +++ b/be/src/exec/parquet_reader.h @@ -79,6 +79,9 @@ class ParquetReaderWrap { Status size(int64_t* size); Status init_parquet_reader(const std::vector& tuple_slot_descs, const std::string& timezone); + Status next_batch(std::shared_ptr* batch, + const std::vector& tuple_slot_descs, + bool* eof); private: void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, @@ -86,6 +89,7 @@ class ParquetReaderWrap { Status column_indices(const std::vector& tuple_slot_descs); Status set_field_null(Tuple* tuple, const SlotDescriptor* slot_desc); Status read_record_batch(const std::vector& tuple_slot_descs, bool* eof); + const std::shared_ptr& get_batch(); Status handle_timestamp(const std::shared_ptr& 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 1e54c05d9362df..a34e4a19e6b233 100644 --- a/be/src/exec/parquet_scanner.h +++ b/be/src/exec/parquet_scanner.h @@ -65,11 +65,11 @@ class ParquetScanner : public BaseScanner { // 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& _ranges; const std::vector& _broker_addresses; diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp index d0a1693440a750..b89b60d5b1fc07 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 5944a47c4fa31f..c6eb413aa7a12e 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 @@ class RlePageBuilder : public PageBuilder { 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 7555e9d0ca9222..36580a13b692ae 100644 --- a/be/src/vec/CMakeLists.txt +++ b/be/src/vec/CMakeLists.txt @@ -101,6 +101,7 @@ set(VEC_FILES exec/vtable_function_node.cpp exec/vbroker_scan_node.cpp exec/vbroker_scanner.cpp + exec/vparquet_scanner.cpp exec/join/vhash_join_node.cpp exprs/vectorized_agg_fn.cpp exprs/vectorized_fn_call.cpp @@ -190,6 +191,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/exec/vparquet_reader.cpp b/be/src/vec/exec/vparquet_reader.cpp new file mode 100644 index 00000000000000..2b854ef43d1ab5 --- /dev/null +++ b/be/src/vec/exec/vparquet_reader.cpp @@ -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. +#include "exec/parquet_reader.h" + +#include +#include +#include + +#include "common/logging.h" +#include "exec/file_reader.h" +#include "gen_cpp/PaloBrokerService_types.h" +#include "gen_cpp/TPaloBrokerService.h" +#include "runtime/broker_mgr.h" +#include "runtime/client_cache.h" +#include "runtime/descriptors.h" +#include "runtime/exec_env.h" +#include "runtime/mem_pool.h" +#include "runtime/tuple.h" +#include "util/thrift_util.h" + +namespace doris { +namespace vectorized { + +// Broker +VParquetReaderWrap::VParquetReaderWrap(ParquetReaderWrap* parquet_reader, + const std::vector& src_slot_desc, std::string time_zone) + : _src_slot_descs(src_slot_desc), + _time_zone(std::move(time_zone)), + _inited(false) { + _reader = std::shared_ptr(parquet_reader); +} + +VParquetReaderWrap::~VParquetReaderWrap() { + _reader->close(); +} + +Status VParquetReaderWrap::next_batch(std::shared_ptr* batch) { + if (!inited) { + RETURN_IF_ERROR(_parquet_reader->init_parquet_reader(_src_slot_descs, _time_zone)); + _inited = true; + } else { + bool eof = false; + auto status = _parquet_reader->read_record_batch(_src_slot_descs, &eof); + if (status.is_end_of_file() || eof) { + *batch = nullptr; + return Status::EndOfFile("End Of Parquet File"); + } else if (!status.ok()) { + return status; + } + } + *batch = _parquet_reader->get_batch(); + return Status::OK(); +} + +} // namespace vectorized +} // namespace doris diff --git a/be/src/vec/exec/vparquet_reader.h b/be/src/vec/exec/vparquet_reader.h new file mode 100644 index 00000000000000..6d5b790bbc6e6c --- /dev/null +++ b/be/src/vec/exec/vparquet_reader.h @@ -0,0 +1,60 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "common/status.h" +#include "exec/parquet_reader.h" +#include "gen_cpp/PaloBrokerService_types.h" +#include "gen_cpp/PlanNodes_types.h" +#include "gen_cpp/Types_types.h" + +namespace doris { + +// Reader of broker parquet file +class VParquetReaderWrap { +public: + VParquetReaderWrap(ParquetReaderWrap* reader, + std::vector& src_slot_descs, std::string time_zone); + virtual ~VParquetReaderWrap(); + + // Read + Status next_batch(std::shared_ptr* batch); + +private: + std::shared_ptr _reader; + const std::vector& _src_slot_descs; + bool _inited; + std::string _timezone; +}; + +} // namespace doris diff --git a/be/src/vec/exec/vparquet_scanner.cpp b/be/src/vec/exec/vparquet_scanner.cpp new file mode 100644 index 00000000000000..b7601f85d6543a --- /dev/null +++ b/be/src/vec/exec/vparquet_scanner.cpp @@ -0,0 +1,308 @@ +// 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& ranges, + const std::vector& broker_addresses, + const std::vector& 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(range.num_of_columns_from_file) + : implicit_cast(_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 || _batch == nullptr || _arrow_batch_cur_idx >= _batch->num_rows()) { + while (!_scanner_eof) { + status = next_arrow_batch(); + if (_scanner_eof) { + return status; + } + if (status.is_end_of_file()) { + // try next file + continue; + } + return status; + } + } + return status; +} + +Status VParquetScanner::init_src_block(Block* block) { + size_t batch_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(batch_pos++).get(); + auto pt = arrow_type_to_primitive_type(array->type()->id()); + if (pt == INVALID_TYPE) { + return Status::NotSupported(fmt::format( + "Not support arrow type:{}", array->type()->name())); + } + auto is_nullable = true; + // let src column be nullable for simplify converting + DataTypePtr data_type = DataTypeFactory::instance().create_data_type(pt, is_nullable); + 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(std::vector& columns, 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 + 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; + } + // if src block is not empty, then finalize the block + if (src_block.rows() > 0) { + break; + } + _cur_file_eof = true; + RETURN_IF_ERROR(next_arrow_batch()); + // there may be different arrow file, so reinit block here + RETURN_IF_ERROR(init_src_block(&src_block)); + } + 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, columns)); + *eof = _scanner_eof; + return Status::OK(); +} + +// eval conjuncts, for example: t1 > 1 +Status VParquetScanner::eval_conjunts(Block* block) { + for (auto& vctx : _pre_filter_vctxs) { + 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) { + int start = range.num_of_columns_from_file; + int rows = block->rows(); + for (int 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(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, std::vector& columns) { + 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_ctxs[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(*ptr)) { + if (nullable_column->has_null()) { + // fill filter if src has null value and dest column is not nullable + IColumn::Filter& filter = assert_cast(*filter_column).get_data(); + const ColumnPtr& null_column_ptr = nullable_column->get_null_map_column_ptr(); + const auto& column_data = assert_cast(*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(); + } + } + columns[dest_index] = (*std::move(ptr)).mutate(); + } + const IColumn::Filter& filter = assert_cast(*filter_column).get_data(); + size_t after_filtered_rows = orig_rows; + for (size_t i = 0; i < columns.size(); ++i) { + columns[i] = (*std::move(columns[i]->filter(filter, 0))).mutate(); + after_filtered_rows = columns[i]->size(); + } + _counter->num_rows_filtered += orig_rows - after_filtered_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(), + "" + } + }; + 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((_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 \ No newline at end of file diff --git a/be/src/vec/exec/vparquet_scanner.h b/be/src/vec/exec/vparquet_scanner.h new file mode 100644 index 00000000000000..89a2bba332a8af --- /dev/null +++ b/be/src/vec/exec/vparquet_scanner.h @@ -0,0 +1,69 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include + +#include +#include "common/status.h" +#include +#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& ranges, + const std::vector& broker_addresses, + const std::vector& pre_filter_texprs, ScannerCounter* counter); + + virtual ~VParquetScanner(); + + // Open this scanner, will initialize information need to + Status open(); + + Status get_next(std::vector& columns, 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, std::vector& columns); + void fill_columns_from_path(Block* block); + +private: + std::shared_ptr _batch; + size_t _arrow_batch_cur_idx; + int _num_of_columns_from_file; +}; + +} // namespace doris diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index 092842f5facf97..071874f30d12c7 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -965,8 +965,9 @@ class FunctionCast final : public IFunctionBase { !(check_and_get_data_type(from_type.get()) || check_and_get_data_type(from_type.get()))) { function = FunctionConvertToTimeType::create(); - } else + } else { function = FunctionTo::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 00000000000000..a58d282745fa1e --- /dev/null +++ b/be/src/vec/utils/arrow_column_to_doris_column.cpp @@ -0,0 +1,279 @@ +// 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 "vec/columns/column_nullable.h" +#include "vec/data_types/data_type_decimal.h" +#include "vec/runtime/vdatetime_value.h" + +#include +#include +#include + +#include "arrow/type.h" +#include "arrow/array/array_binary.h" +#include "arrow/array/array_nested.h" +#include "arrow/scalar.h" +#include "arrow/type_fwd.h" +#include "arrow/type_traits.h" + +#define FOR_ARROW_TYPES(M) \ + M(::arrow::Type::BOOL, TYPE_BOOLEAN) \ + M(::arrow::Type::INT8, TYPE_TINYINT) \ + M(::arrow::Type::UINT8, TYPE_SMALLINT) \ + M(::arrow::Type::INT16, TYPE_SMALLINT) \ + M(::arrow::Type::UINT16, TYPE_INT) \ + M(::arrow::Type::INT32, TYPE_INT) \ + M(::arrow::Type::UINT32, TYPE_BIGINT) \ + M(::arrow::Type::INT64, TYPE_BIGINT) \ + M(::arrow::Type::UINT64, TYPE_LARGEINT) \ + 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::INT32, Int32) \ + 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& column_chars_t = assert_cast(*data_column).get_chars(); + PaddedPODArray& column_offsets = assert_cast(*data_column).get_offsets(); + + auto concrete_array = down_cast(array); + std::shared_ptr 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& column_chars_t = assert_cast(*data_column).get_chars(); + PaddedPODArray& column_offsets = assert_cast(*data_column).get_offsets(); + + auto concrete_array = down_cast(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 > +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(*data_column).get_data(); + /// buffers[0] is a null bitmap and buffers[1] are actual values + std::shared_ptr buffer = array->data()->buffers[1]; + const auto * raw_data = reinterpret_cast(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 &>(*data_column).get_data(); + auto concrete_array = down_cast(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 +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 &>(*data_column).get_data(); + auto concrete_array = down_cast(array); + int64_t divisor = 1; + int64_t multiplier = 1; + if constexpr (std::is_same_v) { + const auto type = std::static_pointer_cast(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) { + multiplier = 24 * 60 * 60; // day => secs + } else if constexpr (std::is_same_v) { + 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(concrete_array->Value(value_i)) / divisor * multiplier, timezone) ; + if constexpr (std::is_same_v) { + v.cast_to_date(); + } + column_data.emplace_back(binary_cast(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 &>(*data_column).get_data(); + auto concrete_array = down_cast(array); + const auto* arrow_decimal_type = static_cast(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(concrete_array->Value(value_i)); + // convert scale to 9 + if (scale != 9) { + value = convert_decimals, + vectorized::DataTypeDecimal>( + 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((*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(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_column, arrow_batch_cur_idx, data_column, num_elements, timezone); + case arrow::Type::DATE64: + return convert_column_with_timestamp_data( + arrow_column, arrow_batch_cur_idx, data_column, num_elements, timezone); + case arrow::Type::TIMESTAMP: + return convert_column_with_timestamp_data( + 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 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 00000000000000..a3dcf6508e73cd --- /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 +#include +#include +#include + +#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 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 4ce1224a13a009..c8c3ed77fc9581 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 @@ -187,6 +187,23 @@ public boolean apply(Expr arg) { public boolean apply(Expr arg) { return arg instanceof NullLiteral; } }; + public static final com.google.common.base.Predicate IS_VARCHAR_SLOT_REF_IMPLICIT_CAST = + new com.google.common.base.Predicate() { + @Override + public boolean apply(Expr arg) { + // exclude explicit cast. for example: cast(k1 as date) + if (!arg.isImplicitCast()) { + return false; + } + List 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 e0d11c0331a12c..313e02441ba5dc 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 @@ -106,6 +106,15 @@ public ArrayList getSlots() { return slots; } + 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 471e6599e86ca0..ec9714cf2f40d2 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 boolean isStringType() { || 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 905f9507378a81..ed271ea87c17ca 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.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; @@ -931,7 +932,7 @@ public static List getSchemaChangeShadowColumnDesc(Table tbl, */ public static void initColumns(Table tbl, List columnExprs, Map>> 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); } /* @@ -941,10 +942,11 @@ public static void initColumns(Table tbl, List columnExprs, public static void initColumns(Table tbl, LoadTaskInfo.ImportColumnDescs columnDescs, Map>> columnToHadoopFunction, Map exprsByName, Analyzer analyzer, TupleDescriptor srcTupleDesc, - Map slotDescByName, TBrokerScanRangeParams params) throws UserException { + Map 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); } /* @@ -959,6 +961,7 @@ private static void initColumns(Table tbl, List columnExprs, Map>> columnToHadoopFunction, Map exprsByName, Analyzer analyzer, TupleDescriptor srcTupleDesc, Map 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. @@ -1044,26 +1047,52 @@ private static void initColumns(Table tbl, List columnExprs, if (!needInitSlotAndAnalyzeExprs) { return; } - + Set exprArgsColumns = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + for (ImportColumnDesc importColumnDesc : copiedColumnExprs) { + if (importColumnDesc.isColumn()) { + continue; + } + List slots = Lists.newArrayList(); + importColumnDesc.getExpr().collect(SlotRef.class, slots); + for (SlotRef slot : slots) { + String slotColumnName = slot.getColumnName(); + exprArgsColumns.add(slotColumnName); + } + } + Set 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(); - String realColName = tbl.getColumn(columnName) == null ? columnName + Column tblColumn = tbl.getColumn(columnName); + String realColName = tblColumn == null ? columnName : tbl.getColumn(columnName).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 (exprArgsColumns.contains(columnName) || formatType != TFileFormatType.FORMAT_PARQUET + || !useVectorizedLoad) { + // columns in expr args should be parsed as varchar type + slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); + slotDesc.setColumn(new Column(realColName, PrimitiveType.VARCHAR)); + excludedColumns.add(realColName); + // 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); + } else { + // in vectorized load, + // 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()); + } 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); } @@ -1082,7 +1111,29 @@ private static void initColumns(Table tbl, List columnExprs, } 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 cloneExprsByName = Maps.newHashMap(exprsByName); + Map 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 exprsByName, + Map mvDefineExpr, Map slotDescByName, + boolean useVectorizedLoad) throws UserException { // analyze all exprs for (Map.Entry entry : exprsByName.entrySet()) { ExprSubstitutionMap smap = new ExprSubstitutionMap(); @@ -1146,7 +1197,44 @@ private static void initColumns(Table tbl, List columnExprs, exprsByName.put(entry.getKey(), expr); } - LOG.debug("after init column, exprMap: {}", exprsByName); + } + /** + * @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 exprsByName, TupleDescriptor srcTupleDesc, + Set excludedColumns) throws UserException { + for (Map.Entry entry : exprsByName.entrySet()) { + List 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 a7b9493b88db89..3473bf65ed440a 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 @@ -36,6 +36,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.UserException; import org.apache.doris.common.util.BrokerUtil; +import org.apache.doris.common.util.VectorizedUtil; import org.apache.doris.common.FeConstants; import org.apache.doris.load.BrokerFileGroup; import org.apache.doris.load.Load; @@ -269,7 +270,8 @@ private void initColumns(ParamCreateContext context) throws UserException { 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 0930bad5f0036f..0e51a68b7741ad 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.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; @@ -138,7 +139,8 @@ public void init(Analyzer analyzer) throws UserException { } 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 c3b96d8575a8af..eacb4a86b55603 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 @@ -263,6 +263,9 @@ private void setOptionalFromTSLPutRequest(TStreamLoadPutRequest request) throws case FILE_STREAM: path = request.getPath(); break; + case FILE_LOCAL: + path = request.getPath(); + break; default: throw new UserException("unsupported file type, type=" + request.getFileType()); } From 3c1bd5385625210ca145c335163a983b720d31cd Mon Sep 17 00:00:00 2001 From: yintao <373141588@qq.com> Date: Sat, 7 May 2022 15:06:32 +0800 Subject: [PATCH 02/21] code format --- be/src/exec/base_scanner.cpp | 5 +- be/src/exec/broker_scan_node.cpp | 6 +- be/src/exec/parquet_reader.cpp | 8 +- be/src/exec/parquet_reader.h | 3 +- be/src/vec/exec/vparquet_reader.cpp | 7 +- be/src/vec/exec/vparquet_reader.h | 4 +- be/src/vec/exec/vparquet_scanner.cpp | 70 ++++--- be/src/vec/exec/vparquet_scanner.h | 10 +- .../utils/arrow_column_to_doris_column.cpp | 198 +++++++++--------- .../vec/utils/arrow_column_to_doris_column.h | 10 +- 10 files changed, 163 insertions(+), 158 deletions(-) diff --git a/be/src/exec/base_scanner.cpp b/be/src/exec/base_scanner.cpp index 8dd4b0f6c5db4a..3d5f8a62aacbac 100644 --- a/be/src/exec/base_scanner.cpp +++ b/be/src/exec/base_scanner.cpp @@ -112,9 +112,10 @@ Status BaseScanner::init_expr_ctxes() { RETURN_IF_ERROR(Expr::prepare(_pre_filter_ctxs, _state, *_row_desc, _mem_tracker)); RETURN_IF_ERROR(Expr::open(_pre_filter_ctxs, _state)); // vec + RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(_state->obj_pool(), _pre_filter_texprs, + &_pre_filter_vctxs)); RETURN_IF_ERROR( - vectorized::VExpr::create_expr_trees(_state->obj_pool(), _pre_filter_texprs, &_pre_filter_vctxs)); - RETURN_IF_ERROR(vectorized::VExpr::prepare(_pre_filter_vctxs, _state, *_row_desc, _mem_tracker)); + vectorized::VExpr::prepare(_pre_filter_vctxs, _state, *_row_desc, _mem_tracker)); RETURN_IF_ERROR(vectorized::VExpr::open(_pre_filter_vctxs, _state)); } diff --git a/be/src/exec/broker_scan_node.cpp b/be/src/exec/broker_scan_node.cpp index 03f4e6fe830aa8..597b1de5880f0c 100644 --- a/be/src/exec/broker_scan_node.cpp +++ b/be/src/exec/broker_scan_node.cpp @@ -226,9 +226,9 @@ std::unique_ptr BrokerScanNode::create_scanner(const TBrokerScanRan switch (scan_range.ranges[0].format_type) { case TFileFormatType::FORMAT_PARQUET: if (_vectorized) { - scan = new vectorized::VParquetScanner(_runtime_state, runtime_profile(), scan_range.params, - scan_range.ranges, scan_range.broker_addresses, - _pre_filter_texprs, counter); + 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, diff --git a/be/src/exec/parquet_reader.cpp b/be/src/exec/parquet_reader.cpp index dbd63ec92e46ba..3d2555161c25b7 100644 --- a/be/src/exec/parquet_reader.cpp +++ b/be/src/exec/parquet_reader.cpp @@ -220,11 +220,9 @@ Status ParquetReaderWrap::read_record_batch(const std::vector& } Status ParquetReaderWrap::next_batch(std::shared_ptr* batch, - const std::vector& tuple_slot_descs, - bool* eof) { - if (_batch->num_rows() == 0 || - _current_line_of_batch != 0 || - _current_line_of_group != 0) { + const std::vector& 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(); diff --git a/be/src/exec/parquet_reader.h b/be/src/exec/parquet_reader.h index 7bf3f235dfed77..c62054800d733d 100644 --- a/be/src/exec/parquet_reader.h +++ b/be/src/exec/parquet_reader.h @@ -80,8 +80,7 @@ class ParquetReaderWrap { Status init_parquet_reader(const std::vector& tuple_slot_descs, const std::string& timezone); Status next_batch(std::shared_ptr* batch, - const std::vector& tuple_slot_descs, - bool* eof); + const std::vector& tuple_slot_descs, bool* eof); private: void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, diff --git a/be/src/vec/exec/vparquet_reader.cpp b/be/src/vec/exec/vparquet_reader.cpp index 2b854ef43d1ab5..4f8d34ced4c80b 100644 --- a/be/src/vec/exec/vparquet_reader.cpp +++ b/be/src/vec/exec/vparquet_reader.cpp @@ -37,10 +37,9 @@ namespace vectorized { // Broker VParquetReaderWrap::VParquetReaderWrap(ParquetReaderWrap* parquet_reader, - const std::vector& src_slot_desc, std::string time_zone) - : _src_slot_descs(src_slot_desc), - _time_zone(std::move(time_zone)), - _inited(false) { + const std::vector& src_slot_desc, + std::string time_zone) + : _src_slot_descs(src_slot_desc), _time_zone(std::move(time_zone)), _inited(false) { _reader = std::shared_ptr(parquet_reader); } diff --git a/be/src/vec/exec/vparquet_reader.h b/be/src/vec/exec/vparquet_reader.h index 6d5b790bbc6e6c..2ffe2c8f291c77 100644 --- a/be/src/vec/exec/vparquet_reader.h +++ b/be/src/vec/exec/vparquet_reader.h @@ -43,8 +43,8 @@ namespace doris { // Reader of broker parquet file class VParquetReaderWrap { public: - VParquetReaderWrap(ParquetReaderWrap* reader, - std::vector& src_slot_descs, std::string time_zone); + VParquetReaderWrap(ParquetReaderWrap* reader, std::vector& src_slot_descs, + std::string time_zone); virtual ~VParquetReaderWrap(); // Read diff --git a/be/src/vec/exec/vparquet_scanner.cpp b/be/src/vec/exec/vparquet_scanner.cpp index b7601f85d6543a..40deeb352b8b85 100644 --- a/be/src/vec/exec/vparquet_scanner.cpp +++ b/be/src/vec/exec/vparquet_scanner.cpp @@ -27,16 +27,17 @@ namespace doris::vectorized { VParquetScanner::VParquetScanner(RuntimeState* state, RuntimeProfile* profile, - const TBrokerScanRangeParams& params, - const std::vector& ranges, - const std::vector& broker_addresses, - const std::vector& pre_filter_texprs, ScannerCounter* counter) - : ParquetScanner(state, profile, params, ranges, broker_addresses, pre_filter_texprs, counter), + const TBrokerScanRangeParams& params, + const std::vector& ranges, + const std::vector& broker_addresses, + const std::vector& 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() { -} +VParquetScanner::~VParquetScanner() {} Status VParquetScanner::open() { RETURN_IF_ERROR(ParquetScanner::open()); @@ -85,7 +86,7 @@ Status VParquetScanner::next_arrow_batch() { } Status VParquetScanner::init_arrow_batch_if_necessary() { - // 1. init batch if first time + // 1. init batch if first time // 2. reset reader if end of file Status status; if (_scanner_eof || _batch == nullptr || _arrow_batch_cur_idx >= _batch->num_rows()) { @@ -114,20 +115,21 @@ Status VParquetScanner::init_src_block(Block* block) { auto* array = _batch->column(batch_pos++).get(); auto pt = arrow_type_to_primitive_type(array->type()->id()); if (pt == INVALID_TYPE) { - return Status::NotSupported(fmt::format( - "Not support arrow type:{}", array->type()->name())); + return Status::NotSupported( + fmt::format("Not support arrow type:{}", array->type()->name())); } auto is_nullable = true; // let src column be nullable for simplify converting DataTypePtr data_type = DataTypeFactory::instance().create_data_type(pt, is_nullable); MutableColumnPtr data_column = data_type->create_column(); - block->insert(ColumnWithTypeAndName(std::move(data_column), data_type, slot_desc->col_name())); + block->insert( + ColumnWithTypeAndName(std::move(data_column), data_type, slot_desc->col_name())); } return Status::OK(); } Status VParquetScanner::get_next(std::vector& columns, bool* eof) { - // overall of type converting: + // 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 SCOPED_TIMER(_read_timer); @@ -189,8 +191,7 @@ Status VParquetScanner::get_next(std::vector& columns, bool* e Status VParquetScanner::eval_conjunts(Block* block) { for (auto& vctx : _pre_filter_vctxs) { size_t orig_rows = block->rows(); - RETURN_IF_ERROR( - VExprContext::filter_block(vctx, block, block->columns())); + RETURN_IF_ERROR(VExprContext::filter_block(vctx, block, block->columns())); _counter->num_rows_unselected += orig_rows - block->rows(); } return Status::OK(); @@ -205,13 +206,16 @@ void VParquetScanner::fill_columns_from_path(Block* block) { 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); + 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(column_from_path.c_str()), column_from_path.size()); + data_column->insert_data(const_cast(column_from_path.c_str()), + column_from_path.size()); } - block->insert(ColumnWithTypeAndName(std::move(data_column), data_type, slot_desc->col_name())); + block->insert(ColumnWithTypeAndName(std::move(data_column), data_type, + slot_desc->col_name())); } } } @@ -237,7 +241,8 @@ Status VParquetScanner::materialize_block(Block* block, std::vector(*filter_column).get_data(); const ColumnPtr& null_column_ptr = nullable_column->get_null_map_column_ptr(); - const auto& column_data = assert_cast(*null_column_ptr).get_data(); + const auto& column_data = + assert_cast(*null_column_ptr).get_data(); for (size_t i = 0; i < null_column_ptr->size(); ++i) { filter[i] &= !column_data[i]; } @@ -269,16 +274,13 @@ Status VParquetScanner::cast_src_block(Block* block) { 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(), - "" - } - }; - auto func_cast = SimpleFunctionFactory::instance().get_function("CAST", arguments, return_type); + ColumnsWithTypeAndName arguments { + arg, + {DataTypeString().create_column_const( + arg.column->size(), remove_nullable(return_type)->get_family_name()), + std::make_shared(), ""}}; + 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); } @@ -286,8 +288,8 @@ Status VParquetScanner::cast_src_block(Block* block) { } Status VParquetScanner::append_batch_to_src_block(Block* block) { - size_t num_elements = - std::min((_state->batch_size() - block->rows()), (_batch->num_rows() - _arrow_batch_cur_idx)); + size_t num_elements = std::min((_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]; @@ -296,13 +298,13 @@ Status VParquetScanner::append_batch_to_src_block(Block* block) { } 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())); + 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 \ No newline at end of file +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/exec/vparquet_scanner.h b/be/src/vec/exec/vparquet_scanner.h index 89a2bba332a8af..9a1b2c9b606441 100644 --- a/be/src/vec/exec/vparquet_scanner.h +++ b/be/src/vec/exec/vparquet_scanner.h @@ -38,10 +38,10 @@ namespace doris::vectorized { class VParquetScanner : public ParquetScanner { public: VParquetScanner(RuntimeState* state, RuntimeProfile* profile, - const TBrokerScanRangeParams& params, - const std::vector& ranges, - const std::vector& broker_addresses, - const std::vector& pre_filter_texprs, ScannerCounter* counter); + const TBrokerScanRangeParams& params, + const std::vector& ranges, + const std::vector& broker_addresses, + const std::vector& pre_filter_texprs, ScannerCounter* counter); virtual ~VParquetScanner(); @@ -66,4 +66,4 @@ class VParquetScanner : public ParquetScanner { int _num_of_columns_from_file; }; -} // namespace doris +} // namespace doris::vectorized diff --git a/be/src/vec/utils/arrow_column_to_doris_column.cpp b/be/src/vec/utils/arrow_column_to_doris_column.cpp index a58d282745fa1e..d9bad4c65e5a1c 100644 --- a/be/src/vec/utils/arrow_column_to_doris_column.cpp +++ b/be/src/vec/utils/arrow_column_to_doris_column.cpp @@ -32,55 +32,56 @@ #include "arrow/type_fwd.h" #include "arrow/type_traits.h" -#define FOR_ARROW_TYPES(M) \ - M(::arrow::Type::BOOL, TYPE_BOOLEAN) \ - M(::arrow::Type::INT8, TYPE_TINYINT) \ - M(::arrow::Type::UINT8, TYPE_SMALLINT) \ - M(::arrow::Type::INT16, TYPE_SMALLINT) \ - M(::arrow::Type::UINT16, TYPE_INT) \ - M(::arrow::Type::INT32, TYPE_INT) \ - M(::arrow::Type::UINT32, TYPE_BIGINT) \ - M(::arrow::Type::INT64, TYPE_BIGINT) \ - M(::arrow::Type::UINT64, TYPE_LARGEINT) \ - 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) \ +#define FOR_ARROW_TYPES(M) \ + M(::arrow::Type::BOOL, TYPE_BOOLEAN) \ + M(::arrow::Type::INT8, TYPE_TINYINT) \ + M(::arrow::Type::UINT8, TYPE_SMALLINT) \ + M(::arrow::Type::INT16, TYPE_SMALLINT) \ + M(::arrow::Type::UINT16, TYPE_INT) \ + M(::arrow::Type::INT32, TYPE_INT) \ + M(::arrow::Type::UINT32, TYPE_BIGINT) \ + M(::arrow::Type::INT64, TYPE_BIGINT) \ + M(::arrow::Type::UINT64, TYPE_LARGEINT) \ + 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::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::INT32, Int32) \ - 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) +#define FOR_ARROW_NUMERIC_TYPES(M) \ + M(arrow::Type::UINT8, UInt8) \ + M(arrow::Type::INT8, Int8) \ + M(arrow::Type::INT16, Int16) \ + M(arrow::Type::INT32, Int32) \ + 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; + switch (type) { +#define DISPATCH(ARROW_TYPE, CPP_TYPE) \ + case ARROW_TYPE: \ + return CPP_TYPE; FOR_ARROW_TYPES(DISPATCH) -# undef DISPATCH - default: - break; +#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) { +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) { @@ -95,16 +96,16 @@ static size_t fill_nullable_column(const arrow::Array* array, size_t array_idx, /// 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& column_chars_t = assert_cast(*data_column).get_chars(); - PaddedPODArray& column_offsets = assert_cast(*data_column).get_offsets(); + MutableColumnPtr& data_column, size_t num_elements) { + PaddedPODArray& column_chars_t = assert_cast(*data_column).get_chars(); + PaddedPODArray& column_offsets = assert_cast(*data_column).get_offsets(); auto concrete_array = down_cast(array); std::shared_ptr 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); + 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'); @@ -115,9 +116,10 @@ static Status convert_column_with_string_data(const arrow::Array* array, size_t } static Status convert_column_with_fixed_size_data(const arrow::Array* array, size_t array_idx, - MutableColumnPtr& data_column, size_t num_elements) { - PaddedPODArray& column_chars_t = assert_cast(*data_column).get_chars(); - PaddedPODArray& column_offsets = assert_cast(*data_column).get_offsets(); + MutableColumnPtr& data_column, + size_t num_elements) { + PaddedPODArray& column_chars_t = assert_cast(*data_column).get_chars(); + PaddedPODArray& column_offsets = assert_cast(*data_column).get_offsets(); auto concrete_array = down_cast(array); uint32_t width = concrete_array->byte_width(); @@ -125,7 +127,7 @@ static Status convert_column_with_fixed_size_data(const arrow::Array* array, siz 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); + const auto* raw_data = array_data + (offset_i * width); column_chars_t.insert(raw_data, raw_data + width); } column_chars_t.emplace_back('\0'); @@ -137,18 +139,18 @@ static Status convert_column_with_fixed_size_data(const arrow::Array* array, siz /// Inserts numeric data right into internal column data to reduce an overhead template > 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(*data_column).get_data(); + MutableColumnPtr& data_column, size_t num_elements) { + auto& column_data = static_cast(*data_column).get_data(); /// buffers[0] is a null bitmap and buffers[1] are actual values std::shared_ptr buffer = array->data()->buffers[1]; - const auto * raw_data = reinterpret_cast(buffer->data()) + array_idx; + const auto* raw_data = reinterpret_cast(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 &>(*data_column).get_data(); + MutableColumnPtr& data_column, size_t num_elements) { + auto& column_data = static_cast&>(*data_column).get_data(); auto concrete_array = down_cast(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)); @@ -178,9 +180,9 @@ static int64_t time_unit_divisor(arrow::TimeUnit::type unit) { template 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 &>(*data_column).get_data(); + MutableColumnPtr& data_column, size_t num_elements, + const std::string& timezone) { + auto& column_data = static_cast&>(*data_column).get_data(); auto concrete_array = down_cast(array); int64_t divisor = 1; int64_t multiplier = 1; @@ -198,7 +200,8 @@ static Status convert_column_with_timestamp_data(const arrow::Array* array, size for (size_t value_i = array_idx; value_i < array_idx + num_elements; ++value_i) { VecDateTimeValue v; - v.from_unixtime(static_cast(concrete_array->Value(value_i)) / divisor * multiplier, timezone) ; + v.from_unixtime(static_cast(concrete_array->Value(value_i)) / divisor * multiplier, + timezone); if constexpr (std::is_same_v) { v.cast_to_date(); } @@ -208,72 +211,77 @@ static Status convert_column_with_timestamp_data(const arrow::Array* array, size } 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 &>(*data_column).get_data(); + MutableColumnPtr& data_column, size_t num_elements) { + auto& column_data = + static_cast&>(*data_column).get_data(); auto concrete_array = down_cast(array); - const auto* arrow_decimal_type = static_cast(array->type().get()); + const auto* arrow_decimal_type = static_cast(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(concrete_array->Value(value_i)); + auto value = + *reinterpret_cast(concrete_array->Value(value_i)); // convert scale to 9 if (scale != 9) { - value = convert_decimals, - vectorized::DataTypeDecimal>( - value, scale, 9); + value = convert_decimals, + vectorized::DataTypeDecimal>(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, +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((*std::move(doirs_column.column)).mutate().get()); + auto* nullable_column = reinterpret_cast( + (*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(arrow_column, arrow_batch_cur_idx, data_column, num_elements); + 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( \ + 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_column, arrow_batch_cur_idx, data_column, num_elements, timezone); - case arrow::Type::DATE64: - return convert_column_with_timestamp_data( - arrow_column, arrow_batch_cur_idx, data_column, num_elements, timezone); - case arrow::Type::TIMESTAMP: - return convert_column_with_timestamp_data( - 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; +#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_column, arrow_batch_cur_idx, data_column, num_elements, timezone); + case arrow::Type::DATE64: + return convert_column_with_timestamp_data( + arrow_column, arrow_batch_cur_idx, data_column, num_elements, timezone); + case arrow::Type::TIMESTAMP: + return convert_column_with_timestamp_data( + 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())); + return Status::NotSupported( + fmt::format("Not support arrow type:{}", arrow_column->type()->name())); } -} // namespace doris +} // 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 index a3dcf6508e73cd..e308a69fe5188f 100644 --- a/be/src/vec/utils/arrow_column_to_doris_column.h +++ b/be/src/vec/utils/arrow_column_to_doris_column.h @@ -27,14 +27,12 @@ #include "vec/core/column_with_type_and_name.h" // This files contains some utilities to convert Doris internal -// data format from Apache Arrow format. +// 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, +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 +} // namespace doris::vectorized From 7f8ba23fe65258a526480fd36ef306b3171ca676 Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Sat, 7 May 2022 15:29:36 +0800 Subject: [PATCH 03/21] fix compile problem --- be/src/vec/utils/arrow_column_to_doris_column.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/be/src/vec/utils/arrow_column_to_doris_column.cpp b/be/src/vec/utils/arrow_column_to_doris_column.cpp index d9bad4c65e5a1c..6d22d03acd28aa 100644 --- a/be/src/vec/utils/arrow_column_to_doris_column.cpp +++ b/be/src/vec/utils/arrow_column_to_doris_column.cpp @@ -31,6 +31,7 @@ #include "arrow/scalar.h" #include "arrow/type_fwd.h" #include "arrow/type_traits.h" +#include "gutil/casts.h" #define FOR_ARROW_TYPES(M) \ M(::arrow::Type::BOOL, TYPE_BOOLEAN) \ From 67064d6e6b0e5562ed4e8e79db7cea7abc13548f Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Sun, 8 May 2022 22:16:37 +0800 Subject: [PATCH 04/21] add unittest for parquet vec --- .../utils/arrow_column_to_doris_column.cpp | 10 +- be/test/CMakeLists.txt | 1 + .../arrow_column_to_doris_column_test.cpp | 651 ++++++++++++++++++ 3 files changed, 658 insertions(+), 4 deletions(-) create mode 100644 be/test/vec/utils/arrow_column_to_doris_column_test.cpp diff --git a/be/src/vec/utils/arrow_column_to_doris_column.cpp b/be/src/vec/utils/arrow_column_to_doris_column.cpp index 6d22d03acd28aa..28cc6c490708ae 100644 --- a/be/src/vec/utils/arrow_column_to_doris_column.cpp +++ b/be/src/vec/utils/arrow_column_to_doris_column.cpp @@ -36,13 +36,13 @@ #define FOR_ARROW_TYPES(M) \ M(::arrow::Type::BOOL, TYPE_BOOLEAN) \ M(::arrow::Type::INT8, TYPE_TINYINT) \ - M(::arrow::Type::UINT8, TYPE_SMALLINT) \ + M(::arrow::Type::UINT8, TYPE_TINYINT) \ M(::arrow::Type::INT16, TYPE_SMALLINT) \ - M(::arrow::Type::UINT16, TYPE_INT) \ + M(::arrow::Type::UINT16, TYPE_SMALLINT) \ M(::arrow::Type::INT32, TYPE_INT) \ - M(::arrow::Type::UINT32, TYPE_BIGINT) \ + M(::arrow::Type::UINT32, TYPE_INT) \ M(::arrow::Type::INT64, TYPE_BIGINT) \ - M(::arrow::Type::UINT64, TYPE_LARGEINT) \ + 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) \ @@ -58,7 +58,9 @@ 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) \ diff --git a/be/test/CMakeLists.txt b/be/test/CMakeLists.txt index 5abf9cceb22386..e2a07ccf41a3fe 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 00000000000000..f7a641b597230d --- /dev/null +++ b/be/test/vec/utils/arrow_column_to_doris_column_test.cpp @@ -0,0 +1,651 @@ + +// 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 + +#include +#include +#include +#include "vec/columns/column_nullable.h" +#include "vec/data_types/data_type_decimal.h" +#include "vec/runtime/vdatetime_value.h" +#include "vec/data_types/data_type_factory.hpp" +#include "vec/functions/simple_function_factory.h" + +#include +#include +#include + +#include "arrow/type.h" +#include "arrow/array/array_binary.h" +#include "arrow/array/array_nested.h" +#include "arrow/scalar.h" +#include "arrow/type_fwd.h" +#include "arrow/type_traits.h" +#include +#include +#include +#include +#include "arrow/array/builder_base.h" +#include "arrow/type_fwd.h" +#include "arrow/type_traits.h" +#include "gutil/casts.h" +#include + +namespace doris::vectorized { + +template ::CType> +ArrowCppType string_to_arrow_datetime(std::shared_ptr 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) { + 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) { + return unix_seconds / (24 * 3600); + } else if constexpr (std::is_same_v) { + return unix_seconds * 1000L; + } else { + assert(false); + } + return 0; +} + +template ::CType> +std::shared_ptr create_constant_numeric_array(size_t num_elements, ArrowCppType value, + std::shared_ptr type, size_t& counter) { + std::vector> 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::ArrayType; + auto array_data = std::make_shared(type, num_elements, buffers); + auto array = std::make_shared(array_data); + return std::static_pointer_cast(array); +} + +template ::CType> +void test_arrow_to_datetime_column(std::shared_ptr 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(num_elements, arrow_datetime, type, counter); + std::string time_zone = "UTC"; + if constexpr (std::is_same_v) { + 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( + (*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(*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(datetime); + ASSERT_EQ(datetime_data[idx], val); + } + } else { + auto val = binary_cast(datetime); + ASSERT_EQ(datetime_data[idx], val); + } + } +} + +template +void test_datetime(std::shared_ptr type, const std::vector& test_cases, size_t num_elements) { + using ArrowCppType = typename arrow::TypeTraits::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(type, value); + VecDateTimeValue tv; + tv.from_date_str(value.c_str(), value.size()); + test_arrow_to_datetime_column(type, column, num_elements, arrow_datetime, tv, counter); + } +} + +TEST(ArrowColumnToDorisColumnTest, test_date32_to_date) { + auto type = std::make_shared(); + std::vector test_cases = { + {"1970-01-01"}, + {"2021-05-30"}, + {"2022-05-08"} + }; + test_datetime, false>(type, test_cases, 32); + test_datetime, true>(type, test_cases, 32); +} + +TEST(ArrowColumnToDorisColumnTest, test_date64_to_datetime) { + auto type = std::make_shared(); + std::vector test_cases = { + {"1970-01-01 12:12:12"}, + {"2021-05-30 22:22:22"}, + {"2022-05-08 00:00:01"} + }; + test_datetime, false>(type, test_cases, 64); + test_datetime, true>(type, test_cases, 64); +} + +TEST(ArrowColumnToDorisColumnTest, test_timestamp_to_datetime) { + auto type = std::make_shared(); + std::vector test_cases = { + {"1970-01-01 12:12:12"}, + {"2021-05-30 22:22:22"}, + {"2022-05-08 00:00:01"} + }; + std::vector zones = { + "UTC", + "GMT", + "CST", + "+01:00", + "-09:00", + "Asia/Shanghai", + "Europe/Zurich" + }; + std::vector 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(unit, zone); + test_datetime, false>(type, test_cases, 64); + test_datetime, true>(type, test_cases, 64); + } + } +} + +template , + typename ArrowCppType = typename arrow::TypeTraits::CType> +void test_arrow_to_numeric_column(std::shared_ptr 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(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( + (*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(*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 > +void test_numeric(std::shared_ptr type, const std::vector& test_cases, size_t num_elements) { + using ArrowCppType = typename arrow::TypeTraits::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(type, column, num_elements, ArrowCppType(value), value, counter); + } +} + +TEST(ArrowColumnToDorisColumnTest, test_int8) { + auto type = std::make_shared(); + std::vector test_cases = { + 1, + -1, + -128, + 127, + int8_t(255) + }; + test_numeric(type, test_cases, 64); + test_numeric(type, test_cases, 64); +} + +TEST(ArrowColumnToDorisColumnTest, test_uint8) { + auto type = std::make_shared(); + std::vector test_cases = { + uint8_t(-1), + uint8_t(1), + uint8_t(-128), + uint8_t(127), + uint8_t(255) + }; + test_numeric(type, test_cases, 64); + test_numeric(type, test_cases, 64); +} + +TEST(ArrowColumnToDorisColumnTest, test_uint16) { + auto type = std::make_shared(); + std::vector test_cases = { + uint16_t(-1), + uint16_t(1), + uint16_t(-128), + uint16_t(127), + uint16_t(65535) + }; + test_numeric(type, test_cases, 64); + test_numeric(type, test_cases, 64); +} + +TEST(ArrowColumnToDorisColumnTest, test_uint32) { + auto type = std::make_shared(); + std::vector test_cases = { + uint32_t(-1), + uint32_t(1), + uint32_t(-65535), + uint32_t(65535), + uint32_t(4294967295) + }; + test_numeric(type, test_cases, 64); + test_numeric(type, test_cases, 64); +} + +TEST(ArrowColumnToDorisColumnTest, test_uint64) { + auto type = std::make_shared(); + std::vector test_cases = { + uint64_t(-1), + uint64_t(1), + uint64_t(-4294967295), + uint64_t(4294967295), + uint64_t(std::numeric_limits::min()), + uint64_t(std::numeric_limits::max()) + }; + test_numeric(type, test_cases, 64); + test_numeric(type, test_cases, 64); +} + +TEST(ArrowColumnToDorisColumnTest, test_float64) { + auto type = std::make_shared(); + std::vector test_cases = { + double(-1.11f), + double(1.11f), + double(-4294967295), + double(4294967295), + double(std::numeric_limits::min()), + double(std::numeric_limits::max()) + }; + test_numeric(type, test_cases, 64); + test_numeric(type, test_cases, 64); +} + +template +std::shared_ptr create_decimal_array(size_t num_elements, int128_t decimal, + std::shared_ptr type, + size_t& counter) { + std::vector> 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(type, num_elements, buffers); + auto array = std::make_shared(array_data); + counter += num_elements; + return array; +} + +template +void test_arrow_to_decimal_column(std::shared_ptr 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(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( + (*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&>(*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 +void test_decimalv2(std::shared_ptr type, const std::vector& test_cases, size_t num_elements) { + using ArrowCppType = typename arrow::TypeTraits::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(decimal_value); + int128_t expect_value = convert_decimals, + vectorized::DataTypeDecimal>(value, + type->scale(), 9); + test_arrow_to_decimal_column(type, column, num_elements, value, expect_value, counter); + } +} + +TEST(ArrowColumnToDorisColumnTest, test_decimalv2) { + std::vector test_cases = { + "1.2345678", + "-12.34567890", + "99999999999.99999999", + "-99999999999.99999999" + }; + auto type_p27s9 = std::make_shared(27, 9); + test_decimalv2(type_p27s9, test_cases, 64); + test_decimalv2(type_p27s9, test_cases, 64); + + auto type_p27s25 = std::make_shared(27, 25); + test_decimalv2(type_p27s25, test_cases, 128); + test_decimalv2(type_p27s25, test_cases, 128); +} + +template +static inline std::shared_ptr 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 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 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(bytes_width); + auto array = std::make_shared(type, num_elements, data_buf, null_bitmap_buf); + counter += num_elements; + return std::static_pointer_cast(array); +} + +template +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(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( + (*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(*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 +void test_fixed_binary(const std::vector& 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(column, num_elements, value, counter); + } +} + +TEST(ArrowColumnToDorisColumnTest, test_fixed_binary) { + std::vector 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 +static inline std::shared_ptr 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 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 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 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::ArrayType; + auto array = std::make_shared(num_elements, offsets_buf, data_buf, null_bitmap); + counter += num_elements; + return std::static_pointer_cast(array); +} + +template ::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(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( + (*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(*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 +void test_binary(const std::vector& 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(column, num_elements, value, counter); + } +} + +TEST(ArrowColumnToDorisColumnTest, test_binary) { + std::vector test_cases = { + "1.2345678", + "-12.34567890", + "99999999999.99999999", + "-99999999999.99999999" + }; + test_binary(test_cases, 64); + test_binary(test_cases, 64); + + test_binary(test_cases, 64); + test_binary(test_cases, 64); +} +} // namespace doris::vectorized From c515fdc1ab0f2243ef3fcb312e5e5a039440e146 Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Sun, 8 May 2022 22:37:20 +0800 Subject: [PATCH 05/21] code format --- be/src/vec/exec/vparquet_reader.cpp | 69 ---------------------------- be/src/vec/exec/vparquet_reader.h | 60 ------------------------ be/src/vec/exec/vparquet_scanner.cpp | 40 ++++++++-------- be/src/vec/exec/vparquet_scanner.h | 16 +++---- 4 files changed, 28 insertions(+), 157 deletions(-) delete mode 100644 be/src/vec/exec/vparquet_reader.cpp delete mode 100644 be/src/vec/exec/vparquet_reader.h diff --git a/be/src/vec/exec/vparquet_reader.cpp b/be/src/vec/exec/vparquet_reader.cpp deleted file mode 100644 index 4f8d34ced4c80b..00000000000000 --- a/be/src/vec/exec/vparquet_reader.cpp +++ /dev/null @@ -1,69 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. -#include "exec/parquet_reader.h" - -#include -#include -#include - -#include "common/logging.h" -#include "exec/file_reader.h" -#include "gen_cpp/PaloBrokerService_types.h" -#include "gen_cpp/TPaloBrokerService.h" -#include "runtime/broker_mgr.h" -#include "runtime/client_cache.h" -#include "runtime/descriptors.h" -#include "runtime/exec_env.h" -#include "runtime/mem_pool.h" -#include "runtime/tuple.h" -#include "util/thrift_util.h" - -namespace doris { -namespace vectorized { - -// Broker -VParquetReaderWrap::VParquetReaderWrap(ParquetReaderWrap* parquet_reader, - const std::vector& src_slot_desc, - std::string time_zone) - : _src_slot_descs(src_slot_desc), _time_zone(std::move(time_zone)), _inited(false) { - _reader = std::shared_ptr(parquet_reader); -} - -VParquetReaderWrap::~VParquetReaderWrap() { - _reader->close(); -} - -Status VParquetReaderWrap::next_batch(std::shared_ptr* batch) { - if (!inited) { - RETURN_IF_ERROR(_parquet_reader->init_parquet_reader(_src_slot_descs, _time_zone)); - _inited = true; - } else { - bool eof = false; - auto status = _parquet_reader->read_record_batch(_src_slot_descs, &eof); - if (status.is_end_of_file() || eof) { - *batch = nullptr; - return Status::EndOfFile("End Of Parquet File"); - } else if (!status.ok()) { - return status; - } - } - *batch = _parquet_reader->get_batch(); - return Status::OK(); -} - -} // namespace vectorized -} // namespace doris diff --git a/be/src/vec/exec/vparquet_reader.h b/be/src/vec/exec/vparquet_reader.h deleted file mode 100644 index 2ffe2c8f291c77..00000000000000 --- a/be/src/vec/exec/vparquet_reader.h +++ /dev/null @@ -1,60 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -#include "common/status.h" -#include "exec/parquet_reader.h" -#include "gen_cpp/PaloBrokerService_types.h" -#include "gen_cpp/PlanNodes_types.h" -#include "gen_cpp/Types_types.h" - -namespace doris { - -// Reader of broker parquet file -class VParquetReaderWrap { -public: - VParquetReaderWrap(ParquetReaderWrap* reader, std::vector& src_slot_descs, - std::string time_zone); - virtual ~VParquetReaderWrap(); - - // Read - Status next_batch(std::shared_ptr* batch); - -private: - std::shared_ptr _reader; - const std::vector& _src_slot_descs; - bool _inited; - std::string _timezone; -}; - -} // namespace doris diff --git a/be/src/vec/exec/vparquet_scanner.cpp b/be/src/vec/exec/vparquet_scanner.cpp index 40deeb352b8b85..392198414443f9 100644 --- a/be/src/vec/exec/vparquet_scanner.cpp +++ b/be/src/vec/exec/vparquet_scanner.cpp @@ -62,7 +62,7 @@ Status VParquetScanner::open() { } // get next available arrow batch -Status VParquetScanner::next_arrow_batch() { +Status VParquetScanner::_next_arrow_batch() { _arrow_batch_cur_idx = 0; // first, init file reader if (_cur_file_reader == nullptr || _cur_file_eof) { @@ -85,13 +85,13 @@ Status VParquetScanner::next_arrow_batch() { return Status::EndOfFile("EOF"); } -Status VParquetScanner::init_arrow_batch_if_necessary() { +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 || _batch == nullptr || _arrow_batch_cur_idx >= _batch->num_rows()) { while (!_scanner_eof) { - status = next_arrow_batch(); + status = _next_arrow_batch(); if (_scanner_eof) { return status; } @@ -105,7 +105,7 @@ Status VParquetScanner::init_arrow_batch_if_necessary() { return status; } -Status VParquetScanner::init_src_block(Block* block) { +Status VParquetScanner::_init_src_block(Block* block) { size_t batch_pos = 0; for (auto i = 0; i < _num_of_columns_from_file; ++i) { SlotDescriptor* slot_desc = _src_slot_descs[i]; @@ -135,7 +135,7 @@ Status VParquetScanner::get_next(std::vector& columns, bool* e SCOPED_TIMER(_read_timer); // init arrow batch { - Status st = init_arrow_batch_if_necessary(); + Status st = _init_arrow_batch_if_necessary(); if (!st.ok()) { if (!st.is_end_of_file()) { return st; @@ -145,17 +145,17 @@ Status VParquetScanner::get_next(std::vector& columns, bool* e } } Block src_block; - RETURN_IF_ERROR(init_src_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)); + 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(); + auto status = _next_arrow_batch(); // if ok, append the batch to the src columns if (status.ok()) { continue; @@ -169,26 +169,26 @@ Status VParquetScanner::get_next(std::vector& columns, bool* e break; } _cur_file_eof = true; - RETURN_IF_ERROR(next_arrow_batch()); + RETURN_IF_ERROR(_next_arrow_batch()); // there may be different arrow file, so reinit block here - RETURN_IF_ERROR(init_src_block(&src_block)); + RETURN_IF_ERROR(_init_src_block(&src_block)); } 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)); + 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)); + _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, columns)); + RETURN_IF_ERROR(_materialize_block(&src_block, columns)); *eof = _scanner_eof; return Status::OK(); } // eval conjuncts, for example: t1 > 1 -Status VParquetScanner::eval_conjunts(Block* block) { +Status VParquetScanner::_eval_conjunts(Block* block) { for (auto& vctx : _pre_filter_vctxs) { size_t orig_rows = block->rows(); RETURN_IF_ERROR(VExprContext::filter_block(vctx, block, block->columns())); @@ -197,7 +197,7 @@ Status VParquetScanner::eval_conjunts(Block* block) { return Status::OK(); } -void VParquetScanner::fill_columns_from_path(Block* block) { +void VParquetScanner::_fill_columns_from_path(Block* block) { const TBrokerRangeDesc& range = _ranges.at(_next_range - 1); if (range.__isset.num_of_columns_from_file) { int start = range.num_of_columns_from_file; @@ -220,7 +220,7 @@ void VParquetScanner::fill_columns_from_path(Block* block) { } } -Status VParquetScanner::materialize_block(Block* block, std::vector& columns) { +Status VParquetScanner::_materialize_block(Block* block, std::vector& columns) { int ctx_idx = 0; size_t orig_rows = block->rows(); auto filter_column = ColumnUInt8::create(orig_rows, 1); @@ -264,7 +264,7 @@ Status VParquetScanner::materialize_block(Block* block, std::vector primitive type(PT0) ==cast_src_block==> // primitive type(PT1) ==materialize_block==> dest primitive type -Status VParquetScanner::cast_src_block(Block* block) { +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]; @@ -287,7 +287,7 @@ Status VParquetScanner::cast_src_block(Block* block) { return Status::OK(); } -Status VParquetScanner::append_batch_to_src_block(Block* block) { +Status VParquetScanner::_append_batch_to_src_block(Block* block) { size_t num_elements = std::min((_state->batch_size() - block->rows()), (_batch->num_rows() - _arrow_batch_cur_idx)); size_t column_pos = 0; @@ -307,4 +307,4 @@ Status VParquetScanner::append_batch_to_src_block(Block* block) { return Status::OK(); } -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized diff --git a/be/src/vec/exec/vparquet_scanner.h b/be/src/vec/exec/vparquet_scanner.h index 9a1b2c9b606441..fdf3f1ab4e1a37 100644 --- a/be/src/vec/exec/vparquet_scanner.h +++ b/be/src/vec/exec/vparquet_scanner.h @@ -51,14 +51,14 @@ class VParquetScanner : public ParquetScanner { Status get_next(std::vector& columns, 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, std::vector& columns); - void fill_columns_from_path(Block* block); + 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, std::vector& columns); + void _fill_columns_from_path(Block* block); private: std::shared_ptr _batch; From e414bb4f0643fb28808e2cf90696f7869b61759e Mon Sep 17 00:00:00 2001 From: yintao <373141588@qq.com> Date: Sun, 8 May 2022 22:38:20 +0800 Subject: [PATCH 06/21] code format --- .../utils/arrow_column_to_doris_column.cpp | 12 +- .../arrow_column_to_doris_column_test.cpp | 212 +++++++----------- 2 files changed, 93 insertions(+), 131 deletions(-) diff --git a/be/src/vec/utils/arrow_column_to_doris_column.cpp b/be/src/vec/utils/arrow_column_to_doris_column.cpp index 28cc6c490708ae..0d2d36cee2bc4c 100644 --- a/be/src/vec/utils/arrow_column_to_doris_column.cpp +++ b/be/src/vec/utils/arrow_column_to_doris_column.cpp @@ -36,13 +36,13 @@ #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::UINT8, TYPE_TINYINT) \ M(::arrow::Type::INT16, TYPE_SMALLINT) \ - M(::arrow::Type::UINT16, TYPE_SMALLINT) \ + M(::arrow::Type::UINT16, TYPE_SMALLINT) \ M(::arrow::Type::INT32, TYPE_INT) \ - M(::arrow::Type::UINT32, TYPE_INT) \ + M(::arrow::Type::UINT32, TYPE_INT) \ M(::arrow::Type::INT64, TYPE_BIGINT) \ - M(::arrow::Type::UINT64, 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) \ @@ -58,9 +58,9 @@ M(arrow::Type::UINT8, UInt8) \ M(arrow::Type::INT8, Int8) \ M(arrow::Type::INT16, Int16) \ - M(arrow::Type::UINT16, UInt16) \ + M(arrow::Type::UINT16, UInt16) \ M(arrow::Type::INT32, Int32) \ - M(arrow::Type::UINT32, UInt32) \ + M(arrow::Type::UINT32, UInt32) \ M(arrow::Type::UINT64, UInt64) \ M(arrow::Type::INT64, Int64) \ M(arrow::Type::HALF_FLOAT, Float32) \ 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 index f7a641b597230d..ba03f9c16188ee 100644 --- a/be/test/vec/utils/arrow_column_to_doris_column_test.cpp +++ b/be/test/vec/utils/arrow_column_to_doris_column_test.cpp @@ -84,9 +84,11 @@ ArrowCppType string_to_arrow_datetime(std::shared_ptr type, const std return 0; } -template ::CType> +template ::CType> std::shared_ptr create_constant_numeric_array(size_t num_elements, ArrowCppType value, - std::shared_ptr type, size_t& counter) { + std::shared_ptr type, + size_t& counter) { std::vector> buffers; buffers.resize(2); size_t null_bitmap_byte_size = (num_elements + 7) / 8; @@ -115,10 +117,12 @@ std::shared_ptr create_constant_numeric_array(size_t num_elements, template ::CType> -void test_arrow_to_datetime_column(std::shared_ptr type, ColumnWithTypeAndName& column, size_t num_elements, - ArrowCppType arrow_datetime, VecDateTimeValue datetime, size_t& counter) { +void test_arrow_to_datetime_column(std::shared_ptr 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(num_elements, arrow_datetime, type, counter); + auto array = create_constant_numeric_array(num_elements, arrow_datetime, + type, counter); std::string time_zone = "UTC"; if constexpr (std::is_same_v) { time_zone = type->timezone(); @@ -141,7 +145,7 @@ void test_arrow_to_datetime_column(std::shared_ptr type, ColumnWithTy if (is_nullable) { ASSERT_NE(nullable_column, nullptr); NullMap& map_data = nullable_column->get_null_map_data(); - if (i % 2 == 0) { + if (i % 2 == 0) { ASSERT_EQ(map_data[idx], true); } else { ASSERT_EQ(map_data[idx], false); @@ -156,7 +160,8 @@ void test_arrow_to_datetime_column(std::shared_ptr type, ColumnWithTy } template -void test_datetime(std::shared_ptr type, const std::vector& test_cases, size_t num_elements) { +void test_datetime(std::shared_ptr type, const std::vector& test_cases, + size_t num_elements) { using ArrowCppType = typename arrow::TypeTraits::CType; size_t counter = 0; auto pt = arrow_type_to_primitive_type(type->id()); @@ -168,17 +173,14 @@ void test_datetime(std::shared_ptr type, const std::vector(type, value); VecDateTimeValue tv; tv.from_date_str(value.c_str(), value.size()); - test_arrow_to_datetime_column(type, column, num_elements, arrow_datetime, tv, counter); + test_arrow_to_datetime_column( + type, column, num_elements, arrow_datetime, tv, counter); } } TEST(ArrowColumnToDorisColumnTest, test_date32_to_date) { auto type = std::make_shared(); - std::vector test_cases = { - {"1970-01-01"}, - {"2021-05-30"}, - {"2022-05-08"} - }; + std::vector test_cases = {{"1970-01-01"}, {"2021-05-30"}, {"2022-05-08"}}; test_datetime, false>(type, test_cases, 32); test_datetime, true>(type, test_cases, 32); } @@ -186,10 +188,7 @@ TEST(ArrowColumnToDorisColumnTest, test_date32_to_date) { TEST(ArrowColumnToDorisColumnTest, test_date64_to_datetime) { auto type = std::make_shared(); std::vector test_cases = { - {"1970-01-01 12:12:12"}, - {"2021-05-30 22:22:22"}, - {"2022-05-08 00:00:01"} - }; + {"1970-01-01 12:12:12"}, {"2021-05-30 22:22:22"}, {"2022-05-08 00:00:01"}}; test_datetime, false>(type, test_cases, 64); test_datetime, true>(type, test_cases, 64); } @@ -197,25 +196,12 @@ TEST(ArrowColumnToDorisColumnTest, test_date64_to_datetime) { TEST(ArrowColumnToDorisColumnTest, test_timestamp_to_datetime) { auto type = std::make_shared(); std::vector test_cases = { - {"1970-01-01 12:12:12"}, - {"2021-05-30 22:22:22"}, - {"2022-05-08 00:00:01"} - }; - std::vector zones = { - "UTC", - "GMT", - "CST", - "+01:00", - "-09:00", - "Asia/Shanghai", - "Europe/Zurich" - }; - std::vector time_units = { - arrow::TimeUnit::SECOND, - arrow::TimeUnit::MICRO, - arrow::TimeUnit::MILLI, - arrow::TimeUnit::NANO - }; + {"1970-01-01 12:12:12"}, {"2021-05-30 22:22:22"}, {"2022-05-08 00:00:01"}}; + std::vector zones = {"UTC", "GMT", "CST", "+01:00", + "-09:00", "Asia/Shanghai", "Europe/Zurich"}; + std::vector 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(unit, zone); @@ -228,10 +214,12 @@ TEST(ArrowColumnToDorisColumnTest, test_timestamp_to_datetime) { template , typename ArrowCppType = typename arrow::TypeTraits::CType> -void test_arrow_to_numeric_column(std::shared_ptr type, ColumnWithTypeAndName& column, size_t num_elements, - ArrowCppType arrow_numeric, CppType numeric, size_t& counter) { +void test_arrow_to_numeric_column(std::shared_ptr 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(num_elements, arrow_numeric, type, counter); + auto array = create_constant_numeric_array(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); @@ -250,7 +238,7 @@ void test_arrow_to_numeric_column(std::shared_ptr type, ColumnWithTyp if (is_nullable) { ASSERT_NE(nullable_column, nullptr); NullMap& map_data = nullable_column->get_null_map_data(); - if (i % 2 == 0) { + if (i % 2 == 0) { ASSERT_EQ(map_data[idx], true); } else { ASSERT_EQ(map_data[idx], false); @@ -264,7 +252,8 @@ void test_arrow_to_numeric_column(std::shared_ptr type, ColumnWithTyp template > -void test_numeric(std::shared_ptr type, const std::vector& test_cases, size_t num_elements) { +void test_numeric(std::shared_ptr type, const std::vector& test_cases, + size_t num_elements) { using ArrowCppType = typename arrow::TypeTraits::CType; size_t counter = 0; auto pt = arrow_type_to_primitive_type(type->id()); @@ -273,94 +262,70 @@ void test_numeric(std::shared_ptr type, const std::vector& t 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(type, column, num_elements, ArrowCppType(value), value, counter); + test_arrow_to_numeric_column( + type, column, num_elements, ArrowCppType(value), value, counter); } } TEST(ArrowColumnToDorisColumnTest, test_int8) { auto type = std::make_shared(); - std::vector test_cases = { - 1, - -1, - -128, - 127, - int8_t(255) - }; + std::vector test_cases = {1, -1, -128, 127, int8_t(255)}; test_numeric(type, test_cases, 64); test_numeric(type, test_cases, 64); } TEST(ArrowColumnToDorisColumnTest, test_uint8) { auto type = std::make_shared(); - std::vector test_cases = { - uint8_t(-1), - uint8_t(1), - uint8_t(-128), - uint8_t(127), - uint8_t(255) - }; + std::vector test_cases = {uint8_t(-1), uint8_t(1), uint8_t(-128), uint8_t(127), + uint8_t(255)}; test_numeric(type, test_cases, 64); test_numeric(type, test_cases, 64); } TEST(ArrowColumnToDorisColumnTest, test_uint16) { auto type = std::make_shared(); - std::vector test_cases = { - uint16_t(-1), - uint16_t(1), - uint16_t(-128), - uint16_t(127), - uint16_t(65535) - }; + std::vector test_cases = {uint16_t(-1), uint16_t(1), uint16_t(-128), uint16_t(127), + uint16_t(65535)}; test_numeric(type, test_cases, 64); test_numeric(type, test_cases, 64); } TEST(ArrowColumnToDorisColumnTest, test_uint32) { auto type = std::make_shared(); - std::vector test_cases = { - uint32_t(-1), - uint32_t(1), - uint32_t(-65535), - uint32_t(65535), - uint32_t(4294967295) - }; + std::vector test_cases = {uint32_t(-1), uint32_t(1), uint32_t(-65535), uint32_t(65535), + uint32_t(4294967295)}; test_numeric(type, test_cases, 64); test_numeric(type, test_cases, 64); } TEST(ArrowColumnToDorisColumnTest, test_uint64) { auto type = std::make_shared(); - std::vector test_cases = { - uint64_t(-1), - uint64_t(1), - uint64_t(-4294967295), - uint64_t(4294967295), - uint64_t(std::numeric_limits::min()), - uint64_t(std::numeric_limits::max()) - }; + std::vector test_cases = {uint64_t(-1), + uint64_t(1), + uint64_t(-4294967295), + uint64_t(4294967295), + uint64_t(std::numeric_limits::min()), + uint64_t(std::numeric_limits::max())}; test_numeric(type, test_cases, 64); test_numeric(type, test_cases, 64); } TEST(ArrowColumnToDorisColumnTest, test_float64) { auto type = std::make_shared(); - std::vector test_cases = { - double(-1.11f), - double(1.11f), - double(-4294967295), - double(4294967295), - double(std::numeric_limits::min()), - double(std::numeric_limits::max()) - }; + std::vector test_cases = {double(-1.11f), + double(1.11f), + double(-4294967295), + double(4294967295), + double(std::numeric_limits::min()), + double(std::numeric_limits::max())}; test_numeric(type, test_cases, 64); test_numeric(type, test_cases, 64); } template std::shared_ptr create_decimal_array(size_t num_elements, int128_t decimal, - std::shared_ptr type, - size_t& counter) { + std::shared_ptr type, + size_t& counter) { std::vector> buffers; buffers.resize(2); auto byte_width = type->byte_width(); @@ -385,7 +350,8 @@ std::shared_ptr create_decimal_array(size_t num_elements, int128_t } template -void test_arrow_to_decimal_column(std::shared_ptr type, ColumnWithTypeAndName& column, size_t num_elements, +void test_arrow_to_decimal_column(std::shared_ptr 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(num_elements, arrow_value, type, counter); @@ -401,13 +367,14 @@ void test_arrow_to_decimal_column(std::shared_ptr type, C } else { data_column = (*std::move(column.column)).mutate(); } - auto& decimal_data = static_cast&>(*data_column).get_data(); + auto& decimal_data = + static_cast&>(*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) { + if (i % 2 == 0) { ASSERT_EQ(map_data[idx], true); } else { ASSERT_EQ(map_data[idx], false); @@ -420,7 +387,8 @@ void test_arrow_to_decimal_column(std::shared_ptr type, C } template -void test_decimalv2(std::shared_ptr type, const std::vector& test_cases, size_t num_elements) { +void test_decimalv2(std::shared_ptr type, + const std::vector& test_cases, size_t num_elements) { using ArrowCppType = typename arrow::TypeTraits::CType; size_t counter = 0; auto pt = arrow_type_to_primitive_type(type->id()); @@ -431,20 +399,18 @@ void test_decimalv2(std::shared_ptr type, const std::vect for (auto& str : test_cases) { DecimalV2Value decimal_value(str); int128_t value = binary_cast(decimal_value); - int128_t expect_value = convert_decimals, - vectorized::DataTypeDecimal>(value, - type->scale(), 9); - test_arrow_to_decimal_column(type, column, num_elements, value, expect_value, counter); + int128_t expect_value = + convert_decimals, + vectorized::DataTypeDecimal>( + value, type->scale(), 9); + test_arrow_to_decimal_column(type, column, num_elements, value, expect_value, + counter); } } TEST(ArrowColumnToDorisColumnTest, test_decimalv2) { - std::vector test_cases = { - "1.2345678", - "-12.34567890", - "99999999999.99999999", - "-99999999999.99999999" - }; + std::vector test_cases = {"1.2345678", "-12.34567890", "99999999999.99999999", + "-99999999999.99999999"}; auto type_p27s9 = std::make_shared(27, 9); test_decimalv2(type_p27s9, test_cases, 64); test_decimalv2(type_p27s9, test_cases, 64); @@ -456,8 +422,8 @@ TEST(ArrowColumnToDorisColumnTest, test_decimalv2) { template static inline std::shared_ptr create_fixed_size_binary_array(int64_t num_elements, - const std::string& value, - size_t& counter) { + 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 data_buf = std::move(data_buf_tmp.ValueOrDie()); @@ -478,16 +444,18 @@ static inline std::shared_ptr create_fixed_size_binary_array(int64 p += bytes_width; } auto type = std::make_shared(bytes_width); - auto array = std::make_shared(type, num_elements, data_buf, null_bitmap_buf); + auto array = std::make_shared(type, num_elements, data_buf, + null_bitmap_buf); counter += num_elements; return std::static_pointer_cast(array); } template void test_arrow_to_fixed_binary_column(ColumnWithTypeAndName& column, size_t num_elements, - const std::string value, size_t& counter) { + const std::string value, size_t& counter) { ASSERT_EQ(column.column->size(), counter); - auto array = create_fixed_size_binary_array(num_elements, value, counter); + auto array = + create_fixed_size_binary_array(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); @@ -508,7 +476,7 @@ void test_arrow_to_fixed_binary_column(ColumnWithTypeAndName& column, size_t num if (is_nullable) { ASSERT_NE(nullable_column, nullptr); NullMap& map_data = nullable_column->get_null_map_data(); - if (i % 2 == 0) { + if (i % 2 == 0) { ASSERT_EQ(map_data[idx], true); ASSERT_EQ(s.size, 0); } else { @@ -530,17 +498,14 @@ void test_fixed_binary(const std::vector& test_cases, size_t num_el 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(column, num_elements, value, counter); + test_arrow_to_fixed_binary_column(column, num_elements, value, + counter); } } TEST(ArrowColumnToDorisColumnTest, test_fixed_binary) { - std::vector test_cases = { - "1.2345678", - "-12.34567890", - "99999999999.99999999", - "-99999999999.99999999" - }; + std::vector 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); @@ -549,8 +514,9 @@ TEST(ArrowColumnToDorisColumnTest, test_fixed_binary) { } template -static inline std::shared_ptr create_binary_array(int64_t num_elements, const std::string& value, - size_t& counter) { +static inline std::shared_ptr 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); @@ -589,7 +555,7 @@ static inline std::shared_ptr create_binary_array(int64_t num_elem template ::CType> void test_arrow_to_binary_column(ColumnWithTypeAndName& column, size_t num_elements, - ArrowCppType value, size_t& counter) { + ArrowCppType value, size_t& counter) { ASSERT_EQ(column.column->size(), counter); auto array = create_binary_array(num_elements, value, counter); auto ret = arrow_column_to_doris_column(array.get(), 0, column, num_elements, "UTC"); @@ -611,7 +577,7 @@ void test_arrow_to_binary_column(ColumnWithTypeAndName& column, size_t num_eleme if (is_nullable) { ASSERT_NE(nullable_column, nullptr); NullMap& map_data = nullable_column->get_null_map_data(); - if (i % 2 == 0) { + if (i % 2 == 0) { ASSERT_EQ(map_data[idx], true); ASSERT_EQ(s.size, 0); } else { @@ -636,12 +602,8 @@ void test_binary(const std::vector& test_cases, size_t num_elements } TEST(ArrowColumnToDorisColumnTest, test_binary) { - std::vector test_cases = { - "1.2345678", - "-12.34567890", - "99999999999.99999999", - "-99999999999.99999999" - }; + std::vector test_cases = {"1.2345678", "-12.34567890", "99999999999.99999999", + "-99999999999.99999999"}; test_binary(test_cases, 64); test_binary(test_cases, 64); From 4e3e21bad67a91afcd336d083afba6ea45a90c89 Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Mon, 9 May 2022 12:44:28 +0800 Subject: [PATCH 07/21] fix fe unittest & be parquet numeric overflow bug --- 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 | 9 ++- .../vec/utils/arrow_column_to_doris_column.h | 1 + .../main/java/org/apache/doris/load/Load.java | 48 ++++++++------ 5 files changed, 99 insertions(+), 25 deletions(-) diff --git a/be/src/vec/data_types/data_type_factory.cpp b/be/src/vec/data_types/data_type_factory.cpp index 555c6417ce8e5f..557b978c8e72ba 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(); + break; + case ::arrow::Type::INT8: + nested = std::make_shared(); + break; + case ::arrow::Type::UINT8: + nested = std::make_shared(); + break; + case ::arrow::Type::INT16: + nested = std::make_shared(); + break; + case ::arrow::Type::UINT16: + nested = std::make_shared(); + break; + case ::arrow::Type::INT32: + nested = std::make_shared(); + break; + case ::arrow::Type::UINT32: + nested = std::make_shared(); + break; + case ::arrow::Type::INT64: + nested = std::make_shared(); + break; + case ::arrow::Type::UINT64: + nested = std::make_shared(); + break; + case ::arrow::Type::HALF_FLOAT: + case ::arrow::Type::FLOAT: + nested = std::make_shared(); + break; + case ::arrow::Type::DOUBLE: + nested = std::make_shared(); + break; + case ::arrow::Type::DATE32: + nested = std::make_shared(); + break; + case ::arrow::Type::DATE64: + case ::arrow::Type::TIMESTAMP: + nested = std::make_shared(); + break; + case ::arrow::Type::BINARY: + case ::arrow::Type::FIXED_SIZE_BINARY: + case ::arrow::Type::STRING: + nested = std::make_shared(); + break; + case ::arrow::Type::DECIMAL: + nested = std::make_shared>(27, 9); + break; + default: + DCHECK(false) << "invalid arrow type:" << (int)type; + break; + } + + if (nested && is_nullable) { + return std::make_shared(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 4ecb29048792eb..3b667c6f725c82 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 #include +#include "arrow/type.h" #include "gen_cpp/data.pb.h" #include "olap/field.h" #include "olap/tablet_schema.h" @@ -87,6 +88,8 @@ class DataTypeFactory { 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 index 392198414443f9..21fef9b207b7f7 100644 --- a/be/src/vec/exec/vparquet_scanner.cpp +++ b/be/src/vec/exec/vparquet_scanner.cpp @@ -113,14 +113,13 @@ Status VParquetScanner::_init_src_block(Block* block) { continue; } auto* array = _batch->column(batch_pos++).get(); - auto pt = arrow_type_to_primitive_type(array->type()->id()); - if (pt == INVALID_TYPE) { + // let src column be nullable for simplify converting + 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())); } - auto is_nullable = true; - // let src column be nullable for simplify converting - DataTypePtr data_type = DataTypeFactory::instance().create_data_type(pt, is_nullable); MutableColumnPtr data_column = data_type->create_column(); block->insert( ColumnWithTypeAndName(std::move(data_column), data_type, slot_desc->col_name())); diff --git a/be/src/vec/utils/arrow_column_to_doris_column.h b/be/src/vec/utils/arrow_column_to_doris_column.h index e308a69fe5188f..841d604d53833f 100644 --- a/be/src/vec/utils/arrow_column_to_doris_column.h +++ b/be/src/vec/utils/arrow_column_to_doris_column.h @@ -35,4 +35,5 @@ 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/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 ed271ea87c17ca..f48d32b13e8960 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 @@ -1065,32 +1065,37 @@ private static void initColumns(Table tbl, List columnExprs, // make column name case match with real column name String columnName = importColumnDesc.getColumnName(); Column tblColumn = tbl.getColumn(columnName); - String realColName = tblColumn == null ? columnName - : tbl.getColumn(columnName).getName(); + String realColName = tblColumn == null ? columnName : tblColumn.getName(); if (importColumnDesc.getExpr() != null) { Expr expr = transformHadoopFunctionExpr(tbl, realColName, importColumnDesc.getExpr()); exprsByName.put(realColName, expr); } else { SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); // only support parquet format now - if (exprArgsColumns.contains(columnName) || formatType != TFileFormatType.FORMAT_PARQUET - || !useVectorizedLoad) { - // columns in expr args should be parsed as varchar type + if (useVectorizedLoad && formatType == TFileFormatType.FORMAT_PARQUET + && tblColumn != null) { + // in vectorized load + if (exprArgsColumns.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); + } 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)); - excludedColumns.add(realColName); // 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); - } else { - // in vectorized load, - // 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()); } slotDesc.setIsMaterialized(true); params.addToSrcSlotIds(slotDesc.getId().asInt()); @@ -1142,14 +1147,17 @@ private static void analyzeAllExprs(Table tbl, Analyzer analyzer, Maptype()->name())); From cdc0aa860104e70eeaa0723f96ec337fbda6ba3c Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Mon, 9 May 2022 13:04:21 +0800 Subject: [PATCH 09/21] fix complile problem --- be/src/vec/exec/vparquet_scanner.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/be/src/vec/exec/vparquet_scanner.cpp b/be/src/vec/exec/vparquet_scanner.cpp index c54d2c1854c814..2cd9aacaae41e8 100644 --- a/be/src/vec/exec/vparquet_scanner.cpp +++ b/be/src/vec/exec/vparquet_scanner.cpp @@ -189,7 +189,7 @@ Status VParquetScanner::get_next(std::vector& columns, bool* e // eval conjuncts, for example: t1 > 1 Status VParquetScanner::_eval_conjunts(Block* block) { - for (auto& vctx : _pre_filter_vctxs) { + 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(); @@ -230,7 +230,7 @@ Status VParquetScanner::_materialize_block(Block* block, std::vector dest primitive type RETURN_IF_ERROR(ctx->execute(block, &result_column_id)); From fd830656be8e79cdb286b401bb15d59a7b751cd3 Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Mon, 9 May 2022 15:08:56 +0800 Subject: [PATCH 10/21] adapter new get_next interface for vec parquet --- be/src/vec/exec/vparquet_scanner.cpp | 24 +++++++++++++----------- be/src/vec/exec/vparquet_scanner.h | 4 ++-- 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/be/src/vec/exec/vparquet_scanner.cpp b/be/src/vec/exec/vparquet_scanner.cpp index 2cd9aacaae41e8..0ca100c1f9fe37 100644 --- a/be/src/vec/exec/vparquet_scanner.cpp +++ b/be/src/vec/exec/vparquet_scanner.cpp @@ -128,7 +128,7 @@ Status VParquetScanner::_init_src_block(Block* block) { return Status::OK(); } -Status VParquetScanner::get_next(std::vector& columns, bool* eof) { +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 @@ -182,7 +182,7 @@ Status VParquetScanner::get_next(std::vector& columns, bool* e _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, columns)); + RETURN_IF_ERROR(_materialize_block(&src_block, block)); *eof = _scanner_eof; return Status::OK(); } @@ -220,7 +220,7 @@ void VParquetScanner::_fill_columns_from_path(Block* block) { } } -Status VParquetScanner::_materialize_block(Block* block, std::vector& columns) { +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); @@ -250,15 +250,17 @@ Status VParquetScanner::_materialize_block(Block* block, std::vectorget_nested_column_ptr(); } } - columns[dest_index] = (*std::move(ptr)).mutate(); + dest_block->insert(vectorized::ColumnWithTypeAndName(std::move(ptr), + slot_desc->get_data_type_ptr(), + slot_desc->col_name())); } - const IColumn::Filter& filter = assert_cast(*filter_column).get_data(); - size_t after_filtered_rows = orig_rows; - for (size_t i = 0; i < columns.size(); ++i) { - columns[i] = (*std::move(columns[i]->filter(filter, 0))).mutate(); - after_filtered_rows = columns[i]->size(); - } - _counter->num_rows_filtered += orig_rows - after_filtered_rows; + size_t dest_size = dest_block->columns(); + // do filter + dest_block->insert(vectorized::ColumnWithTypeAndName(std::move(filter_column), + std::make_shared(), + "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(); } diff --git a/be/src/vec/exec/vparquet_scanner.h b/be/src/vec/exec/vparquet_scanner.h index fdf3f1ab4e1a37..3bcb3b16d511fc 100644 --- a/be/src/vec/exec/vparquet_scanner.h +++ b/be/src/vec/exec/vparquet_scanner.h @@ -48,7 +48,7 @@ class VParquetScanner : public ParquetScanner { // Open this scanner, will initialize information need to Status open(); - Status get_next(std::vector& columns, bool* eof); + Status get_next(Block* block, bool* eof); private: Status _next_arrow_batch(); @@ -57,7 +57,7 @@ class VParquetScanner : public ParquetScanner { Status _append_batch_to_src_block(Block* block); Status _cast_src_block(Block* block); Status _eval_conjunts(Block* block); - Status _materialize_block(Block* block, std::vector& columns); + Status _materialize_block(Block* block, Block* dest_block); void _fill_columns_from_path(Block* block); private: From 3a545dcbb8782d670809baa389fa68cdec475ae3 Mon Sep 17 00:00:00 2001 From: yintao <373141588@qq.com> Date: Mon, 9 May 2022 15:26:39 +0800 Subject: [PATCH 11/21] code format --- be/src/vec/exec/vparquet_scanner.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/be/src/vec/exec/vparquet_scanner.cpp b/be/src/vec/exec/vparquet_scanner.cpp index 0ca100c1f9fe37..bd3b3bd3cb0e75 100644 --- a/be/src/vec/exec/vparquet_scanner.cpp +++ b/be/src/vec/exec/vparquet_scanner.cpp @@ -250,15 +250,14 @@ Status VParquetScanner::_materialize_block(Block* block, Block* dest_block) { 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())); + 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(), - "filter column")); + dest_block->insert(vectorized::ColumnWithTypeAndName( + std::move(filter_column), std::make_shared(), + "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(); From 3fa84138085666113e79c74461bba99560fd76f2 Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Mon, 9 May 2022 17:41:12 +0800 Subject: [PATCH 12/21] code format --- be/test/vec/utils/arrow_column_to_doris_column_test.cpp | 1 - .../java/org/apache/doris/analysis/TupleDescriptor.java | 4 ++++ fe/fe-core/src/main/java/org/apache/doris/load/Load.java | 8 +++++--- .../main/java/org/apache/doris/task/StreamLoadTask.java | 3 +-- 4 files changed, 10 insertions(+), 6 deletions(-) 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 index ba03f9c16188ee..f953e4b9e08e51 100644 --- a/be/test/vec/utils/arrow_column_to_doris_column_test.cpp +++ b/be/test/vec/utils/arrow_column_to_doris_column_test.cpp @@ -1,4 +1,3 @@ - // 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 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 313e02441ba5dc..07970dce36c3c5 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 @@ -106,6 +106,10 @@ public ArrayList getSlots() { return slots; } + /** + * @param slotId + * @return this slot's desc + */ public SlotDescriptor getSlot(int slotId) { for (SlotDescriptor slotDesc : slots) { if (slotDesc.getId().asInt() == slotId) { 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 f48d32b13e8960..cfc585c1620be2 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 @@ -1073,7 +1073,7 @@ private static void initColumns(Table tbl, List columnExprs, SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); // only support parquet format now if (useVectorizedLoad && formatType == TFileFormatType.FORMAT_PARQUET - && tblColumn != null) { + && tblColumn != null) { // in vectorized load if (exprArgsColumns.contains(columnName)) { // columns in expr args should be varchar type @@ -1117,7 +1117,8 @@ private static void initColumns(Table tbl, List columnExprs, 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 + // 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 cloneExprsByName = Maps.newHashMap(exprsByName); @@ -1206,8 +1207,9 @@ private static void analyzeAllExprs(Table tbl, Analyzer analyzer, Map exprsByName, TupleDescriptor srcTupleDesc, 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 eacb4a86b55603..8db2ada9ec3e27 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 @@ -261,8 +261,7 @@ private void setOptionalFromTSLPutRequest(TStreamLoadPutRequest request) throws } switch (request.getFileType()) { case FILE_STREAM: - path = request.getPath(); - break; + // fall through to case FILE_LOCAL case FILE_LOCAL: path = request.getPath(); break; From e79aec18f5eef40a8ecfd03570124ee22508e8e8 Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Tue, 10 May 2022 15:40:52 +0800 Subject: [PATCH 13/21] fix bug and add more comments --- be/src/vec/exec/vparquet_scanner.cpp | 20 +++++++++++++++---- .../doris/analysis/TupleDescriptor.java | 3 ++- .../main/java/org/apache/doris/load/Load.java | 4 ++++ 3 files changed, 22 insertions(+), 5 deletions(-) diff --git a/be/src/vec/exec/vparquet_scanner.cpp b/be/src/vec/exec/vparquet_scanner.cpp index bd3b3bd3cb0e75..85737091d1ca48 100644 --- a/be/src/vec/exec/vparquet_scanner.cpp +++ b/be/src/vec/exec/vparquet_scanner.cpp @@ -89,7 +89,7 @@ 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 || _batch == nullptr || _arrow_batch_cur_idx >= _batch->num_rows()) { + if (!_scanner_eof || _batch == nullptr || _arrow_batch_cur_idx >= _batch->num_rows()) { while (!_scanner_eof) { status = _next_arrow_batch(); if (_scanner_eof) { @@ -107,6 +107,7 @@ Status VParquetScanner::_init_arrow_batch_if_necessary() { 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) { @@ -114,6 +115,7 @@ Status VParquetScanner::_init_src_block(Block* block) { } 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); @@ -132,6 +134,16 @@ 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 { @@ -200,9 +212,9 @@ Status VParquetScanner::_eval_conjunts(Block* block) { void VParquetScanner::_fill_columns_from_path(Block* block) { const TBrokerRangeDesc& range = _ranges.at(_next_range - 1); if (range.__isset.num_of_columns_from_file) { - int start = range.num_of_columns_from_file; - int rows = block->rows(); - for (int i = 0; i < range.columns_from_path.size(); ++i) { + 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(); 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 07970dce36c3c5..cfb2ea0078ac06 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 @@ -107,7 +107,8 @@ public ArrayList getSlots() { } /** - * @param slotId + * get slot desc by slot id + * @param slotId slot id * @return this slot's desc */ public SlotDescriptor getSlot(int slotId) { 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 cfc585c1620be2..314d102a718d2c 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 @@ -1209,6 +1209,10 @@ private static void analyzeAllExprs(Table tbl, Analyzer analyzer, Map Date: Wed, 11 May 2022 13:21:48 +0800 Subject: [PATCH 14/21] remove unused code --- be/src/vec/exec/vparquet_scanner.cpp | 20 ++------------------ 1 file changed, 2 insertions(+), 18 deletions(-) diff --git a/be/src/vec/exec/vparquet_scanner.cpp b/be/src/vec/exec/vparquet_scanner.cpp index 85737091d1ca48..f908ceaa7a90c6 100644 --- a/be/src/vec/exec/vparquet_scanner.cpp +++ b/be/src/vec/exec/vparquet_scanner.cpp @@ -90,17 +90,7 @@ Status VParquetScanner::_init_arrow_batch_if_necessary() { // 2. reset reader if end of file Status status; if (!_scanner_eof || _batch == nullptr || _arrow_batch_cur_idx >= _batch->num_rows()) { - while (!_scanner_eof) { - status = _next_arrow_batch(); - if (_scanner_eof) { - return status; - } - if (status.is_end_of_file()) { - // try next file - continue; - } - return status; - } + return _next_arrow_batch(); } return status; } @@ -176,14 +166,8 @@ Status VParquetScanner::get_next(vectorized::Block* block, bool* eof) { if (!status.is_end_of_file()) { return status; } - // if src block is not empty, then finalize the block - if (src_block.rows() > 0) { - break; - } _cur_file_eof = true; - RETURN_IF_ERROR(_next_arrow_batch()); - // there may be different arrow file, so reinit block here - RETURN_IF_ERROR(_init_src_block(&src_block)); + break; } COUNTER_UPDATE(_rows_read_counter, src_block.rows()); SCOPED_TIMER(_materialize_timer); From 0ff901f87730cf65dbd533e977b0a82d62bb44e1 Mon Sep 17 00:00:00 2001 From: yintao <373141588@qq.com> Date: Thu, 12 May 2022 10:42:02 +0800 Subject: [PATCH 15/21] format code --- be/src/exec/broker_scan_node.cpp | 3 +-- be/src/vec/exec/vparquet_scanner.h | 4 ++-- .../vec/utils/arrow_column_to_doris_column.cpp | 4 ++-- be/src/vec/utils/arrow_column_to_doris_column.h | 4 ++-- .../utils/arrow_column_to_doris_column_test.cpp | 16 +++++++--------- 5 files changed, 14 insertions(+), 17 deletions(-) diff --git a/be/src/exec/broker_scan_node.cpp b/be/src/exec/broker_scan_node.cpp index 42938360741385..8bb3aa256d1a90 100644 --- a/be/src/exec/broker_scan_node.cpp +++ b/be/src/exec/broker_scan_node.cpp @@ -21,8 +21,6 @@ #include #include "common/object_pool.h" -#include "vec/exec/vbroker_scanner.h" -#include "vec/exec/vparquet_scanner.h" #include "exec/json_scanner.h" #include "exec/orc_scanner.h" #include "exec/parquet_scanner.h" @@ -34,6 +32,7 @@ #include "util/runtime_profile.h" #include "util/thread.h" #include "vec/exec/vbroker_scanner.h" +#include "vec/exec/vparquet_scanner.h" namespace doris { diff --git a/be/src/vec/exec/vparquet_scanner.h b/be/src/vec/exec/vparquet_scanner.h index 3bcb3b16d511fc..20f2d89b05e25d 100644 --- a/be/src/vec/exec/vparquet_scanner.h +++ b/be/src/vec/exec/vparquet_scanner.h @@ -21,12 +21,12 @@ #include #include #include -#include #include +#include #include -#include "common/status.h" #include +#include "common/status.h" #include "gen_cpp/PlanNodes_types.h" #include "gen_cpp/Types_types.h" #include "runtime/mem_pool.h" diff --git a/be/src/vec/utils/arrow_column_to_doris_column.cpp b/be/src/vec/utils/arrow_column_to_doris_column.cpp index 0d2d36cee2bc4c..3c1536afbc5f0c 100644 --- a/be/src/vec/utils/arrow_column_to_doris_column.cpp +++ b/be/src/vec/utils/arrow_column_to_doris_column.cpp @@ -21,14 +21,14 @@ #include "vec/data_types/data_type_decimal.h" #include "vec/runtime/vdatetime_value.h" -#include #include +#include #include -#include "arrow/type.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" diff --git a/be/src/vec/utils/arrow_column_to_doris_column.h b/be/src/vec/utils/arrow_column_to_doris_column.h index 841d604d53833f..a244903bd839a1 100644 --- a/be/src/vec/utils/arrow_column_to_doris_column.h +++ b/be/src/vec/utils/arrow_column_to_doris_column.h @@ -17,10 +17,10 @@ #pragma once -#include -#include #include #include +#include +#include #include "common/status.h" #include "runtime/primitive_type.h" 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 index f953e4b9e08e51..90169a119a1f4a 100644 --- a/be/test/vec/utils/arrow_column_to_doris_column_test.cpp +++ b/be/test/vec/utils/arrow_column_to_doris_column_test.cpp @@ -24,29 +24,27 @@ #include #include "vec/columns/column_nullable.h" #include "vec/data_types/data_type_decimal.h" -#include "vec/runtime/vdatetime_value.h" #include "vec/data_types/data_type_factory.hpp" #include "vec/functions/simple_function_factory.h" +#include "vec/runtime/vdatetime_value.h" -#include #include +#include #include -#include "arrow/type.h" -#include "arrow/array/array_binary.h" -#include "arrow/array/array_nested.h" -#include "arrow/scalar.h" -#include "arrow/type_fwd.h" -#include "arrow/type_traits.h" #include #include #include #include +#include +#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 namespace doris::vectorized { From ed7957eee50ff3df1ba3dcedb8427a503a73ad22 Mon Sep 17 00:00:00 2001 From: yintao <373141588@qq.com> Date: Thu, 12 May 2022 10:54:21 +0800 Subject: [PATCH 16/21] code format --- be/src/vec/exec/vparquet_scanner.cpp | 1 + be/src/vec/exec/vparquet_scanner.h | 5 ++-- .../utils/arrow_column_to_doris_column.cpp | 7 ++--- .../vec/utils/arrow_column_to_doris_column.h | 1 + .../arrow_column_to_doris_column_test.cpp | 28 +++++++++---------- 5 files changed, 22 insertions(+), 20 deletions(-) diff --git a/be/src/vec/exec/vparquet_scanner.cpp b/be/src/vec/exec/vparquet_scanner.cpp index f908ceaa7a90c6..5e31807f07392e 100644 --- a/be/src/vec/exec/vparquet_scanner.cpp +++ b/be/src/vec/exec/vparquet_scanner.cpp @@ -16,6 +16,7 @@ // under the License. #include "vec/exec/vparquet_scanner.h" + #include "exec/parquet_reader.h" #include "exprs/expr.h" #include "runtime/descriptors.h" diff --git a/be/src/vec/exec/vparquet_scanner.h b/be/src/vec/exec/vparquet_scanner.h index 20f2d89b05e25d..754aabc4c3d569 100644 --- a/be/src/vec/exec/vparquet_scanner.h +++ b/be/src/vec/exec/vparquet_scanner.h @@ -17,6 +17,9 @@ #pragma once +#include +#include + #include #include #include @@ -24,8 +27,6 @@ #include #include -#include -#include #include "common/status.h" #include "gen_cpp/PlanNodes_types.h" #include "gen_cpp/Types_types.h" diff --git a/be/src/vec/utils/arrow_column_to_doris_column.cpp b/be/src/vec/utils/arrow_column_to_doris_column.cpp index 3c1536afbc5f0c..a02fc92b48ca2f 100644 --- a/be/src/vec/utils/arrow_column_to_doris_column.cpp +++ b/be/src/vec/utils/arrow_column_to_doris_column.cpp @@ -17,10 +17,6 @@ #include "vec/utils/arrow_column_to_doris_column.h" -#include "vec/columns/column_nullable.h" -#include "vec/data_types/data_type_decimal.h" -#include "vec/runtime/vdatetime_value.h" - #include #include #include @@ -32,6 +28,9 @@ #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) \ diff --git a/be/src/vec/utils/arrow_column_to_doris_column.h b/be/src/vec/utils/arrow_column_to_doris_column.h index a244903bd839a1..2e70fee11a2dae 100644 --- a/be/src/vec/utils/arrow_column_to_doris_column.h +++ b/be/src/vec/utils/arrow_column_to_doris_column.h @@ -19,6 +19,7 @@ #include #include + #include #include 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 index 90169a119a1f4a..9cedc8213699da 100644 --- a/be/test/vec/utils/arrow_column_to_doris_column_test.cpp +++ b/be/test/vec/utils/arrow_column_to_doris_column_test.cpp @@ -17,26 +17,21 @@ #include "vec/utils/arrow_column_to_doris_column.h" -#include - -#include -#include -#include -#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" - #include -#include -#include - #include #include +#include +#include #include #include #include + +#include + +#include +#include +#include + #include "arrow/array/array_binary.h" #include "arrow/array/array_nested.h" #include "arrow/array/builder_base.h" @@ -45,6 +40,11 @@ #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 { From 80f1e0dbf25aa28efb66999b89f9db8b71ff4fc8 Mon Sep 17 00:00:00 2001 From: yintao <373141588@qq.com> Date: Thu, 12 May 2022 11:09:38 +0800 Subject: [PATCH 17/21] code format --- be/test/vec/utils/arrow_column_to_doris_column_test.cpp | 1 - 1 file changed, 1 deletion(-) 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 index 9cedc8213699da..0252cf4883ccbd 100644 --- a/be/test/vec/utils/arrow_column_to_doris_column_test.cpp +++ b/be/test/vec/utils/arrow_column_to_doris_column_test.cpp @@ -25,7 +25,6 @@ #include #include #include - #include #include From 3378872507c06ed1413e2b4b19864d9aeb0bf755 Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Thu, 12 May 2022 17:24:58 +0800 Subject: [PATCH 18/21] update fe --- .../main/java/org/apache/doris/load/Load.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) 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 55fa5d835af879..9684db908e7f6d 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 @@ -1047,7 +1047,7 @@ private static void initColumns(Table tbl, List columnExprs, if (!needInitSlotAndAnalyzeExprs) { return; } - Set exprArgsColumns = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + Set exprSrcSlotName = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); for (ImportColumnDesc importColumnDesc : copiedColumnExprs) { if (importColumnDesc.isColumn()) { continue; @@ -1056,14 +1056,16 @@ private static void initColumns(Table tbl, List columnExprs, importColumnDesc.getExpr().collect(SlotRef.class, slots); for (SlotRef slot : slots) { String slotColumnName = slot.getColumnName(); - exprArgsColumns.add(slotColumnName); + exprSrcSlotName.add(slotColumnName); } } + // excludedColumns is columns that should be varchar type Set 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 (tblColumn == null || importColumnDesc.getExpr() == null) { realColName = columnName; @@ -1079,18 +1081,25 @@ private static void initColumns(Table tbl, List columnExprs, if (useVectorizedLoad && formatType == TFileFormatType.FORMAT_PARQUET && tblColumn != null) { // in vectorized load - if (exprArgsColumns.contains(columnName)) { + // 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()); } - // 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)); From a7529b2ce07c2c72bc3e2d323c03368076d95240 Mon Sep 17 00:00:00 2001 From: yintao <373141588@qq.com> Date: Fri, 13 May 2022 14:06:34 +0800 Subject: [PATCH 19/21] format code --- .../java/org/apache/doris/analysis/TupleDescriptor.java | 3 ++- fe/fe-core/src/main/java/org/apache/doris/load/Load.java | 9 +++++---- .../java/org/apache/doris/planner/BrokerScanNode.java | 1 - 3 files changed, 7 insertions(+), 6 deletions(-) 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 9e60e225929571..45fab98fa8315c 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 @@ -106,7 +106,8 @@ public ArrayList getSlots() { } /** - * get slot desc by slot id + * get slot desc by slot id. + * * @param slotId slot id * @return this slot's desc */ 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 280e90b9ad7288..a13d78af16ccb1 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 @@ -1221,15 +1221,16 @@ private static void analyzeAllExprs(Table tbl, Analyzer analyzer, Map exprsByName, TupleDescriptor srcTupleDesc, Set 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 entry : exprsByName.entrySet()) { List casts = Lists.newArrayList(); // exclude explicit cast. for example: cast(k1 as date) 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 ea7dc42297fe23..59fada152ea9e1 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 @@ -38,7 +38,6 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.BrokerUtil; import org.apache.doris.common.util.VectorizedUtil; -import org.apache.doris.common.FeConstants; import org.apache.doris.load.BrokerFileGroup; import org.apache.doris.load.Load; import org.apache.doris.load.loadv2.LoadTask; From 738a778f341be0a16544a9170a64416a5c21aee2 Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Fri, 13 May 2022 16:16:16 +0800 Subject: [PATCH 20/21] fix unit test --- fe/fe-core/src/main/java/org/apache/doris/load/Load.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 a13d78af16ccb1..81e500968edc65 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 @@ -1066,7 +1066,7 @@ private static void initColumns(Table tbl, List columnExprs, String columnName = importColumnDesc.getColumnName(); Column tblColumn = tbl.getColumn(columnName); String realColName; - if (tblColumn == null || importColumnDesc.getExpr() == null) { + if (tblColumn == null || tblColumn.getName() == null || importColumnDesc.getExpr() == null) { realColName = columnName; } else { realColName = tblColumn.getName(); From 0339fb59227bddf369a4188663a42b83385f0f9c Mon Sep 17 00:00:00 2001 From: Tao Yin <373141588@qq.com> Date: Fri, 13 May 2022 16:50:59 +0800 Subject: [PATCH 21/21] fix bug --- be/src/vec/exec/vparquet_scanner.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/be/src/vec/exec/vparquet_scanner.cpp b/be/src/vec/exec/vparquet_scanner.cpp index 5e31807f07392e..6a891850a7256c 100644 --- a/be/src/vec/exec/vparquet_scanner.cpp +++ b/be/src/vec/exec/vparquet_scanner.cpp @@ -90,7 +90,10 @@ 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 || _batch == nullptr || _arrow_batch_cur_idx >= _batch->num_rows()) { + if (_scanner_eof) { + return Status::EndOfFile("EOF"); + } + if (_batch == nullptr || _arrow_batch_cur_idx >= _batch->num_rows()) { return _next_arrow_batch(); } return status;