[refactor](remove broker scan node) it is not useful any more (#16128)

remove broker scannode
remove broker table
remove broker scanner
remove json scanner
remove orc scanner
remove hive external table
remove hudi external table
remove broker external table, user could use broker table value function instead
Co-authored-by: yiguolei <yiguolei@gmail.com>
This commit is contained in:
yiguolei
2023-01-23 19:37:38 +08:00
committed by GitHub
parent 61fccc88d7
commit a3cd0ddbdc
31 changed files with 34 additions and 8846 deletions

View File

@ -46,7 +46,6 @@
#include "vec/exec/vaggregation_node.h"
#include "vec/exec/vanalytic_eval_node.h"
#include "vec/exec/vassert_num_rows_node.h"
#include "vec/exec/vbroker_scan_node.h"
#include "vec/exec/vdata_gen_scan_node.h"
#include "vec/exec/vempty_set_node.h"
#include "vec/exec/vexchange_node.h"
@ -340,7 +339,6 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN
case TPlanNodeType::SELECT_NODE:
case TPlanNodeType::REPEAT_NODE:
case TPlanNodeType::TABLE_FUNCTION_NODE:
case TPlanNodeType::BROKER_SCAN_NODE:
case TPlanNodeType::DATA_GEN_SCAN_NODE:
case TPlanNodeType::FILE_SCAN_NODE:
case TPlanNodeType::JDBC_SCAN_NODE:
@ -445,10 +443,6 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN
*node = pool->add(new vectorized::VExceptNode(pool, tnode, descs));
return Status::OK();
case TPlanNodeType::BROKER_SCAN_NODE:
*node = pool->add(new vectorized::VBrokerScanNode(pool, tnode, descs));
return Status::OK();
case TPlanNodeType::FILE_SCAN_NODE:
*node = pool->add(new vectorized::NewFileScanNode(pool, tnode, descs));
return Status::OK();
@ -529,7 +523,6 @@ void ExecNode::collect_nodes(TPlanNodeType::type node_type, std::vector<ExecNode
void ExecNode::collect_scan_nodes(vector<ExecNode*>* nodes) {
collect_nodes(TPlanNodeType::OLAP_SCAN_NODE, nodes);
collect_nodes(TPlanNodeType::BROKER_SCAN_NODE, nodes);
collect_nodes(TPlanNodeType::ES_HTTP_SCAN_NODE, nodes);
collect_nodes(TPlanNodeType::DATA_GEN_SCAN_NODE, nodes);
collect_nodes(TPlanNodeType::FILE_SCAN_NODE, nodes);

View File

@ -1,59 +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 "operator.h"
#include "vec/exec/vbroker_scan_node.h"
namespace doris::pipeline {
class BrokerScanOperatorBuilder : public OperatorBuilder<vectorized::VBrokerScanNode> {
public:
BrokerScanOperatorBuilder(int32_t id, ExecNode* node)
: OperatorBuilder(id, "BrokerScanOperator", node) {}
bool is_source() const override { return true; }
OperatorPtr build_operator() override;
};
class BrokerScanOperator : public SourceOperator<BrokerScanOperatorBuilder> {
public:
BrokerScanOperator(OperatorBuilderBase* operator_builder, ExecNode* scan_node)
: SourceOperator(operator_builder, scan_node) {}
bool can_read() override { return _node->can_read(); }
bool is_pending_finish() const override { return !_node->can_finish(); }
Status open(RuntimeState* state) override {
SCOPED_TIMER(_runtime_profile->total_time_counter());
RETURN_IF_ERROR(SourceOperator::open(state));
return _node->open(state);
}
Status close(RuntimeState* state) override {
RETURN_IF_ERROR(SourceOperator::close(state));
_node->close(state);
return Status::OK();
}
};
OperatorPtr BrokerScanOperatorBuilder::build_operator() {
return std::make_shared<BrokerScanOperator>(this, _node);
}
} // namespace doris::pipeline

View File

@ -29,7 +29,6 @@
#include "pipeline/exec/analytic_sink_operator.h"
#include "pipeline/exec/analytic_source_operator.h"
#include "pipeline/exec/assert_num_rows_operator.h"
#include "pipeline/exec/broker_scan_operator.h"
#include "pipeline/exec/const_value_operator.h"
#include "pipeline/exec/data_queue.h"
#include "pipeline/exec/datagen_operator.h"
@ -306,12 +305,6 @@ Status PipelineFragmentContext::_build_pipelines(ExecNode* node, PipelinePtr cur
auto node_type = node->type();
switch (node_type) {
// for source
case TPlanNodeType::BROKER_SCAN_NODE: {
OperatorBuilderPtr operator_t =
std::make_shared<BrokerScanOperatorBuilder>(next_operator_builder_id(), node);
RETURN_IF_ERROR(cur_pipe->add_operator(operator_t));
break;
}
case TPlanNodeType::OLAP_SCAN_NODE:
case TPlanNodeType::JDBC_SCAN_NODE:
case TPlanNodeType::ODBC_SCAN_NODE:

View File

@ -840,10 +840,9 @@ void FragmentMgr::_set_scan_concurrency(const TExecPlanFragmentParams& params,
bool FragmentMgr::_is_scan_node(const TPlanNodeType::type& type) {
return type == TPlanNodeType::OLAP_SCAN_NODE || type == TPlanNodeType::MYSQL_SCAN_NODE ||
type == TPlanNodeType::SCHEMA_SCAN_NODE || type == TPlanNodeType::META_SCAN_NODE ||
type == TPlanNodeType::BROKER_SCAN_NODE || type == TPlanNodeType::ES_SCAN_NODE ||
type == TPlanNodeType::ES_HTTP_SCAN_NODE || type == TPlanNodeType::ODBC_SCAN_NODE ||
type == TPlanNodeType::DATA_GEN_SCAN_NODE || type == TPlanNodeType::FILE_SCAN_NODE ||
type == TPlanNodeType::JDBC_SCAN_NODE;
type == TPlanNodeType::ES_SCAN_NODE || type == TPlanNodeType::ES_HTTP_SCAN_NODE ||
type == TPlanNodeType::ODBC_SCAN_NODE || type == TPlanNodeType::DATA_GEN_SCAN_NODE ||
type == TPlanNodeType::FILE_SCAN_NODE || type == TPlanNodeType::JDBC_SCAN_NODE;
}
void FragmentMgr::cancel(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason,

View File

@ -108,12 +108,8 @@ set(VEC_FILES
exec/vassert_num_rows_node.cpp
exec/vrepeat_node.cpp
exec/vtable_function_node.cpp
exec/vbroker_scan_node.cpp
exec/vbroker_scanner.cpp
exec/vjson_scanner.cpp
exec/vjdbc_connector.cpp
exec/vparquet_scanner.cpp
exec/vorc_scanner.cpp
exec/join/vhash_join_node.cpp
exec/join/vjoin_node_base.cpp
exec/join/vnested_loop_join_node.cpp

View File

@ -15,13 +15,14 @@
// specific language governing permissions and limitations
// under the License.
#include "vec/exec/varrow_scanner.h"
#include "exec/arrow/parquet_reader.h"
#include "exprs/expr.h"
#include "io/file_factory.h"
#include "runtime/descriptors.h"
#include "runtime/exec_env.h"
#include "vec/data_types/data_type_factory.hpp"
#include "vec/exec/vorc_scanner.h"
#include "vec/functions/simple_function_factory.h"
#include "vec/utils/arrow_column_to_doris_column.h"

View File

@ -1,347 +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 "vec/exec/vbroker_scan_node.h"
#include "gen_cpp/PlanNodes_types.h"
#include "runtime/memory/mem_tracker.h"
#include "runtime/runtime_state.h"
#include "runtime/tuple.h"
#include "runtime/tuple_row.h"
#include "util/runtime_profile.h"
#include "util/thread.h"
#include "util/types.h"
#include "vec/common/string_ref.h"
#include "vec/exec/vbroker_scanner.h"
#include "vec/exec/vjson_scanner.h"
#include "vec/exec/vorc_scanner.h"
#include "vec/exec/vparquet_scanner.h"
#include "vec/exprs/vexpr_context.h"
namespace doris::vectorized {
VBrokerScanNode::VBrokerScanNode(ObjectPool* pool, const TPlanNode& tnode,
const DescriptorTbl& descs)
: ScanNode(pool, tnode, descs),
_tuple_id(tnode.broker_scan_node.tuple_id),
_runtime_state(nullptr),
_tuple_desc(nullptr),
_num_running_scanners(0),
_scan_finished(false),
_max_buffered_batches(32),
_wait_scanner_timer(nullptr) {}
Status VBrokerScanNode::init(const TPlanNode& tnode, RuntimeState* state) {
RETURN_IF_ERROR(ScanNode::init(tnode, state));
auto& broker_scan_node = tnode.broker_scan_node;
if (broker_scan_node.__isset.pre_filter_exprs) {
_pre_filter_texprs = broker_scan_node.pre_filter_exprs;
}
return Status::OK();
}
Status VBrokerScanNode::prepare(RuntimeState* state) {
VLOG_QUERY << "VBrokerScanNode prepare";
RETURN_IF_ERROR(ScanNode::prepare(state));
// get tuple desc
_runtime_state = state;
_tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id);
if (_tuple_desc == nullptr) {
return Status::InternalError("Failed to get tuple descriptor, _tuple_id={}", _tuple_id);
}
// Initialize slots map
for (auto slot : _tuple_desc->slots()) {
auto pair = _slots_map.emplace(slot->col_name(), slot);
if (!pair.second) {
return Status::InternalError("Failed to insert slot, col_name={}", slot->col_name());
}
}
// Profile
_wait_scanner_timer = ADD_TIMER(runtime_profile(), "WaitScannerTime");
return Status::OK();
}
Status VBrokerScanNode::open(RuntimeState* state) {
START_AND_SCOPE_SPAN(state->get_tracer(), span, "VBrokerScanNode::open");
SCOPED_TIMER(_runtime_profile->total_time_counter());
RETURN_IF_ERROR(ExecNode::open(state));
RETURN_IF_CANCELLED(state);
RETURN_IF_ERROR(start_scanners());
return Status::OK();
}
Status VBrokerScanNode::start_scanners() {
{
std::unique_lock<std::mutex> l(_batch_queue_lock);
_num_running_scanners = 1;
}
_scanner_threads.emplace_back([this, size = 0, length = _scan_ranges.size(),
parent_span = opentelemetry::trace::Tracer::GetCurrentSpan()] {
OpentelemetryScope scope {parent_span};
this->scanner_worker(size, length);
});
return Status::OK();
}
Status VBrokerScanNode::get_next(RuntimeState* state, vectorized::Block* block, bool* eos) {
INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, "VBrokerScanNode::get_next");
SCOPED_TIMER(_runtime_profile->total_time_counter());
// check if CANCELLED.
if (state->is_cancelled()) {
std::unique_lock<std::mutex> l(_batch_queue_lock);
if (update_status(Status::Cancelled("Cancelled"))) {
// Notify all scanners
_queue_writer_cond.notify_all();
}
}
if (_scan_finished.load()) {
*eos = true;
return Status::OK();
}
const int batch_size = _runtime_state->batch_size();
while (true) {
std::shared_ptr<vectorized::Block> scanner_block;
{
std::unique_lock<std::mutex> l(_batch_queue_lock);
while (_process_status.ok() && !_runtime_state->is_cancelled() &&
_num_running_scanners > 0 && _block_queue.empty()) {
SCOPED_TIMER(_wait_scanner_timer);
_queue_reader_cond.wait_for(l, std::chrono::seconds(1));
}
if (!_process_status.ok()) {
// Some scanner process failed.
return _process_status;
}
if (_runtime_state->is_cancelled()) {
if (update_status(Status::Cancelled("Cancelled"))) {
_queue_writer_cond.notify_all();
}
return _process_status;
}
if (!_block_queue.empty()) {
scanner_block = _block_queue.front();
_block_queue.pop_front();
}
}
// All scanner has been finished, and all cached batch has been read
if (!scanner_block) {
if (_mutable_block && !_mutable_block->empty()) {
*block = _mutable_block->to_block();
reached_limit(block, eos);
LOG_IF(INFO, *eos) << "VBrokerScanNode ReachedLimit.";
}
_scan_finished.store(true);
*eos = true;
return Status::OK();
}
// notify one scanner
_queue_writer_cond.notify_one();
if (UNLIKELY(!_mutable_block)) {
_mutable_block.reset(new MutableBlock(scanner_block->clone_empty()));
}
if (_mutable_block->rows() + scanner_block->rows() < batch_size) {
// merge scanner_block into _mutable_block
_mutable_block->add_rows(scanner_block.get(), 0, scanner_block->rows());
continue;
} else {
if (_mutable_block->empty()) {
// directly use scanner_block
*block = *scanner_block;
} else {
// copy _mutable_block firstly, then merge scanner_block into _mutable_block for next.
*block = _mutable_block->to_block();
_mutable_block->set_muatable_columns(scanner_block->clone_empty_columns());
_mutable_block->add_rows(scanner_block.get(), 0, scanner_block->rows());
}
break;
}
}
reached_limit(block, eos);
if (*eos) {
_scan_finished.store(true);
_queue_writer_cond.notify_all();
LOG(INFO) << "VBrokerScanNode ReachedLimit.";
} else {
*eos = false;
}
return Status::OK();
}
Status VBrokerScanNode::close(RuntimeState* state) {
if (is_closed()) {
return Status::OK();
}
START_AND_SCOPE_SPAN(state->get_tracer(), span, "VBrokerScanNode::close");
SCOPED_TIMER(_runtime_profile->total_time_counter());
_scan_finished.store(true);
_queue_writer_cond.notify_all();
_queue_reader_cond.notify_all();
for (int i = 0; i < _scanner_threads.size(); ++i) {
_scanner_threads[i].join();
}
return ExecNode::close(state);
}
Status VBrokerScanNode::scanner_scan(const TBrokerScanRange& scan_range, ScannerCounter* counter) {
//create scanner object and open
std::unique_ptr<BaseScanner> scanner = create_scanner(scan_range, counter);
RETURN_IF_ERROR(scanner->open());
bool scanner_eof = false;
while (!scanner_eof) {
RETURN_IF_CANCELLED(_runtime_state);
// If we have finished all works
if (_scan_finished.load() || !_process_status.ok()) {
return Status::OK();
}
std::shared_ptr<vectorized::Block> block(new vectorized::Block());
RETURN_IF_ERROR(scanner->get_next(block.get(), &scanner_eof));
if (block->rows() == 0) {
continue;
}
auto old_rows = block->rows();
RETURN_IF_ERROR(VExprContext::filter_block(_vconjunct_ctx_ptr, block.get(),
_tuple_desc->slots().size()));
counter->num_rows_unselected += old_rows - block->rows();
if (block->rows() == 0) {
continue;
}
std::unique_lock<std::mutex> l(_batch_queue_lock);
while (_process_status.ok() && !_scan_finished.load() && !_runtime_state->is_cancelled() &&
// stop pushing more batch if
// 1. too many batches in queue, or
// 2. at least one batch in queue and memory exceed limit.
(_block_queue.size() >= _max_buffered_batches || !_block_queue.empty())) {
_queue_writer_cond.wait_for(l, std::chrono::seconds(1));
}
// Process already set failed, so we just return OK
if (!_process_status.ok()) {
return Status::OK();
}
// Scan already finished, just return
if (_scan_finished.load()) {
return Status::OK();
}
// Runtime state is canceled, just return cancel
if (_runtime_state->is_cancelled()) {
return Status::Cancelled("Cancelled");
}
// Queue size Must be smaller than _max_buffered_batches
_block_queue.push_back(std::move(block));
// Notify reader to process
_queue_reader_cond.notify_one();
}
return Status::OK();
}
void VBrokerScanNode::scanner_worker(int start_idx, int length) {
START_AND_SCOPE_SPAN(_runtime_state->get_tracer(), span, "VBrokerScanNode::scanner_worker");
SCOPED_ATTACH_TASK(_runtime_state);
Thread::set_self_name("vbroker_scanner");
Status status = Status::OK();
ScannerCounter counter;
for (int i = 0; i < length && status.ok(); ++i) {
const TBrokerScanRange& scan_range =
_scan_ranges[start_idx + i].scan_range.broker_scan_range;
status = scanner_scan(scan_range, &counter);
if (!status.ok()) {
LOG(WARNING) << "Scanner[" << start_idx + i << "] process failed. status=" << status;
}
}
// Update stats
_runtime_state->update_num_rows_load_filtered(counter.num_rows_filtered);
_runtime_state->update_num_rows_load_unselected(counter.num_rows_unselected);
// scanner is going to finish
{
std::lock_guard<std::mutex> l(_batch_queue_lock);
if (!status.ok()) {
update_status(status);
}
// This scanner will finish
_num_running_scanners--;
}
_queue_reader_cond.notify_all();
// If one scanner failed, others don't need scan any more
if (!status.ok()) {
_queue_writer_cond.notify_all();
}
}
std::unique_ptr<BaseScanner> VBrokerScanNode::create_scanner(const TBrokerScanRange& scan_range,
ScannerCounter* counter) {
BaseScanner* scan = nullptr;
switch (scan_range.ranges[0].format_type) {
case TFileFormatType::FORMAT_PARQUET:
scan = new vectorized::VParquetScanner(_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 vectorized::VORCScanner(_runtime_state, runtime_profile(), scan_range.params,
scan_range.ranges, scan_range.broker_addresses,
_pre_filter_texprs, counter);
break;
case TFileFormatType::FORMAT_JSON:
if (config::enable_simdjson_reader) {
scan = new vectorized::VJsonScanner<vectorized::VSIMDJsonReader>(
_runtime_state, runtime_profile(), scan_range.params, scan_range.ranges,
scan_range.broker_addresses, _pre_filter_texprs, counter);
} else {
scan = new vectorized::VJsonScanner<vectorized::VJsonReader>(
_runtime_state, runtime_profile(), scan_range.params, scan_range.ranges,
scan_range.broker_addresses, _pre_filter_texprs, counter);
}
break;
default:
scan = new vectorized::VBrokerScanner(_runtime_state, runtime_profile(), scan_range.params,
scan_range.ranges, scan_range.broker_addresses,
_pre_filter_texprs, counter);
}
std::unique_ptr<BaseScanner> scanner(scan);
return scanner;
}
// This function is called after plan node has been prepared.
Status VBrokerScanNode::set_scan_ranges(const std::vector<TScanRangeParams>& scan_ranges) {
_scan_ranges = scan_ranges;
return Status::OK();
}
void VBrokerScanNode::debug_string(int ident_level, std::stringstream* out) const {
(*out) << "VBrokerScanNode";
}
} // namespace doris::vectorized

View File

@ -1,115 +0,0 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#pragma once
#include <memory>
#include "common/status.h"
#include "exec/base_scanner.h"
#include "exec/scan_node.h"
#include "gen_cpp/PaloInternalService_types.h"
#include "runtime/descriptors.h"
namespace doris {
class RuntimeState;
class Status;
namespace vectorized {
class VBrokerScanNode final : public ScanNode {
public:
VBrokerScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
~VBrokerScanNode() override = default;
// Called after create this scan node
Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;
// Prepare partition infos & set up timer
Status prepare(RuntimeState* state) override;
// Start broker scan using ParquetScanner or BrokerScanner.
Status open(RuntimeState* state) override;
Status get_next(RuntimeState* state, vectorized::Block* block, bool* eos) override;
// Close the scanner, and report errors.
Status close(RuntimeState* state) override;
// No use
Status set_scan_ranges(const std::vector<TScanRangeParams>& scan_ranges) override;
bool can_read() { return true; }
bool can_finish() const { return _num_running_scanners == 0; }
private:
// Write debug string of this into out.
void debug_string(int indentation_level, std::stringstream* out) const override;
// Update process status to one failed status,
// NOTE: Must hold the mutex of this scan node
bool update_status(const Status& new_status) {
if (_process_status.ok()) {
_process_status = new_status;
return true;
}
return false;
}
std::unique_ptr<BaseScanner> create_scanner(const TBrokerScanRange& scan_range,
ScannerCounter* counter);
Status start_scanners();
void scanner_worker(int start_idx, int length);
// Scan one range
Status scanner_scan(const TBrokerScanRange& scan_range, ScannerCounter* counter);
TupleId _tuple_id;
RuntimeState* _runtime_state;
TupleDescriptor* _tuple_desc;
std::map<std::string, SlotDescriptor*> _slots_map;
std::vector<TScanRangeParams> _scan_ranges;
std::mutex _batch_queue_lock;
std::condition_variable _queue_reader_cond;
std::condition_variable _queue_writer_cond;
std::atomic<int> _num_running_scanners;
std::atomic<bool> _scan_finished;
Status _process_status;
std::vector<std::thread> _scanner_threads;
int _max_buffered_batches;
// The origin preceding filter exprs.
// These exprs will be converted to expr context
// in XXXScanner.
// Because the row descriptor used for these exprs is `src_row_desc`,
// which is initialized in XXXScanner.
std::vector<TExpr> _pre_filter_texprs;
RuntimeProfile::Counter* _wait_scanner_timer;
std::deque<std::shared_ptr<vectorized::Block>> _block_queue;
std::unique_ptr<MutableBlock> _mutable_block;
};
} // namespace vectorized
} // namespace doris

View File

@ -1,471 +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 "vec/exec/vbroker_scanner.h"
#include <fmt/format.h>
#include "common/consts.h"
#include "exec/line_reader.h"
#include "exec/plain_binary_line_reader.h"
#include "exec/plain_text_line_reader.h"
#include "exec/text_converter.h"
#include "exec/text_converter.hpp"
#include "gen_cpp/internal_service.pb.h"
#include "io/file_factory.h"
#include "util/utf8_check.h"
namespace doris::vectorized {
VBrokerScanner::VBrokerScanner(RuntimeState* state, RuntimeProfile* profile,
const TBrokerScanRangeParams& params,
const std::vector<TBrokerRangeDesc>& ranges,
const std::vector<TNetworkAddress>& broker_addresses,
const std::vector<TExpr>& pre_filter_texprs, ScannerCounter* counter)
: BaseScanner(state, profile, params, ranges, broker_addresses, pre_filter_texprs, counter),
_cur_file_reader(nullptr),
_cur_line_reader(nullptr),
_cur_decompressor(nullptr),
_cur_line_reader_eof(false),
_skip_lines(0) {
if (params.__isset.column_separator_length && params.column_separator_length > 1) {
_value_separator = params.column_separator_str;
_value_separator_length = params.column_separator_length;
} else {
_value_separator.push_back(static_cast<char>(params.column_separator));
_value_separator_length = 1;
}
if (params.__isset.line_delimiter_length && params.line_delimiter_length > 1) {
_line_delimiter = params.line_delimiter_str;
_line_delimiter_length = params.line_delimiter_length;
} else {
_line_delimiter.push_back(static_cast<char>(params.line_delimiter));
_line_delimiter_length = 1;
}
if (params.__isset.trim_double_quotes) {
_trim_double_quotes = params.trim_double_quotes;
}
_split_values.reserve(sizeof(Slice) * params.src_slot_ids.size());
_text_converter.reset(new (std::nothrow) TextConverter('\\'));
_src_block_mem_reuse = true;
}
VBrokerScanner::~VBrokerScanner() {
close();
}
Status VBrokerScanner::open() {
RETURN_IF_ERROR(BaseScanner::open()); // base default function
return Status::OK();
}
Status VBrokerScanner::_open_file_reader() {
const TBrokerRangeDesc& range = _ranges[_next_range];
int64_t start_offset = range.start_offset;
if (start_offset != 0) {
start_offset -= 1;
}
//means first range, skip
if (start_offset == 0 && range.header_type.size() > 0) {
std::string header_type = to_lower(range.header_type);
if (header_type == BeConsts::CSV_WITH_NAMES) {
_skip_lines = 1;
} else if (header_type == BeConsts::CSV_WITH_NAMES_AND_TYPES) {
_skip_lines = 2;
}
}
if (range.file_type == TFileType::FILE_STREAM) {
RETURN_IF_ERROR(FileFactory::create_pipe_reader(range.load_id, _cur_file_reader_s));
_real_reader = _cur_file_reader_s.get();
} else {
RETURN_IF_ERROR(FileFactory::create_file_reader(
range.file_type, _state->exec_env(), _profile, _broker_addresses,
_params.properties, range, start_offset, _cur_file_reader));
_real_reader = _cur_file_reader.get();
}
return _real_reader->open();
}
Status VBrokerScanner::_create_decompressor(TFileFormatType::type type) {
if (_cur_decompressor != nullptr) {
delete _cur_decompressor;
_cur_decompressor = nullptr;
}
CompressType compress_type;
switch (type) {
case TFileFormatType::FORMAT_CSV_PLAIN:
case TFileFormatType::FORMAT_JSON:
case TFileFormatType::FORMAT_PROTO:
compress_type = CompressType::UNCOMPRESSED;
break;
case TFileFormatType::FORMAT_CSV_GZ:
compress_type = CompressType::GZIP;
break;
case TFileFormatType::FORMAT_CSV_BZ2:
compress_type = CompressType::BZIP2;
break;
case TFileFormatType::FORMAT_CSV_LZ4FRAME:
compress_type = CompressType::LZ4FRAME;
break;
case TFileFormatType::FORMAT_CSV_LZOP:
compress_type = CompressType::LZOP;
break;
case TFileFormatType::FORMAT_CSV_DEFLATE:
compress_type = CompressType::DEFLATE;
break;
default: {
return Status::InternalError("Unknown format type, cannot inference compress type, type={}",
type);
}
}
RETURN_IF_ERROR(Decompressor::create_decompressor(compress_type, &_cur_decompressor));
return Status::OK();
}
Status VBrokerScanner::_open_line_reader() {
if (_cur_decompressor != nullptr) {
delete _cur_decompressor;
_cur_decompressor = nullptr;
}
if (_cur_line_reader != nullptr) {
delete _cur_line_reader;
_cur_line_reader = nullptr;
}
const TBrokerRangeDesc& range = _ranges[_next_range];
int64_t size = range.size;
if (range.start_offset != 0) {
if (range.format_type != TFileFormatType::FORMAT_CSV_PLAIN) {
return Status::InternalError("For now we do not support split compressed file");
}
size += 1;
// not first range will always skip one line
_skip_lines = 1;
}
// create decompressor.
// _decompressor may be nullptr if this is not a compressed file
RETURN_IF_ERROR(_create_decompressor(range.format_type));
_file_format_type = range.format_type;
// open line reader
switch (range.format_type) {
case TFileFormatType::FORMAT_CSV_PLAIN:
case TFileFormatType::FORMAT_CSV_GZ:
case TFileFormatType::FORMAT_CSV_BZ2:
case TFileFormatType::FORMAT_CSV_LZ4FRAME:
case TFileFormatType::FORMAT_CSV_LZOP:
case TFileFormatType::FORMAT_CSV_DEFLATE:
_cur_line_reader = new PlainTextLineReader(_profile, _real_reader, _cur_decompressor, size,
_line_delimiter, _line_delimiter_length);
break;
case TFileFormatType::FORMAT_PROTO:
_cur_line_reader = new PlainBinaryLineReader(_real_reader);
break;
default: {
return Status::InternalError("Unknown format type, cannot init line reader, type={}",
range.format_type);
}
}
_cur_line_reader_eof = false;
return Status::OK();
}
void VBrokerScanner::close() {
BaseScanner::close();
if (_cur_decompressor != nullptr) {
delete _cur_decompressor;
_cur_decompressor = nullptr;
}
if (_cur_line_reader != nullptr) {
delete _cur_line_reader;
_cur_line_reader = nullptr;
}
}
Status VBrokerScanner::_open_next_reader() {
if (_next_range >= _ranges.size()) {
_scanner_eof = true;
return Status::OK();
}
RETURN_IF_ERROR(_open_file_reader());
RETURN_IF_ERROR(_open_line_reader());
_next_range++;
return Status::OK();
}
Status VBrokerScanner::_line_to_src_tuple(const Slice& line) {
RETURN_IF_ERROR(_line_split_to_values(line));
if (!_success) {
return Status::OK();
}
for (int i = 0; i < _split_values.size(); ++i) {
auto slot_desc = _src_slot_descs[i];
const Slice& value = _split_values[i];
if (slot_desc->is_nullable() && is_null(value)) {
_src_tuple->set_null(slot_desc->null_indicator_offset());
continue;
}
_src_tuple->set_not_null(slot_desc->null_indicator_offset());
void* slot = _src_tuple->get_slot(slot_desc->tuple_offset());
Slice* str_slot = reinterpret_cast<Slice*>(slot);
str_slot->data = value.data;
str_slot->size = value.size;
}
const TBrokerRangeDesc& range = _ranges.at(_next_range - 1);
if (range.__isset.num_of_columns_from_file) {
fill_slots_of_columns_from_path(range.num_of_columns_from_file, range.columns_from_path);
}
return Status::OK();
}
void VBrokerScanner::split_line(const Slice& line) {
_split_values.clear();
if (_file_format_type == TFileFormatType::FORMAT_PROTO) {
PDataRow** ptr = reinterpret_cast<PDataRow**>(line.data);
PDataRow* row = *ptr;
for (const PDataColumn& col : (row)->col()) {
int len = col.value().size();
uint8_t* buf = new uint8_t[len];
memcpy(buf, col.value().c_str(), len);
_split_values.emplace_back(buf, len);
}
delete row;
delete[] ptr;
} else {
const char* value = line.data;
size_t start = 0; // point to the start pos of next col value.
size_t curpos = 0; // point to the start pos of separator matching sequence.
size_t p1 = 0; // point to the current pos of separator matching sequence.
size_t non_space = 0; // point to the last pos of non_space character.
// Separator: AAAA
//
// p1
// ▼
// AAAA
// 1000AAAA2000AAAA
// ▲ ▲
// Start │
// curpos
while (curpos < line.size) {
if (curpos + p1 == line.size || *(value + curpos + p1) != _value_separator[p1]) {
// Not match, move forward:
curpos += (p1 == 0 ? 1 : p1);
p1 = 0;
} else {
p1++;
if (p1 == _value_separator_length) {
// Match a separator
non_space = curpos;
// Trim tailing spaces. Be consistent with hive and trino's behavior.
if (_state->trim_tailing_spaces_for_external_table_query()) {
while (non_space > start && *(value + non_space - 1) == ' ') {
non_space--;
}
}
if (_trim_double_quotes && (non_space - 1) > start &&
*(value + start) == '\"' && *(value + non_space - 1) == '\"') {
start++;
non_space--;
}
_split_values.emplace_back(value + start, non_space - start);
start = curpos + _value_separator_length;
curpos = start;
p1 = 0;
non_space = 0;
}
}
}
CHECK(curpos == line.size) << curpos << " vs " << line.size;
non_space = curpos;
if (_state->trim_tailing_spaces_for_external_table_query()) {
while (non_space > start && *(value + non_space - 1) == ' ') {
non_space--;
}
}
if (_trim_double_quotes && (non_space - 1) > start && *(value + start) == '\"' &&
*(value + non_space - 1) == '\"') {
start++;
non_space--;
}
_split_values.emplace_back(value + start, non_space - start);
}
}
Status VBrokerScanner::_line_split_to_values(const Slice& line) {
bool is_proto_format = _file_format_type == TFileFormatType::FORMAT_PROTO;
if (!is_proto_format && !validate_utf8(line.data, line.size)) {
RETURN_IF_ERROR(_state->append_error_msg_to_file(
[]() -> std::string { return "Unable to display"; },
[]() -> std::string {
fmt::memory_buffer error_msg;
fmt::format_to(error_msg, "{}", "Unable to display");
return fmt::to_string(error_msg);
},
&_scanner_eof));
_counter->num_rows_filtered++;
_success = false;
return Status::OK();
}
split_line(line);
// range of current file
const TBrokerRangeDesc& range = _ranges.at(_next_range - 1);
bool read_by_column_def = false;
if (range.__isset.read_by_column_def) {
read_by_column_def = range.read_by_column_def;
}
const std::vector<std::string>& columns_from_path = range.columns_from_path;
// read data by column definition, resize _split_values to _src_solt_size
if (read_by_column_def) {
// fill slots by NULL
while (_split_values.size() + columns_from_path.size() < _src_slot_descs.size()) {
_split_values.emplace_back(_split_values.back().get_data(), 0);
}
// remove redundant slots
while (_split_values.size() + columns_from_path.size() > _src_slot_descs.size()) {
_split_values.pop_back();
}
} else {
if (_split_values.size() + columns_from_path.size() < _src_slot_descs.size()) {
RETURN_IF_ERROR(_state->append_error_msg_to_file(
[&]() -> std::string {
return is_proto_format ? "" : std::string(line.data, line.size);
},
[&]() -> std::string {
fmt::memory_buffer error_msg;
fmt::format_to(error_msg, "{}",
"actual column number is less than schema column number.");
fmt::format_to(error_msg, "actual number: {}, column separator: [{}], ",
_split_values.size(), _value_separator);
fmt::format_to(error_msg, "line delimiter: [{}], schema number: {}; ",
_line_delimiter, _src_slot_descs.size());
return fmt::to_string(error_msg);
},
&_scanner_eof));
_counter->num_rows_filtered++;
_success = false;
return Status::OK();
} else if (_split_values.size() + columns_from_path.size() > _src_slot_descs.size()) {
RETURN_IF_ERROR(_state->append_error_msg_to_file(
[&]() -> std::string {
return is_proto_format ? "" : std::string(line.data, line.size);
},
[&]() -> std::string {
fmt::memory_buffer error_msg;
fmt::format_to(error_msg, "{}",
"actual column number is more than schema column number.");
fmt::format_to(error_msg, "actual number: {}, column separator: [{}], ",
_split_values.size(), _value_separator);
fmt::format_to(error_msg, "line delimiter: [{}], schema number: {}; ",
_line_delimiter, _src_slot_descs.size());
return fmt::to_string(error_msg);
},
&_scanner_eof));
_counter->num_rows_filtered++;
_success = false;
return Status::OK();
}
}
_success = true;
return Status::OK();
}
Status VBrokerScanner::get_next(Block* output_block, bool* eof) {
SCOPED_TIMER(_read_timer);
RETURN_IF_ERROR(_init_src_block());
const int batch_size = _state->batch_size();
auto columns = _src_block.mutate_columns();
while (columns[0]->size() < batch_size && !_scanner_eof) {
if (_cur_line_reader == nullptr || _cur_line_reader_eof) {
RETURN_IF_ERROR(_open_next_reader());
// If there isn't any more reader, break this
if (_scanner_eof) {
continue;
}
}
const uint8_t* ptr = nullptr;
size_t size = 0;
RETURN_IF_ERROR(_cur_line_reader->read_line(&ptr, &size, &_cur_line_reader_eof));
if (_skip_lines > 0) {
_skip_lines--;
continue;
}
if (size == 0) {
// Read empty row, just continue
continue;
}
{
COUNTER_UPDATE(_rows_read_counter, 1);
SCOPED_TIMER(_materialize_timer);
RETURN_IF_ERROR(_fill_dest_columns(Slice(ptr, size), columns));
if (_success) {
free_expr_local_allocations();
}
}
}
columns.clear();
return _fill_dest_block(output_block, eof);
}
Status VBrokerScanner::_fill_dest_columns(const Slice& line,
std::vector<MutableColumnPtr>& columns) {
RETURN_IF_ERROR(_line_split_to_values(line));
if (UNLIKELY(!_success)) {
// If not success, which means we met an invalid row, return.
return Status::OK();
}
// This check is meaningless, should be removed
// if (!check_array_format(_split_values)) {
// return Status::OK();
// }
int idx = 0;
for (int i = 0; i < _split_values.size(); ++i) {
int dest_index = idx++;
auto src_slot_desc = _src_slot_descs[i];
const Slice& value = _split_values[i];
_text_converter->write_string_column(src_slot_desc, &columns[dest_index], value.data,
value.size);
}
return Status::OK();
}
} // namespace doris::vectorized

View File

@ -1,94 +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 <map>
#include <memory>
#include <sstream>
#include <string>
#include <vector>
#include "common/status.h"
#include "exec/base_scanner.h"
#include "exec/decompressor.h"
#include "exec/line_reader.h"
#include "exec/text_converter.h"
#include "gen_cpp/PlanNodes_types.h"
#include "gen_cpp/Types_types.h"
#include "io/file_reader.h"
#include "runtime/mem_pool.h"
#include "util/runtime_profile.h"
#include "util/slice.h"
namespace doris::vectorized {
class VBrokerScanner final : public BaseScanner {
public:
VBrokerScanner(RuntimeState* state, RuntimeProfile* profile,
const TBrokerScanRangeParams& params,
const std::vector<TBrokerRangeDesc>& ranges,
const std::vector<TNetworkAddress>& broker_addresses,
const std::vector<TExpr>& pre_filter_texprs, ScannerCounter* counter);
~VBrokerScanner() override;
Status open() override;
Status get_next(Block* block, bool* eof) override;
void close() override;
private:
Status _open_file_reader();
Status _create_decompressor(TFileFormatType::type type);
Status _open_line_reader();
// Read next buffer from reader
Status _open_next_reader();
Status _line_to_src_tuple(const Slice& line);
Status _line_split_to_values(const Slice& line);
// Split one text line to values
void split_line(const Slice& line);
std::unique_ptr<TextConverter> _text_converter;
Status _fill_dest_columns(const Slice& line, std::vector<MutableColumnPtr>& columns);
std::string _value_separator;
std::string _line_delimiter;
TFileFormatType::type _file_format_type;
int _value_separator_length;
int _line_delimiter_length;
// Reader
// _cur_file_reader_s is for stream load pipe reader,
// and _cur_file_reader is for other file reader.
// TODO: refactor this to use only shared_ptr or unique_ptr
std::unique_ptr<FileReader> _cur_file_reader;
std::shared_ptr<FileReader> _cur_file_reader_s;
FileReader* _real_reader;
LineReader* _cur_line_reader;
Decompressor* _cur_decompressor;
bool _cur_line_reader_eof;
// When we fetch range start from 0, header_type="csv_with_names" skip first line
// When we fetch range start from 0, header_type="csv_with_names_and_types" skip first two line
// When we fetch range doesn't start from 0 will always skip the first line
int _skip_lines;
std::vector<Slice> _split_values;
bool _trim_double_quotes = false;
};
} // namespace doris::vectorized

File diff suppressed because it is too large Load Diff

View File

@ -1,279 +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 <rapidjson/document.h>
#include <rapidjson/error/en.h>
#include <rapidjson/stringbuffer.h>
#include <rapidjson/writer.h>
#include <simdjson.h>
#include <map>
#include <memory>
#include <sstream>
#include <string>
#include <unordered_map>
#include <vector>
#include "common/status.h"
#include "exec/base_scanner.h"
#include "exec/line_reader.h"
#include "exprs/json_functions.h"
#include "io/file_reader.h"
#include "runtime/descriptors.h"
#include "util/runtime_profile.h"
namespace doris {
class ExprContext;
class RuntimeState;
struct ScannerCounter;
namespace vectorized {
class VJsonReader;
template <typename JsonReader>
class VJsonScanner : public BaseScanner {
public:
VJsonScanner(RuntimeState* state, RuntimeProfile* profile, const TBrokerScanRangeParams& params,
const std::vector<TBrokerRangeDesc>& ranges,
const std::vector<TNetworkAddress>& broker_addresses,
const std::vector<TExpr>& pre_filter_texprs, ScannerCounter* counter);
~VJsonScanner() override;
// Open this scanner, will initialize information needed
Status open() override;
Status get_next(vectorized::Block* output_block, bool* eof) override;
void close() override;
private:
Status _open_vjson_reader();
Status _open_next_reader();
Status _open_file_reader();
Status _open_line_reader();
Status _open_json_reader();
Status _open_based_reader();
Status _get_range_params(std::string& jsonpath, std::string& json_root, bool& strip_outer_array,
bool& num_as_string, bool& fuzzy_parse);
std::string _jsonpath;
std::string _jsonpath_file;
std::string _line_delimiter;
int _line_delimiter_length;
// Reader
// _cur_file_reader_s is for stream load pipe reader,
// and _cur_file_reader is for other file reader.
// TODO: refactor this to use only shared_ptr or unique_ptr
std::unique_ptr<FileReader> _cur_file_reader;
std::shared_ptr<FileReader> _cur_file_reader_s;
FileReader* _real_reader;
LineReader* _cur_line_reader;
JsonReader* _cur_json_reader;
bool _cur_reader_eof;
bool _read_json_by_line;
// When we fetch range doesn't start from 0,
// we will read to one ahead, and skip the first line
bool _skip_next_line;
std::unique_ptr<JsonReader> _cur_vjson_reader = nullptr;
};
class VJsonReader {
public:
VJsonReader(RuntimeState* state, ScannerCounter* counter, RuntimeProfile* profile,
bool strip_outer_array, bool num_as_string, bool fuzzy_parse, bool* scanner_eof,
FileReader* file_reader = nullptr, LineReader* line_reader = nullptr);
~VJsonReader();
Status init(const std::string& jsonpath, const std::string& json_root);
Status read_json_column(std::vector<MutableColumnPtr>& columns,
const std::vector<SlotDescriptor*>& slot_descs, bool* is_empty_row,
bool* eof);
private:
Status (VJsonReader::*_vhandle_json_callback)(
std::vector<vectorized::MutableColumnPtr>& columns,
const std::vector<SlotDescriptor*>& slot_descs, bool* is_empty_row, bool* eof);
Status _vhandle_simple_json(std::vector<MutableColumnPtr>& columns,
const std::vector<SlotDescriptor*>& slot_descs, bool* is_empty_row,
bool* eof);
Status _vhandle_flat_array_complex_json(std::vector<MutableColumnPtr>& columns,
const std::vector<SlotDescriptor*>& slot_descs,
bool* is_empty_row, bool* eof);
Status _vhandle_nested_complex_json(std::vector<MutableColumnPtr>& columns,
const std::vector<SlotDescriptor*>& slot_descs,
bool* is_empty_row, bool* eof);
Status _write_columns_by_jsonpath(rapidjson::Value& objectValue,
const std::vector<SlotDescriptor*>& slot_descs,
std::vector<MutableColumnPtr>& columns, bool* valid);
Status _set_column_value(rapidjson::Value& objectValue, std::vector<MutableColumnPtr>& columns,
const std::vector<SlotDescriptor*>& slot_descs, bool* valid);
Status _write_data_to_column(rapidjson::Value::ConstValueIterator value,
SlotDescriptor* slot_desc, vectorized::IColumn* column_ptr,
bool* valid);
Status _parse_json(bool* is_empty_row, bool* eof);
Status _append_error_msg(const rapidjson::Value& objectValue, std::string error_msg,
std::string col_name, bool* valid);
void _fill_slot(doris::Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool,
const uint8_t* value, int32_t len);
Status _parse_json_doc(size_t* size, bool* eof);
Status _set_tuple_value(rapidjson::Value& objectValue, doris::Tuple* tuple,
const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool,
bool* valid);
Status _write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc,
doris::Tuple* tuple, MemPool* tuple_pool, bool* valid);
std::string _print_json_value(const rapidjson::Value& value);
void _close();
Status _generate_json_paths(const std::string& jsonpath,
std::vector<std::vector<JsonPath>>* vect);
Status _parse_jsonpath_and_json_root(const std::string& jsonpath, const std::string& json_root);
int _next_line;
int _total_lines;
RuntimeState* _state;
ScannerCounter* _counter;
RuntimeProfile* _profile;
FileReader* _file_reader;
LineReader* _line_reader;
bool _closed;
bool _strip_outer_array;
bool _num_as_string;
bool _fuzzy_parse;
RuntimeProfile::Counter* _bytes_read_counter;
RuntimeProfile::Counter* _read_timer;
RuntimeProfile::Counter* _file_read_timer;
std::vector<std::vector<JsonPath>> _parsed_jsonpaths;
std::vector<JsonPath> _parsed_json_root;
char _value_buffer[4 * 1024 * 1024];
char _parse_buffer[512 * 1024];
using Document = rapidjson::GenericDocument<rapidjson::UTF8<>, rapidjson::MemoryPoolAllocator<>,
rapidjson::MemoryPoolAllocator<>>;
rapidjson::MemoryPoolAllocator<> _value_allocator;
rapidjson::MemoryPoolAllocator<> _parse_allocator;
Document _origin_json_doc; // origin json document object from parsed json string
rapidjson::Value* _json_doc; // _json_doc equals _final_json_doc iff not set `json_root`
std::unordered_map<std::string, int> _name_map;
// point to the _scanner_eof of JsonScanner
bool* _scanner_eof;
};
class VSIMDJsonReader {
public:
VSIMDJsonReader(RuntimeState* state, ScannerCounter* counter, RuntimeProfile* profile,
bool strip_outer_array, bool num_as_string, bool fuzzy_parse, bool* scanner_eof,
FileReader* file_reader = nullptr, LineReader* line_reader = nullptr);
~VSIMDJsonReader();
Status init(const std::string& jsonpath, const std::string& json_root);
Status read_json_column(Block& block, const std::vector<SlotDescriptor*>& slot_descs,
bool* is_empty_row, bool* eof);
private:
Status (VSIMDJsonReader::*_vhandle_json_callback)(
Block& block, const std::vector<SlotDescriptor*>& slot_descs, bool* is_empty_row,
bool* eof);
Status _vhandle_simple_json(Block& block, const std::vector<SlotDescriptor*>& slot_descs,
bool* is_empty_row, bool* eof);
Status _vhandle_flat_array_complex_json(Block& block,
const std::vector<SlotDescriptor*>& slot_descs,
bool* is_empty_row, bool* eof);
Status _vhandle_nested_complex_json(Block& block,
const std::vector<SlotDescriptor*>& slot_descs,
bool* is_empty_row, bool* eof);
Status _write_columns_by_jsonpath(simdjson::ondemand::value value,
const std::vector<SlotDescriptor*>& slot_descs, Block& block,
bool* valid);
Status _set_column_value(simdjson::ondemand::value value, Block& block,
const std::vector<SlotDescriptor*>& slot_descs, bool* valid);
Status _write_data_to_column(simdjson::ondemand::value value, SlotDescriptor* slot_desc,
vectorized::IColumn* column_ptr, bool* valid);
Status _parse_json(bool* is_empty_row, bool* eof);
Status _parse_json_doc(size_t* size, bool* eof);
Status _parse_jsonpath_and_json_root(const std::string& jsonpath, const std::string& json_root);
Status _generate_json_paths(const std::string& jsonpath, std::vector<std::string>* vect);
Status _append_error_msg(std::string error_msg, std::string col_name, bool* valid);
std::unique_ptr<simdjson::ondemand::parser> _json_parser = nullptr;
simdjson::ondemand::document _original_json_doc;
simdjson::ondemand::value _json_value;
// for strip outer array
simdjson::ondemand::array_iterator _array_iter;
int _next_line;
int _total_lines;
doris::RuntimeState* _state;
doris::ScannerCounter* _counter;
RuntimeProfile* _profile;
FileReader* _file_reader;
LineReader* _line_reader;
bool _strip_outer_array;
RuntimeProfile::Counter* _bytes_read_counter;
RuntimeProfile::Counter* _read_timer;
RuntimeProfile::Counter* _file_read_timer;
// simdjson pointer string, eg.
// jsonpath simdjson pointer
// `["$.k1[0]", "$.k2.a"]` -> ["/k1/0", "/k2/a"]
// notice array index not support `*`
// so we are not fully compatible with previous implementation by rapidjson
std::vector<std::string> _parsed_jsonpaths;
std::string _parsed_json_root;
bool* _scanner_eof;
static constexpr size_t _buffer_size = 1024 * 1024 * 8;
static constexpr size_t _padded_size = _buffer_size + simdjson::SIMDJSON_PADDING;
char _simdjson_ondemand_padding_buffer[_padded_size];
};
} // namespace vectorized
} // namespace doris

View File

@ -1,40 +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 "vec/exec/vorc_scanner.h"
#include <exec/arrow/orc_reader.h>
namespace doris::vectorized {
VORCScanner::VORCScanner(RuntimeState* state, RuntimeProfile* profile,
const TBrokerScanRangeParams& params,
const std::vector<TBrokerRangeDesc>& ranges,
const std::vector<TNetworkAddress>& broker_addresses,
const std::vector<TExpr>& pre_filter_texprs, ScannerCounter* counter)
: VArrowScanner(state, profile, params, ranges, broker_addresses, pre_filter_texprs,
counter) {}
ArrowReaderWrap* VORCScanner::_new_arrow_reader(const std::vector<SlotDescriptor*>& file_slot_descs,
FileReader* file_reader,
int32_t num_of_columns_from_file,
int64_t range_start_offset, int64_t range_size) {
return new ORCReaderWrap(_state, file_slot_descs, file_reader, num_of_columns_from_file,
range_start_offset, range_size);
}
} // namespace doris::vectorized

View File

@ -1,54 +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 <arrow/array.h>
#include <vec/exec/varrow_scanner.h>
#include <map>
#include <memory>
#include <sstream>
#include <string>
#include <unordered_map>
#include <vector>
#include "common/status.h"
#include "exec/base_scanner.h"
#include "gen_cpp/Types_types.h"
#include "runtime/mem_pool.h"
#include "util/runtime_profile.h"
namespace doris::vectorized {
// VOrc scanner convert the data read from Orc to doris's columns.
class VORCScanner final : public VArrowScanner {
public:
VORCScanner(RuntimeState* state, RuntimeProfile* profile, const TBrokerScanRangeParams& params,
const std::vector<TBrokerRangeDesc>& ranges,
const std::vector<TNetworkAddress>& broker_addresses,
const std::vector<TExpr>& pre_filter_texprs, ScannerCounter* counter);
~VORCScanner() override = default;
protected:
ArrowReaderWrap* _new_arrow_reader(const std::vector<SlotDescriptor*>& file_slot_descs,
FileReader* file_reader, int32_t num_of_columns_from_file,
int64_t range_start_offset, int64_t range_size) override;
};
} // namespace doris::vectorized

View File

@ -273,12 +273,7 @@ set(VEC_TEST_FILES
vec/core/column_nullable_test.cpp
vec/core/column_vector_test.cpp
vec/exec/vgeneric_iterators_test.cpp
vec/exec/vbroker_scan_node_test.cpp
vec/exec/vbroker_scanner_test.cpp
vec/exec/vjson_scanner_test.cpp
vec/exec/vtablet_sink_test.cpp
vec/exec/vorc_scanner_test.cpp
vec/exec/vparquet_scanner_test.cpp
vec/exprs/vexpr_test.cpp
vec/function/function_array_aggregation_test.cpp
vec/function/function_array_element_test.cpp

View File

@ -1,637 +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 "vec/exec/vbroker_scan_node.h"
#include <gtest/gtest.h>
#include <map>
#include <string>
#include <vector>
#include "common/object_pool.h"
#include "exprs/binary_predicate.h"
#include "exprs/cast_functions.h"
#include "exprs/literal.h"
#include "exprs/slot_ref.h"
#include "gen_cpp/Descriptors_types.h"
#include "gen_cpp/PlanNodes_types.h"
#include "io/local_file_reader.h"
#include "runtime/descriptors.h"
#include "runtime/memory/mem_tracker.h"
#include "runtime/primitive_type.h"
#include "runtime/runtime_state.h"
#include "runtime/user_function_cache.h"
namespace doris {
Expr* create_literal(ObjectPool* pool, PrimitiveType type, const void* data);
namespace vectorized {
class VBrokerScanNodeTest : public testing::Test {
public:
VBrokerScanNodeTest() : _runtime_state(TQueryGlobals()) {
init();
_runtime_state.init_mem_trackers();
}
void init();
static void SetUpTestCase() {
UserFunctionCache::instance()->init(
"./be/test/runtime/test_data/user_function_cache/normal");
CastFunctions::init();
}
protected:
virtual void SetUp() {}
virtual void TearDown() {}
private:
void init_desc_table();
RuntimeState _runtime_state;
ObjectPool _obj_pool;
std::map<std::string, SlotDescriptor*> _slots_map;
TBrokerScanRangeParams _params;
DescriptorTbl* _desc_tbl;
TPlanNode _tnode;
};
void VBrokerScanNodeTest::init_desc_table() {
TDescriptorTable t_desc_table;
// table descriptors
TTableDescriptor t_table_desc;
t_table_desc.id = 0;
t_table_desc.tableType = TTableType::OLAP_TABLE;
t_table_desc.numCols = 0;
t_table_desc.numClusteringCols = 0;
t_desc_table.tableDescriptors.push_back(t_table_desc);
t_desc_table.__isset.tableDescriptors = true;
int next_slot_id = 1;
// TSlotDescriptor
// int offset = 1;
// int i = 0;
// k1
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 0;
slot_desc.byteOffset = 0;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = -1;
slot_desc.colName = "k1";
slot_desc.slotIdx = 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
// k2
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 1;
slot_desc.byteOffset = 4;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = -1;
slot_desc.colName = "k2";
slot_desc.slotIdx = 2;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
// k3
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 1;
slot_desc.byteOffset = 8;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = -1;
slot_desc.colName = "k3";
slot_desc.slotIdx = 3;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
// k4(partitioned column)
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 1;
slot_desc.byteOffset = 12;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = -1;
slot_desc.colName = "k4";
slot_desc.slotIdx = 4;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
t_desc_table.__isset.slotDescriptors = true;
{
// TTupleDescriptor dest
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = 0;
t_tuple_desc.byteSize = 16;
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
// source tuple descriptor
// TSlotDescriptor
// int offset = 1;
// int i = 0;
// k1
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 1;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 0;
slot_desc.byteOffset = 0;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 0;
slot_desc.colName = "k1";
slot_desc.slotIdx = 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
// k2
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 1;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 1;
slot_desc.byteOffset = 16;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 1;
slot_desc.colName = "k2";
slot_desc.slotIdx = 2;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
// k3
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 1;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 2;
slot_desc.byteOffset = 32;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 2;
slot_desc.colName = "k3";
slot_desc.slotIdx = 3;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
// k4(partitioned column)
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 1;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 3;
slot_desc.byteOffset = 48;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 3;
slot_desc.colName = "k4";
slot_desc.slotIdx = 4;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
{
// TTupleDescriptor source
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = 1;
t_tuple_desc.byteSize = 64;
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
DescriptorTbl::create(&_obj_pool, t_desc_table, &_desc_tbl);
_runtime_state.set_desc_tbl(_desc_tbl);
}
void VBrokerScanNodeTest::init() {
_params.column_separator = ',';
_params.line_delimiter = '\n';
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
TTypeDesc varchar_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(5000);
node.__set_scalar_type(scalar_type);
varchar_type.types.push_back(node);
}
for (int i = 0; i < 4; ++i) {
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = int_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttoint";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = int_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttoint(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_int_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = 5 + i;
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(i + 1, expr);
_params.src_slot_ids.push_back(5 + i);
}
// _params.__isset.expr_of_dest_slot = true;
_params.__set_dest_tuple_id(0);
_params.__set_src_tuple_id(1);
init_desc_table();
// Node Id
_tnode.node_id = 0;
_tnode.node_type = TPlanNodeType::BROKER_SCAN_NODE;
_tnode.num_children = 0;
_tnode.limit = -1;
_tnode.row_tuples.push_back(0);
_tnode.nullable_tuples.push_back(false);
_tnode.broker_scan_node.tuple_id = 0;
_tnode.__isset.broker_scan_node = true;
}
TEST_F(VBrokerScanNodeTest, normal) {
VBrokerScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl);
scan_node.init(_tnode);
auto status = scan_node.prepare(&_runtime_state);
ASSERT_TRUE(status.ok());
// set scan range
std::vector<TScanRangeParams> scan_ranges;
{
TScanRangeParams scan_range_params;
TBrokerScanRange broker_scan_range;
broker_scan_range.params = _params;
TBrokerRangeDesc range;
range.path = "./be/test/exec/test_data/broker_scanner/normal.csv";
range.start_offset = 0;
range.size = -1;
range.file_type = TFileType::FILE_LOCAL;
range.format_type = TFileFormatType::FORMAT_CSV_PLAIN;
range.splittable = true;
std::vector<std::string> columns_from_path {"1"};
range.__set_columns_from_path(columns_from_path);
range.__set_num_of_columns_from_file(3);
broker_scan_range.ranges.push_back(range);
scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range);
scan_ranges.push_back(scan_range_params);
}
{
TScanRangeParams scan_range_params;
TBrokerScanRange broker_scan_range;
broker_scan_range.params = _params;
TBrokerRangeDesc range;
range.path = "./be/test/exec/test_data/broker_scanner/normal.csv";
range.start_offset = 1;
range.size = 7;
range.file_type = TFileType::FILE_LOCAL;
range.format_type = TFileFormatType::FORMAT_CSV_PLAIN;
range.splittable = true;
std::vector<std::string> columns_from_path {"2"};
range.__set_columns_from_path(columns_from_path);
range.__set_num_of_columns_from_file(3);
broker_scan_range.ranges.push_back(range);
scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range);
scan_ranges.push_back(scan_range_params);
}
scan_node.set_scan_ranges(scan_ranges);
status = scan_node.open(&_runtime_state);
ASSERT_TRUE(status.ok());
doris::vectorized::Block block;
bool eos = false;
status = scan_node.get_next(&_runtime_state, &block, &eos);
ASSERT_EQ(4, block.rows());
ASSERT_EQ(4, block.columns());
ASSERT_TRUE(eos);
auto columns = block.get_columns_with_type_and_name();
ASSERT_EQ(columns.size(), 4);
ASSERT_EQ(columns[0].to_string(0), "1");
ASSERT_EQ(columns[0].to_string(1), "4");
ASSERT_EQ(columns[0].to_string(2), "8");
ASSERT_EQ(columns[0].to_string(3), "4");
ASSERT_EQ(columns[1].to_string(0), "2");
ASSERT_EQ(columns[1].to_string(1), "5");
ASSERT_EQ(columns[1].to_string(2), "9");
ASSERT_EQ(columns[1].to_string(3), "5");
ASSERT_EQ(columns[2].to_string(0), "3");
ASSERT_EQ(columns[2].to_string(1), "6");
ASSERT_EQ(columns[2].to_string(2), "10");
ASSERT_EQ(columns[2].to_string(3), "6");
ASSERT_EQ(columns[3].to_string(0), "1");
ASSERT_EQ(columns[3].to_string(1), "1");
ASSERT_EQ(columns[3].to_string(2), "1");
ASSERT_EQ(columns[3].to_string(3), "2");
block.clear();
status = scan_node.get_next(&_runtime_state, &block, &eos);
ASSERT_EQ(0, block.rows());
ASSERT_TRUE(eos);
scan_node.close(&_runtime_state);
{
std::stringstream ss;
scan_node.runtime_profile()->pretty_print(&ss);
LOG(INFO) << ss.str();
}
}
TEST_F(VBrokerScanNodeTest, where_binary_pre) {
TPlanNode _tnode_ = _tnode;
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
TExpr expr;
{
TExprNode expr_node;
expr_node.__set_node_type(TExprNodeType::BINARY_PRED);
expr_node.type = gen_type_desc(TPrimitiveType::BOOLEAN);
expr_node.__set_num_children(2);
expr_node.__isset.opcode = true;
expr_node.__set_opcode(TExprOpcode::LT);
expr_node.__isset.vector_opcode = true;
expr_node.__set_vector_opcode(TExprOpcode::LT);
expr_node.__isset.fn = true;
expr_node.fn.name.function_name = "lt";
expr_node.fn.binary_type = TFunctionBinaryType::BUILTIN;
expr_node.fn.ret_type = int_type;
expr_node.fn.has_var_args = false;
expr.nodes.push_back(expr_node);
}
{
TExprNode expr_node;
expr_node.__set_node_type(TExprNodeType::SLOT_REF);
expr_node.type = int_type;
expr_node.__set_num_children(0);
expr_node.__isset.slot_ref = true;
TSlotRef slot_ref;
slot_ref.__set_slot_id(1);
slot_ref.__set_tuple_id(0);
expr_node.__set_slot_ref(slot_ref);
expr_node.__isset.output_column = true;
expr_node.__set_output_column(0);
expr.nodes.push_back(expr_node);
}
{
TExprNode expr_node;
expr_node.__set_node_type(TExprNodeType::INT_LITERAL);
expr_node.type = int_type;
expr_node.__set_num_children(0);
expr_node.__isset.int_literal = true;
TIntLiteral int_literal;
int_literal.__set_value(8);
expr_node.__set_int_literal(int_literal);
expr.nodes.push_back(expr_node);
}
_tnode_.__set_vconjunct(expr);
VBrokerScanNode scan_node(&_obj_pool, _tnode_, *_desc_tbl);
auto status = scan_node.init(_tnode_);
ASSERT_TRUE(status.ok());
status = scan_node.prepare(&_runtime_state);
ASSERT_TRUE(status.ok());
// set scan range
std::vector<TScanRangeParams> scan_ranges;
{
TScanRangeParams scan_range_params;
TBrokerScanRange broker_scan_range;
broker_scan_range.params = _params;
TBrokerRangeDesc range;
range.path = "./be/test/exec/test_data/broker_scanner/normal.csv";
range.start_offset = 0;
range.size = -1;
range.file_type = TFileType::FILE_LOCAL;
range.format_type = TFileFormatType::FORMAT_CSV_PLAIN;
range.splittable = true;
std::vector<std::string> columns_from_path {"1"};
range.__set_columns_from_path(columns_from_path);
range.__set_num_of_columns_from_file(3);
broker_scan_range.ranges.push_back(range);
scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range);
scan_ranges.push_back(scan_range_params);
}
scan_node.set_scan_ranges(scan_ranges);
status = scan_node.open(&_runtime_state);
ASSERT_TRUE(status.ok());
doris::vectorized::Block block;
bool eos = false;
status = scan_node.get_next(&_runtime_state, &block, &eos);
ASSERT_EQ(2, block.rows());
ASSERT_EQ(4, block.columns());
auto columns = block.get_columns_with_type_and_name();
ASSERT_EQ(columns.size(), 4);
ASSERT_EQ(columns[0].to_string(0), "1");
ASSERT_EQ(columns[0].to_string(1), "4");
ASSERT_EQ(columns[1].to_string(0), "2");
ASSERT_EQ(columns[1].to_string(1), "5");
ASSERT_EQ(columns[2].to_string(0), "3");
ASSERT_EQ(columns[2].to_string(1), "6");
ASSERT_EQ(columns[3].to_string(0), "1");
ASSERT_EQ(columns[3].to_string(1), "1");
ASSERT_TRUE(eos);
block.clear();
status = scan_node.get_next(&_runtime_state, &block, &eos);
ASSERT_EQ(0, block.rows());
ASSERT_TRUE(eos);
scan_node.close(&_runtime_state);
{
std::stringstream ss;
scan_node.runtime_profile()->pretty_print(&ss);
LOG(INFO) << ss.str();
}
}
} // namespace vectorized
} // namespace doris

View File

@ -1,554 +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 "vec/exec/vbroker_scanner.h"
#include <gtest/gtest.h>
#include <map>
#include <string>
#include <vector>
#include "common/object_pool.h"
#include "exprs/cast_functions.h"
#include "gen_cpp/Descriptors_types.h"
#include "gen_cpp/PlanNodes_types.h"
#include "io/local_file_reader.h"
#include "runtime/descriptors.h"
#include "runtime/memory/mem_tracker.h"
#include "runtime/runtime_state.h"
#include "runtime/user_function_cache.h"
namespace doris {
namespace vectorized {
class VBrokerScannerTest : public testing::Test {
public:
VBrokerScannerTest() : _runtime_state(TQueryGlobals()) {
init();
_profile = _runtime_state.runtime_profile();
_runtime_state.init_mem_trackers();
TUniqueId unique_id;
TQueryOptions query_options;
TQueryGlobals query_globals;
_runtime_state.init(unique_id, query_options, query_globals, nullptr);
}
void init();
static void SetUpTestCase() {
UserFunctionCache::instance()->init(
"./be/test/runtime/test_data/user_function_cache/normal");
CastFunctions::init();
}
protected:
virtual void SetUp() {}
virtual void TearDown() {}
private:
void init_desc_table();
void init_params();
TupleId _dst_tuple_id = 0;
TupleId _src_tuple_id = 1;
RuntimeState _runtime_state;
RuntimeProfile* _profile;
ObjectPool _obj_pool;
TBrokerScanRangeParams _params;
DescriptorTbl* _desc_tbl;
std::vector<TNetworkAddress> _addresses;
ScannerCounter _counter;
std::vector<TExpr> _pre_filter;
};
void VBrokerScannerTest::init_desc_table() {
TDescriptorTable t_desc_table;
// table descriptors
TTableDescriptor t_table_desc;
t_table_desc.id = 0;
t_table_desc.tableType = TTableType::OLAP_TABLE;
t_table_desc.numCols = 0;
t_table_desc.numClusteringCols = 0;
t_desc_table.tableDescriptors.push_back(t_table_desc);
t_desc_table.__isset.tableDescriptors = true;
int next_slot_id = 1;
// TSlotDescriptor
// int offset = 1;
// int i = 0;
// k1
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 0;
slot_desc.byteOffset = 0;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = -1;
slot_desc.colName = "k1";
slot_desc.slotIdx = 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
// k2
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 1;
slot_desc.byteOffset = 4;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = -1;
slot_desc.colName = "k2";
slot_desc.slotIdx = 2;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
// k3
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 2;
slot_desc.byteOffset = 8;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = -1;
slot_desc.colName = "k3";
slot_desc.slotIdx = 3;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
t_desc_table.__isset.slotDescriptors = true;
{
// TTupleDescriptor dest
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = 0;
t_tuple_desc.byteSize = 12;
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
// source tuple descriptor
// TSlotDescriptor
// int offset = 1;
// int i = 0;
// k1
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 1;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 0;
slot_desc.byteOffset = 0;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 0;
slot_desc.colName = "k1";
slot_desc.slotIdx = 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
// k2
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 1;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 1;
slot_desc.byteOffset = 16;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 1;
slot_desc.colName = "k2";
slot_desc.slotIdx = 2;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
// k3
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 1;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 2;
slot_desc.byteOffset = 32;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 2;
slot_desc.colName = "k3";
slot_desc.slotIdx = 3;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
{
// TTupleDescriptor source
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = 1;
t_tuple_desc.byteSize = 48;
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
DescriptorTbl::create(&_obj_pool, t_desc_table, &_desc_tbl);
_runtime_state.set_desc_tbl(_desc_tbl);
}
void VBrokerScannerTest::init_params() {
_params.column_separator = ',';
_params.line_delimiter = '\n';
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
TTypeDesc varchar_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(5000);
node.__set_scalar_type(scalar_type);
varchar_type.types.push_back(node);
}
for (int i = 0; i < 3; ++i) {
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = int_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttoint";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = int_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttoint(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_int_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = 4 + i;
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(i + 1, expr);
_params.src_slot_ids.push_back(4 + i);
}
// _params.__isset.expr_of_dest_slot = true;
_params.__set_dest_tuple_id(_dst_tuple_id);
_params.__set_src_tuple_id(_src_tuple_id);
}
void VBrokerScannerTest::init() {
init_desc_table();
init_params();
}
TEST_F(VBrokerScannerTest, normal) {
std::vector<TBrokerRangeDesc> ranges;
TBrokerRangeDesc range;
range.path = "./be/test/exec/test_data/broker_scanner/normal.csv";
range.start_offset = 0;
range.size = -1;
range.splittable = true;
range.file_type = TFileType::FILE_LOCAL;
range.format_type = TFileFormatType::FORMAT_CSV_PLAIN;
ranges.push_back(range);
VBrokerScanner scanner(&_runtime_state, _profile, _params, ranges, _addresses, _pre_filter,
&_counter);
auto st = scanner.open();
ASSERT_TRUE(st.ok());
std::unique_ptr<vectorized::Block> block(new vectorized::Block());
bool eof = false;
st = scanner.get_next(block.get(), &eof);
ASSERT_TRUE(st.ok());
ASSERT_TRUE(eof);
auto columns = block->get_columns();
ASSERT_EQ(columns.size(), 3);
ASSERT_EQ(columns[0]->get_int(0), 1);
ASSERT_EQ(columns[0]->get_int(1), 4);
ASSERT_EQ(columns[0]->get_int(2), 8);
ASSERT_EQ(columns[1]->get_int(0), 2);
ASSERT_EQ(columns[1]->get_int(1), 5);
ASSERT_EQ(columns[1]->get_int(2), 9);
ASSERT_EQ(columns[2]->get_int(0), 3);
ASSERT_EQ(columns[2]->get_int(1), 6);
ASSERT_EQ(columns[2]->get_int(2), 10);
}
TEST_F(VBrokerScannerTest, normal_with_pre_filter) {
std::vector<TBrokerRangeDesc> ranges;
TBrokerRangeDesc range;
range.path = "./be/test/exec/test_data/broker_scanner/normal.csv";
range.start_offset = 0;
range.size = -1;
range.splittable = true;
range.file_type = TFileType::FILE_LOCAL;
range.format_type = TFileFormatType::FORMAT_CSV_PLAIN;
ranges.push_back(range);
// init pre_filter expr: k1 < '8'
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
TTypeDesc varchar_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(5000);
node.__set_scalar_type(scalar_type);
varchar_type.types.push_back(node);
}
TExpr filter_expr;
{
TExprNode expr_node;
expr_node.__set_node_type(TExprNodeType::BINARY_PRED);
expr_node.type = gen_type_desc(TPrimitiveType::BOOLEAN);
expr_node.__set_num_children(2);
expr_node.__isset.opcode = true;
expr_node.__set_opcode(TExprOpcode::LT);
expr_node.__isset.vector_opcode = true;
expr_node.__set_vector_opcode(TExprOpcode::LT);
expr_node.__isset.fn = true;
expr_node.fn.name.function_name = "lt";
expr_node.fn.binary_type = TFunctionBinaryType::BUILTIN;
expr_node.fn.ret_type = int_type;
expr_node.fn.has_var_args = false;
filter_expr.nodes.push_back(expr_node);
}
{
TExprNode expr_node;
expr_node.__set_node_type(TExprNodeType::SLOT_REF);
expr_node.type = varchar_type;
expr_node.__set_num_children(0);
expr_node.__isset.slot_ref = true;
TSlotRef slot_ref;
slot_ref.__set_slot_id(4);
slot_ref.__set_tuple_id(1);
expr_node.__set_slot_ref(slot_ref);
expr_node.__isset.output_column = true;
expr_node.__set_output_column(0);
filter_expr.nodes.push_back(expr_node);
}
{
TExprNode expr_node;
expr_node.__set_node_type(TExprNodeType::STRING_LITERAL);
expr_node.type = varchar_type;
expr_node.__set_num_children(0);
expr_node.__isset.string_literal = true;
TStringLiteral string_literal;
string_literal.__set_value("8");
expr_node.__set_string_literal(string_literal);
filter_expr.nodes.push_back(expr_node);
}
_pre_filter.push_back(filter_expr);
VBrokerScanner scanner(&_runtime_state, _profile, _params, ranges, _addresses, _pre_filter,
&_counter);
auto st = scanner.open();
ASSERT_TRUE(st.ok());
std::unique_ptr<vectorized::Block> block(new vectorized::Block());
bool eof = false;
// end of file
st = scanner.get_next(block.get(), &eof);
ASSERT_TRUE(st.ok());
ASSERT_TRUE(eof);
auto columns = block->get_columns();
ASSERT_EQ(columns.size(), 3);
ASSERT_EQ(columns[0]->get_int(0), 1);
ASSERT_EQ(columns[0]->get_int(1), 4);
ASSERT_EQ(columns[1]->get_int(0), 2);
ASSERT_EQ(columns[1]->get_int(1), 5);
ASSERT_EQ(columns[2]->get_int(0), 3);
ASSERT_EQ(columns[2]->get_int(1), 6);
}
TEST_F(VBrokerScannerTest, normal2) {
std::vector<TBrokerRangeDesc> ranges;
TBrokerRangeDesc range;
range.path = "./be/test/exec/test_data/broker_scanner/normal2_1.csv";
range.start_offset = 0;
range.size = 7;
range.splittable = true;
range.file_type = TFileType::FILE_LOCAL;
range.format_type = TFileFormatType::FORMAT_CSV_PLAIN;
ranges.push_back(range);
range.path = "./be/test/exec/test_data/broker_scanner/normal2_2.csv";
range.start_offset = 0;
range.size = 4;
ranges.push_back(range);
VBrokerScanner scanner(&_runtime_state, _profile, _params, ranges, _addresses, _pre_filter,
&_counter);
auto st = scanner.open();
ASSERT_TRUE(st.ok());
std::unique_ptr<vectorized::Block> block(new vectorized::Block());
bool eof = false;
st = scanner.get_next(block.get(), &eof);
ASSERT_TRUE(st.ok());
ASSERT_TRUE(eof);
auto columns = block->get_columns();
ASSERT_EQ(columns.size(), 3);
ASSERT_EQ(columns[0]->get_int(0), 1);
ASSERT_EQ(columns[0]->get_int(1), 3);
ASSERT_EQ(columns[1]->get_int(0), 2);
ASSERT_EQ(columns[1]->get_int(1), 4);
ASSERT_EQ(columns[2]->get_int(0), 3);
ASSERT_EQ(columns[2]->get_int(1), 5);
}
TEST_F(VBrokerScannerTest, normal5) {
std::vector<TBrokerRangeDesc> ranges;
TBrokerRangeDesc range;
range.path = "./be/test/exec/test_data/broker_scanner/normal.csv";
range.start_offset = 0;
range.size = 0;
range.splittable = true;
range.file_type = TFileType::FILE_LOCAL;
range.format_type = TFileFormatType::FORMAT_CSV_PLAIN;
ranges.push_back(range);
VBrokerScanner scanner(&_runtime_state, _profile, _params, ranges, _addresses, _pre_filter,
&_counter);
auto st = scanner.open();
ASSERT_TRUE(st.ok());
std::unique_ptr<vectorized::Block> block(new vectorized::Block());
bool eof = false;
// end of file
st = scanner.get_next(block.get(), &eof);
ASSERT_TRUE(st.ok());
ASSERT_TRUE(eof);
auto columns = block->get_columns();
ASSERT_EQ(columns.size(), 0);
}
} // namespace vectorized
} // namespace doris

View File

@ -1,861 +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 "vec/exec/vjson_scanner.h"
#include <gtest/gtest.h>
#include <time.h>
#include <map>
#include <string>
#include <vector>
#include "common/object_pool.h"
#include "exprs/cast_functions.h"
#include "exprs/decimalv2_operators.h"
#include "gen_cpp/Descriptors_types.h"
#include "gen_cpp/PlanNodes_types.h"
#include "io/local_file_reader.h"
#include "runtime/descriptors.h"
#include "runtime/exec_env.h"
#include "runtime/runtime_state.h"
#include "runtime/tuple.h"
#include "runtime/user_function_cache.h"
#include "util/defer_op.h"
#include "vec/exec/vbroker_scan_node.h"
namespace doris {
namespace vectorized {
class VJsonScannerTest : public testing::Test {
public:
VJsonScannerTest() : _runtime_state(TQueryGlobals()) {
init();
_runtime_state.init_mem_trackers();
TUniqueId unique_id;
TQueryOptions query_options;
TQueryGlobals query_globals;
_runtime_state.init(unique_id, query_options, query_globals, nullptr);
}
void init();
static void SetUpTestCase() {
config::enable_simdjson_reader = true;
UserFunctionCache::instance()->init(
"./be/test/runtime/test_data/user_function_cache/normal");
CastFunctions::init();
DecimalV2Operators::init();
}
protected:
virtual void SetUp() {}
virtual void TearDown() {}
private:
int create_src_tuple(TDescriptorTable& t_desc_table, int next_slot_id);
int create_dst_tuple(TDescriptorTable& t_desc_table, int next_slot_id);
void create_expr_info();
void init_desc_table();
RuntimeState _runtime_state;
ObjectPool _obj_pool;
std::map<std::string, SlotDescriptor*> _slots_map;
TBrokerScanRangeParams _params;
DescriptorTbl* _desc_tbl;
TPlanNode _tnode;
};
#define TUPLE_ID_DST 0
#define TUPLE_ID_SRC 1
#define COLUMN_NUMBERS 6
#define DST_TUPLE_SLOT_ID_START 1
#define SRC_TUPLE_SLOT_ID_START 7
int VJsonScannerTest::create_src_tuple(TDescriptorTable& t_desc_table, int next_slot_id) {
const char* columnNames[] = {"category", "author", "title", "price", "largeint", "decimal"};
for (int i = 0; i < COLUMN_NUMBERS; i++) {
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 1;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = i;
slot_desc.byteOffset = i * 16 + 8;
slot_desc.nullIndicatorByte = i / 8;
slot_desc.nullIndicatorBit = i % 8;
slot_desc.colName = columnNames[i];
slot_desc.slotIdx = i + 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
{
// TTupleDescriptor source
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = TUPLE_ID_SRC;
t_tuple_desc.byteSize = COLUMN_NUMBERS * 16 + 8;
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
return next_slot_id;
}
int VJsonScannerTest::create_dst_tuple(TDescriptorTable& t_desc_table, int next_slot_id) {
int32_t byteOffset = 8;
{ //category
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 0;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 0;
slot_desc.colName = "category";
slot_desc.slotIdx = 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
{ // author
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 1;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 1;
slot_desc.colName = "author";
slot_desc.slotIdx = 2;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
{ // title
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 2;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 2;
slot_desc.colName = "title";
slot_desc.slotIdx = 3;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
{ // price
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::DOUBLE);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 3;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 3;
slot_desc.colName = "price";
slot_desc.slotIdx = 4;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 8;
{ // lagreint
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::LARGEINT);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 4;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 4;
slot_desc.colName = "lagreint";
slot_desc.slotIdx = 5;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
{ // decimal
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__isset.precision = true;
scalar_type.__isset.scale = true;
scalar_type.__set_precision(-1);
scalar_type.__set_scale(-1);
scalar_type.__set_type(TPrimitiveType::DECIMALV2);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 5;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 5;
slot_desc.colName = "decimal";
slot_desc.slotIdx = 6;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
t_desc_table.__isset.slotDescriptors = true;
{
// TTupleDescriptor dest
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = TUPLE_ID_DST;
t_tuple_desc.byteSize = byteOffset + 8;
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
return next_slot_id;
}
void VJsonScannerTest::init_desc_table() {
TDescriptorTable t_desc_table;
// table descriptors
TTableDescriptor t_table_desc;
t_table_desc.id = 0;
t_table_desc.tableType = TTableType::BROKER_TABLE;
t_table_desc.numCols = 0;
t_table_desc.numClusteringCols = 0;
t_desc_table.tableDescriptors.push_back(t_table_desc);
t_desc_table.__isset.tableDescriptors = true;
int next_slot_id = 1;
next_slot_id = create_dst_tuple(t_desc_table, next_slot_id);
next_slot_id = create_src_tuple(t_desc_table, next_slot_id);
DescriptorTbl::create(&_obj_pool, t_desc_table, &_desc_tbl);
_runtime_state.set_desc_tbl(_desc_tbl);
}
void VJsonScannerTest::create_expr_info() {
TTypeDesc varchar_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(5000);
node.__set_scalar_type(scalar_type);
varchar_type.types.push_back(node);
}
// category VARCHAR --> VARCHAR
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START; // category id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START);
}
// author VARCHAR --> VARCHAR
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START + 1; // author id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START + 1, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START + 1);
}
// title VARCHAR --> VARCHAR
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START + 2; // log_time id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START + 2, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START + 2);
}
// price VARCHAR --> DOUBLE
{
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::DOUBLE);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = int_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttodouble";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = int_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttodouble(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_double_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START + 3; // price id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START + 3, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START + 3);
}
// largeint VARCHAR --> LargeInt
{
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::LARGEINT);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = int_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttolargeint";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = int_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttolargeint(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_large_int_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START + 4; // price id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START + 4, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START + 4);
}
// decimal VARCHAR --> Decimal
{
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__isset.precision = true;
scalar_type.__isset.scale = true;
scalar_type.__set_precision(-1);
scalar_type.__set_scale(-1);
scalar_type.__set_type(TPrimitiveType::DECIMALV2);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = int_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttodecimalv2";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = int_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttodecimalv2(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::DecimalV2Operators::cast_to_decimalv2_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START + 5; // price id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START + 5, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START + 5);
}
// _params.__isset.expr_of_dest_slot = true;
_params.__set_dest_tuple_id(TUPLE_ID_DST);
_params.__set_src_tuple_id(TUPLE_ID_SRC);
}
void VJsonScannerTest::init() {
create_expr_info();
init_desc_table();
// Node Id
_tnode.node_id = 0;
_tnode.node_type = TPlanNodeType::SCHEMA_SCAN_NODE;
_tnode.num_children = 0;
_tnode.limit = -1;
_tnode.row_tuples.push_back(0);
_tnode.nullable_tuples.push_back(false);
_tnode.broker_scan_node.tuple_id = 0;
_tnode.__isset.broker_scan_node = true;
}
TEST_F(VJsonScannerTest, simple_array_json) {
auto test_fn = [&](bool using_simdjson) {
bool saved_flag = config::enable_simdjson_reader;
if (using_simdjson) {
config::enable_simdjson_reader = true;
}
Defer __defer([&] { config::enable_simdjson_reader = saved_flag; });
VBrokerScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl);
scan_node.init(_tnode);
auto status = scan_node.prepare(&_runtime_state);
EXPECT_TRUE(status.ok());
// set scan range
std::vector<TScanRangeParams> scan_ranges;
{
TScanRangeParams scan_range_params;
TBrokerScanRange broker_scan_range;
broker_scan_range.params = _params;
TBrokerRangeDesc range;
range.start_offset = 0;
range.size = -1;
range.format_type = TFileFormatType::FORMAT_JSON;
range.strip_outer_array = true;
range.__isset.strip_outer_array = true;
range.__set_num_as_string(true);
range.splittable = true;
range.path = "./be/test/exec/test_data/json_scanner/test_simple2.json";
range.file_type = TFileType::FILE_LOCAL;
broker_scan_range.ranges.push_back(range);
scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range);
scan_ranges.push_back(scan_range_params);
}
scan_node.set_scan_ranges(scan_ranges);
status = scan_node.open(&_runtime_state);
EXPECT_TRUE(status.ok());
bool eof = false;
vectorized::Block block;
status = scan_node.get_next(&_runtime_state, &block, &eof);
EXPECT_TRUE(status.ok());
EXPECT_EQ(2, block.rows());
EXPECT_EQ(6, block.columns());
auto columns = block.get_columns_with_type_and_name();
ASSERT_EQ(columns.size(), 6);
ASSERT_EQ(columns[0].to_string(0), "reference");
ASSERT_EQ(columns[0].to_string(1), "fiction");
ASSERT_EQ(columns[1].to_string(0), "NigelRees");
ASSERT_EQ(columns[1].to_string(1), "EvelynWaugh");
ASSERT_EQ(columns[2].to_string(0), "SayingsoftheCentury");
ASSERT_EQ(columns[2].to_string(1), "SwordofHonour");
ASSERT_EQ(columns[3].to_string(0), "8.95");
ASSERT_EQ(columns[3].to_string(1), "12.99");
ASSERT_EQ(columns[4].to_string(0), "1234");
ASSERT_EQ(columns[4].to_string(1), "1180591620717411303424");
ASSERT_EQ(columns[5].to_string(0), "1234.123400000");
ASSERT_EQ(columns[5].to_string(1), "9999999999999.999999000");
block.clear();
status = scan_node.get_next(&_runtime_state, &block, &eof);
ASSERT_EQ(0, block.rows());
ASSERT_TRUE(eof);
scan_node.close(&_runtime_state);
};
test_fn(true);
test_fn(false);
}
TEST_F(VJsonScannerTest, use_jsonpaths_with_file_reader) {
auto test_fn = [&](bool using_simdjson) {
bool saved_flag = config::enable_simdjson_reader;
if (using_simdjson) {
config::enable_simdjson_reader = true;
}
Defer __defer([&] { config::enable_simdjson_reader = saved_flag; });
VBrokerScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl);
scan_node.init(_tnode);
auto status = scan_node.prepare(&_runtime_state);
EXPECT_TRUE(status.ok());
// set scan range
std::vector<TScanRangeParams> scan_ranges;
{
TScanRangeParams scan_range_params;
TBrokerScanRange broker_scan_range;
broker_scan_range.params = _params;
TBrokerRangeDesc range;
range.start_offset = 0;
range.size = -1;
range.format_type = TFileFormatType::FORMAT_JSON;
range.strip_outer_array = true;
range.__isset.strip_outer_array = true;
range.splittable = true;
range.path = "./be/test/exec/test_data/json_scanner/test_simple2.json";
range.file_type = TFileType::FILE_LOCAL;
range.jsonpaths =
"[\"$.category\", \"$.author\", \"$.title\", \"$.price\", \"$.largeint\", "
"\"$.decimal\"]";
range.__isset.jsonpaths = true;
broker_scan_range.ranges.push_back(range);
scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range);
scan_ranges.push_back(scan_range_params);
}
scan_node.set_scan_ranges(scan_ranges);
status = scan_node.open(&_runtime_state);
EXPECT_TRUE(status.ok());
bool eof = false;
vectorized::Block block;
status = scan_node.get_next(&_runtime_state, &block, &eof);
EXPECT_TRUE(status.ok());
EXPECT_EQ(2, block.rows());
EXPECT_EQ(6, block.columns());
auto columns = block.get_columns_with_type_and_name();
ASSERT_EQ(columns.size(), 6);
ASSERT_EQ(columns[0].to_string(0), "reference");
ASSERT_EQ(columns[0].to_string(1), "fiction");
ASSERT_EQ(columns[1].to_string(0), "NigelRees");
ASSERT_EQ(columns[1].to_string(1), "EvelynWaugh");
ASSERT_EQ(columns[2].to_string(0), "SayingsoftheCentury");
ASSERT_EQ(columns[2].to_string(1), "SwordofHonour");
block.clear();
status = scan_node.get_next(&_runtime_state, &block, &eof);
ASSERT_EQ(0, block.rows());
ASSERT_TRUE(eof);
scan_node.close(&_runtime_state);
};
test_fn(true);
test_fn(false);
}
TEST_F(VJsonScannerTest, use_jsonpaths_with_line_reader) {
auto test_fn = [&](bool using_simdjson) {
bool saved_flag = config::enable_simdjson_reader;
if (using_simdjson) {
config::enable_simdjson_reader = true;
}
Defer __defer([&] { config::enable_simdjson_reader = saved_flag; });
VBrokerScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl);
scan_node.init(_tnode);
auto status = scan_node.prepare(&_runtime_state);
EXPECT_TRUE(status.ok());
std::vector<TScanRangeParams> scan_ranges;
{
TScanRangeParams scan_range_params;
TBrokerScanRange broker_scan_range;
broker_scan_range.params = _params;
TBrokerRangeDesc range;
range.start_offset = 0;
range.size = -1;
range.format_type = TFileFormatType::FORMAT_JSON;
range.splittable = true;
range.strip_outer_array = true;
range.__isset.strip_outer_array = true;
range.path = "./be/test/exec/test_data/json_scanner/test_simple2.json";
range.file_type = TFileType::FILE_LOCAL;
range.jsonpaths =
"[\"$.category\", \"$.author\", \"$.title\", \"$.price\", \"$.largeint\", "
"\"$.decimal\"]";
range.__isset.jsonpaths = true;
range.read_json_by_line = true;
range.__isset.read_json_by_line = true;
broker_scan_range.ranges.push_back(range);
scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range);
scan_ranges.push_back(scan_range_params);
}
scan_node.set_scan_ranges(scan_ranges);
status = scan_node.open(&_runtime_state);
EXPECT_TRUE(status.ok());
bool eof = false;
vectorized::Block block;
status = scan_node.get_next(&_runtime_state, &block, &eof);
EXPECT_TRUE(status.ok());
EXPECT_EQ(2, block.rows());
EXPECT_EQ(6, block.columns());
auto columns = block.get_columns_with_type_and_name();
ASSERT_EQ(columns.size(), 6);
ASSERT_EQ(columns[0].to_string(0), "reference");
ASSERT_EQ(columns[0].to_string(1), "fiction");
ASSERT_EQ(columns[1].to_string(0), "NigelRees");
ASSERT_EQ(columns[1].to_string(1), "EvelynWaugh");
ASSERT_EQ(columns[2].to_string(0), "SayingsoftheCentury");
ASSERT_EQ(columns[2].to_string(1), "SwordofHonour");
block.clear();
status = scan_node.get_next(&_runtime_state, &block, &eof);
ASSERT_EQ(0, block.rows());
ASSERT_TRUE(eof);
scan_node.close(&_runtime_state);
};
test_fn(true);
test_fn(false);
}
TEST_F(VJsonScannerTest, use_jsonpaths_mismatch) {
auto test_fn = [&](bool using_simdjson) {
bool saved_flag = config::enable_simdjson_reader;
if (using_simdjson) {
config::enable_simdjson_reader = true;
}
Defer __defer([&] { config::enable_simdjson_reader = saved_flag; });
VBrokerScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl);
scan_node.init(_tnode);
auto status = scan_node.prepare(&_runtime_state);
EXPECT_TRUE(status.ok());
// set scan range
std::vector<TScanRangeParams> scan_ranges;
{
TScanRangeParams scan_range_params;
TBrokerScanRange broker_scan_range;
broker_scan_range.params = _params;
TBrokerRangeDesc range;
range.start_offset = 0;
range.size = -1;
range.format_type = TFileFormatType::FORMAT_JSON;
range.strip_outer_array = true;
range.__isset.strip_outer_array = true;
range.splittable = true;
range.path = "./be/test/exec/test_data/json_scanner/test_simple2.json";
range.file_type = TFileType::FILE_LOCAL;
range.jsonpaths = "[\"$.k1\", \"$.k2\", \"$.k3\", \"$.k4\", \"$.k5\", \"$.k6\"]";
range.__isset.jsonpaths = true;
broker_scan_range.ranges.push_back(range);
scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range);
scan_ranges.push_back(scan_range_params);
}
scan_node.set_scan_ranges(scan_ranges);
status = scan_node.open(&_runtime_state);
EXPECT_TRUE(status.ok());
bool eof = false;
vectorized::Block block;
status = scan_node.get_next(&_runtime_state, &block, &eof);
EXPECT_TRUE(status.ok());
EXPECT_EQ(0, block.rows());
EXPECT_EQ(0, block.columns());
block.clear();
scan_node.close(&_runtime_state);
};
test_fn(true);
test_fn(false);
}
TEST_F(VJsonScannerTest, use_nested_with_jsonpath) {
auto test_fn = [&](bool using_simdjson) {
bool saved_flag = config::enable_simdjson_reader;
if (using_simdjson) {
config::enable_simdjson_reader = true;
}
Defer __defer([&] { config::enable_simdjson_reader = saved_flag; });
VBrokerScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl);
scan_node.init(_tnode);
auto status = scan_node.prepare(&_runtime_state);
EXPECT_TRUE(status.ok());
// set scan range
std::vector<TScanRangeParams> scan_ranges;
{
TScanRangeParams scan_range_params;
TBrokerScanRange broker_scan_range;
broker_scan_range.params = _params;
TBrokerRangeDesc range;
range.start_offset = 0;
range.size = -1;
range.format_type = TFileFormatType::FORMAT_JSON;
range.strip_outer_array = true;
range.__isset.strip_outer_array = true;
range.splittable = true;
range.path = "./be/test/exec/test_data/json_scanner/test_nested.json";
range.file_type = TFileType::FILE_LOCAL;
range.jsonpaths = "[\"$.qid\", \"$.tag\", \"$.creationDate\", \"$.answers[0].user\"]";
range.__isset.jsonpaths = true;
broker_scan_range.ranges.push_back(range);
scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range);
scan_ranges.push_back(scan_range_params);
}
scan_node.set_scan_ranges(scan_ranges);
status = scan_node.open(&_runtime_state);
EXPECT_TRUE(status.ok());
bool eof = false;
vectorized::Block block;
status = scan_node.get_next(&_runtime_state, &block, &eof);
EXPECT_TRUE(status.ok());
EXPECT_EQ(2048, block.rows());
EXPECT_EQ(6, block.columns());
auto columns = block.get_columns_with_type_and_name();
ASSERT_EQ(columns.size(), 6);
EXPECT_EQ(columns[0].to_string(0), "1000000");
EXPECT_EQ(columns[0].to_string(1), "10000005");
EXPECT_EQ(columns[1].to_string(0), "[\"vb6\", \"progress-bar\"]");
EXPECT_EQ(columns[1].to_string(1), "[\"php\", \"arrays\", \"sorting\"]");
EXPECT_EQ(columns[2].to_string(0), "2009-06-16T07:28:42.770");
EXPECT_EQ(columns[2].to_string(1), "2012-04-03T19:25:46.213");
block.clear();
scan_node.close(&_runtime_state);
};
test_fn(true);
test_fn(false);
}
} // namespace vectorized
} // namespace doris

View File

@ -1,884 +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 "vec/exec/vorc_scanner.h"
#include <gtest/gtest.h>
#include <runtime/descriptor_helper.h>
#include <time.h>
#include <map>
#include <string>
#include <vector>
#include "common/object_pool.h"
#include "exprs/cast_functions.h"
#include "exprs/decimalv2_operators.h"
#include "gen_cpp/Descriptors_types.h"
#include "gen_cpp/PlanNodes_types.h"
#include "io/local_file_reader.h"
#include "runtime/descriptors.h"
#include "runtime/runtime_state.h"
#include "runtime/tuple.h"
#include "runtime/user_function_cache.h"
#include "vec/exec/vbroker_scan_node.h"
namespace doris {
namespace vectorized {
class VOrcScannerTest : public testing::Test {
public:
VOrcScannerTest() : _runtime_state(TQueryGlobals()) {
_profile = _runtime_state.runtime_profile();
_runtime_state.init_mem_trackers();
}
~VOrcScannerTest() {}
static void SetUpTestCase() {
UserFunctionCache::instance()->init(
"./be/test/runtime/test_data/user_function_cache/normal");
CastFunctions::init();
DecimalV2Operators::init();
}
protected:
virtual void SetUp() {}
virtual void TearDown() {}
private:
RuntimeState _runtime_state;
RuntimeProfile* _profile;
ObjectPool _obj_pool;
DescriptorTbl* _desc_tbl;
std::vector<TNetworkAddress> _addresses;
ScannerCounter _counter;
std::vector<TExpr> _pre_filter;
};
TEST_F(VOrcScannerTest, normal) {
TBrokerScanRangeParams params;
TTypeDesc varchar_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
varchar_type.types.push_back(node);
}
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
TTypeDesc big_int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::BIGINT);
node.__set_scalar_type(scalar_type);
big_int_type.types.push_back(node);
}
TTypeDesc float_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::FLOAT);
node.__set_scalar_type(scalar_type);
float_type.types.push_back(node);
}
TTypeDesc double_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::DOUBLE);
node.__set_scalar_type(scalar_type);
double_type.types.push_back(node);
}
TTypeDesc date_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::DATE);
node.__set_scalar_type(scalar_type);
date_type.types.push_back(node);
}
//col1 varchar -> bigint
{
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = big_int_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttobigint";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = big_int_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttoint(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_big_int_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = 0;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(8, expr);
params.src_slot_ids.push_back(0);
}
//col2, col3
for (int i = 1; i <= 2; i++) {
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = i;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(8 + i, expr);
params.src_slot_ids.push_back(i);
}
//col5 varchar -> double
{
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = double_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttodouble";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = double_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttoint(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_double_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = 3;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(11, expr);
params.src_slot_ids.push_back(3);
}
//col6 varchar -> float
{
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = float_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttofloat";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = float_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttoint(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_float_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = 4;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(12, expr);
params.src_slot_ids.push_back(4);
}
//col7,col8
for (int i = 5; i <= 6; i++) {
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = int_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttoint";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = int_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttoint(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_int_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = i;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(8 + i, expr);
params.src_slot_ids.push_back(i);
}
//col9 varchar -> var
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = 7;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(15, expr);
params.src_slot_ids.push_back(7);
}
params.__set_src_tuple_id(0);
params.__set_dest_tuple_id(1);
//init_desc_table
TDescriptorTable t_desc_table;
// table descriptors
TTableDescriptor t_table_desc;
t_table_desc.id = 0;
t_table_desc.tableType = TTableType::BROKER_TABLE;
t_table_desc.numCols = 0;
t_table_desc.numClusteringCols = 0;
t_desc_table.tableDescriptors.push_back(t_table_desc);
t_desc_table.__isset.tableDescriptors = true;
TDescriptorTableBuilder dtb;
TTupleDescriptorBuilder src_tuple_builder;
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col1")
.column_pos(1)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col2")
.column_pos(2)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col3")
.column_pos(3)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col5")
.column_pos(4)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col6")
.column_pos(5)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col7")
.column_pos(6)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col8")
.column_pos(7)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col9")
.column_pos(8)
.build());
src_tuple_builder.build(&dtb);
TTupleDescriptorBuilder dest_tuple_builder;
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().type(TYPE_BIGINT).column_name("col1").column_pos(1).build());
dest_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col2")
.column_pos(2)
.build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().string_type(65535).column_name("col3").column_pos(3).build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().type(TYPE_DOUBLE).column_name("col5").column_pos(4).build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().type(TYPE_FLOAT).column_name("col6").column_pos(5).build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().type(TYPE_INT).column_name("col7").column_pos(6).build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().type(TYPE_INT).column_name("col8").column_pos(7).build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().string_type(65535).column_name("col9").column_pos(8).build());
dest_tuple_builder.build(&dtb);
t_desc_table = dtb.desc_tbl();
DescriptorTbl::create(&_obj_pool, t_desc_table, &_desc_tbl);
_runtime_state.set_desc_tbl(_desc_tbl);
std::vector<TBrokerRangeDesc> ranges;
TBrokerRangeDesc rangeDesc;
rangeDesc.start_offset = 0;
rangeDesc.size = -1;
rangeDesc.format_type = TFileFormatType::FORMAT_ORC;
rangeDesc.splittable = false;
rangeDesc.path = "./be/test/exec/test_data/orc_scanner/my-file.orc";
rangeDesc.file_type = TFileType::FILE_LOCAL;
ranges.push_back(rangeDesc);
VORCScanner scanner(&_runtime_state, _profile, params, ranges, _addresses, _pre_filter,
&_counter);
EXPECT_TRUE(scanner.open().ok());
vectorized::Block block;
bool eof = false;
EXPECT_TRUE(scanner.get_next(&block, &eof).ok());
EXPECT_TRUE(eof);
EXPECT_TRUE(scanner.get_next(&block, &eof).ok());
EXPECT_TRUE(eof);
scanner.close();
}
TEST_F(VOrcScannerTest, normal2) {
TBrokerScanRangeParams params;
TTypeDesc varchar_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
varchar_type.types.push_back(node);
}
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::INT);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = 1;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(3, expr);
params.src_slot_ids.push_back(0);
params.src_slot_ids.push_back(1);
params.src_slot_ids.push_back(2);
}
params.__set_src_tuple_id(0);
params.__set_dest_tuple_id(1);
//init_desc_table
TDescriptorTable t_desc_table;
// table descriptors
TTableDescriptor t_table_desc;
t_table_desc.id = 0;
t_table_desc.tableType = TTableType::BROKER_TABLE;
t_table_desc.numCols = 0;
t_table_desc.numClusteringCols = 0;
t_desc_table.tableDescriptors.push_back(t_table_desc);
t_desc_table.__isset.tableDescriptors = true;
TDescriptorTableBuilder dtb;
TTupleDescriptorBuilder src_tuple_builder;
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col1")
.column_pos(1)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col2")
.column_pos(2)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col3")
.column_pos(3)
.build());
src_tuple_builder.build(&dtb);
TTupleDescriptorBuilder dest_tuple_builder;
dest_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.column_name("value_from_col2")
.column_pos(1)
.build());
dest_tuple_builder.build(&dtb);
t_desc_table = dtb.desc_tbl();
DescriptorTbl::create(&_obj_pool, t_desc_table, &_desc_tbl);
_runtime_state.set_desc_tbl(_desc_tbl);
std::vector<TBrokerRangeDesc> ranges;
TBrokerRangeDesc rangeDesc;
rangeDesc.start_offset = 0;
rangeDesc.size = -1;
rangeDesc.format_type = TFileFormatType::FORMAT_ORC;
rangeDesc.splittable = false;
rangeDesc.path = "./be/test/exec/test_data/orc_scanner/my-file.orc";
rangeDesc.file_type = TFileType::FILE_LOCAL;
ranges.push_back(rangeDesc);
VORCScanner scanner(&_runtime_state, _profile, params, ranges, _addresses, _pre_filter,
&_counter);
EXPECT_TRUE(scanner.open().ok());
bool eof = false;
vectorized::Block block;
EXPECT_TRUE(scanner.get_next(&block, &eof).ok());
EXPECT_EQ(10, block.rows());
EXPECT_TRUE(eof);
scanner.close();
}
TEST_F(VOrcScannerTest, normal3) {
TBrokerScanRangeParams params;
TTypeDesc varchar_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
varchar_type.types.push_back(node);
}
TTypeDesc decimal_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::DECIMALV2);
scalar_type.__set_precision(64);
scalar_type.__set_scale(64);
node.__set_scalar_type(scalar_type);
decimal_type.types.push_back(node);
}
TTypeDesc tinyint_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::TINYINT);
node.__set_scalar_type(scalar_type);
tinyint_type.types.push_back(node);
}
TTypeDesc datetime_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::DATETIME);
node.__set_scalar_type(scalar_type);
datetime_type.types.push_back(node);
}
TTypeDesc date_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::DATE);
node.__set_scalar_type(scalar_type);
date_type.types.push_back(node);
}
{
for (int i = 0; i < 5; ++i) {
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = decimal_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttodecimalv2";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = decimal_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("cast_to_decimalv2_val(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::DecimalV2Operators::cast_to_decimalv2_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = i;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(9 + i, expr);
params.src_slot_ids.push_back(i);
}
{
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = tinyint_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttotinyint";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = tinyint_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("cast_to_tiny_int_val(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_tiny_int_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = 5;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(14, expr);
params.src_slot_ids.push_back(5);
}
{
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = datetime_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttodatetime";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = datetime_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("cast_to_datetime_val(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_datetime_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = 6;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(15, expr);
params.src_slot_ids.push_back(6);
}
{
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = date_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttodate";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = date_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttoint(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_date_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = 7;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(16, expr);
params.src_slot_ids.push_back(7);
}
{
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = decimal_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttodecimalv2";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = decimal_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("cast_to_decimalv2_val(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::DecimalV2Operators::cast_to_decimalv2_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = 8;
slot_ref.slot_ref.tuple_id = 0;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
params.expr_of_dest_slot.emplace(17, expr);
params.src_slot_ids.push_back(8);
}
}
params.__set_src_tuple_id(0);
params.__set_dest_tuple_id(1);
//init_desc_table
TDescriptorTable t_desc_table;
// table descriptors
TTableDescriptor t_table_desc;
t_table_desc.id = 0;
t_table_desc.tableType = TTableType::BROKER_TABLE;
t_table_desc.numCols = 0;
t_table_desc.numClusteringCols = 0;
t_desc_table.tableDescriptors.push_back(t_table_desc);
t_desc_table.__isset.tableDescriptors = true;
TDescriptorTableBuilder dtb;
TTupleDescriptorBuilder src_tuple_builder;
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col1")
.column_pos(1)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col2")
.column_pos(2)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col3")
.column_pos(3)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col4")
.column_pos(4)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col5")
.column_pos(5)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col6")
.column_pos(6)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col7")
.column_pos(7)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col8")
.column_pos(8)
.build());
src_tuple_builder.add_slot(TSlotDescriptorBuilder()
.string_type(65535)
.nullable(true)
.column_name("col9")
.column_pos(9)
.build());
src_tuple_builder.build(&dtb);
TTupleDescriptorBuilder dest_tuple_builder;
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().decimal_type(10, 9).column_name("col1").column_pos(1).build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().decimal_type(7, 5).column_name("col2").column_pos(2).build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().decimal_type(10, 9).column_name("col3").column_pos(3).build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().decimal_type(10, 5).column_name("col4").column_pos(4).build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().decimal_type(10, 5).column_name("col5").column_pos(5).build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().type(TYPE_TINYINT).column_name("col6").column_pos(6).build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().type(TYPE_DATETIME).column_name("col7").column_pos(7).build());
dest_tuple_builder.add_slot(TSlotDescriptorBuilder()
.type(TYPE_DATE)
.nullable(true)
.column_name("col8")
.column_pos(8)
.build());
dest_tuple_builder.add_slot(
TSlotDescriptorBuilder().decimal_type(27, 9).column_name("col9").column_pos(9).build());
dest_tuple_builder.build(&dtb);
t_desc_table = dtb.desc_tbl();
DescriptorTbl::create(&_obj_pool, t_desc_table, &_desc_tbl);
_runtime_state.set_desc_tbl(_desc_tbl);
std::vector<TBrokerRangeDesc> ranges;
TBrokerRangeDesc rangeDesc;
rangeDesc.start_offset = 0;
rangeDesc.size = -1;
rangeDesc.format_type = TFileFormatType::FORMAT_ORC;
rangeDesc.splittable = false;
rangeDesc.path = "./be/test/exec/test_data/orc_scanner/decimal_and_timestamp.orc";
rangeDesc.file_type = TFileType::FILE_LOCAL;
ranges.push_back(rangeDesc);
VORCScanner scanner(&_runtime_state, _profile, params, ranges, _addresses, _pre_filter,
&_counter);
EXPECT_TRUE(scanner.open().ok());
bool eof = false;
vectorized::Block block;
EXPECT_TRUE(scanner.get_next(&block, &eof).ok());
EXPECT_EQ(1, block.rows());
EXPECT_TRUE(eof);
scanner.close();
}
} // namespace vectorized
} // namespace doris

View File

@ -1,497 +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 <gtest/gtest.h>
#include <time.h>
#include <map>
#include <string>
#include <vector>
#include "common/object_pool.h"
#include "exprs/cast_functions.h"
#include "gen_cpp/Descriptors_types.h"
#include "gen_cpp/PlanNodes_types.h"
#include "io/local_file_reader.h"
#include "runtime/descriptors.h"
#include "runtime/runtime_state.h"
#include "runtime/tuple.h"
#include "runtime/user_function_cache.h"
#include "vec/exec/vbroker_scan_node.h"
namespace doris {
namespace vectorized {
class VParquetScannerTest : public testing::Test {
public:
VParquetScannerTest() : _runtime_state(TQueryGlobals()) {
init();
_runtime_state.init_mem_trackers();
}
~VParquetScannerTest() {}
void init();
static void SetUpTestCase() {
UserFunctionCache::instance()->init(
"./be/test/runtime/test_data/user_function_cache/normal");
CastFunctions::init();
}
protected:
virtual void SetUp() {}
virtual void TearDown() {}
private:
int create_src_tuple(TDescriptorTable& t_desc_table, int next_slot_id);
int create_dst_tuple(TDescriptorTable& t_desc_table, int next_slot_id);
void create_expr_info();
void init_desc_table();
RuntimeState _runtime_state;
ObjectPool _obj_pool;
std::map<std::string, SlotDescriptor*> _slots_map;
TBrokerScanRangeParams _params;
DescriptorTbl* _desc_tbl;
TPlanNode _tnode;
};
#define TUPLE_ID_DST 0
#define TUPLE_ID_SRC 1
#define COLUMN_NUMBERS 20
#define DST_TUPLE_SLOT_ID_START 1
#define SRC_TUPLE_SLOT_ID_START 21
int VParquetScannerTest::create_src_tuple(TDescriptorTable& t_desc_table, int next_slot_id) {
const char* columnNames[] = {
"log_version", "log_time", "log_time_stamp", "js_version",
"vst_cookie", "vst_ip", "vst_user_id", "vst_user_agent",
"device_resolution", "page_url", "page_refer_url", "page_yyid",
"page_type", "pos_type", "content_id", "media_id",
"spm_cnt", "spm_pre", "scm_cnt", "partition_column"};
for (int i = 0; i < COLUMN_NUMBERS; i++) {
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 1;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = i;
// Skip the first 8 bytes These 8 bytes are used to indicate whether the field is a null value
slot_desc.byteOffset = i * 16 + 8;
slot_desc.nullIndicatorByte = i / 8;
slot_desc.nullIndicatorBit = i % 8;
slot_desc.colName = columnNames[i];
slot_desc.slotIdx = i + 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
{
// TTupleDescriptor source
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = TUPLE_ID_SRC;
//Here 8 bytes in order to handle null values
t_tuple_desc.byteSize = COLUMN_NUMBERS * 16 + 8;
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
return next_slot_id;
}
int VParquetScannerTest::create_dst_tuple(TDescriptorTable& t_desc_table, int next_slot_id) {
int32_t byteOffset =
8; // Skip the first 8 bytes These 8 bytes are used to indicate whether the field is a null value
{ //log_version
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR); //parquet::Type::BYTE
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 0;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 0;
slot_desc.colName = "log_version";
slot_desc.slotIdx = 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
{ // log_time
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::BIGINT); //parquet::Type::INT64
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 1;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 1;
slot_desc.colName = "log_time";
slot_desc.slotIdx = 2;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 8;
{ // log_time_stamp
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::BIGINT); //parquet::Type::INT32
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 2;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 2;
slot_desc.colName = "log_time_stamp";
slot_desc.slotIdx = 3;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 8;
const char* columnNames[] = {
"log_version", "log_time", "log_time_stamp", "js_version",
"vst_cookie", "vst_ip", "vst_user_id", "vst_user_agent",
"device_resolution", "page_url", "page_refer_url", "page_yyid",
"page_type", "pos_type", "content_id", "media_id",
"spm_cnt", "spm_pre", "scm_cnt", "partition_column"};
for (int i = 3; i < COLUMN_NUMBERS; i++, byteOffset += 16) {
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR); //parquet::Type::BYTE
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = i;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = i / 8;
slot_desc.nullIndicatorBit = i % 8;
slot_desc.colName = columnNames[i];
slot_desc.slotIdx = i + 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
t_desc_table.__isset.slotDescriptors = true;
{
// TTupleDescriptor dest
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = TUPLE_ID_DST;
t_tuple_desc.byteSize = byteOffset + 8; //Here 8 bytes in order to handle null values
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
return next_slot_id;
}
void VParquetScannerTest::init_desc_table() {
TDescriptorTable t_desc_table;
// table descriptors
TTableDescriptor t_table_desc;
t_table_desc.id = 0;
t_table_desc.tableType = TTableType::BROKER_TABLE;
t_table_desc.numCols = 0;
t_table_desc.numClusteringCols = 0;
t_desc_table.tableDescriptors.push_back(t_table_desc);
t_desc_table.__isset.tableDescriptors = true;
int next_slot_id = 1;
next_slot_id = create_dst_tuple(t_desc_table, next_slot_id);
next_slot_id = create_src_tuple(t_desc_table, next_slot_id);
DescriptorTbl::create(&_obj_pool, t_desc_table, &_desc_tbl);
_runtime_state.set_desc_tbl(_desc_tbl);
}
void VParquetScannerTest::create_expr_info() {
TTypeDesc varchar_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(5000);
node.__set_scalar_type(scalar_type);
varchar_type.types.push_back(node);
}
// log_version VARCHAR --> VARCHAR
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START; // log_time id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START);
}
// log_time VARCHAR --> BIGINT
{
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::BIGINT);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = int_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttoint";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = int_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttoint(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_big_int_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START + 1; // log_time id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START + 1, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START + 1);
}
// log_time_stamp VARCHAR --> BIGINT
{
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::BIGINT);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = int_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttoint";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = int_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttoint(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_big_int_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START + 2;
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START + 2, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START + 2);
}
// couldn't convert type
for (int i = 3; i < COLUMN_NUMBERS; i++) {
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START + i; // log_time id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START + i, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START + i);
}
// _params.__isset.expr_of_dest_slot = true;
_params.__set_dest_tuple_id(TUPLE_ID_DST);
_params.__set_src_tuple_id(TUPLE_ID_SRC);
}
void VParquetScannerTest::init() {
create_expr_info();
init_desc_table();
// Node Id
_tnode.node_id = 0;
_tnode.node_type = TPlanNodeType::SCHEMA_SCAN_NODE;
_tnode.num_children = 0;
_tnode.limit = -1;
_tnode.row_tuples.push_back(0);
_tnode.nullable_tuples.push_back(false);
_tnode.broker_scan_node.tuple_id = 0;
_tnode.__isset.broker_scan_node = true;
}
TEST_F(VParquetScannerTest, normal) {
VBrokerScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl);
scan_node.init(_tnode);
auto status = scan_node.prepare(&_runtime_state);
EXPECT_TRUE(status.ok());
// set scan range
std::vector<TScanRangeParams> scan_ranges;
{
TScanRangeParams scan_range_params;
TBrokerScanRange broker_scan_range;
broker_scan_range.params = _params;
TBrokerRangeDesc range;
range.start_offset = 0;
range.size = -1;
range.format_type = TFileFormatType::FORMAT_PARQUET;
range.splittable = true;
std::vector<std::string> columns_from_path {"value"};
range.__set_columns_from_path(columns_from_path);
range.__set_num_of_columns_from_file(19);
#if 1
range.path = "./be/test/exec/test_data/parquet_scanner/localfile.parquet";
range.file_type = TFileType::FILE_LOCAL;
#else
range.path = "hdfs://ip:8020/user/xxxx.parq";
range.file_type = TFileType::FILE_BROKER;
TNetworkAddress addr;
addr.__set_hostname("127.0.0.1");
addr.__set_port(8000);
broker_scan_range.broker_addresses.push_back(addr);
#endif
broker_scan_range.ranges.push_back(range);
scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range);
scan_ranges.push_back(scan_range_params);
}
scan_node.set_scan_ranges(scan_ranges);
status = scan_node.open(&_runtime_state);
EXPECT_TRUE(status.ok());
// Get block
vectorized::Block block;
bool eof = false;
for (int i = 0; i < 14; i++) {
status = scan_node.get_next(&_runtime_state, &block, &eof);
EXPECT_TRUE(status.ok());
EXPECT_EQ(2048, block.rows());
EXPECT_FALSE(eof);
block.clear();
}
status = scan_node.get_next(&_runtime_state, &block, &eof);
EXPECT_TRUE(status.ok());
EXPECT_EQ(1328, block.rows());
EXPECT_TRUE(eof);
block.clear();
status = scan_node.get_next(&_runtime_state, &block, &eof);
EXPECT_TRUE(status.ok());
EXPECT_EQ(0, block.rows());
EXPECT_TRUE(eof);
scan_node.close(&_runtime_state);
{
std::stringstream ss;
scan_node.runtime_profile()->pretty_print(&ss);
LOG(INFO) << ss.str();
}
}
} // namespace vectorized
} // namespace doris

View File

@ -34,7 +34,6 @@ import org.apache.doris.common.UserException;
import org.apache.doris.common.util.DebugUtil;
import org.apache.doris.load.BrokerFileGroup;
import org.apache.doris.mysql.privilege.PrivPredicate;
import org.apache.doris.planner.BrokerScanNode;
import org.apache.doris.planner.DataPartition;
import org.apache.doris.planner.OlapTableSink;
import org.apache.doris.planner.PlanFragment;
@ -147,17 +146,9 @@ public class LoadingTaskPlanner {
// Generate plan trees
// 1. Broker scan node
ScanNode scanNode;
boolean useNewScanNode = Config.enable_new_load_scan_node || useNewLoadScanNode;
if (useNewScanNode) {
scanNode = new ExternalFileScanNode(new PlanNodeId(nextNodeId++), scanTupleDesc);
((ExternalFileScanNode) scanNode).setLoadInfo(loadJobId, txnId, table, brokerDesc, fileGroups,
fileStatusesList, filesAdded, strictMode, loadParallelism, userInfo);
} else {
scanNode = new BrokerScanNode(new PlanNodeId(nextNodeId++), scanTupleDesc, "BrokerScanNode",
fileStatusesList, filesAdded);
((BrokerScanNode) scanNode).setLoadInfo(loadJobId, txnId, table, brokerDesc, fileGroups, strictMode,
loadParallelism, userInfo);
}
scanNode = new ExternalFileScanNode(new PlanNodeId(nextNodeId++), scanTupleDesc);
((ExternalFileScanNode) scanNode).setLoadInfo(loadJobId, txnId, table, brokerDesc, fileGroups,
fileStatusesList, filesAdded, strictMode, loadParallelism, userInfo);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
if (Config.enable_vectorized_load) {

View File

@ -1,654 +0,0 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.planner;
import org.apache.doris.analysis.Analyzer;
import org.apache.doris.analysis.BrokerDesc;
import org.apache.doris.analysis.Expr;
import org.apache.doris.analysis.ImportColumnDesc;
import org.apache.doris.analysis.IntLiteral;
import org.apache.doris.analysis.SlotDescriptor;
import org.apache.doris.analysis.SlotRef;
import org.apache.doris.analysis.StorageBackend;
import org.apache.doris.analysis.TupleDescriptor;
import org.apache.doris.analysis.UserIdentity;
import org.apache.doris.catalog.BrokerTable;
import org.apache.doris.catalog.Column;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.FsBroker;
import org.apache.doris.catalog.HdfsResource;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.catalog.Table;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.Config;
import org.apache.doris.common.DdlException;
import org.apache.doris.common.FeConstants;
import org.apache.doris.common.UserException;
import org.apache.doris.common.util.BrokerUtil;
import org.apache.doris.common.util.Util;
import org.apache.doris.common.util.VectorizedUtil;
import org.apache.doris.load.BrokerFileGroup;
import org.apache.doris.load.Load;
import org.apache.doris.load.loadv2.LoadTask;
import org.apache.doris.mysql.privilege.UserProperty;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.resource.Tag;
import org.apache.doris.statistics.StatisticalType;
import org.apache.doris.system.Backend;
import org.apache.doris.system.BeSelectionPolicy;
import org.apache.doris.task.LoadTaskInfo;
import org.apache.doris.thrift.TBrokerFileStatus;
import org.apache.doris.thrift.TBrokerRangeDesc;
import org.apache.doris.thrift.TBrokerScanRange;
import org.apache.doris.thrift.TBrokerScanRangeParams;
import org.apache.doris.thrift.TExplainLevel;
import org.apache.doris.thrift.TFileFormatType;
import org.apache.doris.thrift.TFileType;
import org.apache.doris.thrift.THdfsParams;
import org.apache.doris.thrift.TNetworkAddress;
import org.apache.doris.thrift.TScanRange;
import org.apache.doris.thrift.TScanRangeLocation;
import org.apache.doris.thrift.TScanRangeLocations;
import com.google.common.base.Joiner;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
/**
* Broker scan node
*
* Since https://github.com/apache/doris/pull/5686, Doris can read data from HDFS without broker by
* broker scan node.
* Broker scan node is more likely a file scan node for now.
* With this feature, we can extend BrokerScanNode to query external table which data is stored in HDFS, such as
* Hive and Iceberg, etc.
*/
public class BrokerScanNode extends LoadScanNode {
private static final Logger LOG = LogManager.getLogger(BrokerScanNode.class);
private static final TBrokerFileStatusComparator T_BROKER_FILE_STATUS_COMPARATOR
= new TBrokerFileStatusComparator();
public static class TBrokerFileStatusComparator implements Comparator<TBrokerFileStatus> {
@Override
public int compare(TBrokerFileStatus o1, TBrokerFileStatus o2) {
if (o1.size < o2.size) {
return -1;
} else if (o1.size > o2.size) {
return 1;
}
return 0;
}
}
// File groups need to
private List<TScanRangeLocations> locationsList;
// used both for load statement and select statement
private long totalBytes;
private long bytesPerInstance;
// Parameters need to process
private long loadJobId = -1; // -1 means this scan node is not for a load job
private long txnId = -1;
protected Table targetTable;
protected BrokerDesc brokerDesc;
protected List<BrokerFileGroup> fileGroups;
private boolean strictMode = false;
private int loadParallelism = 1;
private UserIdentity userIdentity;
protected List<List<TBrokerFileStatus>> fileStatusesList;
// file num
protected int filesAdded;
// Only used for external table in select statement
private List<Backend> backends;
private int nextBe = 0;
private Analyzer analyzer;
protected static class ParamCreateContext {
public BrokerFileGroup fileGroup;
public TBrokerScanRangeParams params;
public TupleDescriptor srcTupleDescriptor;
public Map<String, Expr> exprMap;
public Map<String, SlotDescriptor> slotDescByName;
public String timezone;
}
private List<ParamCreateContext> paramCreateContexts;
// For broker load and external broker table
public BrokerScanNode(PlanNodeId id, TupleDescriptor destTupleDesc, String planNodeName,
List<List<TBrokerFileStatus>> fileStatusesList, int filesAdded) {
super(id, destTupleDesc, planNodeName, StatisticalType.BROKER_SCAN_NODE);
this.fileStatusesList = fileStatusesList;
this.filesAdded = filesAdded;
if (ConnectContext.get() != null) {
this.userIdentity = ConnectContext.get().getCurrentUserIdentity();
}
}
// For hive and iceberg scan node
public BrokerScanNode(PlanNodeId id, TupleDescriptor destTupleDesc, String planNodeName,
List<List<TBrokerFileStatus>> fileStatusesList, int filesAdded, StatisticalType statisticalType) {
super(id, destTupleDesc, planNodeName, statisticalType);
this.fileStatusesList = fileStatusesList;
this.filesAdded = filesAdded;
if (ConnectContext.get() != null) {
this.userIdentity = ConnectContext.get().getCurrentUserIdentity();
}
}
@Override
public void init(Analyzer analyzer) throws UserException {
super.init(analyzer);
this.analyzer = analyzer;
if (desc.getTable() != null) {
this.initFileGroup();
}
// Get all broker file status
assignBackends();
getFileStatusAndCalcInstance();
paramCreateContexts = Lists.newArrayList();
for (BrokerFileGroup fileGroup : fileGroups) {
ParamCreateContext context = new ParamCreateContext();
context.fileGroup = fileGroup;
context.timezone = analyzer.getTimezone();
initParams(context);
paramCreateContexts.add(context);
}
}
public List<ParamCreateContext> getParamCreateContexts() {
return paramCreateContexts;
}
protected void initFileGroup() throws UserException {
BrokerTable brokerTable = (BrokerTable) desc.getTable();
try {
fileGroups = Lists.newArrayList(new BrokerFileGroup(brokerTable));
} catch (AnalysisException e) {
throw new UserException(e.getMessage());
}
brokerDesc = new BrokerDesc(brokerTable.getBrokerName(), brokerTable.getBrokerProperties());
targetTable = brokerTable;
}
protected boolean isLoad() {
return desc.getTable() == null;
}
public void setLoadInfo(long loadJobId,
long txnId,
Table targetTable,
BrokerDesc brokerDesc,
List<BrokerFileGroup> fileGroups,
boolean strictMode,
int loadParallelism,
UserIdentity userIdentity) {
this.loadJobId = loadJobId;
this.txnId = txnId;
this.targetTable = targetTable;
this.brokerDesc = brokerDesc;
this.fileGroups = fileGroups;
this.strictMode = strictMode;
this.loadParallelism = loadParallelism;
this.userIdentity = userIdentity;
}
// Called from init, construct source tuple information
private void initParams(ParamCreateContext context)
throws UserException {
TBrokerScanRangeParams params = new TBrokerScanRangeParams();
context.params = params;
BrokerFileGroup fileGroup = context.fileGroup;
params.setColumnSeparator(fileGroup.getColumnSeparator().getBytes(Charset.forName("UTF-8"))[0]);
params.setLineDelimiter(fileGroup.getLineDelimiter().getBytes(Charset.forName("UTF-8"))[0]);
params.setColumnSeparatorStr(fileGroup.getColumnSeparator());
params.setLineDelimiterStr(fileGroup.getLineDelimiter());
params.setColumnSeparatorLength(fileGroup.getColumnSeparator().getBytes(Charset.forName("UTF-8")).length);
params.setLineDelimiterLength(fileGroup.getLineDelimiter().getBytes(Charset.forName("UTF-8")).length);
params.setStrictMode(strictMode);
params.setProperties(brokerDesc.getProperties());
if (params.getSrcSlotIds() == null) {
params.setSrcSlotIds(new java.util.ArrayList<java.lang.Integer>());
}
deleteCondition = fileGroup.getDeleteCondition();
mergeType = fileGroup.getMergeType();
initColumns(context);
initAndSetPrecedingFilter(fileGroup.getPrecedingFilterExpr(), context.srcTupleDescriptor, analyzer);
initAndSetWhereExpr(fileGroup.getWhereExpr(), this.desc, analyzer);
}
/**
* This method is used to calculate the slotDescByName and exprMap.
* The expr in exprMap is analyzed in this function.
* The smap of slot which belongs to expr will be analyzed by src desc.
* slotDescByName: the single slot from columns in load stmt
* exprMap: the expr from column mapping in load stmt.
*
* @param context
* @throws UserException
*/
private void initColumns(ParamCreateContext context) throws UserException {
context.srcTupleDescriptor = analyzer.getDescTbl().createTupleDescriptor();
context.slotDescByName = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
context.exprMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
// for load job, column exprs is got from file group
// for query, there is no column exprs, they will be got from table's schema in "Load.initColumns"
LoadTaskInfo.ImportColumnDescs columnDescs = new LoadTaskInfo.ImportColumnDescs();
if (isLoad()) {
columnDescs.descs = context.fileGroup.getColumnExprList();
if (mergeType == LoadTask.MergeType.MERGE) {
columnDescs.descs.add(ImportColumnDesc.newDeleteSignImportColumnDesc(deleteCondition));
} else if (mergeType == LoadTask.MergeType.DELETE) {
columnDescs.descs.add(ImportColumnDesc.newDeleteSignImportColumnDesc(new IntLiteral(1)));
}
// add columnExpr for sequence column
if (targetTable instanceof OlapTable && ((OlapTable) targetTable).hasSequenceCol()) {
String sequenceCol = ((OlapTable) targetTable).getSequenceMapCol();
if (sequenceCol == null) {
sequenceCol = context.fileGroup.getSequenceCol();
}
columnDescs.descs.add(new ImportColumnDesc(Column.SEQUENCE_COL,
new SlotRef(null, sequenceCol)));
}
}
if (targetTable != null) {
Load.initColumns(targetTable, columnDescs, context.fileGroup.getColumnToHadoopFunction(), context.exprMap,
analyzer, context.srcTupleDescriptor, context.slotDescByName, context.params.getSrcSlotIds(),
formatType(context.fileGroup.getFileFormat(), ""), null, VectorizedUtil.isVectorized());
}
}
protected TScanRangeLocations newLocations(TBrokerScanRangeParams params, BrokerDesc brokerDesc)
throws UserException {
Backend selectedBackend;
if (brokerDesc.isMultiLoadBroker()) {
if (!brokerDesc.getProperties().containsKey(BrokerDesc.MULTI_LOAD_BROKER_BACKEND_KEY)) {
throw new DdlException("backend not found for multi load.");
}
String backendId = brokerDesc.getProperties().get(BrokerDesc.MULTI_LOAD_BROKER_BACKEND_KEY);
selectedBackend = Env.getCurrentSystemInfo().getBackend(Long.valueOf(backendId));
if (selectedBackend == null) {
throw new DdlException("backend " + backendId + " not found for multi load.");
}
} else {
selectedBackend = backends.get(nextBe++);
nextBe = nextBe % backends.size();
}
// Generate on broker scan range
TBrokerScanRange brokerScanRange = new TBrokerScanRange();
brokerScanRange.setParams(params);
if (brokerDesc.getStorageType() == StorageBackend.StorageType.BROKER) {
FsBroker broker = null;
try {
broker = Env.getCurrentEnv().getBrokerMgr()
.getBroker(brokerDesc.getName(), selectedBackend.getHost());
} catch (AnalysisException e) {
throw new UserException(e.getMessage());
}
brokerScanRange.addToBrokerAddresses(new TNetworkAddress(broker.ip, broker.port));
} else if (brokerDesc.getStorageType() == StorageBackend.StorageType.OFS
|| brokerDesc.getStorageType() == StorageBackend.StorageType.JFS) {
FsBroker broker = Env.getCurrentEnv().getBrokerMgr().getAnyAliveBroker();
if (broker == null) {
throw new UserException("No alive broker.");
}
brokerScanRange.addToBrokerAddresses(new TNetworkAddress(broker.ip, broker.port));
} else {
brokerScanRange.setBrokerAddresses(new ArrayList<>());
}
// Scan range
TScanRange scanRange = new TScanRange();
scanRange.setBrokerScanRange(brokerScanRange);
// Locations
TScanRangeLocations locations = new TScanRangeLocations();
locations.setScanRange(scanRange);
TScanRangeLocation location = new TScanRangeLocation();
location.setBackendId(selectedBackend.getId());
location.setServer(new TNetworkAddress(selectedBackend.getHost(), selectedBackend.getBePort()));
locations.addToLocations(location);
return locations;
}
private void getFileStatusAndCalcInstance() throws UserException {
if (fileStatusesList == null || filesAdded == -1) {
// FIXME(cmy): fileStatusesList and filesAdded can be set out of db lock when doing pull load,
// but for now it is very difficult to set them out of db lock when doing broker query.
// So we leave this code block here.
// This will be fixed later.
fileStatusesList = Lists.newArrayList();
filesAdded = 0;
this.getFileStatus();
}
if (isLoad() && filesAdded == 0) {
throw new UserException("No source file in this table(" + targetTable.getName() + ").");
}
totalBytes = 0;
for (List<TBrokerFileStatus> fileStatuses : fileStatusesList) {
if (!brokerDesc.isMultiLoadBroker()) {
Collections.sort(fileStatuses, T_BROKER_FILE_STATUS_COMPARATOR);
}
for (TBrokerFileStatus fileStatus : fileStatuses) {
totalBytes += fileStatus.size;
}
}
numInstances = 1;
if (!brokerDesc.isMultiLoadBroker()) {
numInstances = (int) (totalBytes / Config.min_bytes_per_broker_scanner);
int totalLoadParallelism = loadParallelism * backends.size();
numInstances = Math.min(totalLoadParallelism, numInstances);
numInstances = Math.min(numInstances, Config.max_broker_concurrency);
numInstances = Math.max(1, numInstances);
}
bytesPerInstance = totalBytes / numInstances + 1;
if (bytesPerInstance > Config.max_bytes_per_broker_scanner) {
throw new UserException(
"Scan bytes per broker scanner exceed limit: " + Config.max_bytes_per_broker_scanner);
}
LOG.info("number instance of broker scan node is: {}, bytes per instance: {}", numInstances, bytesPerInstance);
}
protected void getFileStatus() throws UserException {
for (BrokerFileGroup fileGroup : fileGroups) {
boolean isBinaryFileFormat = fileGroup.isBinaryFileFormat();
List<TBrokerFileStatus> fileStatuses = Lists.newArrayList();
for (int i = 0; i < fileGroup.getFilePaths().size(); i++) {
if (brokerDesc.isMultiLoadBroker()) {
TBrokerFileStatus fileStatus = new TBrokerFileStatus(fileGroup.getFilePaths().get(i),
false, fileGroup.getFileSize().get(i), false);
fileStatuses.add(fileStatus);
} else {
BrokerUtil.parseFile(fileGroup.getFilePaths().get(i), brokerDesc, fileStatuses);
}
}
// only get non-empty file or non-binary file
fileStatuses = fileStatuses.stream().filter(f -> {
return f.getSize() > 0 || !isBinaryFileFormat;
}).collect(Collectors.toList());
fileStatusesList.add(fileStatuses);
filesAdded += fileStatuses.size();
for (TBrokerFileStatus fstatus : fileStatuses) {
LOG.info("Add file status is {}", fstatus);
}
}
}
private void assignBackends() throws UserException {
Set<Tag> tags = Sets.newHashSet();
if (userIdentity != null) {
tags = Env.getCurrentEnv().getAuth().getResourceTags(userIdentity.getQualifiedUser());
if (tags == UserProperty.INVALID_RESOURCE_TAGS) {
throw new UserException("No valid resource tag for user: " + userIdentity.getQualifiedUser());
}
} else {
LOG.debug("user info in BrokerScanNode should not be null, add log to observer");
}
backends = Lists.newArrayList();
// broker scan node is used for query or load
BeSelectionPolicy policy = new BeSelectionPolicy.Builder().needQueryAvailable().needLoadAvailable()
.addTags(tags).build();
backends.addAll(policy.getCandidateBackends(Env.getCurrentSystemInfo().getIdToBackend().values()));
if (backends.isEmpty()) {
throw new UserException("No available backends");
}
}
private TFileFormatType formatType(String fileFormat, String path) throws UserException {
if (fileFormat != null) {
if (fileFormat.toLowerCase().equals("parquet")) {
return TFileFormatType.FORMAT_PARQUET;
} else if (fileFormat.toLowerCase().equals("orc")) {
return TFileFormatType.FORMAT_ORC;
} else if (fileFormat.toLowerCase().equals("json")) {
return TFileFormatType.FORMAT_JSON;
// csv/csv_with_name/csv_with_names_and_types treat as csv format
} else if (fileFormat.toLowerCase().equals(FeConstants.csv)
|| fileFormat.toLowerCase().equals(FeConstants.csv_with_names)
|| fileFormat.toLowerCase().equals(FeConstants.csv_with_names_and_types)
// TODO: Add TEXTFILE to TFileFormatType to Support hive text file format.
|| fileFormat.toLowerCase().equals(FeConstants.text)) {
return TFileFormatType.FORMAT_CSV_PLAIN;
} else {
throw new UserException("Not supported file format: " + fileFormat);
}
}
return Util.getFileFormatType(path);
}
public String getHostUri() throws UserException {
return "";
}
private String getHeaderType(String formatType) {
if (formatType != null) {
if (formatType.toLowerCase().equals(FeConstants.csv_with_names)
|| formatType.toLowerCase().equals(FeConstants.csv_with_names_and_types)) {
return formatType;
}
}
return "";
}
// If fileFormat is not null, we use fileFormat instead of check file's suffix
private void processFileGroup(
ParamCreateContext context,
List<TBrokerFileStatus> fileStatuses)
throws UserException {
if (fileStatuses == null || fileStatuses.isEmpty()) {
return;
}
// set hdfs params, used to Hive and Iceberg scan
THdfsParams tHdfsParams = new THdfsParams();
String fsName = getHostUri();
tHdfsParams.setFsName(fsName);
TScanRangeLocations curLocations = newLocations(context.params, brokerDesc);
long curInstanceBytes = 0;
long curFileOffset = 0;
for (int i = 0; i < fileStatuses.size(); ) {
TBrokerFileStatus fileStatus = fileStatuses.get(i);
long leftBytes = fileStatus.size - curFileOffset;
long tmpBytes = curInstanceBytes + leftBytes;
//header_type
String headerType = getHeaderType(context.fileGroup.getFileFormat());
TFileFormatType formatType = formatType(context.fileGroup.getFileFormat(), fileStatus.path);
List<String> columnsFromPath = BrokerUtil.parseColumnsFromPath(fileStatus.path,
context.fileGroup.getColumnNamesFromPath());
int numberOfColumnsFromFile = context.slotDescByName.size() - columnsFromPath.size();
if (tmpBytes > bytesPerInstance) {
// Now only support split plain text
if ((formatType == TFileFormatType.FORMAT_CSV_PLAIN && fileStatus.isSplitable)
|| formatType == TFileFormatType.FORMAT_JSON) {
long rangeBytes = bytesPerInstance - curInstanceBytes;
TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType,
rangeBytes, columnsFromPath, numberOfColumnsFromFile, brokerDesc, headerType);
if (formatType == TFileFormatType.FORMAT_JSON) {
rangeDesc.setStripOuterArray(context.fileGroup.isStripOuterArray());
rangeDesc.setJsonpaths(context.fileGroup.getJsonPaths());
rangeDesc.setJsonRoot(context.fileGroup.getJsonRoot());
rangeDesc.setFuzzyParse(context.fileGroup.isFuzzyParse());
rangeDesc.setNumAsString(context.fileGroup.isNumAsString());
rangeDesc.setReadJsonByLine(context.fileGroup.isReadJsonByLine());
}
curLocations.getScanRange().getBrokerScanRange().addToRanges(rangeDesc);
curFileOffset += rangeBytes;
} else {
TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType,
leftBytes, columnsFromPath, numberOfColumnsFromFile, brokerDesc, headerType);
if (rangeDesc.hdfs_params != null && rangeDesc.hdfs_params.getFsName() == null) {
rangeDesc.hdfs_params.setFsName(fsName);
} else if (rangeDesc.hdfs_params == null) {
rangeDesc.setHdfsParams(tHdfsParams);
}
rangeDesc.setReadByColumnDef(true);
curLocations.getScanRange().getBrokerScanRange().addToRanges(rangeDesc);
curFileOffset = 0;
i++;
}
// New one scan
locationsList.add(curLocations);
curLocations = newLocations(context.params, brokerDesc);
curInstanceBytes = 0;
} else {
TBrokerRangeDesc rangeDesc = createBrokerRangeDesc(curFileOffset, fileStatus, formatType,
leftBytes, columnsFromPath, numberOfColumnsFromFile, brokerDesc, headerType);
if (formatType == TFileFormatType.FORMAT_JSON) {
rangeDesc.setStripOuterArray(context.fileGroup.isStripOuterArray());
rangeDesc.setJsonpaths(context.fileGroup.getJsonPaths());
rangeDesc.setJsonRoot(context.fileGroup.getJsonRoot());
rangeDesc.setFuzzyParse(context.fileGroup.isFuzzyParse());
rangeDesc.setNumAsString(context.fileGroup.isNumAsString());
rangeDesc.setReadJsonByLine(context.fileGroup.isReadJsonByLine());
}
if (rangeDesc.hdfs_params != null && rangeDesc.hdfs_params.getFsName() == null) {
rangeDesc.hdfs_params.setFsName(fsName);
} else if (rangeDesc.hdfs_params == null) {
rangeDesc.setHdfsParams(tHdfsParams);
}
rangeDesc.setReadByColumnDef(true);
curLocations.getScanRange().getBrokerScanRange().addToRanges(rangeDesc);
curFileOffset = 0;
curInstanceBytes += leftBytes;
i++;
}
}
// Put the last file
if (curLocations.getScanRange().getBrokerScanRange().isSetRanges()) {
locationsList.add(curLocations);
}
}
private TBrokerRangeDesc createBrokerRangeDesc(long curFileOffset, TBrokerFileStatus fileStatus,
TFileFormatType formatType, long rangeBytes,
List<String> columnsFromPath, int numberOfColumnsFromFile,
BrokerDesc brokerDesc, String headerType) {
TBrokerRangeDesc rangeDesc = new TBrokerRangeDesc();
rangeDesc.setFileType(brokerDesc.getFileType());
rangeDesc.setFormatType(formatType);
rangeDesc.setPath(fileStatus.path);
rangeDesc.setSplittable(fileStatus.isSplitable);
rangeDesc.setStartOffset(curFileOffset);
rangeDesc.setSize(rangeBytes);
// fileSize only be used when format is orc or parquet and TFileType is broker
// When TFileType is other type, it is not necessary
rangeDesc.setFileSize(fileStatus.size);
// In Backend, will append columnsFromPath to the end of row after data scanned from file.
rangeDesc.setNumOfColumnsFromFile(numberOfColumnsFromFile);
rangeDesc.setColumnsFromPath(columnsFromPath);
rangeDesc.setHeaderType(headerType);
// set hdfs params for hdfs file type.
if (brokerDesc.getFileType() == TFileType.FILE_HDFS) {
THdfsParams tHdfsParams = HdfsResource.generateHdfsParam(brokerDesc.getProperties());
rangeDesc.setHdfsParams(tHdfsParams);
}
return rangeDesc;
}
//TODO(wx):support quantile state column or forbidden it.
@Override
public void finalize(Analyzer analyzer) throws UserException {
locationsList = Lists.newArrayList();
for (int i = 0; i < fileGroups.size(); ++i) {
List<TBrokerFileStatus> fileStatuses = fileStatusesList.get(i);
if (fileStatuses.isEmpty()) {
continue;
}
ParamCreateContext context = paramCreateContexts.get(i);
try {
finalizeParams(context.slotDescByName, context.exprMap, context.params,
context.srcTupleDescriptor, strictMode, context.fileGroup.isNegative(), analyzer);
} catch (AnalysisException e) {
throw new UserException(e.getMessage());
}
processFileGroup(context, fileStatuses);
}
if (LOG.isDebugEnabled()) {
for (TScanRangeLocations locations : locationsList) {
LOG.debug("Scan range is {}", locations);
}
}
if (loadJobId != -1) {
LOG.info("broker load job {} with txn {} has {} scan range: {}",
loadJobId, txnId, locationsList.size(),
brokerDesc.isMultiLoadBroker() ? "local"
: locationsList.stream().map(loc -> loc.locations.get(0).backend_id).toArray());
}
}
@Override
public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
return locationsList;
}
@Override
public String getNodeExplainString(String prefix, TExplainLevel detailLevel) {
StringBuilder output = new StringBuilder();
if (!isLoad()) {
BrokerTable brokerTable = (BrokerTable) targetTable;
output.append(prefix).append("TABLE: ").append(brokerTable.getName()).append("\n");
if (detailLevel != TExplainLevel.BRIEF) {
output.append(prefix).append("PATH: ")
.append(Joiner.on(",").join(brokerTable.getPaths())).append("\",\n");
}
}
output.append(prefix).append("BROKER: ").append(brokerDesc.getName()).append("\n");
return output.toString();
}
}

View File

@ -1,198 +0,0 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.planner;
import org.apache.doris.analysis.Analyzer;
import org.apache.doris.analysis.BrokerDesc;
import org.apache.doris.analysis.ImportColumnDesc;
import org.apache.doris.analysis.StorageBackend;
import org.apache.doris.analysis.TupleDescriptor;
import org.apache.doris.catalog.HMSResource;
import org.apache.doris.catalog.HiveMetaStoreClientHelper;
import org.apache.doris.catalog.HiveTable;
import org.apache.doris.common.FeConstants;
import org.apache.doris.common.UserException;
import org.apache.doris.common.util.Util;
import org.apache.doris.load.BrokerFileGroup;
import org.apache.doris.statistics.StatisticalType;
import org.apache.doris.thrift.TBrokerFileStatus;
import org.apache.doris.thrift.TExplainLevel;
import com.google.common.base.Strings;
import com.google.common.collect.Lists;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
import org.apache.hadoop.hive.metastore.api.Table;
import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
public class HiveScanNode extends BrokerScanNode {
private static final Logger LOG = LogManager.getLogger(HiveScanNode.class);
private static final String HIVE_DEFAULT_COLUMN_SEPARATOR = "\001";
private static final String HIVE_DEFAULT_LINE_DELIMITER = "\n";
private HiveTable hiveTable;
// partition column predicates of hive table
private ExprNodeGenericFuncDesc hivePartitionPredicate;
private List<ImportColumnDesc> parsedColumnExprList = new ArrayList<>();
private String hdfsUri;
private Table remoteHiveTable;
/* hive table properties */
private String columnSeparator;
private String lineDelimiter;
private String fileFormat;
private String path;
private List<String> partitionKeys = new ArrayList<>();
private StorageBackend.StorageType storageType;
/* hive table properties */
public String getHostUri() {
return hdfsUri;
}
public List<ImportColumnDesc> getParsedColumnExprList() {
return parsedColumnExprList;
}
public String getColumnSeparator() {
return columnSeparator;
}
public String getLineDelimiter() {
return lineDelimiter;
}
public String getFileFormat() {
return fileFormat;
}
public String getPath() {
return path;
}
public List<String> getPartitionKeys() {
return partitionKeys;
}
public HiveScanNode(PlanNodeId id, TupleDescriptor destTupleDesc, String planNodeName,
List<List<TBrokerFileStatus>> fileStatusesList, int filesAdded) {
super(id, destTupleDesc, planNodeName, fileStatusesList, filesAdded, StatisticalType.HIVE_SCAN_NODE);
this.hiveTable = (HiveTable) destTupleDesc.getTable();
}
@Override
public void init(Analyzer analyzer) throws UserException {
super.init(analyzer);
}
@Override
protected void initFileGroup() throws UserException {
initHiveTblProperties();
analyzeColumnFromPath();
HiveTable hiveTable = (HiveTable) desc.getTable();
fileGroups = Lists.newArrayList(
new BrokerFileGroup(hiveTable.getId(),
getColumnSeparator(),
getLineDelimiter(),
getPath(),
getFileFormat(),
getPartitionKeys(),
getParsedColumnExprList()));
brokerDesc = new BrokerDesc("HiveTableDesc", storageType, hiveTable.getHiveProperties());
targetTable = hiveTable;
}
private void setStorageType(String location) throws UserException {
String[] strings = StringUtils.split(location, "/");
String storagePrefix = strings[0].split(":")[0];
if (Util.isS3CompatibleStorageSchema(storagePrefix)) {
this.storageType = StorageBackend.StorageType.S3;
} else if (storagePrefix.equalsIgnoreCase("hdfs")) {
this.storageType = StorageBackend.StorageType.HDFS;
} else if (storagePrefix.equalsIgnoreCase(FeConstants.FS_PREFIX_OFS)) {
this.storageType = StorageBackend.StorageType.OFS;
} else if (storagePrefix.equalsIgnoreCase(FeConstants.FS_PREFIX_JFS)) {
this.storageType = StorageBackend.StorageType.JFS;
} else {
throw new UserException("Not supported storage type: " + storagePrefix);
}
}
private void initHiveTblProperties() throws UserException {
this.remoteHiveTable = HiveMetaStoreClientHelper.getTable(hiveTable);
this.fileFormat = HiveMetaStoreClientHelper.HiveFileFormat.getFormat(remoteHiveTable.getSd().getInputFormat());
this.setStorageType(remoteHiveTable.getSd().getLocation());
Map<String, String> serDeInfoParams = remoteHiveTable.getSd().getSerdeInfo().getParameters();
this.columnSeparator = Strings.isNullOrEmpty(serDeInfoParams.get("field.delim"))
? HIVE_DEFAULT_COLUMN_SEPARATOR : serDeInfoParams.get("field.delim");
this.lineDelimiter = Strings.isNullOrEmpty(serDeInfoParams.get("line.delim"))
? HIVE_DEFAULT_LINE_DELIMITER : serDeInfoParams.get("line.delim");
this.path = remoteHiveTable.getSd().getLocation();
for (FieldSchema fieldSchema : remoteHiveTable.getPartitionKeys()) {
this.partitionKeys.add(fieldSchema.getName());
}
}
@Override
protected void getFileStatus() throws UserException {
if (partitionKeys.size() > 0) {
// Hive Table is no longer supported.
// So there we just create an empty predicate
hivePartitionPredicate = new ExprNodeGenericFuncDesc();
}
List<TBrokerFileStatus> fileStatuses = new ArrayList<>();
this.hdfsUri = HiveMetaStoreClientHelper.getHiveDataFiles(hiveTable, hivePartitionPredicate,
fileStatuses, remoteHiveTable, storageType);
fileStatusesList.add(fileStatuses);
filesAdded += fileStatuses.size();
for (TBrokerFileStatus fstatus : fileStatuses) {
LOG.debug("Add file status is {}", fstatus);
}
}
@Override
public String getNodeExplainString(String prefix, TExplainLevel detailLevel) {
StringBuilder output = new StringBuilder();
if (!isLoad()) {
output.append(prefix).append("TABLE: ").append(hiveTable.getName()).append("\n");
output.append(prefix).append("PATH: ")
.append(hiveTable.getHiveProperties().get(HMSResource.HIVE_METASTORE_URIS)).append("\n");
}
return output.toString();
}
/**
* Analyze columns from path, the partition columns
*/
private void analyzeColumnFromPath() {
for (String colName : partitionKeys) {
ImportColumnDesc importColumnDesc = new ImportColumnDesc(colName, null);
parsedColumnExprList.add(importColumnDesc);
}
}
}

View File

@ -1,112 +0,0 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.planner;
import org.apache.doris.analysis.Analyzer;
import org.apache.doris.analysis.BrokerDesc;
import org.apache.doris.analysis.Expr;
import org.apache.doris.analysis.TupleDescriptor;
import org.apache.doris.catalog.IcebergProperty;
import org.apache.doris.catalog.IcebergTable;
import org.apache.doris.common.UserException;
import org.apache.doris.external.iceberg.util.IcebergUtils;
import org.apache.doris.load.BrokerFileGroup;
import org.apache.doris.statistics.StatisticalType;
import org.apache.doris.thrift.TBrokerFileStatus;
import org.apache.doris.thrift.TExplainLevel;
import com.google.common.collect.Lists;
import org.apache.iceberg.expressions.Expression;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.util.ArrayList;
import java.util.List;
import java.util.ListIterator;
public class IcebergScanNode extends BrokerScanNode {
private static final Logger LOG = LogManager.getLogger(IcebergScanNode.class);
private IcebergTable icebergTable;
private final List<Expression> icebergPredicates = new ArrayList<>();
public IcebergScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName,
List<List<TBrokerFileStatus>> fileStatusesList, int filesAdded) {
super(id, desc, planNodeName, fileStatusesList, filesAdded, StatisticalType.ICEBERG_SCAN_NODE);
icebergTable = (IcebergTable) desc.getTable();
}
@Override
public void init(Analyzer analyzer) throws UserException {
super.init(analyzer);
}
@Override
protected void initFileGroup() throws UserException {
fileGroups = Lists.newArrayList(
new BrokerFileGroup(icebergTable.getId(),
null,
icebergTable.getFileFormat()));
brokerDesc = new BrokerDesc("IcebergTableDesc", icebergTable.getStorageType(),
icebergTable.getIcebergProperties());
targetTable = icebergTable;
}
@Override
public String getHostUri() throws UserException {
return icebergTable.getHostUri();
}
@Override
protected void getFileStatus() throws UserException {
// extract iceberg conjuncts
ListIterator<Expr> it = conjuncts.listIterator();
while (it.hasNext()) {
Expression expression = IcebergUtils.convertToIcebergExpr(it.next());
if (expression != null) {
icebergPredicates.add(expression);
}
}
// get iceberg file status
List<TBrokerFileStatus> fileStatuses;
try {
fileStatuses = icebergTable.getIcebergDataFiles(icebergPredicates);
} catch (Exception e) {
LOG.warn("errors while load iceberg table {} data files.", icebergTable.getName(), e);
throw new UserException("errors while load Iceberg table ["
+ icebergTable.getName() + "] data files.");
}
fileStatusesList.add(fileStatuses);
filesAdded += fileStatuses.size();
for (TBrokerFileStatus fstatus : fileStatuses) {
LOG.debug("Add file status is {}", fstatus);
}
}
@Override
public String getNodeExplainString(String prefix, TExplainLevel detailLevel) {
StringBuilder output = new StringBuilder();
if (!isLoad()) {
output.append(prefix).append("TABLE: ").append(icebergTable.getName()).append("\n");
output.append(prefix).append("PATH: ")
.append(icebergTable.getIcebergProperties().get(IcebergProperty.ICEBERG_HIVE_METASTORE_URIS))
.append("\n");
}
return output.toString();
}
}

View File

@ -1,256 +0,0 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.planner;
import org.apache.doris.analysis.Analyzer;
import org.apache.doris.analysis.ArithmeticExpr;
import org.apache.doris.analysis.Expr;
import org.apache.doris.analysis.ExprSubstitutionMap;
import org.apache.doris.analysis.FunctionCallExpr;
import org.apache.doris.analysis.IntLiteral;
import org.apache.doris.analysis.NullLiteral;
import org.apache.doris.analysis.SlotDescriptor;
import org.apache.doris.analysis.SlotRef;
import org.apache.doris.analysis.StringLiteral;
import org.apache.doris.analysis.TupleDescriptor;
import org.apache.doris.catalog.AggregateType;
import org.apache.doris.catalog.Column;
import org.apache.doris.catalog.FunctionSet;
import org.apache.doris.catalog.PrimitiveType;
import org.apache.doris.catalog.Type;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.Config;
import org.apache.doris.common.UserException;
import org.apache.doris.load.loadv2.LoadTask;
import org.apache.doris.rewrite.ExprRewriter;
import org.apache.doris.statistics.StatisticalType;
import org.apache.doris.thrift.TBrokerScanNode;
import org.apache.doris.thrift.TBrokerScanRangeParams;
import org.apache.doris.thrift.TPlanNode;
import org.apache.doris.thrift.TPlanNodeType;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import java.util.List;
import java.util.Map;
public abstract class LoadScanNode extends ScanNode {
protected Expr deleteCondition;
protected LoadTask.MergeType mergeType = LoadTask.MergeType.APPEND;
public LoadScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName) {
super(id, desc, planNodeName, StatisticalType.LOAD_SCAN_NODE);
}
public LoadScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, StatisticalType statisticalType) {
super(id, desc, planNodeName, statisticalType);
}
protected void initAndSetWhereExpr(Expr whereExpr, TupleDescriptor tupleDesc,
Analyzer analyzer) throws UserException {
Expr newWhereExpr = initWhereExpr(whereExpr, tupleDesc, analyzer);
if (newWhereExpr != null) {
addConjuncts(newWhereExpr.getConjuncts());
}
}
protected void initAndSetPrecedingFilter(Expr whereExpr,
TupleDescriptor tupleDesc, Analyzer analyzer) throws UserException {
Expr newWhereExpr = initWhereExpr(whereExpr, tupleDesc, analyzer);
if (newWhereExpr != null) {
addPreFilterConjuncts(newWhereExpr.getConjuncts());
}
}
private Expr initWhereExpr(Expr whereExpr, TupleDescriptor tupleDesc, Analyzer analyzer) throws UserException {
if (whereExpr == null) {
return null;
}
Map<String, SlotDescriptor> dstDescMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
for (SlotDescriptor slotDescriptor : tupleDesc.getSlots()) {
dstDescMap.put(slotDescriptor.getColumn().getName(), slotDescriptor);
}
// substitute SlotRef in filter expression
// where expr must be equal first to transfer some predicates(eg: BetweenPredicate to BinaryPredicate)
Expr newWhereExpr = analyzer.getExprRewriter()
.rewrite(whereExpr, analyzer, ExprRewriter.ClauseType.WHERE_CLAUSE);
List<SlotRef> slots = Lists.newArrayList();
newWhereExpr.collect(SlotRef.class, slots);
ExprSubstitutionMap smap = new ExprSubstitutionMap();
for (SlotRef slot : slots) {
SlotDescriptor slotDesc = dstDescMap.get(slot.getColumnName());
if (slotDesc == null) {
throw new UserException("unknown column reference in where statement, reference="
+ slot.getColumnName());
}
smap.getLhs().add(slot);
smap.getRhs().add(new SlotRef(slotDesc));
}
newWhereExpr = newWhereExpr.clone(smap);
newWhereExpr.analyze(analyzer);
if (!newWhereExpr.getType().equals(Type.BOOLEAN)) {
throw new UserException("where statement is not a valid statement return bool");
}
return newWhereExpr;
}
protected void checkBitmapCompatibility(Analyzer analyzer,
SlotDescriptor slotDesc, Expr expr) throws AnalysisException {
if (slotDesc.getColumn().getAggregationType() == AggregateType.BITMAP_UNION) {
expr.analyze(analyzer);
if (!expr.getType().isBitmapType()) {
String errorMsg = String.format("bitmap column %s require the function return type is BITMAP",
slotDesc.getColumn().getName());
throw new AnalysisException(errorMsg);
}
}
}
protected void checkQuantileStateCompatibility(Analyzer analyzer,
SlotDescriptor slotDesc, Expr expr) throws AnalysisException {
if (slotDesc.getColumn().getAggregationType() == AggregateType.QUANTILE_UNION) {
expr.analyze(analyzer);
if (!expr.getType().isQuantileStateType()) {
String errorMsg = "quantile_state column %s require the function return type is QUANTILE_STATE";
throw new AnalysisException(errorMsg);
}
}
}
protected void finalizeParams(Map<String, SlotDescriptor> slotDescByName,
Map<String, Expr> exprMap,
TBrokerScanRangeParams params,
TupleDescriptor srcTupleDesc,
boolean strictMode,
boolean negative,
Analyzer analyzer) throws UserException {
Map<Integer, Integer> destSidToSrcSidWithoutTrans = Maps.newHashMap();
for (SlotDescriptor destSlotDesc : desc.getSlots()) {
if (!destSlotDesc.isMaterialized()) {
continue;
}
Expr expr = null;
if (exprMap != null) {
expr = exprMap.get(destSlotDesc.getColumn().getName());
}
if (expr == null) {
SlotDescriptor srcSlotDesc = slotDescByName.get(destSlotDesc.getColumn().getName());
if (srcSlotDesc != null) {
destSidToSrcSidWithoutTrans.put(destSlotDesc.getId().asInt(), srcSlotDesc.getId().asInt());
// If dest is allow null, we set source to nullable
if (destSlotDesc.getColumn().isAllowNull()) {
srcSlotDesc.setIsNullable(true);
}
expr = new SlotRef(srcSlotDesc);
} else {
Column column = destSlotDesc.getColumn();
if (column.getDefaultValue() != null) {
if (column.getDefaultValueExprDef() != null) {
expr = column.getDefaultValueExpr();
} else {
expr = new StringLiteral(destSlotDesc.getColumn().getDefaultValue());
}
} else {
if (column.isAllowNull()) {
expr = NullLiteral.create(column.getType());
} else {
throw new AnalysisException("column has no source field, column=" + column.getName());
}
}
}
}
// check hll_hash
if (destSlotDesc.getType().getPrimitiveType() == PrimitiveType.HLL) {
if (!(expr instanceof FunctionCallExpr)) {
throw new AnalysisException("HLL column must use " + FunctionSet.HLL_HASH + " function, like "
+ destSlotDesc.getColumn().getName() + "=" + FunctionSet.HLL_HASH + "(xxx)");
}
FunctionCallExpr fn = (FunctionCallExpr) expr;
if (!fn.getFnName().getFunction().equalsIgnoreCase(FunctionSet.HLL_HASH)
&& !fn.getFnName().getFunction().equalsIgnoreCase("hll_empty")) {
throw new AnalysisException("HLL column must use " + FunctionSet.HLL_HASH + " function, like "
+ destSlotDesc.getColumn().getName() + "=" + FunctionSet.HLL_HASH
+ "(xxx) or " + destSlotDesc.getColumn().getName() + "=hll_empty()");
}
expr.setType(Type.HLL);
}
checkBitmapCompatibility(analyzer, destSlotDesc, expr);
checkQuantileStateCompatibility(analyzer, destSlotDesc, expr);
// check quantile_state
if (negative && destSlotDesc.getColumn().getAggregationType() == AggregateType.SUM) {
expr = new ArithmeticExpr(ArithmeticExpr.Operator.MULTIPLY, expr, new IntLiteral(-1));
expr.analyze(analyzer);
}
// for jsonb type, use jsonb_parse_xxx to parse src string to jsonb.
// and if input string is not a valid json string, return null.
PrimitiveType dstType = destSlotDesc.getType().getPrimitiveType();
PrimitiveType srcType = expr.getType().getPrimitiveType();
if (dstType == PrimitiveType.JSONB
&& (srcType == PrimitiveType.VARCHAR || srcType == PrimitiveType.STRING)) {
List<Expr> args = Lists.newArrayList();
args.add(expr);
String nullable = "notnull";
if (destSlotDesc.getIsNullable() || expr.isNullable()) {
nullable = "nullable";
}
String name = "jsonb_parse_" + nullable + "_error_to_null";
expr = new FunctionCallExpr(name, args);
expr.analyze(analyzer);
} else {
expr = castToSlot(destSlotDesc, expr);
}
params.putToExprOfDestSlot(destSlotDesc.getId().asInt(), expr.treeToThrift());
}
params.setDestSidToSrcSidWithoutTrans(destSidToSrcSidWithoutTrans);
params.setDestTupleId(desc.getId().asInt());
params.setStrictMode(strictMode);
params.setSrcTupleId(srcTupleDesc.getId().asInt());
// LOG.info("brokerScanRange is {}", brokerScanRange);
// Need re compute memory layout after set some slot descriptor to nullable
srcTupleDesc.computeStatAndMemLayout();
}
@Override
protected void toThrift(TPlanNode planNode) {
planNode.setNodeType(TPlanNodeType.BROKER_SCAN_NODE);
TBrokerScanNode brokerScanNode = new TBrokerScanNode(desc.getId().asInt());
if (!preFilterConjuncts.isEmpty()) {
if (Config.enable_vectorized_load && vpreFilterConjunct != null) {
brokerScanNode.addToPreFilterExprs(vpreFilterConjunct.treeToThrift());
} else {
for (Expr e : preFilterConjuncts) {
brokerScanNode.addToPreFilterExprs(e.treeToThrift());
}
}
}
planNode.setBrokerScanNode(brokerScanNode);
}
}

View File

@ -1934,16 +1934,12 @@ public class SingleNodePlanner {
}
break;
case BROKER:
scanNode = new BrokerScanNode(ctx.getNextNodeId(), tblRef.getDesc(), "BrokerScanNode",
null, -1);
break;
throw new RuntimeException("Broker external table is not supported, try to use table function please");
case ELASTICSEARCH:
scanNode = new EsScanNode(ctx.getNextNodeId(), tblRef.getDesc(), "EsScanNode");
break;
case HIVE:
scanNode = new HiveScanNode(ctx.getNextNodeId(), tblRef.getDesc(), "HiveScanNode",
null, -1);
break;
throw new RuntimeException("Hive external table is not supported, try to use hive catalog please");
case ICEBERG:
scanNode = new ExternalFileScanNode(ctx.getNextNodeId(), tblRef.getDesc());
break;
@ -1968,7 +1964,7 @@ public class SingleNodePlanner {
default:
break;
}
if (scanNode instanceof OlapScanNode || scanNode instanceof EsScanNode || scanNode instanceof HiveScanNode
if (scanNode instanceof OlapScanNode || scanNode instanceof EsScanNode
|| scanNode instanceof ExternalFileScanNode) {
if (analyzer.enableInferPredicate()) {
PredicatePushDown.visitScanNode(scanNode, tblRef.getJoinOp(), analyzer);

View File

@ -172,30 +172,26 @@ public class StreamLoadPlanner {
}
// create scan node
if (Config.enable_new_load_scan_node && Config.enable_vectorized_load) {
ExternalFileScanNode fileScanNode = new ExternalFileScanNode(new PlanNodeId(0), scanTupleDesc);
// 1. create file group
DataDescription dataDescription = new DataDescription(destTable.getName(), taskInfo);
dataDescription.analyzeWithoutCheckPriv(db.getFullName());
BrokerFileGroup fileGroup = new BrokerFileGroup(dataDescription);
fileGroup.parse(db, dataDescription);
// 2. create dummy file status
TBrokerFileStatus fileStatus = new TBrokerFileStatus();
if (taskInfo.getFileType() == TFileType.FILE_LOCAL) {
fileStatus.setPath(taskInfo.getPath());
fileStatus.setIsDir(false);
fileStatus.setSize(taskInfo.getFileSize()); // must set to -1, means stream.
} else {
fileStatus.setPath("");
fileStatus.setIsDir(false);
fileStatus.setSize(-1); // must set to -1, means stream.
}
fileScanNode.setLoadInfo(loadId, taskInfo.getTxnId(), destTable, BrokerDesc.createForStreamLoad(),
fileGroup, fileStatus, taskInfo.isStrictMode(), taskInfo.getFileType());
scanNode = fileScanNode;
ExternalFileScanNode fileScanNode = new ExternalFileScanNode(new PlanNodeId(0), scanTupleDesc);
// 1. create file group
DataDescription dataDescription = new DataDescription(destTable.getName(), taskInfo);
dataDescription.analyzeWithoutCheckPriv(db.getFullName());
BrokerFileGroup fileGroup = new BrokerFileGroup(dataDescription);
fileGroup.parse(db, dataDescription);
// 2. create dummy file status
TBrokerFileStatus fileStatus = new TBrokerFileStatus();
if (taskInfo.getFileType() == TFileType.FILE_LOCAL) {
fileStatus.setPath(taskInfo.getPath());
fileStatus.setIsDir(false);
fileStatus.setSize(taskInfo.getFileSize()); // must set to -1, means stream.
} else {
scanNode = new StreamLoadScanNode(loadId, new PlanNodeId(0), scanTupleDesc, destTable, taskInfo);
fileStatus.setPath("");
fileStatus.setIsDir(false);
fileStatus.setSize(-1); // must set to -1, means stream.
}
fileScanNode.setLoadInfo(loadId, taskInfo.getTxnId(), destTable, BrokerDesc.createForStreamLoad(),
fileGroup, fileStatus, taskInfo.isStrictMode(), taskInfo.getFileType());
scanNode = fileScanNode;
scanNode.init(analyzer);
scanNode.finalize(analyzer);

View File

@ -1,215 +0,0 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.planner;
import org.apache.doris.analysis.Analyzer;
import org.apache.doris.analysis.Expr;
import org.apache.doris.analysis.ImportColumnDesc;
import org.apache.doris.analysis.IntLiteral;
import org.apache.doris.analysis.SlotDescriptor;
import org.apache.doris.analysis.SlotRef;
import org.apache.doris.analysis.TupleDescriptor;
import org.apache.doris.catalog.Column;
import org.apache.doris.catalog.OlapTable;
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.statistics.StatisticalType;
import org.apache.doris.task.LoadTaskInfo;
import org.apache.doris.thrift.TBrokerRangeDesc;
import org.apache.doris.thrift.TBrokerScanRange;
import org.apache.doris.thrift.TBrokerScanRangeParams;
import org.apache.doris.thrift.TExplainLevel;
import org.apache.doris.thrift.TFileFormatType;
import org.apache.doris.thrift.TScanRange;
import org.apache.doris.thrift.TScanRangeLocations;
import org.apache.doris.thrift.TUniqueId;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.nio.charset.Charset;
import java.util.List;
import java.util.Map;
/**
* used to scan from stream
*/
public class StreamLoadScanNode extends LoadScanNode {
private static final Logger LOG = LogManager.getLogger(StreamLoadScanNode.class);
private TUniqueId loadId;
// TODO(zc): now we use scanRange
// input parameter
private Table dstTable;
private LoadTaskInfo taskInfo;
// helper
private Analyzer analyzer;
private TupleDescriptor srcTupleDesc;
private TBrokerScanRange brokerScanRange;
// If use case sensitive map, for example,
// the column name 「A」 in the table and the mapping '(a) set (A = a)' in load sql,
// Slotdescbyname stores「a」, later will use 「a」to get table's 「A」 column info, will throw exception.
private final Map<String, SlotDescriptor> slotDescByName = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
private final Map<String, Expr> exprsByName = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
// used to construct for streaming loading
public StreamLoadScanNode(
TUniqueId loadId, PlanNodeId id, TupleDescriptor tupleDesc, Table dstTable, LoadTaskInfo taskInfo) {
super(id, tupleDesc, "StreamLoadScanNode", StatisticalType.STREAM_LOAD_SCAN_NODE);
this.loadId = loadId;
this.dstTable = dstTable;
this.taskInfo = taskInfo;
this.numInstances = 1;
}
@Override
public void init(Analyzer analyzer) throws UserException {
// can't call super.init(), because after super.init, conjuncts would be null
assignConjuncts(analyzer);
this.analyzer = analyzer;
brokerScanRange = new TBrokerScanRange();
deleteCondition = taskInfo.getDeleteCondition();
mergeType = taskInfo.getMergeType();
TBrokerRangeDesc rangeDesc = new TBrokerRangeDesc();
rangeDesc.file_type = taskInfo.getFileType();
rangeDesc.format_type = taskInfo.getFormatType();
if (rangeDesc.format_type == TFileFormatType.FORMAT_JSON) {
if (!taskInfo.getJsonPaths().isEmpty()) {
rangeDesc.setJsonpaths(taskInfo.getJsonPaths());
}
if (!taskInfo.getJsonRoot().isEmpty()) {
rangeDesc.setJsonRoot(taskInfo.getJsonRoot());
}
rangeDesc.setStripOuterArray(taskInfo.isStripOuterArray());
rangeDesc.setNumAsString(taskInfo.isNumAsString());
rangeDesc.setFuzzyParse(taskInfo.isFuzzyParse());
rangeDesc.setReadJsonByLine(taskInfo.isReadJsonByLine());
}
rangeDesc.splittable = false;
switch (taskInfo.getFileType()) {
case FILE_LOCAL:
rangeDesc.path = taskInfo.getPath();
break;
case FILE_STREAM:
rangeDesc.path = "Invalid Path";
rangeDesc.load_id = loadId;
break;
default:
throw new UserException("unsupported file type, type=" + taskInfo.getFileType());
}
rangeDesc.start_offset = 0;
rangeDesc.setHeaderType(taskInfo.getHeaderType());
rangeDesc.size = -1;
brokerScanRange.addToRanges(rangeDesc);
srcTupleDesc = analyzer.getDescTbl().createTupleDescriptor("StreamLoadScanNode");
TBrokerScanRangeParams params = new TBrokerScanRangeParams();
LoadTaskInfo.ImportColumnDescs columnExprDescs = taskInfo.getColumnExprDescs();
if (!columnExprDescs.isColumnDescsRewrited) {
if (mergeType == LoadTask.MergeType.MERGE) {
columnExprDescs.descs.add(ImportColumnDesc.newDeleteSignImportColumnDesc(deleteCondition));
} else if (mergeType == LoadTask.MergeType.DELETE) {
columnExprDescs.descs.add(ImportColumnDesc.newDeleteSignImportColumnDesc(new IntLiteral(1)));
}
if (dstTable instanceof OlapTable && ((OlapTable) dstTable).hasSequenceCol()) {
String sequenceCol = ((OlapTable) dstTable).getSequenceMapCol();
if (sequenceCol == null) {
sequenceCol = taskInfo.getSequenceCol();
}
columnExprDescs.descs.add(new ImportColumnDesc(Column.SEQUENCE_COL,
new SlotRef(null, sequenceCol)));
}
}
if (params.getSrcSlotIds() == null) {
params.setSrcSlotIds(Lists.newArrayList());
}
Load.initColumns(dstTable, columnExprDescs, null /* no hadoop function */, exprsByName, analyzer, srcTupleDesc,
slotDescByName, params.getSrcSlotIds(), taskInfo.getFormatType(), taskInfo.getHiddenColumns(),
VectorizedUtil.isVectorized());
// analyze where statement
initAndSetPrecedingFilter(taskInfo.getPrecedingFilter(), this.srcTupleDesc, analyzer);
initAndSetWhereExpr(taskInfo.getWhereExpr(), this.desc, analyzer);
createDefaultSmap(analyzer);
if (taskInfo.getColumnSeparator() != null) {
String sep = taskInfo.getColumnSeparator().getSeparator();
params.setColumnSeparatorStr(sep);
params.setColumnSeparatorLength(sep.getBytes(Charset.forName("UTF-8")).length);
params.setColumnSeparator(sep.getBytes(Charset.forName("UTF-8"))[0]);
} else {
params.setColumnSeparator((byte) '\t');
params.setColumnSeparatorLength(1);
params.setColumnSeparatorStr("\t");
}
if (taskInfo.getLineDelimiter() != null) {
String sep = taskInfo.getLineDelimiter().getSeparator();
params.setLineDelimiterStr(sep);
params.setLineDelimiterLength(sep.getBytes(Charset.forName("UTF-8")).length);
params.setLineDelimiter(sep.getBytes(Charset.forName("UTF-8"))[0]);
} else {
params.setLineDelimiter((byte) '\n');
params.setLineDelimiterLength(1);
}
params.setTrimDoubleQuotes(taskInfo.getTrimDoubleQuotes());
params.setDestTupleId(desc.getId().asInt());
brokerScanRange.setParams(params);
brokerScanRange.setBrokerAddresses(Lists.newArrayList());
computeStats(analyzer);
}
@Override
public void finalize(Analyzer analyzer) throws UserException {
finalizeParams(slotDescByName, exprsByName, brokerScanRange.params, srcTupleDesc,
taskInfo.isStrictMode(), taskInfo.getNegative(), analyzer);
}
@Override
public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
TScanRangeLocations locations = new TScanRangeLocations();
TScanRange scanRange = new TScanRange();
scanRange.setBrokerScanRange(brokerScanRange);
locations.setScanRange(scanRange);
locations.setLocations(Lists.newArrayList());
return Lists.newArrayList(locations);
}
@Override
public int getNumInstances() {
return 1;
}
@Override
public String getNodeExplainString(String prefix, TExplainLevel detailLevel) {
return "StreamLoadScanNode";
}
}

View File

@ -19,7 +19,6 @@ package org.apache.doris.qe;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.common.Config;
import org.apache.doris.common.UserException;
import org.apache.doris.planner.StreamLoadPlanner;
import org.apache.doris.proto.InternalService;
@ -29,7 +28,6 @@ import org.apache.doris.rpc.RpcException;
import org.apache.doris.system.Backend;
import org.apache.doris.system.BeSelectionPolicy;
import org.apache.doris.task.StreamLoadTask;
import org.apache.doris.thrift.TBrokerRangeDesc;
import org.apache.doris.thrift.TExecPlanFragmentParams;
import org.apache.doris.thrift.TExecPlanFragmentParamsList;
import org.apache.doris.thrift.TFileCompressType;
@ -77,16 +75,10 @@ public class InsertStreamTxnExecutor {
tRequest.setTxnConf(txnConf).setImportLabel(txnEntry.getLabel());
for (Map.Entry<Integer, List<TScanRangeParams>> entry : tRequest.params.per_node_scan_ranges.entrySet()) {
for (TScanRangeParams scanRangeParams : entry.getValue()) {
if (Config.enable_new_load_scan_node && Config.enable_vectorized_load) {
scanRangeParams.scan_range.ext_scan_range.file_scan_range.params.setFormatType(
TFileFormatType.FORMAT_PROTO);
scanRangeParams.scan_range.ext_scan_range.file_scan_range.params.setCompressType(
TFileCompressType.PLAIN);
} else {
for (TBrokerRangeDesc desc : scanRangeParams.scan_range.broker_scan_range.ranges) {
desc.setFormatType(TFileFormatType.FORMAT_PROTO);
}
}
scanRangeParams.scan_range.ext_scan_range.file_scan_range.params.setFormatType(
TFileFormatType.FORMAT_PROTO);
scanRangeParams.scan_range.ext_scan_range.file_scan_range.params.setCompressType(
TFileCompressType.PLAIN);
}
}
txnConf.setFragmentInstanceId(tRequest.params.fragment_instance_id);

View File

@ -43,7 +43,6 @@ import org.apache.doris.load.EtlStatus;
import org.apache.doris.load.Load;
import org.apache.doris.load.Source;
import org.apache.doris.metric.MetricRepo;
import org.apache.doris.planner.BrokerScanNode;
import org.apache.doris.planner.OlapTableSink;
import org.apache.doris.planner.PlanFragment;
import org.apache.doris.task.MasterTaskExecutor;
@ -344,8 +343,7 @@ public class BrokerLoadJobTest {
@Injectable FileGroupAggKey aggKey,
@Mocked OlapTable olapTable,
@Mocked PlanFragment sinkFragment,
@Mocked OlapTableSink olapTableSink,
@Mocked BrokerScanNode scanNode) throws Exception {
@Mocked OlapTableSink olapTableSink) throws Exception {
List<Column> schema = new ArrayList<>();
schema.add(new Column("a", PrimitiveType.BIGINT));
Map<String, String> properties = new HashMap<>();

View File

@ -1,824 +0,0 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.planner;
import org.apache.doris.analysis.Analyzer;
import org.apache.doris.analysis.CastExpr;
import org.apache.doris.analysis.DescriptorTable;
import org.apache.doris.analysis.FunctionName;
import org.apache.doris.analysis.SlotDescriptor;
import org.apache.doris.analysis.TupleDescriptor;
import org.apache.doris.catalog.AggregateType;
import org.apache.doris.catalog.Column;
import org.apache.doris.catalog.Database;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.Function;
import org.apache.doris.catalog.FunctionSet;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.catalog.PrimitiveType;
import org.apache.doris.catalog.ScalarFunction;
import org.apache.doris.catalog.ScalarType;
import org.apache.doris.catalog.Type;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.UserException;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.task.StreamLoadTask;
import org.apache.doris.thrift.TExplainLevel;
import org.apache.doris.thrift.TFileFormatType;
import org.apache.doris.thrift.TFileType;
import org.apache.doris.thrift.TPlanNode;
import org.apache.doris.thrift.TStreamLoadPutRequest;
import com.google.common.collect.Lists;
import mockit.Expectations;
import mockit.Injectable;
import mockit.Mocked;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.junit.Assert;
import org.junit.Test;
import java.util.List;
public class StreamLoadScanNodeTest {
private static final Logger LOG = LogManager.getLogger(StreamLoadScanNodeTest.class);
@Mocked
Env env;
@Injectable
ConnectContext connectContext;
@Injectable
Database db;
@Injectable
OlapTable dstTable;
@Mocked
CastExpr castExpr;
TStreamLoadPutRequest getBaseRequest() {
TStreamLoadPutRequest request = new TStreamLoadPutRequest();
request.setFileType(TFileType.FILE_STREAM);
request.setFormatType(TFileFormatType.FORMAT_CSV_PLAIN);
return request;
}
List<Column> getBaseSchema() {
List<Column> columns = Lists.newArrayList();
Column k1 = new Column("k1", PrimitiveType.BIGINT);
k1.setIsKey(true);
k1.setIsAllowNull(false);
columns.add(k1);
Column k2 = new Column("k2", ScalarType.createVarchar(25));
k2.setIsKey(true);
k2.setIsAllowNull(true);
columns.add(k2);
Column v1 = new Column("v1", PrimitiveType.BIGINT);
v1.setIsKey(false);
v1.setIsAllowNull(true);
v1.setAggregationType(AggregateType.SUM, false);
columns.add(v1);
Column v2 = new Column("v2", ScalarType.createVarchar(25));
v2.setIsKey(false);
v2.setAggregationType(AggregateType.REPLACE, false);
v2.setIsAllowNull(false);
columns.add(v2);
return columns;
}
List<Column> getHllSchema() {
List<Column> columns = Lists.newArrayList();
Column k1 = new Column("k1", PrimitiveType.BIGINT);
k1.setIsKey(true);
k1.setIsAllowNull(false);
columns.add(k1);
Column v1 = new Column("v1", PrimitiveType.HLL);
v1.setIsKey(false);
v1.setIsAllowNull(true);
v1.setAggregationType(AggregateType.HLL_UNION, false);
columns.add(v1);
return columns;
}
List<Column> getSequenceColSchema() {
List<Column> columns = Lists.newArrayList();
Column k1 = new Column("k1", PrimitiveType.BIGINT);
k1.setIsKey(true);
k1.setIsAllowNull(false);
columns.add(k1);
Column k2 = new Column("k2", ScalarType.createVarchar(25));
k2.setIsKey(true);
k2.setIsAllowNull(true);
columns.add(k2);
// sequence column, it's hidden column
Column sequenceCol = new Column(Column.SEQUENCE_COL, PrimitiveType.BIGINT);
sequenceCol.setIsKey(false);
sequenceCol.setAggregationType(AggregateType.REPLACE, false);
sequenceCol.setIsAllowNull(false);
sequenceCol.setIsVisible(false);
columns.add(sequenceCol);
// sequence column, it's visible column for user, it's equals to the hidden column
Column visibleSequenceCol = new Column("visible_sequence_col", PrimitiveType.BIGINT);
visibleSequenceCol.setIsKey(false);
visibleSequenceCol.setAggregationType(AggregateType.REPLACE, false);
visibleSequenceCol.setIsAllowNull(true);
columns.add(visibleSequenceCol);
Column v1 = new Column("v1", ScalarType.createVarchar(25));
v1.setIsKey(false);
v1.setAggregationType(AggregateType.REPLACE, false);
v1.setIsAllowNull(false);
columns.add(v1);
return columns;
}
private StreamLoadScanNode getStreamLoadScanNode(TupleDescriptor dstDesc, TStreamLoadPutRequest request)
throws UserException {
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
return new StreamLoadScanNode(streamLoadTask.getId(), new PlanNodeId(1), dstDesc, dstTable, streamLoadTask);
}
@Test
public void testNormal() throws UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getBaseSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
TStreamLoadPutRequest request = getBaseRequest();
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
new Expectations() {
{
dstTable.getBaseSchema();
result = columns;
dstTable.getBaseSchema(anyBoolean);
result = columns;
dstTable.getFullSchema();
result = columns;
dstTable.getColumn("k1");
result = columns.get(0);
dstTable.getColumn("k2");
result = columns.get(1);
dstTable.getColumn("v1");
result = columns.get(2);
dstTable.getColumn("v2");
result = columns.get(3);
}
};
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
Assert.assertEquals(1, scanNode.getNumInstances());
Assert.assertEquals(1, scanNode.getScanRangeLocations(0).size());
}
@Test(expected = AnalysisException.class)
public void testLostV2() throws UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getBaseSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1, k2, v1");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test(expected = AnalysisException.class)
public void testBadColumns() throws UserException, UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getBaseSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1 k2 v1");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test
public void testColumnsNormal() throws UserException, UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getBaseSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
new Expectations() {
{
dstTable.getColumn("k1");
result = columns.stream().filter(c -> c.getName().equals("k1")).findFirst().get();
dstTable.getColumn("k2");
result = columns.stream().filter(c -> c.getName().equals("k2")).findFirst().get();
dstTable.getColumn("v1");
result = columns.stream().filter(c -> c.getName().equals("v1")).findFirst().get();
dstTable.getColumn("v2");
result = columns.stream().filter(c -> c.getName().equals("v2")).findFirst().get();
}
};
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1,k2,v1, v2=k2");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test
public void testHllColumnsNormal() throws UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getHllSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
new Expectations() {
{
env.getFunction((Function) any, (Function.CompareMode) any);
result = new ScalarFunction(new FunctionName(FunctionSet.HLL_HASH),
Lists.newArrayList(), Type.BIGINT, false, true);
dstTable.getColumn("k1");
result = columns.stream().filter(c -> c.getName().equals("k1")).findFirst().get();
dstTable.getColumn("k2");
result = null;
dstTable.getColumn("v1");
result = columns.stream().filter(c -> c.getName().equals("v1")).findFirst().get();
}
};
TStreamLoadPutRequest request = getBaseRequest();
request.setFileType(TFileType.FILE_STREAM);
request.setColumns("k1,k2, v1=" + FunctionSet.HLL_HASH + "(k2)");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test(expected = UserException.class)
public void testHllColumnsNoHllHash() throws UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getHllSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
new Expectations() {
{
env.getFunction((Function) any, (Function.CompareMode) any);
result = new ScalarFunction(new FunctionName("hll_hash1"), Lists.newArrayList(),
Type.BIGINT, false, true);
minTimes = 0;
}
};
new Expectations() {
{
dstTable.getColumn("k1");
result = columns.stream().filter(c -> c.getName().equals("k1")).findFirst().get();
minTimes = 0;
dstTable.getColumn("k2");
result = null;
minTimes = 0;
dstTable.getColumn("v1");
result = columns.stream().filter(c -> c.getName().equals("v1")).findFirst().get();
minTimes = 0;
}
};
TStreamLoadPutRequest request = getBaseRequest();
request.setFileType(TFileType.FILE_LOCAL);
request.setColumns("k1,k2, v1=hll_hash1(k2)");
StreamLoadTask.fromTStreamLoadPutRequest(request);
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test(expected = UserException.class)
public void testHllColumnsFail() throws UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getHllSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
TStreamLoadPutRequest request = getBaseRequest();
request.setFileType(TFileType.FILE_LOCAL);
request.setColumns("k1,k2, v1=k2");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test(expected = UserException.class)
public void testUnsupportedFType() throws UserException, UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getBaseSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
TStreamLoadPutRequest request = getBaseRequest();
request.setFileType(TFileType.FILE_BROKER);
request.setColumns("k1,k2,v1, v2=k2");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test(expected = UserException.class)
public void testColumnsUnknownRef() throws UserException, UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getBaseSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
new Expectations() {
{
dstTable.getBaseSchema();
minTimes = 0;
result = columns;
dstTable.getBaseSchema(anyBoolean);
minTimes = 0;
result = columns;
dstTable.getFullSchema();
minTimes = 0;
result = columns;
dstTable.getColumn("k1");
minTimes = 0;
result = columns.get(0);
dstTable.getColumn("k2");
minTimes = 0;
result = columns.get(1);
dstTable.getColumn("v1");
minTimes = 0;
result = columns.get(2);
dstTable.getColumn("v2");
minTimes = 0;
result = columns.get(3);
}
};
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1,k2,v1, v2=k3");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test
public void testWhereNormal() throws UserException, UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getBaseSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
new Expectations() {
{
dstTable.getColumn("k1");
result = columns.stream().filter(c -> c.getName().equals("k1")).findFirst().get();
dstTable.getColumn("k2");
result = columns.stream().filter(c -> c.getName().equals("k2")).findFirst().get();
dstTable.getColumn("v1");
result = columns.stream().filter(c -> c.getName().equals("v1")).findFirst().get();
dstTable.getColumn("v2");
result = columns.stream().filter(c -> c.getName().equals("v2")).findFirst().get();
}
};
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1,k2,v1, v2=k1");
request.setWhere("k1 = 1");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test(expected = AnalysisException.class)
public void testWhereBad() throws UserException, UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getBaseSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
new Expectations() {
{
dstTable.getColumn("k1");
result = columns.stream().filter(c -> c.getName().equals("k1")).findFirst().get();
minTimes = 0;
dstTable.getColumn("k2");
result = columns.stream().filter(c -> c.getName().equals("k2")).findFirst().get();
minTimes = 0;
dstTable.getColumn("v1");
result = columns.stream().filter(c -> c.getName().equals("v1")).findFirst().get();
minTimes = 0;
dstTable.getColumn("v2");
result = columns.stream().filter(c -> c.getName().equals("v2")).findFirst().get();
minTimes = 0;
}
};
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1,k2,v1, v2=k2");
request.setWhere("k1 1");
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
StreamLoadScanNode scanNode = new StreamLoadScanNode(streamLoadTask.getId(), new PlanNodeId(1),
dstDesc, dstTable, streamLoadTask);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test(expected = UserException.class)
public void testWhereUnknownRef() throws UserException, UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getBaseSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
new Expectations() {
{
dstTable.getBaseSchema();
minTimes = 0;
result = columns;
dstTable.getBaseSchema(anyBoolean);
minTimes = 0;
result = columns;
dstTable.getFullSchema();
minTimes = 0;
result = columns;
dstTable.getColumn("k1");
minTimes = 0;
result = columns.get(0);
dstTable.getColumn("k2");
minTimes = 0;
result = columns.get(1);
dstTable.getColumn("v1");
minTimes = 0;
result = columns.get(2);
dstTable.getColumn("v2");
minTimes = 0;
result = columns.get(3);
}
};
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1,k2,v1, v2=k1");
request.setWhere("k5 = 1");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test(expected = UserException.class)
public void testWhereNotBool() throws UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getBaseSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
new Expectations() {
{
dstTable.getBaseSchema();
minTimes = 0;
result = columns;
dstTable.getBaseSchema(anyBoolean);
minTimes = 0;
result = columns;
dstTable.getFullSchema();
minTimes = 0;
result = columns;
dstTable.getColumn("k1");
minTimes = 0;
result = columns.get(0);
dstTable.getColumn("k2");
minTimes = 0;
result = columns.get(1);
dstTable.getColumn("v1");
minTimes = 0;
result = columns.get(2);
dstTable.getColumn("v2");
minTimes = 0;
result = columns.get(3);
}
};
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1,k2,v1, v2=k1");
request.setWhere("k1 + v2");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test
public void testSequenceColumnWithSetColumns() throws UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getSequenceColSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
System.out.println(column);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
new Expectations() {
{
db.getTableNullable(anyInt);
result = dstTable;
minTimes = 0;
dstTable.hasSequenceCol();
result = true;
dstTable.getColumn("k1");
result = columns.stream().filter(c -> c.getName().equals("k1")).findFirst().get();
minTimes = 0;
dstTable.getColumn("k2");
result = columns.stream().filter(c -> c.getName().equals("k2")).findFirst().get();
minTimes = 0;
dstTable.getColumn(Column.SEQUENCE_COL);
result = columns.stream().filter(c -> c.getName().equals(Column.SEQUENCE_COL)).findFirst().get();
minTimes = 0;
dstTable.getColumn("visible_sequence_col");
result = columns.stream().filter(c -> c.getName().equals("visible_sequence_col")).findFirst().get();
minTimes = 0;
dstTable.getColumn("v1");
result = columns.stream().filter(c -> c.getName().equals("v1")).findFirst().get();
minTimes = 0;
// there is no "source_sequence" column in the Table
dstTable.getColumn("source_sequence");
result = null;
minTimes = 0;
}
};
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1,k2,source_sequence,v1");
request.setFileType(TFileType.FILE_STREAM);
request.setSequenceCol("source_sequence");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
@Test
public void testSequenceColumnWithoutSetColumns() throws UserException {
Analyzer analyzer = new Analyzer(env, connectContext);
DescriptorTable descTbl = analyzer.getDescTbl();
List<Column> columns = getSequenceColSchema();
TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc");
for (Column column : columns) {
SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc);
slot.setColumn(column);
slot.setIsMaterialized(true);
slot.setIsNullable(column.isAllowNull());
}
new Expectations() {
{
db.getTableNullable(anyInt);
result = dstTable;
minTimes = 0;
dstTable.hasSequenceCol();
result = true;
dstTable.getBaseSchema(anyBoolean);
result = columns;
dstTable.getFullSchema();
result = columns;
dstTable.getColumn("k1");
result = columns.stream().filter(c -> c.getName().equals("k1")).findFirst().get();
minTimes = 0;
dstTable.getColumn("k2");
result = columns.stream().filter(c -> c.getName().equals("k2")).findFirst().get();
minTimes = 0;
dstTable.getColumn(Column.SEQUENCE_COL);
result = columns.stream().filter(c -> c.getName().equals(Column.SEQUENCE_COL)).findFirst().get();
minTimes = 0;
dstTable.getColumn("visible_sequence_col");
result = columns.stream().filter(c -> c.getName().equals("visible_sequence_col")).findFirst().get();
minTimes = 0;
dstTable.getColumn("v1");
result = columns.stream().filter(c -> c.getName().equals("v1")).findFirst().get();
minTimes = 0;
dstTable.hasSequenceCol();
result = true;
minTimes = 0;
}
};
TStreamLoadPutRequest request = getBaseRequest();
request.setFileType(TFileType.FILE_STREAM);
request.setSequenceCol("visible_sequence_col");
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
scanNode.getNodeExplainString("", TExplainLevel.NORMAL);
TPlanNode planNode = new TPlanNode();
scanNode.toThrift(planNode);
}
}